2019-03-08 16:49:10 +00:00
# include <Common/FieldVisitors.h>
# include <Common/typeid_cast.h>
# include <IO/ReadHelpers.h>
# include <Columns/ColumnAggregateFunction.h>
2019-03-29 20:04:04 +00:00
# include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
2019-03-08 16:49:10 +00:00
# include <DataTypes/DataTypeLowCardinality.h>
# include <DataTypes/DataTypeTuple.h>
# include <DataTypes/DataTypeArray.h>
# include <DataTypes/DataTypeFactory.h>
# include <AggregateFunctions/AggregateFunctionFactory.h>
# include <Parsers/ASTFunction.h>
# include <Parsers/ASTLiteral.h>
# include <Parsers/ASTIdentifier.h>
# include <boost/algorithm/string/join.hpp>
2019-03-11 08:24:52 +00:00
namespace DB
{
2019-03-08 16:49:10 +00:00
namespace ErrorCodes
{
extern const int SYNTAX_ERROR ;
extern const int BAD_ARGUMENTS ;
extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS ;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
extern const int LOGICAL_ERROR ;
}
2019-12-31 07:17:39 +00:00
static const std : : vector < String > supported_functions { " any " , " anyLast " , " min " , " max " , " sum " , " groupBitAnd " , " groupBitOr " , " groupBitXor " } ;
2019-03-08 16:49:10 +00:00
2019-03-29 20:04:04 +00:00
String DataTypeCustomSimpleAggregateFunction : : getName ( ) const
2019-03-11 08:24:52 +00:00
{
2019-03-08 16:49:10 +00:00
std : : stringstream stream ;
stream < < " SimpleAggregateFunction( " < < function - > getName ( ) ;
2019-03-11 08:24:52 +00:00
if ( ! parameters . empty ( ) )
{
2019-03-08 16:49:10 +00:00
stream < < " ( " ;
2019-03-11 08:24:52 +00:00
for ( size_t i = 0 ; i < parameters . size ( ) ; + + i )
{
2019-03-08 16:49:10 +00:00
if ( i )
stream < < " , " ;
stream < < applyVisitor ( DB : : FieldVisitorToString ( ) , parameters [ i ] ) ;
}
stream < < " ) " ;
}
2019-03-25 21:51:54 +00:00
for ( const auto & argument_type : argument_types )
2019-03-08 16:49:10 +00:00
stream < < " , " < < argument_type - > getName ( ) ;
stream < < " ) " ;
return stream . str ( ) ;
}
2020-03-10 18:16:14 +00:00
static std : : pair < DataTypePtr , DataTypeCustomDescPtr > create ( const ASTPtr & arguments )
2019-03-08 16:49:10 +00:00
{
String function_name ;
AggregateFunctionPtr function ;
DataTypes argument_types ;
Array params_row ;
if ( ! arguments | | arguments - > children . empty ( ) )
throw Exception ( " Data type SimpleAggregateFunction requires parameters: "
" name of aggregate function and list of data types for arguments " , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2019-03-25 21:54:19 +00:00
if ( const ASTFunction * parametric = arguments - > children [ 0 ] - > as < ASTFunction > ( ) )
2019-03-08 16:49:10 +00:00
{
if ( parametric - > parameters )
throw Exception ( " Unexpected level of parameters to aggregate function " , ErrorCodes : : SYNTAX_ERROR ) ;
function_name = parametric - > name ;
2019-03-25 21:54:19 +00:00
const ASTs & parameters = parametric - > arguments - > as < ASTExpressionList & > ( ) . children ;
2019-03-08 16:49:10 +00:00
params_row . resize ( parameters . size ( ) ) ;
for ( size_t i = 0 ; i < parameters . size ( ) ; + + i )
{
2019-03-25 21:54:19 +00:00
const ASTLiteral * lit = parameters [ i ] - > as < ASTLiteral > ( ) ;
2019-03-08 16:49:10 +00:00
if ( ! lit )
throw Exception ( " Parameters to aggregate functions must be literals " ,
ErrorCodes : : PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS ) ;
params_row [ i ] = lit - > value ;
}
}
2019-08-08 20:02:30 +00:00
else if ( auto opt_name = tryGetIdentifierName ( arguments - > children [ 0 ] ) )
2019-03-08 16:49:10 +00:00
{
function_name = * opt_name ;
}
2019-03-25 21:54:19 +00:00
else if ( arguments - > children [ 0 ] - > as < ASTLiteral > ( ) )
2019-03-08 16:49:10 +00:00
{
throw Exception ( " Aggregate function name for data type SimpleAggregateFunction must be passed as identifier (without quotes) or function " ,
ErrorCodes : : BAD_ARGUMENTS ) ;
}
else
throw Exception ( " Unexpected AST element passed as aggregate function name for data type SimpleAggregateFunction. Must be identifier or function. " ,
ErrorCodes : : BAD_ARGUMENTS ) ;
for ( size_t i = 1 ; i < arguments - > children . size ( ) ; + + i )
argument_types . push_back ( DataTypeFactory : : instance ( ) . get ( arguments - > children [ i ] ) ) ;
if ( function_name . empty ( ) )
throw Exception ( " Logical error: empty name of aggregate function passed " , ErrorCodes : : LOGICAL_ERROR ) ;
function = AggregateFunctionFactory : : instance ( ) . get ( function_name , argument_types , params_row ) ;
// check function
2019-03-11 08:24:52 +00:00
if ( std : : find ( std : : begin ( supported_functions ) , std : : end ( supported_functions ) , function - > getName ( ) ) = = std : : end ( supported_functions ) )
{
2019-03-08 16:49:10 +00:00
throw Exception ( " Unsupported aggregate function " + function - > getName ( ) + " , supported functions are " + boost : : algorithm : : join ( supported_functions , " , " ) ,
ErrorCodes : : BAD_ARGUMENTS ) ;
}
DataTypePtr storage_type = DataTypeFactory : : instance ( ) . get ( argument_types [ 0 ] - > getName ( ) ) ;
2019-03-11 08:24:52 +00:00
if ( ! function - > getReturnType ( ) - > equals ( * removeLowCardinality ( storage_type ) ) )
{
2019-03-08 16:49:10 +00:00
throw Exception ( " Incompatible data types between aggregate function ' " + function - > getName ( ) + " ' which returns " + function - > getReturnType ( ) - > getName ( ) + " and column storage type " + storage_type - > getName ( ) ,
ErrorCodes : : BAD_ARGUMENTS ) ;
}
2019-03-29 20:04:04 +00:00
DataTypeCustomNamePtr custom_name = std : : make_unique < DataTypeCustomSimpleAggregateFunction > ( function , argument_types , params_row ) ;
2019-03-08 16:49:10 +00:00
2019-03-29 20:04:04 +00:00
return std : : make_pair ( storage_type , std : : make_unique < DataTypeCustomDesc > ( std : : move ( custom_name ) , nullptr ) ) ;
2019-03-08 16:49:10 +00:00
}
void registerDataTypeDomainSimpleAggregateFunction ( DataTypeFactory & factory )
{
2019-03-29 20:04:04 +00:00
factory . registerDataTypeCustom ( " SimpleAggregateFunction " , create ) ;
2019-03-08 16:49:10 +00:00
}
}