2011-09-19 03:40:05 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
|
2011-09-25 05:07:47 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionSum.h>
|
2011-09-26 04:00:46 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionAvg.h>
|
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionAny.h>
|
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionAnyLast.h>
|
2013-07-28 20:53:31 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionsMinMax.h>
|
2013-10-28 14:15:56 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionsArgMinMax.h>
|
2011-09-26 04:00:46 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionUniq.h>
|
2012-08-26 12:18:50 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionGroupArray.h>
|
2013-07-28 20:53:31 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionGroupUniqArray.h>
|
2013-07-27 19:57:45 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionQuantile.h>
|
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionQuantileTiming.h>
|
2013-09-11 00:55:31 +00:00
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionIf.h>
|
2011-09-19 03:40:05 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
2012-10-29 02:58:52 +00:00
|
|
|
|
|
|
|
|
|
#include <DB/DataTypes/DataTypeDate.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeDateTime.h>
|
2013-06-25 14:16:16 +00:00
|
|
|
|
#include <DB/DataTypes/DataTypeString.h>
|
|
|
|
|
#include <DB/DataTypes/DataTypeFixedString.h>
|
2011-09-19 03:40:05 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
AggregateFunctionFactory::AggregateFunctionFactory()
|
|
|
|
|
{
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
/** Создать агрегатную функцию с числовым типом в параметре шаблона, в зависимости от типа аргумента.
|
|
|
|
|
*/
|
|
|
|
|
template<template <typename> class AggregateFunctionTemplate>
|
|
|
|
|
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
|
|
|
|
|
{
|
|
|
|
|
if (dynamic_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<UInt16>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<UInt32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<UInt64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Int8>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Int16>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Int32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Int64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionTemplate<Float32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionTemplate<Float64>;
|
|
|
|
|
else
|
|
|
|
|
return NULL;
|
|
|
|
|
}
|
|
|
|
|
|
2013-08-21 13:26:42 +00:00
|
|
|
|
template<template <typename, typename> class AggregateFunctionTemplate, class Data>
|
|
|
|
|
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
|
|
|
|
|
{
|
|
|
|
|
if (dynamic_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionTemplate<UInt8, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<UInt16, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<UInt32, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<UInt64, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Int8, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Int16, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Int32, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Int64, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionTemplate<Float32, Data>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionTemplate<Float64, Data>;
|
|
|
|
|
else
|
|
|
|
|
return NULL;
|
|
|
|
|
}
|
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
|
2013-09-14 22:56:11 +00:00
|
|
|
|
AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const
|
2011-09-19 03:40:05 +00:00
|
|
|
|
{
|
2011-09-25 05:07:47 +00:00
|
|
|
|
if (name == "count")
|
|
|
|
|
return new AggregateFunctionCount;
|
2011-09-26 04:00:46 +00:00
|
|
|
|
else if (name == "any")
|
|
|
|
|
return new AggregateFunctionAny;
|
|
|
|
|
else if (name == "anyLast")
|
|
|
|
|
return new AggregateFunctionAnyLast;
|
|
|
|
|
else if (name == "min")
|
|
|
|
|
return new AggregateFunctionMin;
|
|
|
|
|
else if (name == "max")
|
|
|
|
|
return new AggregateFunctionMax;
|
2013-10-28 14:15:56 +00:00
|
|
|
|
else if (name == "argMin")
|
|
|
|
|
return new AggregateFunctionArgMin;
|
|
|
|
|
else if (name == "argMax")
|
|
|
|
|
return new AggregateFunctionArgMax;
|
2012-08-26 12:18:50 +00:00
|
|
|
|
else if (name == "groupArray")
|
|
|
|
|
return new AggregateFunctionGroupArray;
|
2013-07-28 20:53:31 +00:00
|
|
|
|
else if (name == "groupUniqArray")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
const DataTypeArray * arr = dynamic_cast<const DataTypeArray *>(&*argument_types[0]);
|
|
|
|
|
|
|
|
|
|
AggregateFunctionPtr res;
|
|
|
|
|
|
|
|
|
|
if (!arr)
|
|
|
|
|
res = createWithNumericType<AggregateFunctionGroupUniqArray>(*argument_types[0]);
|
|
|
|
|
else
|
|
|
|
|
res = createWithNumericType<AggregateFunctionGroupUniqArrays>(*arr->getNestedType());
|
|
|
|
|
|
|
|
|
|
if (!res)
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
|
|
return res;
|
|
|
|
|
}
|
2011-09-25 05:07:47 +00:00
|
|
|
|
else if (name == "sum")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionSum>(*argument_types[0]);
|
|
|
|
|
|
|
|
|
|
if (!res)
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
|
|
return res;
|
2011-09-25 05:07:47 +00:00
|
|
|
|
}
|
2011-09-26 04:00:46 +00:00
|
|
|
|
else if (name == "avg")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionAvg>(*argument_types[0]);
|
2011-09-26 04:00:46 +00:00
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
if (!res)
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
|
|
return res;
|
2011-09-26 04:00:46 +00:00
|
|
|
|
}
|
|
|
|
|
else if (name == "uniq")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
const IDataType & argument_type = *argument_types[0];
|
|
|
|
|
|
2013-08-21 13:26:42 +00:00
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqUniquesHashSetData>(*argument_types[0]);
|
|
|
|
|
|
|
|
|
|
if (res)
|
|
|
|
|
return res;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDate *>(&argument_type))
|
|
|
|
|
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDateTime*>(&argument_type))
|
|
|
|
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeString*>(&argument_type) || dynamic_cast<const DataTypeFixedString*>(&argument_type))
|
|
|
|
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqUniquesHashSetData>;
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
else if (name == "uniqHLL12")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
const IDataType & argument_type = *argument_types[0];
|
|
|
|
|
|
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, AggregateFunctionUniqHLL12Data>(*argument_types[0]);
|
2013-06-25 14:16:16 +00:00
|
|
|
|
|
|
|
|
|
if (res)
|
|
|
|
|
return res;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDate *>(&argument_type))
|
2013-08-21 13:26:42 +00:00
|
|
|
|
return new AggregateFunctionUniq<DataTypeDate::FieldType, AggregateFunctionUniqHLL12Data>;
|
2013-06-25 14:16:16 +00:00
|
|
|
|
else if (dynamic_cast<const DataTypeDateTime*>(&argument_type))
|
2013-08-21 13:26:42 +00:00
|
|
|
|
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, AggregateFunctionUniqHLL12Data>;
|
2013-06-25 14:16:16 +00:00
|
|
|
|
else if (dynamic_cast<const DataTypeString*>(&argument_type) || dynamic_cast<const DataTypeFixedString*>(&argument_type))
|
2013-08-21 13:26:42 +00:00
|
|
|
|
return new AggregateFunctionUniq<String, AggregateFunctionUniqHLL12Data>;
|
2011-09-26 04:00:46 +00:00
|
|
|
|
else
|
2013-06-25 14:16:16 +00:00
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2011-09-26 04:00:46 +00:00
|
|
|
|
}
|
2013-04-26 18:57:08 +00:00
|
|
|
|
else if (name == "uniqState")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
const IDataType & argument_type = *argument_types[0];
|
|
|
|
|
|
2013-08-21 13:26:42 +00:00
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniqState, AggregateFunctionUniqUniquesHashSetData>(*argument_types[0]);
|
|
|
|
|
|
|
|
|
|
if (res)
|
|
|
|
|
return res;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDate *>(&argument_type))
|
|
|
|
|
return new AggregateFunctionUniqState<DataTypeDate::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDateTime*>(&argument_type))
|
|
|
|
|
return new AggregateFunctionUniqState<DataTypeDateTime::FieldType, AggregateFunctionUniqUniquesHashSetData>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeString*>(&argument_type) || dynamic_cast<const DataTypeFixedString*>(&argument_type))
|
|
|
|
|
return new AggregateFunctionUniqState<String, AggregateFunctionUniqUniquesHashSetData>;
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
|
|
|
|
else if (name == "uniqHLL12State")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
const IDataType & argument_type = *argument_types[0];
|
|
|
|
|
|
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniqState, AggregateFunctionUniqHLL12Data>(*argument_types[0]);
|
2013-06-25 14:16:16 +00:00
|
|
|
|
|
|
|
|
|
if (res)
|
|
|
|
|
return res;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDate *>(&argument_type))
|
2013-08-21 13:26:42 +00:00
|
|
|
|
return new AggregateFunctionUniqState<DataTypeDate::FieldType, AggregateFunctionUniqHLL12Data>;
|
2013-06-25 14:16:16 +00:00
|
|
|
|
else if (dynamic_cast<const DataTypeDateTime*>(&argument_type))
|
2013-08-21 13:26:42 +00:00
|
|
|
|
return new AggregateFunctionUniqState<DataTypeDateTime::FieldType, AggregateFunctionUniqHLL12Data>;
|
2013-06-25 14:16:16 +00:00
|
|
|
|
else if (dynamic_cast<const DataTypeString*>(&argument_type) || dynamic_cast<const DataTypeFixedString*>(&argument_type))
|
2013-08-21 13:26:42 +00:00
|
|
|
|
return new AggregateFunctionUniqState<String, AggregateFunctionUniqHLL12Data>;
|
2013-04-26 18:57:08 +00:00
|
|
|
|
else
|
2013-06-25 14:16:16 +00:00
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2013-04-26 18:57:08 +00:00
|
|
|
|
}
|
2012-10-29 02:58:52 +00:00
|
|
|
|
else if (name == "median" || name == "quantile")
|
2012-10-23 10:58:53 +00:00
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
2013-06-25 14:16:16 +00:00
|
|
|
|
const IDataType & argument_type = *argument_types[0];
|
|
|
|
|
|
|
|
|
|
if (dynamic_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantile<UInt8>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantile<UInt16>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantile<UInt32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantile<UInt64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantile<Int8>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantile<Int16>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantile<Int32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantile<Int64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantile<Float32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantile<Float64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDate::FieldType, false>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantile<DataTypeDateTime::FieldType, false>;
|
2012-10-23 10:58:53 +00:00
|
|
|
|
else
|
2013-06-25 14:16:16 +00:00
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2012-10-23 10:58:53 +00:00
|
|
|
|
}
|
2013-06-29 23:49:34 +00:00
|
|
|
|
else if (name == "quantiles")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
const IDataType & argument_type = *argument_types[0];
|
|
|
|
|
|
|
|
|
|
if (dynamic_cast<const DataTypeUInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt8>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt16>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<UInt64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionQuantiles<Int8>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionQuantiles<Int16>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionQuantiles<Int32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionQuantiles<Int64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionQuantiles<Float32>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionQuantiles<Float64>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDate::FieldType, false>;
|
|
|
|
|
else if (dynamic_cast<const DataTypeDateTime*>(&argument_type)) return new AggregateFunctionQuantiles<DataTypeDateTime::FieldType, false>;
|
|
|
|
|
else
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
}
|
2013-07-27 19:57:45 +00:00
|
|
|
|
else if (name == "medianTiming" || name == "quantileTiming")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantileTiming>(*argument_types[0]);
|
|
|
|
|
|
|
|
|
|
if (!res)
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
else if (name == "quantilesTiming")
|
|
|
|
|
{
|
|
|
|
|
if (argument_types.size() != 1)
|
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
|
|
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionQuantilesTiming>(*argument_types[0]);
|
|
|
|
|
|
|
|
|
|
if (!res)
|
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
|
|
return res;
|
|
|
|
|
}
|
2013-09-14 22:56:11 +00:00
|
|
|
|
else if (recursion_level == 0 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
|
2013-09-11 00:55:31 +00:00
|
|
|
|
{
|
2013-09-14 22:56:11 +00:00
|
|
|
|
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
|
2013-10-28 16:13:19 +00:00
|
|
|
|
DataTypes nested_dt = argument_types;
|
|
|
|
|
nested_dt.pop_back();
|
|
|
|
|
AggregateFunctionPtr nested = get(String(name.data(), name.size() - 2), nested_dt);
|
2013-09-11 00:55:31 +00:00
|
|
|
|
return new AggregateFunctionIf(nested);
|
|
|
|
|
}
|
2011-09-25 05:07:47 +00:00
|
|
|
|
else
|
|
|
|
|
throw Exception("Unknown aggregate function " + name, ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
|
|
|
|
|
}
|
2011-09-19 03:40:05 +00:00
|
|
|
|
|
2011-09-25 05:07:47 +00:00
|
|
|
|
|
|
|
|
|
AggregateFunctionPtr AggregateFunctionFactory::tryGet(const String & name, const DataTypes & argument_types) const
|
2011-09-19 03:40:05 +00:00
|
|
|
|
{
|
2013-05-24 10:49:19 +00:00
|
|
|
|
return isAggregateFunctionName(name)
|
|
|
|
|
? get(name, argument_types)
|
|
|
|
|
: NULL;
|
|
|
|
|
}
|
|
|
|
|
|
2012-08-23 19:38:45 +00:00
|
|
|
|
|
2013-09-14 22:56:11 +00:00
|
|
|
|
bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int recursion_level) const
|
2013-05-24 10:49:19 +00:00
|
|
|
|
{
|
2013-06-30 10:28:17 +00:00
|
|
|
|
static const char * names[] =
|
|
|
|
|
{
|
|
|
|
|
"count",
|
|
|
|
|
"any",
|
|
|
|
|
"anyLast",
|
|
|
|
|
"min",
|
|
|
|
|
"max",
|
2013-10-28 14:15:56 +00:00
|
|
|
|
"argMin",
|
|
|
|
|
"argMax",
|
2013-06-30 10:28:17 +00:00
|
|
|
|
"sum",
|
|
|
|
|
"avg",
|
|
|
|
|
"uniq",
|
|
|
|
|
"uniqState",
|
2013-08-21 15:33:45 +00:00
|
|
|
|
"uniqHLL12",
|
|
|
|
|
"uniqHLL12State",
|
2013-06-30 10:28:17 +00:00
|
|
|
|
"groupArray",
|
2013-07-28 20:53:31 +00:00
|
|
|
|
"groupUniqArray",
|
2013-06-30 10:28:17 +00:00
|
|
|
|
"median",
|
|
|
|
|
"quantile",
|
|
|
|
|
"quantiles",
|
2013-07-27 19:57:45 +00:00
|
|
|
|
"medianTiming",
|
|
|
|
|
"quantileTiming",
|
|
|
|
|
"quantilesTiming",
|
2013-06-30 10:28:17 +00:00
|
|
|
|
NULL
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
for (const char ** it = names; *it; ++it)
|
|
|
|
|
if (0 == strcmp(*it, name.data()))
|
|
|
|
|
return true;
|
|
|
|
|
|
2013-09-14 22:56:11 +00:00
|
|
|
|
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
|
|
|
|
|
if (recursion_level == 0 && name.size() >= 3 && name[name.size() - 2] == 'I' && name[name.size() - 1] == 'f')
|
|
|
|
|
return isAggregateFunctionName(String(name.data(), name.size() - 2), 1);
|
2013-09-11 00:55:31 +00:00
|
|
|
|
|
2013-06-30 10:28:17 +00:00
|
|
|
|
return false;
|
2011-09-19 03:40:05 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
}
|