simplified the functions (agreement to cast to Float64)

This commit is contained in:
myrrc 2020-10-25 23:33:01 +03:00
parent f18232739b
commit d8370116c1
7 changed files with 49 additions and 267 deletions

View File

@ -4,4 +4,5 @@ toc_priority: 5
# avg {#agg_function-avg}
Calculates the average. Only works for numbers. The result is always Float64.
Calculates the average. Only works for numbers (Integral, floating-point, or Decimals).
The result is always Float64.

View File

@ -28,11 +28,7 @@ but may have different types.
- `NaN`. If all the weights are equal to 0.
- Weighted mean otherwise.
**Return type**
- `Decimal` if both types are [Decimal](../../../sql-reference/data-types/decimal.md)
or if one type is Decimal and other is Integer.
- [Float64](../../../sql-reference/data-types/float.md) otherwise.
**Return type** is always [Float64](../../../sql-reference/data-types/float.md).
**Example**

View File

@ -1,3 +1,4 @@
#include <memory>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionAvg.h>
#include <AggregateFunctions/Helpers.h>
@ -13,23 +14,22 @@ namespace ErrorCodes
namespace
{
constexpr bool allowType(const DataTypePtr& type) noexcept
{
const WhichDataType t(type);
return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal();
}
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
assertNoParameters(name, parameters);
assertUnary(name, argument_types);
AggregateFunctionPtr res;
DataTypePtr data_type = argument_types[0];
if (isDecimal(data_type))
res.reset(createWithDecimalType<AggregateFunctionAvg>(*data_type, *data_type, argument_types));
else
res.reset(createWithNumericType<AggregateFunctionAvg>(*data_type, argument_types));
if (!res)
if (!allowType(argument_types[0]))
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<AggregateFunctionAvg>(argument_types);
}
}

View File

