#include #include #include namespace DB { namespace { 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]; AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) return new AggregateFunctionUniq; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } AggregateFunctionPtr createAggregateFunctionUniqExact(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]; AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) return new AggregateFunctionUniq; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } AggregateFunctionPtr createAggregateFunctionUniqHLL12(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]; AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } AggregateFunctionPtr createAggregateFunctionUniqCombined(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]; AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } AggregateFunctionPtr createAggregateFunctionUniqCombinedRaw(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]; AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } AggregateFunctionPtr createAggregateFunctionUniqCombinedLinearCounting(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]; AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } AggregateFunctionPtr createAggregateFunctionUniqCombinedBiasCorrected(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]; AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) return new AggregateFunctionUniq>; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } } void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory) { factory.registerFunction({"uniq"}, createAggregateFunctionUniq); factory.registerFunction({"uniqHLL12"}, createAggregateFunctionUniqHLL12); factory.registerFunction({"uniqExact"}, createAggregateFunctionUniqExact); factory.registerFunction({"uniqCombinedRaw"}, createAggregateFunctionUniqCombinedRaw); factory.registerFunction({"uniqCombinedLinearCounting"}, createAggregateFunctionUniqCombinedLinearCounting); factory.registerFunction({"uniqCombinedBiasCorrected"}, createAggregateFunctionUniqCombinedBiasCorrected); factory.registerFunction({"uniqCombined"}, createAggregateFunctionUniqCombined); } }