ClickHouse/dbms/src/AggregateFunctions/AggregateFunctionsUniq.cpp

73 lines
3.2 KiB
C++
Raw Normal View History

2015-09-24 12:40:36 +00:00
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionUniq.h>
#include <DB/AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
2015-10-29 01:57:22 +00:00
template <typename Data>
2015-09-24 12:40:36 +00:00
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types)
{
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];
2015-10-29 01:57:22 +00:00
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, Data>(*argument_types[0]);
2015-09-24 12:40:36 +00:00
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
2015-10-29 01:57:22 +00:00
return new AggregateFunctionUniq<DataTypeDate::FieldType, Data>;
2015-09-24 12:40:36 +00:00
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
2015-10-29 01:57:22 +00:00
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, Data>;
2015-09-24 12:40:36 +00:00
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
2015-10-29 01:57:22 +00:00
return new AggregateFunctionUniq<String, Data>;
2015-09-24 12:40:36 +00:00
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
2015-10-29 01:57:22 +00:00
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types)
2015-09-24 12:40:36 +00:00
{
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];
2015-10-29 01:57:22 +00:00
AggregateFunctionPtr res = createWithNumericType<AggregateFunctionUniq, Data>(*argument_types[0]);
2015-09-24 12:40:36 +00:00
if (res)
return res;
else if (typeid_cast<const DataTypeDate *>(&argument_type))
2015-10-29 01:57:22 +00:00
return new AggregateFunctionUniq<DataTypeDate::FieldType, Data<DataTypeDate::FieldType>>;
2015-09-24 12:40:36 +00:00
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
2015-10-29 01:57:22 +00:00
return new AggregateFunctionUniq<DataTypeDateTime::FieldType, Data<DataTypeDateTime::FieldType>>;
2015-09-24 12:40:36 +00:00
else if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
2015-10-29 01:57:22 +00:00
return new AggregateFunctionUniq<String, Data<String>>;
2015-09-24 12:40:36 +00:00
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory)
{
2015-10-29 01:57:22 +00:00
factory.registerFunction({"uniq"}, createAggregateFunctionUniq<AggregateFunctionUniqUniquesHashSetData>);
factory.registerFunction({"uniqHLL12"}, createAggregateFunctionUniq<AggregateFunctionUniqHLL12Data>);
factory.registerFunction({"uniqExact"}, createAggregateFunctionUniq<AggregateFunctionUniqExactData>);
factory.registerFunction({"uniqCombinedRaw"}, createAggregateFunctionUniq<AggregateFunctionUniqCombinedRawData>);
factory.registerFunction({"uniqCombinedLinearCounting"}, createAggregateFunctionUniq<AggregateFunctionUniqCombinedLinearCountingData>);
factory.registerFunction({"uniqCombinedBiasCorrected"}, createAggregateFunctionUniq<AggregateFunctionUniqCombinedBiasCorrectedData>);
factory.registerFunction({"uniqCombined"}, createAggregateFunctionUniq<AggregateFunctionUniqCombinedData>);
2015-09-24 12:40:36 +00:00
}
}