@ -10,10 +10,7 @@
namespace DB
{
template <class T>
using DecimalOrVectorCol = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
/// A type-fixed rational fraction represented by a pair of #Numerator and #Denominator.
/// A type-fixed fraction represented by a pair of #Numerator and #Denominator.
template <class Numerator, class Denominator>
struct RationalFraction
{
@ -26,77 +23,42 @@ struct RationalFraction
template <class Result>
Result NO_SANITIZE_UNDEFINED result() const
{
if constexpr (std::is_floating_point_v<Result>)
if constexpr (std::numeric_limits<Result>::is_iec559)
{
if constexpr (is_big_int_v<Denominator>)
return static_cast<Result>(numerator) / static_cast<Result>(denominator);
else
return static_cast<Result>(numerator) / denominator; /// allow division by zero
}
if constexpr (std::is_floating_point_v<Result> && std::numeric_limits<Result>::is_iec559)
return static_cast<Result>(numerator) / denominator; /// allow division by zero
if (denominator == static_cast<Denominator>(0))
return static_cast<Result>(0);
if constexpr (std::is_same_v<Numerator, Decimal256>)
return static_cast<Result>(numerator / static_cast<Numerator>(denominator));
else
return static_cast<Result>(numerator / denominator);
return static_cast<Result>(numerator / denominator);
}
};
/**
* Motivation: ClickHouse has added the Decimal data type, which basically represents a fraction that stores
* the precise (unlike floating-point) result with respect to some scale.
* The discussion showed that the easiest (and simplest) way is to cast both the columns of numerator and denominator
* to Float64. Another way would be to write some template magic that figures out the appropriate numerator
* and denominator (and the resulting type) in favour of extended integral types (UInt128 e.g.) and Decimals (
* which are a mess themselves). The second way is also a bit useless because now Decimals are not used in functions
* like avg.
*
* These decimal types can't be divided by floating point data types, so functions like avg or avgWeighted
* can't return the Floa64 column as a result when of the input columns is Decimal (because that would, in case of
* avgWeighted, involve division numerator (Decimal) / denominator (Float64)).
*
* The rules for determining the output and intermediate storage types for these functions are different, so
* the struct representing the deduction guide is presented.
*
* Given the initial Columns types (e.g. values and weights for avgWeighted, values for avg),
* the struct calculated the output type and the intermediate storage type (that's used by the RationalFraction).
*/
template <class Column1, class Column2>
struct AvgFunctionTypesDeductionTemplate
{
using Numerator = int;
using Denominator = int;
using Fraction = RationalFraction<Numerator, Denominator>;
using ResultType = bool;
using ResultDataType = bool;
using ResultVectorType = bool;
};
/**
* @tparam InitialNumerator The type that the initial numerator column would have (needed to cast the input IColumn to
* appropriate type).
* @tparam InitialDenominator The type that the initial denominator column would have.
*
* @tparam Deduction Function template that, given the numerator and the denominator, finds the actual
* suitable storage and the resulting column type.
* The ability to explicitly specify the denominator is made for avg (it uses the integral value as the denominator is
* simply the length of the supplied list).
*
* @tparam Derived When deriving from this class, use the child class name as in CRTP, e.g.
* class Self : Agg<char, bool, bool, Self>.
*/
template <class InitialNumerator, class InitialDenominator, template <class, class> class Deduction, class Derived>
template <class Denominator, class Derived>
class AggregateFunctionAvgBase : public
IAggregateFunctionDataHelper<typename Deduction<InitialNumerator, InitialDenominator>::Fraction, Derived>
IAggregateFunctionDataHelper<RationalFraction<Float64, Denominator>, Derived>
{
public:
using Deducted = Deduction<InitialNumerator, InitialDenominator>;
using Numerator = Float64;
using Fraction = RationalFraction<Numerator, Denominator>;
using ResultType = typename Deducted::ResultType;
using ResultDataType = typename Deducted::ResultDataType;
using ResultVectorType = typename Deducted::ResultVectorType;
using ResultType = Float64;
using ResultDataType = DataTypeNumber<Float64>;
using ResultVectorType = ColumnVector<Float64>;
using Numerator = typename Deducted::Numerator;
using Denominator = typename Deducted::Denominator;
using Base = IAggregateFunctionDataHelper<typename Deducted::Fraction, Derived>;
using Base = IAggregateFunctionDataHelper<Fraction, Derived>;
/// ctor for native types
explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}), scale(0) {}
@ -107,10 +69,7 @@ public:
DataTypePtr getReturnType() const override
{
if constexpr (IsDecimalNumber<ResultType>)
return std::make_shared<ResultDataType>(ResultDataType::maxPrecision(), scale);
else
return std::make_shared<ResultDataType>();
return std::make_shared<ResultDataType>();
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
@ -148,38 +107,16 @@ protected:
UInt32 scale;
};
template <class T, class V>
struct AvgFunctionTypesDeduction
{
using Numerator = std::conditional_t<IsDecimalNumber<T>,
std::conditional_t<std::is_same_v<T, Decimal256>,
Decimal256,
Decimal128>,
NearestFieldType<T>>;
using Denominator = V;
using Fraction = RationalFraction<Numerator, Denominator>;
using ResultType = std::conditional_t<IsDecimalNumber<T>, T, Float64>;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<T>, DataTypeNumber<Float64>>;
using ResultVectorType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<Float64>>;
};
template <class InputColumn>
class AggregateFunctionAvg final :
public AggregateFunctionAvgBase<InputColumn, UInt64, AvgFunctionTypesDeduction, AggregateFunctionAvg<InputColumn>>
class AggregateFunctionAvg final : public AggregateFunctionAvgBase<UInt64, AggregateFunctionAvg>
{
public:
using Base =
AggregateFunctionAvgBase<InputColumn, UInt64, AvgFunctionTypesDeduction, AggregateFunctionAvg<InputColumn>>;
using Base::Base;
using AggregateFunctionAvgBase<UInt64, AggregateFunctionAvg>::AggregateFunctionAvgBase;
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final
{
const auto & column = static_cast<const DecimalOrVectorCol<InputColumn> &>(*columns[0]);
const auto & column = static_cast<const ColumnVector<Float64> &>(*columns[0]);
this->data(place).numerator += column.getData()[row_num];
this->data(place).denominator += 1;
++this->data(place).denominator;
}
String getName() const final { return "avg"; }

View File

@ -1,3 +1,4 @@
#include <memory>
#include <type_traits>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionAvgWeighted.h>
@ -14,7 +15,7 @@ namespace ErrorCodes
namespace
{
constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right)
constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept
{
const WhichDataType l_dt(left), r_dt(right);
@ -26,39 +27,6 @@ constexpr bool allowTypes(const DataTypePtr& left, const DataTypePtr& right)
return allow(l_dt) && allow(r_dt);
}
#define AT_SWITCH(LINE) \
switch (which.idx) \
{ \
LINE(Int8); LINE(Int16); LINE(Int32); LINE(Int64); LINE(Int128); LINE(Int256); \
LINE(UInt8); LINE(UInt16); LINE(UInt32); LINE(UInt64); LINE(UInt128); LINE(UInt256); \
LINE(Decimal32); LINE(Decimal64); LINE(Decimal128); LINE(Decimal256); \
LINE(Float32); LINE(Float64); \
default: return nullptr; \
}
template <class First, class ... TArgs>
static IAggregateFunction * create(const IDataType & second_type, TArgs && ... args)
{
const WhichDataType which(second_type);
#define LINE(Type) \
case TypeIndex::Type: return new AggregateFunctionAvgWeighted<First, Type>(std::forward<TArgs>(args)...)
AT_SWITCH(LINE)
#undef LINE
}
// Not using helper functions because there are no templates for binary decimal/numeric function.
template <class... TArgs>
static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
{
const WhichDataType which(first_type);
#define LINE(Type) \
case TypeIndex::Type: return create<Type, TArgs...>(second_type, std::forward<TArgs>(args)...)
AT_SWITCH(LINE)
#undef LINE
}
AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
assertNoParameters(name, parameters);
@ -74,11 +42,7 @@ AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name
" are non-conforming as arguments for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
AggregateFunctionPtr res;
res.reset(create(*data_type, *data_type_weight, argument_types));
assert(res); // type checking should be done in allowTypes.
return res;
return std::make_shared<AggregateFunctionAvgWeighted>(argument_types);
}
}

