2011-09-19 03:40:05 +00:00
# include <DB/AggregateFunctions/AggregateFunctionCount.h>
2011-09-25 05:07:47 +00:00
# include <DB/AggregateFunctions/AggregateFunctionSum.h>
2011-09-26 04:00:46 +00:00
# include <DB/AggregateFunctions/AggregateFunctionAvg.h>
# include <DB/AggregateFunctions/AggregateFunctionAny.h>
# include <DB/AggregateFunctions/AggregateFunctionAnyLast.h>
2013-07-28 20:53:31 +00:00
# include <DB/AggregateFunctions/AggregateFunctionsMinMax.h>
2013-10-28 14:15:56 +00:00
# include <DB/AggregateFunctions/AggregateFunctionsArgMinMax.h>
2011-09-26 04:00:46 +00:00
# include <DB/AggregateFunctions/AggregateFunctionUniq.h>
2012-08-26 12:18:50 +00:00
# include <DB/AggregateFunctions/AggregateFunctionGroupArray.h>
2013-07-28 20:53:31 +00:00
# include <DB/AggregateFunctions/AggregateFunctionGroupUniqArray.h>
2013-07-27 19:57:45 +00:00
# include <DB/AggregateFunctions/AggregateFunctionQuantile.h>
# include <DB/AggregateFunctions/AggregateFunctionQuantileTiming.h>
2013-09-11 00:55:31 +00:00
# include <DB/AggregateFunctions/AggregateFunctionIf.h>
2014-03-27 12:48:09 +00:00
# include <DB/AggregateFunctions/AggregateFunctionArray.h>
2014-05-21 13:27:40 +00:00
# include <DB/AggregateFunctions/AggregateFunctionState.h>
# include <DB/AggregateFunctions/AggregateFunctionMerge.h>
2011-09-19 03:40:05 +00:00
# include <DB/AggregateFunctions/AggregateFunctionFactory.h>
2012-10-29 02:58:52 +00:00
# include <DB/DataTypes/DataTypeDate.h>
# include <DB/DataTypes/DataTypeDateTime.h>
2013-06-25 14:16:16 +00:00
# include <DB/DataTypes/DataTypeString.h>
# include <DB/DataTypes/DataTypeFixedString.h>
2011-09-19 03:40:05 +00:00
namespace DB
{
AggregateFunctionFactory : : AggregateFunctionFactory ( )
{
}
2013-06-25 14:16:16 +00:00
/** Создать агрегатную функцию с числовым типом в параметре шаблона, в зависимости от типа аргумента.
*/
template < template < typename > class AggregateFunctionTemplate >
static IAggregateFunction * createWithNumericType ( const IDataType & argument_type )
{
if ( dynamic_cast < const DataTypeUInt8 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt8 > ;
else if ( dynamic_cast < const DataTypeUInt16 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt16 > ;
else if ( dynamic_cast < const DataTypeUInt32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt32 > ;
else if ( dynamic_cast < const DataTypeUInt64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt64 > ;
else if ( dynamic_cast < const DataTypeInt8 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int8 > ;
else if ( dynamic_cast < const DataTypeInt16 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int16 > ;
else if ( dynamic_cast < const DataTypeInt32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int32 > ;
else if ( dynamic_cast < const DataTypeInt64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int64 > ;
else if ( dynamic_cast < const DataTypeFloat32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Float32 > ;
else if ( dynamic_cast < const DataTypeFloat64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Float64 > ;
else
2014-04-08 07:58:53 +00:00
return nullptr ;
2013-06-25 14:16:16 +00:00
}
2013-08-21 13:26:42 +00:00
template < template < typename , typename > class AggregateFunctionTemplate , class Data >
static IAggregateFunction * createWithNumericType ( const IDataType & argument_type )
{
if ( dynamic_cast < const DataTypeUInt8 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt8 , Data > ;
else if ( dynamic_cast < const DataTypeUInt16 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt16 , Data > ;
else if ( dynamic_cast < const DataTypeUInt32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt32 , Data > ;
else if ( dynamic_cast < const DataTypeUInt64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt64 , Data > ;
else if ( dynamic_cast < const DataTypeInt8 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int8 , Data > ;
else if ( dynamic_cast < const DataTypeInt16 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int16 , Data > ;
else if ( dynamic_cast < const DataTypeInt32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int32 , Data > ;
else if ( dynamic_cast < const DataTypeInt64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int64 , Data > ;
else if ( dynamic_cast < const DataTypeFloat32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Float32 , Data > ;
else if ( dynamic_cast < const DataTypeFloat64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Float64 , Data > ;
else
2014-04-08 07:58:53 +00:00
return nullptr ;
2013-08-21 13:26:42 +00:00
}
2014-02-02 09:08:06 +00:00
template < template < typename , typename > class AggregateFunctionTemplate , template < typename > class Data >
static IAggregateFunction * createWithNumericType ( const IDataType & argument_type )
{
if ( dynamic_cast < const DataTypeUInt8 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt8 , Data < UInt8 > > ;
else if ( dynamic_cast < const DataTypeUInt16 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt16 , Data < UInt16 > > ;
else if ( dynamic_cast < const DataTypeUInt32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt32 , Data < UInt32 > > ;
else if ( dynamic_cast < const DataTypeUInt64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < UInt64 , Data < UInt64 > > ;
else if ( dynamic_cast < const DataTypeInt8 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int8 , Data < Int8 > > ;
else if ( dynamic_cast < const DataTypeInt16 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int16 , Data < Int16 > > ;
else if ( dynamic_cast < const DataTypeInt32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int32 , Data < Int32 > > ;
else if ( dynamic_cast < const DataTypeInt64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Int64 , Data < Int64 > > ;
else if ( dynamic_cast < const DataTypeFloat32 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Float32 , Data < Float32 > > ;
else if ( dynamic_cast < const DataTypeFloat64 * > ( & argument_type ) ) return new AggregateFunctionTemplate < Float64 , Data < Float64 > > ;
else
2014-04-08 07:58:53 +00:00
return nullptr ;
2014-02-02 09:08:06 +00:00
}
2013-06-25 14:16:16 +00:00
2013-09-14 22:56:11 +00:00
AggregateFunctionPtr AggregateFunctionFactory : : get ( const String & name , const DataTypes & argument_types , int recursion_level ) const
2011-09-19 03:40:05 +00:00
{
2011-09-25 05:07:47 +00:00
if ( name = = " count " )
return new AggregateFunctionCount ;
2011-09-26 04:00:46 +00:00
else if ( name = = " any " )
return new AggregateFunctionAny ;
else if ( name = = " anyLast " )
return new AggregateFunctionAnyLast ;
else if ( name = = " min " )
return new AggregateFunctionMin ;
else if ( name = = " max " )
return new AggregateFunctionMax ;
2013-10-28 14:15:56 +00:00
else if ( name = = " argMin " )
return new AggregateFunctionArgMin ;
else if ( name = = " argMax " )
return new AggregateFunctionArgMax ;
2012-08-26 12:18:50 +00:00
else if ( name = = " groupArray " )
return new AggregateFunctionGroupArray ;
2013-07-28 20:53:31 +00:00
else if ( name = = " groupUniqArray " )
{
if ( argument_types . size ( ) ! = 1 )
throw Exception ( " Incorrect number of arguments for aggregate function " + name , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
const DataTypeArray * arr = dynamic_cast < const DataTypeArray * > ( & * argument_types [ 0 ] ) ;
AggregateFunctionPtr res ;
if ( ! arr )
res = createWithNumericType < AggregateFunctionGroupUniqArray > ( * argument_types [ 0 ] ) ;
else
res = createWithNumericType < AggregateFunctionGroupUniqArrays > ( * arr - > getNestedType ( ) ) ;
if ( ! res )
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
return res ;
}
2011-09-25 05:07:47 +00:00
else if ( name = = " sum " )
{
if ( argument_types . size ( ) ! = 1 )
throw Exception ( " Incorrect number of arguments for aggregate function " + name , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2013-06-25 14:16:16 +00:00
AggregateFunctionPtr res = createWithNumericType < AggregateFunctionSum > ( * argument_types [ 0 ] ) ;
if ( ! res )
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
return res ;
2011-09-25 05:07:47 +00:00
}
2011-09-26 04:00:46 +00:00
else if ( name = = " avg " )
{
if ( argument_types . size ( ) ! = 1 )
throw Exception ( " Incorrect number of arguments for aggregate function " + name , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2013-06-25 14:16:16 +00:00
AggregateFunctionPtr res = createWithNumericType < AggregateFunctionAvg > ( * argument_types [ 0 ] ) ;
2011-09-26 04:00:46 +00:00
2013-06-25 14:16:16 +00:00
if ( ! res )
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
return res ;
2011-09-26 04:00:46 +00:00
}
else if ( name = = " uniq " )
{
if ( argument_types . size ( ) ! = 1 )
throw Exception ( " Incorrect number of arguments for aggregate function " + name , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2013-06-25 14:16:16 +00:00
const IDataType & argument_type = * argument_types [ 0 ] ;
2013-08-21 13:26:42 +00:00
AggregateFunctionPtr res = createWithNumericType < AggregateFunctionUniq , AggregateFunctionUniqUniquesHashSetData > ( * argument_types [ 0 ] ) ;
if ( res )
return res ;
else if ( dynamic_cast < const DataTypeDate * > ( & argument_type ) )
return new AggregateFunctionUniq < DataTypeDate : : FieldType , AggregateFunctionUniqUniquesHashSetData > ;
else if ( dynamic_cast < const DataTypeDateTime * > ( & argument_type ) )
return new AggregateFunctionUniq < DataTypeDateTime : : FieldType , AggregateFunctionUniqUniquesHashSetData > ;
else if ( dynamic_cast < const DataTypeString * > ( & argument_type ) | | dynamic_cast < const DataTypeFixedString * > ( & argument_type ) )
return new AggregateFunctionUniq < String , AggregateFunctionUniqUniquesHashSetData > ;
else
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
else if ( name = = " uniqHLL12 " )
{
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 < AggregateFunctionUniq , AggregateFunctionUniqHLL12Data > ( * argument_types [ 0 ] ) ;
2013-06-25 14:16:16 +00:00
if ( res )
return res ;
else if ( dynamic_cast < const DataTypeDate * > ( & argument_type ) )
2013-08-21 13:26:42 +00:00
return new AggregateFunctionUniq < DataTypeDate : : FieldType , AggregateFunctionUniqHLL12Data > ;
2013-06-25 14:16:16 +00:00
else if ( dynamic_cast < const DataTypeDateTime * > ( & argument_type ) )
2013-08-21 13:26:42 +00:00
return new AggregateFunctionUniq < DataTypeDateTime : : FieldType , AggregateFunctionUniqHLL12Data > ;
2013-06-25 14:16:16 +00:00
else if ( dynamic_cast < const DataTypeString * > ( & argument_type ) | | dynamic_cast < const DataTypeFixedString * > ( & argument_type ) )
2013-08-21 13:26:42 +00:00
return new AggregateFunctionUniq < String , AggregateFunctionUniqHLL12Data > ;
2011-09-26 04:00:46 +00:00
else
2013-06-25 14:16:16 +00:00
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2011-09-26 04:00:46 +00:00
}
2014-02-02 09:08:06 +00:00
else if ( name = = " uniqExact " )
{
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 < AggregateFunctionUniq , AggregateFunctionUniqExactData > ( * argument_types [ 0 ] ) ;
if ( res )
return res ;
else if ( dynamic_cast < const DataTypeDate * > ( & argument_type ) )
return new AggregateFunctionUniq < DataTypeDate : : FieldType , AggregateFunctionUniqExactData < DataTypeDate : : FieldType > > ;
else if ( dynamic_cast < const DataTypeDateTime * > ( & argument_type ) )
return new AggregateFunctionUniq < DataTypeDateTime : : FieldType , AggregateFunctionUniqExactData < DataTypeDateTime : : FieldType > > ;
else if ( dynamic_cast < const DataTypeString * > ( & argument_type ) | | dynamic_cast < const DataTypeFixedString * > ( & argument_type ) )
return new AggregateFunctionUniq < String , AggregateFunctionUniqExactData < String > > ;
else
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
2012-10-29 02:58:52 +00:00
else if ( name = = " median " | | name = = " quantile " )
2012-10-23 10:58:53 +00:00
{
if ( argument_types . size ( ) ! = 1 )
throw Exception ( " Incorrect number of arguments for aggregate function " + name , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2013-06-25 14:16:16 +00:00
const IDataType & argument_type = * argument_types [ 0 ] ;
if ( dynamic_cast < const DataTypeUInt8 * > ( & argument_type ) ) return new AggregateFunctionQuantile < UInt8 > ;
else if ( dynamic_cast < const DataTypeUInt16 * > ( & argument_type ) ) return new AggregateFunctionQuantile < UInt16 > ;
else if ( dynamic_cast < const DataTypeUInt32 * > ( & argument_type ) ) return new AggregateFunctionQuantile < UInt32 > ;
else if ( dynamic_cast < const DataTypeUInt64 * > ( & argument_type ) ) return new AggregateFunctionQuantile < UInt64 > ;
else if ( dynamic_cast < const DataTypeInt8 * > ( & argument_type ) ) return new AggregateFunctionQuantile < Int8 > ;
else if ( dynamic_cast < const DataTypeInt16 * > ( & argument_type ) ) return new AggregateFunctionQuantile < Int16 > ;
else if ( dynamic_cast < const DataTypeInt32 * > ( & argument_type ) ) return new AggregateFunctionQuantile < Int32 > ;
else if ( dynamic_cast < const DataTypeInt64 * > ( & argument_type ) ) return new AggregateFunctionQuantile < Int64 > ;
else if ( dynamic_cast < const DataTypeFloat32 * > ( & argument_type ) ) return new AggregateFunctionQuantile < Float32 > ;
else if ( dynamic_cast < const DataTypeFloat64 * > ( & argument_type ) ) return new AggregateFunctionQuantile < Float64 > ;
else if ( dynamic_cast < const DataTypeDate * > ( & argument_type ) ) return new AggregateFunctionQuantile < DataTypeDate : : FieldType , false > ;
else if ( dynamic_cast < const DataTypeDateTime * > ( & argument_type ) ) return new AggregateFunctionQuantile < DataTypeDateTime : : FieldType , false > ;
2012-10-23 10:58:53 +00:00
else
2013-06-25 14:16:16 +00:00
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2012-10-23 10:58:53 +00:00
}
2013-06-29 23:49:34 +00:00
else if ( name = = " quantiles " )
{
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 ] ;
if ( dynamic_cast < const DataTypeUInt8 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < UInt8 > ;
else if ( dynamic_cast < const DataTypeUInt16 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < UInt16 > ;
else if ( dynamic_cast < const DataTypeUInt32 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < UInt32 > ;
else if ( dynamic_cast < const DataTypeUInt64 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < UInt64 > ;
else if ( dynamic_cast < const DataTypeInt8 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < Int8 > ;
else if ( dynamic_cast < const DataTypeInt16 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < Int16 > ;
else if ( dynamic_cast < const DataTypeInt32 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < Int32 > ;
else if ( dynamic_cast < const DataTypeInt64 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < Int64 > ;
else if ( dynamic_cast < const DataTypeFloat32 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < Float32 > ;
else if ( dynamic_cast < const DataTypeFloat64 * > ( & argument_type ) ) return new AggregateFunctionQuantiles < Float64 > ;
else if ( dynamic_cast < const DataTypeDate * > ( & argument_type ) ) return new AggregateFunctionQuantiles < DataTypeDate : : FieldType , false > ;
else if ( dynamic_cast < const DataTypeDateTime * > ( & argument_type ) ) return new AggregateFunctionQuantiles < DataTypeDateTime : : FieldType , false > ;
else
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
}
2013-07-27 19:57:45 +00:00
else if ( name = = " medianTiming " | | name = = " quantileTiming " )
{
if ( argument_types . size ( ) ! = 1 )
throw Exception ( " Incorrect number of arguments for aggregate function " + name , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
AggregateFunctionPtr res = createWithNumericType < AggregateFunctionQuantileTiming > ( * argument_types [ 0 ] ) ;
if ( ! res )
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
return res ;
}
else if ( name = = " quantilesTiming " )
{
if ( argument_types . size ( ) ! = 1 )
throw Exception ( " Incorrect number of arguments for aggregate function " + name , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
AggregateFunctionPtr res = createWithNumericType < AggregateFunctionQuantilesTiming > ( * argument_types [ 0 ] ) ;
if ( ! res )
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
return res ;
}
2014-05-21 13:27:40 +00:00
else if ( recursion_level = = 0 & & name . size ( ) > strlen ( " State " ) & & ! ( strcmp ( name . data ( ) + name . size ( ) - strlen ( " State " ) , " State " ) ) )
{
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
AggregateFunctionPtr nested = get ( String ( name . data ( ) , name . size ( ) - strlen ( " State " ) ) , argument_types , recursion_level + 1 ) ;
return new AggregateFunctionState ( nested ) ;
}
else if ( recursion_level = = 0 & & name . size ( ) > strlen ( " Merge " ) & & ! ( strcmp ( name . data ( ) + name . size ( ) - strlen ( " Merge " ) , " Merge " ) ) )
{
/// Для агрегатных функций вида 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 = dynamic_cast < const DataTypeAggregateFunction * > ( & * argument_types [ 0 ] ) ;
if ( ! function )
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
AggregateFunctionPtr nested = get ( String ( name . data ( ) , name . size ( ) - strlen ( " Merge " ) ) , function - > getArgumentsDataTypes ( ) , recursion_level + 1 ) ;
if ( nested - > getName ( ) ! = function - > getFunctionName ( ) )
throw Exception ( " Illegal type " + argument_types [ 0 ] - > getName ( ) + " of argument for aggregate function " + name , ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
return new AggregateFunctionMerge ( nested ) ;
}
else if ( recursion_level < = 1 & & name . size ( ) > = 3 & & name [ name . size ( ) - 2 ] = = ' I ' & & name [ name . size ( ) - 1 ] = = ' f ' )
2013-09-11 00:55:31 +00:00
{
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 ( ) ;
2014-03-27 12:48:09 +00:00
AggregateFunctionPtr nested = get ( String ( name . data ( ) , name . size ( ) - 2 ) , nested_dt , recursion_level + 1 ) ;
2013-09-11 00:55:31 +00:00
return new AggregateFunctionIf ( nested ) ;
}
2014-05-21 13:27:40 +00:00
else if ( recursion_level < = 2 & & name . size ( ) > strlen ( " Array " ) & & ! ( strcmp ( name . data ( ) + name . size ( ) - strlen ( " Array " ) , " 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 )
{
if ( const DataTypeArray * array = dynamic_cast < const DataTypeArray * > ( & * argument_types [ i ] ) )
nested_arguments . push_back ( array - > getNestedType ( ) ) ;
else
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-05-21 13:27:40 +00:00
AggregateFunctionPtr nested = get ( String ( name . data ( ) , name . size ( ) - strlen ( " Array " ) ) , nested_arguments , recursion_level + 2 ) ; /// + 2, чтобы ни один другой модификатор не мог идти перед Array
2014-03-27 12:48:09 +00:00
return new AggregateFunctionArray ( 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
{
2013-05-24 10:49:19 +00:00
return isAggregateFunctionName ( name )
? get ( name , argument_types )
: NULL ;
}
2012-08-23 19:38:45 +00:00
2013-09-14 22:56:11 +00:00
bool AggregateFunctionFactory : : isAggregateFunctionName ( const String & name , int recursion_level ) const
2013-05-24 10:49:19 +00:00
{
2013-06-30 10:28:17 +00:00
static const char * names [ ] =
{
" count " ,
" any " ,
" anyLast " ,
" min " ,
" max " ,
2013-10-28 14:15:56 +00:00
" argMin " ,
" argMax " ,
2013-06-30 10:28:17 +00:00
" sum " ,
" avg " ,
" uniq " ,
2013-08-21 15:33:45 +00:00
" uniqHLL12 " ,
2014-02-02 09:08:06 +00:00
" uniqExact " ,
2013-06-30 10:28:17 +00:00
" groupArray " ,
2013-07-28 20:53:31 +00:00
" groupUniqArray " ,
2013-06-30 10:28:17 +00:00
" median " ,
" quantile " ,
" quantiles " ,
2013-07-27 19:57:45 +00:00
" medianTiming " ,
" quantileTiming " ,
" quantilesTiming " ,
2013-06-30 10:28:17 +00:00
NULL
} ;
for ( const char * * it = names ; * it ; + + it )
if ( 0 = = strcmp ( * it , name . data ( ) ) )
return true ;
2014-05-21 13:27:40 +00:00
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
if ( recursion_level < = 0 & & name . size ( ) > strlen ( " State " ) & & ! ( strcmp ( name . data ( ) + name . size ( ) - strlen ( " State " ) , " State " ) ) )
return isAggregateFunctionName ( String ( name . data ( ) , name . size ( ) - strlen ( " State " ) ) , recursion_level + 1 ) ;
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
if ( recursion_level < = 0 & & name . size ( ) > strlen ( " Merge " ) & & ! ( strcmp ( name . data ( ) + name . size ( ) - strlen ( " Merge " ) , " Merge " ) ) )
return isAggregateFunctionName ( String ( name . data ( ) , name . size ( ) - strlen ( " Merge " ) ) , recursion_level + 1 ) ;
2013-09-14 22:56:11 +00:00
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
2014-05-21 13:27:40 +00:00
if ( recursion_level < = 1 & & name . size ( ) > = 3 & & name [ name . size ( ) - 2 ] = = ' I ' & & name [ name . size ( ) - 1 ] = = ' f ' )
return isAggregateFunctionName ( String ( name . data ( ) , name . size ( ) - 2 ) , recursion_level + 1 ) ;
2014-03-27 12:48:09 +00:00
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
2014-05-21 13:27:40 +00:00
if ( recursion_level < = 2 & & name . size ( ) > strlen ( " Array " ) & & ! ( strcmp ( name . data ( ) + name . size ( ) - strlen ( " Array " ) , " Array " ) ) )
return isAggregateFunctionName ( String ( name . data ( ) , name . size ( ) - strlen ( " Array " ) ) , recursion_level + 2 ) ; /// + 2, чтобы ни один другой модификатор не мог идти перед Array
2013-09-11 00:55:31 +00:00
2013-06-30 10:28:17 +00:00
return false ;
2011-09-19 03:40:05 +00:00
}
}