Merge pull request #15419 from myrrc/improvement/diff-types-in-avg-weighted

Allow different types in avgWeighted. Allow avg and avgWeighed to operate on extended integral types.
This commit is contained in:
Alexander Kuzmenkov 2020-11-26 17:16:48 +03:00 committed by GitHub
commit 15a0f14445
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 1304 additions and 687 deletions

View File

@ -1,3 +1,4 @@
# Needed when using Apache Avro serialization format
option (ENABLE_AVRO "Enable Avro" ${ENABLE_LIBRARIES})
if (NOT ENABLE_AVRO)

View File

@ -1,3 +1,5 @@
# Needed when securely connecting to an external server, e.g.
# clickhouse-client --host ... --secure
option(ENABLE_SSL "Enable ssl" ${ENABLE_LIBRARIES})
if(NOT ENABLE_SSL)

View File

@ -415,4 +415,4 @@ if not args.keep_created_tables and not args.use_existing_tables:
c.execute(q)
print(f'drop\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}')
reportStageEnd('drop-2')
reportStageEnd('drop-2')

View File

@ -13,9 +13,9 @@ cmake .. \
-DENABLE_CLICKHOUSE_SERVER=ON \
-DENABLE_CLICKHOUSE_CLIENT=ON \
-DUSE_STATIC_LIBRARIES=OFF \
-DCLICKHOUSE_SPLIT_BINARY=ON \
-DSPLIT_SHARED_LIBRARIES=ON \
-DENABLE_LIBRARIES=OFF \
-DUSE_UNWIND=ON \
-DENABLE_UTILS=OFF \
-DENABLE_TESTS=OFF
```

View File

@ -4,4 +4,59 @@ toc_priority: 5
# avg {#agg_function-avg}
Calculates the average. Only works for numbers. The result is always Float64.
Calculates the arithmetic mean.
**Syntax**
``` sql
avgWeighted(x)
```
**Parameter**
- `x` — Values.
`x` must be
[Integer](../../../sql-reference/data-types/int-uint.md),
[floating-point](../../../sql-reference/data-types/float.md), or
[Decimal](../../../sql-reference/data-types/decimal.md).
**Returned value**
- `NaN` if the supplied parameter is empty.
- Mean otherwise.
**Return type** is always [Float64](../../../sql-reference/data-types/float.md).
**Example**
Query:
``` sql
SELECT avg(x) FROM values('x Int8', 0, 1, 2, 3, 4, 5)
```
Result:
``` text
┌─avg(x)─┐
│ 2.5 │
└────────┘
```
**Example**
Query:
``` sql
CREATE table test (t UInt8) ENGINE = Memory;
SELECT avg(t) FROM test
```
Result:
``` text
┌─avg(x)─┐
│ nan │
└────────┘
```

View File

@ -14,17 +14,21 @@ avgWeighted(x, weight)
**Parameters**
- `x` — Values. [Integer](../../../sql-reference/data-types/int-uint.md) or [floating-point](../../../sql-reference/data-types/float.md).
- `weight` — Weights of the values. [Integer](../../../sql-reference/data-types/int-uint.md) or [floating-point](../../../sql-reference/data-types/float.md).
- `x` — Values.
- `weight` — Weights of the values.
Type of `x` and `weight` must be the same.
`x` and `weight` must both be
[Integer](../../../sql-reference/data-types/int-uint.md),
[floating-point](../../../sql-reference/data-types/float.md), or
[Decimal](../../../sql-reference/data-types/decimal.md),
but may have different types.
**Returned value**
- Weighted mean.
- `NaN`. If all the weights are equal to 0.
- `NaN` if all the weights are equal to 0 or the supplied weights parameter is empty.
- Weighted mean otherwise.
Type: [Float64](../../../sql-reference/data-types/float.md).
**Return type** is always [Float64](../../../sql-reference/data-types/float.md).
**Example**
@ -42,3 +46,54 @@ Result:
│ 8 │
└────────────────────────┘
```
**Example**
Query:
``` sql
SELECT avgWeighted(x, w)
FROM values('x Int8, w Float64', (4, 1), (1, 0), (10, 2))
```
Result:
``` text
┌─avgWeighted(x, weight)─┐
│ 8 │
└────────────────────────┘
```
**Example**
Query:
``` sql
SELECT avgWeighted(x, w)
FROM values('x Int8, w Int8', (0, 0), (1, 0), (10, 0))
```
Result:
``` text
┌─avgWeighted(x, weight)─┐
│ nan │
└────────────────────────┘
```
**Example**
Query:
``` sql
CREATE table test (t UInt8) ENGINE = Memory;
SELECT avgWeighted(t) FROM test
```
Result:
``` text
┌─avgWeighted(x, weight)─┐
│ nan │
└────────────────────────┘
```

