2019-05-18 03:21:39 +00:00
# include <Columns/ColumnConst.h>
2020-06-21 16:37:12 +00:00
# include <Columns/ColumnTuple.h>
2020-10-10 06:49:03 +00:00
# include <Columns/ColumnMap.h>
2020-06-21 16:37:12 +00:00
# include <Columns/ColumnsNumber.h>
2019-05-24 02:38:30 +00:00
# include <DataTypes/DataTypesNumber.h>
# include <DataTypes/DataTypeNullable.h>
2020-06-21 16:37:12 +00:00
# include <DataTypes/DataTypeString.h>
# include <DataTypes/DataTypeArray.h>
# include <DataTypes/DataTypeTuple.h>
2020-10-10 06:49:03 +00:00
# include <DataTypes/DataTypeMap.h>
2019-05-24 02:38:30 +00:00
# include <DataTypes/FieldToDataType.h>
2019-09-02 12:57:22 +00:00
# include <Processors/Formats/IRowInputFormat.h>
2019-05-24 02:38:30 +00:00
# include <Functions/FunctionFactory.h>
2019-05-08 01:41:05 +00:00
# include <Interpreters/ExpressionAnalyzer.h>
2019-10-02 19:54:40 +00:00
# include <Interpreters/ReplaceQueryParameterVisitor.h>
2020-07-22 17:13:05 +00:00
# include <Interpreters/TreeRewriter.h>
2020-05-20 20:16:32 +00:00
# include <Interpreters/Context.h>
# include <Interpreters/convertFieldToType.h>
2020-06-21 16:37:12 +00:00
# include <Interpreters/ExpressionActions.h>
2021-03-01 20:10:50 +00:00
# include <Interpreters/castColumn.h>
2019-05-08 01:41:05 +00:00
# include <IO/ReadHelpers.h>
# include <Parsers/ASTExpressionList.h>
# include <Parsers/ASTFunction.h>
2019-05-24 02:38:30 +00:00
# include <Parsers/ASTIdentifier.h>
2019-05-08 01:41:05 +00:00
# include <Parsers/ASTLiteral.h>
2019-10-02 19:54:40 +00:00
# include <Parsers/ASTQueryParameter.h>
2019-05-25 03:31:23 +00:00
# include <Parsers/CommonParsers.h>
2019-09-02 16:26:22 +00:00
# include <Processors/Formats/Impl/ConstantExpressionTemplate.h>
2019-05-08 01:41:05 +00:00
# include <Parsers/ExpressionElementParsers.h>
2019-10-02 17:51:00 +00:00
# include <boost/functional/hash.hpp>
2022-01-30 19:49:48 +00:00
# include <base/sort.h>
2019-05-24 02:38:30 +00:00
2019-05-08 01:41:05 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR ;
2019-05-25 03:31:23 +00:00
extern const int SYNTAX_ERROR ;
2021-11-26 14:49:31 +00:00
extern const int BAD_ARGUMENTS ;
2019-05-08 01:41:05 +00:00
}
2020-03-26 17:13:14 +00:00
2019-09-12 20:23:20 +00:00
struct SpecialParserType
{
2020-03-26 17:13:14 +00:00
SpecialParserType ( ) = default ;
2020-03-26 19:21:31 +00:00
explicit SpecialParserType ( Field : : Types : : Which main_type_ ) : main_type ( main_type_ ) { }
2020-03-26 17:13:14 +00:00
Field : : Types : : Which main_type = Field : : Types : : String ;
2019-09-20 19:44:52 +00:00
bool is_nullable = false ;
2020-03-26 17:13:14 +00:00
bool is_array = false ;
bool is_tuple = false ;
2020-10-10 06:49:03 +00:00
bool is_map = false ;
2020-03-26 17:13:14 +00:00
/// Type and nullability
std : : vector < std : : pair < Field : : Types : : Which , bool > > nested_types ;
2019-09-12 20:23:20 +00:00
2020-03-26 17:13:14 +00:00
bool useDefaultParser ( ) const
{
return main_type = = Field : : Types : : String | | ( ! nested_types . empty ( )
& & std : : all_of (
nested_types . begin ( ) ,
nested_types . end ( ) ,
[ ] ( const auto & type ) { return type . first = = Field : : Types : : String ; } ) ) ;
}
2019-09-12 20:23:20 +00:00
} ;
2019-09-04 16:54:20 +00:00
struct LiteralInfo
2019-05-22 03:29:32 +00:00
{
2020-03-09 02:55:28 +00:00
using ASTLiteralPtr = std : : shared_ptr < ASTLiteral > ;
2019-09-04 16:54:20 +00:00
LiteralInfo ( const ASTLiteralPtr & literal_ , const String & column_name_ , bool force_nullable_ )
: literal ( literal_ ) , dummy_column_name ( column_name_ ) , force_nullable ( force_nullable_ ) { }
ASTLiteralPtr literal ;
String dummy_column_name ;
/// Make column nullable even if expression type is not.
/// (for literals in functions like ifNull and assumeNotNul, which never return NULL even for NULL arguments)
bool force_nullable ;
2019-05-22 03:29:32 +00:00
2019-09-11 19:55:28 +00:00
DataTypePtr type ;
2019-09-12 20:23:20 +00:00
SpecialParserType special_parser ;
2019-09-11 19:55:28 +00:00
} ;
2019-05-24 02:38:30 +00:00
2020-03-26 17:13:14 +00:00
static void fillLiteralInfo ( DataTypes & nested_types , LiteralInfo & info )
{
size_t elements_num = nested_types . size ( ) ;
info . special_parser . nested_types . reserve ( elements_num ) ;
2020-06-17 13:21:06 +00:00
for ( auto & nested_type : nested_types )
2020-03-26 17:13:14 +00:00
{
/// It can be Array(Nullable(nested_type)) or Tuple(..., Nullable(nested_type), ...)
bool is_nullable = false ;
2020-04-22 06:34:20 +00:00
if ( const auto * nullable = dynamic_cast < const DataTypeNullable * > ( nested_type . get ( ) ) )
2020-03-26 17:13:14 +00:00
{
nested_type = nullable - > getNestedType ( ) ;
is_nullable = true ;
}
WhichDataType type_info { nested_type } ;
Field : : Types : : Which field_type ;
/// Promote integers to 64 bit types
if ( type_info . isNativeUInt ( ) )
{
nested_type = std : : make_shared < DataTypeUInt64 > ( ) ;
field_type = Field : : Types : : UInt64 ;
}
else if ( type_info . isNativeInt ( ) )
{
nested_type = std : : make_shared < DataTypeInt64 > ( ) ;
field_type = Field : : Types : : Int64 ;
}
else if ( type_info . isFloat64 ( ) )
{
field_type = Field : : Types : : Float64 ;
}
else if ( type_info . isString ( ) )
{
field_type = Field : : Types : : String ;
}
2020-04-21 12:54:36 +00:00
else if ( type_info . isArray ( ) )
{
field_type = Field : : Types : : Array ;
}
else if ( type_info . isTuple ( ) )
{
field_type = Field : : Types : : Tuple ;
}
2020-10-10 06:49:03 +00:00
else if ( type_info . isMap ( ) )
{
field_type = Field : : Types : : Map ;
}
2020-03-26 17:13:14 +00:00
else
throw Exception ( " Unexpected literal type inside Array: " + nested_type - > getName ( ) + " . It's a bug " ,
ErrorCodes : : LOGICAL_ERROR ) ;
if ( is_nullable )
nested_type = std : : make_shared < DataTypeNullable > ( nested_type ) ;
info . special_parser . nested_types . emplace_back ( field_type , is_nullable ) ;
}
}
2019-09-11 19:55:28 +00:00
/// Extracts ASTLiterals from expression, replaces them with ASTIdentifiers where needed
/// and deduces data types for dummy columns by field type of literal
2019-09-04 16:54:20 +00:00
class ReplaceLiteralsVisitor
{
public :
2019-05-24 02:38:30 +00:00
LiteralsInfo replaced_literals ;
2021-04-10 23:33:54 +00:00
ContextPtr context ;
2019-05-22 03:29:32 +00:00
2021-04-10 23:33:54 +00:00
explicit ReplaceLiteralsVisitor ( ContextPtr context_ ) : context ( context_ ) { }
2019-05-24 02:38:30 +00:00
2019-09-11 19:55:28 +00:00
void visit ( ASTPtr & ast , bool force_nullable )
2019-05-22 03:29:32 +00:00
{
2019-09-04 16:54:20 +00:00
if ( visitIfLiteral ( ast , force_nullable ) )
2019-05-22 19:59:18 +00:00
return ;
2020-04-22 06:34:20 +00:00
if ( auto * function = ast - > as < ASTFunction > ( ) )
2019-09-04 16:54:20 +00:00
visit ( * function , force_nullable ) ;
2019-10-02 19:54:40 +00:00
else if ( ast - > as < ASTQueryParameter > ( ) )
return ;
2019-10-02 17:51:00 +00:00
else if ( ast - > as < ASTIdentifier > ( ) )
2019-05-25 03:31:23 +00:00
throw DB : : Exception ( " Identifier in constant expression " , ErrorCodes : : SYNTAX_ERROR ) ;
2019-05-24 02:38:30 +00:00
else
2019-10-02 17:51:00 +00:00
throw DB : : Exception ( " Syntax error in constant expression " , ErrorCodes : : SYNTAX_ERROR ) ;
2019-05-24 02:38:30 +00:00
}
2019-05-22 19:59:18 +00:00
2019-05-24 02:38:30 +00:00
private :
2019-09-04 16:54:20 +00:00
void visitChildren ( ASTPtr & ast , const ColumnNumbers & dont_visit_children , const std : : vector < char > & force_nullable )
2019-05-24 02:38:30 +00:00
{
for ( size_t i = 0 ; i < ast - > children . size ( ) ; + + i )
if ( std : : find ( dont_visit_children . begin ( ) , dont_visit_children . end ( ) , i ) = = dont_visit_children . end ( ) )
2019-09-04 16:54:20 +00:00
visit ( ast - > children [ i ] , force_nullable [ i ] ) ;
2019-05-24 02:38:30 +00:00
}
2019-09-04 16:54:20 +00:00
void visit ( ASTFunction & function , bool force_nullable )
2019-05-24 02:38:30 +00:00
{
2019-09-20 19:44:52 +00:00
if ( function . name = = " lambda " )
return ;
2019-12-08 21:06:37 +00:00
FunctionOverloadResolverPtr builder = FunctionFactory : : instance ( ) . get ( function . name , context ) ;
2019-10-02 17:51:00 +00:00
/// Do not replace literals which must be constant
ColumnNumbers dont_visit_children = builder - > getArgumentsThatAreAlwaysConstant ( ) ;
2019-09-04 16:54:20 +00:00
/// Allow nullable arguments if function never returns NULL
2019-10-02 17:51:00 +00:00
ColumnNumbers can_always_be_nullable = builder - > getArgumentsThatDontImplyNullableReturnType ( function . arguments - > children . size ( ) ) ;
2019-09-04 16:54:20 +00:00
2019-10-02 17:51:00 +00:00
std : : vector < char > force_nullable_arguments ( function . arguments - > children . size ( ) , force_nullable ) ;
for ( auto & idx : can_always_be_nullable )
if ( idx < force_nullable_arguments . size ( ) )
force_nullable_arguments [ idx ] = true ;
2019-09-04 16:54:20 +00:00
visitChildren ( function . arguments , dont_visit_children , force_nullable_arguments ) ;
2019-05-24 02:38:30 +00:00
}
2019-09-04 16:54:20 +00:00
bool visitIfLiteral ( ASTPtr & ast , bool force_nullable )
2019-05-24 02:38:30 +00:00
{
auto literal = std : : dynamic_pointer_cast < ASTLiteral > ( ast ) ;
if ( ! literal )
return false ;
if ( literal - > begin & & literal - > end )
{
2019-09-06 19:01:44 +00:00
/// Do not replace empty array and array of NULLs
if ( literal - > value . getType ( ) = = Field : : Types : : Array )
{
const Array & array = literal - > value . get < Array > ( ) ;
auto not_null = std : : find_if_not ( array . begin ( ) , array . end ( ) , [ ] ( const auto & elem ) { return elem . isNull ( ) ; } ) ;
if ( not_null = = array . end ( ) )
return true ;
}
2020-10-10 06:49:03 +00:00
else if ( literal - > value . getType ( ) = = Field : : Types : : Map )
{
const Map & map = literal - > value . get < Map > ( ) ;
if ( map . size ( ) % 2 )
return false ;
}
2021-02-15 18:53:24 +00:00
else if ( literal - > value . getType ( ) = = Field : : Types : : Tuple )
{
const Tuple & tuple = literal - > value . get < Tuple > ( ) ;
for ( const auto & value : tuple )
if ( value . isNull ( ) )
return true ;
}
2020-10-10 06:49:03 +00:00
2019-05-24 02:38:30 +00:00
String column_name = " _dummy_ " + std : : to_string ( replaced_literals . size ( ) ) ;
2019-09-04 16:54:20 +00:00
replaced_literals . emplace_back ( literal , column_name , force_nullable ) ;
2019-09-11 19:55:28 +00:00
setDataType ( replaced_literals . back ( ) ) ;
2019-05-24 02:38:30 +00:00
ast = std : : make_shared < ASTIdentifier > ( column_name ) ;
}
return true ;
2019-05-22 03:29:32 +00:00
}
2019-09-11 19:55:28 +00:00
2020-03-18 00:57:00 +00:00
static void setDataType ( LiteralInfo & info )
2019-09-11 19:55:28 +00:00
{
2020-06-17 13:21:06 +00:00
/// Type (Field::Types:Which) of literal in AST can be:
/// 1. simple literal type: String, UInt64, Int64, Float64, Null
/// 2. complex literal type: Array or Tuple of simple literals
/// 3. Array or Tuple of complex literals
2019-09-11 19:55:28 +00:00
/// Null and empty Array literals are considered as tokens, because template with Nullable(Nothing) or Array(Nothing) is useless.
Field : : Types : : Which field_type = info . literal - > value . getType ( ) ;
/// We have to use ParserNumber instead of type->deserializeAsTextQuoted() for arithmetic types
/// to check actual type of literal and avoid possible overflow and precision issues.
2020-03-26 17:13:14 +00:00
info . special_parser = SpecialParserType ( field_type ) ;
2019-09-11 19:55:28 +00:00
/// Do not use 8, 16 and 32 bit types, so template will match all integers
if ( field_type = = Field : : Types : : UInt64 )
info . type = std : : make_shared < DataTypeUInt64 > ( ) ;
else if ( field_type = = Field : : Types : : Int64 )
info . type = std : : make_shared < DataTypeInt64 > ( ) ;
else if ( field_type = = Field : : Types : : Float64 )
info . type = std : : make_shared < DataTypeFloat64 > ( ) ;
else if ( field_type = = Field : : Types : : String )
info . type = std : : make_shared < DataTypeString > ( ) ;
2020-03-26 17:13:14 +00:00
else if ( field_type = = Field : : Types : : Array )
2019-09-11 19:55:28 +00:00
{
2019-09-12 20:23:20 +00:00
info . special_parser . is_array = true ;
2019-09-11 19:55:28 +00:00
info . type = applyVisitor ( FieldToDataType ( ) , info . literal - > value ) ;
2020-03-26 17:13:14 +00:00
DataTypes nested_types = { assert_cast < const DataTypeArray & > ( * info . type ) . getNestedType ( ) } ;
fillLiteralInfo ( nested_types , info ) ;
info . type = std : : make_shared < DataTypeArray > ( nested_types [ 0 ] ) ;
}
else if ( field_type = = Field : : Types : : Tuple )
{
info . special_parser . is_tuple = true ;
info . type = applyVisitor ( FieldToDataType ( ) , info . literal - > value ) ;
auto nested_types = assert_cast < const DataTypeTuple & > ( * info . type ) . getElements ( ) ;
fillLiteralInfo ( nested_types , info ) ;
info . type = std : : make_shared < DataTypeTuple > ( nested_types ) ;
2019-09-11 19:55:28 +00:00
}
2020-10-10 06:49:03 +00:00
else if ( field_type = = Field : : Types : : Map )
{
info . special_parser . is_map = true ;
info . type = applyVisitor ( FieldToDataType ( ) , info . literal - > value ) ;
2020-12-03 03:52:41 +00:00
auto nested_types = assert_cast < const DataTypeMap & > ( * info . type ) . getKeyValueTypes ( ) ;
2020-10-10 06:49:03 +00:00
fillLiteralInfo ( nested_types , info ) ;
info . type = std : : make_shared < DataTypeMap > ( nested_types ) ;
}
2019-09-11 19:55:28 +00:00
else
2021-09-06 15:59:46 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR ,
" Unexpected literal type {} " ,
info . literal - > value . getTypeName ( ) ) ;
2019-09-11 19:55:28 +00:00
/// Allow literal to be NULL, if result column has nullable type or if function never returns NULL
if ( info . force_nullable & & info . type - > canBeInsideNullable ( ) )
2019-09-12 20:23:20 +00:00
{
2019-09-11 19:55:28 +00:00
info . type = makeNullable ( info . type ) ;
2019-09-12 20:23:20 +00:00
info . special_parser . is_nullable = true ;
}
2019-09-11 19:55:28 +00:00
}
2019-05-22 03:29:32 +00:00
} ;
2019-05-08 01:41:05 +00:00
2019-05-22 03:29:32 +00:00
2019-09-04 16:54:20 +00:00
/// Expression template is a sequence of tokens and data types of literals.
/// E.g. template of "position('some string', 'other string') != 0" is
/// ["position", "(", DataTypeString, ",", DataTypeString, ")", "!=", DataTypeUInt64]
2019-09-11 19:55:28 +00:00
ConstantExpressionTemplate : : TemplateStructure : : TemplateStructure ( LiteralsInfo & replaced_literals , TokenIterator expression_begin , TokenIterator expression_end ,
2021-04-10 23:33:54 +00:00
ASTPtr & expression , const IDataType & result_type , bool null_as_default_ , ContextPtr context )
2019-05-08 01:41:05 +00:00
{
2019-10-16 19:52:00 +00:00
null_as_default = null_as_default_ ;
2019-05-22 19:59:18 +00:00
2022-01-30 19:49:48 +00:00
: : sort ( replaced_literals . begin ( ) , replaced_literals . end ( ) , [ ] ( const LiteralInfo & a , const LiteralInfo & b )
2019-05-22 03:29:32 +00:00
{
return a . literal - > begin . value ( ) < b . literal - > begin . value ( ) ;
} ) ;
2019-09-04 16:54:20 +00:00
/// Make sequence of tokens and determine IDataType by Field::Types:Which for each literal.
token_after_literal_idx . reserve ( replaced_literals . size ( ) ) ;
2019-09-12 20:23:20 +00:00
special_parser . resize ( replaced_literals . size ( ) ) ;
2021-03-09 14:46:52 +00:00
serializations . resize ( replaced_literals . size ( ) ) ;
2019-09-04 16:54:20 +00:00
2019-05-22 03:29:32 +00:00
TokenIterator prev_end = expression_begin ;
2019-05-22 19:59:18 +00:00
for ( size_t i = 0 ; i < replaced_literals . size ( ) ; + + i )
2019-05-22 03:29:32 +00:00
{
2019-05-22 19:59:18 +00:00
const LiteralInfo & info = replaced_literals [ i ] ;
2019-05-22 03:29:32 +00:00
if ( info . literal - > begin . value ( ) < prev_end )
2019-09-04 16:54:20 +00:00
throw Exception ( " Cannot replace literals " , ErrorCodes : : LOGICAL_ERROR ) ;
2019-05-22 03:29:32 +00:00
while ( prev_end < info . literal - > begin . value ( ) )
{
tokens . emplace_back ( prev_end - > begin , prev_end - > size ( ) ) ;
+ + prev_end ;
}
token_after_literal_idx . push_back ( tokens . size ( ) ) ;
2019-09-12 20:23:20 +00:00
special_parser [ i ] = info . special_parser ;
2019-05-22 03:29:32 +00:00
2019-09-11 19:55:28 +00:00
literals . insert ( { nullptr , info . type , info . dummy_column_name } ) ;
2019-05-22 03:29:32 +00:00
prev_end = info . literal - > end . value ( ) ;
2021-03-09 14:46:52 +00:00
serializations [ i ] = info . type - > getDefaultSerialization ( ) ;
2019-05-22 03:29:32 +00:00
}
2019-05-08 01:41:05 +00:00
2019-05-22 03:29:32 +00:00
while ( prev_end < expression_end )
{
tokens . emplace_back ( prev_end - > begin , prev_end - > size ( ) ) ;
+ + prev_end ;
}
2019-05-08 01:41:05 +00:00
2019-10-16 19:52:00 +00:00
addNodesToCastResult ( result_type , expression , null_as_default ) ;
2019-05-08 01:41:05 +00:00
2020-07-22 17:13:05 +00:00
auto syntax_result = TreeRewriter ( context ) . analyze ( expression , literals . getNamesAndTypesList ( ) ) ;
2019-10-16 19:52:00 +00:00
result_column_name = expression - > getColumnName ( ) ;
2019-05-22 03:29:32 +00:00
actions_on_literals = ExpressionAnalyzer ( expression , syntax_result , context ) . getActions ( false ) ;
2021-11-26 14:49:31 +00:00
if ( actions_on_literals - > hasArrayJoin ( ) )
throw Exception ( ErrorCodes : : BAD_ARGUMENTS ,
" Array joins are not allowed in constant expressions for IN, VALUES, LIMIT and similar sections. " ) ;
2019-05-08 01:41:05 +00:00
}
2019-09-11 19:55:28 +00:00
size_t ConstantExpressionTemplate : : TemplateStructure : : getTemplateHash ( const ASTPtr & expression ,
const LiteralsInfo & replaced_literals ,
const DataTypePtr & result_column_type ,
2019-10-16 19:52:00 +00:00
bool null_as_default ,
2019-09-13 17:40:48 +00:00
const String & salt )
2019-09-04 16:54:20 +00:00
{
2019-09-13 17:40:48 +00:00
/// TODO distinguish expressions with the same AST and different tokens (e.g. "CAST(expr, 'Type')" and "CAST(expr AS Type)")
2019-09-11 19:55:28 +00:00
SipHash hash_state ;
hash_state . update ( result_column_type - > getName ( ) ) ;
2019-09-04 16:54:20 +00:00
2019-09-11 19:55:28 +00:00
expression - > updateTreeHash ( hash_state ) ;
2019-09-04 16:54:20 +00:00
2019-09-11 19:55:28 +00:00
for ( const auto & info : replaced_literals )
hash_state . update ( info . type - > getName ( ) ) ;
2019-10-16 19:52:00 +00:00
hash_state . update ( null_as_default ) ;
2019-09-11 19:55:28 +00:00
/// Allows distinguish expression in the last column in Values format
2019-09-13 17:40:48 +00:00
hash_state . update ( salt ) ;
2019-09-11 19:55:28 +00:00
2019-10-02 17:51:00 +00:00
IAST : : Hash res128 ;
2021-01-27 00:54:57 +00:00
hash_state . get128 ( res128 ) ;
2019-10-02 17:51:00 +00:00
size_t res = 0 ;
boost : : hash_combine ( res , res128 . first ) ;
boost : : hash_combine ( res , res128 . second ) ;
return res ;
2019-09-11 19:55:28 +00:00
}
2019-09-04 16:54:20 +00:00
2019-09-13 17:40:48 +00:00
ConstantExpressionTemplate : : TemplateStructurePtr
ConstantExpressionTemplate : : Cache : : getFromCacheOrConstruct ( const DataTypePtr & result_column_type ,
2019-10-16 19:52:00 +00:00
bool null_as_default ,
2019-09-13 17:40:48 +00:00
TokenIterator expression_begin ,
TokenIterator expression_end ,
const ASTPtr & expression_ ,
2021-04-10 23:33:54 +00:00
ContextPtr context ,
2019-09-13 17:40:48 +00:00
bool * found_in_cache ,
const String & salt )
2019-09-11 19:55:28 +00:00
{
2019-09-13 17:40:48 +00:00
TemplateStructurePtr res ;
2019-09-11 19:55:28 +00:00
ASTPtr expression = expression_ - > clone ( ) ;
ReplaceLiteralsVisitor visitor ( context ) ;
2019-10-16 19:52:00 +00:00
visitor . visit ( expression , result_column_type - > isNullable ( ) | | null_as_default ) ;
2021-04-10 23:33:54 +00:00
ReplaceQueryParameterVisitor param_visitor ( context - > getQueryParameters ( ) ) ;
2019-10-02 19:54:40 +00:00
param_visitor . visit ( expression ) ;
2019-09-11 19:55:28 +00:00
2019-10-16 19:52:00 +00:00
size_t template_hash = TemplateStructure : : getTemplateHash ( expression , visitor . replaced_literals , result_column_type , null_as_default , salt ) ;
2019-09-13 17:40:48 +00:00
auto iter = cache . find ( template_hash ) ;
if ( iter = = cache . end ( ) )
2019-09-11 19:55:28 +00:00
{
2019-09-13 17:40:48 +00:00
if ( max_size < = cache . size ( ) )
cache . clear ( ) ;
2019-10-16 19:52:00 +00:00
res = std : : make_shared < TemplateStructure > ( visitor . replaced_literals , expression_begin , expression_end ,
expression , * result_column_type , null_as_default , context ) ;
2019-09-13 17:40:48 +00:00
cache . insert ( { template_hash , res } ) ;
if ( found_in_cache )
* found_in_cache = false ;
2019-09-04 16:54:20 +00:00
}
else
2019-09-13 17:40:48 +00:00
{
/// FIXME process collisions correctly
res = iter - > second ;
if ( found_in_cache )
* found_in_cache = true ;
}
return res ;
}
2019-09-04 16:54:20 +00:00
2020-01-14 11:11:01 +00:00
bool ConstantExpressionTemplate : : parseExpression ( ReadBuffer & istr , const FormatSettings & format_settings , const Settings & settings )
2019-05-08 01:41:05 +00:00
{
size_t cur_column = 0 ;
try
{
2020-01-14 11:11:01 +00:00
if ( tryParseExpression ( istr , format_settings , cur_column , settings ) )
2019-09-13 17:40:48 +00:00
{
+ + rows_count ;
2019-09-11 19:55:28 +00:00
return true ;
2019-09-13 17:40:48 +00:00
}
2019-05-12 03:15:08 +00:00
}
catch ( DB : : Exception & e )
2019-05-08 01:41:05 +00:00
{
for ( size_t i = 0 ; i < cur_column ; + + i )
columns [ i ] - > popBack ( 1 ) ;
if ( ! isParseError ( e . code ( ) ) )
throw ;
2019-05-12 03:15:08 +00:00
2019-09-11 19:55:28 +00:00
return false ;
}
for ( size_t i = 0 ; i < cur_column ; + + i )
columns [ i ] - > popBack ( 1 ) ;
return false ;
}
2020-01-14 11:11:01 +00:00
bool ConstantExpressionTemplate : : tryParseExpression ( ReadBuffer & istr , const FormatSettings & format_settings , size_t & cur_column , const Settings & settings )
2019-09-11 19:55:28 +00:00
{
size_t cur_token = 0 ;
size_t num_columns = structure - > literals . columns ( ) ;
while ( cur_column < num_columns )
{
size_t skip_tokens_until = structure - > token_after_literal_idx [ cur_column ] ;
while ( cur_token < skip_tokens_until )
{
/// TODO skip comments
skipWhitespaceIfAny ( istr ) ;
if ( ! checkString ( structure - > tokens [ cur_token + + ] , istr ) )
return false ;
}
skipWhitespaceIfAny ( istr ) ;
const DataTypePtr & type = structure - > literals . getByPosition ( cur_column ) . type ;
2020-01-14 11:11:01 +00:00
if ( format_settings . values . accurate_types_of_literals & & ! structure - > special_parser [ cur_column ] . useDefaultParser ( ) )
2019-09-11 19:55:28 +00:00
{
2020-01-14 11:11:01 +00:00
if ( ! parseLiteralAndAssertType ( istr , type . get ( ) , cur_column , settings ) )
2019-09-11 19:55:28 +00:00
return false ;
}
else
2021-03-09 14:46:52 +00:00
structure - > serializations [ cur_column ] - > deserializeTextQuoted ( * columns [ cur_column ] , istr , format_settings ) ;
2019-09-11 19:55:28 +00:00
+ + cur_column ;
2019-05-08 01:41:05 +00:00
}
2019-09-11 19:55:28 +00:00
while ( cur_token < structure - > tokens . size ( ) )
{
skipWhitespaceIfAny ( istr ) ;
if ( ! checkString ( structure - > tokens [ cur_token + + ] , istr ) )
return false ;
}
return true ;
2019-05-08 01:41:05 +00:00
}
2020-01-14 11:11:01 +00:00
bool ConstantExpressionTemplate : : parseLiteralAndAssertType ( ReadBuffer & istr , const IDataType * complex_type , size_t column_idx , const Settings & settings )
2019-09-04 16:54:20 +00:00
{
2019-09-12 20:23:20 +00:00
using Type = Field : : Types : : Which ;
2019-09-04 16:54:20 +00:00
2019-09-12 20:23:20 +00:00
/// TODO in case of type mismatch return some hints to deduce new template faster
if ( istr . eof ( ) )
return false ;
2019-09-04 16:54:20 +00:00
2019-09-12 20:23:20 +00:00
SpecialParserType type_info = structure - > special_parser [ column_idx ] ;
2019-09-04 16:54:20 +00:00
/// If literal does not fit entirely in the buffer, parsing error will happen.
2019-09-12 20:23:20 +00:00
/// However, it's possible to deduce new template (or use template from cache) after error like it was template mismatch.
2020-10-10 06:49:03 +00:00
if ( type_info . is_array | | type_info . is_tuple | | type_info . is_map )
2019-09-04 16:54:20 +00:00
{
2019-09-12 20:23:20 +00:00
/// TODO faster way to check types without using Parsers
ParserArrayOfLiterals parser_array ;
2020-03-24 12:55:35 +00:00
ParserTupleOfLiterals parser_tuple ;
2019-09-12 20:23:20 +00:00
Tokens tokens_number ( istr . position ( ) , istr . buffer ( ) . end ( ) ) ;
2020-01-14 11:11:01 +00:00
IParser : : Pos iterator ( tokens_number , settings . max_parser_depth ) ;
2019-09-12 20:23:20 +00:00
Expected expected ;
ASTPtr ast ;
2021-02-17 22:05:31 +00:00
if ( ! parser_array . parse ( iterator , ast , expected ) & & ! parser_tuple . parse ( iterator , ast , expected ) )
2019-09-11 19:55:28 +00:00
return false ;
2020-03-26 17:13:14 +00:00
2019-09-04 16:54:20 +00:00
istr . position ( ) = const_cast < char * > ( iterator - > begin ) ;
2020-03-24 12:55:35 +00:00
const Field & collection = ast - > as < ASTLiteral & > ( ) . value ;
auto collection_type = applyVisitor ( FieldToDataType ( ) , collection ) ;
2020-03-26 17:13:14 +00:00
DataTypes nested_types ;
if ( type_info . is_array )
nested_types = { assert_cast < const DataTypeArray & > ( * collection_type ) . getNestedType ( ) } ;
2020-10-13 07:02:03 +00:00
else if ( type_info . is_tuple )
2020-03-26 17:13:14 +00:00
nested_types = assert_cast < const DataTypeTuple & > ( * collection_type ) . getElements ( ) ;
2020-10-13 07:02:03 +00:00
else
2020-12-03 03:52:41 +00:00
nested_types = assert_cast < const DataTypeMap & > ( * collection_type ) . getKeyValueTypes ( ) ;
2019-09-04 16:54:20 +00:00
2020-03-26 17:13:14 +00:00
for ( size_t i = 0 ; i < nested_types . size ( ) ; + + i )
2019-09-04 16:54:20 +00:00
{
2020-03-26 17:13:14 +00:00
const auto & [ nested_field_type , is_nullable ] = type_info . nested_types [ i ] ;
if ( is_nullable )
2020-04-22 06:34:20 +00:00
if ( const auto * nullable = dynamic_cast < const DataTypeNullable * > ( nested_types [ i ] . get ( ) ) )
2020-03-26 17:13:14 +00:00
nested_types [ i ] = nullable - > getNestedType ( ) ;
WhichDataType nested_type_info ( nested_types [ i ] ) ;
bool are_types_compatible =
( nested_type_info . isNativeUInt ( ) & & nested_field_type = = Type : : UInt64 ) | |
( nested_type_info . isNativeInt ( ) & & nested_field_type = = Type : : Int64 ) | |
( nested_type_info . isFloat64 ( ) & & nested_field_type = = Type : : Float64 ) ;
if ( ! are_types_compatible )
return false ;
2019-09-04 16:54:20 +00:00
}
2020-03-26 17:13:14 +00:00
Field array_same_types = convertFieldToType ( collection , * complex_type , nullptr ) ;
columns [ column_idx ] - > insert ( array_same_types ) ;
return true ;
2019-09-04 16:54:20 +00:00
}
else
{
2019-09-12 20:23:20 +00:00
Field number ;
2019-09-20 19:44:52 +00:00
if ( type_info . is_nullable & & 4 < = istr . available ( ) & & 0 = = strncasecmp ( istr . position ( ) , " NULL " , 4 ) )
istr . position ( ) + = 4 ;
else
2019-09-12 20:23:20 +00:00
{
2019-10-02 17:51:00 +00:00
/// ParserNumber::parse(...) is about 20x slower than strtod(...)
/// because of using ASTPtr, Expected and Tokens, which are not needed here.
/// Parse numeric literal in the same way, as ParserNumber does, but use strtod and strtoull directly.
2019-09-12 20:23:20 +00:00
bool negative = * istr . position ( ) = = ' - ' ;
if ( negative | | * istr . position ( ) = = ' + ' )
+ + istr . position ( ) ;
static constexpr size_t MAX_LENGTH_OF_NUMBER = 319 ;
char buf [ MAX_LENGTH_OF_NUMBER + 1 ] ;
size_t bytes_to_copy = std : : min ( istr . available ( ) , MAX_LENGTH_OF_NUMBER ) ;
memcpy ( buf , istr . position ( ) , bytes_to_copy ) ;
buf [ bytes_to_copy ] = 0 ;
char * pos_double = buf ;
errno = 0 ;
Float64 float_value = std : : strtod ( buf , & pos_double ) ;
if ( pos_double = = buf | | errno = = ERANGE | | float_value < 0 )
return false ;
2019-09-04 16:54:20 +00:00
2019-09-12 20:23:20 +00:00
if ( negative )
float_value = - float_value ;
2019-09-04 16:54:20 +00:00
2019-09-12 20:23:20 +00:00
char * pos_integer = buf ;
errno = 0 ;
UInt64 uint_value = std : : strtoull ( buf , & pos_integer , 0 ) ;
if ( pos_integer = = pos_double & & errno ! = ERANGE & & ( ! negative | | uint_value < = ( 1ULL < < 63 ) ) )
{
istr . position ( ) + = pos_integer - buf ;
2020-03-26 17:13:14 +00:00
if ( negative & & type_info . main_type = = Type : : Int64 )
2019-09-12 20:23:20 +00:00
number = static_cast < Int64 > ( - uint_value ) ;
2020-03-26 17:13:14 +00:00
else if ( ! negative & & type_info . main_type = = Type : : UInt64 )
2019-09-12 20:23:20 +00:00
number = uint_value ;
else
return false ;
}
2020-03-26 17:13:14 +00:00
else if ( type_info . main_type = = Type : : Float64 )
2019-09-12 20:23:20 +00:00
{
istr . position ( ) + = pos_double - buf ;
number = float_value ;
}
else
return false ;
2019-09-04 16:54:20 +00:00
}
2019-09-12 20:23:20 +00:00
columns [ column_idx ] - > insert ( number ) ;
return true ;
2019-09-04 16:54:20 +00:00
}
}
2021-03-01 20:10:50 +00:00
ColumnPtr ConstantExpressionTemplate : : evaluateAll ( BlockMissingValues & nulls , size_t column_idx , const DataTypePtr & expected_type , size_t offset )
2019-05-08 01:41:05 +00:00
{
2019-09-11 19:55:28 +00:00
Block evaluated = structure - > literals . cloneWithColumns ( std : : move ( columns ) ) ;
columns = structure - > literals . cloneEmptyColumns ( ) ;
if ( ! structure - > literals . columns ( ) )
2019-05-18 03:21:39 +00:00
evaluated . insert ( { ColumnConst : : create ( ColumnUInt8 : : create ( 1 , 0 ) , rows_count ) , std : : make_shared < DataTypeUInt8 > ( ) , " _dummy " } ) ;
2019-09-11 19:55:28 +00:00
structure - > actions_on_literals - > execute ( evaluated ) ;
2019-05-08 01:41:05 +00:00
2019-09-04 16:54:20 +00:00
if ( ! evaluated | | evaluated . rows ( ) ! = rows_count )
throw Exception ( " Number of rows mismatch after evaluation of batch of constant expressions: got " +
std : : to_string ( evaluated . rows ( ) ) + " rows for " + std : : to_string ( rows_count ) + " expressions " ,
2019-05-08 01:41:05 +00:00
ErrorCodes : : LOGICAL_ERROR ) ;
2019-09-11 19:55:28 +00:00
if ( ! evaluated . has ( structure - > result_column_name ) )
throw Exception ( " Cannot evaluate template " + structure - > result_column_name + " , block structure: \n " + evaluated . dumpStructure ( ) ,
2019-09-04 16:54:20 +00:00
ErrorCodes : : LOGICAL_ERROR ) ;
2019-05-08 01:41:05 +00:00
2019-05-23 03:07:49 +00:00
rows_count = 0 ;
2021-03-01 20:10:50 +00:00
auto res = evaluated . getByName ( structure - > result_column_name ) ;
res . column = res . column - > convertToFullColumnIfConst ( ) ;
2019-10-16 19:52:00 +00:00
if ( ! structure - > null_as_default )
2021-03-01 20:10:50 +00:00
return castColumn ( res , expected_type ) ;
2019-10-16 19:52:00 +00:00
/// Extract column with evaluated expression and mask for NULLs
2021-03-01 20:10:50 +00:00
const auto & tuple = assert_cast < const ColumnTuple & > ( * res . column ) ;
2019-10-16 19:52:00 +00:00
if ( tuple . tupleSize ( ) ! = 2 )
throw Exception ( " Invalid tuple size, it'a a bug " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-04-22 06:34:20 +00:00
const auto & is_null = assert_cast < const ColumnUInt8 & > ( tuple . getColumn ( 1 ) ) ;
2019-10-16 19:52:00 +00:00
for ( size_t i = 0 ; i < is_null . size ( ) ; + + i )
if ( is_null . getUInt ( i ) )
nulls . setBit ( column_idx , offset + i ) ;
2021-03-01 20:10:50 +00:00
res . column = tuple . getColumnPtr ( 0 ) ;
res . type = assert_cast < const DataTypeTuple & > ( * res . type ) . getElements ( ) [ 0 ] ;
return castColumn ( res , expected_type ) ;
2019-05-08 01:41:05 +00:00
}
2019-10-16 19:52:00 +00:00
void ConstantExpressionTemplate : : TemplateStructure : : addNodesToCastResult ( const IDataType & result_column_type , ASTPtr & expr , bool null_as_default )
2019-05-08 01:41:05 +00:00
{
2019-10-16 19:52:00 +00:00
/// Replace "expr" with "CAST(expr, 'TypeName')"
2021-09-01 09:12:23 +00:00
/// or with "(if(isNull(_dummy_0 AS _expression), defaultValueOfTypeName('TypeName'), _CAST(_expression, 'TypeName')), isNull(_expression))" if null_as_default is true
2019-10-16 19:52:00 +00:00
if ( null_as_default )
{
expr - > setAlias ( " _expression " ) ;
2019-05-08 01:41:05 +00:00
2021-09-01 09:12:23 +00:00
auto is_null = makeASTFunction ( " isNull " , std : : make_shared < ASTIdentifier > ( " _expression " ) ) ;
is_null - > setAlias ( " _is_expression_nullable " ) ;
auto default_value = makeASTFunction ( " defaultValueOfTypeName " , std : : make_shared < ASTLiteral > ( result_column_type . getName ( ) ) ) ;
auto cast = makeASTFunction ( " _CAST " , std : : move ( expr ) , std : : make_shared < ASTLiteral > ( result_column_type . getName ( ) ) ) ;
2019-05-08 01:41:05 +00:00
2021-09-01 09:12:23 +00:00
auto cond = makeASTFunction ( " if " , std : : move ( is_null ) , std : : move ( default_value ) , std : : move ( cast ) ) ;
expr = makeASTFunction ( " tuple " , std : : move ( cond ) , std : : make_shared < ASTIdentifier > ( " _is_expression_nullable " ) ) ;
}
else
2019-10-16 19:52:00 +00:00
{
2021-09-01 09:12:23 +00:00
expr = makeASTFunction ( " _CAST " , std : : move ( expr ) , std : : make_shared < ASTLiteral > ( result_column_type . getName ( ) ) ) ;
2019-10-16 19:52:00 +00:00
}
2019-05-08 01:41:05 +00:00
}
}