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>
|
2016-07-12 13:02:52 +00:00
|
|
|
#include <DB/DataTypes/DataTypeNullable.h>
|
2016-07-14 05:22:09 +00:00
|
|
|
#include <DB/Common/StringUtils.h>
|
|
|
|
#include <Poco/String.h>
|
2016-07-12 13:02:52 +00:00
|
|
|
|
2011-09-19 03:40:05 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-12 02:21:15 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int UNKNOWN_AGGREGATE_FUNCTION;
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
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
|
|
|
/// Ничего не проверяет.
|
2016-07-14 05:22:09 +00:00
|
|
|
std::string trimRight(const std::string & in, const char * suffix)
|
2015-02-27 17:38:21 +00:00
|
|
|
{
|
2016-07-14 05:22:09 +00:00
|
|
|
return in.substr(0, in.size() - strlen(suffix));
|
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-11-15 08:31:08 +00:00
|
|
|
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory & factory);
|
2015-09-24 12:40:36 +00:00
|
|
|
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory);
|
|
|
|
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory);
|
2015-11-21 12:47:41 +00:00
|
|
|
void registerAggregateFunctionsQuantileTDigest(AggregateFunctionFactory & factory);
|
2015-09-24 12:40:36 +00:00
|
|
|
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory);
|
|
|
|
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory);
|
|
|
|
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory);
|
|
|
|
void registerAggregateFunctionSum(AggregateFunctionFactory & factory);
|
|
|
|
void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory);
|
|
|
|
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory & factory);
|
2015-12-23 05:43:13 +00:00
|
|
|
void registerAggregateFunctionDebug(AggregateFunctionFactory & factory);
|
2015-09-24 12:40:36 +00:00
|
|
|
|
|
|
|
AggregateFunctionPtr createAggregateFunctionArray(AggregateFunctionPtr & nested);
|
|
|
|
AggregateFunctionPtr createAggregateFunctionIf(AggregateFunctionPtr & nested);
|
|
|
|
AggregateFunctionPtr createAggregateFunctionState(AggregateFunctionPtr & nested);
|
|
|
|
AggregateFunctionPtr createAggregateFunctionMerge(AggregateFunctionPtr & nested);
|
2016-07-12 13:02:52 +00:00
|
|
|
AggregateFunctionPtr createAggregateFunctionNull(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);
|
2015-11-15 08:31:08 +00:00
|
|
|
registerAggregateFunctionsQuantileExact(*this);
|
|
|
|
registerAggregateFunctionsQuantileExactWeighted(*this);
|
2015-09-24 12:40:36 +00:00
|
|
|
registerAggregateFunctionsQuantileDeterministic(*this);
|
|
|
|
registerAggregateFunctionsQuantileTiming(*this);
|
2015-11-21 12:47:41 +00:00
|
|
|
registerAggregateFunctionsQuantileTDigest(*this);
|
2015-09-24 12:40:36 +00:00
|
|
|
registerAggregateFunctionsSequenceMatch(*this);
|
|
|
|
registerAggregateFunctionsMinMaxAny(*this);
|
|
|
|
registerAggregateFunctionsStatistics(*this);
|
|
|
|
registerAggregateFunctionSum(*this);
|
|
|
|
registerAggregateFunctionsUniq(*this);
|
|
|
|
registerAggregateFunctionUniqUpTo(*this);
|
2015-12-23 05:43:13 +00:00
|
|
|
registerAggregateFunctionDebug(*this);
|
2015-03-01 01:06:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2016-07-14 05:22:09 +00:00
|
|
|
void AggregateFunctionFactory::registerFunction(const String & name, Creator creator, CaseSensitiveness case_sensitiveness)
|
2011-09-19 03:40:05 +00:00
|
|
|
{
|
2015-09-24 12:40:36 +00:00
|
|
|
if (creator == nullptr)
|
2016-07-14 05:22:09 +00:00
|
|
|
throw Exception("AggregateFunctionFactory: the aggregate function " + name + " has been provided "
|
2015-09-24 12:40:36 +00:00
|
|
|
" a null constructor", ErrorCodes::LOGICAL_ERROR);
|
2013-06-29 23:49:34 +00:00
|
|
|
|
2016-07-14 05:22:09 +00:00
|
|
|
if (!aggregate_functions.emplace(name, creator).second)
|
|
|
|
throw Exception("AggregateFunctionFactory: the aggregate function name " + name + " is not unique",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
2015-05-17 17:46:21 +00:00
|
|
|
|
2016-07-14 05:22:09 +00:00
|
|
|
if (case_sensitiveness == CaseInsensitive
|
|
|
|
&& !case_insensitive_aggregate_functions.emplace(Poco::toLower(name), creator).second)
|
|
|
|
throw Exception("AggregateFunctionFactory: the case insensitive aggregate function name " + name + " is not unique",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
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
|
2016-07-12 13:02:52 +00:00
|
|
|
{
|
|
|
|
bool has_nullable_types = false;
|
|
|
|
for (const auto & arg_type : argument_types)
|
|
|
|
{
|
|
|
|
if (arg_type.get()->isNullable())
|
|
|
|
{
|
|
|
|
has_nullable_types = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (has_nullable_types)
|
|
|
|
{
|
|
|
|
DataTypes new_argument_types;
|
|
|
|
new_argument_types.reserve(argument_types.size());
|
|
|
|
|
|
|
|
for (const auto & arg_type : argument_types)
|
|
|
|
{
|
|
|
|
if (arg_type.get()->isNullable())
|
|
|
|
{
|
|
|
|
const DataTypeNullable & actual_type = static_cast<const DataTypeNullable &>(*arg_type.get());
|
|
|
|
const DataTypePtr & nested_type = actual_type.getNestedType();
|
|
|
|
new_argument_types.push_back(nested_type);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
new_argument_types.push_back(arg_type);
|
|
|
|
}
|
|
|
|
|
|
|
|
AggregateFunctionPtr function = getImpl(name, new_argument_types, recursion_level);
|
|
|
|
return createAggregateFunctionNull(function);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return getImpl(name, argument_types, recursion_level);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
AggregateFunctionPtr AggregateFunctionFactory::getImpl(const String & name, const DataTypes & argument_types, int recursion_level) const
|
2015-09-24 12:40:36 +00:00
|
|
|
{
|
|
|
|
auto it = aggregate_functions.find(name);
|
|
|
|
if (it != aggregate_functions.end())
|
2015-05-17 22:33:17 +00:00
|
|
|
{
|
2016-07-14 05:22:09 +00:00
|
|
|
auto it = aggregate_functions.find(name);
|
|
|
|
if (it != aggregate_functions.end())
|
|
|
|
return it->second(name, argument_types);
|
2015-05-17 22:33:17 +00:00
|
|
|
}
|
2016-07-14 05:22:09 +00:00
|
|
|
|
|
|
|
if (recursion_level == 0)
|
|
|
|
{
|
|
|
|
auto it = case_insensitive_aggregate_functions.find(Poco::toLower(name));
|
|
|
|
if (it != case_insensitive_aggregate_functions.end())
|
|
|
|
return it->second(name, argument_types);
|
|
|
|
}
|
|
|
|
|
|
|
|
if ((recursion_level == 0) && endsWith(name, "State"))
|
2014-05-21 13:27:40 +00:00
|
|
|
{
|
|
|
|
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
|
2016-07-14 05:22:09 +00:00
|
|
|
AggregateFunctionPtr nested = get(trimRight(name, "State"), argument_types, recursion_level + 1);
|
2015-09-24 12:40:36 +00:00
|
|
|
return createAggregateFunctionState(nested);
|
2014-05-21 13:27:40 +00:00
|
|
|
}
|
2016-07-14 05:22:09 +00:00
|
|
|
|
|
|
|
if ((recursion_level <= 1) && endsWith(name, "Merge"))
|
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
|
|
|
|
2016-07-14 05:22:09 +00:00
|
|
|
AggregateFunctionPtr nested = get(trimRight(name, "Merge"), 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
|
|
|
}
|
2016-07-14 05:22:09 +00:00
|
|
|
|
|
|
|
if ((recursion_level <= 2) && endsWith(name, "If"))
|
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();
|
2016-07-14 05:22:09 +00:00
|
|
|
AggregateFunctionPtr nested = get(trimRight(name, "If"), nested_dt, recursion_level + 1);
|
2015-09-24 12:40:36 +00:00
|
|
|
return createAggregateFunctionIf(nested);
|
2013-09-11 00:55:31 +00:00
|
|
|
}
|
2016-07-14 05:22:09 +00:00
|
|
|
|
|
|
|
if ((recursion_level <= 3) && endsWith(name, "Array"))
|
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
|
2016-07-14 05:22:09 +00:00
|
|
|
AggregateFunctionPtr nested = get(trimRight(name, "Array"), nested_arguments, recursion_level + 3);
|
2015-09-24 12:40:36 +00:00
|
|
|
return createAggregateFunctionArray(nested);
|
2014-03-27 12:48:09 +00:00
|
|
|
}
|
2016-07-14 05:22:09 +00:00
|
|
|
|
|
|
|
throw Exception("Unknown aggregate function " + name, ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION);
|
2011-09-25 05:07:47 +00:00
|
|
|
}
|
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
|
|
|
|
{
|
2016-07-14 05:22:09 +00:00
|
|
|
if (aggregate_functions.count(name))
|
|
|
|
return true;
|
|
|
|
|
|
|
|
if (recursion_level == 0 && case_insensitive_aggregate_functions.count(Poco::toLower(name)))
|
2015-04-24 15:49:30 +00:00
|
|
|
return true;
|
2016-07-14 05:22:09 +00:00
|
|
|
|
2014-05-21 13:27:40 +00:00
|
|
|
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
|
2016-07-14 05:22:09 +00:00
|
|
|
if ((recursion_level <= 0) && endsWith(name, "State"))
|
|
|
|
return isAggregateFunctionName(trimRight(name, "State"), recursion_level + 1);
|
|
|
|
|
2014-05-21 13:27:40 +00:00
|
|
|
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
|
2016-07-14 05:22:09 +00:00
|
|
|
if ((recursion_level <= 1) && endsWith(name, "Merge"))
|
|
|
|
return isAggregateFunctionName(trimRight(name, "Merge"), recursion_level + 1);
|
|
|
|
|
2013-09-14 22:56:11 +00:00
|
|
|
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
|
2016-07-14 05:22:09 +00:00
|
|
|
if ((recursion_level <= 2) && endsWith(name, "If"))
|
|
|
|
return isAggregateFunctionName(trimRight(name, "If"), recursion_level + 1);
|
|
|
|
|
2014-03-27 12:48:09 +00:00
|
|
|
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
|
2016-07-14 05:22:09 +00:00
|
|
|
if ((recursion_level <= 3) && endsWith(name, "Array"))
|
2015-09-24 12:40:36 +00:00
|
|
|
{
|
|
|
|
/// + 3, чтобы ни один другой модификатор не мог идти перед Array
|
2016-07-14 05:22:09 +00:00
|
|
|
return isAggregateFunctionName(trimRight(name, "Array"), recursion_level + 3);
|
2015-09-24 12:40:36 +00:00
|
|
|
}
|
|
|
|
|
2016-07-14 05:22:09 +00:00
|
|
|
return false;
|
2015-09-24 12:40:36 +00:00
|
|
|
}
|
|
|
|
|
2011-09-19 03:40:05 +00:00
|
|
|
}
|