mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
Pass Settings to all aggregate function creator
This commit is contained in:
parent
48ab5f947b
commit
a2f1bde5cc
@ -11,6 +11,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -105,7 +106,7 @@ public:
|
|||||||
|
|
||||||
void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory)
|
void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerFunction("aggThrow", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
factory.registerFunction("aggThrow", [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
Float64 throw_probability = 1.0;
|
Float64 throw_probability = 1.0;
|
||||||
if (parameters.size() == 1)
|
if (parameters.size() == 1)
|
||||||
|
@ -1,28 +1,27 @@
|
|||||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
#include <AggregateFunctions/HelpersMinMaxAny.h>
|
#include <AggregateFunctions/HelpersMinMaxAny.h>
|
||||||
#include <AggregateFunctions/FactoryHelpers.h>
|
|
||||||
#include "registerAggregateFunctions.h"
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionAny(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
|
||||||
{
|
{
|
||||||
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types, parameters));
|
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types, parameters, settings));
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionAnyLast(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
|
||||||
{
|
{
|
||||||
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types, parameters));
|
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types, parameters, settings));
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionAnyHeavy(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionAnyHeavy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
|
||||||
{
|
{
|
||||||
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyHeavyData>(name, argument_types, parameters));
|
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyHeavyData>(name, argument_types, parameters, settings));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -11,6 +11,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -3,10 +3,11 @@
|
|||||||
#include <AggregateFunctions/AggregateFunctionAvg.h>
|
#include <AggregateFunctions/AggregateFunctionAvg.h>
|
||||||
#include <AggregateFunctions/Helpers.h>
|
#include <AggregateFunctions/Helpers.h>
|
||||||
#include <AggregateFunctions/FactoryHelpers.h>
|
#include <AggregateFunctions/FactoryHelpers.h>
|
||||||
#include "registerAggregateFunctions.h"
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -20,7 +21,7 @@ bool allowType(const DataTypePtr& type) noexcept
|
|||||||
return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal();
|
return t.isInt() || t.isUInt() || t.isFloat() || t.isDecimal();
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
template <typename T>
|
template <typename T>
|
||||||
using DecimalOrVectorCol = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
|
using DecimalOrVectorCol = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
|
||||||
|
|
||||||
|
@ -4,10 +4,11 @@
|
|||||||
#include <AggregateFunctions/AggregateFunctionAvgWeighted.h>
|
#include <AggregateFunctions/AggregateFunctionAvgWeighted.h>
|
||||||
#include <AggregateFunctions/Helpers.h>
|
#include <AggregateFunctions/Helpers.h>
|
||||||
#include <AggregateFunctions/FactoryHelpers.h>
|
#include <AggregateFunctions/FactoryHelpers.h>
|
||||||
#include "registerAggregateFunctions.h"
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -60,7 +61,8 @@ static IAggregateFunction * create(const IDataType & first_type, const IDataType
|
|||||||
#undef LINE
|
#undef LINE
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr
|
||||||
|
createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
template <class T>
|
template <class T>
|
||||||
using AvgWeightedFieldType = std::conditional_t<IsDecimalNumber<T>,
|
using AvgWeightedFieldType = std::conditional_t<IsDecimalNumber<T>,
|
||||||
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
|
std::conditional_t<std::is_same_v<T, Decimal256>, Decimal256, Decimal128>,
|
||||||
|
@ -7,6 +7,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -16,7 +18,7 @@ namespace
|
|||||||
{
|
{
|
||||||
|
|
||||||
template <template <typename> class Data>
|
template <template <typename> class Data>
|
||||||
AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -15,7 +16,7 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -21,8 +22,8 @@ namespace
|
|||||||
AggregateFunctionPtr createAggregateFunctionCategoricalIV(
|
AggregateFunctionPtr createAggregateFunctionCategoricalIV(
|
||||||
const std::string & name,
|
const std::string & name,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array & params
|
const Array & params,
|
||||||
)
|
const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, params);
|
assertNoParameters(name, params);
|
||||||
|
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
template <typename T = UInt64>
|
template <typename T = UInt64>
|
||||||
class AggregateFunctionCategoricalIV final : public IAggregateFunctionHelper<AggregateFunctionCategoricalIV<T>>
|
class AggregateFunctionCategoricalIV final : public IAggregateFunctionHelper<AggregateFunctionCategoricalIV<T>>
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
/** Create aggregate function combinator by matching suffix in aggregate function name.
|
/** Create aggregate function combinator by matching suffix in aggregate function name.
|
||||||
*/
|
*/
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter(
|
AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter(
|
||||||
const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const
|
const AggregateFunctionPtr &, const DataTypes & types, const Array & params, const AggregateFunctionProperties & /*properties*/) const
|
||||||
@ -16,7 +17,7 @@ AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter(
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertArityAtMost<1>(name, argument_types);
|
assertArityAtMost<1>(name, argument_types);
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
struct AggregateFunctionCountData
|
struct AggregateFunctionCountData
|
||||||
{
|
{
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -20,7 +21,8 @@ namespace
|
|||||||
AggregateFunctionPtr createAggregateFunctionDeltaSum(
|
AggregateFunctionPtr createAggregateFunctionDeltaSum(
|
||||||
const String & name,
|
const String & name,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array & params)
|
const Array & params,
|
||||||
|
const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, params);
|
assertNoParameters(name, params);
|
||||||
|
|
||||||
|
@ -15,6 +15,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
struct AggregationFunctionDeltaSumData
|
struct AggregationFunctionDeltaSumData
|
||||||
|
@ -6,6 +6,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -16,7 +17,7 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionEntropy(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionEntropy(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
if (argument_types.empty())
|
if (argument_types.empty())
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
/** Calculates Shannon Entropy, using HashMap and computing empirical distribution function.
|
/** Calculates Shannon Entropy, using HashMap and computing empirical distribution function.
|
||||||
* Entropy is measured in bits (base-2 logarithm is used).
|
* Entropy is measured in bits (base-2 logarithm is used).
|
||||||
|
@ -23,6 +23,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -137,7 +138,8 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(
|
|||||||
if (!out_properties.returns_default_when_only_null && has_null_arguments)
|
if (!out_properties.returns_default_when_only_null && has_null_arguments)
|
||||||
return nullptr;
|
return nullptr;
|
||||||
|
|
||||||
return found.creator(name, argument_types, parameters);
|
const Settings * settings = query_context ? &query_context->getSettingsRef() : nullptr;
|
||||||
|
return found.creator(name, argument_types, parameters, settings);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Combinators of aggregate functions.
|
/// Combinators of aggregate functions.
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
class Context;
|
class Context;
|
||||||
class IDataType;
|
class IDataType;
|
||||||
@ -26,7 +27,7 @@ using DataTypes = std::vector<DataTypePtr>;
|
|||||||
* Parameters are for "parametric" aggregate functions.
|
* Parameters are for "parametric" aggregate functions.
|
||||||
* For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments".
|
* For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments".
|
||||||
*/
|
*/
|
||||||
using AggregateFunctionCreator = std::function<AggregateFunctionPtr(const String &, const DataTypes &, const Array &)>;
|
using AggregateFunctionCreator = std::function<AggregateFunctionPtr(const String &, const DataTypes &, const Array &, const Settings *)>;
|
||||||
|
|
||||||
struct AggregateFunctionWithProperties
|
struct AggregateFunctionWithProperties
|
||||||
{
|
{
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -49,7 +50,7 @@ inline AggregateFunctionPtr createAggregateFunctionGroupArrayImpl(const DataType
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
|
||||||
@ -83,7 +84,7 @@ AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string & name,
|
|||||||
return createAggregateFunctionGroupArrayImpl<GroupArrayTrait<true, Sampler::NONE>>(argument_types[0], max_elems);
|
return createAggregateFunctionGroupArrayImpl<GroupArrayTrait<true, Sampler::NONE>>(argument_types[0], max_elems);
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionGroupArraySample(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionGroupArraySample(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
|
||||||
|
@ -26,6 +26,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int TOO_LARGE_ARRAY_SIZE;
|
extern const int TOO_LARGE_ARRAY_SIZE;
|
||||||
|
@ -7,6 +7,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
@ -15,7 +17,7 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
|
|
||||||
|
@ -20,6 +20,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -56,7 +57,7 @@ inline AggregateFunctionPtr createAggregateFunctionMovingImpl(const std::string
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <template <typename, typename> class Function>
|
template <template <typename, typename> class Function>
|
||||||
AggregateFunctionPtr createAggregateFunctionMoving(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionMoving(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
|
||||||
|
@ -22,6 +22,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -33,7 +35,7 @@ namespace
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <template <typename> class Data>
|
template <template <typename> class Data>
|
||||||
AggregateFunctionPtr createAggregateFunctionBitmap(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionBitmap(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
@ -57,7 +59,7 @@ namespace
|
|||||||
// Additional aggregate functions to manipulate bitmaps.
|
// Additional aggregate functions to manipulate bitmaps.
|
||||||
template <template <typename, typename> class AggregateFunctionTemplate>
|
template <template <typename, typename> class AggregateFunctionTemplate>
|
||||||
AggregateFunctionPtr
|
AggregateFunctionPtr
|
||||||
createAggregateFunctionBitmapL2(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
createAggregateFunctionBitmapL2(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
/// Counts bitmap operation on numbers.
|
/// Counts bitmap operation on numbers.
|
||||||
template <typename T, typename Data>
|
template <typename T, typename Data>
|
||||||
class AggregateFunctionBitmap final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>
|
class AggregateFunctionBitmap final : public IAggregateFunctionDataHelper<Data, AggregateFunctionBitmap<T, Data>>
|
||||||
|
@ -16,6 +16,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
enum BitmapKind
|
enum BitmapKind
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -72,7 +73,7 @@ inline AggregateFunctionPtr createAggregateFunctionGroupUniqArrayImpl(const std:
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
|
||||||
|
@ -23,6 +23,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -22,7 +23,7 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
if (params.size() != 1)
|
if (params.size() != 1)
|
||||||
throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
@ -26,6 +26,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -16,6 +16,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int BAD_ARGUMENTS;
|
extern const int BAD_ARGUMENTS;
|
||||||
@ -29,7 +31,7 @@ namespace
|
|||||||
using FuncLogisticRegression = AggregateFunctionMLMethod<LinearModelData, NameLogisticRegression>;
|
using FuncLogisticRegression = AggregateFunctionMLMethod<LinearModelData, NameLogisticRegression>;
|
||||||
template <class Method>
|
template <class Method>
|
||||||
AggregateFunctionPtr
|
AggregateFunctionPtr
|
||||||
createAggregateFunctionMLMethod(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
createAggregateFunctionMLMethod(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
if (parameters.size() > 4)
|
if (parameters.size() > 4)
|
||||||
throw Exception(
|
throw Exception(
|
||||||
|
@ -11,6 +11,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
|
@ -12,11 +12,12 @@ extern const int NOT_IMPLEMENTED;
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionMannWhitneyUTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
|
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
#include <iostream>
|
#include <iostream>
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -6,18 +6,19 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
|
||||||
{
|
{
|
||||||
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types, parameters));
|
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types, parameters, settings));
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionArgMax(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
|
||||||
{
|
{
|
||||||
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types, parameters));
|
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMaxData>(name, argument_types, parameters, settings));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -7,6 +7,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -32,11 +34,16 @@ namespace
|
|||||||
|
|
||||||
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory & factory)
|
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerFunction("maxIntersections", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
factory.registerFunction(
|
||||||
{ return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Count, name, argument_types, parameters); });
|
"maxIntersections", [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) {
|
||||||
|
return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Count, name, argument_types, parameters);
|
||||||
|
});
|
||||||
|
|
||||||
factory.registerFunction("maxIntersectionsPosition", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
factory.registerFunction(
|
||||||
{ return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Position, name, argument_types, parameters); });
|
"maxIntersectionsPosition",
|
||||||
|
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) {
|
||||||
|
return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Position, name, argument_types, parameters);
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -19,6 +19,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
@ -6,18 +6,19 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionMin(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
|
||||||
{
|
{
|
||||||
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types, parameters));
|
return AggregateFunctionPtr(createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types, parameters, settings));
|
||||||
}
|
}
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionArgMin(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings * settings)
|
||||||
{
|
{
|
||||||
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types, parameters));
|
return AggregateFunctionPtr(createAggregateFunctionArgMinMax<AggregateFunctionMinData>(name, argument_types, parameters, settings));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -15,6 +15,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
/** Aggregate function that takes arbitrary number of arbitrary arguments and does nothing.
|
/** Aggregate function that takes arbitrary number of arbitrary arguments and does nothing.
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -80,7 +81,7 @@ static constexpr bool supportBigInt()
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <template <typename, bool> class Function>
|
template <template <typename, bool> class Function>
|
||||||
AggregateFunctionPtr createAggregateFunctionQuantile(const std::string & name, const DataTypes & argument_types, const Array & params)
|
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.
|
/// Second argument type check doesn't depend on the type of the first one.
|
||||||
Function<void, true>::assertSecondArg(argument_types);
|
Function<void, true>::assertSecondArg(argument_types);
|
||||||
|
@ -28,6 +28,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -12,11 +12,12 @@ extern const int NOT_IMPLEMENTED;
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionRankCorrelation(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionRankCorrelation(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
struct RankCorrelationData : public StatisticalSample<Float64, Float64>
|
struct RankCorrelationData : public StatisticalSample<Float64, Float64>
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -9,6 +9,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -7,6 +7,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
@ -15,7 +17,7 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionRetention(const std::string & name, const DataTypes & arguments, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionRetention(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, params);
|
assertNoParameters(name, params);
|
||||||
|
|
||||||
|
@ -17,6 +17,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -23,7 +24,7 @@ namespace
|
|||||||
{
|
{
|
||||||
|
|
||||||
template <template <typename, typename> class AggregateFunction, template <typename> class Data>
|
template <template <typename, typename> class AggregateFunction, template <typename> class Data>
|
||||||
AggregateFunctionPtr createAggregateFunctionSequenceBase(const std::string & name, const DataTypes & argument_types, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionSequenceBase(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
if (params.size() != 1)
|
if (params.size() != 1)
|
||||||
throw Exception{"Aggregate function " + name + " requires exactly one parameter.",
|
throw Exception{"Aggregate function " + name + " requires exactly one parameter.",
|
||||||
|
@ -15,6 +15,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -4,10 +4,12 @@
|
|||||||
#include <AggregateFunctions/FactoryHelpers.h>
|
#include <AggregateFunctions/FactoryHelpers.h>
|
||||||
|
|
||||||
#include <Core/TypeListNumber.h>
|
#include <Core/TypeListNumber.h>
|
||||||
#include "registerAggregateFunctions.h"
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -19,8 +21,8 @@ namespace
|
|||||||
AggregateFunctionPtr createAggregateFunctionSimpleLinearRegression(
|
AggregateFunctionPtr createAggregateFunctionSimpleLinearRegression(
|
||||||
const String & name,
|
const String & name,
|
||||||
const DataTypes & arguments,
|
const DataTypes & arguments,
|
||||||
const Array & params
|
const Array & params,
|
||||||
)
|
const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, params);
|
assertNoParameters(name, params);
|
||||||
assertBinary(name, arguments);
|
assertBinary(name, arguments);
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
class AggregateFunctionCombinatorSimpleState final : public IAggregateFunctionCombinator
|
class AggregateFunctionCombinatorSimpleState final : public IAggregateFunctionCombinator
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
/** Not an aggregate function, but an adapter of aggregate functions.
|
/** Not an aggregate function, but an adapter of aggregate functions.
|
||||||
* Aggregate functions with the `SimpleState` suffix is almost identical to the corresponding ones,
|
* Aggregate functions with the `SimpleState` suffix is almost identical to the corresponding ones,
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
/** Not an aggregate function, but an adapter of aggregate functions,
|
/** Not an aggregate function, but an adapter of aggregate functions,
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -17,7 +18,7 @@ namespace
|
|||||||
{
|
{
|
||||||
|
|
||||||
template <template <typename> class FunctionTemplate>
|
template <template <typename> class FunctionTemplate>
|
||||||
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
@ -31,7 +32,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string &
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <template <typename, typename> class FunctionTemplate>
|
template <template <typename, typename> class FunctionTemplate>
|
||||||
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -17,7 +18,7 @@ namespace
|
|||||||
{
|
{
|
||||||
|
|
||||||
template <template <typename> class FunctionTemplate>
|
template <template <typename> class FunctionTemplate>
|
||||||
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
@ -36,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string &
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <template <typename, typename> class FunctionTemplate>
|
template <template <typename, typename> class FunctionTemplate>
|
||||||
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
|
@ -30,6 +30,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
enum class StatisticsFunctionKind
|
enum class StatisticsFunctionKind
|
||||||
{
|
{
|
||||||
|
@ -14,6 +14,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
@ -56,7 +57,7 @@ struct StudentTTestData : public TTestMoments<Float64>
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionStudentTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionStudentTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
|
@ -7,6 +7,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -50,7 +52,7 @@ template <typename T> using AggregateFunctionSumKahan =
|
|||||||
|
|
||||||
|
|
||||||
template <template <typename> class Function>
|
template <template <typename> class Function>
|
||||||
AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
|
@ -15,6 +15,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
/// Uses addOverflow method (if available) to avoid UB for sumWithOverflow()
|
/// Uses addOverflow method (if available) to avoid UB for sumWithOverflow()
|
||||||
///
|
///
|
||||||
|
@ -9,6 +9,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
@ -75,7 +77,7 @@ auto parseArguments(const std::string & name, const DataTypes & arguments)
|
|||||||
// function template that allows to choose the aggregate function variant that
|
// function template that allows to choose the aggregate function variant that
|
||||||
// accepts either normal arguments or tuple argument.
|
// accepts either normal arguments or tuple argument.
|
||||||
template<template <bool tuple_argument> typename MappedFunction>
|
template<template <bool tuple_argument> typename MappedFunction>
|
||||||
AggregateFunctionPtr createAggregateFunctionMap(const std::string & name, const DataTypes & arguments, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionMap(const std::string & name, const DataTypes & arguments, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
auto [keys_type, values_types, tuple_argument] = parseArguments(name, arguments);
|
auto [keys_type, values_types, tuple_argument] = parseArguments(name, arguments);
|
||||||
|
|
||||||
|
@ -21,6 +21,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -23,6 +23,7 @@ extern "C"
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
class ReadBuffer;
|
class ReadBuffer;
|
||||||
class WriteBuffer;
|
class WriteBuffer;
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -57,7 +58,7 @@ static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_ty
|
|||||||
|
|
||||||
|
|
||||||
template <bool is_weighted>
|
template <bool is_weighted>
|
||||||
AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const DataTypes & argument_types, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
if (!is_weighted)
|
if (!is_weighted)
|
||||||
{
|
{
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
|
@ -14,6 +14,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -29,7 +30,7 @@ namespace
|
|||||||
* It differs, for example, in that it uses a trivial hash function, since `uniq` of many arguments first hashes them out itself.
|
* It differs, for example, in that it uses a trivial hash function, since `uniq` of many arguments first hashes them out itself.
|
||||||
*/
|
*/
|
||||||
template <typename Data, typename DataForVariadic>
|
template <typename Data, typename DataForVariadic>
|
||||||
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, params);
|
assertNoParameters(name, params);
|
||||||
|
|
||||||
@ -73,7 +74,7 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <bool is_exact, template <typename> class Data, typename DataForVariadic>
|
template <bool is_exact, template <typename> class Data, typename DataForVariadic>
|
||||||
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, params);
|
assertNoParameters(name, params);
|
||||||
|
|
||||||
|
@ -27,6 +27,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
/// uniq
|
/// uniq
|
||||||
|
|
||||||
|
@ -11,6 +11,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
@ -131,8 +133,16 @@ namespace
|
|||||||
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)
|
void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory)
|
||||||
{
|
{
|
||||||
using namespace std::placeholders;
|
using namespace std::placeholders;
|
||||||
factory.registerFunction("uniqCombined", std::bind(createAggregateFunctionUniqCombined, false, _1, _2, _3)); // NOLINT
|
factory.registerFunction("uniqCombined",
|
||||||
factory.registerFunction("uniqCombined64", std::bind(createAggregateFunctionUniqCombined, true, _1, _2, _3)); // NOLINT
|
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionUniqCombined(false, name, argument_types, parameters);
|
||||||
|
});
|
||||||
|
factory.registerFunction("uniqCombined64",
|
||||||
|
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
|
{
|
||||||
|
return createAggregateFunctionUniqCombined(true, name, argument_types, parameters);
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -22,6 +22,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
namespace detail
|
namespace detail
|
||||||
{
|
{
|
||||||
/** Hash function for uniqCombined/uniqCombined64 (based on Ret).
|
/** Hash function for uniqCombined/uniqCombined64 (based on Ret).
|
||||||
|
@ -10,6 +10,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
@ -24,7 +25,7 @@ namespace
|
|||||||
constexpr UInt8 uniq_upto_max_threshold = 100;
|
constexpr UInt8 uniq_upto_max_threshold = 100;
|
||||||
|
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, const DataTypes & argument_types, const Array & params)
|
AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
||||||
{
|
{
|
||||||
UInt8 threshold = 5; /// default value
|
UInt8 threshold = 5; /// default value
|
||||||
|
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
|
|
||||||
/** Counts the number of unique values up to no more than specified in the parameter.
|
/** Counts the number of unique values up to no more than specified in the parameter.
|
||||||
|
@ -14,6 +14,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
@ -53,7 +54,7 @@ struct WelchTTestData : public TTestMoments<Float64>
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
AggregateFunctionPtr createAggregateFunctionWelchTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
AggregateFunctionPtr createAggregateFunctionWelchTTest(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertBinary(name, argument_types);
|
assertBinary(name, argument_types);
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
|
@ -11,6 +11,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
@ -23,9 +25,9 @@ namespace
|
|||||||
|
|
||||||
template <template <typename> class Data>
|
template <template <typename> class Data>
|
||||||
AggregateFunctionPtr
|
AggregateFunctionPtr
|
||||||
createAggregateFunctionWindowFunnel(const std::string & name, const DataTypes & arguments, const Array & params, const Settings & settings)
|
createAggregateFunctionWindowFunnel(const std::string & name, const DataTypes & arguments, const Array & params, const Settings * settings)
|
||||||
{
|
{
|
||||||
if (!settings.allow_experimental_funnel_functions)
|
if (settings == nullptr || !settings->allow_experimental_funnel_functions)
|
||||||
{
|
{
|
||||||
throw Exception{
|
throw Exception{
|
||||||
"Aggregate function " + name + " is experimental. Set `allow_experimental_funnel_functions` setting to enable it",
|
"Aggregate function " + name + " is experimental. Set `allow_experimental_funnel_functions` setting to enable it",
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -7,6 +7,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
|
@ -39,6 +39,7 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
/** Create an aggregate function with a numeric type in the template parameter, depending on the type of the argument.
|
/** Create an aggregate function with a numeric type in the template parameter, depending on the type of the argument.
|
||||||
*/
|
*/
|
||||||
|
@ -12,6 +12,8 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
struct Settings;
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||||
@ -19,7 +21,7 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
/// min, max, any, anyLast, anyHeavy, etc...
|
/// min, max, any, anyLast, anyHeavy, etc...
|
||||||
template <template <typename> class AggregateFunctionTemplate, template <typename> class Data>
|
template <template <typename> class AggregateFunctionTemplate, template <typename> class Data>
|
||||||
static IAggregateFunction * createAggregateFunctionSingleValue(const String & name, const DataTypes & argument_types, const Array & parameters)
|
static IAggregateFunction * createAggregateFunctionSingleValue(const String & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
assertUnary(name, argument_types);
|
assertUnary(name, argument_types);
|
||||||
@ -73,7 +75,7 @@ static IAggregateFunction * createAggregateFunctionArgMinMaxSecond(const DataTyp
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <template <typename> class MinMaxData>
|
template <template <typename> class MinMaxData>
|
||||||
static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name, const DataTypes & argument_types, const Array & parameters)
|
static IAggregateFunction * createAggregateFunctionArgMinMax(const String & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
assertNoParameters(name, parameters);
|
||||||
|
|
||||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user