View File

@ -43,13 +43,81 @@ else ()
${ENABLE_CLICKHOUSE_ALL})
endif ()
message(STATUS "ClickHouse modes:")
if (NOT ENABLE_CLICKHOUSE_SERVER)
message(WARNING "ClickHouse server mode is not going to be built.")
else()
message(STATUS "Server mode: ON")
endif()
if (NOT ENABLE_CLICKHOUSE_CLIENT)
message(WARNING "ClickHouse client mode is not going to be built. You won't be able to connect to the server and run
tests")
else()
message(STATUS "Client mode: ON")
endif()
if (ENABLE_CLICKHOUSE_LOCAL)
message(STATUS "Local mode: ON")
else()
message(STATUS "Local mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_BENCHMARK)
message(STATUS "Benchmark mode: ON")
else()
message(STATUS "Benchmark mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG)
message(STATUS "Extract from config mode: ON")
else()
message(STATUS "Extract from config mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_COMPRESSOR)
message(STATUS "Compressor mode: ON")
else()
message(STATUS "Compressor mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_COPIER)
message(STATUS "Copier mode: ON")
else()
message(STATUS "Copier mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_FORMAT)
message(STATUS "Format mode: ON")
else()
message(STATUS "Format mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_OBFUSCATOR)
message(STATUS "Obfuscator mode: ON")
else()
message(STATUS "Obfuscator mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_ODBC_BRIDGE)
message(STATUS "ODBC bridge mode: ON")
else()
message(STATUS "ODBC bridge mode: OFF")
endif()
if (ENABLE_CLICKHOUSE_INSTALL)
message(STATUS "ClickHouse install: ON")
else()
message(STATUS "ClickHouse install: OFF")
endif()
if(NOT (MAKE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES))
set(CLICKHOUSE_ONE_SHARED ON)
endif()
configure_file (config_tools.h.in ${ConfigIncludePath}/config_tools.h)
macro(clickhouse_target_link_split_lib target name)
if(NOT CLICKHOUSE_ONE_SHARED)
target_link_libraries(${target} PRIVATE clickhouse-${name}-lib)

View File

@ -1,3 +1,4 @@
#include <memory>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionAvg.h>
#include <AggregateFunctions/Helpers.h>
@ -13,43 +14,37 @@ namespace ErrorCodes
namespace
{
template <typename T>
struct Avg
bool allowType(const DataTypePtr& type) noexcept
{
using FieldType = std::conditional_t<IsDecimalNumber<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
NearestFieldType<T>>;
// using FieldType = std::conditional_t<IsDecimalNumber<T>, Decimal128, NearestFieldType<T>>;
using Function = AggregateFunctionAvg<T, AggregateFunctionAvgData<FieldType, UInt64>>;
};
template <typename T>
using AggregateFuncAvg = typename Avg<T>::Function;
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<AggregateFuncAvg>(*data_type, *data_type, argument_types));
else
res.reset(createWithNumericType<AggregateFuncAvg>(*data_type, argument_types));
const DataTypePtr& data_type = argument_types[0];
if (!allowType(data_type))
throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
AggregateFunctionPtr res;
if (isDecimal(data_type))
res.reset(createWithDecimalType<AggregateFunctionAvg>(
*data_type, argument_types, getDecimalScale(*data_type)));
else
res.reset(createWithNumericType<AggregateFunctionAvg>(*data_type, argument_types));
if (!res)
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory)
{
factory.registerFunction("avg", createAggregateFunctionAvg, AggregateFunctionFactory::CaseInsensitive);
}
}

View File

@ -1,78 +1,102 @@
#pragma once
#include <type_traits>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include "Core/DecimalFunctions.h"
namespace DB
{
namespace ErrorCodes
{
}
template <class T>
using DecimalOrVectorCol = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
template <typename T, typename Denominator>
struct AggregateFunctionAvgData
{
using NumeratorType = T;
using DenominatorType = Denominator;
template <class T> constexpr bool DecimalOrExtendedInt =
IsDecimalNumber<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>;
T numerator{0};
/**
* Helper class to encapsulate values conversion for avg and avgWeighted.
*/
template <class Numerator, class Denominator>
struct AvgFraction
{
Numerator numerator{0};
Denominator denominator{0};
template <typename ResultT>
ResultT NO_SANITIZE_UNDEFINED result() const
/// Allow division by zero as sometimes we need to return NaN.
/// Invoked only is either Numerator or Denominator are Decimal.
Float64 NO_SANITIZE_UNDEFINED divideIfAnyDecimal(UInt32 num_scale, UInt32 denom_scale) const
{
if constexpr (std::is_floating_point_v<ResultT>)
if constexpr (std::numeric_limits<ResultT>::is_iec559)
{
if constexpr (is_big_int_v<Denominator>)
return static_cast<ResultT>(numerator) / static_cast<ResultT>(denominator);
else
return static_cast<ResultT>(numerator) / denominator; /// allow division by zero
}
if constexpr (IsDecimalNumber<Numerator> && IsDecimalNumber<Denominator>)
{
// According to the docs, num(S1) / denom(S2) would have scale S1
if (denominator == static_cast<Denominator>(0))
return static_cast<ResultT>(0);
if constexpr (std::is_same_v<Numerator, Decimal256> && std::is_same_v<Denominator, Decimal128>)
///Special case as Decimal256 / Decimal128 = compile error (as Decimal128 is not parametrized by a wide
///int), but an __int128 instead
return DecimalUtils::convertTo<Float64>(
numerator / (denominator.template convertTo<Decimal256>()), num_scale);
else
return DecimalUtils::convertTo<Float64>(numerator / denominator, num_scale);
}
if constexpr (std::is_same_v<T, Decimal256>)
return static_cast<ResultT>(numerator / static_cast<T>(denominator));
/// Numerator is always casted to Float64 to divide correctly if the denominator is not Float64.
Float64 num_converted;
if constexpr (IsDecimalNumber<Numerator>)
num_converted = DecimalUtils::convertTo<Float64>(numerator, num_scale);
else
return static_cast<ResultT>(numerator / denominator);
num_converted = static_cast<Float64>(numerator); /// all other types, including extended integral.
std::conditional_t<DecimalOrExtendedInt<Denominator>,
Float64, Denominator> denom_converted;
if constexpr (IsDecimalNumber<Denominator>)
denom_converted = DecimalUtils::convertTo<Float64>(denominator, denom_scale);
else if constexpr (DecimalOrExtendedInt<Denominator>)
/// no way to divide Float64 and extended integral type without an explicit cast.
denom_converted = static_cast<Float64>(denominator);
else
denom_converted = denominator; /// can divide on float, no cast required.
return num_converted / denom_converted;
}
Float64 NO_SANITIZE_UNDEFINED divide() const
{
if constexpr (DecimalOrExtendedInt<Denominator>) /// if extended int
return static_cast<Float64>(numerator) / static_cast<Float64>(denominator);
else
return static_cast<Float64>(numerator) / denominator;
}
};
/// Calculates arithmetic mean of numbers.
template <typename T, typename Data, typename Derived>
class AggregateFunctionAvgBase : public IAggregateFunctionDataHelper<Data, Derived>
/**
* @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<AvgFraction<Numerator, Denominator>, Derived>
{
public:
using ResultType = std::conditional_t<IsDecimalNumber<T>, T, Float64>;
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<T>, DataTypeNumber<Float64>>;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<Float64>>;
using Fraction = AvgFraction<Numerator, Denominator>;
using Base = IAggregateFunctionDataHelper<Fraction, Derived>;
/// ctor for native types
AggregateFunctionAvgBase(const DataTypes & argument_types_) : IAggregateFunctionDataHelper<Data, Derived>(argument_types_, {}), scale(0) {}
explicit AggregateFunctionAvgBase(const DataTypes & argument_types_,
UInt32 num_scale_ = 0, UInt32 denom_scale_ = 0)
: Base(argument_types_, {}), num_scale(num_scale_), denom_scale(denom_scale_) {}
/// ctor for Decimals
AggregateFunctionAvgBase(const IDataType & data_type, const DataTypes & argument_types_)
: IAggregateFunctionDataHelper<Data, Derived>(argument_types_, {}), scale(getDecimalScale(data_type))
{
}
DataTypePtr getReturnType() const override
{
if constexpr (IsDecimalNumber<T>)
return std::make_shared<ResultDataType>(ResultDataType::maxPrecision(), scale);
else
return std::make_shared<ResultDataType>();
}
DataTypePtr getReturnType() const final { return std::make_shared<DataTypeNumber<Float64>>(); }
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
{
@ -84,7 +108,7 @@ public:
{
writeBinary(this->data(place).numerator, buf);
if constexpr (std::is_unsigned_v<typename Data::DenominatorType>)
if constexpr (std::is_unsigned_v<Denominator>)
writeVarUInt(this->data(place).denominator, buf);
else /// Floating point denominator type can be used
writeBinary(this->data(place).denominator, buf);
@ -94,7 +118,7 @@ public:
{
readBinary(this->data(place).numerator, buf);
if constexpr (std::is_unsigned_v<typename Data::DenominatorType>)
if constexpr (std::is_unsigned_v<Denominator>)
readVarUInt(this->data(place).denominator, buf);
else /// Floating point denominator type can be used
readBinary(this->data(place).denominator, buf);
@ -102,29 +126,34 @@ public:
void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override
{
auto & column = static_cast<ColVecResult &>(to);
column.getData().push_back(this->data(place).template result<ResultType>());
if constexpr (IsDecimalNumber<Numerator> || IsDecimalNumber<Denominator>)
static_cast<ColumnVector<Float64> &>(to).getData().push_back(
this->data(place).divideIfAnyDecimal(num_scale, denom_scale));
else
static_cast<ColumnVector<Float64> &>(to).getData().push_back(this->data(place).divide());
}
protected:
UInt32 scale;
private:
UInt32 num_scale;
UInt32 denom_scale;
};
template <typename T, typename Data>
class AggregateFunctionAvg final : public AggregateFunctionAvgBase<T, Data, AggregateFunctionAvg<T, Data>>
template <class T>
using AvgFieldType = std::conditional_t<IsDecimalNumber<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
NearestFieldType<T>>;
template <class T>
class AggregateFunctionAvg final : public AggregateFunctionAvgBase<AvgFieldType<T>, UInt64, AggregateFunctionAvg<T>>
{
public:
using AggregateFunctionAvgBase<T, Data, AggregateFunctionAvg<T, Data>>::AggregateFunctionAvgBase;
using AggregateFunctionAvgBase<AvgFieldType<T>, UInt64, AggregateFunctionAvg<T>>::AggregateFunctionAvgBase;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const final
{
const auto & column = static_cast<const ColVecType &>(*columns[0]);
this->data(place).numerator += column.getData()[row_num];
this->data(place).denominator += 1;
this->data(place).numerator += static_cast<const DecimalOrVectorCol<T> &>(*columns[0]).getData()[row_num];
++this->data(place).denominator;
}
String getName() const override { return "avg"; }
String getName() const final { return "avg"; }
};
}

View File

@ -1,3 +1,5 @@
#include <memory>
#include <type_traits>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionAvgWeighted.h>
#include <AggregateFunctions/Helpers.h>
@ -13,47 +15,91 @@ namespace ErrorCodes
namespace
{
template <typename T>
struct AvgWeighted
bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept
{
using FieldType = std::conditional_t<IsDecimalNumber<T>,
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
NearestFieldType<T>>;
// using FieldType = std::conditional_t<IsDecimalNumber<T>, Decimal128, NearestFieldType<T>>;
using Function = AggregateFunctionAvgWeighted<T, AggregateFunctionAvgData<FieldType, FieldType>>;
};
const WhichDataType l_dt(left), r_dt(right);
template <typename T>
using AggregateFuncAvgWeighted = typename AvgWeighted<T>::Function;
constexpr auto allow = [](WhichDataType t)
{
return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal();
};
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);
assertBinary(name, argument_types);
AggregateFunctionPtr res;
const auto data_type = static_cast<const DataTypePtr>(argument_types[0]);
const auto data_type_weight = static_cast<const DataTypePtr>(argument_types[1]);
if (!data_type->equals(*data_type_weight))
throw Exception("Different types " + data_type->getName() + " and " + data_type_weight->getName() + " of arguments for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (isDecimal(data_type))
res.reset(createWithDecimalType<AggregateFuncAvgWeighted>(*data_type, *data_type, argument_types));
if (!allowTypes(data_type, data_type_weight))
throw Exception(
"Types " + data_type->getName() +
" and " + data_type_weight->getName() +
" are non-conforming as arguments for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
AggregateFunctionPtr ptr;
const bool left_decimal = isDecimal(data_type);
const bool right_decimal = isDecimal(data_type_weight);
if (left_decimal && right_decimal)
ptr.reset(create(*data_type, *data_type_weight,
argument_types,
getDecimalScale(*data_type), getDecimalScale(*data_type_weight)));
else if (left_decimal)
ptr.reset(create(*data_type, *data_type_weight, argument_types,
getDecimalScale(*data_type)));
else if (right_decimal)
ptr.reset(create(*data_type, *data_type_weight, argument_types,
// numerator is not decimal, so its scale is 0
0, getDecimalScale(*data_type_weight)));
else
res.reset(createWithNumericType<AggregateFuncAvgWeighted>(*data_type, argument_types));
ptr.reset(create(*data_type, *data_type_weight, argument_types));
if (!res)
throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
return ptr;
}
}
void registerAggregateFunctionAvgWeighted(AggregateFunctionFactory & factory)
{
factory.registerFunction("avgWeighted", createAggregateFunctionAvgWeighted, AggregateFunctionFactory::CaseSensitive);
}
}

View File

@ -1,26 +1,44 @@
#pragma once
#include <type_traits>
#include <AggregateFunctions/AggregateFunctionAvg.h>
namespace DB
{
template <typename T, typename Data>
class AggregateFunctionAvgWeighted final : public AggregateFunctionAvgBase<T, Data, AggregateFunctionAvgWeighted<T, Data>>
template <class T>
using AvgWeightedFieldType = std::conditional_t<IsDecimalNumber<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 <class T, class U>
using MaxFieldType = std::conditional_t<(sizeof(AvgWeightedFieldType<T>) > sizeof(AvgWeightedFieldType<U>)),
AvgWeightedFieldType<T>, AvgWeightedFieldType<U>>;
template <class Value, class Weight>
class AggregateFunctionAvgWeighted final :
public AggregateFunctionAvgBase<
MaxFieldType<Value, Weight>, AvgWeightedFieldType<Weight>, AggregateFunctionAvgWeighted<Value, Weight>>
{
public:
using AggregateFunctionAvgBase<T, Data, AggregateFunctionAvgWeighted<T, Data>>::AggregateFunctionAvgBase;
using Base = AggregateFunctionAvgBase<
MaxFieldType<Value, Weight>, AvgWeightedFieldType<Weight>, AggregateFunctionAvgWeighted<Value, Weight>>;
using Base::Base;
using ValueT = MaxFieldType<Value, Weight>;
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override
{
const auto & values = static_cast<const ColVecType &>(*columns[0]);
const auto & weights = static_cast<const ColVecType &>(*columns[1]);
const auto& weights = static_cast<const DecimalOrVectorCol<Weight> &>(*columns[1]);
this->data(place).numerator += static_cast<typename Data::NumeratorType>(values.getData()[row_num]) * weights.getData()[row_num];
this->data(place).denominator += weights.getData()[row_num];
this->data(place).numerator += static_cast<ValueT>(
static_cast<const DecimalOrVectorCol<Value> &>(*columns[0]).getData()[row_num]) *
static_cast<ValueT>(weights.getData()[row_num]);
this->data(place).denominator += static_cast<AvgWeightedFieldType<Weight>>(weights.getData()[row_num]);
}
String getName() const override { return "avgWeighted"; }
};
}

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".
*/
@ -87,7 +88,6 @@ private:
std::optional<AggregateFunctionProperties> tryGetPropertiesImpl(const String & name) const;
private:
using AggregateFunctions = std::unordered_map<String, Value>;
AggregateFunctions aggregate_functions;

View File

@ -15,6 +15,7 @@
M(Float32) \
M(Float64)
// No UInt128 here because of the name conflict
#define FOR_NUMERIC_TYPES(M) \
M(UInt8) \
M(UInt16) \

View File

@ -82,7 +82,7 @@ public:
bool isNumeric() const override { return false; }
bool canBeInsideNullable() const override { return true; }
bool isFixedAndContiguous() const override { return true; }
bool isFixedAndContiguous() const final { return true; }
size_t sizeOfValueIfFixed() const override { return sizeof(T); }
size_t size() const override { return data.size(); }

View File

@ -112,7 +112,6 @@ def get_db_engine(args):
return "" # Will use default engine
def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file):
# print(client_options)
if args.database:
@ -149,10 +148,12 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std
pattern = "{client} --send_logs_level={logs_level} --testmode --multiquery {options} < " + pattern
command = pattern.format(**params)
#print(command)
# print(command)
proc = Popen(command, shell=True, env=os.environ)
start_time = datetime.now()
while (datetime.now() - start_time).total_seconds() < args.timeout and proc.poll() is None:
sleep(0.01)
@ -317,6 +318,7 @@ def run_tests_array(all_tests_with_params):
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
proc, stdout, stderr, total_time = run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file)
if proc.returncode is None:
try:
proc.kill()
@ -347,7 +349,7 @@ def run_tests_array(all_tests_with_params):
if stderr:
print(stderr)
# Stop on fatal errors like segmentation fault. They are send to client via logs.
# Stop on fatal errors like segmentation fault. They are sent to client via logs.
if ' <Fatal> ' in stderr:
SERVER_DIED = True
@ -486,7 +488,7 @@ def collect_build_flags(client):
elif b'-fsanitize=memory' in stdout:
result.append(BuildFlags.MEMORY)
else:
raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'")
@ -497,7 +499,7 @@ def collect_build_flags(client):
elif b'RelWithDebInfo' in stdout or b'Release' in stdout:
result.append(BuildFlags.RELEASE)
else:
raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'")
@ -506,7 +508,7 @@ def collect_build_flags(client):
if b'ON' in stdout or b'1' in stdout:
result.append(BuildFlags.UNBUNDLED)
else:
raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.settings WHERE name = 'default_database_engine'")
@ -515,7 +517,7 @@ def collect_build_flags(client):
if b'Ordinary' in stdout:
result.append(BuildFlags.DATABASE_ORDINARY)
else:
raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE)
(stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'")
@ -544,8 +546,12 @@ def main(args):
return stdout.startswith(b'1')
if not check_server_started(args.client, args.server_check_retries):
raise Exception("clickhouse-server is not responding. Cannot execute 'SELECT 1' query.")
raise Exception(
"Server is not responding. Cannot execute 'SELECT 1' query. \
Note: if you are using unbundled mode, you also have to specify -c option.")
build_flags = collect_build_flags(args.client)
if args.use_skip_list:
tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags)
else:
@ -790,8 +796,13 @@ if __name__ == '__main__':
parser=ArgumentParser(description='ClickHouse functional tests')
parser.add_argument('-q', '--queries', help='Path to queries dir')
parser.add_argument('--tmp', help='Path to tmp dir')
parser.add_argument('-b', '--binary', default='clickhouse', help='Path to clickhouse binary or name of binary in PATH')
parser.add_argument('-c', '--client', help='Client program')
parser.add_argument('-b', '--binary', default='clickhouse',
help='Path to clickhouse (if bundled, clickhouse-server otherwise) binary or name of binary in PATH')
parser.add_argument('-c', '--client',
help='Path to clickhouse-client (if unbundled, useless otherwise) binary of name of binary in PATH')
parser.add_argument('--extract_from_config', help='extract-from-config program')
parser.add_argument('--configclient', help='Client config (if you use not default ports)')
parser.add_argument('--configserver', default= '/etc/clickhouse-server/config.xml', help='Preprocessed server config')
@ -865,10 +876,14 @@ if __name__ == '__main__':
if args.client is None:
if find_binary(args.binary + '-client'):
args.client = args.binary + '-client'
print("Using " + args.client + " as client program (expecting unbundled mode)")
elif find_binary(args.binary):
args.client = args.binary + ' client'
print("Using " + args.client + " as client program (expecting bundled mode)")
else:
print("No 'clickhouse' binary found in PATH", file=sys.stderr)
print("No 'clickhouse' or 'clickhouse-client' client binary found", file=sys.stderr)
parser.print_help()
exit(1)

View File

@ -0,0 +1,38 @@
<test>
<settings>
<allow_experimental_bigint_types>1</allow_experimental_bigint_types>
<max_threads>1</max_threads>
<max_insert_threads>8</max_insert_threads>
</settings>
<preconditions>
<table_exists>hits_100m_single</table_exists>
</preconditions>
<create_query>CREATE TABLE perf_avg(
num UInt64,
num_u Decimal256(75) DEFAULT toDecimal256(num / 400000, 75),
num_f Float64 DEFAULT num / 100
) ENGINE = MergeTree() ORDER BY num
</create_query>
<fill_query>
INSERT INTO perf_avg(num)
SELECT toUInt64(UserID / (WatchID + 1) * 1000000)
FROM hits_100m_single
LIMIT 50000000
</fill_query>
<query>SELECT avg(num) FROM perf_avg FORMAT Null</query>
<query>SELECT avg(2 * num) FROM perf_avg FORMAT Null</query>
<query>SELECT avg(num_u) FROM perf_avg FORMAT Null</query>
<query>SELECT avg(num_f) FROM perf_avg FORMAT Null</query>
<query>SELECT avgWeighted(num_f, num) FROM perf_avg FORMAT Null</query>
<query>SELECT avgWeighted(num_f, num_f) FROM perf_avg FORMAT Null</query>
<query>SELECT avgWeighted(num_f, num_u) FROM perf_avg FORMAT Null</query>
<query>SELECT avgWeighted(num_u, num_f) FROM perf_avg FORMAT Null</query>
<query>SELECT avgWeighted(num_u, num) FROM perf_avg FORMAT Null</query>
<query>SELECT avgWeighted(num_u, num_u) FROM perf_avg FORMAT Null</query>
<drop_query>DROP TABLE IF EXISTS perf_avg</drop_query>
</test>

View File

@ -5,9 +5,6 @@
-1275.0000 -424.99999983 -255.00000000 -1275.0000 -424.99999983 -255.00000000
101.0000 101.00000000 101.00000000 101.0000 101.00000000 101.00000000
-101.0000 -101.00000000 -101.00000000 -101.0000 -101.00000000 -101.00000000
0.0000 0.00000000 0.00000000
25.5000 8.49999999 5.10000000
-25.5000 -8.49999999 -5.10000000
(101,101,101) (101,101,101) (101,101,101) (101,101,101) (102,100,101)
5 5 5
10 10 10

View File

@ -20,10 +20,6 @@ SELECT sum(a), sum(b), sum(c), sumWithOverflow(a), sumWithOverflow(b), sumWithOv
SELECT sum(a+1), sum(b+1), sum(c+1), sumWithOverflow(a+1), sumWithOverflow(b+1), sumWithOverflow(c+1) FROM decimal;
SELECT sum(a-1), sum(b-1), sum(c-1), sumWithOverflow(a-1), sumWithOverflow(b-1), sumWithOverflow(c-1) FROM decimal;
SELECT avg(a), avg(b), avg(c) FROM decimal;
SELECT avg(a), avg(b), avg(c) FROM decimal WHERE a > 0;
SELECT avg(a), avg(b), avg(c) FROM decimal WHERE a < 0;
SELECT (uniq(a), uniq(b), uniq(c)),
(uniqCombined(a), uniqCombined(b), uniqCombined(c)),
(uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)),

View File

@ -5,9 +5,6 @@
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 0.0000 0.0000000 0.00000000
0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8)
0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8)
0.0000 0.0000000 0.00000000 Decimal(9, 4) Decimal(18, 7) Decimal(38, 8)
(0,0,0) (0,0,0) (0,0,0) (0,0,0) (0,0,0)
0 0 0
0 0 0

View File

@ -16,10 +16,6 @@ SELECT sum(a), sum(b), sum(c), sumWithOverflow(a), sumWithOverflow(b), sumWithOv
SELECT sum(a+1), sum(b+1), sum(c+1), sumWithOverflow(a+1), sumWithOverflow(b+1), sumWithOverflow(c+1) FROM decimal;
SELECT sum(a-1), sum(b-1), sum(c-1), sumWithOverflow(a-1), sumWithOverflow(b-1), sumWithOverflow(c-1) FROM decimal;
SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal;
SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal WHERE a > 0;
SELECT avg(a) as aa, avg(b) as ab, avg(c) as ac, toTypeName(aa), toTypeName(ab),toTypeName(ac) FROM decimal WHERE a < 0;
SELECT (uniq(a), uniq(b), uniq(c)),
(uniqCombined(a), uniqCombined(b), uniqCombined(c)),
(uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)),

View File

@ -4,6 +4,6 @@
[499500.00]
[499500.00000]
[499500.0000000000]
1545081300 [('ed87e57c-9331-462a-80b4-9f0c005e88c8',0.4400)]
1545081300 [('ed87e57c-9331-462a-80b4-9f0c005e88c8',0.44)]
4341757 5657967 2018-11-01 16:47:46 txt 321.380000000000 315.080000000000 0.000000000000 2018-11-02 00:00:00
4360430 5681495 2018-11-02 09:00:07 txt 274.350000000000 268.970000000000 0.000000000000 2018-11-02 00:00:00

View File

@ -41,7 +41,7 @@ nan
\N
\N
\N
0.00
0
\N
0
\N

View File

@ -0,0 +1,5 @@
nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan nan
-0.5 -0.5 -0.5 -0.5 -0.5 -0.5 127.493856 32355.57552 499999.5 499999.5 499999.5 499999.5 499999.5 -0.000500002 0.49999949943727 -0.000005 -0.000005257366687274546
-2767.546272
999999
-0.50000449943727

View File

@ -0,0 +1,43 @@
SET allow_experimental_bigint_types=1;
-- no UInt128 as for now
CREATE TABLE IF NOT EXISTS test_01035_avg (
i8 Int8 DEFAULT i64,
i16 Int16 DEFAULT i64,
i32 Int32 DEFAULT i64,
i64 Int64 DEFAULT if(u64 % 2 = 0, toInt64(u64), toInt64(-u64)),
i128 Int128 DEFAULT i64,
i256 Int256 DEFAULT i64,
u8 UInt8 DEFAULT u64,
u16 UInt16 DEFAULT u64,
u32 UInt32 DEFAULT u64,
u64 UInt64,
u256 UInt256 DEFAULT u64,
f32 Float32 DEFAULT u64,
f64 Float64 DEFAULT u64,
d32 Decimal32(4) DEFAULT toDecimal32(i32 / 1000, 4),
d64 Decimal64(18) DEFAULT toDecimal64(u64 / 1000000, 8),
d128 Decimal128(20) DEFAULT toDecimal128(i128 / 100000, 20),
d256 Decimal256(40) DEFAULT toDecimal256(i256 / 100000, 40)
) ENGINE = MergeTree() ORDER BY i64;
SELECT avg(i8), avg(i16), avg(i32), avg(i64), avg(i128), avg(i256),
avg(u8), avg(u16), avg(u32), avg(u64), avg(u256),
avg(f32), avg(f64),
avg(d32), avg(d64), avg(d128), avg(d256) FROM test_01035_avg;
INSERT INTO test_01035_avg (u64) SELECT number FROM system.numbers LIMIT 1000000;
SELECT avg(i8), avg(i16), avg(i32), avg(i64), avg(i128), avg(i256),
avg(u8), avg(u16), avg(u32), avg(u64), avg(u256),
avg(f32), avg(f64),
avg(d32), avg(d64), avg(d128), avg(d256) FROM test_01035_avg;
SELECT avg(i8 * i16) FROM test_01035_avg;
SELECT avg(f32 + f64) FROM test_01035_avg;
SELECT avg(d128 - d64) FROM test_01035_avg;
DROP TABLE IF EXISTS test_01035_avg;

View File

@ -1,3 +0,0 @@
2.3333333333333335
nan
1

View File

@ -1,11 +0,0 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));"
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));"
echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(toDecimal64(0, 0), toFloat64(0))" 2>&1)" \
| grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Different types .* of arguments for aggregate function avgWeighted'

