ClickHouse/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp

244 lines
9.4 KiB
C++
Raw Normal View History

2011-09-19 03:40:05 +00:00
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
2015-09-24 12:40:36 +00:00
#include <DB/DataTypes/DataTypeAggregateFunction.h>
#include <DB/DataTypes/DataTypeArray.h>
2011-09-19 03:40:05 +00:00
namespace DB
{
2015-09-24 12:40:36 +00:00
namespace
2011-09-19 03:40:05 +00:00
{
2015-09-24 12:40:36 +00:00
constexpr size_t strlen_constexpr(const char * in)
{
2015-09-24 12:40:36 +00:00
return (*in == '\0') ? 0 : 1 + strlen_constexpr(in + 1);
}
2015-09-24 12:40:36 +00:00
struct SuffixState
{
2015-09-24 12:40:36 +00:00
static constexpr auto name = "State";
static constexpr auto length = strlen_constexpr(name);
};
2015-09-24 12:40:36 +00:00
struct SuffixIf
{
static constexpr auto name = "If";
static constexpr auto length = strlen_constexpr(name);
};
2015-09-24 12:40:36 +00:00
struct SuffixArray
{
2015-09-24 12:40:36 +00:00
static constexpr auto name = "Array";
static constexpr auto length = strlen_constexpr(name);
};
2015-09-24 12:40:36 +00:00
struct SuffixMerge
{
static constexpr auto name = "Merge";
static constexpr auto length = strlen_constexpr(name);
};
2015-09-24 12:40:36 +00:00
template <typename Suffix>
inline bool endsWith(const std::string & in)
{
2015-09-24 12:40:36 +00:00
return (in.length() > Suffix::length)
&& (in.compare(in.length() - Suffix::length, Suffix::length, Suffix::name) == 0);
}
2015-09-24 12:40:36 +00:00
/// Ничего не проверяет.
template <typename Suffix>
inline std::string trimRight(const std::string & in)
{
2015-09-24 12:40:36 +00:00
return in.substr(0, in.length() - Suffix::length);
}
}
2015-09-24 12:40:36 +00:00
void registerAggregateFunctionAvg(AggregateFunctionFactory & factory);
void registerAggregateFunctionCount(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory);
void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory);
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory);
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory);
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory);
void registerAggregateFunctionSum(AggregateFunctionFactory & factory);
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory);
AggregateFunctionPtr createAggregateFunctionArray(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionIf(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionState(AggregateFunctionPtr & nested);
AggregateFunctionPtr createAggregateFunctionMerge(AggregateFunctionPtr & nested);
2015-09-24 12:40:36 +00:00
AggregateFunctionFactory::AggregateFunctionFactory()
{
2015-09-24 12:40:36 +00:00
registerAggregateFunctionAvg(*this);
registerAggregateFunctionCount(*this);
registerAggregateFunctionGroupArray(*this);
registerAggregateFunctionGroupUniqArray(*this);
registerAggregateFunctionsQuantile(*this);
registerAggregateFunctionsQuantileDeterministic(*this);
registerAggregateFunctionsQuantileTiming(*this);
registerAggregateFunctionsSequenceMatch(*this);
registerAggregateFunctionsMinMaxAny(*this);
registerAggregateFunctionsStatistics(*this);
registerAggregateFunctionSum(*this);
registerAggregateFunctionsUniq(*this);
registerAggregateFunctionUniqUpTo(*this);
}
2015-09-24 12:40:36 +00:00
void AggregateFunctionFactory::registerFunction(const std::vector<std::string> & names, Creator creator)
2011-09-19 03:40:05 +00:00
{
2015-09-24 12:40:36 +00:00
if (names.empty())
throw Exception("AggregateFunctionFactory: no name given for aggregate function", ErrorCodes::LOGICAL_ERROR);
if (creator == nullptr)
throw Exception("AggregateFunctionFactory: the aggregate function " + names[0] + " has been provided "
" a null constructor", ErrorCodes::LOGICAL_ERROR);
2015-09-24 12:40:36 +00:00
bool is_first = true;
2015-09-24 12:40:36 +00:00
for (const auto & name : names)
{
2015-09-24 12:40:36 +00:00
Descriptor desc;
desc.creator = creator;
desc.is_alias = !is_first;
2015-09-24 12:40:36 +00:00
auto res = aggregate_functions.emplace(name, desc);
if (!res.second)
throw Exception("AggregateFunctionFactory: the aggregate function name " + name + " is not unique",
ErrorCodes::LOGICAL_ERROR);
2015-09-24 12:40:36 +00:00
is_first = false;
}
2015-09-24 12:40:36 +00:00
}
2015-09-24 12:40:36 +00:00
AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const
{
auto it = aggregate_functions.find(name);
if (it != aggregate_functions.end())
{
2015-09-24 12:40:36 +00:00
const auto & desc = it->second;
const auto & creator = desc.creator;
return creator(name, argument_types);
}
2015-09-24 12:40:36 +00:00
else if ((recursion_level == 0) && endsWith<SuffixState>(name))
2014-05-21 13:27:40 +00:00
{
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
2015-09-24 12:40:36 +00:00
AggregateFunctionPtr nested = get(trimRight<SuffixState>(name), argument_types, recursion_level + 1);
return createAggregateFunctionState(nested);
2014-05-21 13:27:40 +00:00
}
2015-09-24 12:40:36 +00:00
else if ((recursion_level <= 1) && endsWith<SuffixMerge>(name))
2014-05-21 13:27:40 +00:00
{
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeAggregateFunction * function = typeid_cast<const DataTypeAggregateFunction *>(&*argument_types[0]);
2014-05-21 13:27:40 +00:00
if (!function)
2015-09-24 12:40:36 +00:00
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
2014-05-21 13:27:40 +00:00
2015-09-24 12:40:36 +00:00
AggregateFunctionPtr nested = get(trimRight<SuffixMerge>(name), function->getArgumentsDataTypes(), recursion_level + 1);
2014-05-21 13:27:40 +00:00
if (nested->getName() != function->getFunctionName())
2015-09-24 12:40:36 +00:00
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
2014-05-21 13:27:40 +00:00
2015-09-24 12:40:36 +00:00
return createAggregateFunctionMerge(nested);
2014-05-21 13:27:40 +00:00
}
2015-09-24 12:40:36 +00:00
else if ((recursion_level <= 2) && endsWith<SuffixIf>(name))
{
if (argument_types.empty())
throw Exception{
"Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
DataTypes nested_dt = argument_types;
nested_dt.pop_back();
2015-09-24 12:40:36 +00:00
AggregateFunctionPtr nested = get(trimRight<SuffixIf>(name), nested_dt, recursion_level + 1);
return createAggregateFunctionIf(nested);
}
2015-09-24 12:40:36 +00:00
else if ((recursion_level <= 3) && endsWith<SuffixArray>(name))
{
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
size_t num_agruments = argument_types.size();
DataTypes nested_arguments;
for (size_t i = 0; i < num_agruments; ++i)
{
if (const DataTypeArray * array = typeid_cast<const DataTypeArray *>(&*argument_types[i]))
nested_arguments.push_back(array->getNestedType());
else
2015-09-24 12:40:36 +00:00
throw Exception("Illegal type " + argument_types[i]->getName() + " of argument #" + toString(i + 1) +
" for aggregate function " + name + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
2015-09-24 12:40:36 +00:00
/// + 3, чтобы ни один другой модификатор не мог идти перед Array
AggregateFunctionPtr nested = get(trimRight<SuffixArray>(name), nested_arguments, recursion_level + 3);
return createAggregateFunctionArray(nested);
}
2011-09-25 05:07:47 +00:00
else
throw Exception("Unknown aggregate function " + name, ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
}
2011-09-19 03:40:05 +00:00
2011-09-25 05:07:47 +00:00
AggregateFunctionPtr AggregateFunctionFactory::tryGet(const String & name, const DataTypes & argument_types) const
2011-09-19 03:40:05 +00:00
{
return isAggregateFunctionName(name)
? get(name, argument_types)
2015-09-24 12:40:36 +00:00
: nullptr;
}
2015-04-24 15:49:30 +00:00
bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int recursion_level) const
{
2015-09-24 12:40:36 +00:00
if (aggregate_functions.count(name) > 0)
2015-04-24 15:49:30 +00:00
return true;
2014-05-21 13:27:40 +00:00
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
2015-09-24 12:40:36 +00:00
else if ((recursion_level <= 0) && endsWith<SuffixState>(name))
return isAggregateFunctionName(trimRight<SuffixState>(name), recursion_level + 1);
2014-05-21 13:27:40 +00:00
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
2015-09-24 12:40:36 +00:00
else if ((recursion_level <= 1) && endsWith<SuffixMerge>(name))
return isAggregateFunctionName(trimRight<SuffixMerge>(name), recursion_level + 1);
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
2015-09-24 12:40:36 +00:00
else if ((recursion_level <= 2) && endsWith<SuffixIf>(name))
return isAggregateFunctionName(trimRight<SuffixIf>(name), recursion_level + 1);
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
2015-09-24 12:40:36 +00:00
else if ((recursion_level <= 3) && endsWith<SuffixArray>(name))
{
/// + 3, чтобы ни один другой модификатор не мог идти перед Array
return isAggregateFunctionName(trimRight<SuffixArray>(name), recursion_level + 3);
}
else
return false;
}
AggregateFunctionFactory::Details AggregateFunctionFactory::getDetails(const AggregateFunctionFactory::AggregateFunctions::value_type & entry)
{
const auto & desc = entry.second;
return {entry.first, desc.is_alias};
}
2015-09-24 12:40:36 +00:00
AggregateFunctionFactory::const_iterator AggregateFunctionFactory::begin() const
{
return boost::make_transform_iterator(aggregate_functions.begin(), getDetails);
}
AggregateFunctionFactory::const_iterator AggregateFunctionFactory::end() const
{
return boost::make_transform_iterator(aggregate_functions.end(), getDetails);
2011-09-19 03:40:05 +00:00
}
}