Remove useless header files

This commit is contained in:
Alexey Milovidov 2023-11-06 03:49:55 +01:00
parent a710ac6011
commit 184e6f840e
71 changed files with 7491 additions and 7872 deletions

View File

@ -12,7 +12,7 @@
#include <Common/Config/ConfigReloader.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
#include <Common/TransformEndianness.hpp>
#include <Common/transformEndianness.h>
#include <Core/Settings.h>
#include <Interpreters/executeQuery.h>
#include <Parsers/Access/ASTGrantQuery.h>

View File

@ -1,7 +1,18 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionAnalysisOfVariance.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <IO/VarInt.h>
#include <array>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnNullable.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/Moments.h>
#include "Common/NaNUtils.h"
#include <Common/assert_cast.h>
namespace DB
{
@ -13,6 +24,82 @@ namespace ErrorCodes
namespace
{
using AggregateFunctionAnalysisOfVarianceData = AnalysisOfVarianceMoments<Float64>;
/// One way analysis of variance
/// Provides a statistical test of whether two or more population means are equal (null hypothesis)
/// Has an assumption that subjects from group i have normal distribution.
/// Accepts two arguments - a value and a group number which this value belongs to.
/// Groups are enumerated starting from 0 and there should be at least two groups to perform a test
/// Moreover there should be at least one group with the number of observations greater than one.
class AggregateFunctionAnalysisOfVariance final : public IAggregateFunctionDataHelper<AggregateFunctionAnalysisOfVarianceData, AggregateFunctionAnalysisOfVariance>
{
public:
explicit AggregateFunctionAnalysisOfVariance(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper(arguments, params, createResultType())
{}
DataTypePtr createResultType() const
{
DataTypes types {std::make_shared<DataTypeNumber<Float64>>(), std::make_shared<DataTypeNumber<Float64>>() };
Strings names {"f_statistic", "p_value"};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
String getName() const override { return "analysisOfVariance"; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
data(place).add(columns[0]->getFloat64(row_num), columns[1]->getUInt(row_num));
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
data(place).merge(data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
data(place).read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto f_stat = data(place).getFStatistic();
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
if (unlikely(!std::isfinite(f_stat) || f_stat < 0))
{
column_stat.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
column_value.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
return;
}
auto p_value = data(place).getPValue(f_stat);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
column_stat.getData().push_back(f_stat);
column_value.getData().push_back(p_value);
}
};
AggregateFunctionPtr createAggregateFunctionAnalysisOfVariance(const std::string & name, const DataTypes & arguments, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);

View File

@ -1,97 +0,0 @@
#pragma once
#include <IO/VarInt.h>
#include <IO/WriteHelpers.h>
#include <array>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsCommon.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/Moments.h>
#include "Common/NaNUtils.h"
#include <Common/assert_cast.h>
#include <Core/Types.h>
namespace DB
{
using AggregateFunctionAnalysisOfVarianceData = AnalysisOfVarianceMoments<Float64>;
/// One way analysis of variance
/// Provides a statistical test of whether two or more population means are equal (null hypothesis)
/// Has an assumption that subjects from group i have normal distribution.
/// Accepts two arguments - a value and a group number which this value belongs to.
/// Groups are enumerated starting from 0 and there should be at least two groups to perform a test
/// Moreover there should be at least one group with the number of observations greater than one.
class AggregateFunctionAnalysisOfVariance final : public IAggregateFunctionDataHelper<AggregateFunctionAnalysisOfVarianceData, AggregateFunctionAnalysisOfVariance>
{
public:
explicit AggregateFunctionAnalysisOfVariance(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper(arguments, params, createResultType())
{}
DataTypePtr createResultType() const
{
DataTypes types {std::make_shared<DataTypeNumber<Float64>>(), std::make_shared<DataTypeNumber<Float64>>() };
Strings names {"f_statistic", "p_value"};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
String getName() const override { return "analysisOfVariance"; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
data(place).add(columns[0]->getFloat64(row_num), columns[1]->getUInt(row_num));
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
data(place).merge(data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
data(place).read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto f_stat = data(place).getFStatistic();
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
if (unlikely(!std::isfinite(f_stat) || f_stat < 0))
{
column_stat.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
column_value.getData().push_back(std::numeric_limits<Float64>::quiet_NaN());
return;
}
auto p_value = data(place).getPValue(f_stat);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
column_stat.getData().push_back(f_stat);
column_value.getData().push_back(p_value);
}
};
}

View File

@ -1,12 +1,14 @@
#include <memory>
#include <type_traits>
#include <AggregateFunctions/AggregateFunctionAvg.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionAvgWeighted.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
@ -16,6 +18,88 @@ namespace ErrorCodes
namespace
{
template <typename T>
using AvgWeightedFieldType = std::conditional_t<is_decimal<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
std::conditional_t<DecimalOrExtendedInt<T>,
Float64, // no way to do UInt128 * UInt128, better cast to Float64
NearestFieldType<T>>>;
template <typename T, typename U>
using MaxFieldType = std::conditional_t<(sizeof(AvgWeightedFieldType<T>) > sizeof(AvgWeightedFieldType<U>)),
AvgWeightedFieldType<T>, AvgWeightedFieldType<U>>;
template <typename Value, typename Weight>
class AggregateFunctionAvgWeighted final :
public AggregateFunctionAvgBase<
MaxFieldType<Value, Weight>, AvgWeightedFieldType<Weight>, AggregateFunctionAvgWeighted<Value, Weight>>
{
public:
using Base = AggregateFunctionAvgBase<
MaxFieldType<Value, Weight>, AvgWeightedFieldType<Weight>, AggregateFunctionAvgWeighted<Value, Weight>>;
using Base::Base;
using Numerator = typename Base::Numerator;
using Denominator = typename Base::Denominator;
using Fraction = typename Base::Fraction;
void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
const auto& weights = static_cast<const ColumnVectorOrDecimal<Weight> &>(*columns[1]);
this->data(place).numerator += static_cast<Numerator>(
static_cast<const ColumnVectorOrDecimal<Value> &>(*columns[0]).getData()[row_num]) *
static_cast<Numerator>(weights.getData()[row_num]);
this->data(place).denominator += static_cast<Denominator>(weights.getData()[row_num]);
}
String getName() const override { return "avgWeighted"; }
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override
{
bool can_be_compiled = Base::isCompilable();
can_be_compiled &= canBeNativeType<Weight>();
return can_be_compiled;
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * numerator_type = toNativeType<Numerator>(b);
auto * numerator_ptr = aggregate_data_ptr;
auto * numerator_value = b.CreateLoad(numerator_type, numerator_ptr);
auto numerator_data_type = toNativeDataType<Numerator>();
auto * argument = nativeCast(b, arguments[0], numerator_data_type);
auto * weight = nativeCast(b, arguments[1], numerator_data_type);
llvm::Value * value_weight_multiplication = argument->getType()->isIntegerTy() ? b.CreateMul(argument, weight) : b.CreateFMul(argument, weight);
auto * numerator_result_value = numerator_type->isIntegerTy() ? b.CreateAdd(numerator_value, value_weight_multiplication) : b.CreateFAdd(numerator_value, value_weight_multiplication);
b.CreateStore(numerator_result_value, numerator_ptr);
auto * denominator_type = toNativeType<Denominator>(b);
static constexpr size_t denominator_offset = offsetof(Fraction, denominator);
auto * denominator_ptr = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregate_data_ptr, denominator_offset);
auto * weight_cast_to_denominator = nativeCast(b, arguments[1], toNativeDataType<Denominator>());
auto * denominator_value = b.CreateLoad(denominator_type, denominator_ptr);
auto * denominator_value_updated = denominator_type->isIntegerTy() ? b.CreateAdd(denominator_value, weight_cast_to_denominator) : b.CreateFAdd(denominator_value, weight_cast_to_denominator);
b.CreateStore(denominator_value_updated, denominator_ptr);
}
#endif
};
bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept
{
const WhichDataType l_dt(left), r_dt(right);

View File

@ -1,90 +0,0 @@
#pragma once
#include <type_traits>
#include <AggregateFunctions/AggregateFunctionAvg.h>
namespace DB
{
struct Settings;
template <typename T>
using AvgWeightedFieldType = std::conditional_t<is_decimal<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
std::conditional_t<DecimalOrExtendedInt<T>,
Float64, // no way to do UInt128 * UInt128, better cast to Float64
NearestFieldType<T>>>;
template <typename T, typename U>
using MaxFieldType = std::conditional_t<(sizeof(AvgWeightedFieldType<T>) > sizeof(AvgWeightedFieldType<U>)),
AvgWeightedFieldType<T>, AvgWeightedFieldType<U>>;
template <typename Value, typename Weight>
class AggregateFunctionAvgWeighted final :
public AggregateFunctionAvgBase<
MaxFieldType<Value, Weight>, AvgWeightedFieldType<Weight>, AggregateFunctionAvgWeighted<Value, Weight>>
{
public:
using Base = AggregateFunctionAvgBase<
MaxFieldType<Value, Weight>, AvgWeightedFieldType<Weight>, AggregateFunctionAvgWeighted<Value, Weight>>;
using Base::Base;
using Numerator = typename Base::Numerator;
using Denominator = typename Base::Denominator;
using Fraction = typename Base::Fraction;
void NO_SANITIZE_UNDEFINED add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
const auto& weights = static_cast<const ColumnVectorOrDecimal<Weight> &>(*columns[1]);
this->data(place).numerator += static_cast<Numerator>(
static_cast<const ColumnVectorOrDecimal<Value> &>(*columns[0]).getData()[row_num]) *
static_cast<Numerator>(weights.getData()[row_num]);
this->data(place).denominator += static_cast<Denominator>(weights.getData()[row_num]);
}
String getName() const override { return "avgWeighted"; }
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override
{
bool can_be_compiled = Base::isCompilable();
can_be_compiled &= canBeNativeType<Weight>();
return can_be_compiled;
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * numerator_type = toNativeType<Numerator>(b);
auto * numerator_ptr = aggregate_data_ptr;
auto * numerator_value = b.CreateLoad(numerator_type, numerator_ptr);
auto numerator_data_type = toNativeDataType<Numerator>();
auto * argument = nativeCast(b, arguments[0], numerator_data_type);
auto * weight = nativeCast(b, arguments[1], numerator_data_type);
llvm::Value * value_weight_multiplication = argument->getType()->isIntegerTy() ? b.CreateMul(argument, weight) : b.CreateFMul(argument, weight);
auto * numerator_result_value = numerator_type->isIntegerTy() ? b.CreateAdd(numerator_value, value_weight_multiplication) : b.CreateFAdd(numerator_value, value_weight_multiplication);
b.CreateStore(numerator_result_value, numerator_ptr);
auto * denominator_type = toNativeType<Denominator>(b);
static constexpr size_t denominator_offset = offsetof(Fraction, denominator);
auto * denominator_ptr = b.CreateConstInBoundsGEP1_64(b.getInt8Ty(), aggregate_data_ptr, denominator_offset);
auto * weight_cast_to_denominator = nativeCast(b, arguments[1], toNativeDataType<Denominator>());
auto * denominator_value = b.CreateLoad(denominator_type, denominator_ptr);
auto * denominator_value_updated = denominator_type->isIntegerTy() ? b.CreateAdd(denominator_value, weight_cast_to_denominator) : b.CreateFAdd(denominator_value, weight_cast_to_denominator);
b.CreateStore(denominator_value_updated, denominator_ptr);
}
#endif
};
}

View File

@ -1,11 +1,27 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionBitwise.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include "config.h"
#if USE_EMBEDDED_COMPILER
# include <llvm/IR/IRBuilder.h>
# include <DataTypes/Native.h>
#endif
namespace DB
{
struct Settings;
namespace ErrorCodes
@ -16,6 +32,179 @@ namespace ErrorCodes
namespace
{
template <typename T>
struct AggregateFunctionGroupBitOrData
{
T value = 0;
static const char * name() { return "groupBitOr"; }
void update(T x) { value |= x; }
#if USE_EMBEDDED_COMPILER
static void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * value_ptr)
{
auto type = toNativeType<T>(builder);
builder.CreateStore(llvm::Constant::getNullValue(type), value_ptr);
}
static llvm::Value* compileUpdate(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs)
{
return builder.CreateOr(lhs, rhs);
}
#endif
};
template <typename T>
struct AggregateFunctionGroupBitAndData
{
T value = -1; /// Two's complement arithmetic, sign extension.
static const char * name() { return "groupBitAnd"; }
void update(T x) { value &= x; }
#if USE_EMBEDDED_COMPILER
static void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * value_ptr)
{
auto type = toNativeType<T>(builder);
builder.CreateStore(llvm::ConstantInt::get(type, -1), value_ptr);
}
static llvm::Value* compileUpdate(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs)
{
return builder.CreateAnd(lhs, rhs);
}
#endif
};
template <typename T>
struct AggregateFunctionGroupBitXorData
{
T value = 0;
static const char * name() { return "groupBitXor"; }
void update(T x) { value ^= x; }
#if USE_EMBEDDED_COMPILER
static void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * value_ptr)
{
auto type = toNativeType<T>(builder);
builder.CreateStore(llvm::Constant::getNullValue(type), value_ptr);
}
static llvm::Value* compileUpdate(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs)
{
return builder.CreateXor(lhs, rhs);
}
#endif
};
/// Counts bitwise operation on numbers.
template <typename T, typename Data>
class AggregateFunctionBitwise final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitwise<T, Data>>
{
public:
explicit AggregateFunctionBitwise(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionBitwise<T, Data>>({type}, {}, createResultType())
{}
String getName() const override { return Data::name(); }
static DataTypePtr createResultType()
{
return std::make_shared<DataTypeNumber<T>>();
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).update(assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num]);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).update(this->data(rhs).value);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinary(this->data(place).value, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinary(this->data(place).value, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnVector<T> &>(to).getData().push_back(this->data(place).value);
}
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override
{
auto return_type = this->getResultType();
return canBeNativeType(*return_type);
}
void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr) const override
{
auto * value_ptr = aggregate_data_ptr;
Data::compileCreate(builder, value_ptr);
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * return_type = toNativeType(b, this->getResultType());
auto * value_ptr = aggregate_data_ptr;
auto * value = b.CreateLoad(return_type, value_ptr);
auto * result_value = Data::compileUpdate(builder, value, arguments[0].value);
b.CreateStore(result_value, value_ptr);
}
void compileMerge(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_dst_ptr, llvm::Value * aggregate_data_src_ptr) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * return_type = toNativeType(b, this->getResultType());
auto * value_dst_ptr = aggregate_data_dst_ptr;
auto * value_dst = b.CreateLoad(return_type, value_dst_ptr);
auto * value_src_ptr = aggregate_data_src_ptr;
auto * value_src = b.CreateLoad(return_type, value_src_ptr);
auto * result_value = Data::compileUpdate(builder, value_dst, value_src);
b.CreateStore(result_value, value_dst_ptr);
}
llvm::Value * compileGetResult(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * return_type = toNativeType(b, this->getResultType());
auto * value_ptr = aggregate_data_ptr;
return b.CreateLoad(return_type, value_ptr);
}
#endif
};
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{

View File

@ -1,197 +0,0 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include "config.h"
#if USE_EMBEDDED_COMPILER
# include <llvm/IR/IRBuilder.h>
# include <DataTypes/Native.h>
#endif
namespace DB
{
struct Settings;
template <typename T>
struct AggregateFunctionGroupBitOrData
{
T value = 0;
static const char * name() { return "groupBitOr"; }
void update(T x) { value |= x; }
#if USE_EMBEDDED_COMPILER
static void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * value_ptr)
{
auto type = toNativeType<T>(builder);
builder.CreateStore(llvm::Constant::getNullValue(type), value_ptr);
}
static llvm::Value* compileUpdate(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs)
{
return builder.CreateOr(lhs, rhs);
}
#endif
};
template <typename T>
struct AggregateFunctionGroupBitAndData
{
T value = -1; /// Two's complement arithmetic, sign extension.
static const char * name() { return "groupBitAnd"; }
void update(T x) { value &= x; }
#if USE_EMBEDDED_COMPILER
static void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * value_ptr)
{
auto type = toNativeType<T>(builder);
builder.CreateStore(llvm::ConstantInt::get(type, -1), value_ptr);
}
static llvm::Value* compileUpdate(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs)
{
return builder.CreateAnd(lhs, rhs);
}
#endif
};
template <typename T>
struct AggregateFunctionGroupBitXorData
{
T value = 0;
static const char * name() { return "groupBitXor"; }
void update(T x) { value ^= x; }
#if USE_EMBEDDED_COMPILER
static void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * value_ptr)
{
auto type = toNativeType<T>(builder);
builder.CreateStore(llvm::Constant::getNullValue(type), value_ptr);
}
static llvm::Value* compileUpdate(llvm::IRBuilderBase & builder, llvm::Value * lhs, llvm::Value * rhs)
{
return builder.CreateXor(lhs, rhs);
}
#endif
};
/// Counts bitwise operation on numbers.
template <typename T, typename Data>
class AggregateFunctionBitwise final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitwise<T, Data>>
{
public:
explicit AggregateFunctionBitwise(const DataTypePtr & type)
: IAggregateFunctionDataHelper<Data, AggregateFunctionBitwise<T, Data>>({type}, {}, createResultType())
{}
String getName() const override { return Data::name(); }
static DataTypePtr createResultType()
{
return std::make_shared<DataTypeNumber<T>>();
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).update(assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num]);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).update(this->data(rhs).value);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinary(this->data(place).value, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinary(this->data(place).value, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnVector<T> &>(to).getData().push_back(this->data(place).value);
}
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override
{
auto return_type = this->getResultType();
return canBeNativeType(*return_type);
}
void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr) const override
{
auto * value_ptr = aggregate_data_ptr;
Data::compileCreate(builder, value_ptr);
}
void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const ValuesWithType & arguments) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * return_type = toNativeType(b, this->getResultType());
auto * value_ptr = aggregate_data_ptr;
auto * value = b.CreateLoad(return_type, value_ptr);
auto * result_value = Data::compileUpdate(builder, value, arguments[0].value);
b.CreateStore(result_value, value_ptr);
}
void compileMerge(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_dst_ptr, llvm::Value * aggregate_data_src_ptr) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * return_type = toNativeType(b, this->getResultType());
auto * value_dst_ptr = aggregate_data_dst_ptr;
auto * value_dst = b.CreateLoad(return_type, value_dst_ptr);
auto * value_src_ptr = aggregate_data_src_ptr;
auto * value_src = b.CreateLoad(return_type, value_src_ptr);
auto * result_value = Data::compileUpdate(builder, value_dst, value_src);
b.CreateStore(result_value, value_dst_ptr);
}
llvm::Value * compileGetResult(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr) const override
{
llvm::IRBuilder<> & b = static_cast<llvm::IRBuilder<> &>(builder);
auto * return_type = toNativeType(b, this->getResultType());
auto * value_ptr = aggregate_data_ptr;
return b.CreateLoad(return_type, value_ptr);
}
#endif
};
}

View File

@ -1,7 +1,14 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionBoundingRatio.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Common/assert_cast.h>
#include <Common/transformEndianness.h>
namespace DB
{
@ -10,11 +17,169 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace
{
/** Tracks the leftmost and rightmost (x, y) data points.
*/
struct AggregateFunctionBoundingRatioData
{
struct Point
{
Float64 x;
Float64 y;
};
bool empty = true;
Point left;
Point right;
void add(Float64 x, Float64 y)
{
Point point{x, y};
if (empty)
{
left = point;
right = point;
empty = false;
}
else if (point.x < left.x)
{
left = point;
}
else if (point.x > right.x)
{
right = point;
}
}
void merge(const AggregateFunctionBoundingRatioData & other)
{
if (empty)
{
*this = other;
}
else
{
if (other.left.x < left.x)
left = other.left;
if (other.right.x > right.x)
right = other.right;
}
}
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
template <std::endian endian>
inline void transformEndianness(AggregateFunctionBoundingRatioData::Point & p)
{
DB::transformEndianness<endian>(p.x);
DB::transformEndianness<endian>(p.y);
}
void AggregateFunctionBoundingRatioData::serialize(WriteBuffer & buf) const
{
writeBinaryLittleEndian(empty, buf);
if (!empty)
{
writeBinaryLittleEndian(left, buf);
writeBinaryLittleEndian(right, buf);
}
}
void AggregateFunctionBoundingRatioData::deserialize(ReadBuffer & buf)
{
readBinaryLittleEndian(empty, buf);
if (!empty)
{
readBinaryLittleEndian(left, buf);
readBinaryLittleEndian(right, buf);
}
}
inline void writeBinary(const AggregateFunctionBoundingRatioData::Point & p, WriteBuffer & buf)
{
writePODBinary(p, buf);
}
inline void readBinary(AggregateFunctionBoundingRatioData::Point & p, ReadBuffer & buf)
{
readPODBinary(p, buf);
}
class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>
{
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)
{
if (data.empty)
return std::numeric_limits<Float64>::quiet_NaN();
return (data.right.y - data.left.y) / (data.right.x - data.left.x);
}
public:
String getName() const override
{
return "boundingRatio";
}
explicit AggregateFunctionBoundingRatio(const DataTypes & arguments)
: IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>(arguments, {}, std::make_shared<DataTypeFloat64>())
{
const auto * x_arg = arguments.at(0).get();
const auto * y_arg = arguments.at(1).get();
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Illegal types of arguments of aggregate function {}, must have number representation.",
getName());
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
/// NOTE Slightly inefficient.
const auto x = columns[0]->getFloat64(row_num);
const auto y = columns[1]->getFloat64(row_num);
data(place).add(x, y);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
data(place).merge(data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnFloat64 &>(to).getData().push_back(getBoundingRatio(data(place)));
}
};
AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);

View File

@ -1,177 +0,0 @@
#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Common/assert_cast.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/** Tracks the leftmost and rightmost (x, y) data points.
*/
struct AggregateFunctionBoundingRatioData
{
struct Point
{
Float64 x;
Float64 y;
};
bool empty = true;
Point left;
Point right;
void add(Float64 x, Float64 y)
{
Point point{x, y};
if (empty)
{
left = point;
right = point;
empty = false;
}
else if (point.x < left.x)
{
left = point;
}
else if (point.x > right.x)
{
right = point;
}
}
void merge(const AggregateFunctionBoundingRatioData & other)
{
if (empty)
{
*this = other;
}
else
{
if (other.left.x < left.x)
left = other.left;
if (other.right.x > right.x)
right = other.right;
}
}
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
};
template <std::endian endian>
inline void transformEndianness(AggregateFunctionBoundingRatioData::Point & p)
{
transformEndianness<endian>(p.x);
transformEndianness<endian>(p.y);
}
void AggregateFunctionBoundingRatioData::serialize(WriteBuffer & buf) const
{
writeBinaryLittleEndian(empty, buf);
if (!empty)
{
writeBinaryLittleEndian(left, buf);
writeBinaryLittleEndian(right, buf);
}
}
void AggregateFunctionBoundingRatioData::deserialize(ReadBuffer & buf)
{
readBinaryLittleEndian(empty, buf);
if (!empty)
{
readBinaryLittleEndian(left, buf);
readBinaryLittleEndian(right, buf);
}
}
inline void writeBinary(const AggregateFunctionBoundingRatioData::Point & p, WriteBuffer & buf)
{
writePODBinary(p, buf);
}
inline void readBinary(AggregateFunctionBoundingRatioData::Point & p, ReadBuffer & buf)
{
readPODBinary(p, buf);
}
class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>
{
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)
{
if (data.empty)
return std::numeric_limits<Float64>::quiet_NaN();
return (data.right.y - data.left.y) / (data.right.x - data.left.x);
}
public:
String getName() const override
{
return "boundingRatio";
}
explicit AggregateFunctionBoundingRatio(const DataTypes & arguments)
: IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>(arguments, {}, std::make_shared<DataTypeFloat64>())
{
const auto * x_arg = arguments.at(0).get();
const auto * y_arg = arguments.at(1).get();
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Illegal types of arguments of aggregate function {}, must have number representation.",
getName());
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
/// NOTE Slightly inefficient.
const auto x = columns[0]->getFloat64(row_num);
const auto y = columns[1]->getFloat64(row_num);
data(place).add(x, y);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
data(place).merge(data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnFloat64 &>(to).getData().push_back(getBoundingRatio(data(place)));
}
};
}

View File

@ -1,9 +1,15 @@
#include <AggregateFunctions/AggregateFunctionDeltaSum.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
@ -18,6 +24,113 @@ namespace ErrorCodes
namespace
{
template <typename T>
struct AggregationFunctionDeltaSumData
{
T sum = 0;
T last = 0;
T first = 0;
bool seen = false;
};
template <typename T>
class AggregationFunctionDeltaSum final
: public IAggregateFunctionDataHelper<AggregationFunctionDeltaSumData<T>, AggregationFunctionDeltaSum<T>>
{
public:
AggregationFunctionDeltaSum(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregationFunctionDeltaSumData<T>, AggregationFunctionDeltaSum<T>>{arguments, params, createResultType()}
{}
AggregationFunctionDeltaSum()
: IAggregateFunctionDataHelper<AggregationFunctionDeltaSumData<T>, AggregationFunctionDeltaSum<T>>{}
{}
String getName() const override { return "deltaSum"; }
static DataTypePtr createResultType() { return std::make_shared<DataTypeNumber<T>>(); }
bool allocatesMemoryInArena() const override { return false; }
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto value = assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
if ((this->data(place).last < value) && this->data(place).seen)
{
this->data(place).sum += (value - this->data(place).last);
}
this->data(place).last = value;
if (!this->data(place).seen)
{
this->data(place).first = value;
this->data(place).seen = true;
}
}
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto place_data = &this->data(place);
auto rhs_data = &this->data(rhs);
if ((place_data->last < rhs_data->first) && place_data->seen && rhs_data->seen)
{
// If the lhs last number seen is less than the first number the rhs saw, the lhs is before
// the rhs, for example [0, 2] [4, 7]. So we want to add the deltasums, but also add the
// difference between lhs last number and rhs first number (the 2 and 4). Then we want to
// take last value from the rhs, so first and last become 0 and 7.
place_data->sum += rhs_data->sum + (rhs_data->first - place_data->last);
place_data->last = rhs_data->last;
}
else if ((rhs_data->first < place_data->last && rhs_data->seen && place_data->seen))
{
// In the opposite scenario, the lhs comes after the rhs, e.g. [4, 6] [1, 2]. Since we
// assume the input interval states are sorted by time, we assume this is a counter
// reset, and therefore do *not* add the difference between our first value and the
// rhs last value.
place_data->sum += rhs_data->sum;
place_data->last = rhs_data->last;
}
else if (rhs_data->seen && !place_data->seen)
{
// If we're here then the lhs is an empty state and the rhs does have some state, so
// we'll just take that state.
place_data->first = rhs_data->first;
place_data->last = rhs_data->last;
place_data->sum = rhs_data->sum;
place_data->seen = rhs_data->seen;
}
// Otherwise lhs either has data or is uninitialized, so we don't need to modify its values.
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinaryLittleEndian(this->data(place).sum, buf);
writeBinaryLittleEndian(this->data(place).first, buf);
writeBinaryLittleEndian(this->data(place).last, buf);
writeBinaryLittleEndian(this->data(place).seen, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinaryLittleEndian(this->data(place).sum, buf);
readBinaryLittleEndian(this->data(place).first, buf);
readBinaryLittleEndian(this->data(place).last, buf);
readBinaryLittleEndian(this->data(place).seen, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnVector<T> &>(to).getData().push_back(this->data(place).sum);
}
};
AggregateFunctionPtr createAggregateFunctionDeltaSum(
const String & name,
const DataTypes & arguments,

View File

@ -1,126 +0,0 @@
#pragma once
#include <type_traits>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
struct Settings;
template <typename T>
struct AggregationFunctionDeltaSumData
{
T sum = 0;
T last = 0;
T first = 0;
bool seen = false;
};
template <typename T>
class AggregationFunctionDeltaSum final
: public IAggregateFunctionDataHelper<AggregationFunctionDeltaSumData<T>, AggregationFunctionDeltaSum<T>>
{
public:
AggregationFunctionDeltaSum(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregationFunctionDeltaSumData<T>, AggregationFunctionDeltaSum<T>>{arguments, params, createResultType()}
{}
AggregationFunctionDeltaSum()
: IAggregateFunctionDataHelper<AggregationFunctionDeltaSumData<T>, AggregationFunctionDeltaSum<T>>{}
{}
String getName() const override { return "deltaSum"; }
static DataTypePtr createResultType() { return std::make_shared<DataTypeNumber<T>>(); }
bool allocatesMemoryInArena() const override { return false; }
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto value = assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
if ((this->data(place).last < value) && this->data(place).seen)
{
this->data(place).sum += (value - this->data(place).last);
}
this->data(place).last = value;
if (!this->data(place).seen)
{
this->data(place).first = value;
this->data(place).seen = true;
}
}
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto place_data = &this->data(place);
auto rhs_data = &this->data(rhs);
if ((place_data->last < rhs_data->first) && place_data->seen && rhs_data->seen)
{
// If the lhs last number seen is less than the first number the rhs saw, the lhs is before
// the rhs, for example [0, 2] [4, 7]. So we want to add the deltasums, but also add the
// difference between lhs last number and rhs first number (the 2 and 4). Then we want to
// take last value from the rhs, so first and last become 0 and 7.
place_data->sum += rhs_data->sum + (rhs_data->first - place_data->last);
place_data->last = rhs_data->last;
}
else if ((rhs_data->first < place_data->last && rhs_data->seen && place_data->seen))
{
// In the opposite scenario, the lhs comes after the rhs, e.g. [4, 6] [1, 2]. Since we
// assume the input interval states are sorted by time, we assume this is a counter
// reset, and therefore do *not* add the difference between our first value and the
// rhs last value.
place_data->sum += rhs_data->sum;
place_data->last = rhs_data->last;
}
else if (rhs_data->seen && !place_data->seen)
{
// If we're here then the lhs is an empty state and the rhs does have some state, so
// we'll just take that state.
place_data->first = rhs_data->first;
place_data->last = rhs_data->last;
place_data->sum = rhs_data->sum;
place_data->seen = rhs_data->seen;
}
// Otherwise lhs either has data or is uninitialized, so we don't need to modify its values.
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinaryLittleEndian(this->data(place).sum, buf);
writeBinaryLittleEndian(this->data(place).first, buf);
writeBinaryLittleEndian(this->data(place).last, buf);
writeBinaryLittleEndian(this->data(place).seen, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinaryLittleEndian(this->data(place).sum, buf);
readBinaryLittleEndian(this->data(place).first, buf);
readBinaryLittleEndian(this->data(place).last, buf);
readBinaryLittleEndian(this->data(place).seen, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnVector<T> &>(to).getData().push_back(this->data(place).sum);
}
};
}

View File

@ -1,9 +1,15 @@
#include <AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
@ -17,6 +23,159 @@ namespace ErrorCodes
namespace
{
template <typename ValueType, typename TimestampType>
struct AggregationFunctionDeltaSumTimestampData
{
ValueType sum = 0;
ValueType first = 0;
ValueType last = 0;
TimestampType first_ts = 0;
TimestampType last_ts = 0;
bool seen = false;
};
template <typename ValueType, typename TimestampType>
class AggregationFunctionDeltaSumTimestamp final
: public IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
>
{
public:
AggregationFunctionDeltaSumTimestamp(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
>{arguments, params, createResultType()}
{}
AggregationFunctionDeltaSumTimestamp()
: IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
>{}
{}
bool allocatesMemoryInArena() const override { return false; }
String getName() const override { return "deltaSumTimestamp"; }
static DataTypePtr createResultType() { return std::make_shared<DataTypeNumber<ValueType>>(); }
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto value = assert_cast<const ColumnVector<ValueType> &>(*columns[0]).getData()[row_num];
auto ts = assert_cast<const ColumnVector<TimestampType> &>(*columns[1]).getData()[row_num];
if ((this->data(place).last < value) && this->data(place).seen)
{
this->data(place).sum += (value - this->data(place).last);
}
this->data(place).last = value;
this->data(place).last_ts = ts;
if (!this->data(place).seen)
{
this->data(place).first = value;
this->data(place).seen = true;
this->data(place).first_ts = ts;
}
}
// before returns true if lhs is before rhs or false if it is not or can't be determined
bool ALWAYS_INLINE before (
const AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType> * lhs,
const AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType> * rhs
) const
{
if (lhs->last_ts < rhs->first_ts)
{
return true;
}
if (lhs->last_ts == rhs->first_ts && (lhs->last_ts < rhs->last_ts || lhs->first_ts < rhs->first_ts))
{
return true;
}
return false;
}
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto place_data = &this->data(place);
auto rhs_data = &this->data(rhs);
if (!place_data->seen && rhs_data->seen)
{
place_data->sum = rhs_data->sum;
place_data->seen = true;
place_data->first = rhs_data->first;
place_data->first_ts = rhs_data->first_ts;
place_data->last = rhs_data->last;
place_data->last_ts = rhs_data->last_ts;
}
else if (place_data->seen && !rhs_data->seen)
return;
else if (before(place_data, rhs_data))
{
// This state came before the rhs state
if (rhs_data->first > place_data->last)
place_data->sum += (rhs_data->first - place_data->last);
place_data->sum += rhs_data->sum;
place_data->last = rhs_data->last;
place_data->last_ts = rhs_data->last_ts;
}
else if (before(rhs_data, place_data))
{
// This state came after the rhs state
if (place_data->first > rhs_data->last)
place_data->sum += (place_data->first - rhs_data->last);
place_data->sum += rhs_data->sum;
place_data->first = rhs_data->first;
place_data->first_ts = rhs_data->first_ts;
}
else
{
// If none of those conditions matched, it means both states we are merging have all
// same timestamps. We have to pick either the smaller or larger value so that the
// result is deterministic.
if (place_data->first < rhs_data->first)
{
place_data->first = rhs_data->first;
place_data->last = rhs_data->last;
}
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinaryLittleEndian(this->data(place).sum, buf);
writeBinaryLittleEndian(this->data(place).first, buf);
writeBinaryLittleEndian(this->data(place).first_ts, buf);
writeBinaryLittleEndian(this->data(place).last, buf);
writeBinaryLittleEndian(this->data(place).last_ts, buf);
writeBinaryLittleEndian(this->data(place).seen, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinaryLittleEndian(this->data(place).sum, buf);
readBinaryLittleEndian(this->data(place).first, buf);
readBinaryLittleEndian(this->data(place).first_ts, buf);
readBinaryLittleEndian(this->data(place).last, buf);
readBinaryLittleEndian(this->data(place).last_ts, buf);
readBinaryLittleEndian(this->data(place).seen, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnVector<ValueType> &>(to).getData().push_back(this->data(place).sum);
}
};
AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
const String & name,
const DataTypes & arguments,

View File

@ -1,171 +0,0 @@
#pragma once
#include <type_traits>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
template <typename ValueType, typename TimestampType>
struct AggregationFunctionDeltaSumTimestampData
{
ValueType sum = 0;
ValueType first = 0;
ValueType last = 0;
TimestampType first_ts = 0;
TimestampType last_ts = 0;
bool seen = false;
};
template <typename ValueType, typename TimestampType>
class AggregationFunctionDeltaSumTimestamp final
: public IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
>
{
public:
AggregationFunctionDeltaSumTimestamp(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
>{arguments, params, createResultType()}
{}
AggregationFunctionDeltaSumTimestamp()
: IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
>{}
{}
bool allocatesMemoryInArena() const override { return false; }
String getName() const override { return "deltaSumTimestamp"; }
static DataTypePtr createResultType() { return std::make_shared<DataTypeNumber<ValueType>>(); }
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto value = assert_cast<const ColumnVector<ValueType> &>(*columns[0]).getData()[row_num];
auto ts = assert_cast<const ColumnVector<TimestampType> &>(*columns[1]).getData()[row_num];
if ((this->data(place).last < value) && this->data(place).seen)
{
this->data(place).sum += (value - this->data(place).last);
}
this->data(place).last = value;
this->data(place).last_ts = ts;
if (!this->data(place).seen)
{
this->data(place).first = value;
this->data(place).seen = true;
this->data(place).first_ts = ts;
}
}
// before returns true if lhs is before rhs or false if it is not or can't be determined
bool ALWAYS_INLINE before (
const AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType> * lhs,
const AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType> * rhs
) const
{
if (lhs->last_ts < rhs->first_ts)
{
return true;
}
if (lhs->last_ts == rhs->first_ts && (lhs->last_ts < rhs->last_ts || lhs->first_ts < rhs->first_ts))
{
return true;
}
return false;
}
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto place_data = &this->data(place);
auto rhs_data = &this->data(rhs);
if (!place_data->seen && rhs_data->seen)
{
place_data->sum = rhs_data->sum;
place_data->seen = true;
place_data->first = rhs_data->first;
place_data->first_ts = rhs_data->first_ts;
place_data->last = rhs_data->last;
place_data->last_ts = rhs_data->last_ts;
}
else if (place_data->seen && !rhs_data->seen)
return;
else if (before(place_data, rhs_data))
{
// This state came before the rhs state
if (rhs_data->first > place_data->last)
place_data->sum += (rhs_data->first - place_data->last);
place_data->sum += rhs_data->sum;
place_data->last = rhs_data->last;
place_data->last_ts = rhs_data->last_ts;
}
else if (before(rhs_data, place_data))
{
// This state came after the rhs state
if (place_data->first > rhs_data->last)
place_data->sum += (place_data->first - rhs_data->last);
place_data->sum += rhs_data->sum;
place_data->first = rhs_data->first;
place_data->first_ts = rhs_data->first_ts;
}
else
{
// If none of those conditions matched, it means both states we are merging have all
// same timestamps. We have to pick either the smaller or larger value so that the
// result is deterministic.
if (place_data->first < rhs_data->first)
{
place_data->first = rhs_data->first;
place_data->last = rhs_data->last;
}
}
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinaryLittleEndian(this->data(place).sum, buf);
writeBinaryLittleEndian(this->data(place).first, buf);
writeBinaryLittleEndian(this->data(place).first_ts, buf);
writeBinaryLittleEndian(this->data(place).last, buf);
writeBinaryLittleEndian(this->data(place).last_ts, buf);
writeBinaryLittleEndian(this->data(place).seen, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinaryLittleEndian(this->data(place).sum, buf);
readBinaryLittleEndian(this->data(place).first, buf);
readBinaryLittleEndian(this->data(place).first_ts, buf);
readBinaryLittleEndian(this->data(place).last, buf);
readBinaryLittleEndian(this->data(place).last_ts, buf);
readBinaryLittleEndian(this->data(place).seen, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnVector<ValueType> &>(to).getData().push_back(this->data(place).sum);
}
};
}

View File

@ -1,8 +1,18 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionEntropy.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/UniqVariadicHash.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <cmath>
namespace DB
{
@ -16,6 +26,133 @@ namespace ErrorCodes
namespace
{
/** Calculates Shannon Entropy, using HashMap and computing empirical distribution function.
* Entropy is measured in bits (base-2 logarithm is used).
*/
template <typename Value>
struct EntropyData
{
using Weight = UInt64;
using HashingMap = HashMapWithStackMemory<Value, Weight, HashCRC32<Value>, 4>;
/// For the case of pre-hashed values.
using TrivialMap = HashMapWithStackMemory<Value, Weight, UInt128TrivialHash, 4>;
using Map = std::conditional_t<std::is_same_v<UInt128, Value>, TrivialMap, HashingMap>;
Map map;
void add(const Value & x)
{
if (!isNaN(x))
++map[x];
}
void add(const Value & x, const Weight & weight)
{
if (!isNaN(x))
map[x] += weight;
}
void merge(const EntropyData & rhs)
{
for (const auto & pair : rhs.map)
map[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
map.write(buf);
}
void deserialize(ReadBuffer & buf)
{
typename Map::Reader reader(buf);
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
}
}
Float64 get() const
{
UInt64 total_value = 0;
for (const auto & pair : map)
total_value += pair.getMapped();
Float64 shannon_entropy = 0;
for (const auto & pair : map)
{
Float64 frequency = Float64(pair.getMapped()) / total_value;
shannon_entropy -= frequency * log2(frequency);
}
return shannon_entropy;
}
};
template <typename Value>
class AggregateFunctionEntropy final : public IAggregateFunctionDataHelper<EntropyData<Value>, AggregateFunctionEntropy<Value>>
{
private:
size_t num_args;
public:
explicit AggregateFunctionEntropy(const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<EntropyData<Value>, AggregateFunctionEntropy<Value>>(argument_types_, {}, createResultType())
, num_args(argument_types_.size())
{
}
String getName() const override { return "entropy"; }
static DataTypePtr createResultType()
{
return std::make_shared<DataTypeNumber<Float64>>();
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
if constexpr (!std::is_same_v<UInt128, Value>)
{
/// Here we manage only with numerical types
const auto & column = assert_cast<const ColumnVector <Value> &>(*columns[0]);
this->data(place).add(column.getData()[row_num]);
}
else
{
this->data(place).add(UniqVariadicHash<true, false>::apply(num_args, columns, row_num));
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(const_cast<AggregateDataPtr>(place)).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & column = assert_cast<ColumnVector<Float64> &>(to);
column.getData().push_back(this->data(place).get());
}
};
AggregateFunctionPtr createAggregateFunctionEntropy(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{

View File

@ -1,145 +0,0 @@
#pragma once
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/UniqVariadicHash.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnVector.h>
#include <Common/assert_cast.h>
#include <cmath>
namespace DB
{
struct Settings;
/** Calculates Shannon Entropy, using HashMap and computing empirical distribution function.
* Entropy is measured in bits (base-2 logarithm is used).
*/
template <typename Value>
struct EntropyData
{
using Weight = UInt64;
using HashingMap = HashMapWithStackMemory<Value, Weight, HashCRC32<Value>, 4>;
/// For the case of pre-hashed values.
using TrivialMap = HashMapWithStackMemory<Value, Weight, UInt128TrivialHash, 4>;
using Map = std::conditional_t<std::is_same_v<UInt128, Value>, TrivialMap, HashingMap>;
Map map;
void add(const Value & x)
{
if (!isNaN(x))
++map[x];
}
void add(const Value & x, const Weight & weight)
{
if (!isNaN(x))
map[x] += weight;
}
void merge(const EntropyData & rhs)
{
for (const auto & pair : rhs.map)
map[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
map.write(buf);
}
void deserialize(ReadBuffer & buf)
{
typename Map::Reader reader(buf);
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
}
}
Float64 get() const
{
UInt64 total_value = 0;
for (const auto & pair : map)
total_value += pair.getMapped();
Float64 shannon_entropy = 0;
for (const auto & pair : map)
{
Float64 frequency = Float64(pair.getMapped()) / total_value;
shannon_entropy -= frequency * log2(frequency);
}
return shannon_entropy;
}
};
template <typename Value>
class AggregateFunctionEntropy final : public IAggregateFunctionDataHelper<EntropyData<Value>, AggregateFunctionEntropy<Value>>
{
private:
size_t num_args;
public:
explicit AggregateFunctionEntropy(const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<EntropyData<Value>, AggregateFunctionEntropy<Value>>(argument_types_, {}, createResultType())
, num_args(argument_types_.size())
{
}
String getName() const override { return "entropy"; }
static DataTypePtr createResultType()
{
return std::make_shared<DataTypeNumber<Float64>>();
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
if constexpr (!std::is_same_v<UInt128, Value>)
{
/// Here we manage only with numerical types
const auto & column = assert_cast<const ColumnVector <Value> &>(*columns[0]);
this->data(place).add(column.getData()[row_num]);
}
else
{
this->data(place).add(UniqVariadicHash<true, false>::apply(num_args, columns, row_num));
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(const_cast<AggregateDataPtr>(place)).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & column = assert_cast<ColumnVector<Float64> &>(to);
column.getData().push_back(this->data(place).get());
}
};
}

View File

@ -1,9 +1,31 @@
#include <AggregateFunctions/AggregateFunctionHistogram.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <Common/FieldVisitorConvertToNumber.h>
#include <Common/NaNUtils.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/VarInt.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <math.h>
#include <queue>
#include <stddef.h>
namespace DB
{
@ -16,12 +38,357 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int UNSUPPORTED_PARAMETER;
extern const int PARAMETER_OUT_OF_BOUND;
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int INCORRECT_DATA;
}
namespace
{
/** distance compression algorithm implementation
* http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
*/
class AggregateFunctionHistogramData
{
public:
using Mean = Float64;
using Weight = Float64;
constexpr static size_t bins_count_limit = 250;
private:
struct WeightedValue
{
Mean mean;
Weight weight;
WeightedValue operator+(const WeightedValue & other) const
{
return {mean + other.weight * (other.mean - mean) / (other.weight + weight), other.weight + weight};
}
};
// quantity of stored weighted-values
UInt32 size;
// calculated lower and upper bounds of seen points
Mean lower_bound;
Mean upper_bound;
// Weighted values representation of histogram.
WeightedValue points[0];
void sort()
{
::sort(points, points + size,
[](const WeightedValue & first, const WeightedValue & second)
{
return first.mean < second.mean;
});
}
template <typename T>
struct PriorityQueueStorage
{
size_t size = 0;
T * data_ptr;
explicit PriorityQueueStorage(T * value)
: data_ptr(value)
{
}
void push_back(T val) /// NOLINT
{
data_ptr[size] = std::move(val);
++size;
}
void pop_back() { --size; } /// NOLINT
T * begin() { return data_ptr; }
T * end() const { return data_ptr + size; }
bool empty() const { return size == 0; }
T & front() { return *data_ptr; }
const T & front() const { return *data_ptr; }
using value_type = T;
using reference = T&;
using const_reference = const T&;
using size_type = size_t;
};
/**
* Repeatedly fuse most close values until max_bins bins left
*/
void compress(UInt32 max_bins)
{
sort();
auto new_size = size;
if (size <= max_bins)
return;
// Maintain doubly-linked list of "active" points
// and store neighbour pairs in priority queue by distance
UInt32 previous[size + 1];
UInt32 next[size + 1];
bool active[size + 1];
std::fill(active, active + size, true);
active[size] = false;
auto delete_node = [&](UInt32 i)
{
previous[next[i]] = previous[i];
next[previous[i]] = next[i];
active[i] = false;
};
for (size_t i = 0; i <= size; ++i)
{
previous[i] = static_cast<UInt32>(i - 1);
next[i] = static_cast<UInt32>(i + 1);
}
next[size] = 0;
previous[0] = size;
using QueueItem = std::pair<Mean, UInt32>;
QueueItem storage[2 * size - max_bins];
std::priority_queue<
QueueItem,
PriorityQueueStorage<QueueItem>,
std::greater<QueueItem>>
queue{std::greater<QueueItem>(),
PriorityQueueStorage<QueueItem>(storage)};
auto quality = [&](UInt32 i) { return points[next[i]].mean - points[i].mean; };
for (size_t i = 0; i + 1 < size; ++i)
queue.push({quality(static_cast<UInt32>(i)), i});
while (new_size > max_bins && !queue.empty())
{
auto min_item = queue.top();
queue.pop();
auto left = min_item.second;
auto right = next[left];
if (!active[left] || !active[right] || quality(left) > min_item.first)
continue;
points[left] = points[left] + points[right];
delete_node(right);
if (active[next[left]])
queue.push({quality(left), left});
if (active[previous[left]])
queue.push({quality(previous[left]), previous[left]});
--new_size;
}
size_t left = 0;
for (size_t right = 0; right < size; ++right)
{
if (active[right])
{
points[left] = points[right];
++left;
}
}
size = new_size;
}
/***
* Delete too close points from histogram.
* Assumes that points are sorted.
*/
void unique()
{
if (size == 0)
return;
size_t left = 0;
for (auto right = left + 1; right < size; ++right)
{
// Fuse points if their text representations differ only in last digit
auto min_diff = 10 * (points[left].mean + points[right].mean) * std::numeric_limits<Mean>::epsilon();
if (points[left].mean + std::fabs(min_diff) >= points[right].mean)
{
points[left] = points[left] + points[right];
}
else
{
++left;
points[left] = points[right];
}
}
size = static_cast<UInt32>(left + 1);
}
public:
AggregateFunctionHistogramData()
: size(0)
, lower_bound(std::numeric_limits<Mean>::max())
, upper_bound(std::numeric_limits<Mean>::lowest())
{
static_assert(offsetof(AggregateFunctionHistogramData, points) == sizeof(AggregateFunctionHistogramData), "points should be last member");
}
static size_t structSize(size_t max_bins)
{
return sizeof(AggregateFunctionHistogramData) + max_bins * 2 * sizeof(WeightedValue);
}
void insertResultInto(ColumnVector<Mean> & to_lower, ColumnVector<Mean> & to_upper, ColumnVector<Weight> & to_weights, UInt32 max_bins)
{
compress(max_bins);
unique();
for (size_t i = 0; i < size; ++i)
{
to_lower.insertValue((i == 0) ? lower_bound : (points[i].mean + points[i - 1].mean) / 2);
to_upper.insertValue((i + 1 == size) ? upper_bound : (points[i].mean + points[i + 1].mean) / 2);
// linear density approximation
Weight lower_weight = (i == 0) ? points[i].weight : ((points[i - 1].weight) + points[i].weight * 3) / 4;
Weight upper_weight = (i + 1 == size) ? points[i].weight : (points[i + 1].weight + points[i].weight * 3) / 4;
to_weights.insertValue((lower_weight + upper_weight) / 2);
}
}
void add(Mean value, Weight weight, UInt32 max_bins)
{
// nans break sort and compression
// infs don't fit in bins partition method
if (!isFinite(value))
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid value (inf or nan) for aggregation by 'histogram' function");
points[size] = {value, weight};
++size;
lower_bound = std::min(lower_bound, value);
upper_bound = std::max(upper_bound, value);
if (size >= max_bins * 2)
compress(max_bins);
}
void merge(const AggregateFunctionHistogramData & other, UInt32 max_bins)
{
lower_bound = std::min(lower_bound, other.lower_bound);
upper_bound = std::max(upper_bound, other.upper_bound);
for (size_t i = 0; i < other.size; ++i)
add(other.points[i].mean, other.points[i].weight, max_bins);
}
void write(WriteBuffer & buf) const
{
writeBinary(lower_bound, buf);
writeBinary(upper_bound, buf);
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(points), size * sizeof(WeightedValue));
}
void read(ReadBuffer & buf, UInt32 max_bins)
{
readBinary(lower_bound, buf);
readBinary(upper_bound, buf);
readVarUInt(size, buf);
if (size > max_bins * 2)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too many bins");
static constexpr size_t max_size = 1_GiB;
if (size > max_size)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Too large array size in histogram (maximum: {})", max_size);
buf.readStrict(reinterpret_cast<char *>(points), size * sizeof(WeightedValue));
}
};
template <typename T>
class AggregateFunctionHistogram final: public IAggregateFunctionDataHelper<AggregateFunctionHistogramData, AggregateFunctionHistogram<T>>
{
private:
using Data = AggregateFunctionHistogramData;
const UInt32 max_bins;
public:
AggregateFunctionHistogram(UInt32 max_bins_, const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionHistogramData, AggregateFunctionHistogram<T>>(arguments, params, createResultType())
, max_bins(max_bins_)
{
}
size_t sizeOfData() const override
{
return Data::structSize(max_bins);
}
static DataTypePtr createResultType()
{
DataTypes types;
auto mean = std::make_shared<DataTypeNumber<Data::Mean>>();
auto weight = std::make_shared<DataTypeNumber<Data::Weight>>();
// lower bound
types.emplace_back(mean);
// upper bound
types.emplace_back(mean);
// weight
types.emplace_back(weight);
auto tuple = std::make_shared<DataTypeTuple>(types);
return std::make_shared<DataTypeArray>(tuple);
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto val = assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
this->data(place).add(static_cast<Data::Mean>(val), 1, max_bins);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), max_bins);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf, max_bins);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & data = this->data(place);
auto & to_array = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = to_array.getOffsets();
auto & to_tuple = assert_cast<ColumnTuple &>(to_array.getData());
auto & to_lower = assert_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(0));
auto & to_upper = assert_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(1));
auto & to_weights = assert_cast<ColumnVector<Data::Weight> &>(to_tuple.getColumn(2));
data.insertResultInto(to_lower, to_upper, to_weights, max_bins);
offsets_to.push_back(to_tuple.size());
}
String getName() const override { return "histogram"; }
};
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
if (params.size() != 1)

View File

@ -1,382 +0,0 @@
#pragma once
#include <base/sort.h>
#include <Common/NaNUtils.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/VarInt.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <math.h>
#include <queue>
#include <stddef.h>
namespace DB
{
struct Settings;
class Arena;
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
extern const int INCORRECT_DATA;
}
/**
* distance compression algorithm implementation
* http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf
*/
class AggregateFunctionHistogramData
{
public:
using Mean = Float64;
using Weight = Float64;
constexpr static size_t bins_count_limit = 250;
private:
struct WeightedValue
{
Mean mean;
Weight weight;
WeightedValue operator+(const WeightedValue & other) const
{
return {mean + other.weight * (other.mean - mean) / (other.weight + weight), other.weight + weight};
}
};
// quantity of stored weighted-values
UInt32 size;
// calculated lower and upper bounds of seen points
Mean lower_bound;
Mean upper_bound;
// Weighted values representation of histogram.
WeightedValue points[0];
void sort()
{
::sort(points, points + size,
[](const WeightedValue & first, const WeightedValue & second)
{
return first.mean < second.mean;
});
}
template <typename T>
struct PriorityQueueStorage
{
size_t size = 0;
T * data_ptr;
explicit PriorityQueueStorage(T * value)
: data_ptr(value)
{
}
void push_back(T val) /// NOLINT
{
data_ptr[size] = std::move(val);
++size;
}
void pop_back() { --size; } /// NOLINT
T * begin() { return data_ptr; }
T * end() const { return data_ptr + size; }
bool empty() const { return size == 0; }
T & front() { return *data_ptr; }
const T & front() const { return *data_ptr; }
using value_type = T;
using reference = T&;
using const_reference = const T&;
using size_type = size_t;
};
/**
* Repeatedly fuse most close values until max_bins bins left
*/
void compress(UInt32 max_bins)
{
sort();
auto new_size = size;
if (size <= max_bins)
return;
// Maintain doubly-linked list of "active" points
// and store neighbour pairs in priority queue by distance
UInt32 previous[size + 1];
UInt32 next[size + 1];
bool active[size + 1];
std::fill(active, active + size, true);
active[size] = false;
auto delete_node = [&](UInt32 i)
{
previous[next[i]] = previous[i];
next[previous[i]] = next[i];
active[i] = false;
};
for (size_t i = 0; i <= size; ++i)
{
previous[i] = static_cast<UInt32>(i - 1);
next[i] = static_cast<UInt32>(i + 1);
}
next[size] = 0;
previous[0] = size;
using QueueItem = std::pair<Mean, UInt32>;
QueueItem storage[2 * size - max_bins];
std::priority_queue<
QueueItem,
PriorityQueueStorage<QueueItem>,
std::greater<QueueItem>>
queue{std::greater<QueueItem>(),
PriorityQueueStorage<QueueItem>(storage)};
auto quality = [&](UInt32 i) { return points[next[i]].mean - points[i].mean; };
for (size_t i = 0; i + 1 < size; ++i)
queue.push({quality(static_cast<UInt32>(i)), i});
while (new_size > max_bins && !queue.empty())
{
auto min_item = queue.top();
queue.pop();
auto left = min_item.second;
auto right = next[left];
if (!active[left] || !active[right] || quality(left) > min_item.first)
continue;
points[left] = points[left] + points[right];
delete_node(right);
if (active[next[left]])
queue.push({quality(left), left});
if (active[previous[left]])
queue.push({quality(previous[left]), previous[left]});
--new_size;
}
size_t left = 0;
for (size_t right = 0; right < size; ++right)
{
if (active[right])
{
points[left] = points[right];
++left;
}
}
size = new_size;
}
/***
* Delete too close points from histogram.
* Assumes that points are sorted.
*/
void unique()
{
if (size == 0)
return;
size_t left = 0;
for (auto right = left + 1; right < size; ++right)
{
// Fuse points if their text representations differ only in last digit
auto min_diff = 10 * (points[left].mean + points[right].mean) * std::numeric_limits<Mean>::epsilon();
if (points[left].mean + std::fabs(min_diff) >= points[right].mean)
{
points[left] = points[left] + points[right];
}
else
{
++left;
points[left] = points[right];
}
}
size = static_cast<UInt32>(left + 1);
}
public:
AggregateFunctionHistogramData()
: size(0)
, lower_bound(std::numeric_limits<Mean>::max())
, upper_bound(std::numeric_limits<Mean>::lowest())
{
static_assert(offsetof(AggregateFunctionHistogramData, points) == sizeof(AggregateFunctionHistogramData), "points should be last member");
}
static size_t structSize(size_t max_bins)
{
return sizeof(AggregateFunctionHistogramData) + max_bins * 2 * sizeof(WeightedValue);
}
void insertResultInto(ColumnVector<Mean> & to_lower, ColumnVector<Mean> & to_upper, ColumnVector<Weight> & to_weights, UInt32 max_bins)
{
compress(max_bins);
unique();
for (size_t i = 0; i < size; ++i)
{
to_lower.insertValue((i == 0) ? lower_bound : (points[i].mean + points[i - 1].mean) / 2);
to_upper.insertValue((i + 1 == size) ? upper_bound : (points[i].mean + points[i + 1].mean) / 2);
// linear density approximation
Weight lower_weight = (i == 0) ? points[i].weight : ((points[i - 1].weight) + points[i].weight * 3) / 4;
Weight upper_weight = (i + 1 == size) ? points[i].weight : (points[i + 1].weight + points[i].weight * 3) / 4;
to_weights.insertValue((lower_weight + upper_weight) / 2);
}
}
void add(Mean value, Weight weight, UInt32 max_bins)
{
// nans break sort and compression
// infs don't fit in bins partition method
if (!isFinite(value))
throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid value (inf or nan) for aggregation by 'histogram' function");
points[size] = {value, weight};
++size;
lower_bound = std::min(lower_bound, value);
upper_bound = std::max(upper_bound, value);
if (size >= max_bins * 2)
compress(max_bins);
}
void merge(const AggregateFunctionHistogramData & other, UInt32 max_bins)
{
lower_bound = std::min(lower_bound, other.lower_bound);
upper_bound = std::max(upper_bound, other.upper_bound);
for (size_t i = 0; i < other.size; ++i)
add(other.points[i].mean, other.points[i].weight, max_bins);
}
void write(WriteBuffer & buf) const
{
writeBinary(lower_bound, buf);
writeBinary(upper_bound, buf);
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(points), size * sizeof(WeightedValue));
}
void read(ReadBuffer & buf, UInt32 max_bins)
{
readBinary(lower_bound, buf);
readBinary(upper_bound, buf);
readVarUInt(size, buf);
if (size > max_bins * 2)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too many bins");
static constexpr size_t max_size = 1_GiB;
if (size > max_size)
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Too large array size in histogram (maximum: {})", max_size);
buf.readStrict(reinterpret_cast<char *>(points), size * sizeof(WeightedValue));
}
};
template <typename T>
class AggregateFunctionHistogram final: public IAggregateFunctionDataHelper<AggregateFunctionHistogramData, AggregateFunctionHistogram<T>>
{
private:
using Data = AggregateFunctionHistogramData;
const UInt32 max_bins;
public:
AggregateFunctionHistogram(UInt32 max_bins_, const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionHistogramData, AggregateFunctionHistogram<T>>(arguments, params, createResultType())
, max_bins(max_bins_)
{
}
size_t sizeOfData() const override
{
return Data::structSize(max_bins);
}
static DataTypePtr createResultType()
{
DataTypes types;
auto mean = std::make_shared<DataTypeNumber<Data::Mean>>();
auto weight = std::make_shared<DataTypeNumber<Data::Weight>>();
// lower bound
types.emplace_back(mean);
// upper bound
types.emplace_back(mean);
// weight
types.emplace_back(weight);
auto tuple = std::make_shared<DataTypeTuple>(types);
return std::make_shared<DataTypeArray>(tuple);
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto val = assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num];
this->data(place).add(static_cast<Data::Mean>(val), 1, max_bins);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), max_bins);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf, max_bins);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & data = this->data(place);
auto & to_array = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = to_array.getOffsets();
auto & to_tuple = assert_cast<ColumnTuple &>(to_array.getData());
auto & to_lower = assert_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(0));
auto & to_upper = assert_cast<ColumnVector<Data::Mean> &>(to_tuple.getColumn(1));
auto & to_weights = assert_cast<ColumnVector<Data::Weight> &>(to_tuple.getColumn(2));
data.insertResultInto(to_lower, to_upper, to_weights, max_bins);
offsets_to.push_back(to_tuple.size());
}
String getName() const override { return "histogram"; }
};
}

View File

@ -1,57 +1,272 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionIntervalLengthSum.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <base/range.h>
#include <unordered_set>
#include <AggregateFunctions/Combinators/AggregateFunctionNull.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_LARGE_ARRAY_SIZE;
}
struct Settings;
namespace
{
template <template <typename> class Data>
AggregateFunctionPtr
createAggregateFunctionIntervalLengthSum(const std::string & name, const DataTypes & arguments, const Array &, const Settings *)
/** Calculate total length of intervals without intersections. Each interval is the pair of numbers [begin, end];
* Returns UInt64 for integral types (UInt/Int*, Date/DateTime) and returns Float64 for Float*.
*
* Implementation simply stores intervals sorted by beginning and sums lengths at final.
*/
template <typename T>
struct AggregateFunctionIntervalLengthSumData
{
constexpr static size_t MAX_ARRAY_SIZE = 0xFFFFFF;
using Segment = std::pair<T, T>;
using Segments = PODArrayWithStackMemory<Segment, 64>;
bool sorted = false;
Segments segments;
void add(T begin, T end)
{
if (arguments.size() != 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function {} requires two timestamps argument.", name);
/// Reversed intervals are counted by absolute value of their length.
if (unlikely(end < begin))
std::swap(begin, end);
else if (unlikely(begin == end))
return;
auto args = {arguments[0].get(), arguments[1].get()};
if (sorted && !segments.empty())
sorted = segments.back().first <= begin;
segments.emplace_back(begin, end);
}
if (WhichDataType{args.begin()[0]}.idx != WhichDataType{args.begin()[1]}.idx)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal types {} and {} of arguments "
"of aggregate function {}, both arguments should have same data type",
args.begin()[0]->getName(), args.begin()[1]->getName(), name);
void merge(const AggregateFunctionIntervalLengthSumData & other)
{
if (other.segments.empty())
return;
for (const auto & arg : args)
const auto size = segments.size();
segments.insert(std::begin(other.segments), std::end(other.segments));
/// either sort whole container or do so partially merging ranges afterwards
if (!sorted && !other.sorted)
{
if (!isNativeNumber(arg) && !isDate(arg) && !isDateTime(arg))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of aggregate function {}, must "
"be native integral type, Date/DateTime or Float", arg->getName(), name);
::sort(std::begin(segments), std::end(segments));
}
else
{
const auto begin = std::begin(segments);
const auto middle = std::next(begin, size);
const auto end = std::end(segments);
if (!sorted)
::sort(begin, middle);
if (!other.sorted)
::sort(middle, end);
std::inplace_merge(begin, middle, end);
}
AggregateFunctionPtr res(createWithBasicNumberOrDateOrDateTime<AggregateFunctionIntervalLengthSum, Data>(*arguments[0], arguments));
sorted = true;
}
if (res)
return res;
void sort()
{
if (sorted)
return;
::sort(std::begin(segments), std::end(segments));
sorted = true;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(sorted, buf);
writeBinary(segments.size(), buf);
for (const auto & time_gap : segments)
{
writeBinary(time_gap.first, buf);
writeBinary(time_gap.second, buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(sorted, buf);
size_t size;
readBinary(size, buf);
if (unlikely(size > MAX_ARRAY_SIZE))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {})", MAX_ARRAY_SIZE);
segments.clear();
segments.reserve(size);
Segment segment;
for (size_t i = 0; i < size; ++i)
{
readBinary(segment.first, buf);
readBinary(segment.second, buf);
segments.emplace_back(segment);
}
}
};
template <typename T, typename Data>
class AggregateFunctionIntervalLengthSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionIntervalLengthSum<T, Data>>
{
private:
static auto NO_SANITIZE_UNDEFINED length(typename Data::Segment segment)
{
return segment.second - segment.first;
}
template <typename TResult>
TResult getIntervalLengthSum(Data & data) const
{
if (data.segments.empty())
return 0;
data.sort();
TResult res = 0;
typename Data::Segment curr_segment = data.segments[0];
for (size_t i = 1, size = data.segments.size(); i < size; ++i)
{
const typename Data::Segment & next_segment = data.segments[i];
/// Check if current interval intersects with next one then add length, otherwise advance interval end.
if (curr_segment.second < next_segment.first)
{
res += length(curr_segment);
curr_segment = next_segment;
}
else if (next_segment.second > curr_segment.second)
{
curr_segment.second = next_segment.second;
}
}
res += length(curr_segment);
return res;
}
public:
String getName() const override { return "intervalLengthSum"; }
explicit AggregateFunctionIntervalLengthSum(const DataTypes & arguments)
: IAggregateFunctionDataHelper<Data, AggregateFunctionIntervalLengthSum<T, Data>>(arguments, {}, createResultType())
{
}
static DataTypePtr createResultType()
{
if constexpr (std::is_floating_point_v<T>)
return std::make_shared<DataTypeFloat64>();
return std::make_shared<DataTypeUInt64>();
}
bool allocatesMemoryInArena() const override { return false; }
AggregateFunctionPtr getOwnNullAdapter(
const AggregateFunctionPtr & nested_function,
const DataTypes & arguments,
const Array & params,
const AggregateFunctionProperties & /*properties*/) const override
{
return std::make_shared<AggregateFunctionNullVariadic<false, false>>(nested_function, arguments, params);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
auto begin = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
auto end = assert_cast<const ColumnVector<T> *>(columns[1])->getData()[row_num];
this->data(place).add(begin, end);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if constexpr (std::is_floating_point_v<T>)
assert_cast<ColumnFloat64 &>(to).getData().push_back(getIntervalLengthSum<Float64>(this->data(place)));
else
assert_cast<ColumnUInt64 &>(to).getData().push_back(getIntervalLengthSum<UInt64>(this->data(place)));
}
};
template <template <typename> class Data>
AggregateFunctionPtr
createAggregateFunctionIntervalLengthSum(const std::string & name, const DataTypes & arguments, const Array &, const Settings *)
{
if (arguments.size() != 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function {} requires two timestamps argument.", name);
auto args = {arguments[0].get(), arguments[1].get()};
if (WhichDataType{args.begin()[0]}.idx != WhichDataType{args.begin()[1]}.idx)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of aggregate function {}, must "
"be native integral type, Date/DateTime or Float", arguments.front().get()->getName(), name);
"Illegal types {} and {} of arguments "
"of aggregate function {}, both arguments should have same data type",
args.begin()[0]->getName(), args.begin()[1]->getName(), name);
for (const auto & arg : args)
{
if (!isNativeNumber(arg) && !isDate(arg) && !isDateTime(arg))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of aggregate function {}, must "
"be native integral type, Date/DateTime or Float", arg->getName(), name);
}
AggregateFunctionPtr res(createWithBasicNumberOrDateOrDateTime<AggregateFunctionIntervalLengthSum, Data>(*arguments[0], arguments));
if (res)
return res;
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of aggregate function {}, must "
"be native integral type, Date/DateTime or Float", arguments.front().get()->getName(), name);
}
}

View File

@ -1,232 +0,0 @@
#pragma once
#include <unordered_set>
#include <AggregateFunctions/Combinators/AggregateFunctionNull.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <base/arithmeticOverflow.h>
#include <base/sort.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
}
/** Calculate total length of intervals without intersections. Each interval is the pair of numbers [begin, end];
* Returns UInt64 for integral types (UInt/Int*, Date/DateTime) and returns Float64 for Float*.
*
* Implementation simply stores intervals sorted by beginning and sums lengths at final.
*/
template <typename T>
struct AggregateFunctionIntervalLengthSumData
{
constexpr static size_t MAX_ARRAY_SIZE = 0xFFFFFF;
using Segment = std::pair<T, T>;
using Segments = PODArrayWithStackMemory<Segment, 64>;
bool sorted = false;
Segments segments;
void add(T begin, T end)
{
/// Reversed intervals are counted by absolute value of their length.
if (unlikely(end < begin))
std::swap(begin, end);
else if (unlikely(begin == end))
return;
if (sorted && !segments.empty())
sorted = segments.back().first <= begin;
segments.emplace_back(begin, end);
}
void merge(const AggregateFunctionIntervalLengthSumData & other)
{
if (other.segments.empty())
return;
const auto size = segments.size();
segments.insert(std::begin(other.segments), std::end(other.segments));
/// either sort whole container or do so partially merging ranges afterwards
if (!sorted && !other.sorted)
{
::sort(std::begin(segments), std::end(segments));
}
else
{
const auto begin = std::begin(segments);
const auto middle = std::next(begin, size);
const auto end = std::end(segments);
if (!sorted)
::sort(begin, middle);
if (!other.sorted)
::sort(middle, end);
std::inplace_merge(begin, middle, end);
}
sorted = true;
}
void sort()
{
if (sorted)
return;
::sort(std::begin(segments), std::end(segments));
sorted = true;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(sorted, buf);
writeBinary(segments.size(), buf);
for (const auto & time_gap : segments)
{
writeBinary(time_gap.first, buf);
writeBinary(time_gap.second, buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(sorted, buf);
size_t size;
readBinary(size, buf);
if (unlikely(size > MAX_ARRAY_SIZE))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {})", MAX_ARRAY_SIZE);
segments.clear();
segments.reserve(size);
Segment segment;
for (size_t i = 0; i < size; ++i)
{
readBinary(segment.first, buf);
readBinary(segment.second, buf);
segments.emplace_back(segment);
}
}
};
template <typename T, typename Data>
class AggregateFunctionIntervalLengthSum final : public IAggregateFunctionDataHelper<Data, AggregateFunctionIntervalLengthSum<T, Data>>
{
private:
static auto NO_SANITIZE_UNDEFINED length(typename Data::Segment segment)
{
return segment.second - segment.first;
}
template <typename TResult>
TResult getIntervalLengthSum(Data & data) const
{
if (data.segments.empty())
return 0;
data.sort();
TResult res = 0;
typename Data::Segment curr_segment = data.segments[0];
for (size_t i = 1, size = data.segments.size(); i < size; ++i)
{
const typename Data::Segment & next_segment = data.segments[i];
/// Check if current interval intersects with next one then add length, otherwise advance interval end.
if (curr_segment.second < next_segment.first)
{
res += length(curr_segment);
curr_segment = next_segment;
}
else if (next_segment.second > curr_segment.second)
{
curr_segment.second = next_segment.second;
}
}
res += length(curr_segment);
return res;
}
public:
String getName() const override { return "intervalLengthSum"; }
explicit AggregateFunctionIntervalLengthSum(const DataTypes & arguments)
: IAggregateFunctionDataHelper<Data, AggregateFunctionIntervalLengthSum<T, Data>>(arguments, {}, createResultType())
{
}
static DataTypePtr createResultType()
{
if constexpr (std::is_floating_point_v<T>)
return std::make_shared<DataTypeFloat64>();
return std::make_shared<DataTypeUInt64>();
}
bool allocatesMemoryInArena() const override { return false; }
AggregateFunctionPtr getOwnNullAdapter(
const AggregateFunctionPtr & nested_function,
const DataTypes & arguments,
const Array & params,
const AggregateFunctionProperties & /*properties*/) const override
{
return std::make_shared<AggregateFunctionNullVariadic<false, false>>(nested_function, arguments, params);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
auto begin = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
auto end = assert_cast<const ColumnVector<T> *>(columns[1])->getData()[row_num];
this->data(place).add(begin, end);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if constexpr (std::is_floating_point_v<T>)
assert_cast<ColumnFloat64 &>(to).getData().push_back(getIntervalLengthSum<Float64>(this->data(place)));
else
assert_cast<ColumnUInt64 &>(to).getData().push_back(getIntervalLengthSum<UInt64>(this->data(place)));
}
};
}

View File

@ -1,19 +1,339 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionKolmogorovSmirnovTest.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/Exception.h>
#include <Common/assert_cast.h>
#include <Common/PODArray_fwd.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/ReadHelpers.h>
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace DB
{
struct Settings;
namespace
{
struct KolmogorovSmirnov : public StatisticalSample<Float64, Float64>
{
enum class Alternative
{
TwoSided,
Less,
Greater
};
std::pair<Float64, Float64> getResult(Alternative alternative, String method)
{
::sort(x.begin(), x.end());
::sort(y.begin(), y.end());
Float64 max_s = std::numeric_limits<Float64>::min();
Float64 min_s = std::numeric_limits<Float64>::max();
Float64 now_s = 0;
UInt64 pos_x = 0;
UInt64 pos_y = 0;
UInt64 pos_tmp;
UInt64 n1 = x.size();
UInt64 n2 = y.size();
const Float64 n1_d = 1. / n1;
const Float64 n2_d = 1. / n2;
const Float64 tol = 1e-7;
// reference: https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test
while (pos_x < x.size() && pos_y < y.size())
{
if (likely(fabs(x[pos_x] - y[pos_y]) >= tol))
{
if (x[pos_x] < y[pos_y])
{
now_s += n1_d;
++pos_x;
}
else
{
now_s -= n2_d;
++pos_y;
}
}
else
{
pos_tmp = pos_x + 1;
while (pos_tmp < x.size() && unlikely(fabs(x[pos_tmp] - x[pos_x]) <= tol))
pos_tmp++;
now_s += n1_d * (pos_tmp - pos_x);
pos_x = pos_tmp;
pos_tmp = pos_y + 1;
while (pos_tmp < y.size() && unlikely(fabs(y[pos_tmp] - y[pos_y]) <= tol))
pos_tmp++;
now_s -= n2_d * (pos_tmp - pos_y);
pos_y = pos_tmp;
}
max_s = std::max(max_s, now_s);
min_s = std::min(min_s, now_s);
}
now_s += n1_d * (x.size() - pos_x) - n2_d * (y.size() - pos_y);
min_s = std::min(min_s, now_s);
max_s = std::max(max_s, now_s);
Float64 d = 0;
if (alternative == Alternative::TwoSided)
d = std::max(std::abs(max_s), std::abs(min_s));
else if (alternative == Alternative::Less)
d = -min_s;
else if (alternative == Alternative::Greater)
d = max_s;
UInt64 g = std::__gcd(n1, n2);
UInt64 nx_g = n1 / g;
UInt64 ny_g = n2 / g;
if (method == "auto")
method = std::max(n1, n2) <= 10000 ? "exact" : "asymptotic";
else if (method == "exact" && nx_g >= std::numeric_limits<Int32>::max() / ny_g)
method = "asymptotic";
Float64 p_value = std::numeric_limits<Float64>::infinity();
if (method == "exact")
{
/* reference:
* Gunar Schröer and Dietrich Trenkler
* Exact and Randomization Distributions of Kolmogorov-Smirnov, Tests for Two or Three Samples
*
* and
*
* Thomas Viehmann
* Numerically more stable computation of the p-values for the two-sample Kolmogorov-Smirnov test
*/
if (n2 > n1)
std::swap(n1, n2);
const Float64 f_n1 = static_cast<Float64>(n1);
const Float64 f_n2 = static_cast<Float64>(n2);
const Float64 k_d = (0.5 + floor(d * f_n2 * f_n1 - tol)) / (f_n2 * f_n1);
PaddedPODArray<Float64> c(n1 + 1);
auto check = alternative == Alternative::TwoSided ?
[](const Float64 & q, const Float64 & r, const Float64 & s) { return fabs(r - s) >= q; }
: [](const Float64 & q, const Float64 & r, const Float64 & s) { return r - s >= q; };
c[0] = 0;
for (UInt64 j = 1; j <= n1; j++)
if (check(k_d, 0., j / f_n1))
c[j] = 1.;
else
c[j] = c[j - 1];
for (UInt64 i = 1; i <= n2; i++)
{
if (check(k_d, i / f_n2, 0.))
c[0] = 1.;
for (UInt64 j = 1; j <= n1; j++)
if (check(k_d, i / f_n2, j / f_n1))
c[j] = 1.;
else
{
Float64 v = i / static_cast<Float64>(i + j);
Float64 w = j / static_cast<Float64>(i + j);
c[j] = v * c[j] + w * c[j - 1];
}
}
p_value = c[n1];
}
else if (method == "asymp" || method == "asymptotic")
{
Float64 n = std::min(n1, n2);
Float64 m = std::max(n1, n2);
Float64 p = sqrt((n * m) / (n + m)) * d;
if (alternative == Alternative::TwoSided)
{
/* reference:
* J.DURBIN
* Distribution theory for tests based on the sample distribution function
*/
Float64 new_val, old_val, s, w, z;
UInt64 k_max = static_cast<UInt64>(sqrt(2 - log(tol)));
if (p < 1)
{
z = - (M_PI_2 * M_PI_4) / (p * p);
w = log(p);
s = 0;
for (UInt64 k = 1; k < k_max; k += 2)
s += exp(k * k * z - w);
p = s / 0.398942280401432677939946059934;
}
else
{
z = -2 * p * p;
s = -1;
UInt64 k = 1;
old_val = 0;
new_val = 1;
while (fabs(old_val - new_val) > tol)
{
old_val = new_val;
new_val += 2 * s * exp(z * k * k);
s *= -1;
k++;
}
p = new_val;
}
p_value = 1 - p;
}
else
{
/* reference:
* J. L. HODGES, Jr
* The significance probability of the Smirnov two-sample test
*/
// Use Hodges' suggested approximation Eqn 5.3
// Requires m to be the larger of (n1, n2)
Float64 expt = -2 * p * p - 2 * p * (m + 2 * n) / sqrt(m * n * (m + n)) / 3.0;
p_value = exp(expt);
}
}
return {d, p_value};
}
};
class AggregateFunctionKolmogorovSmirnov final:
public IAggregateFunctionDataHelper<KolmogorovSmirnov, AggregateFunctionKolmogorovSmirnov>
{
private:
using Alternative = typename KolmogorovSmirnov::Alternative;
Alternative alternative = Alternative::TwoSided;
String method = "auto";
public:
explicit AggregateFunctionKolmogorovSmirnov(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<KolmogorovSmirnov, AggregateFunctionKolmogorovSmirnov> ({arguments}, {}, createResultType())
{
if (params.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName());
if (params.empty())
return;
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
alternative = Alternative::Less;
else if (param == "greater")
alternative = Alternative::Greater;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown parameter in aggregate function {}. "
"It must be one of: 'two-sided', 'less', 'greater'", getName());
if (params.size() != 2)
return;
if (params[1].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a String", getName());
method = params[1].get<String>();
if (method != "auto" && method != "exact" && method != "asymp" && method != "asymptotic")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown method in aggregate function {}. "
"It must be one of: 'auto', 'exact', 'asymp' (or 'asymptotic')", getName());
}
String getName() const override
{
return "kolmogorovSmirnovTest";
}
bool allocatesMemoryInArena() const override { return true; }
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
Strings names
{
"d_statistic",
"p_value"
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value, arena);
else
this->data(place).addX(value, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).merge(this->data(rhs), arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if (!this->data(place).size_x || !this->data(place).size_y)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());
auto [d_statistic, p_value] = this->data(place).getResult(alternative, method);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
column_stat.getData().push_back(d_statistic);
column_value.getData().push_back(p_value);
}
};
AggregateFunctionPtr createAggregateFunctionKolmogorovSmirnovTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{

View File

@ -1,331 +0,0 @@
#pragma once
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/Exception.h>
#include <Common/assert_cast.h>
#include <Common/PODArray_fwd.h>
#include <base/types.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
struct KolmogorovSmirnov : public StatisticalSample<Float64, Float64>
{
enum class Alternative
{
TwoSided,
Less,
Greater
};
std::pair<Float64, Float64> getResult(Alternative alternative, String method)
{
::sort(x.begin(), x.end());
::sort(y.begin(), y.end());
Float64 max_s = std::numeric_limits<Float64>::min();
Float64 min_s = std::numeric_limits<Float64>::max();
Float64 now_s = 0;
UInt64 pos_x = 0;
UInt64 pos_y = 0;
UInt64 pos_tmp;
UInt64 n1 = x.size();
UInt64 n2 = y.size();
const Float64 n1_d = 1. / n1;
const Float64 n2_d = 1. / n2;
const Float64 tol = 1e-7;
// reference: https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test
while (pos_x < x.size() && pos_y < y.size())
{
if (likely(fabs(x[pos_x] - y[pos_y]) >= tol))
{
if (x[pos_x] < y[pos_y])
{
now_s += n1_d;
++pos_x;
}
else
{
now_s -= n2_d;
++pos_y;
}
}
else
{
pos_tmp = pos_x + 1;
while (pos_tmp < x.size() && unlikely(fabs(x[pos_tmp] - x[pos_x]) <= tol))
pos_tmp++;
now_s += n1_d * (pos_tmp - pos_x);
pos_x = pos_tmp;
pos_tmp = pos_y + 1;
while (pos_tmp < y.size() && unlikely(fabs(y[pos_tmp] - y[pos_y]) <= tol))
pos_tmp++;
now_s -= n2_d * (pos_tmp - pos_y);
pos_y = pos_tmp;
}
max_s = std::max(max_s, now_s);
min_s = std::min(min_s, now_s);
}
now_s += n1_d * (x.size() - pos_x) - n2_d * (y.size() - pos_y);
min_s = std::min(min_s, now_s);
max_s = std::max(max_s, now_s);
Float64 d = 0;
if (alternative == Alternative::TwoSided)
d = std::max(std::abs(max_s), std::abs(min_s));
else if (alternative == Alternative::Less)
d = -min_s;
else if (alternative == Alternative::Greater)
d = max_s;
UInt64 g = std::__gcd(n1, n2);
UInt64 nx_g = n1 / g;
UInt64 ny_g = n2 / g;
if (method == "auto")
method = std::max(n1, n2) <= 10000 ? "exact" : "asymptotic";
else if (method == "exact" && nx_g >= std::numeric_limits<Int32>::max() / ny_g)
method = "asymptotic";
Float64 p_value = std::numeric_limits<Float64>::infinity();
if (method == "exact")
{
/* reference:
* Gunar Schröer and Dietrich Trenkler
* Exact and Randomization Distributions of Kolmogorov-Smirnov, Tests for Two or Three Samples
*
* and
*
* Thomas Viehmann
* Numerically more stable computation of the p-values for the two-sample Kolmogorov-Smirnov test
*/
if (n2 > n1)
std::swap(n1, n2);
const Float64 f_n1 = static_cast<Float64>(n1);
const Float64 f_n2 = static_cast<Float64>(n2);
const Float64 k_d = (0.5 + floor(d * f_n2 * f_n1 - tol)) / (f_n2 * f_n1);
PaddedPODArray<Float64> c(n1 + 1);
auto check = alternative == Alternative::TwoSided ?
[](const Float64 & q, const Float64 & r, const Float64 & s) { return fabs(r - s) >= q; }
: [](const Float64 & q, const Float64 & r, const Float64 & s) { return r - s >= q; };
c[0] = 0;
for (UInt64 j = 1; j <= n1; j++)
if (check(k_d, 0., j / f_n1))
c[j] = 1.;
else
c[j] = c[j - 1];
for (UInt64 i = 1; i <= n2; i++)
{
if (check(k_d, i / f_n2, 0.))
c[0] = 1.;
for (UInt64 j = 1; j <= n1; j++)
if (check(k_d, i / f_n2, j / f_n1))
c[j] = 1.;
else
{
Float64 v = i / static_cast<Float64>(i + j);
Float64 w = j / static_cast<Float64>(i + j);
c[j] = v * c[j] + w * c[j - 1];
}
}
p_value = c[n1];
}
else if (method == "asymp" || method == "asymptotic")
{
Float64 n = std::min(n1, n2);
Float64 m = std::max(n1, n2);
Float64 p = sqrt((n * m) / (n + m)) * d;
if (alternative == Alternative::TwoSided)
{
/* reference:
* J.DURBIN
* Distribution theory for tests based on the sample distribution function
*/
Float64 new_val, old_val, s, w, z;
UInt64 k_max = static_cast<UInt64>(sqrt(2 - log(tol)));
if (p < 1)
{
z = - (M_PI_2 * M_PI_4) / (p * p);
w = log(p);
s = 0;
for (UInt64 k = 1; k < k_max; k += 2)
s += exp(k * k * z - w);
p = s / 0.398942280401432677939946059934;
}
else
{
z = -2 * p * p;
s = -1;
UInt64 k = 1;
old_val = 0;
new_val = 1;
while (fabs(old_val - new_val) > tol)
{
old_val = new_val;
new_val += 2 * s * exp(z * k * k);
s *= -1;
k++;
}
p = new_val;
}
p_value = 1 - p;
}
else
{
/* reference:
* J. L. HODGES, Jr
* The significance probability of the Smirnov two-sample test
*/
// Use Hodges' suggested approximation Eqn 5.3
// Requires m to be the larger of (n1, n2)
Float64 expt = -2 * p * p - 2 * p * (m + 2 * n) / sqrt(m * n * (m + n)) / 3.0;
p_value = exp(expt);
}
}
return {d, p_value};
}
};
class AggregateFunctionKolmogorovSmirnov final:
public IAggregateFunctionDataHelper<KolmogorovSmirnov, AggregateFunctionKolmogorovSmirnov>
{
private:
using Alternative = typename KolmogorovSmirnov::Alternative;
Alternative alternative = Alternative::TwoSided;
String method = "auto";
public:
explicit AggregateFunctionKolmogorovSmirnov(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<KolmogorovSmirnov, AggregateFunctionKolmogorovSmirnov> ({arguments}, {}, createResultType())
{
if (params.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName());
if (params.empty())
return;
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
alternative = Alternative::Less;
else if (param == "greater")
alternative = Alternative::Greater;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown parameter in aggregate function {}. "
"It must be one of: 'two-sided', 'less', 'greater'", getName());
if (params.size() != 2)
return;
if (params[1].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a String", getName());
method = params[1].get<String>();
if (method != "auto" && method != "exact" && method != "asymp" && method != "asymptotic")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown method in aggregate function {}. "
"It must be one of: 'auto', 'exact', 'asymp' (or 'asymptotic')", getName());
}
String getName() const override
{
return "kolmogorovSmirnovTest";
}
bool allocatesMemoryInArena() const override { return true; }
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
Strings names
{
"d_statistic",
"p_value"
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value, arena);
else
this->data(place).addX(value, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).merge(this->data(rhs), arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if (!this->data(place).size_x || !this->data(place).size_y)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());
auto [d_statistic, p_value] = this->data(place).getResult(alternative, method);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
column_stat.getData().push_back(d_statistic);
column_value.getData().push_back(p_value);
}
};
}

View File

@ -1,12 +1,30 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <numeric>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsDateTime.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <Common/PODArray_fwd.h>
#include <Common/assert_cast.h>
#include <boost/math/distributions/normal.hpp>
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace DB
@ -16,29 +34,321 @@ struct Settings;
namespace
{
AggregateFunctionPtr
createAggregateFunctionLargestTriangleThreeBuckets(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
struct LargestTriangleThreeBucketsData : public StatisticalSample<Float64, Float64>
{
void add(const Float64 xval, const Float64 yval, Arena * arena)
{
assertBinary(name, argument_types);
if (!(isNumber(argument_types[0]) || isDateOrDate32(argument_types[0]) || isDateTime(argument_types[0])
|| isDateTime64(argument_types[0])))
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the first argument",
name);
if (!(isNumber(argument_types[1]) || isDateOrDate32(argument_types[1]) || isDateTime(argument_types[1])
|| isDateTime64(argument_types[1])))
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the second argument",
name);
return std::make_shared<AggregateFunctionLargestTriangleThreeBuckets>(argument_types, parameters);
this->addX(xval, arena);
this->addY(yval, arena);
}
void sort(Arena * arena)
{
// sort the this->x and this->y in ascending order of this->x using index
std::vector<size_t> index(this->x.size());
std::iota(index.begin(), index.end(), 0);
::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; });
SampleX temp_x{};
SampleY temp_y{};
for (size_t i = 0; i < this->x.size(); ++i)
{
temp_x.push_back(this->x[index[i]], arena);
temp_y.push_back(this->y[index[i]], arena);
}
for (size_t i = 0; i < this->x.size(); ++i)
{
this->x[i] = temp_x[i];
this->y[i] = temp_y[i];
}
}
PODArray<std::pair<Float64, Float64>> getResult(size_t total_buckets, Arena * arena)
{
// Sort the data
this->sort(arena);
PODArray<std::pair<Float64, Float64>> result;
// Handle special cases for small data list
if (this->x.size() <= total_buckets)
{
for (size_t i = 0; i < this->x.size(); ++i)
{
result.emplace_back(std::make_pair(this->x[i], this->y[i]));
}
return result;
}
// Handle special cases for 0 or 1 or 2 buckets
if (total_buckets == 0)
return result;
if (total_buckets == 1)
{
result.emplace_back(std::make_pair(this->x.front(), this->y.front()));
return result;
}
if (total_buckets == 2)
{
result.emplace_back(std::make_pair(this->x.front(), this->y.front()));
result.emplace_back(std::make_pair(this->x.back(), this->y.back()));
return result;
}
// Find the size of each bucket
size_t single_bucket_size = this->x.size() / total_buckets;
// Include the first data point
result.emplace_back(std::make_pair(this->x[0], this->y[0]));
for (size_t i = 1; i < total_buckets - 1; ++i) // Skip the first and last bucket
{
size_t start_index = i * single_bucket_size;
size_t end_index = (i + 1) * single_bucket_size;
// Compute the average point in the next bucket
Float64 avg_x = 0;
Float64 avg_y = 0;
for (size_t j = end_index; j < (i + 2) * single_bucket_size; ++j)
{
avg_x += this->x[j];
avg_y += this->y[j];
}
avg_x /= single_bucket_size;
avg_y /= single_bucket_size;
// Find the point in the current bucket that forms the largest triangle
size_t max_index = start_index;
Float64 max_area = 0.0;
for (size_t j = start_index; j < end_index; ++j)
{
Float64 area = std::abs(
0.5
* (result.back().first * this->y[j] + this->x[j] * avg_y + avg_x * result.back().second - result.back().first * avg_y
- this->x[j] * result.back().second - avg_x * this->y[j]));
if (area > max_area)
{
max_area = area;
max_index = j;
}
}
// Include the selected point
result.emplace_back(std::make_pair(this->x[max_index], this->y[max_index]));
}
// Include the last data point
result.emplace_back(std::make_pair(this->x.back(), this->y.back()));
return result;
}
};
class AggregateFunctionLargestTriangleThreeBuckets final : public IAggregateFunctionDataHelper<LargestTriangleThreeBucketsData, AggregateFunctionLargestTriangleThreeBuckets>
{
private:
UInt64 total_buckets{0};
TypeIndex x_type;
TypeIndex y_type;
public:
explicit AggregateFunctionLargestTriangleThreeBuckets(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<LargestTriangleThreeBucketsData, AggregateFunctionLargestTriangleThreeBuckets>({arguments}, {}, createResultType(arguments))
{
if (params.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require one parameter", getName());
if (params[0].getType() != Field::Types::UInt64)
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName());
total_buckets = params[0].get<UInt64>();
this->x_type = WhichDataType(arguments[0]).idx;
this->y_type = WhichDataType(arguments[1]).idx;
}
static constexpr auto name = "largestTriangleThreeBuckets";
String getName() const override { return name; }
bool allocatesMemoryInArena() const override { return true; }
static DataTypePtr createResultType(const DataTypes & arguments)
{
TypeIndex x_type = arguments[0]->getTypeId();
TypeIndex y_type = arguments[1]->getTypeId();
UInt32 x_scale = 0;
UInt32 y_scale = 0;
if (const auto * datetime64_type = typeid_cast<const DataTypeDateTime64 *>(arguments[0].get()))
{
x_scale = datetime64_type->getScale();
}
if (const auto * datetime64_type = typeid_cast<const DataTypeDateTime64 *>(arguments[1].get()))
{
y_scale = datetime64_type->getScale();
}
DataTypes types = {getDataTypeFromTypeIndex(x_type, x_scale), getDataTypeFromTypeIndex(y_type, y_scale)};
auto tuple = std::make_shared<DataTypeTuple>(std::move(types));
return std::make_shared<DataTypeArray>(tuple);
}
static DataTypePtr getDataTypeFromTypeIndex(TypeIndex type_index, UInt32 scale)
{
DataTypePtr data_type;
switch (type_index)
{
case TypeIndex::Date:
data_type = std::make_shared<DataTypeDate>();
break;
case TypeIndex::Date32:
data_type = std::make_shared<DataTypeDate32>();
break;
case TypeIndex::DateTime:
data_type = std::make_shared<DataTypeDateTime>();
break;
case TypeIndex::DateTime64:
data_type = std::make_shared<DataTypeDateTime64>(scale);
break;
default:
data_type = std::make_shared<DataTypeNumber<Float64>>();
}
return data_type;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 x = getFloat64DataFromColumn(columns[0], row_num, this->x_type);
Float64 y = getFloat64DataFromColumn(columns[1], row_num, this->y_type);
this->data(place).add(x, y, arena);
}
Float64 getFloat64DataFromColumn(const IColumn * column, size_t row_num, TypeIndex type_index) const
{
switch (type_index)
{
case TypeIndex::Date:
return static_cast<const ColumnDate &>(*column).getData()[row_num];
case TypeIndex::Date32:
return static_cast<const ColumnDate32 &>(*column).getData()[row_num];
case TypeIndex::DateTime:
return static_cast<const ColumnDateTime &>(*column).getData()[row_num];
case TypeIndex::DateTime64:
return static_cast<const ColumnDateTime64 &>(*column).getData()[row_num];
default:
return column->getFloat64(row_num);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
const auto & b = this->data(rhs);
a.merge(b, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override
{
auto res = this->data(place).getResult(total_buckets, arena);
auto & col = assert_cast<ColumnArray &>(to);
auto & col_offsets = assert_cast<ColumnArray::ColumnOffsets &>(col.getOffsetsColumn());
auto column_x_adder_func = getColumnAdderFunc(x_type);
auto column_y_adder_func = getColumnAdderFunc(y_type);
for (size_t i = 0; i < res.size(); ++i)
{
auto & column_tuple = assert_cast<ColumnTuple &>(col.getData());
column_x_adder_func(column_tuple.getColumn(0), res[i].first);
column_y_adder_func(column_tuple.getColumn(1), res[i].second);
}
col_offsets.getData().push_back(col.getData().size());
}
std::function<void(IColumn &, Float64)> getColumnAdderFunc(TypeIndex type_index) const
{
switch (type_index)
{
case TypeIndex::Date:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDate &>(column);
col.getData().push_back(static_cast<UInt16>(value));
};
case TypeIndex::Date32:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDate32 &>(column);
col.getData().push_back(static_cast<UInt32>(value));
};
case TypeIndex::DateTime:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDateTime &>(column);
col.getData().push_back(static_cast<UInt32>(value));
};
case TypeIndex::DateTime64:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDateTime64 &>(column);
col.getData().push_back(static_cast<UInt64>(value));
};
default:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnFloat64 &>(column);
col.getData().push_back(value);
};
}
}
};
AggregateFunctionPtr
createAggregateFunctionLargestTriangleThreeBuckets(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertBinary(name, argument_types);
if (!(isNumber(argument_types[0]) || isDateOrDate32(argument_types[0]) || isDateTime(argument_types[0])
|| isDateTime64(argument_types[0])))
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the first argument",
name);
if (!(isNumber(argument_types[1]) || isDateOrDate32(argument_types[1]) || isDateTime(argument_types[1])
|| isDateTime64(argument_types[1])))
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Aggregate function {} only supports Date, Date32, DateTime, DateTime64 and Number as the second argument",
name);
return std::make_shared<AggregateFunctionLargestTriangleThreeBuckets>(argument_types, parameters);
}
}

View File

@ -33,295 +33,6 @@ namespace ErrorCodes
}
struct LargestTriangleThreeBucketsData : public StatisticalSample<Float64, Float64>
{
void add(const Float64 xval, const Float64 yval, Arena * arena)
{
this->addX(xval, arena);
this->addY(yval, arena);
}
void sort(Arena * arena)
{
// sort the this->x and this->y in ascending order of this->x using index
std::vector<size_t> index(this->x.size());
std::iota(index.begin(), index.end(), 0);
::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; });
SampleX temp_x{};
SampleY temp_y{};
for (size_t i = 0; i < this->x.size(); ++i)
{
temp_x.push_back(this->x[index[i]], arena);
temp_y.push_back(this->y[index[i]], arena);
}
for (size_t i = 0; i < this->x.size(); ++i)
{
this->x[i] = temp_x[i];
this->y[i] = temp_y[i];
}
}
PODArray<std::pair<Float64, Float64>> getResult(size_t total_buckets, Arena * arena)
{
// Sort the data
this->sort(arena);
PODArray<std::pair<Float64, Float64>> result;
// Handle special cases for small data list
if (this->x.size() <= total_buckets)
{
for (size_t i = 0; i < this->x.size(); ++i)
{
result.emplace_back(std::make_pair(this->x[i], this->y[i]));
}
return result;
}
// Handle special cases for 0 or 1 or 2 buckets
if (total_buckets == 0)
return result;
if (total_buckets == 1)
{
result.emplace_back(std::make_pair(this->x.front(), this->y.front()));
return result;
}
if (total_buckets == 2)
{
result.emplace_back(std::make_pair(this->x.front(), this->y.front()));
result.emplace_back(std::make_pair(this->x.back(), this->y.back()));
return result;
}
// Find the size of each bucket
size_t single_bucket_size = this->x.size() / total_buckets;
// Include the first data point
result.emplace_back(std::make_pair(this->x[0], this->y[0]));
for (size_t i = 1; i < total_buckets - 1; ++i) // Skip the first and last bucket
{
size_t start_index = i * single_bucket_size;
size_t end_index = (i + 1) * single_bucket_size;
// Compute the average point in the next bucket
Float64 avg_x = 0;
Float64 avg_y = 0;
for (size_t j = end_index; j < (i + 2) * single_bucket_size; ++j)
{
avg_x += this->x[j];
avg_y += this->y[j];
}
avg_x /= single_bucket_size;
avg_y /= single_bucket_size;
// Find the point in the current bucket that forms the largest triangle
size_t max_index = start_index;
Float64 max_area = 0.0;
for (size_t j = start_index; j < end_index; ++j)
{
Float64 area = std::abs(
0.5
* (result.back().first * this->y[j] + this->x[j] * avg_y + avg_x * result.back().second - result.back().first * avg_y
- this->x[j] * result.back().second - avg_x * this->y[j]));
if (area > max_area)
{
max_area = area;
max_index = j;
}
}
// Include the selected point
result.emplace_back(std::make_pair(this->x[max_index], this->y[max_index]));
}
// Include the last data point
result.emplace_back(std::make_pair(this->x.back(), this->y.back()));
return result;
}
};
class AggregateFunctionLargestTriangleThreeBuckets final : public IAggregateFunctionDataHelper<LargestTriangleThreeBucketsData, AggregateFunctionLargestTriangleThreeBuckets>
{
private:
UInt64 total_buckets{0};
TypeIndex x_type;
TypeIndex y_type;
public:
explicit AggregateFunctionLargestTriangleThreeBuckets(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<LargestTriangleThreeBucketsData, AggregateFunctionLargestTriangleThreeBuckets>({arguments}, {}, createResultType(arguments))
{
if (params.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require one parameter", getName());
if (params[0].getType() != Field::Types::UInt64)
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a UInt64", getName());
total_buckets = params[0].get<UInt64>();
this->x_type = WhichDataType(arguments[0]).idx;
this->y_type = WhichDataType(arguments[1]).idx;
}
static constexpr auto name = "largestTriangleThreeBuckets";
String getName() const override { return name; }
bool allocatesMemoryInArena() const override { return true; }
static DataTypePtr createResultType(const DataTypes & arguments)
{
TypeIndex x_type = arguments[0]->getTypeId();
TypeIndex y_type = arguments[1]->getTypeId();
UInt32 x_scale = 0;
UInt32 y_scale = 0;
if (const auto * datetime64_type = typeid_cast<const DataTypeDateTime64 *>(arguments[0].get()))
{
x_scale = datetime64_type->getScale();
}
if (const auto * datetime64_type = typeid_cast<const DataTypeDateTime64 *>(arguments[1].get()))
{
y_scale = datetime64_type->getScale();
}
DataTypes types = {getDataTypeFromTypeIndex(x_type, x_scale), getDataTypeFromTypeIndex(y_type, y_scale)};
auto tuple = std::make_shared<DataTypeTuple>(std::move(types));
return std::make_shared<DataTypeArray>(tuple);
}
static DataTypePtr getDataTypeFromTypeIndex(TypeIndex type_index, UInt32 scale)
{
DataTypePtr data_type;
switch (type_index)
{
case TypeIndex::Date:
data_type = std::make_shared<DataTypeDate>();
break;
case TypeIndex::Date32:
data_type = std::make_shared<DataTypeDate32>();
break;
case TypeIndex::DateTime:
data_type = std::make_shared<DataTypeDateTime>();
break;
case TypeIndex::DateTime64:
data_type = std::make_shared<DataTypeDateTime64>(scale);
break;
default:
data_type = std::make_shared<DataTypeNumber<Float64>>();
}
return data_type;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 x = getFloat64DataFromColumn(columns[0], row_num, this->x_type);
Float64 y = getFloat64DataFromColumn(columns[1], row_num, this->y_type);
this->data(place).add(x, y, arena);
}
Float64 getFloat64DataFromColumn(const IColumn * column, size_t row_num, TypeIndex type_index) const
{
switch (type_index)
{
case TypeIndex::Date:
return static_cast<const ColumnDate &>(*column).getData()[row_num];
case TypeIndex::Date32:
return static_cast<const ColumnDate32 &>(*column).getData()[row_num];
case TypeIndex::DateTime:
return static_cast<const ColumnDateTime &>(*column).getData()[row_num];
case TypeIndex::DateTime64:
return static_cast<const ColumnDateTime64 &>(*column).getData()[row_num];
default:
return column->getFloat64(row_num);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
const auto & b = this->data(rhs);
a.merge(b, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override
{
auto res = this->data(place).getResult(total_buckets, arena);
auto & col = assert_cast<ColumnArray &>(to);
auto & col_offsets = assert_cast<ColumnArray::ColumnOffsets &>(col.getOffsetsColumn());
auto column_x_adder_func = getColumnAdderFunc(x_type);
auto column_y_adder_func = getColumnAdderFunc(y_type);
for (size_t i = 0; i < res.size(); ++i)
{
auto & column_tuple = assert_cast<ColumnTuple &>(col.getData());
column_x_adder_func(column_tuple.getColumn(0), res[i].first);
column_y_adder_func(column_tuple.getColumn(1), res[i].second);
}
col_offsets.getData().push_back(col.getData().size());
}
std::function<void(IColumn &, Float64)> getColumnAdderFunc(TypeIndex type_index) const
{
switch (type_index)
{
case TypeIndex::Date:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDate &>(column);
col.getData().push_back(static_cast<UInt16>(value));
};
case TypeIndex::Date32:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDate32 &>(column);
col.getData().push_back(static_cast<UInt32>(value));
};
case TypeIndex::DateTime:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDateTime &>(column);
col.getData().push_back(static_cast<UInt32>(value));
};
case TypeIndex::DateTime64:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnDateTime64 &>(column);
col.getData().push_back(static_cast<UInt64>(value));
};
default:
return [](IColumn & column, Float64 value)
{
auto & col = assert_cast<ColumnFloat64 &>(column);
col.getData().push_back(value);
};
}
}
};
}

View File

@ -1,21 +1,254 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionMannWhitney.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <Common/PODArray_fwd.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/ReadHelpers.h>
#include <limits>
#include <boost/math/distributions/normal.hpp>
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace DB
{
struct Settings;
namespace
{
struct MannWhitneyData : public StatisticalSample<Float64, Float64>
{
/*Since null hypothesis is "for randomly selected values X and Y from two populations,
*the probability of X being greater than Y is equal to the probability of Y being greater than X".
*Or "the distribution F of first sample equals to the distribution G of second sample".
*Then alternative for this hypothesis (H1) is "two-sided"(F != G), "less"(F < G), "greater" (F > G). */
enum class Alternative
{
TwoSided,
Less,
Greater
};
/// The behaviour equals to the similar function from scipy.
/// https://github.com/scipy/scipy/blob/ab9e9f17e0b7b2d618c4d4d8402cd4c0c200d6c0/scipy/stats/stats.py#L6978
std::pair<Float64, Float64> getResult(Alternative alternative, bool continuity_correction)
{
ConcatenatedSamples both(this->x, this->y);
RanksArray ranks;
Float64 tie_correction;
/// Compute ranks according to both samples.
std::tie(ranks, tie_correction) = computeRanksAndTieCorrection(both);
const Float64 n1 = this->size_x;
const Float64 n2 = this->size_y;
Float64 r1 = 0;
for (size_t i = 0; i < n1; ++i)
r1 += ranks[i];
const Float64 u1 = n1 * n2 + (n1 * (n1 + 1.)) / 2. - r1;
const Float64 u2 = n1 * n2 - u1;
/// The distribution of U-statistic under null hypothesis H0 is symmetric with respect to meanrank.
const Float64 meanrank = n1 * n2 /2. + 0.5 * continuity_correction;
const Float64 sd = std::sqrt(tie_correction * n1 * n2 * (n1 + n2 + 1) / 12.0);
Float64 u = 0;
if (alternative == Alternative::TwoSided)
/// There is no difference which u_i to take as u, because z will be differ only in sign and we take std::abs() from it.
u = std::max(u1, u2);
else if (alternative == Alternative::Less)
u = u1;
else if (alternative == Alternative::Greater)
u = u2;
Float64 z = (u - meanrank) / sd;
if (unlikely(!std::isfinite(z)))
return {std::numeric_limits<Float64>::quiet_NaN(), std::numeric_limits<Float64>::quiet_NaN()};
if (alternative == Alternative::TwoSided)
z = std::abs(z);
auto standard_normal_distribution = boost::math::normal_distribution<Float64>();
auto cdf = boost::math::cdf(standard_normal_distribution, z);
Float64 p_value = 0;
if (alternative == Alternative::TwoSided)
p_value = 2 - 2 * cdf;
else
p_value = 1 - cdf;
return {u2, p_value};
}
private:
using Sample = typename StatisticalSample<Float64, Float64>::SampleX;
/// We need to compute ranks according to all samples. Use this class to avoid extra copy and memory allocation.
class ConcatenatedSamples
{
public:
ConcatenatedSamples(const Sample & first_, const Sample & second_)
: first(first_), second(second_) {}
const Float64 & operator[](size_t ind) const
{
if (ind < first.size())
return first[ind];
return second[ind % first.size()];
}
size_t size() const
{
return first.size() + second.size();
}
private:
const Sample & first;
const Sample & second;
};
};
class AggregateFunctionMannWhitney final:
public IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney>
{
private:
using Alternative = typename MannWhitneyData::Alternative;
Alternative alternative;
bool continuity_correction{true};
public:
explicit AggregateFunctionMannWhitney(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney> ({arguments}, {}, createResultType())
{
if (params.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName());
if (params.empty())
{
alternative = Alternative::TwoSided;
return;
}
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
alternative = Alternative::Less;
else if (param == "greater")
alternative = Alternative::Greater;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown parameter in aggregate function {}. "
"It must be one of: 'two-sided', 'less', 'greater'", getName());
if (params.size() != 2)
return;
if (params[1].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a UInt64", getName());
continuity_correction = static_cast<bool>(params[1].get<UInt64>());
}
String getName() const override
{
return "mannWhitneyUTest";
}
bool allocatesMemoryInArena() const override { return true; }
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
Strings names
{
"u_statistic",
"p_value"
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value, arena);
else
this->data(place).addX(value, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
const auto & b = this->data(rhs);
a.merge(b, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if (!this->data(place).size_x || !this->data(place).size_y)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());
auto [u_statistic, p_value] = this->data(place).getResult(alternative, continuity_correction);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
column_stat.getData().push_back(u_statistic);
column_value.getData().push_back(p_value);
}
};
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{

View File

@ -1,249 +0,0 @@
#pragma once
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <Common/PODArray_fwd.h>
#include <base/types.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <limits>
#include <boost/math/distributions/normal.hpp>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
struct MannWhitneyData : public StatisticalSample<Float64, Float64>
{
/*Since null hypothesis is "for randomly selected values X and Y from two populations,
*the probability of X being greater than Y is equal to the probability of Y being greater than X".
*Or "the distribution F of first sample equals to the distribution G of second sample".
*Then alternative for this hypothesis (H1) is "two-sided"(F != G), "less"(F < G), "greater" (F > G). */
enum class Alternative
{
TwoSided,
Less,
Greater
};
/// The behaviour equals to the similar function from scipy.
/// https://github.com/scipy/scipy/blob/ab9e9f17e0b7b2d618c4d4d8402cd4c0c200d6c0/scipy/stats/stats.py#L6978
std::pair<Float64, Float64> getResult(Alternative alternative, bool continuity_correction)
{
ConcatenatedSamples both(this->x, this->y);
RanksArray ranks;
Float64 tie_correction;
/// Compute ranks according to both samples.
std::tie(ranks, tie_correction) = computeRanksAndTieCorrection(both);
const Float64 n1 = this->size_x;
const Float64 n2 = this->size_y;
Float64 r1 = 0;
for (size_t i = 0; i < n1; ++i)
r1 += ranks[i];
const Float64 u1 = n1 * n2 + (n1 * (n1 + 1.)) / 2. - r1;
const Float64 u2 = n1 * n2 - u1;
/// The distribution of U-statistic under null hypothesis H0 is symmetric with respect to meanrank.
const Float64 meanrank = n1 * n2 /2. + 0.5 * continuity_correction;
const Float64 sd = std::sqrt(tie_correction * n1 * n2 * (n1 + n2 + 1) / 12.0);
Float64 u = 0;
if (alternative == Alternative::TwoSided)
/// There is no difference which u_i to take as u, because z will be differ only in sign and we take std::abs() from it.
u = std::max(u1, u2);
else if (alternative == Alternative::Less)
u = u1;
else if (alternative == Alternative::Greater)
u = u2;
Float64 z = (u - meanrank) / sd;
if (unlikely(!std::isfinite(z)))
return {std::numeric_limits<Float64>::quiet_NaN(), std::numeric_limits<Float64>::quiet_NaN()};
if (alternative == Alternative::TwoSided)
z = std::abs(z);
auto standard_normal_distribution = boost::math::normal_distribution<Float64>();
auto cdf = boost::math::cdf(standard_normal_distribution, z);
Float64 p_value = 0;
if (alternative == Alternative::TwoSided)
p_value = 2 - 2 * cdf;
else
p_value = 1 - cdf;
return {u2, p_value};
}
private:
using Sample = typename StatisticalSample<Float64, Float64>::SampleX;
/// We need to compute ranks according to all samples. Use this class to avoid extra copy and memory allocation.
class ConcatenatedSamples
{
public:
ConcatenatedSamples(const Sample & first_, const Sample & second_)
: first(first_), second(second_) {}
const Float64 & operator[](size_t ind) const
{
if (ind < first.size())
return first[ind];
return second[ind % first.size()];
}
size_t size() const
{
return first.size() + second.size();
}
private:
const Sample & first;
const Sample & second;
};
};
class AggregateFunctionMannWhitney final:
public IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney>
{
private:
using Alternative = typename MannWhitneyData::Alternative;
Alternative alternative;
bool continuity_correction{true};
public:
explicit AggregateFunctionMannWhitney(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<MannWhitneyData, AggregateFunctionMannWhitney> ({arguments}, {}, createResultType())
{
if (params.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName());
if (params.empty())
{
alternative = Alternative::TwoSided;
return;
}
if (params[0].getType() != Field::Types::String)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require first parameter to be a String", getName());
const auto & param = params[0].get<String>();
if (param == "two-sided")
alternative = Alternative::TwoSided;
else if (param == "less")
alternative = Alternative::Less;
else if (param == "greater")
alternative = Alternative::Greater;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown parameter in aggregate function {}. "
"It must be one of: 'two-sided', 'less', 'greater'", getName());
if (params.size() != 2)
return;
if (params[1].getType() != Field::Types::UInt64)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function {} require second parameter to be a UInt64", getName());
continuity_correction = static_cast<bool>(params[1].get<UInt64>());
}
String getName() const override
{
return "mannWhitneyUTest";
}
bool allocatesMemoryInArena() const override { return true; }
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
Strings names
{
"u_statistic",
"p_value"
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value, arena);
else
this->data(place).addX(value, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
const auto & b = this->data(rhs);
a.merge(b, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if (!this->data(place).size_x || !this->data(place).size_y)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());
auto [u_statistic, p_value] = this->data(place).getResult(alternative, continuity_correction);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
column_stat.getData().push_back(u_statistic);
column_value.getData().push_back(p_value);
}
};
}

View File

@ -1,8 +1,21 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionMaxIntersections.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h>
#include <Common/NaNUtils.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#define AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE 0xFFFFFF
namespace DB
{
@ -11,24 +24,187 @@ struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int TOO_LARGE_ARRAY_SIZE;
}
namespace
{
AggregateFunctionPtr createAggregateFunctionMaxIntersections(
AggregateFunctionIntersectionsKind kind,
const std::string & name, const DataTypes & argument_types, const Array & parameters)
/** maxIntersections: returns maximum count of the intersected intervals defined by start_column and end_column values,
* maxIntersectionsPosition: returns leftmost position of maximum intersection of intervals.
*/
/// Similar to GroupArrayNumericData.
template <typename T>
struct MaxIntersectionsData
{
/// Left or right end of the interval and signed weight; with positive sign for begin of interval and negative sign for end of interval.
using Value = std::pair<T, Int64>;
// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena
using Allocator = MixedAlignedArenaAllocator<alignof(Value), 4096>;
using Array = PODArray<Value, 32, Allocator>;
Array value;
};
enum class AggregateFunctionIntersectionsKind
{
Count,
Position
};
template <typename PointType>
class AggregateFunctionIntersectionsMax final
: public IAggregateFunctionDataHelper<MaxIntersectionsData<PointType>, AggregateFunctionIntersectionsMax<PointType>>
{
private:
AggregateFunctionIntersectionsKind kind;
public:
AggregateFunctionIntersectionsMax(AggregateFunctionIntersectionsKind kind_, const DataTypes & arguments)
: IAggregateFunctionDataHelper<MaxIntersectionsData<PointType>, AggregateFunctionIntersectionsMax<PointType>>(arguments, {}, createResultType(kind_))
, kind(kind_)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);
if (!isNativeNumber(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: first argument must be represented by integer", getName());
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionIntersectionsMax>(*argument_types[0], kind, argument_types));
if (!res)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types {} and {} of argument for aggregate function {}",
argument_types[0]->getName(), argument_types[1]->getName(), name);
if (!isNativeNumber(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: second argument must be represented by integer", getName());
return res;
if (!arguments[0]->equals(*arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: arguments must have the same type", getName());
}
String getName() const override
{
return kind == AggregateFunctionIntersectionsKind::Count
? "maxIntersections"
: "maxIntersectionsPosition";
}
static DataTypePtr createResultType(AggregateFunctionIntersectionsKind kind_)
{
if (kind_ == AggregateFunctionIntersectionsKind::Count)
return std::make_shared<DataTypeUInt64>();
else
return std::make_shared<DataTypeNumber<PointType>>();
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
PointType left = assert_cast<const ColumnVector<PointType> &>(*columns[0]).getData()[row_num];
PointType right = assert_cast<const ColumnVector<PointType> &>(*columns[1]).getData()[row_num];
if (!isNaN(left))
this->data(place).value.push_back(std::make_pair(left, Int64(1)), arena);
if (!isNaN(right))
this->data(place).value.push_back(std::make_pair(right, Int64(-1)), arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & cur_elems = this->data(place);
auto & rhs_elems = this->data(rhs);
cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
const auto & value = this->data(place).value;
size_t size = value.size();
writeVarUInt(size, buf);
/// In this version, pairs were serialized with padding.
/// We must ensure that padding bytes are zero-filled.
static_assert(offsetof(typename MaxIntersectionsData<PointType>::Value, first) == 0);
static_assert(offsetof(typename MaxIntersectionsData<PointType>::Value, second) > 0);
char zero_padding[offsetof(typename MaxIntersectionsData<PointType>::Value, second) - sizeof(value[0].first)]{};
for (size_t i = 0; i < size; ++i)
{
writePODBinary(value[i].first, buf);
writePODBinary(zero_padding, buf);
if constexpr (std::endian::native == std::endian::little)
writePODBinary(value[i].second, buf);
else
writePODBinary(std::byteswap(value[i].second), buf);
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
size_t size = 0;
readVarUInt(size, buf);
if (unlikely(size > AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Too large array size (maximum: {})", AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE);
auto & value = this->data(place).value;
value.resize(size, arena);
buf.readStrict(reinterpret_cast<char *>(value.data()), size * sizeof(value[0]));
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
Int64 current_intersections = 0;
Int64 max_intersections = 0;
PointType position_of_max_intersections = 0;
/// const_cast because we will sort the array
auto & array = this->data(place).value;
/// Sort by position; for equal position, sort by weight to get deterministic result.
::sort(array.begin(), array.end());
for (const auto & point_weight : array)
{
current_intersections += point_weight.second;
if (current_intersections > max_intersections)
{
max_intersections = current_intersections;
position_of_max_intersections = point_weight.first;
}
}
if (kind == AggregateFunctionIntersectionsKind::Count)
{
auto & result_column = assert_cast<ColumnUInt64 &>(to).getData();
result_column.push_back(max_intersections);
}
else
{
auto & result_column = assert_cast<ColumnVector<PointType> &>(to).getData();
result_column.push_back(position_of_max_intersections);
}
}
};
AggregateFunctionPtr createAggregateFunctionMaxIntersections(
AggregateFunctionIntersectionsKind kind,
const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionIntersectionsMax>(*argument_types[0], kind, argument_types));
if (!res)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal types {} and {} of argument for aggregate function {}",
argument_types[0]->getName(), argument_types[1]->getName(), name);
return res;
}
}
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory & factory)

View File

@ -1,189 +0,0 @@
#pragma once
#include <base/sort.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h>
#include <Common/NaNUtils.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#define AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE 0xFFFFFF
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int TOO_LARGE_ARRAY_SIZE;
}
/** maxIntersections: returns maximum count of the intersected intervals defined by start_column and end_column values,
* maxIntersectionsPosition: returns leftmost position of maximum intersection of intervals.
*/
/// Similar to GroupArrayNumericData.
template <typename T>
struct MaxIntersectionsData
{
/// Left or right end of the interval and signed weight; with positive sign for begin of interval and negative sign for end of interval.
using Value = std::pair<T, Int64>;
// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena
using Allocator = MixedAlignedArenaAllocator<alignof(Value), 4096>;
using Array = PODArray<Value, 32, Allocator>;
Array value;
};
enum class AggregateFunctionIntersectionsKind
{
Count,
Position
};
template <typename PointType>
class AggregateFunctionIntersectionsMax final
: public IAggregateFunctionDataHelper<MaxIntersectionsData<PointType>, AggregateFunctionIntersectionsMax<PointType>>
{
private:
AggregateFunctionIntersectionsKind kind;
public:
AggregateFunctionIntersectionsMax(AggregateFunctionIntersectionsKind kind_, const DataTypes & arguments)
: IAggregateFunctionDataHelper<MaxIntersectionsData<PointType>, AggregateFunctionIntersectionsMax<PointType>>(arguments, {}, createResultType(kind_))
, kind(kind_)
{
if (!isNativeNumber(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: first argument must be represented by integer", getName());
if (!isNativeNumber(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: second argument must be represented by integer", getName());
if (!arguments[0]->equals(*arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: arguments must have the same type", getName());
}
String getName() const override
{
return kind == AggregateFunctionIntersectionsKind::Count
? "maxIntersections"
: "maxIntersectionsPosition";
}
static DataTypePtr createResultType(AggregateFunctionIntersectionsKind kind_)
{
if (kind_ == AggregateFunctionIntersectionsKind::Count)
return std::make_shared<DataTypeUInt64>();
else
return std::make_shared<DataTypeNumber<PointType>>();
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
PointType left = assert_cast<const ColumnVector<PointType> &>(*columns[0]).getData()[row_num];
PointType right = assert_cast<const ColumnVector<PointType> &>(*columns[1]).getData()[row_num];
if (!isNaN(left))
this->data(place).value.push_back(std::make_pair(left, Int64(1)), arena);
if (!isNaN(right))
this->data(place).value.push_back(std::make_pair(right, Int64(-1)), arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & cur_elems = this->data(place);
auto & rhs_elems = this->data(rhs);
cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
const auto & value = this->data(place).value;
size_t size = value.size();
writeVarUInt(size, buf);
/// In this version, pairs were serialized with padding.
/// We must ensure that padding bytes are zero-filled.
static_assert(offsetof(typename MaxIntersectionsData<PointType>::Value, first) == 0);
static_assert(offsetof(typename MaxIntersectionsData<PointType>::Value, second) > 0);
char zero_padding[offsetof(typename MaxIntersectionsData<PointType>::Value, second) - sizeof(value[0].first)]{};
for (size_t i = 0; i < size; ++i)
{
writePODBinary(value[i].first, buf);
writePODBinary(zero_padding, buf);
if constexpr (std::endian::native == std::endian::little)
writePODBinary(value[i].second, buf);
else
writePODBinary(std::byteswap(value[i].second), buf);
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
size_t size = 0;
readVarUInt(size, buf);
if (unlikely(size > AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Too large array size (maximum: {})", AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE);
auto & value = this->data(place).value;
value.resize(size, arena);
buf.readStrict(reinterpret_cast<char *>(value.data()), size * sizeof(value[0]));
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
Int64 current_intersections = 0;
Int64 max_intersections = 0;
PointType position_of_max_intersections = 0;
/// const_cast because we will sort the array
auto & array = this->data(place).value;
/// Sort by position; for equal position, sort by weight to get deterministic result.
::sort(array.begin(), array.end());
for (const auto & point_weight : array)
{
current_intersections += point_weight.second;
if (current_intersections > max_intersections)
{
max_intersections = current_intersections;
position_of_max_intersections = point_weight.first;
}
}
if (kind == AggregateFunctionIntersectionsKind::Count)
{
auto & result_column = assert_cast<ColumnUInt64 &>(to).getData();
result_column.push_back(max_intersections);
}
else
{
auto & result_column = assert_cast<ColumnVector<PointType> &>(to).getData();
result_column.push_back(position_of_max_intersections);
}
}
};
}

View File

@ -1,8 +1,16 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionMeanZTest.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Moments.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <cmath>
namespace ErrorCodes
{
@ -18,6 +26,121 @@ struct Settings;
namespace
{
/// Returns tuple of (z-statistic, p-value, confidence-interval-low, confidence-interval-high)
template <typename Data>
class AggregateFunctionMeanZTest :
public IAggregateFunctionDataHelper<Data, AggregateFunctionMeanZTest<Data>>
{
private:
Float64 pop_var_x;
Float64 pop_var_y;
Float64 confidence_level;
public:
AggregateFunctionMeanZTest(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<Data, AggregateFunctionMeanZTest<Data>>({arguments}, params, createResultType())
{
pop_var_x = params.at(0).safeGet<Float64>();
pop_var_y = params.at(1).safeGet<Float64>();
confidence_level = params.at(2).safeGet<Float64>();
if (!std::isfinite(pop_var_x) || !std::isfinite(pop_var_y) || !std::isfinite(confidence_level))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} requires finite parameter values.", Data::name);
}
if (pop_var_x < 0.0 || pop_var_y < 0.0)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Population variance parameters must be larger than or equal to zero "
"in aggregate function {}.", Data::name);
}
if (confidence_level <= 0.0 || confidence_level >= 1.0)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Confidence level parameter must be between 0 and 1 in aggregate function {}.", Data::name);
}
}
String getName() const override
{
return Data::name;
}
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
Strings names
{
"z_statistic",
"p_value",
"confidence_interval_low",
"confidence_interval_high"
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value);
else
this->data(place).addX(value);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto [z_stat, p_value] = this->data(place).getResult(pop_var_x, pop_var_y);
auto [ci_low, ci_high] = this->data(place).getConfidenceIntervals(pop_var_x, pop_var_y, confidence_level);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
auto & column_ci_low = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(2));
auto & column_ci_high = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(3));
column_stat.getData().push_back(z_stat);
column_value.getData().push_back(p_value);
column_ci_low.getData().push_back(ci_low);
column_ci_high.getData().push_back(ci_high);
}
};
struct MeanZTestData : public ZTestMoments<Float64>
{
static constexpr auto name = "meanZTest";

View File

@ -1,141 +0,0 @@
#pragma once
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <Core/Types.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <cmath>
namespace DB
{
struct Settings;
class ReadBuffer;
class WriteBuffer;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/// Returns tuple of (z-statistic, p-value, confidence-interval-low, confidence-interval-high)
template <typename Data>
class AggregateFunctionMeanZTest :
public IAggregateFunctionDataHelper<Data, AggregateFunctionMeanZTest<Data>>
{
private:
Float64 pop_var_x;
Float64 pop_var_y;
Float64 confidence_level;
public:
AggregateFunctionMeanZTest(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<Data, AggregateFunctionMeanZTest<Data>>({arguments}, params, createResultType())
{
pop_var_x = params.at(0).safeGet<Float64>();
pop_var_y = params.at(1).safeGet<Float64>();
confidence_level = params.at(2).safeGet<Float64>();
if (!std::isfinite(pop_var_x) || !std::isfinite(pop_var_y) || !std::isfinite(confidence_level))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} requires finite parameter values.", Data::name);
}
if (pop_var_x < 0.0 || pop_var_y < 0.0)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Population variance parameters must be larger than or equal to zero "
"in aggregate function {}.", Data::name);
}
if (confidence_level <= 0.0 || confidence_level >= 1.0)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Confidence level parameter must be between 0 and 1 in aggregate function {}.", Data::name);
}
}
String getName() const override
{
return Data::name;
}
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
std::make_shared<DataTypeNumber<Float64>>(),
};
Strings names
{
"z_statistic",
"p_value",
"confidence_interval_low",
"confidence_interval_high"
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value);
else
this->data(place).addX(value);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto [z_stat, p_value] = this->data(place).getResult(pop_var_x, pop_var_y);
auto [ci_low, ci_high] = this->data(place).getConfidenceIntervals(pop_var_x, pop_var_y, confidence_level);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));
auto & column_tuple = assert_cast<ColumnTuple &>(to);
auto & column_stat = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(0));
auto & column_value = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(1));
auto & column_ci_low = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(2));
auto & column_ci_high = assert_cast<ColumnVector<Float64> &>(column_tuple.getColumn(3));
column_stat.getData().push_back(z_stat);
column_value.getData().push_back(p_value);
column_ci_low.getData().push_back(ci_low);
column_ci_high.getData().push_back(ci_high);
}
};
}

View File

@ -1,5 +1,5 @@
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <AggregateFunctions/QuantileReservoirSampler.h>
#include <AggregateFunctions/ReservoirSampler.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
@ -9,16 +9,106 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
namespace
{
/** Quantile calculation with "reservoir sample" algorithm.
* It collects pseudorandom subset of limited size from a stream of values,
* and approximate quantile from it.
* The result is non-deterministic. Also look at QuantileReservoirSamplerDeterministic.
*
* This algorithm is quite inefficient in terms of precision for memory usage,
* but very efficient in CPU (though less efficient than QuantileTiming and than QuantileExact for small sets).
*/
template <typename Value>
struct QuantileReservoirSampler
{
using Data = ReservoirSampler<Value, ReservoirSamplerOnEmpty::RETURN_NAN_OR_ZERO>;
Data data;
void add(const Value & x)
{
data.insert(x);
}
template <typename Weight>
void add(const Value &, const Weight &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method add with weight is not implemented for ReservoirSampler");
}
void merge(const QuantileReservoirSampler & rhs)
{
data.merge(rhs.data);
}
void serialize(WriteBuffer & buf) const
{
data.write(buf);
}
void deserialize(ReadBuffer & buf)
{
data.read(buf);
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level)
{
if (data.empty())
return {};
if constexpr (is_decimal<Value>)
return Value(static_cast<typename Value::NativeType>(data.quantileInterpolated(level)));
else
return static_cast<Value>(data.quantileInterpolated(level));
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result)
{
bool is_empty = data.empty();
for (size_t i = 0; i < size; ++i)
{
if (is_empty)
{
result[i] = Value{};
}
else
{
if constexpr (is_decimal<Value>)
result[indices[i]] = Value(static_cast<typename Value::NativeType>(data.quantileInterpolated(levels[indices[i]])));
else
result[indices[i]] = Value(data.quantileInterpolated(levels[indices[i]]));
}
}
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64 level)
{
return data.quantileInterpolated(level);
}
void getManyFloat(const Float64 * levels, const size_t * indices, size_t size, Float64 * result)
{
for (size_t i = 0; i < size; ++i)
result[indices[i]] = data.quantileInterpolated(levels[indices[i]]);
}
};
template <typename Value, bool float_return> using FuncQuantile = AggregateFunctionQuantile<Value, QuantileReservoirSampler<Value>, NameQuantile, false, std::conditional_t<float_return, Float64, void>, false>;
template <typename Value, bool float_return> using FuncQuantiles = AggregateFunctionQuantile<Value, QuantileReservoirSampler<Value>, NameQuantiles, false, std::conditional_t<float_return, Float64, void>, true>;

View File

@ -1,10 +1,15 @@
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <AggregateFunctions/QuantileApprox.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Core/Field.h>
#include <cmath>
#include <Common/RadixSort.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
namespace DB
{
@ -12,11 +17,471 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
namespace
{
template <typename T>
class ApproxSampler
{
public:
struct Stats
{
T value; // the sampled value
Int64 g; // the minimum rank jump from the previous value's minimum rank
Int64 delta; // the maximum span of the rank
Stats() = default;
Stats(T value_, Int64 g_, Int64 delta_) : value(value_), g(g_), delta(delta_) {}
};
struct QueryResult
{
size_t index;
Int64 rank;
T value;
QueryResult(size_t index_, Int64 rank_, T value_) : index(index_), rank(rank_), value(value_) { }
};
ApproxSampler() = default;
explicit ApproxSampler(
double relative_error_,
size_t compress_threshold_ = default_compress_threshold,
size_t count_ = 0,
bool compressed_ = false)
: relative_error(relative_error_)
, compress_threshold(compress_threshold_)
, count(count_)
, compressed(compressed_)
{
sampled.reserve(compress_threshold);
backup_sampled.reserve(compress_threshold);
head_sampled.reserve(default_head_size);
}
bool isCompressed() const { return compressed; }
void setCompressed() { compressed = true; }
void insert(T x)
{
head_sampled.push_back(x);
compressed = false;
if (head_sampled.size() >= default_head_size)
{
withHeadBufferInserted();
if (sampled.size() >= compress_threshold)
compress();
}
}
void query(const Float64 * percentiles, const size_t * indices, size_t size, T * result) const
{
if (!head_sampled.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot operate on an uncompressed summary, call compress() first");
if (sampled.empty())
{
for (size_t i = 0; i < size; ++i)
result[i] = T();
return;
}
Int64 current_max = std::numeric_limits<Int64>::min();
for (const auto & stats : sampled)
current_max = std::max(stats.delta + stats.g, current_max);
Int64 target_error = current_max/2;
size_t index= 0;
auto min_rank = sampled[0].g;
for (size_t i = 0; i < size; ++i)
{
double percentile = percentiles[indices[i]];
if (percentile <= relative_error)
{
result[indices[i]] = sampled.front().value;
}
else if (percentile >= 1 - relative_error)
{
result[indices[i]] = sampled.back().value;
}
else
{
QueryResult res = findApproxQuantile(index, min_rank, target_error, percentile);
index = res.index;
min_rank = res.rank;
result[indices[i]] = res.value;
}
}
}
void compress()
{
if (compressed)
return;
withHeadBufferInserted();
doCompress(2 * relative_error * count);
compressed = true;
}
void merge(const ApproxSampler & other)
{
if (other.count == 0)
return;
else if (count == 0)
{
compress_threshold = other.compress_threshold;
relative_error = other.relative_error;
count = other.count;
compressed = other.compressed;
sampled.resize(other.sampled.size());
memcpy(sampled.data(), other.sampled.data(), sizeof(Stats) * other.sampled.size());
return;
}
else
{
// Merge the two buffers.
// The GK algorithm is a bit unclear about it, but we need to adjust the statistics during the
// merging. The main idea is that samples that come from one side will suffer from the lack of
// precision of the other.
// As a concrete example, take two QuantileSummaries whose samples (value, g, delta) are:
// `a = [(0, 1, 0), (20, 99, 0)]` and `b = [(10, 1, 0), (30, 49, 0)]`
// This means `a` has 100 values, whose minimum is 0 and maximum is 20,
// while `b` has 50 values, between 10 and 30.
// The resulting samples of the merge will be:
// a+b = [(0, 1, 0), (10, 1, ??), (20, 99, ??), (30, 49, 0)]
// The values of `g` do not change, as they represent the minimum number of values between two
// consecutive samples. The values of `delta` should be adjusted, however.
// Take the case of the sample `10` from `b`. In the original stream, it could have appeared
// right after `0` (as expressed by `g=1`) or right before `20`, so `delta=99+0-1=98`.
// In the GK algorithm's style of working in terms of maximum bounds, one can observe that the
// maximum additional uncertainty over samples coming from `b` is `max(g_a + delta_a) =
// floor(2 * eps_a * n_a)`. Likewise, additional uncertainty over samples from `a` is
// `floor(2 * eps_b * n_b)`.
// Only samples that interleave the other side are affected. That means that samples from
// one side that are lesser (or greater) than all samples from the other side are just copied
// unmodified.
// If the merging instances have different `relativeError`, the resulting instance will carry
// the largest one: `eps_ab = max(eps_a, eps_b)`.
// The main invariant of the GK algorithm is kept:
// `max(g_ab + delta_ab) <= floor(2 * eps_ab * (n_a + n_b))` since
// `max(g_ab + delta_ab) <= floor(2 * eps_a * n_a) + floor(2 * eps_b * n_b)`
// Finally, one can see how the `insert(x)` operation can be expressed as `merge([(x, 1, 0])`
compress();
backup_sampled.clear();
backup_sampled.reserve(sampled.size() + other.sampled.size());
double merged_relative_error = std::max(relative_error, other.relative_error);
size_t merged_count = count + other.count;
Int64 additional_self_delta = static_cast<Int64>(std::floor(2 * other.relative_error * other.count));
Int64 additional_other_delta = static_cast<Int64>(std::floor(2 * relative_error * count));
// Do a merge of two sorted lists until one of the lists is fully consumed
size_t self_idx = 0;
size_t other_idx = 0;
while (self_idx < sampled.size() && other_idx < other.sampled.size())
{
const Stats & self_sample = sampled[self_idx];
const Stats & other_sample = other.sampled[other_idx];
// Detect next sample
Stats next_sample;
Int64 additional_delta = 0;
if (self_sample.value < other_sample.value)
{
++self_idx;
next_sample = self_sample;
additional_delta = other_idx > 0 ? additional_self_delta : 0;
}
else
{
++other_idx;
next_sample = other_sample;
additional_delta = self_idx > 0 ? additional_other_delta : 0;
}
// Insert it
next_sample.delta += additional_delta;
backup_sampled.emplace_back(std::move(next_sample));
}
// Copy the remaining samples from the other list
// (by construction, at most one `while` loop will run)
while (self_idx < sampled.size())
{
backup_sampled.emplace_back(sampled[self_idx]);
++self_idx;
}
while (other_idx < other.sampled.size())
{
backup_sampled.emplace_back(other.sampled[other_idx]);
++other_idx;
}
std::swap(sampled, backup_sampled);
relative_error = merged_relative_error;
count = merged_count;
compress_threshold = other.compress_threshold;
doCompress(2 * merged_relative_error * merged_count);
compressed = true;
}
}
void write(WriteBuffer & buf) const
{
writeBinaryLittleEndian(compress_threshold, buf);
writeBinaryLittleEndian(relative_error, buf);
writeBinaryLittleEndian(count, buf);
writeBinaryLittleEndian(sampled.size(), buf);
for (const auto & stats : sampled)
{
writeBinaryLittleEndian(stats.value, buf);
writeBinaryLittleEndian(stats.g, buf);
writeBinaryLittleEndian(stats.delta, buf);
}
}
void read(ReadBuffer & buf)
{
readBinaryLittleEndian(compress_threshold, buf);
readBinaryLittleEndian(relative_error, buf);
readBinaryLittleEndian(count, buf);
size_t sampled_len = 0;
readBinaryLittleEndian(sampled_len, buf);
sampled.resize(sampled_len);
for (size_t i = 0; i < sampled_len; ++i)
{
auto stats = sampled[i];
readBinaryLittleEndian(stats.value, buf);
readBinaryLittleEndian(stats.g, buf);
readBinaryLittleEndian(stats.delta, buf);
}
}
private:
QueryResult findApproxQuantile(size_t index, Int64 min_rank_at_index, double target_error, double percentile) const
{
Stats curr_sample = sampled[index];
Int64 rank = static_cast<Int64>(std::ceil(percentile * count));
size_t i = index;
Int64 min_rank = min_rank_at_index;
while (i < sampled.size() - 1)
{
Int64 max_rank = min_rank + curr_sample.delta;
if (max_rank - target_error <= rank && rank <= min_rank + target_error)
return {i, min_rank, curr_sample.value};
else
{
++i;
curr_sample = sampled[i];
min_rank += curr_sample.g;
}
}
return {sampled.size()-1, 0, sampled.back().value};
}
void withHeadBufferInserted()
{
if (head_sampled.empty())
return;
bool use_radix_sort = head_sampled.size() >= 256 && (is_arithmetic_v<T> && !is_big_int_v<T>);
if (use_radix_sort)
RadixSort<RadixSortNumTraits<T>>::executeLSD(head_sampled.data(), head_sampled.size());
else
::sort(head_sampled.begin(), head_sampled.end());
backup_sampled.clear();
backup_sampled.reserve(sampled.size() + head_sampled.size());
size_t sample_idx = 0;
size_t ops_idx = 0;
size_t current_count = count;
for (; ops_idx < head_sampled.size(); ++ops_idx)
{
T current_sample = head_sampled[ops_idx];
// Add all the samples before the next observation.
while (sample_idx < sampled.size() && sampled[sample_idx].value <= current_sample)
{
backup_sampled.emplace_back(sampled[sample_idx]);
++sample_idx;
}
// If it is the first one to insert, of if it is the last one
++current_count;
Int64 delta;
if (backup_sampled.empty() || (sample_idx == sampled.size() && ops_idx == (head_sampled.size() - 1)))
delta = 0;
else
delta = static_cast<Int64>(std::floor(2 * relative_error * current_count));
backup_sampled.emplace_back(current_sample, 1, delta);
}
// Add all the remaining existing samples
for (; sample_idx < sampled.size(); ++sample_idx)
backup_sampled.emplace_back(sampled[sample_idx]);
std::swap(sampled, backup_sampled);
head_sampled.clear();
count = current_count;
}
void doCompress(double merge_threshold)
{
if (sampled.empty())
return;
backup_sampled.clear();
// Start for the last element, which is always part of the set.
// The head contains the current new head, that may be merged with the current element.
Stats head = sampled.back();
ssize_t i = sampled.size() - 2;
// Do not compress the last element
while (i >= 1)
{
// The current sample:
const auto & sample1 = sampled[i];
// Do we need to compress?
if (sample1.g + head.g + head.delta < merge_threshold)
{
// Do not insert yet, just merge the current element into the head.
head.g += sample1.g;
}
else
{
// Prepend the current head, and keep the current sample as target for merging.
backup_sampled.push_back(head);
head = sample1;
}
--i;
}
backup_sampled.push_back(head);
// If necessary, add the minimum element:
auto curr_head = sampled.front();
// don't add the minimum element if `currentSamples` has only one element (both `currHead` and
// `head` point to the same element)
if (curr_head.value <= head.value && sampled.size() > 1)
backup_sampled.emplace_back(sampled.front());
std::reverse(backup_sampled.begin(), backup_sampled.end());
std::swap(sampled, backup_sampled);
}
double relative_error;
size_t compress_threshold;
size_t count = 0;
bool compressed;
PaddedPODArray<Stats> sampled;
PaddedPODArray<Stats> backup_sampled;
PaddedPODArray<T> head_sampled;
static constexpr size_t default_compress_threshold = 10000;
static constexpr size_t default_head_size = 50000;
};
template <typename Value>
class QuantileGK
{
private:
using Data = ApproxSampler<Value>;
mutable Data data;
public:
QuantileGK() = default;
explicit QuantileGK(size_t accuracy) : data(1.0 / static_cast<double>(accuracy)) { }
void add(const Value & x)
{
data.insert(x);
}
template <typename Weight>
void add(const Value &, const Weight &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method add with weight is not implemented for GKSampler");
}
void merge(const QuantileGK & rhs)
{
if (!data.isCompressed())
data.compress();
data.merge(rhs.data);
}
void serialize(WriteBuffer & buf) const
{
/// Always compress before serialization
if (!data.isCompressed())
data.compress();
data.write(buf);
}
void deserialize(ReadBuffer & buf)
{
data.read(buf);
data.setCompressed();
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level)
{
if (!data.isCompressed())
data.compress();
Value res;
size_t indice = 0;
data.query(&level, &indice, 1, &res);
return res;
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result)
{
if (!data.isCompressed())
data.compress();
data.query(levels, indices, size, result);
}
Float64 getFloat64(Float64 /*level*/)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat64 is not implemented for GKSampler");
}
void getManyFloat(const Float64 * /*levels*/, const size_t * /*indices*/, size_t /*size*/, Float64 * /*result*/)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getManyFloat is not implemented for GKSampler");
}
};
template <typename Value, bool _> using FuncQuantileGK = AggregateFunctionQuantile<Value, QuantileGK<Value>, NameQuantileGK, false, void, false>;
template <typename Value, bool _> using FuncQuantilesGK = AggregateFunctionQuantile<Value, QuantileGK<Value>, NameQuantilesGK, false, void, true>;

View File

@ -1,5 +1,5 @@
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <AggregateFunctions/QuantileReservoirSamplerDeterministic.h>
#include <AggregateFunctions/ReservoirSamplerDeterministic.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
@ -9,16 +9,106 @@
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
namespace
{
/** Quantile calculation with "reservoir sample" algorithm.
* It collects pseudorandom subset of limited size from a stream of values,
* and approximate quantile from it.
* The function accept second argument, named "determinator"
* and a hash function from it is calculated and used as a source for randomness
* to apply random sampling.
* The function is deterministic, but care should be taken with choose of "determinator" argument.
*/
template <typename Value>
struct QuantileReservoirSamplerDeterministic
{
using Data = ReservoirSamplerDeterministic<Value, ReservoirSamplerDeterministicOnEmpty::RETURN_NAN_OR_ZERO>;
Data data;
void add(const Value &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method add without determinator is not implemented for ReservoirSamplerDeterministic");
}
template <typename Determinator>
void add(const Value & x, const Determinator & determinator)
{
data.insert(x, determinator);
}
void merge(const QuantileReservoirSamplerDeterministic & rhs)
{
data.merge(rhs.data);
}
void serialize(WriteBuffer & buf) const
{
data.write(buf);
}
void deserialize(ReadBuffer & buf)
{
data.read(buf);
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level)
{
if (data.empty())
return {};
if constexpr (is_decimal<Value>)
return static_cast<typename Value::NativeType>(data.quantileInterpolated(level));
else
return static_cast<Value>(data.quantileInterpolated(level));
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result)
{
bool is_empty = data.empty();
for (size_t i = 0; i < size; ++i)
{
if (is_empty)
{
result[i] = Value{};
}
else
{
if constexpr (is_decimal<Value>)
result[indices[i]] = static_cast<typename Value::NativeType>(data.quantileInterpolated(levels[indices[i]]));
else
result[indices[i]] = static_cast<Value>(data.quantileInterpolated(levels[indices[i]]));
}
}
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64 level)
{
return data.quantileInterpolated(level);
}
void getManyFloat(const Float64 * levels, const size_t * indices, size_t size, Float64 * result)
{
for (size_t i = 0; i < size; ++i)
result[indices[i]] = data.quantileInterpolated(levels[indices[i]]);
}
};
template <typename Value, bool float_return> using FuncQuantileDeterministic = AggregateFunctionQuantile<Value, QuantileReservoirSamplerDeterministic<Value>, NameQuantileDeterministic, true, std::conditional_t<float_return, Float64, void>, false>;
template <typename Value, bool float_return> using FuncQuantilesDeterministic = AggregateFunctionQuantile<Value, QuantileReservoirSamplerDeterministic<Value>, NameQuantilesDeterministic, true, std::conditional_t<float_return, Float64, void>, true>;

View File

@ -1,24 +1,214 @@
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <AggregateFunctions/QuantileExactWeighted.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Core/Field.h>
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
namespace
{
/** Calculates quantile by counting number of occurrences for each value in a hash map.
*
* It uses O(distinct(N)) memory. Can be naturally applied for values with weight.
* In case of many identical values, it can be more efficient than QuantileExact even when weight is not used.
*/
template <typename Value>
struct QuantileExactWeighted
{
struct Int128Hash
{
size_t operator()(Int128 x) const
{
return CityHash_v1_0_2::Hash128to64({x >> 64, x & 0xffffffffffffffffll});
}
};
using Weight = UInt64;
using UnderlyingType = NativeType<Value>;
using Hasher = HashCRC32<UnderlyingType>;
/// When creating, the hash table must be small.
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;
Map map;
void add(const Value & x)
{
/// We must skip NaNs as they are not compatible with comparison sorting.
if (!isNaN(x))
++map[x];
}
void add(const Value & x, Weight weight)
{
if (!isNaN(x))
map[x] += weight;
}
void merge(const QuantileExactWeighted & rhs)
{
for (const auto & pair : rhs.map)
map[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
map.write(buf);
}
void deserialize(ReadBuffer & buf)
{
typename Map::Reader reader(buf);
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
}
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level) const
{
size_t size = map.size();
if (0 == size)
return std::numeric_limits<Value>::quiet_NaN();
/// Copy the data to a temporary array to get the element you need in order.
using Pair = typename Map::value_type;
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
/// Note: 64-bit integer weight can overflow.
/// We do some implementation specific behaviour (return approximate or garbage results).
/// Float64 is used as accumulator here to get approximate results.
/// But weight can be already overflowed in computations in 'add' and 'merge' methods.
/// It will be reasonable to change the type of weight to Float64 in the map,
/// but we don't do that for compatibility of serialized data.
size_t i = 0;
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
array[i] = pair.getValue();
++i;
}
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 threshold = std::ceil(sum_weight * level);
Float64 accumulated = 0;
const Pair * it = array;
const Pair * end = array + size;
while (it < end)
{
accumulated += it->second;
if (accumulated >= threshold)
break;
++it;
}
if (it == end)
--it;
return it->first;
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const
{
size_t size = map.size();
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
result[i] = Value();
return;
}
/// Copy the data to a temporary array to get the element you need in order.
using Pair = typename Map::value_type;
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
size_t i = 0;
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
array[i] = pair.getValue();
++i;
}
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
const Pair * it = array;
const Pair * end = array + size;
size_t level_index = 0;
Float64 threshold = std::ceil(sum_weight * levels[indices[level_index]]);
while (it < end)
{
accumulated += it->second;
while (accumulated >= threshold)
{
result[indices[level_index]] = it->first;
++level_index;
if (level_index == num_levels)
return;
threshold = std::ceil(sum_weight * levels[indices[level_index]]);
}
++it;
}
while (level_index < num_levels)
{
result[indices[level_index]] = array[size - 1].first;
++level_index;
}
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat is not implemented for QuantileExact");
}
void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getManyFloat is not implemented for QuantileExact");
}
};
template <typename Value, bool _> using FuncQuantileExactWeighted = AggregateFunctionQuantile<Value, QuantileExactWeighted<Value>, NameQuantileExactWeighted, true, void, false>;
template <typename Value, bool _> using FuncQuantilesExactWeighted = AggregateFunctionQuantile<Value, QuantileExactWeighted<Value>, NameQuantilesExactWeighted, true, void, true>;

View File

@ -1,58 +1,353 @@
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <AggregateFunctions/QuantileInterpolatedWeighted.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Core/Field.h>
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
namespace
{
template <typename Value, bool _> using FuncQuantileInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantileInterpolatedWeighted, true, void, false>;
template <typename Value, bool _> using FuncQuantilesInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantilesInterpolatedWeighted, true, void, true>;
template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
/** Approximates Quantile by:
* - sorting input values and weights
* - building a cumulative distribution based on weights
* - performing linear interpolation between the weights and values
*
*/
template <typename Value>
struct QuantileInterpolatedWeighted
{
struct Int128Hash
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
size_t operator()(Int128 x) const
{
return CityHash_v1_0_2::Hash128to64({x >> 64, x & 0xffffffffffffffffll});
}
};
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);
using Weight = UInt64;
using UnderlyingType = NativeType<Value>;
using Hasher = HashCRC32<UnderlyingType>;
/// When creating, the hash table must be small.
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;
Map map;
void add(const Value & x)
{
/// We must skip NaNs as they are not compatible with comparison sorting.
if (!isNaN(x))
++map[x];
}
void add(const Value & x, Weight weight)
{
if (!isNaN(x))
map[x] += weight;
}
void merge(const QuantileInterpolatedWeighted & rhs)
{
for (const auto & pair : rhs.map)
map[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
map.write(buf);
}
void deserialize(ReadBuffer & buf)
{
typename Map::Reader reader(buf);
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
}
}
Value get(Float64 level) const
{
return getImpl<Value>(level);
}
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result) const
{
getManyImpl<Value>(levels, indices, size, result);
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat is not implemented for QuantileInterpolatedWeighted");
}
void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getManyFloat is not implemented for QuantileInterpolatedWeighted");
}
private:
using Pair = typename std::pair<UnderlyingType, Float64>;
/// Get the value of the `level` quantile. The level must be between 0 and 1.
template <typename T>
T getImpl(Float64 level) const
{
size_t size = map.size();
if (0 == size)
return std::numeric_limits<Value>::quiet_NaN();
/// Maintain a vector of pair of values and weights for easier sorting and for building
/// a cumulative distribution using the provided weights.
std::vector<Pair> value_weight_pairs;
value_weight_pairs.reserve(size);
/// Note: weight provided must be a 64-bit integer
/// Float64 is used as accumulator here to get approximate results.
/// But weight used in the internal array is stored as Float64 as we
/// do some quantile estimation operation which involves division and
/// require Float64 level of precision.
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
auto value = pair.getKey();
auto weight = pair.getMapped();
value_weight_pairs.push_back({value, weight});
}
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
/// vector for populating and storing the cumulative sum using the provided weights.
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
std::vector<Float64> weights_cum_sum;
weights_cum_sum.reserve(size);
for (size_t idx = 0; idx < size; ++idx)
{
accumulated += value_weight_pairs[idx].second;
weights_cum_sum.push_back(accumulated);
}
/// The following estimation of quantile is general and the idea is:
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
/// calculates a simple cumulative distribution based on weights
if (sum_weight != 0)
{
for (size_t idx = 0; idx < size; ++idx)
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
}
/// perform linear interpolation
size_t idx = 0;
if (size >= 2)
{
if (level >= value_weight_pairs[size - 2].second)
{
idx = size - 2;
}
else
{
size_t start = 0, end = size - 1;
while (start <= end)
{
size_t mid = start + (end - start) / 2;
if (mid > size)
break;
if (level > value_weight_pairs[mid + 1].second)
start = mid + 1;
else
{
idx = mid;
end = mid - 1;
}
}
}
}
size_t l = idx;
size_t u = idx + 1 < size ? idx + 1 : idx;
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
if (level < xl)
yr = yl;
if (level > xr)
yl = yr;
return static_cast<T>(interpolate(level, xl, xr, yl, yr));
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
template <typename T>
void getManyImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const
{
size_t size = map.size();
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
result[i] = Value();
return;
}
std::vector<Pair> value_weight_pairs;
value_weight_pairs.reserve(size);
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
auto value = pair.getKey();
auto weight = pair.getMapped();
value_weight_pairs.push_back({value, weight});
}
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
/// vector for populating and storing the cumulative sum using the provided weights.
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
std::vector<Float64> weights_cum_sum;
weights_cum_sum.reserve(size);
for (size_t idx = 0; idx < size; ++idx)
{
accumulated += value_weight_pairs[idx].second;
weights_cum_sum.emplace_back(accumulated);
}
/// The following estimation of quantile is general and the idea is:
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
/// calculates a simple cumulative distribution based on weights
if (sum_weight != 0)
{
for (size_t idx = 0; idx < size; ++idx)
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
}
for (size_t level_index = 0; level_index < num_levels; ++level_index)
{
/// perform linear interpolation for every level
auto level = levels[indices[level_index]];
size_t idx = 0;
if (size >= 2)
{
if (level >= value_weight_pairs[size - 2].second)
{
idx = size - 2;
}
else
{
size_t start = 0, end = size - 1;
while (start <= end)
{
size_t mid = start + (end - start) / 2;
if (mid > size)
break;
if (level > value_weight_pairs[mid + 1].second)
start = mid + 1;
else
{
idx = mid;
end = mid - 1;
}
}
}
}
size_t l = idx;
size_t u = idx + 1 < size ? idx + 1 : idx;
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
if (level < xl)
yr = yl;
if (level > xr)
yl = yr;
result[indices[level_index]] = static_cast<T>(interpolate(level, xl, xr, yl, yr));
}
}
/// This ignores overflows or NaN's that might arise during add, sub and mul operations and doesn't aim to provide exact
/// results since `the quantileInterpolatedWeighted` function itself relies mainly on approximation.
UnderlyingType NO_SANITIZE_UNDEFINED interpolate(Float64 level, Float64 xl, Float64 xr, UnderlyingType yl, UnderlyingType yr) const
{
UnderlyingType dy = yr - yl;
Float64 dx = xr - xl;
dx = dx == 0 ? 1 : dx; /// to handle NaN behavior that might arise during integer division below.
/// yl + (dy / dx) * (level - xl)
return static_cast<UnderlyingType>(yl + (dy / dx) * (level - xl));
}
};
template <typename Value, bool _> using FuncQuantileInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantileInterpolatedWeighted, true, void, false>;
template <typename Value, bool _> using FuncQuantilesInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantilesInterpolatedWeighted, true, void, true>;
template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) return std::make_shared<Function<TYPE, true>>(argument_types, params);
FOR_BASIC_NUMERIC_TYPES(DISPATCH)
if (which.idx == TypeIndex::TYPE) return std::make_shared<Function<TYPE, true>>(argument_types, params);
FOR_BASIC_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Date) return std::make_shared<Function<DataTypeDate::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime) return std::make_shared<Function<DataTypeDateTime::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::Date) return std::make_shared<Function<DataTypeDate::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime) return std::make_shared<Function<DataTypeDateTime::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
if (which.idx == TypeIndex::Int128) return std::make_shared<Function<Int128, true>>(argument_types, params);
if (which.idx == TypeIndex::UInt128) return std::make_shared<Function<UInt128, true>>(argument_types, params);
if (which.idx == TypeIndex::Int256) return std::make_shared<Function<Int256, true>>(argument_types, params);
if (which.idx == TypeIndex::UInt256) return std::make_shared<Function<UInt256, true>>(argument_types, params);
if (which.idx == TypeIndex::Int128) return std::make_shared<Function<Int128, true>>(argument_types, params);
if (which.idx == TypeIndex::UInt128) return std::make_shared<Function<UInt128, true>>(argument_types, params);
if (which.idx == TypeIndex::Int256) return std::make_shared<Function<Int256, true>>(argument_types, params);
if (which.idx == TypeIndex::UInt256) return std::make_shared<Function<UInt256, true>>(argument_types, params);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_type->getName(), name);
}
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_type->getName(), name);
}
}
void registerAggregateFunctionsQuantileInterpolatedWeighted(AggregateFunctionFactory & factory)

View File

@ -1,7 +1,13 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionRankCorrelation.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <Common/PODArray_fwd.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
namespace ErrorCodes
@ -16,6 +22,83 @@ struct Settings;
namespace
{
struct RankCorrelationData : public StatisticalSample<Float64, Float64>
{
Float64 getResult()
{
RanksArray ranks_x;
std::tie(ranks_x, std::ignore) = computeRanksAndTieCorrection(this->x);
RanksArray ranks_y;
std::tie(ranks_y, std::ignore) = computeRanksAndTieCorrection(this->y);
/// Sizes can be non-equal due to skipped NaNs.
const Float64 size = static_cast<Float64>(std::min(this->size_x, this->size_y));
/// Count d^2 sum
Float64 answer = 0;
for (size_t j = 0; j < size; ++j)
answer += (ranks_x[j] - ranks_y[j]) * (ranks_x[j] - ranks_y[j]);
answer *= 6;
answer /= size * (size * size - 1);
answer = 1 - answer;
return answer;
}
};
class AggregateFunctionRankCorrelation :
public IAggregateFunctionDataHelper<RankCorrelationData, AggregateFunctionRankCorrelation>
{
public:
explicit AggregateFunctionRankCorrelation(const DataTypes & arguments)
: IAggregateFunctionDataHelper<RankCorrelationData, AggregateFunctionRankCorrelation> ({arguments}, {}, std::make_shared<DataTypeNumber<Float64>>())
{}
String getName() const override
{
return "rankCorr";
}
bool allocatesMemoryInArena() const override { return true; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 new_x = columns[0]->getFloat64(row_num);
Float64 new_y = columns[1]->getFloat64(row_num);
this->data(place).addX(new_x, arena);
this->data(place).addY(new_y, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
const auto & b = this->data(rhs);
a.merge(b, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto answer = this->data(place).getResult();
auto & column = static_cast<ColumnVector<Float64> &>(to);
column.getData().push_back(answer);
}
};
AggregateFunctionPtr createAggregateFunctionRankCorrelation(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{

View File

@ -1,98 +0,0 @@
#pragma once
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/StatCommon.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <Common/PODArray_fwd.h>
#include <base/types.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
namespace DB
{
struct Settings;
struct RankCorrelationData : public StatisticalSample<Float64, Float64>
{
Float64 getResult()
{
RanksArray ranks_x;
std::tie(ranks_x, std::ignore) = computeRanksAndTieCorrection(this->x);
RanksArray ranks_y;
std::tie(ranks_y, std::ignore) = computeRanksAndTieCorrection(this->y);
/// Sizes can be non-equal due to skipped NaNs.
const Float64 size = static_cast<Float64>(std::min(this->size_x, this->size_y));
/// Count d^2 sum
Float64 answer = 0;
for (size_t j = 0; j < size; ++j)
answer += (ranks_x[j] - ranks_y[j]) * (ranks_x[j] - ranks_y[j]);
answer *= 6;
answer /= size * (size * size - 1);
answer = 1 - answer;
return answer;
}
};
class AggregateFunctionRankCorrelation :
public IAggregateFunctionDataHelper<RankCorrelationData, AggregateFunctionRankCorrelation>
{
public:
explicit AggregateFunctionRankCorrelation(const DataTypes & arguments)
:IAggregateFunctionDataHelper<RankCorrelationData, AggregateFunctionRankCorrelation> ({arguments}, {}, std::make_shared<DataTypeNumber<Float64>>())
{}
String getName() const override
{
return "rankCorr";
}
bool allocatesMemoryInArena() const override { return true; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Float64 new_x = columns[0]->getFloat64(row_num);
Float64 new_y = columns[1]->getFloat64(row_num);
this->data(place).addX(new_x, arena);
this->data(place).addY(new_y, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & a = this->data(place);
const auto & b = this->data(rhs);
a.merge(b, arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto answer = this->data(place).getResult();
auto & column = static_cast<ColumnVector<Float64> &>(to);
column.getData().push_back(answer);
}
};
}

View File

@ -1,21 +1,150 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionRetention.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <unordered_set>
#include <Columns/ColumnArray.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <bitset>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
struct AggregateFunctionRetentionData
{
static constexpr auto max_events = 32;
using Events = std::bitset<max_events>;
Events events;
void add(UInt8 event)
{
events.set(event);
}
void merge(const AggregateFunctionRetentionData & other)
{
events |= other.events;
}
void serialize(WriteBuffer & buf) const
{
UInt32 event_value = static_cast<UInt32>(events.to_ulong());
writeBinary(event_value, buf);
}
void deserialize(ReadBuffer & buf)
{
UInt32 event_value;
readBinary(event_value, buf);
events = event_value;
}
};
/**
* The max size of events is 32, that's enough for retention analytics
*
* Usage:
* - retention(cond1, cond2, cond3, ....)
* - returns [cond1_flag, cond1_flag && cond2_flag, cond1_flag && cond3_flag, ...]
*/
class AggregateFunctionRetention final
: public IAggregateFunctionDataHelper<AggregateFunctionRetentionData, AggregateFunctionRetention>
{
private:
UInt8 events_size;
public:
String getName() const override
{
return "retention";
}
explicit AggregateFunctionRetention(const DataTypes & arguments)
: IAggregateFunctionDataHelper<AggregateFunctionRetentionData, AggregateFunctionRetention>(arguments, {}, std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt8>()))
{
for (const auto i : collections::range(0, arguments.size()))
{
const auto * cond_arg = arguments[i].get();
if (!isUInt8(cond_arg))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of aggregate function {}, must be UInt8",
cond_arg->getName(), i, getName());
}
events_size = static_cast<UInt8>(arguments.size());
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
for (const auto i : collections::range(0, events_size))
{
auto event = assert_cast<const ColumnVector<UInt8> *>(columns[i])->getData()[row_num];
if (event)
{
this->data(place).add(i);
}
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & data_to = assert_cast<ColumnUInt8 &>(assert_cast<ColumnArray &>(to).getData()).getData();
auto & offsets_to = assert_cast<ColumnArray &>(to).getOffsets();
ColumnArray::Offset current_offset = data_to.size();
data_to.resize(current_offset + events_size);
const bool first_flag = this->data(place).events.test(0);
data_to[current_offset] = first_flag;
++current_offset;
for (size_t i = 1; i < events_size; ++i)
{
data_to[current_offset] = (first_flag && this->data(place).events.test(i));
++current_offset;
}
offsets_to.push_back(current_offset);
}
};
AggregateFunctionPtr createAggregateFunctionRetention(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
{
assertNoParameters(name, params);

View File

@ -1,143 +0,0 @@
#pragma once
#include <unordered_set>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnArray.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <base/range.h>
#include <bitset>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
struct AggregateFunctionRetentionData
{
static constexpr auto max_events = 32;
using Events = std::bitset<max_events>;
Events events;
void add(UInt8 event)
{
events.set(event);
}
void merge(const AggregateFunctionRetentionData & other)
{
events |= other.events;
}
void serialize(WriteBuffer & buf) const
{
UInt32 event_value = static_cast<UInt32>(events.to_ulong());
writeBinary(event_value, buf);
}
void deserialize(ReadBuffer & buf)
{
UInt32 event_value;
readBinary(event_value, buf);
events = event_value;
}
};
/**
* The max size of events is 32, that's enough for retention analytics
*
* Usage:
* - retention(cond1, cond2, cond3, ....)
* - returns [cond1_flag, cond1_flag && cond2_flag, cond1_flag && cond3_flag, ...]
*/
class AggregateFunctionRetention final
: public IAggregateFunctionDataHelper<AggregateFunctionRetentionData, AggregateFunctionRetention>
{
private:
UInt8 events_size;
public:
String getName() const override
{
return "retention";
}
explicit AggregateFunctionRetention(const DataTypes & arguments)
: IAggregateFunctionDataHelper<AggregateFunctionRetentionData, AggregateFunctionRetention>(arguments, {}, std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt8>()))
{
for (const auto i : collections::range(0, arguments.size()))
{
const auto * cond_arg = arguments[i].get();
if (!isUInt8(cond_arg))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument {} of aggregate function {}, must be UInt8",
cond_arg->getName(), i, getName());
}
events_size = static_cast<UInt8>(arguments.size());
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
for (const auto i : collections::range(0, events_size))
{
auto event = assert_cast<const ColumnVector<UInt8> *>(columns[i])->getData()[row_num];
if (event)
{
this->data(place).add(i);
}
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & data_to = assert_cast<ColumnUInt8 &>(assert_cast<ColumnArray &>(to).getData()).getData();
auto & offsets_to = assert_cast<ColumnArray &>(to).getOffsets();
ColumnArray::Offset current_offset = data_to.size();
data_to.resize(current_offset + events_size);
const bool first_flag = this->data(place).events.test(0);
data_to[current_offset] = first_flag;
++current_offset;
for (size_t i = 1; i < events_size; ++i)
{
data_to[current_offset] = (first_flag && this->data(place).events.test(i));
++current_offset;
}
offsets_to.push_back(current_offset);
}
};
}

View File

@ -1,15 +1,22 @@
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSequenceMatch.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <base/range.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <bitset>
#include <stack>
namespace DB
{
struct Settings;
namespace ErrorCodes
@ -18,11 +25,689 @@ namespace ErrorCodes
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_SLOW;
extern const int SYNTAX_ERROR;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
namespace
{
/// helper type for comparing `std::pair`s using solely the .first member
template <template <typename> class Comparator>
struct ComparePairFirst final
{
template <typename T1, typename T2>
bool operator()(const std::pair<T1, T2> & lhs, const std::pair<T1, T2> & rhs) const
{
return Comparator<T1>{}(lhs.first, rhs.first);
}
};
static constexpr size_t max_events = 32;
template <typename T>
struct AggregateFunctionSequenceMatchData final
{
using Timestamp = T;
using Events = std::bitset<max_events>;
using TimestampEvents = std::pair<Timestamp, Events>;
using Comparator = ComparePairFirst<std::less>;
bool sorted = true;
PODArrayWithStackMemory<TimestampEvents, 64> events_list;
/// sequenceMatch conditions met at least once in events_list
Events conditions_met;
void add(const Timestamp timestamp, const Events & events)
{
/// store information exclusively for rows with at least one event
if (events.any())
{
events_list.emplace_back(timestamp, events);
sorted = false;
conditions_met |= events;
}
}
void merge(const AggregateFunctionSequenceMatchData & other)
{
if (other.events_list.empty())
return;
events_list.insert(std::begin(other.events_list), std::end(other.events_list));
sorted = false;
conditions_met |= other.conditions_met;
}
void sort()
{
if (sorted)
return;
::sort(std::begin(events_list), std::end(events_list), Comparator{});
sorted = true;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(sorted, buf);
writeBinary(events_list.size(), buf);
for (const auto & events : events_list)
{
writeBinary(events.first, buf);
writeBinary(events.second.to_ulong(), buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(sorted, buf);
size_t size;
readBinary(size, buf);
/// If we lose these flags, functionality is broken
/// If we serialize/deserialize these flags, we have compatibility issues
/// If we set these flags to 1, we have a minor performance penalty, which seems acceptable
conditions_met.set();
events_list.clear();
events_list.reserve(size);
for (size_t i = 0; i < size; ++i)
{
Timestamp timestamp;
readBinary(timestamp, buf);
UInt64 events;
readBinary(events, buf);
events_list.emplace_back(timestamp, Events{events});
}
}
};
/// Max number of iterations to match the pattern against a sequence, exception thrown when exceeded
constexpr auto sequence_match_max_iterations = 1000000;
template <typename T, typename Data, typename Derived>
class AggregateFunctionSequenceBase : public IAggregateFunctionDataHelper<Data, Derived>
{
public:
AggregateFunctionSequenceBase(const DataTypes & arguments, const Array & params, const String & pattern_, const DataTypePtr & result_type_)
: IAggregateFunctionDataHelper<Data, Derived>(arguments, params, result_type_)
, pattern(pattern_)
{
arg_count = arguments.size();
parsePattern();
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
typename Data::Events events;
for (const auto i : collections::range(1, arg_count))
{
const auto event = assert_cast<const ColumnUInt8 *>(columns[i])->getData()[row_num];
events.set(i - 1, event);
}
this->data(place).add(timestamp, events);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
bool haveSameStateRepresentationImpl(const IAggregateFunction & rhs) const override
{
return this->getName() == rhs.getName() && this->haveEqualArgumentTypes(rhs);
}
private:
enum class PatternActionType
{
SpecificEvent,
AnyEvent,
KleeneStar,
TimeLessOrEqual,
TimeLess,
TimeGreaterOrEqual,
TimeGreater,
TimeEqual
};
struct PatternAction final
{
PatternActionType type;
std::uint64_t extra;
PatternAction() = default;
explicit PatternAction(const PatternActionType type_, const std::uint64_t extra_ = 0) : type{type_}, extra{extra_} {}
};
using PatternActions = PODArrayWithStackMemory<PatternAction, 64>;
Derived & derived() { return static_cast<Derived &>(*this); }
void parsePattern()
{
actions.clear();
actions.emplace_back(PatternActionType::KleeneStar);
dfa_states.clear();
dfa_states.emplace_back(true);
pattern_has_time = false;
const char * pos = pattern.data();
const char * begin = pos;
const char * end = pos + pattern.size();
auto throw_exception = [&](const std::string & msg)
{
throw Exception(ErrorCodes::SYNTAX_ERROR, "{} '{}' at position {}", msg, std::string(pos, end), toString(pos - begin));
};
auto match = [&pos, end](const char * str) mutable
{
size_t length = strlen(str);
if (pos + length <= end && 0 == memcmp(pos, str, length))
{
pos += length;
return true;
}
return false;
};
while (pos < end)
{
if (match("(?"))
{
if (match("t"))
{
PatternActionType type;
if (match("<="))
type = PatternActionType::TimeLessOrEqual;
else if (match("<"))
type = PatternActionType::TimeLess;
else if (match(">="))
type = PatternActionType::TimeGreaterOrEqual;
else if (match(">"))
type = PatternActionType::TimeGreater;
else if (match("=="))
type = PatternActionType::TimeEqual;
else
throw_exception("Unknown time condition");
UInt64 duration = 0;
const auto * prev_pos = pos;
pos = tryReadIntText(duration, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number");
if (actions.back().type != PatternActionType::SpecificEvent &&
actions.back().type != PatternActionType::AnyEvent &&
actions.back().type != PatternActionType::KleeneStar)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Temporal condition should be preceded by an event condition");
pattern_has_time = true;
actions.emplace_back(type, duration);
}
else
{
UInt64 event_number = 0;
const auto * prev_pos = pos;
pos = tryReadIntText(event_number, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number");
if (event_number > arg_count - 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Event number {} is out of range", event_number);
actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1);
dfa_states.back().transition = DFATransition::SpecificEvent;
dfa_states.back().event = static_cast<uint32_t>(event_number - 1);
dfa_states.emplace_back();
conditions_in_pattern.set(event_number - 1);
}
if (!match(")"))
throw_exception("Expected closing parenthesis, found");
}
else if (match(".*"))
{
actions.emplace_back(PatternActionType::KleeneStar);
dfa_states.back().has_kleene = true;
}
else if (match("."))
{
actions.emplace_back(PatternActionType::AnyEvent);
dfa_states.back().transition = DFATransition::AnyEvent;
dfa_states.emplace_back();
}
else
throw_exception("Could not parse pattern, unexpected starting symbol");
}
}
protected:
/// Uses a DFA based approach in order to better handle patterns without
/// time assertions.
///
/// NOTE: This implementation relies on the assumption that the pattern is *small*.
///
/// This algorithm performs in O(mn) (with m the number of DFA states and N the number
/// of events) with a memory consumption and memory allocations in O(m). It means that
/// if n >>> m (which is expected to be the case), this algorithm can be considered linear.
template <typename EventEntry>
bool dfaMatch(EventEntry & events_it, const EventEntry events_end) const
{
using ActiveStates = std::vector<bool>;
/// Those two vectors keep track of which states should be considered for the current
/// event as well as the states which should be considered for the next event.
ActiveStates active_states(dfa_states.size(), false);
ActiveStates next_active_states(dfa_states.size(), false);
active_states[0] = true;
/// Keeps track of dead-ends in order not to iterate over all the events to realize that
/// the match failed.
size_t n_active = 1;
for (/* empty */; events_it != events_end && n_active > 0 && !active_states.back(); ++events_it)
{
n_active = 0;
next_active_states.assign(dfa_states.size(), false);
for (size_t state = 0; state < dfa_states.size(); ++state)
{
if (!active_states[state])
{
continue;
}
switch (dfa_states[state].transition)
{
case DFATransition::None:
break;
case DFATransition::AnyEvent:
next_active_states[state + 1] = true;
++n_active;
break;
case DFATransition::SpecificEvent:
if (events_it->second.test(dfa_states[state].event))
{
next_active_states[state + 1] = true;
++n_active;
}
break;
}
if (dfa_states[state].has_kleene)
{
next_active_states[state] = true;
++n_active;
}
}
swap(active_states, next_active_states);
}
return active_states.back();
}
template <typename EventEntry>
bool backtrackingMatch(EventEntry & events_it, const EventEntry events_end) const
{
const auto action_begin = std::begin(actions);
const auto action_end = std::end(actions);
auto action_it = action_begin;
const auto events_begin = events_it;
auto base_it = events_it;
/// an iterator to action plus an iterator to row in events list plus timestamp at the start of sequence
using backtrack_info = std::tuple<decltype(action_it), EventEntry, EventEntry>;
std::stack<backtrack_info> back_stack;
/// backtrack if possible
const auto do_backtrack = [&]
{
while (!back_stack.empty())
{
auto & top = back_stack.top();
action_it = std::get<0>(top);
events_it = std::next(std::get<1>(top));
base_it = std::get<2>(top);
back_stack.pop();
if (events_it != events_end)
return true;
}
return false;
};
size_t i = 0;
while (action_it != action_end && events_it != events_end)
{
if (action_it->type == PatternActionType::SpecificEvent)
{
if (events_it->second.test(action_it->extra))
{
/// move to the next action and events
base_it = events_it;
++action_it, ++events_it;
}
else if (!do_backtrack())
/// backtracking failed, bail out
break;
}
else if (action_it->type == PatternActionType::AnyEvent)
{
base_it = events_it;
++action_it, ++events_it;
}
else if (action_it->type == PatternActionType::KleeneStar)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (action_it->type == PatternActionType::TimeLessOrEqual)
{
if (events_it->first <= base_it->first + action_it->extra)
{
/// condition satisfied, move onto next action
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (!do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeLess)
{
if (events_it->first < base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (!do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeGreaterOrEqual)
{
if (events_it->first >= base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (++events_it == events_end && !do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeGreater)
{
if (events_it->first > base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (++events_it == events_end && !do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeEqual)
{
if (events_it->first == base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (++events_it == events_end && !do_backtrack())
break;
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown PatternActionType");
if (++i > sequence_match_max_iterations)
throw Exception(ErrorCodes::TOO_SLOW, "Pattern application proves too difficult, exceeding max iterations ({})",
sequence_match_max_iterations);
}
/// if there are some actions remaining
if (action_it != action_end)
{
/// match multiple empty strings at end
while (action_it->type == PatternActionType::KleeneStar ||
action_it->type == PatternActionType::TimeLessOrEqual ||
action_it->type == PatternActionType::TimeLess ||
(action_it->type == PatternActionType::TimeGreaterOrEqual && action_it->extra == 0))
++action_it;
}
if (events_it == events_begin)
++events_it;
return action_it == action_end;
}
/// Splits the pattern into deterministic parts separated by non-deterministic fragments
/// (time constraints and Kleene stars), and tries to match the deterministic parts in their specified order,
/// ignoring the non-deterministic fragments.
/// This function can quickly check that a full match is not possible if some deterministic fragment is missing.
template <typename EventEntry>
bool couldMatchDeterministicParts(const EventEntry events_begin, const EventEntry events_end, bool limit_iterations = true) const
{
size_t events_processed = 0;
auto events_it = events_begin;
const auto actions_end = std::end(actions);
auto actions_it = std::begin(actions);
auto det_part_begin = actions_it;
auto match_deterministic_part = [&events_it, events_end, &events_processed, det_part_begin, actions_it, limit_iterations]()
{
auto events_it_init = events_it;
auto det_part_it = det_part_begin;
while (det_part_it != actions_it && events_it != events_end)
{
/// matching any event
if (det_part_it->type == PatternActionType::AnyEvent)
++events_it, ++det_part_it;
/// matching specific event
else
{
if (events_it->second.test(det_part_it->extra))
++events_it, ++det_part_it;
/// abandon current matching, try to match the deterministic fragment further in the list
else
{
events_it = ++events_it_init;
det_part_it = det_part_begin;
}
}
if (limit_iterations && ++events_processed > sequence_match_max_iterations)
throw Exception(ErrorCodes::TOO_SLOW, "Pattern application proves too difficult, exceeding max iterations ({})",
sequence_match_max_iterations);
}
return det_part_it == actions_it;
};
for (; actions_it != actions_end; ++actions_it)
if (actions_it->type != PatternActionType::SpecificEvent && actions_it->type != PatternActionType::AnyEvent)
{
if (!match_deterministic_part())
return false;
det_part_begin = std::next(actions_it);
}
return match_deterministic_part();
}
private:
enum class DFATransition : char
{
/// .-------.
/// | |
/// `-------'
None,
/// .-------. (?[0-9])
/// | | ----------
/// `-------'
SpecificEvent,
/// .-------. .
/// | | ----------
/// `-------'
AnyEvent,
};
struct DFAState
{
explicit DFAState(bool has_kleene_ = false)
: has_kleene{has_kleene_}, event{0}, transition{DFATransition::None}
{}
/// .-------.
/// | | - - -
/// `-------'
/// |_^
bool has_kleene;
/// In the case of a state transitions with a `SpecificEvent`,
/// `event` contains the value of the event.
uint32_t event;
/// The kind of transition out of this state.
DFATransition transition;
};
using DFAStates = std::vector<DFAState>;
protected:
/// `True` if the parsed pattern contains time assertions (?t...), `false` otherwise.
bool pattern_has_time;
/// sequenceMatch conditions met at least once in the pattern
std::bitset<max_events> conditions_in_pattern;
private:
std::string pattern;
size_t arg_count;
PatternActions actions;
DFAStates dfa_states;
};
template <typename T, typename Data>
class AggregateFunctionSequenceMatch final : public AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceMatch<T, Data>>
{
public:
AggregateFunctionSequenceMatch(const DataTypes & arguments, const Array & params, const String & pattern_)
: AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceMatch<T, Data>>(arguments, params, pattern_, std::make_shared<DataTypeUInt8>()) {}
using AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceMatch<T, Data>>::AggregateFunctionSequenceBase;
String getName() const override { return "sequenceMatch"; }
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & output = assert_cast<ColumnUInt8 &>(to).getData();
if ((this->conditions_in_pattern & this->data(place).conditions_met) != this->conditions_in_pattern)
{
output.push_back(false);
return;
}
this->data(place).sort();
const auto & data_ref = this->data(place);
const auto events_begin = std::begin(data_ref.events_list);
const auto events_end = std::end(data_ref.events_list);
auto events_it = events_begin;
bool match = (this->pattern_has_time ?
(this->couldMatchDeterministicParts(events_begin, events_end) && this->backtrackingMatch(events_it, events_end)) :
this->dfaMatch(events_it, events_end));
output.push_back(match);
}
};
template <typename T, typename Data>
class AggregateFunctionSequenceCount final : public AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceCount<T, Data>>
{
public:
AggregateFunctionSequenceCount(const DataTypes & arguments, const Array & params, const String & pattern_)
: AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceCount<T, Data>>(arguments, params, pattern_, std::make_shared<DataTypeUInt64>()) {}
using AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceCount<T, Data>>::AggregateFunctionSequenceBase;
String getName() const override { return "sequenceCount"; }
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & output = assert_cast<ColumnUInt64 &>(to).getData();
if ((this->conditions_in_pattern & this->data(place).conditions_met) != this->conditions_in_pattern)
{
output.push_back(0);
return;
}
this->data(place).sort();
output.push_back(count(place));
}
private:
UInt64 count(ConstAggregateDataPtr __restrict place) const
{
const auto & data_ref = this->data(place);
const auto events_begin = std::begin(data_ref.events_list);
const auto events_end = std::end(data_ref.events_list);
auto events_it = events_begin;
size_t count = 0;
// check if there is a chance of matching the sequence at least once
if (this->couldMatchDeterministicParts(events_begin, events_end))
{
while (events_it != events_end && this->backtrackingMatch(events_it, events_end))
++count;
}
return count;
}
};
template <template <typename, typename> typename AggregateFunction, template <typename> typename Data>
AggregateFunctionPtr createAggregateFunctionSequenceBase(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)

View File

@ -1,702 +0,0 @@
#pragma once
#include <AggregateFunctions/IAggregateFunction.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <base/range.h>
#include <base/sort.h>
#include <Common/PODArray.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <bitset>
#include <stack>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int TOO_SLOW;
extern const int SYNTAX_ERROR;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
/// helper type for comparing `std::pair`s using solely the .first member
template <template <typename> class Comparator>
struct ComparePairFirst final
{
template <typename T1, typename T2>
bool operator()(const std::pair<T1, T2> & lhs, const std::pair<T1, T2> & rhs) const
{
return Comparator<T1>{}(lhs.first, rhs.first);
}
};
static constexpr size_t max_events = 32;
template <typename T>
struct AggregateFunctionSequenceMatchData final
{
using Timestamp = T;
using Events = std::bitset<max_events>;
using TimestampEvents = std::pair<Timestamp, Events>;
using Comparator = ComparePairFirst<std::less>;
bool sorted = true;
PODArrayWithStackMemory<TimestampEvents, 64> events_list;
/// sequenceMatch conditions met at least once in events_list
Events conditions_met;
void add(const Timestamp timestamp, const Events & events)
{
/// store information exclusively for rows with at least one event
if (events.any())
{
events_list.emplace_back(timestamp, events);
sorted = false;
conditions_met |= events;
}
}
void merge(const AggregateFunctionSequenceMatchData & other)
{
if (other.events_list.empty())
return;
events_list.insert(std::begin(other.events_list), std::end(other.events_list));
sorted = false;
conditions_met |= other.conditions_met;
}
void sort()
{
if (sorted)
return;
::sort(std::begin(events_list), std::end(events_list), Comparator{});
sorted = true;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(sorted, buf);
writeBinary(events_list.size(), buf);
for (const auto & events : events_list)
{
writeBinary(events.first, buf);
writeBinary(events.second.to_ulong(), buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(sorted, buf);
size_t size;
readBinary(size, buf);
/// If we lose these flags, functionality is broken
/// If we serialize/deserialize these flags, we have compatibility issues
/// If we set these flags to 1, we have a minor performance penalty, which seems acceptable
conditions_met.set();
events_list.clear();
events_list.reserve(size);
for (size_t i = 0; i < size; ++i)
{
Timestamp timestamp;
readBinary(timestamp, buf);
UInt64 events;
readBinary(events, buf);
events_list.emplace_back(timestamp, Events{events});
}
}
};
/// Max number of iterations to match the pattern against a sequence, exception thrown when exceeded
constexpr auto sequence_match_max_iterations = 1000000;
template <typename T, typename Data, typename Derived>
class AggregateFunctionSequenceBase : public IAggregateFunctionDataHelper<Data, Derived>
{
public:
AggregateFunctionSequenceBase(const DataTypes & arguments, const Array & params, const String & pattern_, const DataTypePtr & result_type_)
: IAggregateFunctionDataHelper<Data, Derived>(arguments, params, result_type_)
, pattern(pattern_)
{
arg_count = arguments.size();
parsePattern();
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
typename Data::Events events;
for (const auto i : collections::range(1, arg_count))
{
const auto event = assert_cast<const ColumnUInt8 *>(columns[i])->getData()[row_num];
events.set(i - 1, event);
}
this->data(place).add(timestamp, events);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
bool haveSameStateRepresentationImpl(const IAggregateFunction & rhs) const override
{
return this->getName() == rhs.getName() && this->haveEqualArgumentTypes(rhs);
}
private:
enum class PatternActionType
{
SpecificEvent,
AnyEvent,
KleeneStar,
TimeLessOrEqual,
TimeLess,
TimeGreaterOrEqual,
TimeGreater,
TimeEqual
};
struct PatternAction final
{
PatternActionType type;
std::uint64_t extra;
PatternAction() = default;
explicit PatternAction(const PatternActionType type_, const std::uint64_t extra_ = 0) : type{type_}, extra{extra_} {}
};
using PatternActions = PODArrayWithStackMemory<PatternAction, 64>;
Derived & derived() { return static_cast<Derived &>(*this); }
void parsePattern()
{
actions.clear();
actions.emplace_back(PatternActionType::KleeneStar);
dfa_states.clear();
dfa_states.emplace_back(true);
pattern_has_time = false;
const char * pos = pattern.data();
const char * begin = pos;
const char * end = pos + pattern.size();
auto throw_exception = [&](const std::string & msg)
{
throw Exception(ErrorCodes::SYNTAX_ERROR, "{} '{}' at position {}", msg, std::string(pos, end), toString(pos - begin));
};
auto match = [&pos, end](const char * str) mutable
{
size_t length = strlen(str);
if (pos + length <= end && 0 == memcmp(pos, str, length))
{
pos += length;
return true;
}
return false;
};
while (pos < end)
{
if (match("(?"))
{
if (match("t"))
{
PatternActionType type;
if (match("<="))
type = PatternActionType::TimeLessOrEqual;
else if (match("<"))
type = PatternActionType::TimeLess;
else if (match(">="))
type = PatternActionType::TimeGreaterOrEqual;
else if (match(">"))
type = PatternActionType::TimeGreater;
else if (match("=="))
type = PatternActionType::TimeEqual;
else
throw_exception("Unknown time condition");
UInt64 duration = 0;
const auto * prev_pos = pos;
pos = tryReadIntText(duration, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number");
if (actions.back().type != PatternActionType::SpecificEvent &&
actions.back().type != PatternActionType::AnyEvent &&
actions.back().type != PatternActionType::KleeneStar)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Temporal condition should be preceded by an event condition");
pattern_has_time = true;
actions.emplace_back(type, duration);
}
else
{
UInt64 event_number = 0;
const auto * prev_pos = pos;
pos = tryReadIntText(event_number, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number");
if (event_number > arg_count - 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Event number {} is out of range", event_number);
actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1);
dfa_states.back().transition = DFATransition::SpecificEvent;
dfa_states.back().event = static_cast<uint32_t>(event_number - 1);
dfa_states.emplace_back();
conditions_in_pattern.set(event_number - 1);
}
if (!match(")"))
throw_exception("Expected closing parenthesis, found");
}
else if (match(".*"))
{
actions.emplace_back(PatternActionType::KleeneStar);
dfa_states.back().has_kleene = true;
}
else if (match("."))
{
actions.emplace_back(PatternActionType::AnyEvent);
dfa_states.back().transition = DFATransition::AnyEvent;
dfa_states.emplace_back();
}
else
throw_exception("Could not parse pattern, unexpected starting symbol");
}
}
protected:
/// Uses a DFA based approach in order to better handle patterns without
/// time assertions.
///
/// NOTE: This implementation relies on the assumption that the pattern is *small*.
///
/// This algorithm performs in O(mn) (with m the number of DFA states and N the number
/// of events) with a memory consumption and memory allocations in O(m). It means that
/// if n >>> m (which is expected to be the case), this algorithm can be considered linear.
template <typename EventEntry>
bool dfaMatch(EventEntry & events_it, const EventEntry events_end) const
{
using ActiveStates = std::vector<bool>;
/// Those two vectors keep track of which states should be considered for the current
/// event as well as the states which should be considered for the next event.
ActiveStates active_states(dfa_states.size(), false);
ActiveStates next_active_states(dfa_states.size(), false);
active_states[0] = true;
/// Keeps track of dead-ends in order not to iterate over all the events to realize that
/// the match failed.
size_t n_active = 1;
for (/* empty */; events_it != events_end && n_active > 0 && !active_states.back(); ++events_it)
{
n_active = 0;
next_active_states.assign(dfa_states.size(), false);
for (size_t state = 0; state < dfa_states.size(); ++state)
{
if (!active_states[state])
{
continue;
}
switch (dfa_states[state].transition)
{
case DFATransition::None:
break;
case DFATransition::AnyEvent:
next_active_states[state + 1] = true;
++n_active;
break;
case DFATransition::SpecificEvent:
if (events_it->second.test(dfa_states[state].event))
{
next_active_states[state + 1] = true;
++n_active;
}
break;
}
if (dfa_states[state].has_kleene)
{
next_active_states[state] = true;
++n_active;
}
}
swap(active_states, next_active_states);
}
return active_states.back();
}
template <typename EventEntry>
bool backtrackingMatch(EventEntry & events_it, const EventEntry events_end) const
{
const auto action_begin = std::begin(actions);
const auto action_end = std::end(actions);
auto action_it = action_begin;
const auto events_begin = events_it;
auto base_it = events_it;
/// an iterator to action plus an iterator to row in events list plus timestamp at the start of sequence
using backtrack_info = std::tuple<decltype(action_it), EventEntry, EventEntry>;
std::stack<backtrack_info> back_stack;
/// backtrack if possible
const auto do_backtrack = [&]
{
while (!back_stack.empty())
{
auto & top = back_stack.top();
action_it = std::get<0>(top);
events_it = std::next(std::get<1>(top));
base_it = std::get<2>(top);
back_stack.pop();
if (events_it != events_end)
return true;
}
return false;
};
size_t i = 0;
while (action_it != action_end && events_it != events_end)
{
if (action_it->type == PatternActionType::SpecificEvent)
{
if (events_it->second.test(action_it->extra))
{
/// move to the next action and events
base_it = events_it;
++action_it, ++events_it;
}
else if (!do_backtrack())
/// backtracking failed, bail out
break;
}
else if (action_it->type == PatternActionType::AnyEvent)
{
base_it = events_it;
++action_it, ++events_it;
}
else if (action_it->type == PatternActionType::KleeneStar)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (action_it->type == PatternActionType::TimeLessOrEqual)
{
if (events_it->first <= base_it->first + action_it->extra)
{
/// condition satisfied, move onto next action
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (!do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeLess)
{
if (events_it->first < base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (!do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeGreaterOrEqual)
{
if (events_it->first >= base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (++events_it == events_end && !do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeGreater)
{
if (events_it->first > base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (++events_it == events_end && !do_backtrack())
break;
}
else if (action_it->type == PatternActionType::TimeEqual)
{
if (events_it->first == base_it->first + action_it->extra)
{
back_stack.emplace(action_it, events_it, base_it);
base_it = events_it;
++action_it;
}
else if (++events_it == events_end && !do_backtrack())
break;
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown PatternActionType");
if (++i > sequence_match_max_iterations)
throw Exception(ErrorCodes::TOO_SLOW, "Pattern application proves too difficult, exceeding max iterations ({})",
sequence_match_max_iterations);
}
/// if there are some actions remaining
if (action_it != action_end)
{
/// match multiple empty strings at end
while (action_it->type == PatternActionType::KleeneStar ||
action_it->type == PatternActionType::TimeLessOrEqual ||
action_it->type == PatternActionType::TimeLess ||
(action_it->type == PatternActionType::TimeGreaterOrEqual && action_it->extra == 0))
++action_it;
}
if (events_it == events_begin)
++events_it;
return action_it == action_end;
}
/// Splits the pattern into deterministic parts separated by non-deterministic fragments
/// (time constraints and Kleene stars), and tries to match the deterministic parts in their specified order,
/// ignoring the non-deterministic fragments.
/// This function can quickly check that a full match is not possible if some deterministic fragment is missing.
template <typename EventEntry>
bool couldMatchDeterministicParts(const EventEntry events_begin, const EventEntry events_end, bool limit_iterations = true) const
{
size_t events_processed = 0;
auto events_it = events_begin;
const auto actions_end = std::end(actions);
auto actions_it = std::begin(actions);
auto det_part_begin = actions_it;
auto match_deterministic_part = [&events_it, events_end, &events_processed, det_part_begin, actions_it, limit_iterations]()
{
auto events_it_init = events_it;
auto det_part_it = det_part_begin;
while (det_part_it != actions_it && events_it != events_end)
{
/// matching any event
if (det_part_it->type == PatternActionType::AnyEvent)
++events_it, ++det_part_it;
/// matching specific event
else
{
if (events_it->second.test(det_part_it->extra))
++events_it, ++det_part_it;
/// abandon current matching, try to match the deterministic fragment further in the list
else
{
events_it = ++events_it_init;
det_part_it = det_part_begin;
}
}
if (limit_iterations && ++events_processed > sequence_match_max_iterations)
throw Exception(ErrorCodes::TOO_SLOW, "Pattern application proves too difficult, exceeding max iterations ({})",
sequence_match_max_iterations);
}
return det_part_it == actions_it;
};
for (; actions_it != actions_end; ++actions_it)
if (actions_it->type != PatternActionType::SpecificEvent && actions_it->type != PatternActionType::AnyEvent)
{
if (!match_deterministic_part())
return false;
det_part_begin = std::next(actions_it);
}
return match_deterministic_part();
}
private:
enum class DFATransition : char
{
/// .-------.
/// | |
/// `-------'
None,
/// .-------. (?[0-9])
/// | | ----------
/// `-------'
SpecificEvent,
/// .-------. .
/// | | ----------
/// `-------'
AnyEvent,
};
struct DFAState
{
explicit DFAState(bool has_kleene_ = false)
: has_kleene{has_kleene_}, event{0}, transition{DFATransition::None}
{}
/// .-------.
/// | | - - -
/// `-------'
/// |_^
bool has_kleene;
/// In the case of a state transitions with a `SpecificEvent`,
/// `event` contains the value of the event.
uint32_t event;
/// The kind of transition out of this state.
DFATransition transition;
};
using DFAStates = std::vector<DFAState>;
protected:
/// `True` if the parsed pattern contains time assertions (?t...), `false` otherwise.
bool pattern_has_time;
/// sequenceMatch conditions met at least once in the pattern
std::bitset<max_events> conditions_in_pattern;
private:
std::string pattern;
size_t arg_count;
PatternActions actions;
DFAStates dfa_states;
};
template <typename T, typename Data>
class AggregateFunctionSequenceMatch final : public AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceMatch<T, Data>>
{
public:
AggregateFunctionSequenceMatch(const DataTypes & arguments, const Array & params, const String & pattern_)
: AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceMatch<T, Data>>(arguments, params, pattern_, std::make_shared<DataTypeUInt8>()) {}
using AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceMatch<T, Data>>::AggregateFunctionSequenceBase;
String getName() const override { return "sequenceMatch"; }
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & output = assert_cast<ColumnUInt8 &>(to).getData();
if ((this->conditions_in_pattern & this->data(place).conditions_met) != this->conditions_in_pattern)
{
output.push_back(false);
return;
}
this->data(place).sort();
const auto & data_ref = this->data(place);
const auto events_begin = std::begin(data_ref.events_list);
const auto events_end = std::end(data_ref.events_list);
auto events_it = events_begin;
bool match = (this->pattern_has_time ?
(this->couldMatchDeterministicParts(events_begin, events_end) && this->backtrackingMatch(events_it, events_end)) :
this->dfaMatch(events_it, events_end));
output.push_back(match);
}
};
template <typename T, typename Data>
class AggregateFunctionSequenceCount final : public AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceCount<T, Data>>
{
public:
AggregateFunctionSequenceCount(const DataTypes & arguments, const Array & params, const String & pattern_)
: AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceCount<T, Data>>(arguments, params, pattern_, std::make_shared<DataTypeUInt64>()) {}
using AggregateFunctionSequenceBase<T, Data, AggregateFunctionSequenceCount<T, Data>>::AggregateFunctionSequenceBase;
String getName() const override { return "sequenceCount"; }
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & output = assert_cast<ColumnUInt64 &>(to).getData();
if ((this->conditions_in_pattern & this->data(place).conditions_met) != this->conditions_in_pattern)
{
output.push_back(0);
return;
}
this->data(place).sort();
output.push_back(count(place));
}
private:
UInt64 count(ConstAggregateDataPtr __restrict place) const
{
const auto & data_ref = this->data(place);
const auto events_begin = std::begin(data_ref.events_list);
const auto events_end = std::end(data_ref.events_list);
auto events_it = events_begin;
size_t count = 0;
// check if there is a chance of matching the sequence at least once
if (this->couldMatchDeterministicParts(events_begin, events_end))
{
while (events_it != events_end && this->backtrackingMatch(events_it, events_end))
++count;
}
return count;
}
};
}

View File

@ -1,14 +1,25 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSequenceNextNode.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Context.h>
#include <Common/CurrentThread.h>
#include <base/range.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnNullable.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <bitset>
namespace DB
@ -24,11 +35,409 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_AGGREGATE_FUNCTION;
extern const int TOO_LARGE_ARRAY_SIZE;
}
namespace
{
enum class SequenceDirection
{
Forward,
Backward,
};
enum SequenceBase
{
Head,
Tail,
FirstMatch,
LastMatch,
};
/// This is for security
static const UInt64 max_node_size_deserialize = 0xFFFFFF;
/// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl
template <typename Node, size_t MaxEventsSize>
struct NodeBase
{
UInt64 size; /// size of payload
DataTypeDateTime::FieldType event_time;
std::bitset<MaxEventsSize> events_bitset;
bool can_be_base;
char * data() { return reinterpret_cast<char *>(this) + sizeof(Node); }
const char * data() const { return reinterpret_cast<const char *>(this) + sizeof(Node); }
Node * clone(Arena * arena) const
{
return reinterpret_cast<Node *>(
const_cast<char *>(arena->alignedInsert(reinterpret_cast<const char *>(this), sizeof(Node) + size, alignof(Node))));
}
void write(WriteBuffer & buf) const
{
writeVarUInt(size, buf);
buf.write(data(), size);
writeBinary(event_time, buf);
UInt64 ulong_bitset = events_bitset.to_ulong();
writeBinary(ulong_bitset, buf);
writeBinary(can_be_base, buf);
}
static Node * read(ReadBuffer & buf, Arena * arena)
{
UInt64 size;
readVarUInt(size, buf);
if (unlikely(size > max_node_size_deserialize))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large node state size");
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + size, alignof(Node)));
node->size = size;
buf.readStrict(node->data(), size);
readBinary(node->event_time, buf);
UInt64 ulong_bitset;
readBinary(ulong_bitset, buf);
node->events_bitset = ulong_bitset;
readBinary(node->can_be_base, buf);
return node;
}
};
/// It stores String, timestamp, bitset of matched events.
template <size_t MaxEventsSize>
struct NodeString : public NodeBase<NodeString<MaxEventsSize>, MaxEventsSize>
{
using Node = NodeString<MaxEventsSize>;
static Node * allocate(const IColumn & column, size_t row_num, Arena * arena)
{
StringRef string = assert_cast<const ColumnString &>(column).getDataAt(row_num);
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node)));
node->size = string.size;
memcpy(node->data(), string.data, string.size);
return node;
}
void insertInto(IColumn & column)
{
assert_cast<ColumnString &>(column).insertData(this->data(), this->size);
}
bool compare(const Node * rhs) const
{
auto cmp = strncmp(this->data(), rhs->data(), std::min(this->size, rhs->size));
return (cmp == 0) ? this->size < rhs->size : cmp < 0;
}
};
/// TODO : Support other types than string
template <typename Node>
struct SequenceNextNodeGeneralData
{
using Allocator = MixedAlignedArenaAllocator<alignof(Node *), 4096>;
using Array = PODArray<Node *, 32, Allocator>;
Array value;
bool sorted = false;
struct Comparator final
{
bool operator()(const Node * lhs, const Node * rhs) const
{
return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time;
}
};
void sort()
{
if (!sorted)
{
std::stable_sort(std::begin(value), std::end(value), Comparator{});
sorted = true;
}
}
};
/// Implementation of sequenceFirstNode
template <typename T, typename Node>
class SequenceNextNodeImpl final
: public IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, SequenceNextNodeImpl<T, Node>>
{
using Self = SequenceNextNodeImpl<T, Node>;
using Data = SequenceNextNodeGeneralData<Node>;
static Data & data(AggregateDataPtr __restrict place) { return *reinterpret_cast<Data *>(place); }
static const Data & data(ConstAggregateDataPtr __restrict place) { return *reinterpret_cast<const Data *>(place); }
static constexpr size_t base_cond_column_idx = 2;
static constexpr size_t event_column_idx = 1;
SequenceBase seq_base_kind;
SequenceDirection seq_direction;
const size_t min_required_args;
DataTypePtr & data_type;
UInt8 events_size;
UInt64 max_elems;
public:
SequenceNextNodeImpl(
const DataTypePtr & data_type_,
const DataTypes & arguments,
const Array & parameters_,
SequenceBase seq_base_kind_,
SequenceDirection seq_direction_,
size_t min_required_args_,
UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, Self>(arguments, parameters_, data_type_)
, seq_base_kind(seq_base_kind_)
, seq_direction(seq_direction_)
, min_required_args(min_required_args_)
, data_type(this->argument_types[0])
, events_size(arguments.size() - min_required_args)
, max_elems(max_elems_)
{
}
String getName() const override { return "sequenceNextNode"; }
bool haveSameStateRepresentationImpl(const IAggregateFunction & rhs) const override
{
return this->getName() == rhs.getName() && this->haveEqualArgumentTypes(rhs);
}
void insert(Data & a, const Node * v, Arena * arena) const
{
++a.total_values;
a.value.push_back(v->clone(arena), arena);
}
void create(AggregateDataPtr __restrict place) const override /// NOLINT
{
new (place) Data;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Node * node = Node::allocate(*columns[event_column_idx], row_num, arena);
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
/// The events_bitset variable stores matched events in the form of bitset.
/// Each Nth-bit indicates that the Nth-event are matched.
/// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005.
/// 0x00000000
/// + 1 (bit of event1)
/// + 4 (bit of event3)
node->events_bitset.reset();
for (UInt8 i = 0; i < events_size; ++i)
if (assert_cast<const ColumnVector<UInt8> *>(columns[min_required_args + i])->getData()[row_num])
node->events_bitset.set(i);
node->event_time = static_cast<DataTypeDateTime::FieldType>(timestamp);
node->can_be_base = assert_cast<const ColumnVector<UInt8> *>(columns[base_cond_column_idx])->getData()[row_num];
data(place).value.push_back(node, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
if (data(rhs).value.empty())
return;
if (data(place).value.size() >= max_elems)
return;
auto & a = data(place).value;
auto & b = data(rhs).value;
const auto a_size = a.size();
const UInt64 new_elems = std::min(data(rhs).value.size(), static_cast<size_t>(max_elems) - data(place).value.size());
for (UInt64 i = 0; i < new_elems; ++i)
a.push_back(b[i]->clone(arena), arena);
/// Either sort whole container or do so partially merging ranges afterwards
using Comparator = typename SequenceNextNodeGeneralData<Node>::Comparator;
if (!data(place).sorted && !data(rhs).sorted)
std::stable_sort(std::begin(a), std::end(a), Comparator{});
else
{
const auto begin = std::begin(a);
const auto middle = std::next(begin, a_size);
const auto end = std::end(a);
if (!data(place).sorted)
std::stable_sort(begin, middle, Comparator{});
if (!data(rhs).sorted)
std::stable_sort(middle, end, Comparator{});
std::inplace_merge(begin, middle, end, Comparator{});
}
data(place).sorted = true;
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
/// Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node.
this->data(const_cast<AggregateDataPtr>(place)).sort();
writeBinary(data(place).sorted, buf);
auto & value = data(place).value;
size_t size = std::min(static_cast<size_t>(events_size + 1), value.size());
switch (seq_base_kind)
{
case SequenceBase::Head:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[i]->write(buf);
break;
case SequenceBase::Tail:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[value.size() - size + i]->write(buf);
break;
case SequenceBase::FirstMatch:
case SequenceBase::LastMatch:
writeVarUInt(value.size(), buf);
for (auto & node : value)
node->write(buf);
break;
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
readBinary(data(place).sorted, buf);
UInt64 size;
readVarUInt(size, buf);
if (unlikely(size == 0))
return;
if (unlikely(size > max_node_size_deserialize))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Too large array size (maximum: {})", max_node_size_deserialize);
auto & value = data(place).value;
value.resize(size, arena);
for (UInt64 i = 0; i < size; ++i)
value[i] = Node::read(buf, arena);
}
inline std::optional<size_t> getBaseIndex(Data & data) const
{
if (data.value.size() == 0)
return {};
switch (seq_base_kind)
{
case SequenceBase::Head:
if (data.value[0]->can_be_base)
return 0;
break;
case SequenceBase::Tail:
if (data.value[data.value.size() - 1]->can_be_base)
return data.value.size() - 1;
break;
case SequenceBase::FirstMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base)
return i;
}
break;
case SequenceBase::LastMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
auto reversed_i = data.value.size() - i - 1;
if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base)
return reversed_i;
}
break;
}
return {};
}
/// This method returns an index of next node that matched the events.
/// matched events in the chain of events are represented as a bitmask.
/// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on.
UInt32 getNextNodeIndex(Data & data) const
{
const UInt32 unmatched_idx = static_cast<UInt32>(data.value.size());
if (data.value.size() <= events_size)
return unmatched_idx;
data.sort();
std::optional<size_t> base_opt = getBaseIndex(data);
if (!base_opt.has_value())
return unmatched_idx;
UInt32 base = static_cast<UInt32>(base_opt.value());
if (events_size == 0)
return data.value.size() > 0 ? base : unmatched_idx;
UInt32 i = 0;
switch (seq_direction)
{
case SequenceDirection::Forward:
for (i = 0; i < events_size && base + i < data.value.size(); ++i)
if (!data.value[base + i]->events_bitset.test(i))
break;
return (i == events_size) ? base + i : unmatched_idx;
case SequenceDirection::Backward:
for (i = 0; i < events_size && i < base; ++i)
if (!data.value[base - i]->events_bitset.test(i))
break;
return (i == events_size) ? base - i : unmatched_idx;
}
UNREACHABLE();
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & value = data(place).value;
UInt32 event_idx = getNextNodeIndex(this->data(place));
if (event_idx < value.size())
{
ColumnNullable & to_concrete = assert_cast<ColumnNullable &>(to);
value[event_idx]->insertInto(to_concrete.getNestedColumn());
to_concrete.getNullMapData().push_back(0);
}
else
{
to.insertDefault();
}
}
bool allocatesMemoryInArena() const override { return true; }
};
template <typename T>
inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(
const DataTypePtr data_type, const DataTypes & argument_types, const Array & parameters, SequenceDirection direction, SequenceBase base)

View File

@ -1,432 +0,0 @@
#pragma once
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnNullable.h>
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <type_traits>
#include <bitset>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int TOO_LARGE_ARRAY_SIZE;
}
enum class SequenceDirection
{
Forward,
Backward,
};
enum SequenceBase
{
Head,
Tail,
FirstMatch,
LastMatch,
};
/// This is for security
static const UInt64 max_node_size_deserialize = 0xFFFFFF;
/// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl
template <typename Node, size_t MaxEventsSize>
struct NodeBase
{
UInt64 size; /// size of payload
DataTypeDateTime::FieldType event_time;
std::bitset<MaxEventsSize> events_bitset;
bool can_be_base;
char * data() { return reinterpret_cast<char *>(this) + sizeof(Node); }
const char * data() const { return reinterpret_cast<const char *>(this) + sizeof(Node); }
Node * clone(Arena * arena) const
{
return reinterpret_cast<Node *>(
const_cast<char *>(arena->alignedInsert(reinterpret_cast<const char *>(this), sizeof(Node) + size, alignof(Node))));
}
void write(WriteBuffer & buf) const
{
writeVarUInt(size, buf);
buf.write(data(), size);
writeBinary(event_time, buf);
UInt64 ulong_bitset = events_bitset.to_ulong();
writeBinary(ulong_bitset, buf);
writeBinary(can_be_base, buf);
}
static Node * read(ReadBuffer & buf, Arena * arena)
{
UInt64 size;
readVarUInt(size, buf);
if (unlikely(size > max_node_size_deserialize))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large node state size");
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + size, alignof(Node)));
node->size = size;
buf.readStrict(node->data(), size);
readBinary(node->event_time, buf);
UInt64 ulong_bitset;
readBinary(ulong_bitset, buf);
node->events_bitset = ulong_bitset;
readBinary(node->can_be_base, buf);
return node;
}
};
/// It stores String, timestamp, bitset of matched events.
template <size_t MaxEventsSize>
struct NodeString : public NodeBase<NodeString<MaxEventsSize>, MaxEventsSize>
{
using Node = NodeString<MaxEventsSize>;
static Node * allocate(const IColumn & column, size_t row_num, Arena * arena)
{
StringRef string = assert_cast<const ColumnString &>(column).getDataAt(row_num);
Node * node = reinterpret_cast<Node *>(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node)));
node->size = string.size;
memcpy(node->data(), string.data, string.size);
return node;
}
void insertInto(IColumn & column)
{
assert_cast<ColumnString &>(column).insertData(this->data(), this->size);
}
bool compare(const Node * rhs) const
{
auto cmp = strncmp(this->data(), rhs->data(), std::min(this->size, rhs->size));
return (cmp == 0) ? this->size < rhs->size : cmp < 0;
}
};
/// TODO : Support other types than string
template <typename Node>
struct SequenceNextNodeGeneralData
{
using Allocator = MixedAlignedArenaAllocator<alignof(Node *), 4096>;
using Array = PODArray<Node *, 32, Allocator>;
Array value;
bool sorted = false;
struct Comparator final
{
bool operator()(const Node * lhs, const Node * rhs) const
{
return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time;
}
};
void sort()
{
if (!sorted)
{
std::stable_sort(std::begin(value), std::end(value), Comparator{});
sorted = true;
}
}
};
/// Implementation of sequenceFirstNode
template <typename T, typename Node>
class SequenceNextNodeImpl final
: public IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, SequenceNextNodeImpl<T, Node>>
{
using Self = SequenceNextNodeImpl<T, Node>;
using Data = SequenceNextNodeGeneralData<Node>;
static Data & data(AggregateDataPtr __restrict place) { return *reinterpret_cast<Data *>(place); }
static const Data & data(ConstAggregateDataPtr __restrict place) { return *reinterpret_cast<const Data *>(place); }
static constexpr size_t base_cond_column_idx = 2;
static constexpr size_t event_column_idx = 1;
SequenceBase seq_base_kind;
SequenceDirection seq_direction;
const size_t min_required_args;
DataTypePtr & data_type;
UInt8 events_size;
UInt64 max_elems;
public:
SequenceNextNodeImpl(
const DataTypePtr & data_type_,
const DataTypes & arguments,
const Array & parameters_,
SequenceBase seq_base_kind_,
SequenceDirection seq_direction_,
size_t min_required_args_,
UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
: IAggregateFunctionDataHelper<SequenceNextNodeGeneralData<Node>, Self>(arguments, parameters_, data_type_)
, seq_base_kind(seq_base_kind_)
, seq_direction(seq_direction_)
, min_required_args(min_required_args_)
, data_type(this->argument_types[0])
, events_size(arguments.size() - min_required_args)
, max_elems(max_elems_)
{
}
String getName() const override { return "sequenceNextNode"; }
bool haveSameStateRepresentationImpl(const IAggregateFunction & rhs) const override
{
return this->getName() == rhs.getName() && this->haveEqualArgumentTypes(rhs);
}
void insert(Data & a, const Node * v, Arena * arena) const
{
++a.total_values;
a.value.push_back(v->clone(arena), arena);
}
void create(AggregateDataPtr __restrict place) const override /// NOLINT
{
new (place) Data;
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
Node * node = Node::allocate(*columns[event_column_idx], row_num, arena);
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
/// The events_bitset variable stores matched events in the form of bitset.
/// Each Nth-bit indicates that the Nth-event are matched.
/// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005.
/// 0x00000000
/// + 1 (bit of event1)
/// + 4 (bit of event3)
node->events_bitset.reset();
for (UInt8 i = 0; i < events_size; ++i)
if (assert_cast<const ColumnVector<UInt8> *>(columns[min_required_args + i])->getData()[row_num])
node->events_bitset.set(i);
node->event_time = static_cast<DataTypeDateTime::FieldType>(timestamp);
node->can_be_base = assert_cast<const ColumnVector<UInt8> *>(columns[base_cond_column_idx])->getData()[row_num];
data(place).value.push_back(node, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
if (data(rhs).value.empty())
return;
if (data(place).value.size() >= max_elems)
return;
auto & a = data(place).value;
auto & b = data(rhs).value;
const auto a_size = a.size();
const UInt64 new_elems = std::min(data(rhs).value.size(), static_cast<size_t>(max_elems) - data(place).value.size());
for (UInt64 i = 0; i < new_elems; ++i)
a.push_back(b[i]->clone(arena), arena);
/// Either sort whole container or do so partially merging ranges afterwards
using Comparator = typename SequenceNextNodeGeneralData<Node>::Comparator;
if (!data(place).sorted && !data(rhs).sorted)
std::stable_sort(std::begin(a), std::end(a), Comparator{});
else
{
const auto begin = std::begin(a);
const auto middle = std::next(begin, a_size);
const auto end = std::end(a);
if (!data(place).sorted)
std::stable_sort(begin, middle, Comparator{});
if (!data(rhs).sorted)
std::stable_sort(middle, end, Comparator{});
std::inplace_merge(begin, middle, end, Comparator{});
}
data(place).sorted = true;
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
/// Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node.
this->data(const_cast<AggregateDataPtr>(place)).sort();
writeBinary(data(place).sorted, buf);
auto & value = data(place).value;
size_t size = std::min(static_cast<size_t>(events_size + 1), value.size());
switch (seq_base_kind)
{
case SequenceBase::Head:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[i]->write(buf);
break;
case SequenceBase::Tail:
writeVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
value[value.size() - size + i]->write(buf);
break;
case SequenceBase::FirstMatch:
case SequenceBase::LastMatch:
writeVarUInt(value.size(), buf);
for (auto & node : value)
node->write(buf);
break;
}
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
readBinary(data(place).sorted, buf);
UInt64 size;
readVarUInt(size, buf);
if (unlikely(size == 0))
return;
if (unlikely(size > max_node_size_deserialize))
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE,
"Too large array size (maximum: {})", max_node_size_deserialize);
auto & value = data(place).value;
value.resize(size, arena);
for (UInt64 i = 0; i < size; ++i)
value[i] = Node::read(buf, arena);
}
inline std::optional<size_t> getBaseIndex(Data & data) const
{
if (data.value.size() == 0)
return {};
switch (seq_base_kind)
{
case SequenceBase::Head:
if (data.value[0]->can_be_base)
return 0;
break;
case SequenceBase::Tail:
if (data.value[data.value.size() - 1]->can_be_base)
return data.value.size() - 1;
break;
case SequenceBase::FirstMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base)
return i;
}
break;
case SequenceBase::LastMatch:
for (size_t i = 0; i < data.value.size(); ++i)
{
auto reversed_i = data.value.size() - i - 1;
if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base)
return reversed_i;
}
break;
}
return {};
}
/// This method returns an index of next node that matched the events.
/// matched events in the chain of events are represented as a bitmask.
/// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on.
UInt32 getNextNodeIndex(Data & data) const
{
const UInt32 unmatched_idx = static_cast<UInt32>(data.value.size());
if (data.value.size() <= events_size)
return unmatched_idx;
data.sort();
std::optional<size_t> base_opt = getBaseIndex(data);
if (!base_opt.has_value())
return unmatched_idx;
UInt32 base = static_cast<UInt32>(base_opt.value());
if (events_size == 0)
return data.value.size() > 0 ? base : unmatched_idx;
UInt32 i = 0;
switch (seq_direction)
{
case SequenceDirection::Forward:
for (i = 0; i < events_size && base + i < data.value.size(); ++i)
if (!data.value[base + i]->events_bitset.test(i))
break;
return (i == events_size) ? base + i : unmatched_idx;
case SequenceDirection::Backward:
for (i = 0; i < events_size && i < base; ++i)
if (!data.value[base - i]->events_bitset.test(i))
break;
return (i == events_size) ? base - i : unmatched_idx;
}
UNREACHABLE();
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & value = data(place).value;
UInt32 event_idx = getNextNodeIndex(this->data(place));
if (event_idx < value.size())
{
ColumnNullable & to_concrete = assert_cast<ColumnNullable &>(to);
value[event_idx]->insertInto(to_concrete.getNestedColumn());
to_concrete.getNullMapData().push_back(0);
}
else
{
to.insertDefault();
}
}
bool allocatesMemoryInArena() const override { return true; }
};
}

View File

@ -1,10 +1,21 @@
#include <AggregateFunctions/AggregateFunctionSimpleLinearRegression.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <limits>
namespace DB
{
struct Settings;
namespace ErrorCodes
@ -15,6 +26,167 @@ namespace ErrorCodes
namespace
{
template <typename T>
struct AggregateFunctionSimpleLinearRegressionData final
{
size_t count = 0;
T sum_x = 0;
T sum_y = 0;
T sum_xx = 0;
T sum_xy = 0;
void add(T x, T y)
{
count += 1;
sum_x += x;
sum_y += y;
sum_xx += x * x;
sum_xy += x * y;
}
void merge(const AggregateFunctionSimpleLinearRegressionData & other)
{
count += other.count;
sum_x += other.sum_x;
sum_y += other.sum_y;
sum_xx += other.sum_xx;
sum_xy += other.sum_xy;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(count, buf);
writeBinary(sum_x, buf);
writeBinary(sum_y, buf);
writeBinary(sum_xx, buf);
writeBinary(sum_xy, buf);
}
void deserialize(ReadBuffer & buf)
{
readBinary(count, buf);
readBinary(sum_x, buf);
readBinary(sum_y, buf);
readBinary(sum_xx, buf);
readBinary(sum_xy, buf);
}
T getK() const
{
T divisor = sum_xx * count - sum_x * sum_x;
if (divisor == 0)
return std::numeric_limits<T>::quiet_NaN();
return (sum_xy * count - sum_x * sum_y) / divisor;
}
T getB(T k) const
{
if (count == 0)
return std::numeric_limits<T>::quiet_NaN();
return (sum_y - k * sum_x) / count;
}
};
/// Calculates simple linear regression parameters.
/// Result is a tuple (k, b) for y = k * x + b equation, solved by least squares approximation.
template <typename X, typename Y, typename Ret = Float64>
class AggregateFunctionSimpleLinearRegression final : public IAggregateFunctionDataHelper<
AggregateFunctionSimpleLinearRegressionData<Ret>,
AggregateFunctionSimpleLinearRegression<X, Y, Ret>
>
{
public:
AggregateFunctionSimpleLinearRegression(
const DataTypes & arguments,
const Array & params
):
IAggregateFunctionDataHelper<
AggregateFunctionSimpleLinearRegressionData<Ret>,
AggregateFunctionSimpleLinearRegression<X, Y, Ret>
> {arguments, params, createResultType()}
{
// notice: arguments has been checked before
}
String getName() const override
{
return "simpleLinearRegression";
}
void add(
AggregateDataPtr __restrict place,
const IColumn ** columns,
size_t row_num,
Arena *
) const override
{
auto col_x = assert_cast<const ColumnVector<X> *>(columns[0]);
auto col_y = assert_cast<const ColumnVector<Y> *>(columns[1]);
X x = col_x->getData()[row_num];
Y y = col_y->getData()[row_num];
this->data(place).add(x, y);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Ret>>(),
std::make_shared<DataTypeNumber<Ret>>(),
};
Strings names
{
"k",
"b",
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(
AggregateDataPtr __restrict place,
IColumn & to,
Arena *) const override
{
Ret k = this->data(place).getK();
Ret b = this->data(place).getB(k);
auto & col_tuple = assert_cast<ColumnTuple &>(to);
auto & col_k = assert_cast<ColumnVector<Ret> &>(col_tuple.getColumn(0));
auto & col_b = assert_cast<ColumnVector<Ret> &>(col_tuple.getColumn(1));
col_k.getData().push_back(k);
col_b.getData().push_back(b);
}
};
AggregateFunctionPtr createAggregateFunctionSimpleLinearRegression(
const String & name,
const DataTypes & arguments,

View File

@ -1,182 +0,0 @@
#pragma once
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnTuple.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <limits>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
}
template <typename T>
struct AggregateFunctionSimpleLinearRegressionData final
{
size_t count = 0;
T sum_x = 0;
T sum_y = 0;
T sum_xx = 0;
T sum_xy = 0;
void add(T x, T y)
{
count += 1;
sum_x += x;
sum_y += y;
sum_xx += x * x;
sum_xy += x * y;
}
void merge(const AggregateFunctionSimpleLinearRegressionData & other)
{
count += other.count;
sum_x += other.sum_x;
sum_y += other.sum_y;
sum_xx += other.sum_xx;
sum_xy += other.sum_xy;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(count, buf);
writeBinary(sum_x, buf);
writeBinary(sum_y, buf);
writeBinary(sum_xx, buf);
writeBinary(sum_xy, buf);
}
void deserialize(ReadBuffer & buf)
{
readBinary(count, buf);
readBinary(sum_x, buf);
readBinary(sum_y, buf);
readBinary(sum_xx, buf);
readBinary(sum_xy, buf);
}
T getK() const
{
T divisor = sum_xx * count - sum_x * sum_x;
if (divisor == 0)
return std::numeric_limits<T>::quiet_NaN();
return (sum_xy * count - sum_x * sum_y) / divisor;
}
T getB(T k) const
{
if (count == 0)
return std::numeric_limits<T>::quiet_NaN();
return (sum_y - k * sum_x) / count;
}
};
/// Calculates simple linear regression parameters.
/// Result is a tuple (k, b) for y = k * x + b equation, solved by least squares approximation.
template <typename X, typename Y, typename Ret = Float64>
class AggregateFunctionSimpleLinearRegression final : public IAggregateFunctionDataHelper<
AggregateFunctionSimpleLinearRegressionData<Ret>,
AggregateFunctionSimpleLinearRegression<X, Y, Ret>
>
{
public:
AggregateFunctionSimpleLinearRegression(
const DataTypes & arguments,
const Array & params
):
IAggregateFunctionDataHelper<
AggregateFunctionSimpleLinearRegressionData<Ret>,
AggregateFunctionSimpleLinearRegression<X, Y, Ret>
> {arguments, params, createResultType()}
{
// notice: arguments has been checked before
}
String getName() const override
{
return "simpleLinearRegression";
}
void add(
AggregateDataPtr __restrict place,
const IColumn ** columns,
size_t row_num,
Arena *
) const override
{
auto col_x = assert_cast<const ColumnVector<X> *>(columns[0]);
auto col_y = assert_cast<const ColumnVector<Y> *>(columns[1]);
X x = col_x->getData()[row_num];
Y y = col_y->getData()[row_num];
this->data(place).add(x, y);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
static DataTypePtr createResultType()
{
DataTypes types
{
std::make_shared<DataTypeNumber<Ret>>(),
std::make_shared<DataTypeNumber<Ret>>(),
};
Strings names
{
"k",
"b",
};
return std::make_shared<DataTypeTuple>(
std::move(types),
std::move(names)
);
}
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(
AggregateDataPtr __restrict place,
IColumn & to,
Arena *) const override
{
Ret k = this->data(place).getK();
Ret b = this->data(place).getB(k);
auto & col_tuple = assert_cast<ColumnTuple &>(to);
auto & col_k = assert_cast<ColumnVector<Ret> &>(col_tuple.getColumn(0));
auto & col_b = assert_cast<ColumnVector<Ret> &>(col_tuple.getColumn(1));
col_k.getData().push_back(k);
col_b.getData().push_back(b);
}
};
}

View File

@ -1,8 +1,18 @@
#include <AggregateFunctions/AggregateFunctionSparkbar.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <array>
#include <string_view>
#include <DataTypes/DataTypeString.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnString.h>
#include <Common/PODArray.h>
#include <IO/ReadBufferFromString.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/IColumn.h>
namespace DB
{
@ -13,11 +23,309 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
}
namespace
{
template<typename X, typename Y>
struct AggregateFunctionSparkbarData
{
/// TODO: calculate histogram instead of storing all points
using Points = HashMap<X, Y>;
Points points;
X min_x = std::numeric_limits<X>::max();
X max_x = std::numeric_limits<X>::lowest();
Y min_y = std::numeric_limits<Y>::max();
Y max_y = std::numeric_limits<Y>::lowest();
Y insert(const X & x, const Y & y)
{
if (isNaN(y) || y <= 0)
return 0;
auto [it, inserted] = points.insert({x, y});
if (!inserted)
{
if constexpr (std::is_floating_point_v<Y>)
{
it->getMapped() += y;
return it->getMapped();
}
else
{
Y res;
bool has_overfllow = common::addOverflow(it->getMapped(), y, res);
it->getMapped() = has_overfllow ? std::numeric_limits<Y>::max() : res;
}
}
return it->getMapped();
}
void add(X x, Y y)
{
auto new_y = insert(x, y);
min_x = std::min(x, min_x);
max_x = std::max(x, max_x);
min_y = std::min(y, min_y);
max_y = std::max(new_y, max_y);
}
void merge(const AggregateFunctionSparkbarData & other)
{
if (other.points.empty())
return;
for (auto & point : other.points)
{
auto new_y = insert(point.getKey(), point.getMapped());
max_y = std::max(new_y, max_y);
}
min_x = std::min(other.min_x, min_x);
max_x = std::max(other.max_x, max_x);
min_y = std::min(other.min_y, min_y);
max_y = std::max(other.max_y, max_y);
}
void serialize(WriteBuffer & buf) const
{
writeBinary(min_x, buf);
writeBinary(max_x, buf);
writeBinary(min_y, buf);
writeBinary(max_y, buf);
writeVarUInt(points.size(), buf);
for (const auto & elem : points)
{
writeBinary(elem.getKey(), buf);
writeBinary(elem.getMapped(), buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(min_x, buf);
readBinary(max_x, buf);
readBinary(min_y, buf);
readBinary(max_y, buf);
size_t size;
readVarUInt(size, buf);
X x;
Y y;
for (size_t i = 0; i < size; ++i)
{
readBinary(x, buf);
readBinary(y, buf);
insert(x, y);
}
}
};
template<typename X, typename Y>
class AggregateFunctionSparkbar final
: public IAggregateFunctionDataHelper<AggregateFunctionSparkbarData<X, Y>, AggregateFunctionSparkbar<X, Y>>
{
private:
static constexpr size_t BAR_LEVELS = 8;
const size_t width = 0;
/// Range for x specified in parameters.
const bool is_specified_range_x = false;
const X begin_x = std::numeric_limits<X>::min();
const X end_x = std::numeric_limits<X>::max();
size_t updateFrame(ColumnString::Chars & frame, Y value) const
{
static constexpr std::array<std::string_view, BAR_LEVELS + 1> bars{" ", "", "", "", "", "", "", "", ""};
const auto & bar = (isNaN(value) || value < 1 || static_cast<Y>(BAR_LEVELS) < value) ? bars[0] : bars[static_cast<UInt8>(value)];
frame.insert(bar.begin(), bar.end());
return bar.size();
}
/**
* The minimum value of y is rendered as the lowest height "",
* the maximum value of y is rendered as the highest height "", and the middle value will be rendered proportionally.
* If a bucket has no y value, it will be rendered as " ".
*/
void render(ColumnString & to_column, const AggregateFunctionSparkbarData<X, Y> & data) const
{
auto & values = to_column.getChars();
auto & offsets = to_column.getOffsets();
if (data.points.empty())
{
values.push_back('\0');
offsets.push_back(offsets.empty() ? 1 : offsets.back() + 1);
return;
}
auto from_x = is_specified_range_x ? begin_x : data.min_x;
auto to_x = is_specified_range_x ? end_x : data.max_x;
if (from_x >= to_x)
{
size_t sz = updateFrame(values, 8);
values.push_back('\0');
offsets.push_back(offsets.empty() ? sz + 1 : offsets.back() + sz + 1);
return;
}
PaddedPODArray<Y> histogram(width, 0);
PaddedPODArray<UInt64> count_histogram(width, 0); /// The number of points in each bucket
for (const auto & point : data.points)
{
if (point.getKey() < from_x || to_x < point.getKey())
continue;
X delta = to_x - from_x;
if (delta < std::numeric_limits<X>::max())
delta = delta + 1;
X value = point.getKey() - from_x;
Float64 w = histogram.size();
size_t index = std::min<size_t>(static_cast<size_t>(w / delta * value), histogram.size() - 1);
Y res;
bool has_overfllow = false;
if constexpr (std::is_floating_point_v<Y>)
res = histogram[index] + point.getMapped();
else
has_overfllow = common::addOverflow(histogram[index], point.getMapped(), res);
if (unlikely(has_overfllow))
{
/// In case of overflow, just saturate
/// Do not count new values, because we do not know how many of them were added
histogram[index] = std::numeric_limits<Y>::max();
}
else
{
histogram[index] = res;
count_histogram[index] += 1;
}
}
for (size_t i = 0; i < histogram.size(); ++i)
{
if (count_histogram[i] > 0)
histogram[i] /= count_histogram[i];
}
Y y_max = 0;
for (auto & y : histogram)
{
if (isNaN(y) || y <= 0)
continue;
y_max = std::max(y_max, y);
}
if (y_max == 0)
{
values.push_back('\0');
offsets.push_back(offsets.empty() ? 1 : offsets.back() + 1);
return;
}
/// Scale the histogram to the range [0, BAR_LEVELS]
for (auto & y : histogram)
{
if (isNaN(y) || y <= 0)
{
y = 0;
continue;
}
constexpr auto levels_num = static_cast<Y>(BAR_LEVELS - 1);
if constexpr (std::is_floating_point_v<Y>)
{
y = y / (y_max / levels_num) + 1;
}
else
{
Y scaled;
bool has_overfllow = common::mulOverflow<Y>(y, levels_num, scaled);
if (has_overfllow)
y = y / (y_max / levels_num) + 1;
else
y = scaled / y_max + 1;
}
}
size_t sz = 0;
for (const auto & y : histogram)
sz += updateFrame(values, y);
values.push_back('\0');
offsets.push_back(offsets.empty() ? sz + 1 : offsets.back() + sz + 1);
}
public:
AggregateFunctionSparkbar(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionSparkbarData<X, Y>, AggregateFunctionSparkbar>(arguments, params, std::make_shared<DataTypeString>())
, width(params.empty() ? 0 : params.at(0).safeGet<UInt64>())
, is_specified_range_x(params.size() >= 3)
, begin_x(is_specified_range_x ? static_cast<X>(params.at(1).safeGet<X>()) : std::numeric_limits<X>::min())
, end_x(is_specified_range_x ? static_cast<X>(params.at(2).safeGet<X>()) : std::numeric_limits<X>::max())
{
if (width < 2 || 1024 < width)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter width must be in range [2, 1024]");
if (begin_x >= end_x)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter `min_x` must be less than `max_x`");
}
String getName() const override
{
return "sparkbar";
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * /*arena*/) const override
{
X x = assert_cast<const ColumnVector<X> *>(columns[0])->getData()[row_num];
if (begin_x <= x && x <= end_x)
{
Y y = assert_cast<const ColumnVector<Y> *>(columns[1])->getData()[row_num];
this->data(place).add(x, y);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr __restrict rhs, Arena * /*arena*/) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * /*arena*/) const override
{
auto & to_column = assert_cast<ColumnString &>(to);
const auto & data = this->data(place);
render(to_column, data);
}
};
template <template <typename, typename> class AggregateFunctionTemplate, typename Data, typename ... TArgs>
IAggregateFunction * createWithUIntegerOrTimeType(const std::string & name, const IDataType & argument_type, TArgs && ... args)
{

View File

@ -1,323 +0,0 @@
#pragma once
#include <base/arithmeticOverflow.h>
#include <array>
#include <string_view>
#include <DataTypes/DataTypeString.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <base/range.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnString.h>
#include <Common/PODArray.h>
#include <IO/ReadBufferFromString.h>
#include <Common/HashTable/HashMap.h>
#include <Columns/IColumn.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
template<typename X, typename Y>
struct AggregateFunctionSparkbarData
{
/// TODO: calculate histogram instead of storing all points
using Points = HashMap<X, Y>;
Points points;
X min_x = std::numeric_limits<X>::max();
X max_x = std::numeric_limits<X>::lowest();
Y min_y = std::numeric_limits<Y>::max();
Y max_y = std::numeric_limits<Y>::lowest();
Y insert(const X & x, const Y & y)
{
if (isNaN(y) || y <= 0)
return 0;
auto [it, inserted] = points.insert({x, y});
if (!inserted)
{
if constexpr (std::is_floating_point_v<Y>)
{
it->getMapped() += y;
return it->getMapped();
}
else
{
Y res;
bool has_overfllow = common::addOverflow(it->getMapped(), y, res);
it->getMapped() = has_overfllow ? std::numeric_limits<Y>::max() : res;
}
}
return it->getMapped();
}
void add(X x, Y y)
{
auto new_y = insert(x, y);
min_x = std::min(x, min_x);
max_x = std::max(x, max_x);
min_y = std::min(y, min_y);
max_y = std::max(new_y, max_y);
}
void merge(const AggregateFunctionSparkbarData & other)
{
if (other.points.empty())
return;
for (auto & point : other.points)
{
auto new_y = insert(point.getKey(), point.getMapped());
max_y = std::max(new_y, max_y);
}
min_x = std::min(other.min_x, min_x);
max_x = std::max(other.max_x, max_x);
min_y = std::min(other.min_y, min_y);
max_y = std::max(other.max_y, max_y);
}
void serialize(WriteBuffer & buf) const
{
writeBinary(min_x, buf);
writeBinary(max_x, buf);
writeBinary(min_y, buf);
writeBinary(max_y, buf);
writeVarUInt(points.size(), buf);
for (const auto & elem : points)
{
writeBinary(elem.getKey(), buf);
writeBinary(elem.getMapped(), buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(min_x, buf);
readBinary(max_x, buf);
readBinary(min_y, buf);
readBinary(max_y, buf);
size_t size;
readVarUInt(size, buf);
X x;
Y y;
for (size_t i = 0; i < size; ++i)
{
readBinary(x, buf);
readBinary(y, buf);
insert(x, y);
}
}
};
template<typename X, typename Y>
class AggregateFunctionSparkbar final
: public IAggregateFunctionDataHelper<AggregateFunctionSparkbarData<X, Y>, AggregateFunctionSparkbar<X, Y>>
{
private:
static constexpr size_t BAR_LEVELS = 8;
const size_t width = 0;
/// Range for x specified in parameters.
const bool is_specified_range_x = false;
const X begin_x = std::numeric_limits<X>::min();
const X end_x = std::numeric_limits<X>::max();
size_t updateFrame(ColumnString::Chars & frame, Y value) const
{
static constexpr std::array<std::string_view, BAR_LEVELS + 1> bars{" ", "", "", "", "", "", "", "", ""};
const auto & bar = (isNaN(value) || value < 1 || static_cast<Y>(BAR_LEVELS) < value) ? bars[0] : bars[static_cast<UInt8>(value)];
frame.insert(bar.begin(), bar.end());
return bar.size();
}
/**
* The minimum value of y is rendered as the lowest height "",
* the maximum value of y is rendered as the highest height "", and the middle value will be rendered proportionally.
* If a bucket has no y value, it will be rendered as " ".
*/
void render(ColumnString & to_column, const AggregateFunctionSparkbarData<X, Y> & data) const
{
auto & values = to_column.getChars();
auto & offsets = to_column.getOffsets();
if (data.points.empty())
{
values.push_back('\0');
offsets.push_back(offsets.empty() ? 1 : offsets.back() + 1);
return;
}
auto from_x = is_specified_range_x ? begin_x : data.min_x;
auto to_x = is_specified_range_x ? end_x : data.max_x;
if (from_x >= to_x)
{
size_t sz = updateFrame(values, 8);
values.push_back('\0');
offsets.push_back(offsets.empty() ? sz + 1 : offsets.back() + sz + 1);
return;
}
PaddedPODArray<Y> histogram(width, 0);
PaddedPODArray<UInt64> count_histogram(width, 0); /// The number of points in each bucket
for (const auto & point : data.points)
{
if (point.getKey() < from_x || to_x < point.getKey())
continue;
X delta = to_x - from_x;
if (delta < std::numeric_limits<X>::max())
delta = delta + 1;
X value = point.getKey() - from_x;
Float64 w = histogram.size();
size_t index = std::min<size_t>(static_cast<size_t>(w / delta * value), histogram.size() - 1);
Y res;
bool has_overfllow = false;
if constexpr (std::is_floating_point_v<Y>)
res = histogram[index] + point.getMapped();
else
has_overfllow = common::addOverflow(histogram[index], point.getMapped(), res);
if (unlikely(has_overfllow))
{
/// In case of overflow, just saturate
/// Do not count new values, because we do not know how many of them were added
histogram[index] = std::numeric_limits<Y>::max();
}
else
{
histogram[index] = res;
count_histogram[index] += 1;
}
}
for (size_t i = 0; i < histogram.size(); ++i)
{
if (count_histogram[i] > 0)
histogram[i] /= count_histogram[i];
}
Y y_max = 0;
for (auto & y : histogram)
{
if (isNaN(y) || y <= 0)
continue;
y_max = std::max(y_max, y);
}
if (y_max == 0)
{
values.push_back('\0');
offsets.push_back(offsets.empty() ? 1 : offsets.back() + 1);
return;
}
/// Scale the histogram to the range [0, BAR_LEVELS]
for (auto & y : histogram)
{
if (isNaN(y) || y <= 0)
{
y = 0;
continue;
}
constexpr auto levels_num = static_cast<Y>(BAR_LEVELS - 1);
if constexpr (std::is_floating_point_v<Y>)
{
y = y / (y_max / levels_num) + 1;
}
else
{
Y scaled;
bool has_overfllow = common::mulOverflow<Y>(y, levels_num, scaled);
if (has_overfllow)
y = y / (y_max / levels_num) + 1;
else
y = scaled / y_max + 1;
}
}
size_t sz = 0;
for (const auto & y : histogram)
sz += updateFrame(values, y);
values.push_back('\0');
offsets.push_back(offsets.empty() ? sz + 1 : offsets.back() + sz + 1);
}
public:
AggregateFunctionSparkbar(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionSparkbarData<X, Y>, AggregateFunctionSparkbar>(arguments, params, std::make_shared<DataTypeString>())
, width(params.empty() ? 0 : params.at(0).safeGet<UInt64>())
, is_specified_range_x(params.size() >= 3)
, begin_x(is_specified_range_x ? static_cast<X>(params.at(1).safeGet<X>()) : std::numeric_limits<X>::min())
, end_x(is_specified_range_x ? static_cast<X>(params.at(2).safeGet<X>()) : std::numeric_limits<X>::max())
{
if (width < 2 || 1024 < width)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter width must be in range [2, 1024]");
if (begin_x >= end_x)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parameter `min_x` must be less than `max_x`");
}
String getName() const override
{
return "sparkbar";
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * /*arena*/) const override
{
X x = assert_cast<const ColumnVector<X> *>(columns[0])->getData()[row_num];
if (begin_x <= x && x <= end_x)
{
Y y = assert_cast<const ColumnVector<Y> *>(columns[1])->getData()[row_num];
this->data(place).add(x, y);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr __restrict rhs, Arena * /*arena*/) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
bool allocatesMemoryInArena() const override { return false; }
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * /*arena*/) const override
{
auto & to_column = assert_cast<ColumnString &>(to);
const auto & data = this->data(place);
render(to_column, data);
}
};
}

View File

@ -1,7 +1,15 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/AggregateFunctionStatistics.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <cmath>
namespace DB
@ -16,6 +24,454 @@ namespace ErrorCodes
namespace
{
/// This function returns true if both values are large and comparable.
/// It is used to calculate the mean value by merging two sources.
/// It means that if the sizes of both sources are large and comparable, then we must apply a special
/// formula guaranteeing more stability.
bool areComparable(UInt64 a, UInt64 b)
{
const Float64 sensitivity = 0.001;
const UInt64 threshold = 10000;
if ((a == 0) || (b == 0))
return false;
auto res = std::minmax(a, b);
return (((1 - static_cast<Float64>(res.first) / res.second) < sensitivity) && (res.first > threshold));
}
/** Statistical aggregate functions
* varSamp - sample variance
* stddevSamp - mean sample quadratic deviation
* varPop - variance
* stddevPop - standard deviation
* covarSamp - selective covariance
* covarPop - covariance
* corr - correlation
*/
/** Parallel and incremental algorithm for calculating variance.
* Source: "Updating formulae and a pairwise algorithm for computing sample variances"
* (Chan et al., Stanford University, 12.1979)
*/
template <typename T, typename Op>
class AggregateFunctionVarianceData
{
public:
void update(const IColumn & column, size_t row_num)
{
T received = assert_cast<const ColumnVector<T> &>(column).getData()[row_num];
Float64 val = static_cast<Float64>(received);
Float64 delta = val - mean;
++count;
mean += delta / count;
m2 += delta * (val - mean);
}
void mergeWith(const AggregateFunctionVarianceData & source)
{
UInt64 total_count = count + source.count;
if (total_count == 0)
return;
Float64 factor = static_cast<Float64>(count * source.count) / total_count;
Float64 delta = mean - source.mean;
if (areComparable(count, source.count))
mean = (source.count * source.mean + count * mean) / total_count;
else
mean = source.mean + delta * (static_cast<Float64>(count) / total_count);
m2 += source.m2 + delta * delta * factor;
count = total_count;
}
void serialize(WriteBuffer & buf) const
{
writeVarUInt(count, buf);
writeBinary(mean, buf);
writeBinary(m2, buf);
}
void deserialize(ReadBuffer & buf)
{
readVarUInt(count, buf);
readBinary(mean, buf);
readBinary(m2, buf);
}
void publish(IColumn & to) const
{
assert_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(m2, count));
}
private:
UInt64 count = 0;
Float64 mean = 0.0;
Float64 m2 = 0.0;
};
/** The main code for the implementation of varSamp, stddevSamp, varPop, stddevPop.
*/
template <typename T, typename Op>
class AggregateFunctionVariance final
: public IAggregateFunctionDataHelper<AggregateFunctionVarianceData<T, Op>, AggregateFunctionVariance<T, Op>>
{
public:
explicit AggregateFunctionVariance(const DataTypePtr & arg)
: IAggregateFunctionDataHelper<AggregateFunctionVarianceData<T, Op>, AggregateFunctionVariance<T, Op>>({arg}, {}, std::make_shared<DataTypeFloat64>())
{}
String getName() const override { return Op::name; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).update(*columns[0], row_num);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).mergeWith(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
this->data(place).publish(to);
}
};
/** Implementing the varSamp function.
*/
struct AggregateFunctionVarSampImpl
{
static constexpr auto name = "varSampStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
if (count < 2)
return std::numeric_limits<Float64>::infinity();
else
return m2 / (count - 1);
}
};
/** Implementing the stddevSamp function.
*/
struct AggregateFunctionStdDevSampImpl
{
static constexpr auto name = "stddevSampStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
return sqrt(AggregateFunctionVarSampImpl::apply(m2, count));
}
};
/** Implementing the varPop function.
*/
struct AggregateFunctionVarPopImpl
{
static constexpr auto name = "varPopStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
if (count == 0)
return std::numeric_limits<Float64>::infinity();
else if (count == 1)
return 0.0;
else
return m2 / count;
}
};
/** Implementing the stddevPop function.
*/
struct AggregateFunctionStdDevPopImpl
{
static constexpr auto name = "stddevPopStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
return sqrt(AggregateFunctionVarPopImpl::apply(m2, count));
}
};
/** If `compute_marginal_moments` flag is set this class provides the successor
* CovarianceData support of marginal moments for calculating the correlation.
*/
template <bool compute_marginal_moments>
class BaseCovarianceData
{
protected:
void incrementMarginalMoments(Float64, Float64) {}
void mergeWith(const BaseCovarianceData &) {}
void serialize(WriteBuffer &) const {}
void deserialize(const ReadBuffer &) {}
};
template <>
class BaseCovarianceData<true>
{
protected:
void incrementMarginalMoments(Float64 left_incr, Float64 right_incr)
{
left_m2 += left_incr;
right_m2 += right_incr;
}
void mergeWith(const BaseCovarianceData & source)
{
left_m2 += source.left_m2;
right_m2 += source.right_m2;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(left_m2, buf);
writeBinary(right_m2, buf);
}
void deserialize(ReadBuffer & buf)
{
readBinary(left_m2, buf);
readBinary(right_m2, buf);
}
Float64 left_m2 = 0.0;
Float64 right_m2 = 0.0;
};
/** Parallel and incremental algorithm for calculating covariance.
* Source: "Numerically Stable, Single-Pass, Parallel Statistics Algorithms"
* (J. Bennett et al., Sandia National Laboratories,
* 2009 IEEE International Conference on Cluster Computing)
*/
template <typename T, typename U, typename Op, bool compute_marginal_moments>
class CovarianceData : public BaseCovarianceData<compute_marginal_moments>
{
private:
using Base = BaseCovarianceData<compute_marginal_moments>;
public:
void update(const IColumn & column_left, const IColumn & column_right, size_t row_num)
{
T left_received = assert_cast<const ColumnVector<T> &>(column_left).getData()[row_num];
Float64 left_val = static_cast<Float64>(left_received);
Float64 left_delta = left_val - left_mean;
U right_received = assert_cast<const ColumnVector<U> &>(column_right).getData()[row_num];
Float64 right_val = static_cast<Float64>(right_received);
Float64 right_delta = right_val - right_mean;
Float64 old_right_mean = right_mean;
++count;
left_mean += left_delta / count;
right_mean += right_delta / count;
co_moment += (left_val - left_mean) * (right_val - old_right_mean);
/// Update the marginal moments, if any.
if (compute_marginal_moments)
{
Float64 left_incr = left_delta * (left_val - left_mean);
Float64 right_incr = right_delta * (right_val - right_mean);
Base::incrementMarginalMoments(left_incr, right_incr);
}
}
void mergeWith(const CovarianceData & source)
{
UInt64 total_count = count + source.count;
if (total_count == 0)
return;
Float64 factor = static_cast<Float64>(count * source.count) / total_count;
Float64 left_delta = left_mean - source.left_mean;
Float64 right_delta = right_mean - source.right_mean;
if (areComparable(count, source.count))
{
left_mean = (source.count * source.left_mean + count * left_mean) / total_count;
right_mean = (source.count * source.right_mean + count * right_mean) / total_count;
}
else
{
left_mean = source.left_mean + left_delta * (static_cast<Float64>(count) / total_count);
right_mean = source.right_mean + right_delta * (static_cast<Float64>(count) / total_count);
}
co_moment += source.co_moment + left_delta * right_delta * factor;
count = total_count;
/// Update the marginal moments, if any.
if (compute_marginal_moments)
{
Float64 left_incr = left_delta * left_delta * factor;
Float64 right_incr = right_delta * right_delta * factor;
Base::mergeWith(source);
Base::incrementMarginalMoments(left_incr, right_incr);
}
}
void serialize(WriteBuffer & buf) const
{
writeVarUInt(count, buf);
writeBinary(left_mean, buf);
writeBinary(right_mean, buf);
writeBinary(co_moment, buf);
Base::serialize(buf);
}
void deserialize(ReadBuffer & buf)
{
readVarUInt(count, buf);
readBinary(left_mean, buf);
readBinary(right_mean, buf);
readBinary(co_moment, buf);
Base::deserialize(buf);
}
void publish(IColumn & to) const
{
if constexpr (compute_marginal_moments)
assert_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, Base::left_m2, Base::right_m2, count));
else
assert_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, count));
}
private:
UInt64 count = 0;
Float64 left_mean = 0.0;
Float64 right_mean = 0.0;
Float64 co_moment = 0.0;
};
template <typename T, typename U, typename Op, bool compute_marginal_moments = false>
class AggregateFunctionCovariance final
: public IAggregateFunctionDataHelper<
CovarianceData<T, U, Op, compute_marginal_moments>,
AggregateFunctionCovariance<T, U, Op, compute_marginal_moments>>
{
public:
explicit AggregateFunctionCovariance(const DataTypes & args) : IAggregateFunctionDataHelper<
CovarianceData<T, U, Op, compute_marginal_moments>,
AggregateFunctionCovariance<T, U, Op, compute_marginal_moments>>(args, {}, std::make_shared<DataTypeFloat64>())
{}
String getName() const override { return Op::name; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).update(*columns[0], *columns[1], row_num);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).mergeWith(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
this->data(place).publish(to);
}
};
/** Implementing the covarSamp function.
*/
struct AggregateFunctionCovarSampImpl
{
static constexpr auto name = "covarSampStable";
static inline Float64 apply(Float64 co_moment, UInt64 count)
{
if (count < 2)
return std::numeric_limits<Float64>::infinity();
else
return co_moment / (count - 1);
}
};
/** Implementing the covarPop function.
*/
struct AggregateFunctionCovarPopImpl
{
static constexpr auto name = "covarPopStable";
static inline Float64 apply(Float64 co_moment, UInt64 count)
{
if (count == 0)
return std::numeric_limits<Float64>::infinity();
else if (count == 1)
return 0.0;
else
return co_moment / count;
}
};
/** `corr` function implementation.
*/
struct AggregateFunctionCorrImpl
{
static constexpr auto name = "corrStable";
static inline Float64 apply(Float64 co_moment, Float64 left_m2, Float64 right_m2, UInt64 count)
{
if (count < 2)
return std::numeric_limits<Float64>::infinity();
else
return co_moment / sqrt(left_m2 * right_m2);
}
};
template <typename T>
using AggregateFunctionVarSampStable = AggregateFunctionVariance<T, AggregateFunctionVarSampImpl>;
template <typename T>
using AggregateFunctionStddevSampStable = AggregateFunctionVariance<T, AggregateFunctionStdDevSampImpl>;
template <typename T>
using AggregateFunctionVarPopStable = AggregateFunctionVariance<T, AggregateFunctionVarPopImpl>;
template <typename T>
using AggregateFunctionStddevPopStable = AggregateFunctionVariance<T, AggregateFunctionStdDevPopImpl>;
template <typename T, typename U>
using AggregateFunctionCovarSampStable = AggregateFunctionCovariance<T, U, AggregateFunctionCovarSampImpl>;
template <typename T, typename U>
using AggregateFunctionCovarPopStable = AggregateFunctionCovariance<T, U, AggregateFunctionCovarPopImpl>;
template <typename T, typename U>
using AggregateFunctionCorrStable = AggregateFunctionCovariance<T, U, AggregateFunctionCorrImpl, true>;
template <template <typename> typename FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)

View File

@ -1,468 +0,0 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <cmath>
namespace DB
{
struct Settings;
namespace detail
{
/// This function returns true if both values are large and comparable.
/// It is used to calculate the mean value by merging two sources.
/// It means that if the sizes of both sources are large and comparable, then we must apply a special
/// formula guaranteeing more stability.
bool areComparable(UInt64 a, UInt64 b)
{
const Float64 sensitivity = 0.001;
const UInt64 threshold = 10000;
if ((a == 0) || (b == 0))
return false;
auto res = std::minmax(a, b);
return (((1 - static_cast<Float64>(res.first) / res.second) < sensitivity) && (res.first > threshold));
}
}
/** Statistical aggregate functions
* varSamp - sample variance
* stddevSamp - mean sample quadratic deviation
* varPop - variance
* stddevPop - standard deviation
* covarSamp - selective covariance
* covarPop - covariance
* corr - correlation
*/
/** Parallel and incremental algorithm for calculating variance.
* Source: "Updating formulae and a pairwise algorithm for computing sample variances"
* (Chan et al., Stanford University, 12.1979)
*/
template <typename T, typename Op>
class AggregateFunctionVarianceData
{
public:
void update(const IColumn & column, size_t row_num)
{
T received = assert_cast<const ColumnVector<T> &>(column).getData()[row_num];
Float64 val = static_cast<Float64>(received);
Float64 delta = val - mean;
++count;
mean += delta / count;
m2 += delta * (val - mean);
}
void mergeWith(const AggregateFunctionVarianceData & source)
{
UInt64 total_count = count + source.count;
if (total_count == 0)
return;
Float64 factor = static_cast<Float64>(count * source.count) / total_count;
Float64 delta = mean - source.mean;
if (detail::areComparable(count, source.count))
mean = (source.count * source.mean + count * mean) / total_count;
else
mean = source.mean + delta * (static_cast<Float64>(count) / total_count);
m2 += source.m2 + delta * delta * factor;
count = total_count;
}
void serialize(WriteBuffer & buf) const
{
writeVarUInt(count, buf);
writeBinary(mean, buf);
writeBinary(m2, buf);
}
void deserialize(ReadBuffer & buf)
{
readVarUInt(count, buf);
readBinary(mean, buf);
readBinary(m2, buf);
}
void publish(IColumn & to) const
{
assert_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(m2, count));
}
private:
UInt64 count = 0;
Float64 mean = 0.0;
Float64 m2 = 0.0;
};
/** The main code for the implementation of varSamp, stddevSamp, varPop, stddevPop.
*/
template <typename T, typename Op>
class AggregateFunctionVariance final
: public IAggregateFunctionDataHelper<AggregateFunctionVarianceData<T, Op>, AggregateFunctionVariance<T, Op>>
{
public:
explicit AggregateFunctionVariance(const DataTypePtr & arg)
: IAggregateFunctionDataHelper<AggregateFunctionVarianceData<T, Op>, AggregateFunctionVariance<T, Op>>({arg}, {}, std::make_shared<DataTypeFloat64>())
{}
String getName() const override { return Op::name; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).update(*columns[0], row_num);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).mergeWith(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
this->data(place).publish(to);
}
};
/** Implementing the varSamp function.
*/
struct AggregateFunctionVarSampImpl
{
static constexpr auto name = "varSampStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
if (count < 2)
return std::numeric_limits<Float64>::infinity();
else
return m2 / (count - 1);
}
};
/** Implementing the stddevSamp function.
*/
struct AggregateFunctionStdDevSampImpl
{
static constexpr auto name = "stddevSampStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
return sqrt(AggregateFunctionVarSampImpl::apply(m2, count));
}
};
/** Implementing the varPop function.
*/
struct AggregateFunctionVarPopImpl
{
static constexpr auto name = "varPopStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
if (count == 0)
return std::numeric_limits<Float64>::infinity();
else if (count == 1)
return 0.0;
else
return m2 / count;
}
};
/** Implementing the stddevPop function.
*/
struct AggregateFunctionStdDevPopImpl
{
static constexpr auto name = "stddevPopStable";
static inline Float64 apply(Float64 m2, UInt64 count)
{
return sqrt(AggregateFunctionVarPopImpl::apply(m2, count));
}
};
/** If `compute_marginal_moments` flag is set this class provides the successor
* CovarianceData support of marginal moments for calculating the correlation.
*/
template <bool compute_marginal_moments>
class BaseCovarianceData
{
protected:
void incrementMarginalMoments(Float64, Float64) {}
void mergeWith(const BaseCovarianceData &) {}
void serialize(WriteBuffer &) const {}
void deserialize(const ReadBuffer &) {}
};
template <>
class BaseCovarianceData<true>
{
protected:
void incrementMarginalMoments(Float64 left_incr, Float64 right_incr)
{
left_m2 += left_incr;
right_m2 += right_incr;
}
void mergeWith(const BaseCovarianceData & source)
{
left_m2 += source.left_m2;
right_m2 += source.right_m2;
}
void serialize(WriteBuffer & buf) const
{
writeBinary(left_m2, buf);
writeBinary(right_m2, buf);
}
void deserialize(ReadBuffer & buf)
{
readBinary(left_m2, buf);
readBinary(right_m2, buf);
}
Float64 left_m2 = 0.0;
Float64 right_m2 = 0.0;
};
/** Parallel and incremental algorithm for calculating covariance.
* Source: "Numerically Stable, Single-Pass, Parallel Statistics Algorithms"
* (J. Bennett et al., Sandia National Laboratories,
* 2009 IEEE International Conference on Cluster Computing)
*/
template <typename T, typename U, typename Op, bool compute_marginal_moments>
class CovarianceData : public BaseCovarianceData<compute_marginal_moments>
{
private:
using Base = BaseCovarianceData<compute_marginal_moments>;
public:
void update(const IColumn & column_left, const IColumn & column_right, size_t row_num)
{
T left_received = assert_cast<const ColumnVector<T> &>(column_left).getData()[row_num];
Float64 left_val = static_cast<Float64>(left_received);
Float64 left_delta = left_val - left_mean;
U right_received = assert_cast<const ColumnVector<U> &>(column_right).getData()[row_num];
Float64 right_val = static_cast<Float64>(right_received);
Float64 right_delta = right_val - right_mean;
Float64 old_right_mean = right_mean;
++count;
left_mean += left_delta / count;
right_mean += right_delta / count;
co_moment += (left_val - left_mean) * (right_val - old_right_mean);
/// Update the marginal moments, if any.
if (compute_marginal_moments)
{
Float64 left_incr = left_delta * (left_val - left_mean);
Float64 right_incr = right_delta * (right_val - right_mean);
Base::incrementMarginalMoments(left_incr, right_incr);
}
}
void mergeWith(const CovarianceData & source)
{
UInt64 total_count = count + source.count;
if (total_count == 0)
return;
Float64 factor = static_cast<Float64>(count * source.count) / total_count;
Float64 left_delta = left_mean - source.left_mean;
Float64 right_delta = right_mean - source.right_mean;
if (detail::areComparable(count, source.count))
{
left_mean = (source.count * source.left_mean + count * left_mean) / total_count;
right_mean = (source.count * source.right_mean + count * right_mean) / total_count;
}
else
{
left_mean = source.left_mean + left_delta * (static_cast<Float64>(count) / total_count);
right_mean = source.right_mean + right_delta * (static_cast<Float64>(count) / total_count);
}
co_moment += source.co_moment + left_delta * right_delta * factor;
count = total_count;
/// Update the marginal moments, if any.
if (compute_marginal_moments)
{
Float64 left_incr = left_delta * left_delta * factor;
Float64 right_incr = right_delta * right_delta * factor;
Base::mergeWith(source);
Base::incrementMarginalMoments(left_incr, right_incr);
}
}
void serialize(WriteBuffer & buf) const
{
writeVarUInt(count, buf);
writeBinary(left_mean, buf);
writeBinary(right_mean, buf);
writeBinary(co_moment, buf);
Base::serialize(buf);
}
void deserialize(ReadBuffer & buf)
{
readVarUInt(count, buf);
readBinary(left_mean, buf);
readBinary(right_mean, buf);
readBinary(co_moment, buf);
Base::deserialize(buf);
}
void publish(IColumn & to) const
{
if constexpr (compute_marginal_moments)
assert_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, Base::left_m2, Base::right_m2, count));
else
assert_cast<ColumnFloat64 &>(to).getData().push_back(Op::apply(co_moment, count));
}
private:
UInt64 count = 0;
Float64 left_mean = 0.0;
Float64 right_mean = 0.0;
Float64 co_moment = 0.0;
};
template <typename T, typename U, typename Op, bool compute_marginal_moments = false>
class AggregateFunctionCovariance final
: public IAggregateFunctionDataHelper<
CovarianceData<T, U, Op, compute_marginal_moments>,
AggregateFunctionCovariance<T, U, Op, compute_marginal_moments>>
{
public:
explicit AggregateFunctionCovariance(const DataTypes & args) : IAggregateFunctionDataHelper<
CovarianceData<T, U, Op, compute_marginal_moments>,
AggregateFunctionCovariance<T, U, Op, compute_marginal_moments>>(args, {}, std::make_shared<DataTypeFloat64>())
{}
String getName() const override { return Op::name; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).update(*columns[0], *columns[1], row_num);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).mergeWith(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
this->data(place).publish(to);
}
};
/** Implementing the covarSamp function.
*/
struct AggregateFunctionCovarSampImpl
{
static constexpr auto name = "covarSampStable";
static inline Float64 apply(Float64 co_moment, UInt64 count)
{
if (count < 2)
return std::numeric_limits<Float64>::infinity();
else
return co_moment / (count - 1);
}
};
/** Implementing the covarPop function.
*/
struct AggregateFunctionCovarPopImpl
{
static constexpr auto name = "covarPopStable";
static inline Float64 apply(Float64 co_moment, UInt64 count)
{
if (count == 0)
return std::numeric_limits<Float64>::infinity();
else if (count == 1)
return 0.0;
else
return co_moment / count;
}
};
/** `corr` function implementation.
*/
struct AggregateFunctionCorrImpl
{
static constexpr auto name = "corrStable";
static inline Float64 apply(Float64 co_moment, Float64 left_m2, Float64 right_m2, UInt64 count)
{
if (count < 2)
return std::numeric_limits<Float64>::infinity();
else
return co_moment / sqrt(left_m2 * right_m2);
}
};
template <typename T>
using AggregateFunctionVarSampStable = AggregateFunctionVariance<T, AggregateFunctionVarSampImpl>;
template <typename T>
using AggregateFunctionStddevSampStable = AggregateFunctionVariance<T, AggregateFunctionStdDevSampImpl>;
template <typename T>
using AggregateFunctionVarPopStable = AggregateFunctionVariance<T, AggregateFunctionVarPopImpl>;
template <typename T>
using AggregateFunctionStddevPopStable = AggregateFunctionVariance<T, AggregateFunctionStdDevPopImpl>;
template <typename T, typename U>
using AggregateFunctionCovarSampStable = AggregateFunctionCovariance<T, U, AggregateFunctionCovarSampImpl>;
template <typename T, typename U>
using AggregateFunctionCovarPopStable = AggregateFunctionCovariance<T, U, AggregateFunctionCovarPopImpl>;
template <typename T, typename U>
using AggregateFunctionCorrStable = AggregateFunctionCovariance<T, U, AggregateFunctionCorrImpl, true>;
}

View File

@ -1,7 +1,8 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionSumCount.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <DataTypes/DataTypeTuple.h>
#include <AggregateFunctions/AggregateFunctionAvg.h>
namespace DB
@ -16,6 +17,59 @@ namespace ErrorCodes
namespace
{
template <typename T>
class AggregateFunctionSumCount final : public AggregateFunctionAvg<T>
{
public:
using Base = AggregateFunctionAvg<T>;
explicit AggregateFunctionSumCount(const DataTypes & argument_types_, UInt32 num_scale_ = 0)
: Base(argument_types_, createResultType(num_scale_), num_scale_)
{}
static DataTypePtr createResultType(UInt32 num_scale_)
{
auto second_elem = std::make_shared<DataTypeUInt64>();
return std::make_shared<DataTypeTuple>(DataTypes{getReturnTypeFirstElement(num_scale_), std::move(second_elem)});
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const final
{
assert_cast<ColumnVectorOrDecimal<AvgFieldType<T>> &>((assert_cast<ColumnTuple &>(to)).getColumn(0)).getData().push_back(
this->data(place).numerator);
assert_cast<ColumnUInt64 &>((assert_cast<ColumnTuple &>(to)).getColumn(1)).getData().push_back(
this->data(place).denominator);
}
String getName() const final { return "sumCount"; }
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override
{
return false;
}
#endif
private:
static auto getReturnTypeFirstElement(UInt32 num_scale_)
{
using FieldType = AvgFieldType<T>;
if constexpr (!is_decimal<T>)
return std::make_shared<DataTypeNumber<FieldType>>();
else
{
using DataType = DataTypeDecimal<FieldType>;
return std::make_shared<DataType>(DataType::maxPrecision(), num_scale_);
}
}
};
bool allowType(const DataTypePtr& type) noexcept
{
const WhichDataType t(type);

View File

@ -1,61 +0,0 @@
#pragma once
#include <type_traits>
#include <DataTypes/DataTypeTuple.h>
#include <AggregateFunctions/AggregateFunctionAvg.h>
namespace DB
{
template <typename T>
class AggregateFunctionSumCount final : public AggregateFunctionAvg<T>
{
public:
using Base = AggregateFunctionAvg<T>;
explicit AggregateFunctionSumCount(const DataTypes & argument_types_, UInt32 num_scale_ = 0)
: Base(argument_types_, createResultType(num_scale_), num_scale_)
{}
static DataTypePtr createResultType(UInt32 num_scale_)
{
auto second_elem = std::make_shared<DataTypeUInt64>();
return std::make_shared<DataTypeTuple>(DataTypes{getReturnTypeFirstElement(num_scale_), std::move(second_elem)});
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const final
{
assert_cast<ColumnVectorOrDecimal<AvgFieldType<T>> &>((assert_cast<ColumnTuple &>(to)).getColumn(0)).getData().push_back(
this->data(place).numerator);
assert_cast<ColumnUInt64 &>((assert_cast<ColumnTuple &>(to)).getColumn(1)).getData().push_back(
this->data(place).denominator);
}
String getName() const final { return "sumCount"; }
#if USE_EMBEDDED_COMPILER
bool isCompilable() const override
{
return false;
}
#endif
private:
static auto getReturnTypeFirstElement(UInt32 num_scale_)
{
using FieldType = AvgFieldType<T>;
if constexpr (!is_decimal<T>)
return std::make_shared<DataTypeNumber<FieldType>>();
else
{
using DataType = DataTypeDecimal<FieldType>;
return std::make_shared<DataType>(DataType::maxPrecision(), num_scale_);
}
}
};
}

View File

@ -1,5 +1,4 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionTopK.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <Common/FieldVisitorConvertToNumber.h>
@ -7,6 +6,20 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeIPv4andIPv6.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadHelpersArena.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnArray.h>
#include <Common/SpaceSaving.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
@ -25,6 +38,229 @@ namespace ErrorCodes
namespace
{
static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF;
template <typename T>
struct AggregateFunctionTopKData
{
using Set = SpaceSaving<T, HashCRC32<T>>;
Set value;
};
template <typename T, bool is_weighted>
class AggregateFunctionTopK
: public IAggregateFunctionDataHelper<AggregateFunctionTopKData<T>, AggregateFunctionTopK<T, is_weighted>>
{
protected:
using State = AggregateFunctionTopKData<T>;
UInt64 threshold;
UInt64 reserved;
public:
AggregateFunctionTopK(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionTopKData<T>, AggregateFunctionTopK<T, is_weighted>>(argument_types_, params, createResultType(argument_types_))
, threshold(threshold_), reserved(load_factor * threshold)
{}
AggregateFunctionTopK(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params, const DataTypePtr & result_type_)
: IAggregateFunctionDataHelper<AggregateFunctionTopKData<T>, AggregateFunctionTopK<T, is_weighted>>(argument_types_, params, result_type_)
, threshold(threshold_), reserved(load_factor * threshold)
{}
String getName() const override { return is_weighted ? "topKWeighted" : "topK"; }
static DataTypePtr createResultType(const DataTypes & argument_types_)
{
return std::make_shared<DataTypeArray>(argument_types_[0]);
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
if constexpr (is_weighted)
set.insert(assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num], columns[1]->getUInt(row_num));
else
set.insert(assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num]);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
set.merge(this->data(rhs).value);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).value.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
auto & set = this->data(place).value;
set.resize(reserved);
set.read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
const typename State::Set & set = this->data(place).value;
auto result_vec = set.topK(threshold);
size_t size = result_vec.size();
offsets_to.push_back(offsets_to.back() + size);
typename ColumnVector<T>::Container & data_to = assert_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(old_size + size);
size_t i = 0;
for (auto it = result_vec.begin(); it != result_vec.end(); ++it, ++i)
data_to[old_size + i] = it->key;
}
};
/// Generic implementation, it uses serialized representation as object descriptor.
struct AggregateFunctionTopKGenericData
{
using Set = SpaceSaving<StringRef, StringRefHash>;
Set value;
};
/** Template parameter with true value should be used for columns that store their elements in memory continuously.
* For such columns topK() can be implemented more efficiently (especially for small numeric arrays).
*/
template <bool is_plain_column, bool is_weighted>
class AggregateFunctionTopKGeneric
: public IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData, AggregateFunctionTopKGeneric<is_plain_column, is_weighted>>
{
private:
using State = AggregateFunctionTopKGenericData;
UInt64 threshold;
UInt64 reserved;
static void deserializeAndInsert(StringRef str, IColumn & data_to);
public:
AggregateFunctionTopKGeneric(
UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData, AggregateFunctionTopKGeneric<is_plain_column, is_weighted>>(argument_types_, params, createResultType(argument_types_))
, threshold(threshold_), reserved(load_factor * threshold) {}
String getName() const override { return is_weighted ? "topKWeighted" : "topK"; }
static DataTypePtr createResultType(const DataTypes & argument_types_)
{
return std::make_shared<DataTypeArray>(argument_types_[0]);
}
bool allocatesMemoryInArena() const override
{
return true;
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).value.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
auto & set = this->data(place).value;
set.clear();
// Specialized here because there's no deserialiser for StringRef
size_t size = 0;
readVarUInt(size, buf);
if (unlikely(size > TOP_K_MAX_SIZE))
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"Too large size ({}) for aggregate function '{}' state (maximum is {})",
size,
getName(),
TOP_K_MAX_SIZE);
set.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto ref = readStringBinaryInto(*arena, buf);
UInt64 count;
UInt64 error;
readVarUInt(count, buf);
readVarUInt(error, buf);
set.insert(ref, count, error);
arena->rollback(ref.size);
}
set.readAlphaMap(buf);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
if constexpr (is_plain_column)
{
if constexpr (is_weighted)
set.insert(columns[0]->getDataAt(row_num), columns[1]->getUInt(row_num));
else
set.insert(columns[0]->getDataAt(row_num));
}
else
{
const char * begin = nullptr;
StringRef str_serialized = columns[0]->serializeValueIntoArena(row_num, *arena, begin);
if constexpr (is_weighted)
set.insert(str_serialized, columns[1]->getUInt(row_num));
else
set.insert(str_serialized);
arena->rollback(str_serialized.size);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
set.merge(this->data(rhs).value);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
IColumn & data_to = arr_to.getData();
auto result_vec = this->data(place).value.topK(threshold);
offsets_to.push_back(offsets_to.back() + result_vec.size());
for (auto & elem : result_vec)
{
if constexpr (is_plain_column)
data_to.insertData(elem.key.data, elem.key.size);
else
data_to.deserializeAndInsertFromArena(elem.key.data);
}
}
};
/// Substitute return type for Date and DateTime
template <bool is_weighted>
class AggregateFunctionTopKDate : public AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted>

View File

@ -1,250 +0,0 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadHelpersArena.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnArray.h>
#include <Common/SpaceSaving.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
struct Settings;
static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF;
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
}
template <typename T>
struct AggregateFunctionTopKData
{
using Set = SpaceSaving<T, HashCRC32<T>>;
Set value;
};
template <typename T, bool is_weighted>
class AggregateFunctionTopK
: public IAggregateFunctionDataHelper<AggregateFunctionTopKData<T>, AggregateFunctionTopK<T, is_weighted>>
{
protected:
using State = AggregateFunctionTopKData<T>;
UInt64 threshold;
UInt64 reserved;
public:
AggregateFunctionTopK(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionTopKData<T>, AggregateFunctionTopK<T, is_weighted>>(argument_types_, params, createResultType(argument_types_))
, threshold(threshold_), reserved(load_factor * threshold)
{}
AggregateFunctionTopK(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params, const DataTypePtr & result_type_)
: IAggregateFunctionDataHelper<AggregateFunctionTopKData<T>, AggregateFunctionTopK<T, is_weighted>>(argument_types_, params, result_type_)
, threshold(threshold_), reserved(load_factor * threshold)
{}
String getName() const override { return is_weighted ? "topKWeighted" : "topK"; }
static DataTypePtr createResultType(const DataTypes & argument_types_)
{
return std::make_shared<DataTypeArray>(argument_types_[0]);
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
if constexpr (is_weighted)
set.insert(assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num], columns[1]->getUInt(row_num));
else
set.insert(assert_cast<const ColumnVector<T> &>(*columns[0]).getData()[row_num]);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
set.merge(this->data(rhs).value);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).value.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
auto & set = this->data(place).value;
set.resize(reserved);
set.read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
const typename State::Set & set = this->data(place).value;
auto result_vec = set.topK(threshold);
size_t size = result_vec.size();
offsets_to.push_back(offsets_to.back() + size);
typename ColumnVector<T>::Container & data_to = assert_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(old_size + size);
size_t i = 0;
for (auto it = result_vec.begin(); it != result_vec.end(); ++it, ++i)
data_to[old_size + i] = it->key;
}
};
/// Generic implementation, it uses serialized representation as object descriptor.
struct AggregateFunctionTopKGenericData
{
using Set = SpaceSaving<StringRef, StringRefHash>;
Set value;
};
/** Template parameter with true value should be used for columns that store their elements in memory continuously.
* For such columns topK() can be implemented more efficiently (especially for small numeric arrays).
*/
template <bool is_plain_column, bool is_weighted>
class AggregateFunctionTopKGeneric
: public IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData, AggregateFunctionTopKGeneric<is_plain_column, is_weighted>>
{
private:
using State = AggregateFunctionTopKGenericData;
UInt64 threshold;
UInt64 reserved;
static void deserializeAndInsert(StringRef str, IColumn & data_to);
public:
AggregateFunctionTopKGeneric(
UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData, AggregateFunctionTopKGeneric<is_plain_column, is_weighted>>(argument_types_, params, createResultType(argument_types_))
, threshold(threshold_), reserved(load_factor * threshold) {}
String getName() const override { return is_weighted ? "topKWeighted" : "topK"; }
static DataTypePtr createResultType(const DataTypes & argument_types_)
{
return std::make_shared<DataTypeArray>(argument_types_[0]);
}
bool allocatesMemoryInArena() const override
{
return true;
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).value.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
auto & set = this->data(place).value;
set.clear();
// Specialized here because there's no deserialiser for StringRef
size_t size = 0;
readVarUInt(size, buf);
if (unlikely(size > TOP_K_MAX_SIZE))
throw Exception(
ErrorCodes::ARGUMENT_OUT_OF_BOUND,
"Too large size ({}) for aggregate function '{}' state (maximum is {})",
size,
getName(),
TOP_K_MAX_SIZE);
set.resize(size);
for (size_t i = 0; i < size; ++i)
{
auto ref = readStringBinaryInto(*arena, buf);
UInt64 count;
UInt64 error;
readVarUInt(count, buf);
readVarUInt(error, buf);
set.insert(ref, count, error);
arena->rollback(ref.size);
}
set.readAlphaMap(buf);
}
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
if constexpr (is_plain_column)
{
if constexpr (is_weighted)
set.insert(columns[0]->getDataAt(row_num), columns[1]->getUInt(row_num));
else
set.insert(columns[0]->getDataAt(row_num));
}
else
{
const char * begin = nullptr;
StringRef str_serialized = columns[0]->serializeValueIntoArena(row_num, *arena, begin);
if constexpr (is_weighted)
set.insert(str_serialized, columns[1]->getUInt(row_num));
else
set.insert(str_serialized);
arena->rollback(str_serialized.size);
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto & set = this->data(place).value;
if (set.capacity() != reserved)
set.resize(reserved);
set.merge(this->data(rhs).value);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
ColumnArray & arr_to = assert_cast<ColumnArray &>(to);
ColumnArray::Offsets & offsets_to = arr_to.getOffsets();
IColumn & data_to = arr_to.getData();
auto result_vec = this->data(place).value.topK(threshold);
offsets_to.push_back(offsets_to.back() + result_vec.size());
for (auto & elem : result_vec)
{
if constexpr (is_plain_column)
data_to.insertData(elem.key.data, elem.key.size);
else
data_to.deserializeAndInsertFromArena(elem.key.data);
}
}
};
}

View File

@ -1,5 +1,3 @@
#include <AggregateFunctions/AggregateFunctionUniqCombined.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
@ -10,6 +8,23 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeIPv4andIPv6.h>
#include <base/bit_cast.h>
#include <Common/CombinedCardinalityEstimator.h>
#include <Common/SipHash.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/UniqCombinedBiasData.h>
#include <AggregateFunctions/UniqVariadicHash.h>
#include <Columns/ColumnVector.h>
#include <functional>
@ -26,119 +41,332 @@ namespace ErrorCodes
namespace
{
template <UInt8 K, typename HashValueType>
struct WithK
/** Hash function for uniqCombined/uniqCombined64 (based on Ret).
*/
template <typename T, typename Ret>
struct AggregateFunctionUniqCombinedTraits
{
static Ret hash(T x)
{
template <typename T>
using AggregateFunction = AggregateFunctionUniqCombined<T, K, HashValueType>;
template <bool is_exact, bool argument_is_tuple>
using AggregateFunctionVariadic = AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>;
};
template <UInt8 K, typename HashValueType>
AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types, const Array & params)
{
/// We use exact hash function if the arguments are not contiguous in memory, because only exact hash function has support for this case.
bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types);
if (argument_types.size() == 1)
{
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res(createWithNumericType<WithK<K, HashValueType>::template AggregateFunction>(*argument_types[0], argument_types, params));
WhichDataType which(argument_type);
if (res)
return res;
else if (which.isDate())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate::FieldType>>(argument_types, params);
else if (which.isDate32())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate32::FieldType>>(argument_types, params);
else if (which.isDateTime())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDateTime::FieldType>>(argument_types, params);
else if (which.isStringOrFixedString())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<String>>(argument_types, params);
else if (which.isUUID())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeUUID::FieldType>>(argument_types, params);
else if (which.isIPv4())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeIPv4::FieldType>>(argument_types, params);
else if (which.isIPv6())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeIPv6::FieldType>>(argument_types, params);
else if (which.isTuple())
{
if (use_exact_hash_function)
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<true, true>>(argument_types, params);
else
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<false, true>>(argument_types, params);
}
}
/// "Variadic" method also works as a fallback generic case for a single argument.
if (use_exact_hash_function)
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<true, false>>(argument_types, params);
if constexpr (sizeof(T) > sizeof(UInt64))
return static_cast<Ret>(DefaultHash64<T>(x));
else
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<false, false>>(argument_types, params);
return static_cast<Ret>(intHash64(x));
}
};
template <UInt8 K>
AggregateFunctionPtr createAggregateFunctionWithHashType(bool use_64_bit_hash, const DataTypes & argument_types, const Array & params)
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<UInt128, Ret>
{
static Ret hash(UInt128 x)
{
if (use_64_bit_hash)
return createAggregateFunctionWithK<K, UInt64>(argument_types, params);
return static_cast<Ret>(sipHash64(x));
}
};
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<Float32, Ret>
{
static Ret hash(Float32 x)
{
UInt64 res = bit_cast<UInt64>(x);
return static_cast<Ret>(intHash64(res));
}
};
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<Float64, Ret>
{
static Ret hash(Float64 x)
{
UInt64 res = bit_cast<UInt64>(x);
return static_cast<Ret>(intHash64(res));
}
};
// Unlike HashTableGrower always grows to power of 2.
struct UniqCombinedHashTableGrower : public HashTableGrowerWithPrecalculation<>
{
void increaseSize() { increaseSizeDegree(1); }
};
template <typename Key, UInt8 K>
struct AggregateFunctionUniqCombinedDataWithKey
{
// TODO(ilezhankin): pre-generate values for |UniqCombinedBiasData|,
// at the moment gen-bias-data.py script doesn't work.
// We want to migrate from |HashSet| to |HyperLogLogCounter| when the sizes in memory become almost equal.
// The size per element in |HashSet| is sizeof(Key)*2 bytes, and the overall size of |HyperLogLogCounter| is 2^K * 6 bits.
// For Key=UInt32 we can calculate: 2^X * 4 * 2 ≤ 2^(K-3) * 6 ⇒ X ≤ K-4.
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, UniqCombinedHashTableGrower>, 16, K - 5 + (sizeof(Key) == sizeof(UInt32)), K, TrivialHash, Key>;
Set set;
};
template <typename Key>
struct AggregateFunctionUniqCombinedDataWithKey<Key, 17>
{
using Set = CombinedCardinalityEstimator<Key,
HashSet<Key, TrivialHash, UniqCombinedHashTableGrower>,
16,
12 + (sizeof(Key) == sizeof(UInt32)),
17,
TrivialHash,
Key,
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
HyperLogLogMode::FullFeatured>;
Set set;
};
template <typename T, UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey<HashValueType, K>
{
};
/// For String keys, 64 bit hash is always used (both for uniqCombined and uniqCombined64),
/// because of backwards compatibility (64 bit hash was already used for uniqCombined).
template <UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData<String, K, HashValueType> : public AggregateFunctionUniqCombinedDataWithKey<UInt64 /*always*/, K>
{
};
template <UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData<IPv6, K, HashValueType> : public AggregateFunctionUniqCombinedDataWithKey<UInt64 /*always*/, K>
{
};
template <typename T, UInt8 K, typename HashValueType>
class AggregateFunctionUniqCombined final
: public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K, HashValueType>, AggregateFunctionUniqCombined<T, K, HashValueType>>
{
public:
AggregateFunctionUniqCombined(const DataTypes & argument_types_, const Array & params_)
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K, HashValueType>, AggregateFunctionUniqCombined<T, K, HashValueType>>(argument_types_, params_, std::make_shared<DataTypeUInt64>())
{}
String getName() const override
{
if constexpr (std::is_same_v<HashValueType, UInt64>)
return "uniqCombined64";
else
return createAggregateFunctionWithK<K, UInt32>(argument_types, params);
return "uniqCombined";
}
AggregateFunctionPtr createAggregateFunctionUniqCombined(bool use_64_bit_hash,
const std::string & name, const DataTypes & argument_types, const Array & params)
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
/// log2 of the number of cells in HyperLogLog.
/// Reasonable default value, selected to be comparable in quality with "uniq" aggregate function.
UInt8 precision = 17;
if (!params.empty())
if constexpr (std::is_same_v<T, String> || std::is_same_v<T, IPv6>)
{
if (params.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires one parameter or less.",
name);
UInt64 precision_param = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);
// This range is hardcoded below
if (precision_param > 20 || precision_param < 12)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Parameter for aggregate function {} is out of range: [12, 20].",
name);
precision = precision_param;
StringRef value = columns[0]->getDataAt(row_num);
this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
}
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Incorrect number of arguments for aggregate function {}", name);
switch (precision) // NOLINT(bugprone-switch-missing-default-case)
else
{
case 12:
return createAggregateFunctionWithHashType<12>(use_64_bit_hash, argument_types, params);
case 13:
return createAggregateFunctionWithHashType<13>(use_64_bit_hash, argument_types, params);
case 14:
return createAggregateFunctionWithHashType<14>(use_64_bit_hash, argument_types, params);
case 15:
return createAggregateFunctionWithHashType<15>(use_64_bit_hash, argument_types, params);
case 16:
return createAggregateFunctionWithHashType<16>(use_64_bit_hash, argument_types, params);
case 17:
return createAggregateFunctionWithHashType<17>(use_64_bit_hash, argument_types, params);
case 18:
return createAggregateFunctionWithHashType<18>(use_64_bit_hash, argument_types, params);
case 19:
return createAggregateFunctionWithHashType<19>(use_64_bit_hash, argument_types, params);
case 20:
return createAggregateFunctionWithHashType<20>(use_64_bit_hash, argument_types, params);
const auto & value = assert_cast<const ColumnVector<T> &>(*columns[0]).getElement(row_num);
this->data(place).set.insert(AggregateFunctionUniqCombinedTraits<T, HashValueType>::hash(value));
}
UNREACHABLE();
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).set.merge(this->data(rhs).set);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).set.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).set.read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}
};
/** For multiple arguments. To compute, hashes them.
* You can pass multiple arguments as is; You can also pass one argument - a tuple.
* But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples.
*/
template <bool is_exact, bool argument_is_tuple, UInt8 K, typename HashValueType>
class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>>
{
private:
size_t num_args = 0;
public:
explicit AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>>(arguments, params, std::make_shared<DataTypeUInt64>())
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
else
num_args = arguments.size();
}
String getName() const override
{
if constexpr (std::is_same_v<HashValueType, UInt64>)
return "uniqCombined64";
else
return "uniqCombined";
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).set.insert(typename AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>::Set::value_type(
UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num)));
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).set.merge(this->data(rhs).set);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).set.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).set.read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}
};
template <UInt8 K, typename HashValueType>
struct WithK
{
template <typename T>
using AggregateFunction = AggregateFunctionUniqCombined<T, K, HashValueType>;
template <bool is_exact, bool argument_is_tuple>
using AggregateFunctionVariadic = AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>;
};
template <UInt8 K, typename HashValueType>
AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types, const Array & params)
{
/// We use exact hash function if the arguments are not contiguous in memory, because only exact hash function has support for this case.
bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types);
if (argument_types.size() == 1)
{
const IDataType & argument_type = *argument_types[0];
AggregateFunctionPtr res(createWithNumericType<WithK<K, HashValueType>::template AggregateFunction>(*argument_types[0], argument_types, params));
WhichDataType which(argument_type);
if (res)
return res;
else if (which.isDate())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate::FieldType>>(argument_types, params);
else if (which.isDate32())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDate32::FieldType>>(argument_types, params);
else if (which.isDateTime())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeDateTime::FieldType>>(argument_types, params);
else if (which.isStringOrFixedString())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<String>>(argument_types, params);
else if (which.isUUID())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeUUID::FieldType>>(argument_types, params);
else if (which.isIPv4())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeIPv4::FieldType>>(argument_types, params);
else if (which.isIPv6())
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunction<DataTypeIPv6::FieldType>>(argument_types, params);
else if (which.isTuple())
{
if (use_exact_hash_function)
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<true, true>>(argument_types, params);
else
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<false, true>>(argument_types, params);
}
}
/// "Variadic" method also works as a fallback generic case for a single argument.
if (use_exact_hash_function)
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<true, false>>(argument_types, params);
else
return std::make_shared<typename WithK<K, HashValueType>::template AggregateFunctionVariadic<false, false>>(argument_types, params);
}
template <UInt8 K>
AggregateFunctionPtr createAggregateFunctionWithHashType(bool use_64_bit_hash, const DataTypes & argument_types, const Array & params)
{
if (use_64_bit_hash)
return createAggregateFunctionWithK<K, UInt64>(argument_types, params);
else
return createAggregateFunctionWithK<K, UInt32>(argument_types, params);
}
AggregateFunctionPtr createAggregateFunctionUniqCombined(bool use_64_bit_hash,
const std::string & name, const DataTypes & argument_types, const Array & params)
{
/// log2 of the number of cells in HyperLogLog.
/// Reasonable default value, selected to be comparable in quality with "uniq" aggregate function.
UInt8 precision = 17;
if (!params.empty())
{
if (params.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires one parameter or less.",
name);
UInt64 precision_param = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);
// This range is hardcoded below
if (precision_param > 20 || precision_param < 12)
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Parameter for aggregate function {} is out of range: [12, 20].",
name);
precision = precision_param;
}
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Incorrect number of arguments for aggregate function {}", name);
switch (precision) // NOLINT(bugprone-switch-missing-default-case)
{
case 12:
return createAggregateFunctionWithHashType<12>(use_64_bit_hash, argument_types, params);
case 13:
return createAggregateFunctionWithHashType<13>(use_64_bit_hash, argument_types, params);
case 14:
return createAggregateFunctionWithHashType<14>(use_64_bit_hash, argument_types, params);
case 15:
return createAggregateFunctionWithHashType<15>(use_64_bit_hash, argument_types, params);
case 16:
return createAggregateFunctionWithHashType<16>(use_64_bit_hash, argument_types, params);
case 17:
return createAggregateFunctionWithHashType<17>(use_64_bit_hash, argument_types, params);
case 18:
return createAggregateFunctionWithHashType<18>(use_64_bit_hash, argument_types, params);
case 19:
return createAggregateFunctionWithHashType<19>(use_64_bit_hash, argument_types, params);
case 20:
return createAggregateFunctionWithHashType<20>(use_64_bit_hash, argument_types, params);
}
UNREACHABLE();
}
}
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)

View File

@ -1,240 +0,0 @@
#pragma once
#include <base/bit_cast.h>
#include <Common/CombinedCardinalityEstimator.h>
#include <Common/SipHash.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/UniqCombinedBiasData.h>
#include <AggregateFunctions/UniqVariadicHash.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
namespace DB
{
struct Settings;
namespace detail
{
/** Hash function for uniqCombined/uniqCombined64 (based on Ret).
*/
template <typename T, typename Ret>
struct AggregateFunctionUniqCombinedTraits
{
static Ret hash(T x)
{
if constexpr (sizeof(T) > sizeof(UInt64))
return static_cast<Ret>(DefaultHash64<T>(x));
else
return static_cast<Ret>(intHash64(x));
}
};
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<UInt128, Ret>
{
static Ret hash(UInt128 x)
{
return static_cast<Ret>(sipHash64(x));
}
};
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<Float32, Ret>
{
static Ret hash(Float32 x)
{
UInt64 res = bit_cast<UInt64>(x);
return static_cast<Ret>(intHash64(res));
}
};
template <typename Ret>
struct AggregateFunctionUniqCombinedTraits<Float64, Ret>
{
static Ret hash(Float64 x)
{
UInt64 res = bit_cast<UInt64>(x);
return static_cast<Ret>(intHash64(res));
}
};
}
// Unlike HashTableGrower always grows to power of 2.
struct UniqCombinedHashTableGrower : public HashTableGrowerWithPrecalculation<>
{
void increaseSize() { increaseSizeDegree(1); }
};
template <typename Key, UInt8 K>
struct AggregateFunctionUniqCombinedDataWithKey
{
// TODO(ilezhankin): pre-generate values for |UniqCombinedBiasData|,
// at the moment gen-bias-data.py script doesn't work.
// We want to migrate from |HashSet| to |HyperLogLogCounter| when the sizes in memory become almost equal.
// The size per element in |HashSet| is sizeof(Key)*2 bytes, and the overall size of |HyperLogLogCounter| is 2^K * 6 bits.
// For Key=UInt32 we can calculate: 2^X * 4 * 2 ≤ 2^(K-3) * 6 ⇒ X ≤ K-4.
using Set = CombinedCardinalityEstimator<Key, HashSet<Key, TrivialHash, UniqCombinedHashTableGrower>, 16, K - 5 + (sizeof(Key) == sizeof(UInt32)), K, TrivialHash, Key>;
Set set;
};
template <typename Key>
struct AggregateFunctionUniqCombinedDataWithKey<Key, 17>
{
using Set = CombinedCardinalityEstimator<Key,
HashSet<Key, TrivialHash, UniqCombinedHashTableGrower>,
16,
12 + (sizeof(Key) == sizeof(UInt32)),
17,
TrivialHash,
Key,
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
HyperLogLogMode::FullFeatured>;
Set set;
};
template <typename T, UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey<HashValueType, K>
{
};
/// For String keys, 64 bit hash is always used (both for uniqCombined and uniqCombined64),
/// because of backwards compatibility (64 bit hash was already used for uniqCombined).
template <UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData<String, K, HashValueType> : public AggregateFunctionUniqCombinedDataWithKey<UInt64 /*always*/, K>
{
};
template <UInt8 K, typename HashValueType>
struct AggregateFunctionUniqCombinedData<IPv6, K, HashValueType> : public AggregateFunctionUniqCombinedDataWithKey<UInt64 /*always*/, K>
{
};
template <typename T, UInt8 K, typename HashValueType>
class AggregateFunctionUniqCombined final
: public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K, HashValueType>, AggregateFunctionUniqCombined<T, K, HashValueType>>
{
public:
AggregateFunctionUniqCombined(const DataTypes & argument_types_, const Array & params_)
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<T, K, HashValueType>, AggregateFunctionUniqCombined<T, K, HashValueType>>(argument_types_, params_, std::make_shared<DataTypeUInt64>())
{}
String getName() const override
{
if constexpr (std::is_same_v<HashValueType, UInt64>)
return "uniqCombined64";
else
return "uniqCombined";
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
if constexpr (std::is_same_v<T, String> || std::is_same_v<T, IPv6>)
{
StringRef value = columns[0]->getDataAt(row_num);
this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
}
else
{
const auto & value = assert_cast<const ColumnVector<T> &>(*columns[0]).getElement(row_num);
this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits<T, HashValueType>::hash(value));
}
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).set.merge(this->data(rhs).set);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).set.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).set.read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}
};
/** For multiple arguments. To compute, hashes them.
* You can pass multiple arguments as is; You can also pass one argument - a tuple.
* But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples.
*/
template <bool is_exact, bool argument_is_tuple, UInt8 K, typename HashValueType>
class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>>
{
private:
size_t num_args = 0;
public:
explicit AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>,
AggregateFunctionUniqCombinedVariadic<is_exact, argument_is_tuple, K, HashValueType>>(arguments, params, std::make_shared<DataTypeUInt64>())
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
else
num_args = arguments.size();
}
String getName() const override
{
if constexpr (std::is_same_v<HashValueType, UInt64>)
return "uniqCombined64";
else
return "uniqCombined";
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).set.insert(typename AggregateFunctionUniqCombinedData<UInt64, K, HashValueType>::Set::value_type(
UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num)));
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).set.merge(this->data(rhs).set);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).set.write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).set.read(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}
};
}

View File

@ -1,12 +1,24 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/AggregateFunctionUniqUpTo.h>
#include <Common/FieldVisitorConvertToNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/UniqVariadicHash.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
@ -24,6 +36,258 @@ namespace ErrorCodes
namespace
{
/** Counts the number of unique values up to no more than specified in the parameter.
*
* Example: uniqUpTo(3)(UserID)
* - will count the number of unique visitors, return 1, 2, 3 or 4 if visitors > = 4.
*
* For strings, a non-cryptographic hash function is used, due to which the calculation may be a bit inaccurate.
*/
template <typename T>
struct AggregateFunctionUniqUpToData
{
/** If count == threshold + 1 - this means that it is "overflowed" (values greater than threshold).
* In this case (for example, after calling the merge function), the `data` array does not necessarily contain the initialized values
* - example: combine a state in which there are few values, with another state that has overflowed;
* then set count to `threshold + 1`, and values from another state are not copied.
*/
UInt8 count = 0;
char data_ptr[0];
T load(size_t i) const
{
return unalignedLoad<T>(data_ptr + i * sizeof(T));
}
void store(size_t i, const T & x)
{
unalignedStore<T>(data_ptr + i * sizeof(T), x);
}
size_t size() const
{
return count;
}
/// threshold - for how many elements there is room in a `data`.
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE insert(T x, UInt8 threshold)
{
/// The state is already full - nothing needs to be done.
if (count > threshold)
return;
/// Linear search for the matching element.
for (size_t i = 0; i < count; ++i)
if (load(i) == x)
return;
/// Did not find the matching element. If there is room for one more element, insert it.
if (count < threshold)
store(count, x);
/// After increasing count, the state may be overflowed.
++count;
}
void merge(const AggregateFunctionUniqUpToData<T> & rhs, UInt8 threshold)
{
if (count > threshold)
return;
if (rhs.count > threshold)
{
/// If `rhs` is overflowed, then set `count` too also overflowed for the current state.
count = rhs.count;
return;
}
for (size_t i = 0; i < rhs.count; ++i)
insert(rhs.load(i), threshold);
}
void write(WriteBuffer & wb, UInt8 threshold) const
{
writeBinary(count, wb);
/// Write values only if the state is not overflowed. Otherwise, they are not needed, and only the fact that the state is overflowed is important.
if (count <= threshold)
wb.write(data_ptr, count * sizeof(T));
}
void read(ReadBuffer & rb, UInt8 threshold)
{
readBinary(count, rb);
if (count <= threshold)
rb.readStrict(data_ptr, count * sizeof(T));
}
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
insert(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], threshold);
}
};
/// For strings, their hashes are remembered.
template <>
struct AggregateFunctionUniqUpToData<String> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
/// Keep in mind that calculations are approximate.
StringRef value = column.getDataAt(row_num);
insert(CityHash_v1_0_2::CityHash64(value.data, value.size), threshold);
}
};
template <>
struct AggregateFunctionUniqUpToData<UInt128> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
UInt128 value = assert_cast<const ColumnVector<UInt128> &>(column).getData()[row_num];
insert(sipHash64(value), threshold);
}
};
template <>
struct AggregateFunctionUniqUpToData<UInt256> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
UInt256 value = assert_cast<const ColumnVector<UInt256> &>(column).getData()[row_num];
insert(sipHash64(value), threshold);
}
};
template <>
struct AggregateFunctionUniqUpToData<Int256> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
Int256 value = assert_cast<const ColumnVector<Int256> &>(column).getData()[row_num];
insert(sipHash64(value), threshold);
}
};
template <typename T>
class AggregateFunctionUniqUpTo final : public IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<T>, AggregateFunctionUniqUpTo<T>>
{
private:
UInt8 threshold;
public:
AggregateFunctionUniqUpTo(UInt8 threshold_, const DataTypes & argument_types_, const Array & params_)
: IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<T>, AggregateFunctionUniqUpTo<T>>(argument_types_, params_, std::make_shared<DataTypeUInt64>())
, threshold(threshold_)
{
}
size_t sizeOfData() const override
{
return sizeof(AggregateFunctionUniqUpToData<T>) + sizeof(T) * threshold;
}
String getName() const override { return "uniqUpTo"; }
bool allocatesMemoryInArena() const override { return false; }
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).add(*columns[0], row_num, threshold);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), threshold);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf, threshold);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf, threshold);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size());
}
};
/** For multiple arguments. To compute, hashes them.
* You can pass multiple arguments as is; You can also pass one argument - a tuple.
* But (for the possibility of effective implementation), you can not pass several arguments, among which there are tuples.
*/
template <bool is_exact, bool argument_is_tuple>
class AggregateFunctionUniqUpToVariadic final
: public IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<UInt64>, AggregateFunctionUniqUpToVariadic<is_exact, argument_is_tuple>>
{
private:
size_t num_args = 0;
UInt8 threshold;
public:
AggregateFunctionUniqUpToVariadic(const DataTypes & arguments, const Array & params, UInt8 threshold_)
: IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<UInt64>, AggregateFunctionUniqUpToVariadic<is_exact, argument_is_tuple>>(arguments, params, std::make_shared<DataTypeUInt64>())
, threshold(threshold_)
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
else
num_args = arguments.size();
}
size_t sizeOfData() const override
{
return sizeof(AggregateFunctionUniqUpToData<UInt64>) + sizeof(UInt64) * threshold;
}
String getName() const override { return "uniqUpTo"; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).insert(UInt64(UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num)), threshold);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), threshold);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf, threshold);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf, threshold);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size());
}
};
constexpr UInt8 uniq_upto_max_threshold = 100;

View File

@ -1,278 +0,0 @@
#pragma once
#include <base/unaligned.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/UniqVariadicHash.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeUUID.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
struct Settings;
/** Counts the number of unique values up to no more than specified in the parameter.
*
* Example: uniqUpTo(3)(UserID)
* - will count the number of unique visitors, return 1, 2, 3 or 4 if visitors > = 4.
*
* For strings, a non-cryptographic hash function is used, due to which the calculation may be a bit inaccurate.
*/
template <typename T>
struct AggregateFunctionUniqUpToData
{
/** If count == threshold + 1 - this means that it is "overflowed" (values greater than threshold).
* In this case (for example, after calling the merge function), the `data` array does not necessarily contain the initialized values
* - example: combine a state in which there are few values, with another state that has overflowed;
* then set count to `threshold + 1`, and values from another state are not copied.
*/
UInt8 count = 0;
char data_ptr[0];
T load(size_t i) const
{
return unalignedLoad<T>(data_ptr + i * sizeof(T));
}
void store(size_t i, const T & x)
{
unalignedStore<T>(data_ptr + i * sizeof(T), x);
}
size_t size() const
{
return count;
}
/// threshold - for how many elements there is room in a `data`.
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE insert(T x, UInt8 threshold)
{
/// The state is already full - nothing needs to be done.
if (count > threshold)
return;
/// Linear search for the matching element.
for (size_t i = 0; i < count; ++i)
if (load(i) == x)
return;
/// Did not find the matching element. If there is room for one more element, insert it.
if (count < threshold)
store(count, x);
/// After increasing count, the state may be overflowed.
++count;
}
void merge(const AggregateFunctionUniqUpToData<T> & rhs, UInt8 threshold)
{
if (count > threshold)
return;
if (rhs.count > threshold)
{
/// If `rhs` is overflowed, then set `count` too also overflowed for the current state.
count = rhs.count;
return;
}
for (size_t i = 0; i < rhs.count; ++i)
insert(rhs.load(i), threshold);
}
void write(WriteBuffer & wb, UInt8 threshold) const
{
writeBinary(count, wb);
/// Write values only if the state is not overflowed. Otherwise, they are not needed, and only the fact that the state is overflowed is important.
if (count <= threshold)
wb.write(data_ptr, count * sizeof(T));
}
void read(ReadBuffer & rb, UInt8 threshold)
{
readBinary(count, rb);
if (count <= threshold)
rb.readStrict(data_ptr, count * sizeof(T));
}
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
insert(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], threshold);
}
};
/// For strings, their hashes are remembered.
template <>
struct AggregateFunctionUniqUpToData<String> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
/// Keep in mind that calculations are approximate.
StringRef value = column.getDataAt(row_num);
insert(CityHash_v1_0_2::CityHash64(value.data, value.size), threshold);
}
};
template <>
struct AggregateFunctionUniqUpToData<UInt128> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
UInt128 value = assert_cast<const ColumnVector<UInt128> &>(column).getData()[row_num];
insert(sipHash64(value), threshold);
}
};
template <>
struct AggregateFunctionUniqUpToData<UInt256> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
UInt256 value = assert_cast<const ColumnVector<UInt256> &>(column).getData()[row_num];
insert(sipHash64(value), threshold);
}
};
template <>
struct AggregateFunctionUniqUpToData<Int256> : AggregateFunctionUniqUpToData<UInt64>
{
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(const IColumn & column, size_t row_num, UInt8 threshold)
{
Int256 value = assert_cast<const ColumnVector<Int256> &>(column).getData()[row_num];
insert(sipHash64(value), threshold);
}
};
template <typename T>
class AggregateFunctionUniqUpTo final : public IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<T>, AggregateFunctionUniqUpTo<T>>
{
private:
UInt8 threshold;
public:
AggregateFunctionUniqUpTo(UInt8 threshold_, const DataTypes & argument_types_, const Array & params_)
: IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<T>, AggregateFunctionUniqUpTo<T>>(argument_types_, params_, std::make_shared<DataTypeUInt64>())
, threshold(threshold_)
{
}
size_t sizeOfData() const override
{
return sizeof(AggregateFunctionUniqUpToData<T>) + sizeof(T) * threshold;
}
String getName() const override { return "uniqUpTo"; }
bool allocatesMemoryInArena() const override { return false; }
/// ALWAYS_INLINE is required to have better code layout for uniqUpTo function
void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).add(*columns[0], row_num, threshold);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), threshold);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf, threshold);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf, threshold);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size());
}
};
/** For multiple arguments. To compute, hashes them.
* You can pass multiple arguments as is; You can also pass one argument - a tuple.
* But (for the possibility of effective implementation), you can not pass several arguments, among which there are tuples.
*/
template <bool is_exact, bool argument_is_tuple>
class AggregateFunctionUniqUpToVariadic final
: public IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<UInt64>, AggregateFunctionUniqUpToVariadic<is_exact, argument_is_tuple>>
{
private:
size_t num_args = 0;
UInt8 threshold;
public:
AggregateFunctionUniqUpToVariadic(const DataTypes & arguments, const Array & params, UInt8 threshold_)
: IAggregateFunctionDataHelper<AggregateFunctionUniqUpToData<UInt64>, AggregateFunctionUniqUpToVariadic<is_exact, argument_is_tuple>>(arguments, params, std::make_shared<DataTypeUInt64>())
, threshold(threshold_)
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
else
num_args = arguments.size();
}
size_t sizeOfData() const override
{
return sizeof(AggregateFunctionUniqUpToData<UInt64>) + sizeof(UInt64) * threshold;
}
String getName() const override { return "uniqUpTo"; }
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).insert(UInt64(UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num)), threshold);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), threshold);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf, threshold);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).read(buf, threshold);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size());
}
};
}

View File

@ -1,7 +1,11 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/AggregateFunctionVarianceMatrix.h>
#include <Columns/ColumnArray.h>
#include <Common/PODArray_fwd.h>
#include <DataTypes/DataTypeArray.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/Moments.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
@ -16,6 +20,149 @@ namespace ErrorCodes
namespace
{
enum class StatisticsMatrixFunctionKind
{
covarPopMatrix,
covarSampMatrix,
corrMatrix
};
template <StatisticsMatrixFunctionKind _kind>
struct AggregateFunctionVarianceMatrixData
{
using DataType = std::conditional_t<_kind == StatisticsMatrixFunctionKind::corrMatrix, CorrMoments<Float64>, CovarMoments<Float64>>;
AggregateFunctionVarianceMatrixData() = default;
explicit AggregateFunctionVarianceMatrixData(const size_t _num_args)
: num_args(_num_args)
{
data_matrix.resize_fill(num_args * (num_args + 1) / 2, DataType());
}
void add(const IColumn ** column, const size_t row_num)
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].add(column[i]->getFloat64(row_num), column[j]->getFloat64(row_num));
}
void merge(const AggregateFunctionVarianceMatrixData & other)
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].merge(other.data_matrix[i * (i + 1) / 2 + j]);
}
void serialize(WriteBuffer & buf) const
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].write(buf);
}
void deserialize(ReadBuffer & buf)
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].read(buf);
}
void insertResultInto(IColumn & to) const
{
auto & data_to = assert_cast<ColumnFloat64 &>(assert_cast<ColumnArray &>(assert_cast<ColumnArray &>(to).getData()).getData()).getData();
auto & root_offsets_to = assert_cast<ColumnArray &>(to).getOffsets();
auto & nested_offsets_to = assert_cast<ColumnArray &>(assert_cast<ColumnArray &>(to).getData()).getOffsets();
for (size_t i = 0; i < num_args; ++i)
{
for (size_t j = 0; j < num_args; ++j)
{
auto & data = i < j ? data_matrix[j * (j + 1) / 2 + i] : data_matrix[i * (i + 1) / 2 + j];
if constexpr (kind == StatisticsMatrixFunctionKind::covarPopMatrix)
data_to.push_back(data.getPopulation());
if constexpr (kind == StatisticsMatrixFunctionKind::covarSampMatrix)
data_to.push_back(data.getSample());
if constexpr (kind == StatisticsMatrixFunctionKind::corrMatrix)
data_to.push_back(data.get());
}
nested_offsets_to.push_back(nested_offsets_to.back() + num_args);
}
root_offsets_to.push_back(root_offsets_to.back() + num_args);
}
static constexpr StatisticsMatrixFunctionKind kind = _kind;
PaddedPODArray<DataType> data_matrix;
size_t num_args;
};
template <typename Data>
class AggregateFunctionVarianceMatrix final
: public IAggregateFunctionDataHelper<Data, AggregateFunctionVarianceMatrix<Data>>
{
public:
explicit AggregateFunctionVarianceMatrix(const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<Data, AggregateFunctionVarianceMatrix<Data>>(argument_types_, {}, createResultType())
{}
AggregateFunctionVarianceMatrix(const IDataType &, const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<Data, AggregateFunctionVarianceMatrix<Data>>(argument_types_, {}, createResultType())
{}
String getName() const override
{
if constexpr (Data::kind == StatisticsMatrixFunctionKind::covarPopMatrix)
return "covarPopMatrix";
if constexpr (Data::kind == StatisticsMatrixFunctionKind::covarSampMatrix)
return "covarSampMatrix";
if constexpr (Data::kind == StatisticsMatrixFunctionKind::corrMatrix)
return "corrMatrix";
UNREACHABLE();
}
void create(AggregateDataPtr __restrict place) const override
{
new (place) Data(this->argument_types.size());
}
static DataTypePtr createResultType()
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>()));
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).add(columns, row_num);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
this->data(place).insertResultInto(to);
}
};
using AggregateFunctionCovarPopMatrix = AggregateFunctionVarianceMatrix<AggregateFunctionVarianceMatrixData<StatisticsMatrixFunctionKind::covarPopMatrix>>;
using AggregateFunctionCovarSampMatrix = AggregateFunctionVarianceMatrix<AggregateFunctionVarianceMatrixData<StatisticsMatrixFunctionKind::covarSampMatrix>>;
using AggregateFunctionCorrMatrix = AggregateFunctionVarianceMatrix<AggregateFunctionVarianceMatrixData<StatisticsMatrixFunctionKind::corrMatrix>>;
template <typename FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionVarianceMatrix(
const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)

View File

@ -1,159 +0,0 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Common/PODArray.h>
#include <Common/PODArray_fwd.h>
#include <DataTypes/DataTypeArray.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <AggregateFunctions/Moments.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
struct Settings;
enum class StatisticsMatrixFunctionKind
{
covarPopMatrix,
covarSampMatrix,
corrMatrix
};
template <StatisticsMatrixFunctionKind _kind>
struct AggregateFunctionVarianceMatrixData
{
using DataType = std::conditional_t<_kind == StatisticsMatrixFunctionKind::corrMatrix, CorrMoments<Float64>, CovarMoments<Float64>>;
AggregateFunctionVarianceMatrixData() = default;
explicit AggregateFunctionVarianceMatrixData(const size_t _num_args)
: num_args(_num_args)
{
data_matrix.resize_fill(num_args * (num_args + 1) / 2, DataType());
}
void add(const IColumn ** column, const size_t row_num)
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].add(column[i]->getFloat64(row_num), column[j]->getFloat64(row_num));
}
void merge(const AggregateFunctionVarianceMatrixData & other)
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].merge(other.data_matrix[i * (i + 1) / 2 + j]);
}
void serialize(WriteBuffer & buf) const
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].write(buf);
}
void deserialize(ReadBuffer & buf)
{
for (size_t i = 0; i < num_args; ++i)
for (size_t j = 0; j <= i; ++j)
data_matrix[i * (i + 1) / 2 + j].read(buf);
}
void insertResultInto(IColumn & to) const
{
auto & data_to = assert_cast<ColumnFloat64 &>(assert_cast<ColumnArray &>(assert_cast<ColumnArray &>(to).getData()).getData()).getData();
auto & root_offsets_to = assert_cast<ColumnArray &>(to).getOffsets();
auto & nested_offsets_to = assert_cast<ColumnArray &>(assert_cast<ColumnArray &>(to).getData()).getOffsets();
for (size_t i = 0; i < num_args; ++i)
{
for (size_t j = 0; j < num_args; ++j)
{
auto & data = i < j ? data_matrix[j * (j + 1) / 2 + i] : data_matrix[i * (i + 1) / 2 + j];
if constexpr (kind == StatisticsMatrixFunctionKind::covarPopMatrix)
data_to.push_back(data.getPopulation());
if constexpr (kind == StatisticsMatrixFunctionKind::covarSampMatrix)
data_to.push_back(data.getSample());
if constexpr (kind == StatisticsMatrixFunctionKind::corrMatrix)
data_to.push_back(data.get());
}
nested_offsets_to.push_back(nested_offsets_to.back() + num_args);
}
root_offsets_to.push_back(root_offsets_to.back() + num_args);
}
static constexpr StatisticsMatrixFunctionKind kind = _kind;
PaddedPODArray<DataType> data_matrix;
size_t num_args;
};
template <typename Data>
class AggregateFunctionVarianceMatrix final
: public IAggregateFunctionDataHelper<Data, AggregateFunctionVarianceMatrix<Data>>
{
public:
explicit AggregateFunctionVarianceMatrix(const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<Data, AggregateFunctionVarianceMatrix<Data>>(argument_types_, {}, createResultType())
{}
AggregateFunctionVarianceMatrix(const IDataType &, const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<Data, AggregateFunctionVarianceMatrix<Data>>(argument_types_, {}, createResultType())
{}
String getName() const override
{
if constexpr (Data::kind == StatisticsMatrixFunctionKind::covarPopMatrix)
return "covarPopMatrix";
if constexpr (Data::kind == StatisticsMatrixFunctionKind::covarSampMatrix)
return "covarSampMatrix";
if constexpr (Data::kind == StatisticsMatrixFunctionKind::corrMatrix)
return "corrMatrix";
UNREACHABLE();
}
void create(AggregateDataPtr __restrict place) const override
{
new (place) Data(this->argument_types.size());
}
static DataTypePtr createResultType()
{
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>()));
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{
this->data(place).add(columns, row_num);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
this->data(place).insertResultInto(to);
}
};
using AggregateFunctionCovarPopMatrix = AggregateFunctionVarianceMatrix<AggregateFunctionVarianceMatrixData<StatisticsMatrixFunctionKind::covarPopMatrix>>;
using AggregateFunctionCovarSampMatrix = AggregateFunctionVarianceMatrix<AggregateFunctionVarianceMatrixData<StatisticsMatrixFunctionKind::covarSampMatrix>>;
using AggregateFunctionCorrMatrix = AggregateFunctionVarianceMatrix<AggregateFunctionVarianceMatrixData<StatisticsMatrixFunctionKind::corrMatrix>>;
}

View File

@ -1,13 +1,15 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionWindowFunnel.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDate32.h>
#include <DataTypes/DataTypeDateTime.h>
#include <base/range.h>
#include <unordered_set>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/assert_cast.h>
namespace DB
@ -18,11 +20,280 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
namespace
{
static constexpr size_t max_events = 32;
template <typename T>
struct AggregateFunctionWindowFunnelData
{
using TimestampEvent = std::pair<T, UInt8>;
using TimestampEvents = PODArrayWithStackMemory<TimestampEvent, 64>;
bool sorted = true;
TimestampEvents events_list;
size_t size() const
{
return events_list.size();
}
void add(T timestamp, UInt8 event)
{
/// Since most events should have already been sorted by timestamp.
if (sorted && events_list.size() > 0)
{
if (events_list.back().first == timestamp)
sorted = events_list.back().second <= event;
else
sorted = events_list.back().first <= timestamp;
}
events_list.emplace_back(timestamp, event);
}
void merge(const AggregateFunctionWindowFunnelData & other)
{
if (other.events_list.empty())
return;
const auto size = events_list.size();
events_list.insert(std::begin(other.events_list), std::end(other.events_list));
/// either sort whole container or do so partially merging ranges afterwards
if (!sorted && !other.sorted)
std::stable_sort(std::begin(events_list), std::end(events_list));
else
{
const auto begin = std::begin(events_list);
const auto middle = std::next(begin, size);
const auto end = std::end(events_list);
if (!sorted)
std::stable_sort(begin, middle);
if (!other.sorted)
std::stable_sort(middle, end);
std::inplace_merge(begin, middle, end);
}
sorted = true;
}
void sort()
{
if (!sorted)
{
std::stable_sort(std::begin(events_list), std::end(events_list));
sorted = true;
}
}
void serialize(WriteBuffer & buf) const
{
writeBinary(sorted, buf);
writeBinary(events_list.size(), buf);
for (const auto & events : events_list)
{
writeBinary(events.first, buf);
writeBinary(events.second, buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(sorted, buf);
size_t size;
readBinary(size, buf);
/// TODO Protection against huge size
events_list.clear();
events_list.reserve(size);
T timestamp;
UInt8 event;
for (size_t i = 0; i < size; ++i)
{
readBinary(timestamp, buf);
readBinary(event, buf);
events_list.emplace_back(timestamp, event);
}
}
};
/** Calculates the max event level in a sliding window.
* The max size of events is 32, that's enough for funnel analytics
*
* Usage:
* - windowFunnel(window)(timestamp, cond1, cond2, cond3, ....)
*/
template <typename T, typename Data>
class AggregateFunctionWindowFunnel final
: public IAggregateFunctionDataHelper<Data, AggregateFunctionWindowFunnel<T, Data>>
{
private:
UInt64 window;
UInt8 events_size;
/// When the 'strict_deduplication' is set, it applies conditions only for the not repeating values.
bool strict_deduplication;
/// When the 'strict_order' is set, it doesn't allow interventions of other events.
/// In the case of 'A->B->D->C', it stops finding 'A->B->C' at the 'D' and the max event level is 2.
bool strict_order;
/// Applies conditions only to events with strictly increasing timestamps
bool strict_increase;
/// Loop through the entire events_list, update the event timestamp value
/// The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window.
/// If found, returns the max event level, else return 0.
/// The algorithm works in O(n) time, but the overall function works in O(n * log(n)) due to sorting.
UInt8 getEventLevel(Data & data) const
{
if (data.size() == 0)
return 0;
if (!strict_order && events_size == 1)
return 1;
data.sort();
/// events_timestamp stores the timestamp of the first and previous i-th level event happen within time window
std::vector<std::optional<std::pair<UInt64, UInt64>>> events_timestamp(events_size);
bool first_event = false;
for (size_t i = 0; i < data.events_list.size(); ++i)
{
const T & timestamp = data.events_list[i].first;
const auto & event_idx = data.events_list[i].second - 1;
if (strict_order && event_idx == -1)
{
if (first_event)
break;
else
continue;
}
else if (event_idx == 0)
{
events_timestamp[0] = std::make_pair(timestamp, timestamp);
first_event = true;
}
else if (strict_deduplication && events_timestamp[event_idx].has_value())
{
return data.events_list[i - 1].second;
}
else if (strict_order && first_event && !events_timestamp[event_idx - 1].has_value())
{
for (size_t event = 0; event < events_timestamp.size(); ++event)
{
if (!events_timestamp[event].has_value())
return event;
}
}
else if (events_timestamp[event_idx - 1].has_value())
{
auto first_timestamp = events_timestamp[event_idx - 1]->first;
bool time_matched = timestamp <= first_timestamp + window;
if (strict_increase)
time_matched = time_matched && events_timestamp[event_idx - 1]->second < timestamp;
if (time_matched)
{
events_timestamp[event_idx] = std::make_pair(first_timestamp, timestamp);
if (event_idx + 1 == events_size)
return events_size;
}
}
}
for (size_t event = events_timestamp.size(); event > 0; --event)
{
if (events_timestamp[event - 1].has_value())
return event;
}
return 0;
}
public:
String getName() const override
{
return "windowFunnel";
}
AggregateFunctionWindowFunnel(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<Data, AggregateFunctionWindowFunnel<T, Data>>(arguments, params, std::make_shared<DataTypeUInt8>())
{
events_size = arguments.size() - 1;
window = params.at(0).safeGet<UInt64>();
strict_deduplication = false;
strict_order = false;
strict_increase = false;
for (size_t i = 1; i < params.size(); ++i)
{
String option = params.at(i).safeGet<String>();
if (option == "strict_deduplication")
strict_deduplication = true;
else if (option == "strict_order")
strict_order = true;
else if (option == "strict_increase")
strict_increase = true;
else if (option == "strict")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "strict is replaced with strict_deduplication in Aggregate function {}", getName());
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} doesn't support a parameter: {}", getName(), option);
}
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
bool has_event = false;
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
/// reverse iteration and stable sorting are needed for events that are qualified by more than one condition.
for (auto i = events_size; i > 0; --i)
{
auto event = assert_cast<const ColumnVector<UInt8> *>(columns[i])->getData()[row_num];
if (event)
{
this->data(place).add(timestamp, i);
has_event = true;
}
}
if (strict_order && !has_event)
this->data(place).add(timestamp, 0);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt8 &>(to).getData().push_back(getEventLevel(this->data(place)));
}
};
template <template <typename> class Data>
AggregateFunctionPtr
createAggregateFunctionWindowFunnel(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)

View File

@ -1,287 +0,0 @@
#pragma once
#include <unordered_set>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/assert_cast.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
static constexpr size_t max_events = 32;
template <typename T>
struct AggregateFunctionWindowFunnelData
{
using TimestampEvent = std::pair<T, UInt8>;
using TimestampEvents = PODArrayWithStackMemory<TimestampEvent, 64>;
bool sorted = true;
TimestampEvents events_list;
size_t size() const
{
return events_list.size();
}
void add(T timestamp, UInt8 event)
{
/// Since most events should have already been sorted by timestamp.
if (sorted && events_list.size() > 0)
{
if (events_list.back().first == timestamp)
sorted = events_list.back().second <= event;
else
sorted = events_list.back().first <= timestamp;
}
events_list.emplace_back(timestamp, event);
}
void merge(const AggregateFunctionWindowFunnelData & other)
{
if (other.events_list.empty())
return;
const auto size = events_list.size();
events_list.insert(std::begin(other.events_list), std::end(other.events_list));
/// either sort whole container or do so partially merging ranges afterwards
if (!sorted && !other.sorted)
std::stable_sort(std::begin(events_list), std::end(events_list));
else
{
const auto begin = std::begin(events_list);
const auto middle = std::next(begin, size);
const auto end = std::end(events_list);
if (!sorted)
std::stable_sort(begin, middle);
if (!other.sorted)
std::stable_sort(middle, end);
std::inplace_merge(begin, middle, end);
}
sorted = true;
}
void sort()
{
if (!sorted)
{
std::stable_sort(std::begin(events_list), std::end(events_list));
sorted = true;
}
}
void serialize(WriteBuffer & buf) const
{
writeBinary(sorted, buf);
writeBinary(events_list.size(), buf);
for (const auto & events : events_list)
{
writeBinary(events.first, buf);
writeBinary(events.second, buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(sorted, buf);
size_t size;
readBinary(size, buf);
/// TODO Protection against huge size
events_list.clear();
events_list.reserve(size);
T timestamp;
UInt8 event;
for (size_t i = 0; i < size; ++i)
{
readBinary(timestamp, buf);
readBinary(event, buf);
events_list.emplace_back(timestamp, event);
}
}
};
/** Calculates the max event level in a sliding window.
* The max size of events is 32, that's enough for funnel analytics
*
* Usage:
* - windowFunnel(window)(timestamp, cond1, cond2, cond3, ....)
*/
template <typename T, typename Data>
class AggregateFunctionWindowFunnel final
: public IAggregateFunctionDataHelper<Data, AggregateFunctionWindowFunnel<T, Data>>
{
private:
UInt64 window;
UInt8 events_size;
/// When the 'strict_deduplication' is set, it applies conditions only for the not repeating values.
bool strict_deduplication;
/// When the 'strict_order' is set, it doesn't allow interventions of other events.
/// In the case of 'A->B->D->C', it stops finding 'A->B->C' at the 'D' and the max event level is 2.
bool strict_order;
/// Applies conditions only to events with strictly increasing timestamps
bool strict_increase;
/// Loop through the entire events_list, update the event timestamp value
/// The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window.
/// If found, returns the max event level, else return 0.
/// The algorithm works in O(n) time, but the overall function works in O(n * log(n)) due to sorting.
UInt8 getEventLevel(Data & data) const
{
if (data.size() == 0)
return 0;
if (!strict_order && events_size == 1)
return 1;
data.sort();
/// events_timestamp stores the timestamp of the first and previous i-th level event happen within time window
std::vector<std::optional<std::pair<UInt64, UInt64>>> events_timestamp(events_size);
bool first_event = false;
for (size_t i = 0; i < data.events_list.size(); ++i)
{
const T & timestamp = data.events_list[i].first;
const auto & event_idx = data.events_list[i].second - 1;
if (strict_order && event_idx == -1)
{
if (first_event)
break;
else
continue;
}
else if (event_idx == 0)
{
events_timestamp[0] = std::make_pair(timestamp, timestamp);
first_event = true;
}
else if (strict_deduplication && events_timestamp[event_idx].has_value())
{
return data.events_list[i - 1].second;
}
else if (strict_order && first_event && !events_timestamp[event_idx - 1].has_value())
{
for (size_t event = 0; event < events_timestamp.size(); ++event)
{
if (!events_timestamp[event].has_value())
return event;
}
}
else if (events_timestamp[event_idx - 1].has_value())
{
auto first_timestamp = events_timestamp[event_idx - 1]->first;
bool time_matched = timestamp <= first_timestamp + window;
if (strict_increase)
time_matched = time_matched && events_timestamp[event_idx - 1]->second < timestamp;
if (time_matched)
{
events_timestamp[event_idx] = std::make_pair(first_timestamp, timestamp);
if (event_idx + 1 == events_size)
return events_size;
}
}
}
for (size_t event = events_timestamp.size(); event > 0; --event)
{
if (events_timestamp[event - 1].has_value())
return event;
}
return 0;
}
public:
String getName() const override
{
return "windowFunnel";
}
AggregateFunctionWindowFunnel(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper<Data, AggregateFunctionWindowFunnel<T, Data>>(arguments, params, std::make_shared<DataTypeUInt8>())
{
events_size = arguments.size() - 1;
window = params.at(0).safeGet<UInt64>();
strict_deduplication = false;
strict_order = false;
strict_increase = false;
for (size_t i = 1; i < params.size(); ++i)
{
String option = params.at(i).safeGet<String>();
if (option == "strict_deduplication")
strict_deduplication = true;
else if (option == "strict_order")
strict_order = true;
else if (option == "strict_increase")
strict_increase = true;
else if (option == "strict")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "strict is replaced with strict_deduplication in Aggregate function {}", getName());
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} doesn't support a parameter: {}", getName(), option);
}
}
bool allocatesMemoryInArena() const override { return false; }
void add(AggregateDataPtr __restrict place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
bool has_event = false;
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
/// reverse iteration and stable sorting are needed for events that are qualified by more than one condition.
for (auto i = events_size; i > 0; --i)
{
auto event = assert_cast<const ColumnVector<UInt8> *>(columns[i])->getData()[row_num];
if (event)
{
this->data(place).add(timestamp, i);
has_event = true;
}
}
if (strict_order && !has_event)
this->data(place).add(timestamp, 0);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).serialize(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
this->data(place).deserialize(buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
assert_cast<ColumnUInt8 &>(to).getData().push_back(getEventLevel(this->data(place)));
}
};
}

View File

@ -1,477 +0,0 @@
#pragma once
#include <cmath>
#include <base/sort.h>
#include <Common/RadixSort.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
template <typename T>
class ApproxSampler
{
public:
struct Stats
{
T value; // the sampled value
Int64 g; // the minimum rank jump from the previous value's minimum rank
Int64 delta; // the maximum span of the rank
Stats() = default;
Stats(T value_, Int64 g_, Int64 delta_) : value(value_), g(g_), delta(delta_) {}
};
struct QueryResult
{
size_t index;
Int64 rank;
T value;
QueryResult(size_t index_, Int64 rank_, T value_) : index(index_), rank(rank_), value(value_) { }
};
ApproxSampler() = default;
explicit ApproxSampler(
double relative_error_,
size_t compress_threshold_ = default_compress_threshold,
size_t count_ = 0,
bool compressed_ = false)
: relative_error(relative_error_)
, compress_threshold(compress_threshold_)
, count(count_)
, compressed(compressed_)
{
sampled.reserve(compress_threshold);
backup_sampled.reserve(compress_threshold);
head_sampled.reserve(default_head_size);
}
bool isCompressed() const { return compressed; }
void setCompressed() { compressed = true; }
void insert(T x)
{
head_sampled.push_back(x);
compressed = false;
if (head_sampled.size() >= default_head_size)
{
withHeadBufferInserted();
if (sampled.size() >= compress_threshold)
compress();
}
}
void query(const Float64 * percentiles, const size_t * indices, size_t size, T * result) const
{
if (!head_sampled.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot operate on an uncompressed summary, call compress() first");
if (sampled.empty())
{
for (size_t i = 0; i < size; ++i)
result[i] = T();
return;
}
Int64 current_max = std::numeric_limits<Int64>::min();
for (const auto & stats : sampled)
current_max = std::max(stats.delta + stats.g, current_max);
Int64 target_error = current_max/2;
size_t index= 0;
auto min_rank = sampled[0].g;
for (size_t i = 0; i < size; ++i)
{
double percentile = percentiles[indices[i]];
if (percentile <= relative_error)
{
result[indices[i]] = sampled.front().value;
}
else if (percentile >= 1 - relative_error)
{
result[indices[i]] = sampled.back().value;
}
else
{
QueryResult res = findApproxQuantile(index, min_rank, target_error, percentile);
index = res.index;
min_rank = res.rank;
result[indices[i]] = res.value;
}
}
}
void compress()
{
if (compressed)
return;
withHeadBufferInserted();
doCompress(2 * relative_error * count);
compressed = true;
}
void merge(const ApproxSampler & other)
{
if (other.count == 0)
return;
else if (count == 0)
{
compress_threshold = other.compress_threshold;
relative_error = other.relative_error;
count = other.count;
compressed = other.compressed;
sampled.resize(other.sampled.size());
memcpy(sampled.data(), other.sampled.data(), sizeof(Stats) * other.sampled.size());
return;
}
else
{
// Merge the two buffers.
// The GK algorithm is a bit unclear about it, but we need to adjust the statistics during the
// merging. The main idea is that samples that come from one side will suffer from the lack of
// precision of the other.
// As a concrete example, take two QuantileSummaries whose samples (value, g, delta) are:
// `a = [(0, 1, 0), (20, 99, 0)]` and `b = [(10, 1, 0), (30, 49, 0)]`
// This means `a` has 100 values, whose minimum is 0 and maximum is 20,
// while `b` has 50 values, between 10 and 30.
// The resulting samples of the merge will be:
// a+b = [(0, 1, 0), (10, 1, ??), (20, 99, ??), (30, 49, 0)]
// The values of `g` do not change, as they represent the minimum number of values between two
// consecutive samples. The values of `delta` should be adjusted, however.
// Take the case of the sample `10` from `b`. In the original stream, it could have appeared
// right after `0` (as expressed by `g=1`) or right before `20`, so `delta=99+0-1=98`.
// In the GK algorithm's style of working in terms of maximum bounds, one can observe that the
// maximum additional uncertainty over samples coming from `b` is `max(g_a + delta_a) =
// floor(2 * eps_a * n_a)`. Likewise, additional uncertainty over samples from `a` is
// `floor(2 * eps_b * n_b)`.
// Only samples that interleave the other side are affected. That means that samples from
// one side that are lesser (or greater) than all samples from the other side are just copied
// unmodified.
// If the merging instances have different `relativeError`, the resulting instance will carry
// the largest one: `eps_ab = max(eps_a, eps_b)`.
// The main invariant of the GK algorithm is kept:
// `max(g_ab + delta_ab) <= floor(2 * eps_ab * (n_a + n_b))` since
// `max(g_ab + delta_ab) <= floor(2 * eps_a * n_a) + floor(2 * eps_b * n_b)`
// Finally, one can see how the `insert(x)` operation can be expressed as `merge([(x, 1, 0])`
compress();
backup_sampled.clear();
backup_sampled.reserve(sampled.size() + other.sampled.size());
double merged_relative_error = std::max(relative_error, other.relative_error);
size_t merged_count = count + other.count;
Int64 additional_self_delta = static_cast<Int64>(std::floor(2 * other.relative_error * other.count));
Int64 additional_other_delta = static_cast<Int64>(std::floor(2 * relative_error * count));
// Do a merge of two sorted lists until one of the lists is fully consumed
size_t self_idx = 0;
size_t other_idx = 0;
while (self_idx < sampled.size() && other_idx < other.sampled.size())
{
const Stats & self_sample = sampled[self_idx];
const Stats & other_sample = other.sampled[other_idx];
// Detect next sample
Stats next_sample;
Int64 additional_delta = 0;
if (self_sample.value < other_sample.value)
{
++self_idx;
next_sample = self_sample;
additional_delta = other_idx > 0 ? additional_self_delta : 0;
}
else
{
++other_idx;
next_sample = other_sample;
additional_delta = self_idx > 0 ? additional_other_delta : 0;
}
// Insert it
next_sample.delta += additional_delta;
backup_sampled.emplace_back(std::move(next_sample));
}
// Copy the remaining samples from the other list
// (by construction, at most one `while` loop will run)
while (self_idx < sampled.size())
{
backup_sampled.emplace_back(sampled[self_idx]);
++self_idx;
}
while (other_idx < other.sampled.size())
{
backup_sampled.emplace_back(other.sampled[other_idx]);
++other_idx;
}
std::swap(sampled, backup_sampled);
relative_error = merged_relative_error;
count = merged_count;
compress_threshold = other.compress_threshold;
doCompress(2 * merged_relative_error * merged_count);
compressed = true;
}
}
void write(WriteBuffer & buf) const
{
writeBinaryLittleEndian(compress_threshold, buf);
writeBinaryLittleEndian(relative_error, buf);
writeBinaryLittleEndian(count, buf);
writeBinaryLittleEndian(sampled.size(), buf);
for (const auto & stats : sampled)
{
writeBinaryLittleEndian(stats.value, buf);
writeBinaryLittleEndian(stats.g, buf);
writeBinaryLittleEndian(stats.delta, buf);
}
}
void read(ReadBuffer & buf)
{
readBinaryLittleEndian(compress_threshold, buf);
readBinaryLittleEndian(relative_error, buf);
readBinaryLittleEndian(count, buf);
size_t sampled_len = 0;
readBinaryLittleEndian(sampled_len, buf);
sampled.resize(sampled_len);
for (size_t i = 0; i < sampled_len; ++i)
{
auto stats = sampled[i];
readBinaryLittleEndian(stats.value, buf);
readBinaryLittleEndian(stats.g, buf);
readBinaryLittleEndian(stats.delta, buf);
}
}
private:
QueryResult findApproxQuantile(size_t index, Int64 min_rank_at_index, double target_error, double percentile) const
{
Stats curr_sample = sampled[index];
Int64 rank = static_cast<Int64>(std::ceil(percentile * count));
size_t i = index;
Int64 min_rank = min_rank_at_index;
while (i < sampled.size() - 1)
{
Int64 max_rank = min_rank + curr_sample.delta;
if (max_rank - target_error <= rank && rank <= min_rank + target_error)
return {i, min_rank, curr_sample.value};
else
{
++i;
curr_sample = sampled[i];
min_rank += curr_sample.g;
}
}
return {sampled.size()-1, 0, sampled.back().value};
}
void withHeadBufferInserted()
{
if (head_sampled.empty())
return;
bool use_radix_sort = head_sampled.size() >= 256 && (is_arithmetic_v<T> && !is_big_int_v<T>);
if (use_radix_sort)
RadixSort<RadixSortNumTraits<T>>::executeLSD(head_sampled.data(), head_sampled.size());
else
::sort(head_sampled.begin(), head_sampled.end());
backup_sampled.clear();
backup_sampled.reserve(sampled.size() + head_sampled.size());
size_t sample_idx = 0;
size_t ops_idx = 0;
size_t current_count = count;
for (; ops_idx < head_sampled.size(); ++ops_idx)
{
T current_sample = head_sampled[ops_idx];
// Add all the samples before the next observation.
while (sample_idx < sampled.size() && sampled[sample_idx].value <= current_sample)
{
backup_sampled.emplace_back(sampled[sample_idx]);
++sample_idx;
}
// If it is the first one to insert, of if it is the last one
++current_count;
Int64 delta;
if (backup_sampled.empty() || (sample_idx == sampled.size() && ops_idx == (head_sampled.size() - 1)))
delta = 0;
else
delta = static_cast<Int64>(std::floor(2 * relative_error * current_count));
backup_sampled.emplace_back(current_sample, 1, delta);
}
// Add all the remaining existing samples
for (; sample_idx < sampled.size(); ++sample_idx)
backup_sampled.emplace_back(sampled[sample_idx]);
std::swap(sampled, backup_sampled);
head_sampled.clear();
count = current_count;
}
void doCompress(double merge_threshold)
{
if (sampled.empty())
return;
backup_sampled.clear();
// Start for the last element, which is always part of the set.
// The head contains the current new head, that may be merged with the current element.
Stats head = sampled.back();
ssize_t i = sampled.size() - 2;
// Do not compress the last element
while (i >= 1)
{
// The current sample:
const auto & sample1 = sampled[i];
// Do we need to compress?
if (sample1.g + head.g + head.delta < merge_threshold)
{
// Do not insert yet, just merge the current element into the head.
head.g += sample1.g;
}
else
{
// Prepend the current head, and keep the current sample as target for merging.
backup_sampled.push_back(head);
head = sample1;
}
--i;
}
backup_sampled.push_back(head);
// If necessary, add the minimum element:
auto curr_head = sampled.front();
// don't add the minimum element if `currentSamples` has only one element (both `currHead` and
// `head` point to the same element)
if (curr_head.value <= head.value && sampled.size() > 1)
backup_sampled.emplace_back(sampled.front());
std::reverse(backup_sampled.begin(), backup_sampled.end());
std::swap(sampled, backup_sampled);
}
double relative_error;
size_t compress_threshold;
size_t count = 0;
bool compressed;
PaddedPODArray<Stats> sampled;
PaddedPODArray<Stats> backup_sampled;
PaddedPODArray<T> head_sampled;
static constexpr size_t default_compress_threshold = 10000;
static constexpr size_t default_head_size = 50000;
};
template <typename Value>
class QuantileGK
{
private:
using Data = ApproxSampler<Value>;
mutable Data data;
public:
QuantileGK() = default;
explicit QuantileGK(size_t accuracy) : data(1.0 / static_cast<double>(accuracy)) { }
void add(const Value & x)
{
data.insert(x);
}
template <typename Weight>
void add(const Value &, const Weight &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method add with weight is not implemented for GKSampler");
}
void merge(const QuantileGK & rhs)
{
if (!data.isCompressed())
data.compress();
data.merge(rhs.data);
}
void serialize(WriteBuffer & buf) const
{
/// Always compress before serialization
if (!data.isCompressed())
data.compress();
data.write(buf);
}
void deserialize(ReadBuffer & buf)
{
data.read(buf);
data.setCompressed();
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level)
{
if (!data.isCompressed())
data.compress();
Value res;
size_t indice = 0;
data.query(&level, &indice, 1, &res);
return res;
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result)
{
if (!data.isCompressed())
data.compress();
data.query(levels, indices, size, result);
}
Float64 getFloat64(Float64 /*level*/)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat64 is not implemented for GKSampler");
}
void getManyFloat(const Float64 * /*levels*/, const size_t * /*indices*/, size_t /*size*/, Float64 * /*result*/)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getManyFloat is not implemented for GKSampler");
}
};
}

View File

@ -1,203 +0,0 @@
#pragma once
#include <base/sort.h>
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Calculates quantile by counting number of occurrences for each value in a hash map.
*
* It uses O(distinct(N)) memory. Can be naturally applied for values with weight.
* In case of many identical values, it can be more efficient than QuantileExact even when weight is not used.
*/
template <typename Value>
struct QuantileExactWeighted
{
struct Int128Hash
{
size_t operator()(Int128 x) const
{
return CityHash_v1_0_2::Hash128to64({x >> 64, x & 0xffffffffffffffffll});
}
};
using Weight = UInt64;
using UnderlyingType = NativeType<Value>;
using Hasher = HashCRC32<UnderlyingType>;
/// When creating, the hash table must be small.
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;
Map map;
void add(const Value & x)
{
/// We must skip NaNs as they are not compatible with comparison sorting.
if (!isNaN(x))
++map[x];
}
void add(const Value & x, Weight weight)
{
if (!isNaN(x))
map[x] += weight;
}
void merge(const QuantileExactWeighted & rhs)
{
for (const auto & pair : rhs.map)
map[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
map.write(buf);
}
void deserialize(ReadBuffer & buf)
{
typename Map::Reader reader(buf);
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
}
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level) const
{
size_t size = map.size();
if (0 == size)
return std::numeric_limits<Value>::quiet_NaN();
/// Copy the data to a temporary array to get the element you need in order.
using Pair = typename Map::value_type;
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
/// Note: 64-bit integer weight can overflow.
/// We do some implementation specific behaviour (return approximate or garbage results).
/// Float64 is used as accumulator here to get approximate results.
/// But weight can be already overflowed in computations in 'add' and 'merge' methods.
/// It will be reasonable to change the type of weight to Float64 in the map,
/// but we don't do that for compatibility of serialized data.
size_t i = 0;
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
array[i] = pair.getValue();
++i;
}
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 threshold = std::ceil(sum_weight * level);
Float64 accumulated = 0;
const Pair * it = array;
const Pair * end = array + size;
while (it < end)
{
accumulated += it->second;
if (accumulated >= threshold)
break;
++it;
}
if (it == end)
--it;
return it->first;
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const
{
size_t size = map.size();
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
result[i] = Value();
return;
}
/// Copy the data to a temporary array to get the element you need in order.
using Pair = typename Map::value_type;
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
size_t i = 0;
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
array[i] = pair.getValue();
++i;
}
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
const Pair * it = array;
const Pair * end = array + size;
size_t level_index = 0;
Float64 threshold = std::ceil(sum_weight * levels[indices[level_index]]);
while (it < end)
{
accumulated += it->second;
while (accumulated >= threshold)
{
result[indices[level_index]] = it->first;
++level_index;
if (level_index == num_levels)
return;
threshold = std::ceil(sum_weight * levels[indices[level_index]]);
}
++it;
}
while (level_index < num_levels)
{
result[indices[level_index]] = array[size - 1].first;
++level_index;
}
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat is not implemented for QuantileExact");
}
void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getManyFloat is not implemented for QuantileExact");
}
};
}

View File

@ -1,308 +0,0 @@
#pragma once
#include <base/sort.h>
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Approximates Quantile by:
* - sorting input values and weights
* - building a cumulative distribution based on weights
* - performing linear interpolation between the weights and values
*
*/
template <typename Value>
struct QuantileInterpolatedWeighted
{
struct Int128Hash
{
size_t operator()(Int128 x) const
{
return CityHash_v1_0_2::Hash128to64({x >> 64, x & 0xffffffffffffffffll});
}
};
using Weight = UInt64;
using UnderlyingType = NativeType<Value>;
using Hasher = HashCRC32<UnderlyingType>;
/// When creating, the hash table must be small.
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;
Map map;
void add(const Value & x)
{
/// We must skip NaNs as they are not compatible with comparison sorting.
if (!isNaN(x))
++map[x];
}
void add(const Value & x, Weight weight)
{
if (!isNaN(x))
map[x] += weight;
}
void merge(const QuantileInterpolatedWeighted & rhs)
{
for (const auto & pair : rhs.map)
map[pair.getKey()] += pair.getMapped();
}
void serialize(WriteBuffer & buf) const
{
map.write(buf);
}
void deserialize(ReadBuffer & buf)
{
typename Map::Reader reader(buf);
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
}
}
Value get(Float64 level) const
{
return getImpl<Value>(level);
}
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result) const
{
getManyImpl<Value>(levels, indices, size, result);
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat is not implemented for QuantileInterpolatedWeighted");
}
void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getManyFloat is not implemented for QuantileInterpolatedWeighted");
}
private:
using Pair = typename std::pair<UnderlyingType, Float64>;
/// Get the value of the `level` quantile. The level must be between 0 and 1.
template <typename T>
T getImpl(Float64 level) const
{
size_t size = map.size();
if (0 == size)
return std::numeric_limits<Value>::quiet_NaN();
/// Maintain a vector of pair of values and weights for easier sorting and for building
/// a cumulative distribution using the provided weights.
std::vector<Pair> value_weight_pairs;
value_weight_pairs.reserve(size);
/// Note: weight provided must be a 64-bit integer
/// Float64 is used as accumulator here to get approximate results.
/// But weight used in the internal array is stored as Float64 as we
/// do some quantile estimation operation which involves division and
/// require Float64 level of precision.
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
auto value = pair.getKey();
auto weight = pair.getMapped();
value_weight_pairs.push_back({value, weight});
}
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
/// vector for populating and storing the cumulative sum using the provided weights.
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
std::vector<Float64> weights_cum_sum;
weights_cum_sum.reserve(size);
for (size_t idx = 0; idx < size; ++idx)
{
accumulated += value_weight_pairs[idx].second;
weights_cum_sum.push_back(accumulated);
}
/// The following estimation of quantile is general and the idea is:
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
/// calculates a simple cumulative distribution based on weights
if (sum_weight != 0)
{
for (size_t idx = 0; idx < size; ++idx)
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
}
/// perform linear interpolation
size_t idx = 0;
if (size >= 2)
{
if (level >= value_weight_pairs[size - 2].second)
{
idx = size - 2;
}
else
{
size_t start = 0, end = size - 1;
while (start <= end)
{
size_t mid = start + (end - start) / 2;
if (mid > size)
break;
if (level > value_weight_pairs[mid + 1].second)
start = mid + 1;
else
{
idx = mid;
end = mid - 1;
}
}
}
}
size_t l = idx;
size_t u = idx + 1 < size ? idx + 1 : idx;
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
if (level < xl)
yr = yl;
if (level > xr)
yl = yr;
return static_cast<T>(interpolate(level, xl, xr, yl, yr));
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
template <typename T>
void getManyImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const
{
size_t size = map.size();
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
result[i] = Value();
return;
}
std::vector<Pair> value_weight_pairs;
value_weight_pairs.reserve(size);
Float64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.getMapped();
auto value = pair.getKey();
auto weight = pair.getMapped();
value_weight_pairs.push_back({value, weight});
}
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;
/// vector for populating and storing the cumulative sum using the provided weights.
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
std::vector<Float64> weights_cum_sum;
weights_cum_sum.reserve(size);
for (size_t idx = 0; idx < size; ++idx)
{
accumulated += value_weight_pairs[idx].second;
weights_cum_sum.emplace_back(accumulated);
}
/// The following estimation of quantile is general and the idea is:
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
/// calculates a simple cumulative distribution based on weights
if (sum_weight != 0)
{
for (size_t idx = 0; idx < size; ++idx)
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
}
for (size_t level_index = 0; level_index < num_levels; ++level_index)
{
/// perform linear interpolation for every level
auto level = levels[indices[level_index]];
size_t idx = 0;
if (size >= 2)
{
if (level >= value_weight_pairs[size - 2].second)
{
idx = size - 2;
}
else
{
size_t start = 0, end = size - 1;
while (start <= end)
{
size_t mid = start + (end - start) / 2;
if (mid > size)
break;
if (level > value_weight_pairs[mid + 1].second)
start = mid + 1;
else
{
idx = mid;
end = mid - 1;
}
}
}
}
size_t l = idx;
size_t u = idx + 1 < size ? idx + 1 : idx;
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
if (level < xl)
yr = yl;
if (level > xr)
yl = yr;
result[indices[level_index]] = static_cast<T>(interpolate(level, xl, xr, yl, yr));
}
}
/// This ignores overflows or NaN's that might arise during add, sub and mul operations and doesn't aim to provide exact
/// results since `the quantileInterpolatedWeighted` function itself relies mainly on approximation.
UnderlyingType NO_SANITIZE_UNDEFINED interpolate(Float64 level, Float64 xl, Float64 xr, UnderlyingType yl, UnderlyingType yr) const
{
UnderlyingType dy = yr - yl;
Float64 dx = xr - xl;
dx = dx == 0 ? 1 : dx; /// to handle NaN behavior that might arise during integer division below.
/// yl + (dy / dx) * (level - xl)
return static_cast<UnderlyingType>(yl + (dy / dx) * (level - xl));
}
};
}

View File

@ -1,102 +0,0 @@
#pragma once
#include <AggregateFunctions/ReservoirSampler.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Quantile calculation with "reservoir sample" algorithm.
* It collects pseudorandom subset of limited size from a stream of values,
* and approximate quantile from it.
* The result is non-deterministic. Also look at QuantileReservoirSamplerDeterministic.
*
* This algorithm is quite inefficient in terms of precision for memory usage,
* but very efficient in CPU (though less efficient than QuantileTiming and than QuantileExact for small sets).
*/
template <typename Value>
struct QuantileReservoirSampler
{
using Data = ReservoirSampler<Value, ReservoirSamplerOnEmpty::RETURN_NAN_OR_ZERO>;
Data data;
void add(const Value & x)
{
data.insert(x);
}
template <typename Weight>
void add(const Value &, const Weight &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method add with weight is not implemented for ReservoirSampler");
}
void merge(const QuantileReservoirSampler & rhs)
{
data.merge(rhs.data);
}
void serialize(WriteBuffer & buf) const
{
data.write(buf);
}
void deserialize(ReadBuffer & buf)
{
data.read(buf);
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level)
{
if (data.empty())
return {};
if constexpr (is_decimal<Value>)
return Value(static_cast<typename Value::NativeType>(data.quantileInterpolated(level)));
else
return static_cast<Value>(data.quantileInterpolated(level));
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result)
{
bool is_empty = data.empty();
for (size_t i = 0; i < size; ++i)
{
if (is_empty)
{
result[i] = Value{};
}
else
{
if constexpr (is_decimal<Value>)
result[indices[i]] = Value(static_cast<typename Value::NativeType>(data.quantileInterpolated(levels[indices[i]])));
else
result[indices[i]] = Value(data.quantileInterpolated(levels[indices[i]]));
}
}
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64 level)
{
return data.quantileInterpolated(level);
}
void getManyFloat(const Float64 * levels, const size_t * indices, size_t size, Float64 * result)
{
for (size_t i = 0; i < size; ++i)
result[indices[i]] = data.quantileInterpolated(levels[indices[i]]);
}
};
}

View File

@ -1,102 +0,0 @@
#pragma once
#include <AggregateFunctions/ReservoirSamplerDeterministic.h>
namespace DB
{
struct Settings;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Quantile calculation with "reservoir sample" algorithm.
* It collects pseudorandom subset of limited size from a stream of values,
* and approximate quantile from it.
* The function accept second argument, named "determinator"
* and a hash function from it is calculated and used as a source for randomness
* to apply random sampling.
* The function is deterministic, but care should be taken with choose of "determinator" argument.
*/
template <typename Value>
struct QuantileReservoirSamplerDeterministic
{
using Data = ReservoirSamplerDeterministic<Value, ReservoirSamplerDeterministicOnEmpty::RETURN_NAN_OR_ZERO>;
Data data;
void add(const Value &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method add without determinator is not implemented for ReservoirSamplerDeterministic");
}
template <typename Determinator>
void add(const Value & x, const Determinator & determinator)
{
data.insert(x, determinator);
}
void merge(const QuantileReservoirSamplerDeterministic & rhs)
{
data.merge(rhs.data);
}
void serialize(WriteBuffer & buf) const
{
data.write(buf);
}
void deserialize(ReadBuffer & buf)
{
data.read(buf);
}
/// Get the value of the `level` quantile. The level must be between 0 and 1.
Value get(Float64 level)
{
if (data.empty())
return {};
if constexpr (is_decimal<Value>)
return static_cast<typename Value::NativeType>(data.quantileInterpolated(level));
else
return static_cast<Value>(data.quantileInterpolated(level));
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result)
{
bool is_empty = data.empty();
for (size_t i = 0; i < size; ++i)
{
if (is_empty)
{
result[i] = Value{};
}
else
{
if constexpr (is_decimal<Value>)
result[indices[i]] = static_cast<typename Value::NativeType>(data.quantileInterpolated(levels[indices[i]]));
else
result[indices[i]] = static_cast<Value>(data.quantileInterpolated(levels[indices[i]]));
}
}
}
/// The same, but in the case of an empty state, NaN is returned.
Float64 getFloat(Float64 level)
{
return data.quantileInterpolated(level);
}
void getManyFloat(const Float64 * levels, const size_t * indices, size_t size, Float64 * result)
{
for (size_t i = 0; i < size; ++i)
result[indices[i]] = data.quantileInterpolated(levels[indices[i]]);
}
};
}

View File

@ -4,7 +4,7 @@
#include <Common/HyperLogLogBiasEstimator.h>
#include <Common/CompactArray.h>
#include <Common/HashTable/Hash.h>
#include <Common/TransformEndianness.hpp>
#include <Common/transformEndianness.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>

View File

@ -13,8 +13,6 @@
* (~ 700 MB/sec, 15 million strings per second)
*/
#include "TransformEndianness.hpp"
#include <bit>
#include <string>
#include <type_traits>
@ -24,9 +22,11 @@
#include <base/unaligned.h>
#include <base/hex.h>
#include <Common/Exception.h>
#include <Common/transformEndianness.h>
#include <city.h>
namespace DB::ErrorCodes
{
extern const int LOGICAL_ERROR;

View File

@ -19,7 +19,7 @@
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <Common/TransformEndianness.hpp>
#include <Common/transformEndianness.h>
#include <Common/memcpySmall.h>
#include <Common/typeid_cast.h>

View File

@ -17,7 +17,7 @@
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
#include <Common/LocalDateTime.h>
#include <Common/TransformEndianness.hpp>
#include <Common/transformEndianness.h>
#include <base/StringRef.h>
#include <base/arithmeticOverflow.h>
#include <base/sort.h>

View File

@ -15,7 +15,7 @@
#include <Common/DateLUT.h>
#include <Common/LocalDate.h>
#include <Common/LocalDateTime.h>
#include <Common/TransformEndianness.hpp>
#include <Common/transformEndianness.h>
#include <base/find_symbols.h>
#include <base/StringRef.h>
#include <base/DecomposedFloat.h>