mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
dbms: added aggregate function quantileExactWeighted and companions [#METR-18778].
This commit is contained in:
parent
3a1a8c9567
commit
49580a9048
@ -34,7 +34,7 @@ public:
|
||||
return new DataTypeFloat64;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->isNumeric())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
return new DataTypeArray(new typename DataTypeFromFieldType<T>::Type);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
}
|
||||
|
||||
@ -117,7 +117,7 @@ public:
|
||||
return new DataTypeArray(type);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
}
|
||||
|
@ -49,7 +49,7 @@ public:
|
||||
return new DataTypeArray(new typename DataTypeFromFieldType<T>::Type);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -51,7 +51,7 @@ public:
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (returns_float)
|
||||
type = new DataTypeFloat64;
|
||||
@ -126,7 +126,7 @@ public:
|
||||
return new DataTypeArray(type);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (returns_float)
|
||||
type = new DataTypeFloat64;
|
||||
|
@ -56,7 +56,7 @@ public:
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
}
|
||||
@ -143,7 +143,7 @@ public:
|
||||
return new DataTypeArray(type);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
}
|
||||
|
@ -0,0 +1,286 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Common/HashTable/HashMap.h>
|
||||
|
||||
#include <DB/Core/FieldVisitors.h>
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IBinaryAggregateFunction.h>
|
||||
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** В качестве состояния используется хэш-таблица вида: значение -> сколько раз встретилось.
|
||||
*/
|
||||
template <typename T>
|
||||
struct AggregateFunctionQuantileExactWeightedData
|
||||
{
|
||||
using Key = T;
|
||||
using Weight = UInt64;
|
||||
|
||||
/// При создании, хэш-таблица должна быть небольшой.
|
||||
using Map = HashMap<
|
||||
Key, Weight,
|
||||
HashCRC32<Key>,
|
||||
HashTableGrower<4>,
|
||||
HashTableAllocatorWithStackMemory<sizeof(std::pair<Key, Weight>) * (1 << 3)>
|
||||
>;
|
||||
|
||||
Map map;
|
||||
};
|
||||
|
||||
|
||||
/** Точно вычисляет квантиль по множеству значений, для каждого из которых задан вес - сколько раз значение встречалось.
|
||||
* Можно рассматривать набор пар value, weight - как набор гистограмм,
|
||||
* в которых value - значение, округлённое до середины столбика, а weight - высота столбика.
|
||||
* В качестве типа аргумента может быть только числовой тип (в том числе, дата и дата-с-временем).
|
||||
* Тип результата совпадает с типом аргумента.
|
||||
*/
|
||||
template <typename ValueType, typename WeightType>
|
||||
class AggregateFunctionQuantileExactWeighted final
|
||||
: public IBinaryAggregateFunction<
|
||||
AggregateFunctionQuantileExactWeightedData<ValueType>,
|
||||
AggregateFunctionQuantileExactWeighted<ValueType, WeightType>>
|
||||
{
|
||||
private:
|
||||
double level;
|
||||
DataTypePtr type;
|
||||
|
||||
public:
|
||||
AggregateFunctionQuantileExactWeighted(double level_ = 0.5) : level(level_) {}
|
||||
|
||||
String getName() const override { return "quantileExactWeighted"; }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgumentsImpl(const DataTypes & arguments)
|
||||
{
|
||||
type = arguments[0];
|
||||
}
|
||||
|
||||
void setParameters(const Array & params) override
|
||||
{
|
||||
if (params.size() != 1)
|
||||
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
level = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
|
||||
}
|
||||
|
||||
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
|
||||
{
|
||||
this->data(place)
|
||||
.map[static_cast<const ColumnVector<ValueType> &>(column_value).getData()[row_num]]
|
||||
+= static_cast<const ColumnVector<WeightType> &>(column_weight).getData()[row_num];
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
|
||||
{
|
||||
auto & map = this->data(place).map;
|
||||
const auto & rhs_map = this->data(rhs).map;
|
||||
|
||||
for (const auto & pair : rhs_map)
|
||||
map[pair.first] += pair.second;
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).map.write(buf);
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
|
||||
{
|
||||
typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::Reader reader(buf);
|
||||
|
||||
auto & map = this->data(place).map;
|
||||
while (reader.next())
|
||||
{
|
||||
const auto & pair = reader.get();
|
||||
map[pair.first] += pair.second;
|
||||
}
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
auto & map = this->data(place).map;
|
||||
size_t size = map.size();
|
||||
|
||||
if (0 == size)
|
||||
{
|
||||
static_cast<ColumnVector<ValueType> &>(to).getData().push_back(ValueType());
|
||||
return;
|
||||
}
|
||||
|
||||
/// Копируем данные во временный массив, чтобы получить нужный по порядку элемент.
|
||||
using Pair = typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::value_type;
|
||||
std::unique_ptr<Pair> array_holder(new Pair[size]);
|
||||
Pair * array = array_holder.get();
|
||||
|
||||
size_t i = 0;
|
||||
UInt64 sum_weight = 0;
|
||||
for (const auto & pair : map)
|
||||
{
|
||||
sum_weight += pair.second;
|
||||
array[i] = pair;
|
||||
++i;
|
||||
}
|
||||
|
||||
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
|
||||
|
||||
UInt64 threshold = sum_weight * level;
|
||||
UInt64 accumulated = 0;
|
||||
|
||||
const Pair * it = array;
|
||||
const Pair * end = array + size;
|
||||
while (it < end && accumulated < threshold)
|
||||
{
|
||||
accumulated += it->second;
|
||||
++it;
|
||||
}
|
||||
|
||||
if (it == end)
|
||||
--it;
|
||||
|
||||
static_cast<ColumnVector<ValueType> &>(to).getData().push_back(it->first);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** То же самое, но позволяет вычислить сразу несколько квантилей.
|
||||
* Для этого, принимает в качестве параметров несколько уровней. Пример: quantilesExactWeighted(0.5, 0.8, 0.9, 0.95)(ConnectTiming, Weight).
|
||||
* Возвращает массив результатов.
|
||||
*/
|
||||
template <typename ValueType, typename WeightType>
|
||||
class AggregateFunctionQuantilesExactWeighted final
|
||||
: public IBinaryAggregateFunction<
|
||||
AggregateFunctionQuantileExactWeightedData<ValueType>,
|
||||
AggregateFunctionQuantilesExactWeighted<ValueType, WeightType>>
|
||||
{
|
||||
private:
|
||||
using Levels = std::vector<double>;
|
||||
Levels levels;
|
||||
DataTypePtr type;
|
||||
|
||||
public:
|
||||
String getName() const override { return "quantilesExactWeighted"; }
|
||||
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return new DataTypeArray(type);
|
||||
}
|
||||
|
||||
void setArgumentsImpl(const DataTypes & arguments)
|
||||
{
|
||||
type = arguments[0];
|
||||
}
|
||||
|
||||
void setParameters(const Array & params) override
|
||||
{
|
||||
if (params.empty())
|
||||
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
size_t size = params.size();
|
||||
levels.resize(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
levels[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[i]);
|
||||
}
|
||||
|
||||
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
|
||||
{
|
||||
this->data(place)
|
||||
.map[static_cast<const ColumnVector<ValueType> &>(column_value).getData()[row_num]]
|
||||
+= static_cast<const ColumnVector<WeightType> &>(column_weight).getData()[row_num];
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
|
||||
{
|
||||
auto & map = this->data(place).map;
|
||||
const auto & rhs_map = this->data(rhs).map;
|
||||
|
||||
for (const auto & pair : rhs_map)
|
||||
map[pair.first] += pair.second;
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
||||
{
|
||||
this->data(place).map.write(buf);
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
|
||||
{
|
||||
typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::Reader reader(buf);
|
||||
|
||||
auto & map = this->data(place).map;
|
||||
while (reader.next())
|
||||
{
|
||||
const auto & pair = reader.get();
|
||||
map[pair.first] += pair.second;
|
||||
}
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
||||
{
|
||||
auto & map = this->data(place).map;
|
||||
size_t size = map.size();
|
||||
|
||||
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
|
||||
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
|
||||
|
||||
size_t num_levels = levels.size();
|
||||
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + num_levels);
|
||||
|
||||
typename ColumnVector<ValueType>::Container_t & data_to = static_cast<ColumnVector<ValueType> &>(arr_to.getData()).getData();
|
||||
|
||||
if (0 == size)
|
||||
{
|
||||
for (size_t i = 0; i < num_levels; ++i)
|
||||
data_to.push_back(ValueType());
|
||||
return;
|
||||
}
|
||||
|
||||
/// Копируем данные во временный массив, чтобы получить нужный по порядку элемент.
|
||||
using Pair = typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::value_type;
|
||||
std::unique_ptr<Pair> array_holder(new Pair[size]);
|
||||
Pair * array = array_holder.get();
|
||||
|
||||
size_t i = 0;
|
||||
UInt64 sum_weight = 0;
|
||||
for (const auto & pair : map)
|
||||
{
|
||||
sum_weight += pair.second;
|
||||
array[i] = pair;
|
||||
++i;
|
||||
}
|
||||
|
||||
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
|
||||
|
||||
UInt64 accumulated = 0;
|
||||
|
||||
const Pair * it = array;
|
||||
const Pair * end = array + size;
|
||||
|
||||
for (const auto & level : levels)
|
||||
{
|
||||
UInt64 threshold = sum_weight * level;
|
||||
|
||||
while (it < end && accumulated < threshold)
|
||||
{
|
||||
accumulated += it->second;
|
||||
++it;
|
||||
}
|
||||
|
||||
data_to.push_back(it < end ? it->first : it[-1].first);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -557,7 +557,7 @@ public:
|
||||
return new DataTypeFloat32;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
}
|
||||
|
||||
@ -676,7 +676,7 @@ public:
|
||||
return new DataTypeArray(new DataTypeFloat32);
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
return new typename DataTypeFromFieldType<typename NearestFieldType<T>::Type>::Type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->isNumeric())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
|
@ -337,7 +337,7 @@ public:
|
||||
return new DataTypeUInt64;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -145,7 +145,7 @@ public:
|
||||
return new DataTypeUInt64;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -539,7 +539,7 @@ public:
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
|
||||
|
@ -122,7 +122,7 @@ public:
|
||||
return new DataTypeFloat64;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument) override
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->behavesAsNumber())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
|
@ -8,11 +8,12 @@ namespace DB
|
||||
template <typename T, typename Derived>
|
||||
class IBinaryAggregateFunction : public IAggregateFunctionHelper<T>
|
||||
{
|
||||
private:
|
||||
Derived & getDerived() { return static_cast<Derived &>(*this); }
|
||||
const Derived & getDerived() const { return static_cast<const Derived &>(*this); }
|
||||
|
||||
public:
|
||||
void setArguments(const DataTypes & arguments) override
|
||||
void setArguments(const DataTypes & arguments) override final
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception{
|
||||
@ -23,7 +24,7 @@ public:
|
||||
getDerived().setArgumentsImpl(arguments);
|
||||
}
|
||||
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num) const override
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num) const override final
|
||||
{
|
||||
getDerived().addImpl(place, *columns[0], *columns[1], row_num);
|
||||
}
|
||||
|
@ -11,9 +11,13 @@ namespace DB
|
||||
template <typename T, typename Derived>
|
||||
class INullaryAggregateFunction : public IAggregateFunctionHelper<T>
|
||||
{
|
||||
private:
|
||||
Derived & getDerived() { return static_cast<Derived &>(*this); }
|
||||
const Derived & getDerived() const { return static_cast<const Derived &>(*this); }
|
||||
|
||||
public:
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
void setArguments(const DataTypes & arguments) override
|
||||
void setArguments(const DataTypes & arguments) override final
|
||||
{
|
||||
if (arguments.size() != 0)
|
||||
throw Exception("Passed " + toString(arguments.size()) + " arguments to nullary aggregate function " + this->getName(),
|
||||
@ -21,9 +25,9 @@ public:
|
||||
}
|
||||
|
||||
/// Добавить значение.
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override final
|
||||
{
|
||||
static_cast<const Derived &>(*this).addImpl(place);
|
||||
getDerived().addImpl(place);
|
||||
}
|
||||
|
||||
/** Реализуйте это в классе-наследнике:
|
||||
|
@ -11,25 +11,29 @@ namespace DB
|
||||
template <typename T, typename Derived>
|
||||
class IUnaryAggregateFunction : public IAggregateFunctionHelper<T>
|
||||
{
|
||||
private:
|
||||
Derived & getDerived() { return static_cast<Derived &>(*this); }
|
||||
const Derived & getDerived() const { return static_cast<const Derived &>(*this); }
|
||||
|
||||
public:
|
||||
void setArguments(const DataTypes & arguments) override
|
||||
void setArguments(const DataTypes & arguments) override final
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Passed " + toString(arguments.size()) + " arguments to unary aggregate function " + this->getName(),
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
setArgument(arguments[0]);
|
||||
|
||||
getDerived().setArgument(arguments[0]);
|
||||
}
|
||||
|
||||
virtual void setArgument(const DataTypePtr & argument) = 0;
|
||||
|
||||
/// Добавить значение.
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
|
||||
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override final
|
||||
{
|
||||
static_cast<const Derived &>(*this).addImpl(place, *columns[0], row_num);
|
||||
getDerived().addImpl(place, *columns[0], row_num);
|
||||
}
|
||||
|
||||
/** Реализуйте это в классе-наследнике:
|
||||
* void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const;
|
||||
* void setArgument(const DataTypePtr & argument);
|
||||
*/
|
||||
};
|
||||
|
||||
|
@ -61,6 +61,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory);
|
||||
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory & factory);
|
||||
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory);
|
||||
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory & factory);
|
||||
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory & factory);
|
||||
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory);
|
||||
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory);
|
||||
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory);
|
||||
@ -83,6 +84,8 @@ AggregateFunctionFactory::AggregateFunctionFactory()
|
||||
registerAggregateFunctionGroupArray(*this);
|
||||
registerAggregateFunctionGroupUniqArray(*this);
|
||||
registerAggregateFunctionsQuantile(*this);
|
||||
registerAggregateFunctionsQuantileExact(*this);
|
||||
registerAggregateFunctionsQuantileExactWeighted(*this);
|
||||
registerAggregateFunctionsQuantileDeterministic(*this);
|
||||
registerAggregateFunctionsQuantileTiming(*this);
|
||||
registerAggregateFunctionsSequenceMatch(*this);
|
||||
@ -91,7 +94,6 @@ AggregateFunctionFactory::AggregateFunctionFactory()
|
||||
registerAggregateFunctionSum(*this);
|
||||
registerAggregateFunctionsUniq(*this);
|
||||
registerAggregateFunctionUniqUpTo(*this);
|
||||
registerAggregateFunctionsQuantileExact(*this);
|
||||
}
|
||||
|
||||
|
||||
|
@ -0,0 +1,48 @@
|
||||
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <DB/AggregateFunctions/Helpers.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionQuantileExactWeighted.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionQuantileExactWeighted(const std::string & name, const DataTypes & argument_types)
|
||||
{
|
||||
if (argument_types.size() != 2)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantileExactWeighted>(*argument_types[0], *argument_types[1]);
|
||||
|
||||
if (!res)
|
||||
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
AggregateFunctionPtr createAggregateFunctionQuantilesExactWeighted(const std::string & name, const DataTypes & argument_types)
|
||||
{
|
||||
if (argument_types.size() != 2)
|
||||
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
AggregateFunctionPtr res = createWithTwoNumericTypes<AggregateFunctionQuantilesExactWeighted>(*argument_types[0], *argument_types[1]);
|
||||
|
||||
if (!res)
|
||||
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
|
||||
+ " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction({"quantileExactWeighted", "medianExactWeighted"}, createAggregateFunctionQuantileExactWeighted);
|
||||
factory.registerFunction({"quantilesExactWeighted"}, createAggregateFunctionQuantilesExactWeighted);
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,6 @@
|
||||
500
|
||||
[0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1000]
|
||||
[0,33,101,225,317,448,549,634,708,776,838,896,950,976,996,1000,1000]
|
||||
500
|
||||
[0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1001]
|
||||
[0,33,101,225,317,448,549,634,708,776,838,896,950,976,996,1001,1001]
|
@ -0,0 +1,7 @@
|
||||
SELECT quantileExactWeighted(0.5)(number, 1) FROM (SELECT number FROM system.numbers LIMIT 1001);
|
||||
SELECT quantilesExactWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, 1) FROM (SELECT number FROM system.numbers LIMIT 1001);
|
||||
SELECT quantilesExactWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, number) FROM (SELECT number FROM system.numbers LIMIT 1001);
|
||||
|
||||
SELECT quantileTimingWeighted(0.5)(number, 1) FROM (SELECT number FROM system.numbers LIMIT 1001);
|
||||
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, 1) FROM (SELECT number FROM system.numbers LIMIT 1001);
|
||||
SELECT quantilesTimingWeighted(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(number, number) FROM (SELECT number FROM system.numbers LIMIT 1001);
|
Loading…
Reference in New Issue
Block a user