2017-04-01 09:19:00 +00:00
|
|
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
|
|
|
#include <AggregateFunctions/Helpers.h>
|
|
|
|
#include <AggregateFunctions/AggregateFunctionQuantileDeterministic.h>
|
2015-09-24 12:40:36 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2017-07-10 23:30:17 +00:00
|
|
|
AggregateFunctionPtr createAggregateFunctionQuantileDeterministic(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
2015-09-24 12:40:36 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (argument_types.size() != 2)
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2015-09-24 12:40:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto determinator_type = argument_types[1].get();
|
|
|
|
if (!typeid_cast<const DataTypeInt32 *>(determinator_type) &&
|
|
|
|
!typeid_cast<const DataTypeUInt32 *>(determinator_type) &&
|
|
|
|
!typeid_cast<const DataTypeInt64 *>(determinator_type) &&
|
|
|
|
!typeid_cast<const DataTypeUInt64 *>(determinator_type))
|
|
|
|
{
|
|
|
|
throw Exception{
|
|
|
|
"Illegal type " + determinator_type->getName() + " of second argument for aggregate function " + name +
|
|
|
|
", Int32, UInt32, Int64 or UInt64 required",
|
2017-07-12 19:20:57 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-09-24 12:40:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const IDataType & argument_type = *argument_types[0];
|
2015-09-24 12:40:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt8>>();
|
|
|
|
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt16>>();
|
|
|
|
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt32>>();
|
|
|
|
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<UInt64>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int8>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int16>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int32>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Int64>>();
|
|
|
|
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Float32>>();
|
|
|
|
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<Float64>>();
|
|
|
|
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<DataTypeDate::FieldType, false>>();
|
|
|
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return std::make_shared<AggregateFunctionQuantileDeterministic<DataTypeDateTime::FieldType, false>>();
|
|
|
|
else
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2015-09-24 12:40:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-07-10 23:30:17 +00:00
|
|
|
AggregateFunctionPtr createAggregateFunctionQuantilesDeterministic(const std::string & name, const DataTypes & argument_types, const Array & parameters)
|
2015-09-24 12:40:36 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (argument_types.size() != 2)
|
|
|
|
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2015-09-24 12:40:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto determinator_type = argument_types[1].get();
|
|
|
|
if (!typeid_cast<const DataTypeInt32 *>(determinator_type) &&
|
|
|
|
!typeid_cast<const DataTypeUInt32 *>(determinator_type) &&
|
|
|
|
!typeid_cast<const DataTypeInt64 *>(determinator_type) &&
|
|
|
|
!typeid_cast<const DataTypeUInt64 *>(determinator_type))
|
|
|
|
{
|
|
|
|
throw Exception{
|
|
|
|
"Illegal type " + determinator_type->getName() + " of second argument for aggregate function " + name +
|
|
|
|
", Int32, UInt32, Int64 or UInt64 required",
|
2017-07-12 19:20:57 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2015-09-24 12:40:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const IDataType & argument_type = *argument_types[0];
|
2015-09-24 12:40:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt8>>();
|
|
|
|
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt16>>();
|
|
|
|
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt32>>();
|
|
|
|
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<UInt64>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int8>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int16>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int32>>();
|
|
|
|
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Int64>>();
|
|
|
|
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Float32>>();
|
|
|
|
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<Float64>>();
|
|
|
|
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<DataTypeDate::FieldType, false>>();
|
|
|
|
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return std::make_shared<AggregateFunctionQuantilesDeterministic<DataTypeDateTime::FieldType, false>>();
|
|
|
|
else
|
|
|
|
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2015-09-24 12:40:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
factory.registerFunction("quantileDeterministic", createAggregateFunctionQuantileDeterministic);
|
|
|
|
factory.registerFunction("medianDeterministic", createAggregateFunctionQuantileDeterministic);
|
|
|
|
factory.registerFunction("quantilesDeterministic", createAggregateFunctionQuantilesDeterministic);
|
2015-09-24 12:40:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|