View File

@ -5,134 +5,18 @@
namespace DB
{
template <class Values, class Weights>
struct AvgWeightedFunctionTypesDeduction
{
template <class> struct NextAvgType { };
template <> struct NextAvgType<Int8> { using Type = Int16; };
template <> struct NextAvgType<Int16> { using Type = Int32; };
template <> struct NextAvgType<Int32> { using Type = Int64; };
template <> struct NextAvgType<Int64> { using Type = Int128; };
template <> struct NextAvgType<UInt8> { using Type = UInt16; };
template <> struct NextAvgType<UInt16> { using Type = UInt32; };
template <> struct NextAvgType<UInt32> { using Type = UInt64; };
template <> struct NextAvgType<UInt64> { using Type = UInt128; };
// Promoted to Float as these types don't go well when operating with above ones
template <> struct NextAvgType<UInt128> { using Type = Float64; };
template <> struct NextAvgType<UInt256> { using Type = Float64; };
template <> struct NextAvgType<Int128> { using Type = Float64; };
template <> struct NextAvgType<Int256> { using Type = Float64; };
template <> struct NextAvgType<Decimal32> { using Type = Decimal128; };
template <> struct NextAvgType<Decimal64> { using Type = Decimal128; };
template <> struct NextAvgType<Decimal128> { using Type = Decimal128; };
template <> struct NextAvgType<Decimal256> { using Type = Decimal256; };
template <> struct NextAvgType<Float32> { using Type = Float64; };
template <> struct NextAvgType<Float64> { using Type = Float64; };
template <class T> using NextAvgTypeT = typename NextAvgType<T>::Type;
template <class T, class U> using Largest = std::conditional_t<(sizeof(T) > sizeof(U)), T, U>;
struct GetNumDenom
{
using U = Values;
using V = Weights;
static constexpr bool UDecimal = IsDecimalNumber<U>;
static constexpr bool VDecimal = IsDecimalNumber<V>;
static constexpr bool BothDecimal = UDecimal && VDecimal;
static constexpr bool NoneDecimal = !UDecimal && !VDecimal;
/// we do not include extended integral types here as they produce errors while diving on Decimals.
template <class T> static constexpr bool IsIntegral = std::is_integral_v<T>;
template <class T> static constexpr bool IsExtendedIntegral =
std::is_same_v<T, Int128> || std::is_same_v<T, Int256>
|| std::is_same_v<T, UInt128> || std::is_same_v<T, UInt256>;
static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal;
using Num = std::conditional_t<BothOrNoneDecimal,
/// When both types are Decimal, we can perform computations in the Decimals only.
/// When none of the types is Decimal, the result is always correct, the numerator is the next largest type up
/// to Float64.
NextAvgTypeT<Largest<U, V>>,
std::conditional_t<UDecimal,
/// When the numerator only is Decimal, we have to check the denominator:
/// - If it's non-floating point, then we can set the numerator as the next Largest decimal.
/// - Otherwise we won't be able to divide Decimal by double, so we leave the numerator as Float64.
std::conditional_t<IsIntegral<V>,
NextAvgTypeT<U>,
Float64>,
/// When the denominator only is Decimal, it would be converted to Float64 (as integral / Decimal
/// produces a compile error, vice versa allowed), so we just cast the numerator to Flaoat64;
Float64>>;
/**
* When both types are Decimal, we can perform computations in the Decimals only.
* When none of the types is Decimal, the result is always correct, the numerator is the next largest type up to
* Float64.
* We use #V only as the denominator accumulates the sum of the weights.
*
* When the numerator only is Decimal, we set the denominator to next Largest type.
* - If the denominator was floating-point, the numerator would be Float64.
* - If not, the numerator would be Decimal (as the denominator is integral).
*
* When the denominator only is Decimal, it will be casted to Float64 as integral / Decimal produces a compile
* time error.
*
* Extended integer types can't be multiplied by doubles (I don't know, why), so we also convert them to
* double.
*/
using Denom = std::conditional_t<(VDecimal && !UDecimal) || IsExtendedIntegral<V>,
Float64,
NextAvgTypeT<V>>;
};
using Numerator = typename GetNumDenom::Num;
using Denominator = typename GetNumDenom::Denom;
using Fraction = RationalFraction<Numerator, Denominator>;
/// If either Numerator or Denominator are Decimal, the result is also Decimal as everything was checked in
/// GetNumDenom.
using T = std::conditional_t<IsDecimalNumber<Numerator> && IsDecimalNumber<Denominator>,
Largest<Numerator, Denominator>,
std::conditional_t<IsDecimalNumber<Numerator>,
Numerator,
std::conditional_t<IsDecimalNumber<Denominator>,
Denominator,
bool>>>; // both numerator and denominator are non-decimal.
using ResultType = std::conditional_t<IsDecimalNumber<T>, T, Float64>;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<T>, DataTypeNumber<Float64>>;
using ResultVectorType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<Float64>>;
};
/**
* @tparam Values The values column type.
* @tparam Weights The weights column type.
*/
template <class Values, class Weights>
class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase<
Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted<Values, Weights>>
class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase<Float64, AggregateFunctionAvgWeighted>
{
public:
using Base = AggregateFunctionAvgBase<
Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted<Values, Weights>>;
using Base::Base;
using AggregateFunctionAvgBase<Float64, AggregateFunctionAvgWeighted>::AggregateFunctionAvgBase;
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
const auto & values = static_cast<const DecimalOrVectorCol<Values> &>(*columns[0]);
const auto & weights = static_cast<const DecimalOrVectorCol<Weights> &>(*columns[1]);
const auto & values = static_cast<const ColumnVector<Float64> &>(*columns[0]);
const auto & weights = static_cast<const ColumnVector<Float64> &>(*columns[1]);
using Numerator = typename Base::Numerator;
using Denominator = typename Base::Denominator;
const Numerator value = Numerator(values.getData()[row_num]);
const Denominator weight = Denominator(weights.getData()[row_num]);
const auto value = values.getData()[row_num];
const auto weight = weights.getData()[row_num];
this->data(place).numerator += value * weight;
this->data(place).denominator += weight;

View File

@ -21,7 +21,8 @@ class IDataType;
using DataTypePtr = std::shared_ptr<const IDataType>;
using DataTypes = std::vector<DataTypePtr>;
/** Creator have arguments: name of aggregate function, types of arguments, values of parameters.
/**
* The invoker has arguments: name of aggregate function, types of arguments, values of parameters.
* Parameters are for "parametric" aggregate functions.
* For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments".
*/
@ -89,7 +90,6 @@ private:
std::optional<AggregateFunctionProperties> tryGetPropertiesImpl(const String & name, int recursion_level) const;
private:
using AggregateFunctions = std::unordered_map<String, Value>;
AggregateFunctions aggregate_functions;