View File

@ -0,0 +1,230 @@
2.3333333333333335
nan
1
1
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
8
nan
1
1
1
1
1
1
1
1
1
2
2
2
2
2
2
2
2
2
2
2
2
2
2
2
2
1

View File

@ -0,0 +1,44 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 5), (2, 4), (3, 3), (4, 2), (5, 1)]) AS t));"
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, weight) FROM (SELECT t.1 AS x, t.2 AS weight FROM (SELECT arrayJoin([(1, 0), (2, 0), (3, 0), (4, 0), (5, 0)]) AS t));"
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, y) FROM (select toDecimal256(1, 0) x, toDecimal256(1, 1) y);"
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, y) FROM (select toDecimal32(1, 0) x, toDecimal256(1, 1) y);"
types=("Int8" "Int16" "Int32" "Int64" "UInt8" "UInt16" "UInt32" "UInt64" "Float32" "Float64")
for left in "${types[@]}"
do
for right in "${types[@]}"
do
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (4, 1), (1, 0), (10, 2))"
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(x, w) FROM values('x ${left}, w ${right}', (0, 0), (1, 0))"
done
done
exttypes=("Int128" "Int256" "UInt256")
for left in "${exttypes[@]}"
do
for right in "${exttypes[@]}"
do
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(to${left}(1), to${right}(2))"
done
done
# Decimal types
dtypes=("32" "64" "128" "256")
for left in "${dtypes[@]}"
do
for right in "${dtypes[@]}"
do
${CLICKHOUSE_CLIENT} --query="SELECT avgWeighted(toDecimal${left}(2, 4), toDecimal${right}(1, 4))"
done
done
echo "$(${CLICKHOUSE_CLIENT} --server_logs_file=/dev/null --query="SELECT avgWeighted(['string'], toFloat64(0))" 2>&1)" \
| grep -c 'Code: 43. DB::Exception: .* DB::Exception:.* Types .* are non-conforming as arguments for aggregate function avgWeighted'