#include #include #include #include #include #include namespace DB { struct Settings; namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } namespace { /// Substitute return type for Date and DateTime template class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray { public: explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) : AggregateFunctionGroupUniqArray(argument_type, parameters_, max_elems_) {} DataTypePtr getReturnType() const override { return std::make_shared(std::make_shared()); } }; template class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray { public: explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) : AggregateFunctionGroupUniqArray(argument_type, parameters_, max_elems_) {} DataTypePtr getReturnType() const override { return std::make_shared(std::make_shared()); } }; template static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args) { WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate(argument_type, std::forward(args)...); else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime(argument_type, std::forward(args)...); else { /// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric if (argument_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) return new AggregateFunctionGroupUniqArrayGeneric(argument_type, std::forward(args)...); else return new AggregateFunctionGroupUniqArrayGeneric(argument_type, std::forward(args)...); } } template inline AggregateFunctionPtr createAggregateFunctionGroupUniqArrayImpl(const std::string & name, const DataTypePtr & argument_type, TArgs ... args) { AggregateFunctionPtr res(createWithNumericType(*argument_type, argument_type, std::forward(args)...)); if (!res) res = AggregateFunctionPtr(createWithExtraTypes(argument_type, std::forward(args)...)); if (!res) throw Exception("Illegal type " + argument_type->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return res; } AggregateFunctionPtr createAggregateFunctionGroupUniqArray( const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { assertUnary(name, argument_types); bool limit_size = false; UInt64 max_elems = std::numeric_limits::max(); if (parameters.empty()) { // no limit } else if (parameters.size() == 1) { auto type = parameters[0].getType(); if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS); if ((type == Field::Types::Int64 && parameters[0].get() < 0) || (type == Field::Types::UInt64 && parameters[0].get() == 0)) throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS); limit_size = true; max_elems = parameters[0].get(); } else throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 0 or 1", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!limit_size) return createAggregateFunctionGroupUniqArrayImpl(name, argument_types[0], parameters); else return createAggregateFunctionGroupUniqArrayImpl(name, argument_types[0], parameters, max_elems); } } void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupUniqArray", { createAggregateFunctionGroupUniqArray, properties }); } }