mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #38252 from bharatnc/ncb/weighted-quantile-approx
add quantileInterpolatedWeighted function
This commit is contained in:
commit
70e79de69b
@ -0,0 +1,68 @@
|
||||
---
|
||||
slug: /en/sql-reference/aggregate-functions/reference/quantileInterpolatedWeighted
|
||||
sidebar_position: 203
|
||||
---
|
||||
|
||||
# quantileInterpolatedWeighted
|
||||
|
||||
Computes [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using linear interpolation, taking into account the weight of each element.
|
||||
|
||||
To get the interpolated value, all the passed values are combined into an array, which are then sorted by their corresponding weights. Quantile interpolation is then performed using the [weighted percentile method](https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method) by building a cumulative distribution based on weights and then a linear interpolation is performed using the weights and the values to compute the quantiles.
|
||||
|
||||
When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
quantileInterpolatedWeighted(level)(expr, weight)
|
||||
```
|
||||
|
||||
Alias: `medianInterpolatedWeighted`.
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `level` — Level of quantile. Optional parameter. Constant floating-point number from 0 to 1. We recommend using a `level` value in the range of `[0.01, 0.99]`. Default value: 0.5. At `level=0.5` the function calculates [median](https://en.wikipedia.org/wiki/Median).
|
||||
- `expr` — Expression over the column values resulting in numeric [data types](../../../sql-reference/data-types/index.md#data_types), [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md).
|
||||
- `weight` — Column with weights of sequence members. Weight is a number of value occurrences.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Quantile of the specified level.
|
||||
|
||||
Type:
|
||||
|
||||
- [Float64](../../../sql-reference/data-types/float.md) for numeric data type input.
|
||||
- [Date](../../../sql-reference/data-types/date.md) if input values have the `Date` type.
|
||||
- [DateTime](../../../sql-reference/data-types/datetime.md) if input values have the `DateTime` type.
|
||||
|
||||
**Example**
|
||||
|
||||
Input table:
|
||||
|
||||
``` text
|
||||
┌─n─┬─val─┐
|
||||
│ 0 │ 3 │
|
||||
│ 1 │ 2 │
|
||||
│ 2 │ 1 │
|
||||
│ 5 │ 4 │
|
||||
└───┴─────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT quantileInterpolatedWeighted(n, val) FROM t
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─quantileInterpolatedWeighted(n, val)─┐
|
||||
│ 1 │
|
||||
└──────────────────────────────────────┘
|
||||
```
|
||||
|
||||
**See Also**
|
||||
|
||||
- [median](../../../sql-reference/aggregate-functions/reference/median.md#median)
|
||||
- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles)
|
@ -9,7 +9,7 @@ sidebar_position: 201
|
||||
|
||||
Syntax: `quantiles(level1, level2, …)(x)`
|
||||
|
||||
All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values.
|
||||
All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantileInterpolatedWeighted`, `quantilesTDigest`, `quantilesBFloat16`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values.
|
||||
|
||||
## quantilesExactExclusive
|
||||
|
||||
|
@ -232,6 +232,9 @@ struct NameQuantilesExactInclusive { static constexpr auto name = "quantilesExac
|
||||
struct NameQuantileExactWeighted { static constexpr auto name = "quantileExactWeighted"; };
|
||||
struct NameQuantilesExactWeighted { static constexpr auto name = "quantilesExactWeighted"; };
|
||||
|
||||
struct NameQuantileInterpolatedWeighted { static constexpr auto name = "quantileInterpolatedWeighted"; };
|
||||
struct NameQuantilesInterpolatedWeighted { static constexpr auto name = "quantilesInterpolatedWeighted"; };
|
||||
|
||||
struct NameQuantileTiming { static constexpr auto name = "quantileTiming"; };
|
||||
struct NameQuantileTimingWeighted { static constexpr auto name = "quantileTimingWeighted"; };
|
||||
struct NameQuantilesTiming { static constexpr auto name = "quantilesTiming"; };
|
||||
|
@ -0,0 +1,70 @@
|
||||
#include <AggregateFunctions/AggregateFunctionQuantile.h>
|
||||
#include <AggregateFunctions/QuantileInterpolatedWeighted.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename Value, bool _> using FuncQuantileInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantileInterpolatedWeighted, true, void, false>;
|
||||
template <typename Value, bool _> using FuncQuantilesInterpolatedWeighted = AggregateFunctionQuantile<Value, QuantileInterpolatedWeighted<Value>, NameQuantilesInterpolatedWeighted, true, void, true>;
|
||||
|
||||
template <template <typename, bool> class Function>
|
||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||
{
|
||||
/// Second argument type check doesn't depend on the type of the first one.
|
||||
Function<void, true>::assertSecondArg(argument_types);
|
||||
|
||||
const DataTypePtr & argument_type = argument_types[0];
|
||||
WhichDataType which(argument_type);
|
||||
|
||||
#define DISPATCH(TYPE) \
|
||||
if (which.idx == TypeIndex::TYPE) return std::make_shared<Function<TYPE, true>>(argument_types, params);
|
||||
FOR_BASIC_NUMERIC_TYPES(DISPATCH)
|
||||
#undef DISPATCH
|
||||
if (which.idx == TypeIndex::Date) return std::make_shared<Function<DataTypeDate::FieldType, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::DateTime) return std::make_shared<Function<DataTypeDateTime::FieldType, false>>(argument_types, params);
|
||||
|
||||
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
|
||||
|
||||
if (which.idx == TypeIndex::Int128) return std::make_shared<Function<Int128, true>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::UInt128) return std::make_shared<Function<UInt128, true>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::Int256) return std::make_shared<Function<Int256, true>>(argument_types, params);
|
||||
if (which.idx == TypeIndex::UInt256) return std::make_shared<Function<UInt256, true>>(argument_types, params);
|
||||
|
||||
throw Exception("Illegal type " + argument_type->getName() + " of argument for aggregate function " + name,
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
}
|
||||
|
||||
void registerAggregateFunctionsQuantileInterpolatedWeighted(AggregateFunctionFactory & factory)
|
||||
{
|
||||
/// For aggregate functions returning array we cannot return NULL on empty set.
|
||||
AggregateFunctionProperties properties = { .returns_default_when_only_null = true };
|
||||
|
||||
factory.registerFunction(NameQuantileInterpolatedWeighted::name, createAggregateFunctionQuantile<FuncQuantileInterpolatedWeighted>);
|
||||
factory.registerFunction(NameQuantilesInterpolatedWeighted::name, { createAggregateFunctionQuantile<FuncQuantilesInterpolatedWeighted>, properties });
|
||||
|
||||
/// 'median' is an alias for 'quantile'
|
||||
factory.registerAlias("medianInterpolatedWeighted", NameQuantileInterpolatedWeighted::name);
|
||||
}
|
||||
|
||||
}
|
308
src/AggregateFunctions/QuantileInterpolatedWeighted.h
Normal file
308
src/AggregateFunctions/QuantileInterpolatedWeighted.h
Normal file
@ -0,0 +1,308 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct Settings;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
/** Approximates Quantile by:
|
||||
* - sorting input values and weights
|
||||
* - building a cumulative distribution based on weights
|
||||
* - performing linear interpolation between the weights and values
|
||||
*
|
||||
*/
|
||||
template <typename Value>
|
||||
struct QuantileInterpolatedWeighted
|
||||
{
|
||||
struct Int128Hash
|
||||
{
|
||||
size_t operator()(Int128 x) const
|
||||
{
|
||||
return CityHash_v1_0_2::Hash128to64({x >> 64, x & 0xffffffffffffffffll});
|
||||
}
|
||||
};
|
||||
|
||||
using Weight = UInt64;
|
||||
using UnderlyingType = NativeType<Value>;
|
||||
using Hasher = std::conditional_t<std::is_same_v<Value, Decimal128>, Int128Hash, HashCRC32<UnderlyingType>>;
|
||||
|
||||
/// When creating, the hash table must be small.
|
||||
using Map = HashMapWithStackMemory<UnderlyingType, Weight, Hasher, 4>;
|
||||
|
||||
Map map;
|
||||
|
||||
void add(const Value & x)
|
||||
{
|
||||
/// We must skip NaNs as they are not compatible with comparison sorting.
|
||||
if (!isNaN(x))
|
||||
++map[x];
|
||||
}
|
||||
|
||||
void add(const Value & x, Weight weight)
|
||||
{
|
||||
if (!isNaN(x))
|
||||
map[x] += weight;
|
||||
}
|
||||
|
||||
void merge(const QuantileInterpolatedWeighted & rhs)
|
||||
{
|
||||
for (const auto & pair : rhs.map)
|
||||
map[pair.getKey()] += pair.getMapped();
|
||||
}
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
map.write(buf);
|
||||
}
|
||||
|
||||
void deserialize(ReadBuffer & buf)
|
||||
{
|
||||
typename Map::Reader reader(buf);
|
||||
while (reader.next())
|
||||
{
|
||||
const auto & pair = reader.get();
|
||||
map[pair.first] = pair.second;
|
||||
}
|
||||
}
|
||||
|
||||
Value get(Float64 level) const
|
||||
{
|
||||
return getImpl<Value>(level);
|
||||
}
|
||||
|
||||
void getMany(const Float64 * levels, const size_t * indices, size_t size, Value * result) const
|
||||
{
|
||||
getManyImpl<Value>(levels, indices, size, result);
|
||||
}
|
||||
|
||||
/// The same, but in the case of an empty state, NaN is returned.
|
||||
Float64 getFloat(Float64) const
|
||||
{
|
||||
throw Exception("Method getFloat is not implemented for QuantileInterpolatedWeighted", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void getManyFloat(const Float64 *, const size_t *, size_t, Float64 *) const
|
||||
{
|
||||
throw Exception("Method getManyFloat is not implemented for QuantileInterpolatedWeighted", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
private:
|
||||
using Pair = typename std::pair<UnderlyingType, Float64>;
|
||||
|
||||
/// Get the value of the `level` quantile. The level must be between 0 and 1.
|
||||
template <typename T>
|
||||
T getImpl(Float64 level) const
|
||||
{
|
||||
size_t size = map.size();
|
||||
|
||||
if (0 == size)
|
||||
return std::numeric_limits<Value>::quiet_NaN();
|
||||
|
||||
/// Maintain a vector of pair of values and weights for easier sorting and for building
|
||||
/// a cumulative distribution using the provided weights.
|
||||
std::vector<Pair> value_weight_pairs;
|
||||
value_weight_pairs.reserve(size);
|
||||
|
||||
/// Note: weight provided must be a 64-bit integer
|
||||
/// Float64 is used as accumulator here to get approximate results.
|
||||
/// But weight used in the internal array is stored as Float64 as we
|
||||
/// do some quantile estimation operation which involves division and
|
||||
/// require Float64 level of precision.
|
||||
|
||||
Float64 sum_weight = 0;
|
||||
for (const auto & pair : map)
|
||||
{
|
||||
sum_weight += pair.getMapped();
|
||||
auto value = pair.getKey();
|
||||
auto weight = pair.getMapped();
|
||||
value_weight_pairs.push_back({value, weight});
|
||||
}
|
||||
|
||||
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
|
||||
|
||||
Float64 accumulated = 0;
|
||||
|
||||
/// vector for populating and storing the cumulative sum using the provided weights.
|
||||
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
|
||||
std::vector<Float64> weights_cum_sum;
|
||||
weights_cum_sum.reserve(size);
|
||||
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
{
|
||||
accumulated += value_weight_pairs[idx].second;
|
||||
weights_cum_sum.push_back(accumulated);
|
||||
}
|
||||
|
||||
/// The following estimation of quantile is general and the idea is:
|
||||
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
|
||||
|
||||
/// calculates a simple cumulative distribution based on weights
|
||||
if (sum_weight != 0)
|
||||
{
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
|
||||
}
|
||||
|
||||
/// perform linear interpolation
|
||||
size_t idx = 0;
|
||||
if (size >= 2)
|
||||
{
|
||||
if (level >= value_weight_pairs[size - 2].second)
|
||||
{
|
||||
idx = size - 2;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t start = 0, end = size - 1;
|
||||
while (start <= end)
|
||||
{
|
||||
size_t mid = start + (end - start) / 2;
|
||||
if (mid > size)
|
||||
break;
|
||||
if (level > value_weight_pairs[mid + 1].second)
|
||||
start = mid + 1;
|
||||
else
|
||||
{
|
||||
idx = mid;
|
||||
end = mid - 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
size_t l = idx;
|
||||
size_t u = idx + 1 < size ? idx + 1 : idx;
|
||||
|
||||
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
|
||||
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
|
||||
|
||||
if (level < xl)
|
||||
yr = yl;
|
||||
if (level > xr)
|
||||
yl = yr;
|
||||
|
||||
return static_cast<T>(interpolate(level, xl, xr, yl, yr));
|
||||
}
|
||||
|
||||
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
|
||||
/// indices - an array of index levels such that the corresponding elements will go in ascending order.
|
||||
template <typename T>
|
||||
void getManyImpl(const Float64 * levels, const size_t * indices, size_t num_levels, Value * result) const
|
||||
{
|
||||
size_t size = map.size();
|
||||
|
||||
if (0 == size)
|
||||
{
|
||||
for (size_t i = 0; i < num_levels; ++i)
|
||||
result[i] = Value();
|
||||
return;
|
||||
}
|
||||
|
||||
std::vector<Pair> value_weight_pairs;
|
||||
value_weight_pairs.reserve(size);
|
||||
|
||||
Float64 sum_weight = 0;
|
||||
for (const auto & pair : map)
|
||||
{
|
||||
sum_weight += pair.getMapped();
|
||||
auto value = pair.getKey();
|
||||
auto weight = pair.getMapped();
|
||||
value_weight_pairs.push_back({value, weight});
|
||||
}
|
||||
|
||||
::sort(value_weight_pairs.begin(), value_weight_pairs.end(), [](const Pair & a, const Pair & b) { return a.first < b.first; });
|
||||
|
||||
Float64 accumulated = 0;
|
||||
|
||||
/// vector for populating and storing the cumulative sum using the provided weights.
|
||||
/// example: [0,1,2,3,4,5] -> [0,1,3,6,10,15]
|
||||
std::vector<Float64> weights_cum_sum;
|
||||
weights_cum_sum.reserve(size);
|
||||
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
{
|
||||
accumulated += value_weight_pairs[idx].second;
|
||||
weights_cum_sum.emplace_back(accumulated);
|
||||
}
|
||||
|
||||
|
||||
/// The following estimation of quantile is general and the idea is:
|
||||
/// https://en.wikipedia.org/wiki/Percentile#The_weighted_percentile_method
|
||||
|
||||
/// calculates a simple cumulative distribution based on weights
|
||||
if (sum_weight != 0)
|
||||
{
|
||||
for (size_t idx = 0; idx < size; ++idx)
|
||||
value_weight_pairs[idx].second = (weights_cum_sum[idx] - 0.5 * value_weight_pairs[idx].second) / sum_weight;
|
||||
}
|
||||
|
||||
for (size_t level_index = 0; level_index < num_levels; ++level_index)
|
||||
{
|
||||
/// perform linear interpolation for every level
|
||||
auto level = levels[indices[level_index]];
|
||||
|
||||
size_t idx = 0;
|
||||
if (size >= 2)
|
||||
{
|
||||
if (level >= value_weight_pairs[size - 2].second)
|
||||
{
|
||||
idx = size - 2;
|
||||
}
|
||||
else
|
||||
{
|
||||
size_t start = 0, end = size - 1;
|
||||
while (start <= end)
|
||||
{
|
||||
size_t mid = start + (end - start) / 2;
|
||||
if (mid > size)
|
||||
break;
|
||||
if (level > value_weight_pairs[mid + 1].second)
|
||||
start = mid + 1;
|
||||
else
|
||||
{
|
||||
idx = mid;
|
||||
end = mid - 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
size_t l = idx;
|
||||
size_t u = idx + 1 < size ? idx + 1 : idx;
|
||||
|
||||
Float64 xl = value_weight_pairs[l].second, xr = value_weight_pairs[u].second;
|
||||
UnderlyingType yl = value_weight_pairs[l].first, yr = value_weight_pairs[u].first;
|
||||
|
||||
if (level < xl)
|
||||
yr = yl;
|
||||
if (level > xr)
|
||||
yl = yr;
|
||||
|
||||
result[indices[level_index]] = static_cast<T>(interpolate(level, xl, xr, yl, yr));
|
||||
}
|
||||
}
|
||||
|
||||
/// This ignores overflows or NaN's that might arise during add, sub and mul operations and doesn't aim to provide exact
|
||||
/// results since `the quantileInterpolatedWeighted` function itself relies mainly on approximation.
|
||||
UnderlyingType NO_SANITIZE_UNDEFINED interpolate(Float64 level, Float64 xl, Float64 xr, UnderlyingType yl, UnderlyingType yr) const
|
||||
{
|
||||
UnderlyingType dy = yr - yl;
|
||||
Float64 dx = xr - xl;
|
||||
dx = dx == 0 ? 1 : dx; /// to handle NaN behavior that might arise during integer division below.
|
||||
|
||||
/// yl + (dy / dx) * (level - xl)
|
||||
return static_cast<UnderlyingType>(yl + (dy / dx) * (level - xl));
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -21,6 +21,7 @@ void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileInterpolatedWeighted(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactLow(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactHigh(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionsQuantileExactInclusive(AggregateFunctionFactory &);
|
||||
@ -106,6 +107,7 @@ void registerAggregateFunctions()
|
||||
registerAggregateFunctionsQuantileDeterministic(factory);
|
||||
registerAggregateFunctionsQuantileExact(factory);
|
||||
registerAggregateFunctionsQuantileExactWeighted(factory);
|
||||
registerAggregateFunctionsQuantileInterpolatedWeighted(factory);
|
||||
registerAggregateFunctionsQuantileExactLow(factory);
|
||||
registerAggregateFunctionsQuantileExactHigh(factory);
|
||||
registerAggregateFunctionsQuantileExactInclusive(factory);
|
||||
|
@ -25,6 +25,7 @@ static const std::unordered_map<String, String> quantile_fuse_name_mapping = {
|
||||
{NameQuantileExactInclusive::name, NameQuantilesExactInclusive::name},
|
||||
{NameQuantileExactLow::name, NameQuantilesExactLow::name},
|
||||
{NameQuantileExactWeighted::name, NameQuantilesExactWeighted::name},
|
||||
{NameQuantileInterpolatedWeighted::name, NameQuantilesInterpolatedWeighted::name},
|
||||
{NameQuantileTDigest::name, NameQuantilesTDigest::name},
|
||||
{NameQuantileTDigestWeighted::name, NameQuantilesTDigestWeighted::name},
|
||||
{NameQuantileTiming::name, NameQuantilesTiming::name},
|
||||
@ -61,9 +62,11 @@ void GatherFunctionQuantileData::FuseQuantileAggregatesData::addFuncNode(ASTPtr
|
||||
|
||||
const auto & arguments = func->arguments->children;
|
||||
|
||||
|
||||
bool need_two_args = func->name == NameQuantileDeterministic::name || func->name == NameQuantileExactWeighted::name
|
||||
|| func->name == NameQuantileTimingWeighted::name || func->name == NameQuantileTDigestWeighted::name
|
||||
|| func->name == NameQuantileBFloat16Weighted::name;
|
||||
|| func->name == NameQuantileInterpolatedWeighted::name || func->name == NameQuantileTimingWeighted::name
|
||||
|| func->name == NameQuantileTDigestWeighted::name || func->name == NameQuantileBFloat16Weighted::name;
|
||||
|
||||
if (arguments.size() != (need_two_args ? 2 : 1))
|
||||
return;
|
||||
|
||||
|
@ -1,3 +1,4 @@
|
||||
10 [1,1,1,1,10,10,10,10,100,100,100]
|
||||
10 [1,1,2,4,7,10,35,61,87,100,100]
|
||||
100 100
|
||||
61 61
|
||||
|
@ -1,3 +1,4 @@
|
||||
SELECT quantileExactWeighted(0.5)(x, 1) AS q5, quantilesExactWeighted(0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1)(x, 1) AS qs FROM (SELECT arrayJoin([1, 1, 1, 10, 10, 10, 10, 100, 100, 100]) AS x);
|
||||
SELECT quantileInterpolatedWeighted(0.5)(x, 1) AS q5, quantilesInterpolatedWeighted(0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1)(x, 1) AS qs FROM (SELECT arrayJoin([1, 1, 1, 10, 10, 10, 10, 100, 100, 100]) AS x);
|
||||
SELECT quantileExact(0)(x), quantileTiming(0)(x) FROM (SELECT number + 100 AS x FROM system.numbers LIMIT 10000);
|
||||
SELECT quantileExact(x), quantileTiming(x) FROM (SELECT number % 123 AS x FROM system.numbers LIMIT 10000);
|
||||
|
@ -6,6 +6,8 @@
|
||||
['2016-06-15 23:00:00']
|
||||
2016-06-15 23:00:00
|
||||
['2016-06-15 23:00:00']
|
||||
2016-06-15 23:00:00
|
||||
['2016-06-15 23:00:00']
|
||||
30000
|
||||
[30000]
|
||||
30000
|
||||
|
@ -15,6 +15,9 @@ SELECT quantilesExact(0.2)(d) FROM datetime;
|
||||
SELECT quantileExactWeighted(0.2)(d, 1) FROM datetime;
|
||||
SELECT quantilesExactWeighted(0.2)(d, 1) FROM datetime;
|
||||
|
||||
SELECT quantileInterpolatedWeighted(0.2)(d, 1) FROM datetime;
|
||||
SELECT quantilesInterpolatedWeighted(0.2)(d, 1) FROM datetime;
|
||||
|
||||
SELECT quantileTiming(0.2)(d) FROM datetime;
|
||||
SELECT quantilesTiming(0.2)(d) FROM datetime;
|
||||
|
||||
|
@ -0,0 +1,12 @@
|
||||
quantileInterpolatedWeighted
|
||||
0 0 0 Decimal(38, 8)
|
||||
-25.5 -8.49999999 -5.1 Decimal(38, 8)
|
||||
0 0 0
|
||||
9.7 3.23333333 1.94
|
||||
19.9 6.63333332 3.98
|
||||
30.1 10.03333333 6.02
|
||||
40.3 13.43333332 8.06
|
||||
50 16.66666666 10
|
||||
[-50,-40.4,-30.3,-20.2,-10.1,0,10.1,20.2,30.3,40.4,50]
|
||||
[-16.66666666,-13.46666666,-10.09999999,-6.73333332,-3.36666666,0,3.36666666,6.73333332,10.09999999,13.46666666,16.66666666]
|
||||
[-10,-8.08,-6.06,-4.04,-2.02,0,2.02,4.04,6.06,8.08,10]
|
@ -0,0 +1,27 @@
|
||||
DROP TABLE IF EXISTS decimal;
|
||||
|
||||
CREATE TABLE decimal
|
||||
(
|
||||
a Decimal32(4),
|
||||
b Decimal64(8),
|
||||
c Decimal128(8)
|
||||
) ENGINE = Memory;
|
||||
|
||||
INSERT INTO decimal (a, b, c)
|
||||
SELECT toDecimal32(number - 50, 4), toDecimal64(number - 50, 8) / 3, toDecimal128(number - 50, 8) / 5
|
||||
FROM system.numbers LIMIT 101;
|
||||
|
||||
SELECT 'quantileInterpolatedWeighted';
|
||||
SELECT medianInterpolatedWeighted(a, 1), medianInterpolatedWeighted(b, 2), medianInterpolatedWeighted(c, 3) as x, toTypeName(x) FROM decimal;
|
||||
SELECT quantileInterpolatedWeighted(a, 1), quantileInterpolatedWeighted(b, 2), quantileInterpolatedWeighted(c, 3) as x, toTypeName(x) FROM decimal WHERE a < 0;
|
||||
SELECT quantileInterpolatedWeighted(0.0)(a, 1), quantileInterpolatedWeighted(0.0)(b, 2), quantileInterpolatedWeighted(0.0)(c, 3) FROM decimal WHERE a >= 0;
|
||||
SELECT quantileInterpolatedWeighted(0.2)(a, 1), quantileInterpolatedWeighted(0.2)(b, 2), quantileInterpolatedWeighted(0.2)(c, 3) FROM decimal WHERE a >= 0;
|
||||
SELECT quantileInterpolatedWeighted(0.4)(a, 1), quantileInterpolatedWeighted(0.4)(b, 2), quantileInterpolatedWeighted(0.4)(c, 3) FROM decimal WHERE a >= 0;
|
||||
SELECT quantileInterpolatedWeighted(0.6)(a, 1), quantileInterpolatedWeighted(0.6)(b, 2), quantileInterpolatedWeighted(0.6)(c, 3) FROM decimal WHERE a >= 0;
|
||||
SELECT quantileInterpolatedWeighted(0.8)(a, 1), quantileInterpolatedWeighted(0.8)(b, 2), quantileInterpolatedWeighted(0.8)(c, 3) FROM decimal WHERE a >= 0;
|
||||
SELECT quantileInterpolatedWeighted(1.0)(a, 1), quantileInterpolatedWeighted(1.0)(b, 2), quantileInterpolatedWeighted(1.0)(c, 3) FROM decimal WHERE a >= 0;
|
||||
SELECT quantilesInterpolatedWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(a, 1) FROM decimal;
|
||||
SELECT quantilesInterpolatedWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(b, 2) FROM decimal;
|
||||
SELECT quantilesInterpolatedWeighted(0.0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)(c, 3) FROM decimal;
|
||||
|
||||
DROP TABLE IF EXISTS decimal;
|
Loading…
Reference in New Issue
Block a user