2011-09-19 03:40:05 +00:00
|
|
|
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
2015-10-12 07:05:54 +00:00
|
|
|
#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)
|
2013-06-25 14:16:16 +00:00
|
|
|
{
|
2015-09-24 12:40:36 +00:00
|
|
|
return (*in == '\0') ? 0 : 1 + strlen_constexpr(in + 1);
|
2013-06-25 14:16:16 +00:00
|
|
|
}
|
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
struct SuffixState
|
2013-08-21 13:26:42 +00:00
|
|
|
{
|
2015-09-24 12:40:36 +00:00
|
|
|
static constexpr auto name = "State";
|
|
|
|
static constexpr auto length = strlen_constexpr(name);
|
|
|
|
};
|
2013-08-21 13:26:42 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
struct SuffixIf
|
|
|
|
{
|
|
|
|
static constexpr auto name = "If";
|
|
|
|
static constexpr auto length = strlen_constexpr(name);
|
|
|
|
};
|
2014-08-18 05:45:41 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
struct SuffixArray
|
2014-02-02 09:08:06 +00:00
|
|
|
{
|
2015-09-24 12:40:36 +00:00
|
|
|
static constexpr auto name = "Array";
|
|
|
|
static constexpr auto length = strlen_constexpr(name);
|
|
|
|
};
|
2014-02-02 09:08:06 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
struct SuffixMerge
|
|
|
|
{
|
|
|
|
static constexpr auto name = "Merge";
|
|
|
|
static constexpr auto length = strlen_constexpr(name);
|
|
|
|
};
|
2013-06-25 14:16:16 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
template <typename Suffix>
|
|
|
|
inline bool endsWith(const std::string & in)
|
2015-02-27 17:38:21 +00:00
|
|
|
{
|
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-02-27 17:38:21 +00:00
|
|
|
}
|
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
/// Ничего не проверяет.
|
|
|
|
template <typename Suffix>
|
|
|
|
inline std::string trimRight(const std::string & in)
|
2015-02-27 17:38:21 +00:00
|
|
|
{
|
2015-09-24 12:40:36 +00:00
|
|
|
return in.substr(0, in.length() - Suffix::length);
|
2015-02-27 17:38:21 +00:00
|
|
|
}
|
|
|
|
|
2014-08-18 05:45:41 +00:00
|
|
|
}
|
|
|
|
|
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);
|
2015-11-15 03:11:24 +00:00
|
|
|
void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory & factory);
|
2015-09-24 12:40:36 +00:00
|
|
|
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);
|
2014-08-18 05:45:41 +00:00
|
|
|
|
2015-03-01 01:06:49 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
AggregateFunctionFactory::AggregateFunctionFactory()
|
2015-03-01 01:06:49 +00:00
|
|
|
{
|
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-11-15 03:11:24 +00:00
|
|
|
registerAggregateFunctionsQuantileExact(*this);
|
2015-03-01 01:06:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
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);
|
2013-06-29 23:49:34 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
bool is_first = true;
|
2015-05-15 15:01:05 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
for (const auto & name : names)
|
2015-05-15 15:01:05 +00:00
|
|
|
{
|
2015-09-24 12:40:36 +00:00
|
|
|
Descriptor desc;
|
|
|
|
desc.creator = creator;
|
|
|
|
desc.is_alias = !is_first;
|
2015-05-17 17:46:21 +00:00
|
|
|
|
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-05-17 17:46:21 +00:00
|
|
|
|
2015-09-24 12:40:36 +00:00
|
|
|
is_first = false;
|
2015-05-17 17:46:21 +00:00
|
|
|
}
|
2015-09-24 12:40:36 +00:00
|
|
|
}
|
2015-05-17 17:46:21 +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-05-17 22:33:17 +00:00
|
|
|
{
|
2015-09-24 12:40:36 +00:00
|
|
|
const auto & desc = it->second;
|
|
|
|
const auto & creator = desc.creator;
|
|
|
|
return creator(name, argument_types);
|
2015-05-17 22:33:17 +00:00
|
|
|
}
|
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);
|
2014-06-26 00:58:14 +00:00
|
|
|
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))
|
2013-09-11 00:55:31 +00:00
|
|
|
{
|
2015-02-19 16:27:19 +00:00
|
|
|
if (argument_types.empty())
|
|
|
|
throw Exception{
|
|
|
|
"Incorrect number of arguments for aggregate function " + name,
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
|
|
|
|
};
|
|
|
|
|
2013-09-14 22:56:11 +00:00
|
|
|
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
|
2013-10-28 16:13:19 +00:00
|
|
|
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);
|
2013-09-11 00:55:31 +00:00
|
|
|
}
|
2015-09-24 12:40:36 +00:00
|
|
|
else if ((recursion_level <= 3) && endsWith<SuffixArray>(name))
|
2014-03-27 12:48:09 +00:00
|
|
|
{
|
|
|
|
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
|
|
|
|
size_t num_agruments = argument_types.size();
|
|
|
|
|
|
|
|
DataTypes nested_arguments;
|
|
|
|
for (size_t i = 0; i < num_agruments; ++i)
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
if (const DataTypeArray * array = typeid_cast<const DataTypeArray *>(&*argument_types[i]))
|
2014-03-27 12:48:09 +00:00
|
|
|
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);
|
2014-03-27 12:48:09 +00:00
|
|
|
}
|
2015-09-24 12:40:36 +00:00
|
|
|
/// + 3, чтобы ни один другой модификатор не мог идти перед Array
|
|
|
|
AggregateFunctionPtr nested = get(trimRight<SuffixArray>(name), nested_arguments, recursion_level + 3);
|
|
|
|
return createAggregateFunctionArray(nested);
|
2014-03-27 12:48:09 +00:00
|
|
|
}
|
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
|
|
|
{
|
2013-05-24 10:49:19 +00:00
|
|
|
return isAggregateFunctionName(name)
|
|
|
|
? get(name, argument_types)
|
2015-09-24 12:40:36 +00:00
|
|
|
: nullptr;
|
2013-05-24 10:49:19 +00:00
|
|
|
}
|
|
|
|
|
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);
|
2013-09-14 22:56:11 +00:00
|
|
|
/// Для агрегатных функций вида 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);
|
2014-03-27 12:48:09 +00:00
|
|
|
/// Для агрегатных функций вида 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};
|
|
|
|
}
|
2013-09-11 00:55:31 +00:00
|
|
|
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
}
|