ClickHouse/src/AggregateFunctions/AggregateFunctionDistinct.cpp

60 lines
2.1 KiB
C++
Raw Normal View History

2020-05-14 02:14:50 +00:00
#include <AggregateFunctions/AggregateFunctionDistinct.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <AggregateFunctions/Helpers.h>
2020-05-14 02:14:50 +00:00
#include <Common/typeid_cast.h>
#include "registerAggregateFunctions.h"
namespace DB
{
2020-05-16 00:15:44 +00:00
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
2020-05-16 00:15:44 +00:00
}
2020-05-14 02:14:50 +00:00
2020-05-16 00:15:44 +00:00
class AggregateFunctionCombinatorDistinct final : public IAggregateFunctionCombinator
{
public:
String getName() const override { return "Distinct"; }
DataTypes transformArguments(const DataTypes & arguments) const override
2020-05-14 02:14:50 +00:00
{
2020-05-16 00:15:44 +00:00
if (arguments.empty())
throw Exception("Incorrect number of arguments for aggregate function with " + getName() + " suffix",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
2020-05-14 02:14:50 +00:00
return arguments;
2020-05-16 00:15:44 +00:00
}
2020-05-14 02:14:50 +00:00
2020-05-16 00:15:44 +00:00
AggregateFunctionPtr transformAggregateFunction(
const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array &) const override
2020-05-14 02:14:50 +00:00
{
AggregateFunctionPtr res;
if (arguments.size() == 1)
{
res = AggregateFunctionPtr(createWithNumericType<AggregateFunctionDistinctSingleNumericImpl>(*arguments[0], nested_function, arguments));
if (res)
return res;
if (arguments[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
return std::make_shared<AggregateFunctionDistinctSingleGenericImpl<true>>(nested_function, arguments);
else
return std::make_shared<AggregateFunctionDistinctSingleGenericImpl<false>>(nested_function, arguments);
}
if (!res)
throw Exception("Illegal type " /* + argument_type->getName() + */
" of argument for aggregate function " + nested_function->getName() + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
2020-05-14 02:14:50 +00:00
}
2020-05-16 00:15:44 +00:00
};
void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFactory & factory)
{
factory.registerCombinator(std::make_shared<AggregateFunctionCombinatorDistinct>());
}
2020-05-14 02:14:50 +00:00
}