added resul type deduction structs

This commit is contained in:
myrrc 2020-10-22 18:47:21 +03:00
parent 338ecb6fe1
commit 3d479cdd8c
4 changed files with 176 additions and 109 deletions

View File

@ -29,7 +29,10 @@ but may have different types.
- Weighted mean otherwise.
**Return type**
- [Float64](../../../sql-reference/data-types/float.md).
- `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.
**Example**

View File

@ -13,17 +13,6 @@ namespace ErrorCodes
namespace
{
template <class T>
using AvgNumerator = std::conditional_t<
IsDecimalNumber<T>,
std::conditional_t<std::is_same_v<T, Decimal256>,
Decimal256,
Decimal128>,
NearestFieldType<T>>;
template <typename T>
using AggregateFuncAvg = AggregateFunctionAvg<T, AvgNumerator<T>, UInt64>;
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
assertNoParameters(name, parameters);
@ -33,9 +22,9 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const
DataTypePtr data_type = argument_types[0];
if (isDecimal(data_type))
res.reset(createWithDecimalType<AggregateFuncAvg>(*data_type, *data_type, argument_types));
res.reset(createWithDecimalType<AggregateFunctionAvg>(*data_type, *data_type, argument_types));
else
res.reset(createWithNumericType<AggregateFuncAvg>(*data_type, argument_types));
res.reset(createWithNumericType<AggregateFunctionAvg>(*data_type, argument_types));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,

View File

@ -44,18 +44,57 @@ struct RationalFraction
};
/**
* @tparam Numerator The type that the initial numerator column would have (needed to cast the input IColumn to
* 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.
*
* 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 Denominator The type that the initial denominator column would have.
* @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.
*
* @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 Numerator, class Denominator, class Derived>
class AggregateFunctionAvgBase : public IAggregateFunctionDataHelper<RationalFraction<Numerator, Denominator>, Derived>
template <class InitialNumerator, class InitialDenominator, template <class, class> class Deduction, class Derived>
class AggregateFunctionAvgBase : public
IAggregateFunctionDataHelper<typename Deduction<InitialNumerator, InitialDenominator>::Fraction, Derived>
{
public:
using Base = IAggregateFunctionDataHelper<RationalFraction<Numerator, Denominator>, Derived>;
using Deducted = Deduction<InitialNumerator, InitialDenominator>;
using ResultType = typename Deducted::ResultType;
using ResultDataType = typename Deducted::ResultDataType;
using ResultVectorType = typename Deducted::ResultVectorType;
using Numerator = typename Deducted::Numerator;
using Denominator = typename Deducted::Denominator;
using Base = IAggregateFunctionDataHelper<typename Deducted::Fraction, Derived>;
/// ctor for native types
explicit AggregateFunctionAvgBase(const DataTypes & argument_types_): Base(argument_types_, {}), scale(0) {}
@ -66,7 +105,10 @@ public:
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeNumber<Float64>>();
if constexpr (IsDecimalNumber<ResultType>)
return std::make_shared<ResultDataType>(ResultDataType::maxPrecision(), scale);
else
return std::make_shared<ResultDataType>();
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
@ -97,20 +139,39 @@ public:
void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override
{
static_cast<ColumnVector<Float64> &>(to).getData().push_back(this->data(place).template result<Float64>());
static_cast<ResultVectorType &>(to).getData().push_back(this->data(place).template result<ResultType>());
}
protected:
UInt32 scale;
};
template <class InputColumn, class Numerator, class Denominator>
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<Numerator, Denominator, AggregateFunctionAvg<InputColumn, Numerator, Denominator>>
public AggregateFunctionAvgBase<InputColumn, UInt64, AvgFunctionTypesDeduction, AggregateFunctionAvg<InputColumn>>
{
public:
using AggregateFunctionAvgBase<Numerator, Denominator,
AggregateFunctionAvg<InputColumn, Numerator, Denominator>>::AggregateFunctionAvgBase;
using Base =
AggregateFunctionAvgBase<InputColumn, UInt64, AvgFunctionTypesDeduction, AggregateFunctionAvg<InputColumn>>;
using Base::Base;
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final
{

View File

@ -5,102 +5,118 @@
namespace DB
{
template <class T> struct NextAvgType { using Type = T; };
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<Int128> { using Type = Int256; };
template <> struct NextAvgType<Int256> { using Type = Int256; };
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; };
template <> struct NextAvgType<UInt128> { using Type = UInt256; };
template <> struct NextAvgType<UInt256> { using Type = UInt256; };
template <> struct NextAvgType<Decimal32> { using Type = Decimal128; };
template <> struct NextAvgType<Decimal64> { using Type = Decimal128; };
template <> struct NextAvgType<Decimal128> { using Type = Decimal256; };
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>;
template <class U, class V>
struct GetNumDenom
template <class Values, class Weights>
struct AvgWeightedFunctionTypesDeduction
{
static constexpr bool UDecimal = IsDecimalNumber<U>;
static constexpr bool VDecimal = IsDecimalNumber<V>;
static constexpr bool BothDecimal = UDecimal && VDecimal;
static constexpr bool NoneDecimal = !UDecimal && !VDecimal;
template <class T> struct NextAvgType { using Type = T; };
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<Int128> { using Type = Int256; };
template <> struct NextAvgType<Int256> { using Type = Int256; };
template <class T>
static constexpr bool IsIntegral = std::is_integral_v<T>
|| std::is_same_v<T, Int128> || std::is_same_v<T, Int256>
|| std::is_same_v<T, UInt128> || std::is_same_v<T, UInt256>;
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; };
template <> struct NextAvgType<UInt128> { using Type = UInt256; };
template <> struct NextAvgType<UInt256> { using Type = UInt256; };
static constexpr bool BothOrNoneDecimal = BothDecimal || NoneDecimal;
template <> struct NextAvgType<Decimal32> { using Type = Decimal128; };
template <> struct NextAvgType<Decimal64> { using Type = Decimal128; };
template <> struct NextAvgType<Decimal128> { using Type = Decimal256; };
template <> struct NextAvgType<Decimal256> { using Type = Decimal256; };
using Num = std::conditional<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>>,
template <> struct NextAvgType<Float32> { using Type = Float64; };
template <> struct NextAvgType<Float64> { using Type = Float64; };
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, we check the numerator (as the above case).
std::conditional_t<IsIntegral<U>,
NextAvgTypeT<U>,
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>;
/**
* 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, the numerator is either integral (so we leave the Decimal), or Float64,
* so we set the denominator to Float64;
*/
using Denom = std::conditional<VDecimal && !UDecimal && !IsIntegral<U>,
Float64,
NextAvgTypeT<V>>;
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;
template <class T>
static constexpr bool IsIntegral = std::is_integral_v<T>
|| 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<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, we check the numerator (as the above case).
std::conditional_t<IsIntegral<U>,
NextAvgTypeT<U>,
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, the numerator is either integral (so we leave the Decimal), or Float64,
* so we set the denominator to Float64;
*/
using Denom = std::conditional<VDecimal && !UDecimal && !IsIntegral<U>,
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>>;
};
template <class U, class V> using AvgWeightedNum = typename GetNumDenom<U, V>::Num;
template <class U, class V> using AvgWeightedDenom = typename GetNumDenom<U, V>::Denom;
template<class Num, class Denom, class Derived>
using AggFuncAvgWeightedBase = AggregateFunctionAvgBase<
AvgWeightedNum<Num, Denom>,
AvgWeightedDenom<Num, Denom>, Derived>;
/**
* @tparam Values The values column type.
* @tparam Weights The weights column type.
*/
template <class Values, class Weights>
class AggregateFunctionAvgWeighted final :
AggFuncAvgWeightedBase<Values, Weights, AggregateFunctionAvgWeighted<Values, Weights>>
class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase<
Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted<Values, Weights>>
{
public:
using AggFuncAvgWeightedBase<Values, Weights, AggregateFunctionAvgWeighted<Values, Weights>>
::AggFuncAvgWeightedBase;
using Base = AggregateFunctionAvgBase<
Values, Weights, AvgWeightedFunctionTypesDeduction, AggregateFunctionAvgWeighted<Values, Weights>>;
using Base::Base;
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
@ -110,9 +126,7 @@ public:
const auto value = values.getData()[row_num];
const auto weight = weights.getData()[row_num];
using TargetNum = AvgWeightedNum<Values, Weights>;
this->data(place).numerator += static_cast<TargetNum>(value) * weight;
this->data(place).numerator += static_cast<typename Base::Numerator>(value) * weight;
this->data(place).denominator += weight;
}