2015-09-04 21:39:10 +00:00
# include <Poco/Util/Application.h>
2016-11-20 12:43:20 +00:00
# include <Poco/String.h>
2015-09-04 21:39:10 +00:00
2017-04-01 09:19:00 +00:00
# include <DataTypes/FieldToDataType.h>
2013-05-24 10:49:19 +00:00
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTFunction.h>
# include <Parsers/ASTIdentifier.h>
# include <Parsers/ASTLiteral.h>
# include <Parsers/ASTAsterisk.h>
2017-06-19 03:05:05 +00:00
# include <Parsers/ASTQualifiedAsterisk.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTExpressionList.h>
# include <Parsers/ASTSelectQuery.h>
2018-02-26 09:05:06 +00:00
# include <Parsers/ASTSelectWithUnionQuery.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTSubquery.h>
# include <Parsers/ASTOrderByElement.h>
2018-02-05 14:03:01 +00:00
# include <Parsers/formatAST.h>
2013-05-24 10:49:19 +00:00
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeSet.h>
2017-12-25 00:38:45 +00:00
# include <DataTypes/DataTypeNullable.h>
2017-12-25 18:58:39 +00:00
# include <DataTypes/NestedUtils.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypesNumber.h>
2015-01-14 02:44:25 +00:00
2017-04-01 09:19:00 +00:00
# include <Columns/ColumnSet.h>
2017-07-21 06:35:58 +00:00
# include <Columns/ColumnConst.h>
2013-05-24 10:49:19 +00:00
2018-02-26 09:05:06 +00:00
# include <Interpreters/InterpreterSelectWithUnionQuery.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ExpressionAnalyzer.h>
# include <Interpreters/ExpressionActions.h>
# include <Interpreters/InJoinSubqueriesPreprocessor.h>
# include <Interpreters/LogicalExpressionsOptimizer.h>
2018-03-04 16:15:31 +00:00
# include <Interpreters/PredicateExpressionsOptimizer.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ExternalDictionaries.h>
2018-02-28 04:30:27 +00:00
# include <Interpreters/convertFieldToType.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/Set.h>
# include <Interpreters/Join.h>
2018-05-15 10:44:54 +00:00
# include <Interpreters/ProjectionManipulation.h>
# include <Interpreters/evaluateConstantExpression.h>
2013-05-24 10:49:19 +00:00
2017-04-01 09:19:00 +00:00
# include <AggregateFunctions/AggregateFunctionFactory.h>
2017-07-10 23:30:17 +00:00
# include <AggregateFunctions/parseAggregateFunctionParameters.h>
2015-04-16 06:12:35 +00:00
2017-04-01 09:19:00 +00:00
# include <Storages/StorageDistributed.h>
# include <Storages/StorageMemory.h>
# include <Storages/StorageSet.h>
# include <Storages/StorageJoin.h>
2014-03-14 14:52:48 +00:00
2017-04-01 09:19:00 +00:00
# include <DataStreams/LazyBlockInputStream.h>
# include <DataStreams/copyData.h>
2014-03-14 14:52:48 +00:00
2017-04-01 09:19:00 +00:00
# include <Dictionaries/IDictionary.h>
2015-05-13 16:11:07 +00:00
2017-04-01 09:19:00 +00:00
# include <Common/typeid_cast.h>
2018-01-15 19:07:47 +00:00
# include <Common/StringUtils/StringUtils.h>
2014-06-26 00:58:14 +00:00
2017-04-01 09:19:00 +00:00
# include <Parsers/formatAST.h>
2013-05-24 10:49:19 +00:00
2017-04-01 09:19:00 +00:00
# include <Functions/FunctionFactory.h>
# include <Functions/IFunction.h>
2014-12-17 15:26:24 +00:00
2017-06-06 17:18:32 +00:00
# include <ext/range.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeFactory.h>
2018-02-06 19:34:53 +00:00
# include <DataTypes/DataTypeFunction.h>
# include <Functions/FunctionsMiscellaneous.h>
2018-05-14 15:05:45 +00:00
# include <DataTypes/DataTypeTuple.h>
2018-07-23 15:54:10 +00:00
# include <Parsers/queryToString.h>
2018-07-26 12:58:23 +00:00
# include <Parsers/ExpressionListParsers.h>
# include <Parsers/parseQuery.h>
2018-07-27 16:21:43 +00:00
# include <Parsers/queryToString.h>
2018-08-22 06:42:37 +00:00
# include <Interpreters/evaluateQualified.h>
2014-12-17 11:53:17 +00:00
2013-05-24 10:49:19 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2018-04-05 20:52:01 +00:00
extern const int BAD_ARGUMENTS ;
2017-04-01 07:20:54 +00:00
extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS ;
extern const int UNKNOWN_IDENTIFIER ;
extern const int CYCLIC_ALIASES ;
extern const int INCORRECT_RESULT_OF_SCALAR_SUBQUERY ;
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_ROWS ;
2017-04-01 07:20:54 +00:00
extern const int NOT_FOUND_COLUMN_IN_BLOCK ;
extern const int INCORRECT_ELEMENT_OF_SET ;
extern const int ALIAS_REQUIRED ;
extern const int EMPTY_NESTED_TABLE ;
extern const int NOT_AN_AGGREGATE ;
extern const int UNEXPECTED_EXPRESSION ;
extern const int DUPLICATE_COLUMN ;
extern const int FUNCTION_CANNOT_HAVE_PARAMETERS ;
extern const int ILLEGAL_AGGREGATION ;
extern const int SUPPORT_IS_DISABLED ;
extern const int TOO_DEEP_AST ;
2018-03-02 06:33:17 +00:00
extern const int TOO_BIG_AST ;
2017-12-20 07:36:30 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
2018-04-24 07:16:39 +00:00
extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND ;
2018-06-05 19:46:49 +00:00
extern const int TYPE_MISMATCH ;
2018-07-23 15:54:10 +00:00
extern const int INVALID_JOIN_ON_EXPRESSION ;
2018-08-29 15:15:42 +00:00
extern const int EXPECTED_ALL_OR_ANY ;
2016-01-11 21:46:36 +00:00
}
2013-05-24 10:49:19 +00:00
2014-07-31 08:14:19 +00:00
/** Calls to these functions in the GROUP BY statement would be
* replaced by their immediate argument .
2014-07-31 07:24:07 +00:00
*/
2014-08-19 23:12:31 +00:00
const std : : unordered_set < String > injective_function_names
{
2017-04-01 07:20:54 +00:00
" negate " ,
" bitNot " ,
" reverse " ,
" reverseUTF8 " ,
" toString " ,
" toFixedString " ,
" IPv4NumToString " ,
" IPv4StringToNum " ,
" hex " ,
" unhex " ,
" bitmaskToList " ,
" bitmaskToArray " ,
" tuple " ,
" regionToName " ,
" concatAssumeInjective " ,
2014-07-31 07:24:07 +00:00
} ;
2015-05-13 16:11:07 +00:00
const std : : unordered_set < String > possibly_injective_function_names
{
2017-04-01 07:20:54 +00:00
" dictGetString " ,
" dictGetUInt8 " ,
" dictGetUInt16 " ,
" dictGetUInt32 " ,
" dictGetUInt64 " ,
" dictGetInt8 " ,
" dictGetInt16 " ,
" dictGetInt32 " ,
" dictGetInt64 " ,
" dictGetFloat32 " ,
" dictGetFloat64 " ,
" dictGetDate " ,
" dictGetDateTime "
2015-05-13 16:11:07 +00:00
} ;
2015-09-18 13:36:10 +00:00
namespace
{
bool functionIsInOperator ( const String & name )
2015-07-26 07:08:46 +00:00
{
2017-04-01 07:20:54 +00:00
return name = = " in " | | name = = " notIn " ;
2015-07-26 07:08:46 +00:00
}
2015-09-18 13:36:10 +00:00
bool functionIsInOrGlobalInOperator ( const String & name )
2015-07-26 07:08:46 +00:00
{
2017-04-01 07:20:54 +00:00
return name = = " in " | | name = = " notIn " | | name = = " globalIn " | | name = = " globalNotIn " ;
2015-07-26 07:08:46 +00:00
}
2017-12-25 21:57:29 +00:00
void removeDuplicateColumns ( NamesAndTypesList & columns )
2016-08-20 17:10:00 +00:00
{
2017-04-01 07:20:54 +00:00
std : : set < String > names ;
2017-12-25 21:57:19 +00:00
for ( auto it = columns . begin ( ) ; it ! = columns . end ( ) ; )
{
if ( names . emplace ( it - > name ) . second )
+ + it ;
else
columns . erase ( it + + ) ;
}
2016-08-20 17:10:00 +00:00
}
2015-09-18 13:36:10 +00:00
}
2015-07-26 07:08:46 +00:00
2017-01-14 09:00:19 +00:00
ExpressionAnalyzer : : ExpressionAnalyzer (
2018-08-27 17:58:43 +00:00
const ASTPtr & query_ ,
2017-04-01 07:20:54 +00:00
const Context & context_ ,
2017-09-08 03:47:27 +00:00
const StoragePtr & storage_ ,
2018-02-26 09:05:06 +00:00
const NamesAndTypesList & source_columns_ ,
const Names & required_result_columns_ ,
2017-04-01 07:20:54 +00:00
size_t subquery_depth_ ,
2018-02-08 17:10:35 +00:00
bool do_global_ ,
const SubqueriesForSets & subqueries_for_set_ )
2018-08-27 17:58:43 +00:00
: query ( query_ ) , context ( context_ ) , settings ( context . getSettings ( ) ) ,
2018-02-26 09:05:06 +00:00
subquery_depth ( subquery_depth_ ) ,
2018-08-05 03:28:31 +00:00
source_columns ( source_columns_ ) , required_result_columns ( required_result_columns_ ) ,
2018-02-27 21:03:28 +00:00
storage ( storage_ ) ,
2018-02-08 17:10:35 +00:00
do_global ( do_global_ ) , subqueries_for_sets ( subqueries_for_set_ )
2017-04-01 07:20:54 +00:00
{
2018-08-27 17:58:43 +00:00
select_query = typeid_cast < ASTSelectQuery * > ( query . get ( ) ) ;
2017-01-14 09:00:19 +00:00
2018-02-27 21:03:28 +00:00
if ( ! storage & & select_query )
{
auto select_database = select_query - > database ( ) ;
auto select_table = select_query - > table ( ) ;
2017-01-14 09:00:19 +00:00
2018-02-27 21:03:28 +00:00
if ( select_table
& & ! typeid_cast < const ASTSelectWithUnionQuery * > ( select_table . get ( ) )
& & ! typeid_cast < const ASTFunction * > ( select_table . get ( ) ) )
{
String database = select_database
? typeid_cast < const ASTIdentifier & > ( * select_database ) . name
: " " ;
const String & table = typeid_cast < const ASTIdentifier & > ( * select_table ) . name ;
storage = context . tryGetTable ( database , table ) ;
}
}
2018-02-27 19:00:55 +00:00
2018-02-28 01:29:55 +00:00
if ( storage & & source_columns . empty ( ) )
2018-03-13 15:00:28 +00:00
source_columns = storage - > getColumns ( ) . getAllPhysical ( ) ;
2018-02-28 01:29:55 +00:00
else
removeDuplicateColumns ( source_columns ) ;
2018-02-27 19:00:55 +00:00
addAliasColumns ( ) ;
2013-10-17 13:32:32 +00:00
2017-06-19 02:14:18 +00:00
translateQualifiedNames ( ) ;
2017-04-02 17:37:49 +00:00
/// Depending on the user's profile, check for the execution rights
/// distributed subqueries inside the IN or JOIN sections and process these subqueries.
2017-04-01 07:20:54 +00:00
InJoinSubqueriesPreprocessor ( context ) . process ( select_query ) ;
2015-09-18 13:36:10 +00:00
2017-04-02 17:37:49 +00:00
/// Optimizes logical expressions.
2017-04-01 07:20:54 +00:00
LogicalExpressionsOptimizer ( select_query , settings ) . perform ( ) ;
2015-02-16 17:25:44 +00:00
2017-04-02 17:37:49 +00:00
/// Creates a dictionary `aliases`: alias -> ASTPtr
2018-08-27 17:58:43 +00:00
addASTAliases ( query ) ;
2014-07-04 19:13:38 +00:00
2017-04-01 07:20:54 +00:00
/// Common subexpression elimination. Rewrite rules.
normalizeTree ( ) ;
2013-10-17 13:32:32 +00:00
2018-07-20 15:50:39 +00:00
/// Remove unneeded columns according to 'required_result_columns'.
2018-03-01 01:25:06 +00:00
/// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside.
/// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost)
/// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations.
removeUnneededColumnsFromSelectClause ( ) ;
2017-04-02 17:37:49 +00:00
/// Executing scalar subqueries - replacing them with constant values.
2017-04-01 07:20:54 +00:00
executeScalarSubqueries ( ) ;
2015-07-25 11:47:50 +00:00
2017-07-16 22:55:40 +00:00
/// Optimize if with constant condition after constants was substituted instead of sclalar subqueries.
2017-04-01 07:20:54 +00:00
optimizeIfWithConstantCondition ( ) ;
2016-11-24 14:18:44 +00:00
2017-04-01 07:20:54 +00:00
/// GROUP BY injective function elimination.
optimizeGroupBy ( ) ;
2014-07-31 07:24:07 +00:00
2017-04-02 17:37:49 +00:00
/// Remove duplicate items from ORDER BY.
2017-04-01 07:20:54 +00:00
optimizeOrderBy ( ) ;
2015-03-12 03:31:15 +00:00
2017-04-01 07:20:54 +00:00
// Remove duplicated elements from LIMIT BY clause.
optimizeLimitBy ( ) ;
2016-12-28 15:12:54 +00:00
2018-08-31 10:00:46 +00:00
/// Remove duplicated columns from USING(...).
optimizeUsing ( ) ;
2017-04-01 07:20:54 +00:00
/// array_join_alias_to_name, array_join_result_to_source.
getArrayJoinedColumns ( ) ;
2013-10-17 13:32:32 +00:00
2018-08-20 15:49:39 +00:00
/// Push the predicate expression down to the subqueries.
2018-08-22 06:42:37 +00:00
rewrite_subqueries = PredicateExpressionsOptimizer ( select_query , settings , context ) . optimize ( ) ;
2018-03-04 16:15:31 +00:00
2018-02-26 21:00:42 +00:00
/// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`.
2017-04-01 07:20:54 +00:00
collectUsedColumns ( ) ;
2014-07-04 19:13:38 +00:00
2017-04-02 17:37:49 +00:00
/// external_tables, subqueries_for_sets for global subqueries.
/// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers.
2017-04-01 07:20:54 +00:00
initGlobalSubqueriesAndExternalTables ( ) ;
2015-10-15 11:07:39 +00:00
2017-04-01 07:20:54 +00:00
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
2017-04-02 17:37:49 +00:00
/// This analysis should be performed after processing global subqueries, because otherwise,
/// if the aggregate function contains a global subquery, then `analyzeAggregation` method will save
/// in `aggregate_descriptions` the information about the parameters of this aggregate function, among which
/// global subquery. Then, when you call `initGlobalSubqueriesAndExternalTables` method, this
/// the global subquery will be replaced with a temporary table, resulting in aggregate_descriptions
/// will contain out-of-date information, which will lead to an error when the query is executed.
2017-04-01 07:20:54 +00:00
analyzeAggregation ( ) ;
2014-07-04 19:13:38 +00:00
}
2018-07-25 16:11:20 +00:00
void ExpressionAnalyzer : : translateQualifiedNames ( )
{
if ( ! select_query | | ! select_query - > tables | | select_query - > tables - > children . empty ( ) )
return ;
auto & element = static_cast < ASTTablesInSelectQueryElement & > ( * select_query - > tables - > children [ 0 ] ) ;
if ( ! element . table_expression ) /// This is ARRAY JOIN without a table at the left side.
return ;
2018-07-26 15:11:53 +00:00
auto & table_expression = static_cast < ASTTableExpression & > ( * element . table_expression ) ;
auto * join = select_query - > join ( ) ;
std : : vector < DatabaseAndTableWithAlias > tables = { getTableNameWithAliasFromTableExpression ( table_expression , context ) } ;
2018-07-25 16:11:20 +00:00
2018-07-26 15:11:53 +00:00
if ( join )
{
const auto & join_table_expression = static_cast < const ASTTableExpression & > ( * join - > table_expression ) ;
tables . emplace_back ( getTableNameWithAliasFromTableExpression ( join_table_expression , context ) ) ;
}
2018-07-25 16:11:20 +00:00
2018-08-27 17:58:43 +00:00
translateQualifiedNamesImpl ( query , tables ) ;
2017-06-19 02:14:18 +00:00
}
2018-07-26 15:11:53 +00:00
void ExpressionAnalyzer : : translateQualifiedNamesImpl ( ASTPtr & ast , const std : : vector < DatabaseAndTableWithAlias > & tables )
2018-07-25 16:11:20 +00:00
{
if ( auto * identifier = typeid_cast < ASTIdentifier * > ( ast . get ( ) ) )
{
if ( identifier - > kind = = ASTIdentifier : : Column )
{
2018-07-26 15:11:53 +00:00
/// Select first table name with max number of qualifiers which can be stripped.
size_t max_num_qualifiers_to_strip = 0 ;
size_t best_table_pos = 0 ;
for ( size_t table_pos = 0 ; table_pos < tables . size ( ) ; + + table_pos )
{
const auto & table = tables [ table_pos ] ;
auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName ( * identifier , table ) ;
if ( num_qualifiers_to_strip > max_num_qualifiers_to_strip )
2017-06-19 02:14:18 +00:00
{
2018-07-26 15:11:53 +00:00
max_num_qualifiers_to_strip = num_qualifiers_to_strip ;
best_table_pos = table_pos ;
2017-06-19 02:14:18 +00:00
}
}
2018-07-26 15:11:53 +00:00
stripIdentifier ( ast , max_num_qualifiers_to_strip ) ;
2018-07-25 16:11:20 +00:00
2018-07-26 15:11:53 +00:00
/// In case if column from the joined table are in source columns, change it's name to qualified.
if ( best_table_pos & & source_columns . contains ( ast - > getColumnName ( ) ) )
tables [ best_table_pos ] . makeQualifiedName ( ast ) ;
2017-06-19 02:14:18 +00:00
}
}
2017-06-19 03:18:30 +00:00
else if ( typeid_cast < ASTQualifiedAsterisk * > ( ast . get ( ) ) )
2017-06-19 03:05:05 +00:00
{
if ( ast - > children . size ( ) ! = 1 )
throw Exception ( " Logical error: qualified asterisk must have exactly one child " , ErrorCodes : : LOGICAL_ERROR ) ;
ASTIdentifier * ident = typeid_cast < ASTIdentifier * > ( ast - > children [ 0 ] . get ( ) ) ;
if ( ! ident )
throw Exception ( " Logical error: qualified asterisk must have identifier as its child " , ErrorCodes : : LOGICAL_ERROR ) ;
size_t num_components = ident - > children . size ( ) ;
if ( num_components > 2 )
throw Exception ( " Qualified asterisk cannot have more than two qualifiers " , ErrorCodes : : UNKNOWN_ELEMENT_IN_AST ) ;
2018-07-26 15:11:53 +00:00
for ( const auto & table_names : tables )
2017-06-19 03:05:05 +00:00
{
2018-07-26 15:11:53 +00:00
/// database.table.*, table.* or alias.*
if ( ( num_components = = 2
& & ! table_names . database . empty ( )
& & static_cast < const ASTIdentifier & > ( * ident - > children [ 0 ] ) . name = = table_names . database
& & static_cast < const ASTIdentifier & > ( * ident - > children [ 1 ] ) . name = = table_names . table )
| | ( num_components = = 0
& & ( ( ! table_names . table . empty ( ) & & ident - > name = = table_names . table )
| | ( ! table_names . alias . empty ( ) & & ident - > name = = table_names . alias ) ) ) )
{
/// Replace to plain asterisk.
ast = std : : make_shared < ASTAsterisk > ( ) ;
}
2017-06-19 03:05:05 +00:00
}
}
2018-07-26 15:11:53 +00:00
else if ( auto * join = typeid_cast < ASTTableJoin * > ( ast . get ( ) ) )
{
2018-07-31 03:04:35 +00:00
/// Don't translate on_expression here in order to resolve equation parts later.
2018-07-26 15:11:53 +00:00
if ( join - > using_expression_list )
translateQualifiedNamesImpl ( join - > using_expression_list , tables ) ;
}
2017-06-19 02:14:18 +00:00
else
{
2018-08-29 22:09:04 +00:00
/// If the WHERE clause or HAVING consists of a single quailified column, the reference must be translated not only in children, but also in where_expression and having_expression.
if ( ASTSelectQuery * select = typeid_cast < ASTSelectQuery * > ( ast . get ( ) ) )
{
if ( select - > prewhere_expression )
translateQualifiedNamesImpl ( select - > prewhere_expression , tables ) ;
if ( select - > where_expression )
translateQualifiedNamesImpl ( select - > where_expression , tables ) ;
if ( select - > having_expression )
translateQualifiedNamesImpl ( select - > having_expression , tables ) ;
2018-08-30 21:13:34 +00:00
}
2017-06-19 02:14:18 +00:00
for ( auto & child : ast - > children )
{
2018-02-26 09:05:06 +00:00
/// Do not go to FROM, JOIN, subqueries.
2017-06-19 02:14:18 +00:00
if ( ! typeid_cast < const ASTTableExpression * > ( child . get ( ) )
2018-02-26 09:05:06 +00:00
& & ! typeid_cast < const ASTSelectWithUnionQuery * > ( child . get ( ) ) )
2017-06-19 02:14:18 +00:00
{
2018-07-26 15:11:53 +00:00
translateQualifiedNamesImpl ( child , tables ) ;
2017-06-19 02:14:18 +00:00
}
}
}
}
2016-11-24 14:18:44 +00:00
void ExpressionAnalyzer : : optimizeIfWithConstantCondition ( )
{
2018-08-27 17:58:43 +00:00
optimizeIfWithConstantConditionImpl ( query ) ;
2016-11-24 14:18:44 +00:00
}
bool ExpressionAnalyzer : : tryExtractConstValueFromCondition ( const ASTPtr & condition , bool & value ) const
{
2017-04-01 07:20:54 +00:00
/// numeric constant in condition
if ( const ASTLiteral * literal = typeid_cast < ASTLiteral * > ( condition . get ( ) ) )
{
if ( literal - > value . getType ( ) = = Field : : Types : : Int64 | |
literal - > value . getType ( ) = = Field : : Types : : UInt64 )
{
value = literal - > value . get < Int64 > ( ) ;
return true ;
}
}
/// cast of numeric constant in condition to UInt8
if ( const ASTFunction * function = typeid_cast < ASTFunction * > ( condition . get ( ) ) )
{
if ( function - > name = = " CAST " )
{
if ( ASTExpressionList * expr_list = typeid_cast < ASTExpressionList * > ( function - > arguments . get ( ) ) )
{
const ASTPtr & type_ast = expr_list - > children . at ( 1 ) ;
if ( const ASTLiteral * type_literal = typeid_cast < ASTLiteral * > ( type_ast . get ( ) ) )
{
if ( type_literal - > value . getType ( ) = = Field : : Types : : String & &
type_literal - > value . get < std : : string > ( ) = = " UInt8 " )
return tryExtractConstValueFromCondition ( expr_list - > children . at ( 0 ) , value ) ;
}
}
}
}
return false ;
2016-11-24 14:18:44 +00:00
}
2018-08-27 17:58:43 +00:00
void ExpressionAnalyzer : : optimizeIfWithConstantConditionImpl ( ASTPtr & current_ast )
2016-11-24 14:18:44 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! current_ast )
return ;
for ( ASTPtr & child : current_ast - > children )
{
ASTFunction * function_node = typeid_cast < ASTFunction * > ( child . get ( ) ) ;
if ( ! function_node | | function_node - > name ! = " if " )
{
2018-08-27 17:58:43 +00:00
optimizeIfWithConstantConditionImpl ( child ) ;
2017-04-01 07:20:54 +00:00
continue ;
}
2018-08-27 17:58:43 +00:00
optimizeIfWithConstantConditionImpl ( function_node - > arguments ) ;
2017-04-01 07:20:54 +00:00
ASTExpressionList * args = typeid_cast < ASTExpressionList * > ( function_node - > arguments . get ( ) ) ;
2018-08-24 09:57:42 +00:00
if ( args - > children . size ( ) ! = 3 )
throw Exception ( " Wrong number of arguments for function 'if' ( " + toString ( args - > children . size ( ) ) + " instead of 3) " ,
ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2017-04-01 07:20:54 +00:00
2018-08-24 09:57:42 +00:00
ASTPtr condition_expr = args - > children [ 0 ] ;
ASTPtr then_expr = args - > children [ 1 ] ;
ASTPtr else_expr = args - > children [ 2 ] ;
2017-04-01 07:20:54 +00:00
bool condition ;
if ( tryExtractConstValueFromCondition ( condition_expr , condition ) )
{
ASTPtr replace_ast = condition ? then_expr : else_expr ;
ASTPtr child_copy = child ;
String replace_alias = replace_ast - > tryGetAlias ( ) ;
String if_alias = child - > tryGetAlias ( ) ;
if ( replace_alias . empty ( ) )
{
replace_ast - > setAlias ( if_alias ) ;
child = replace_ast ;
}
else
{
/// Only copy of one node is required here.
/// But IAST has only method for deep copy of subtree.
/// This can be a reason of performance degradation in case of deep queries.
ASTPtr replace_ast_deep_copy = replace_ast - > clone ( ) ;
replace_ast_deep_copy - > setAlias ( if_alias ) ;
child = replace_ast_deep_copy ;
}
if ( ! if_alias . empty ( ) )
{
auto alias_it = aliases . find ( if_alias ) ;
if ( alias_it ! = aliases . end ( ) & & alias_it - > second . get ( ) = = child_copy . get ( ) )
alias_it - > second = child ;
}
}
}
2016-11-24 14:18:44 +00:00
}
2014-07-04 19:13:38 +00:00
void ExpressionAnalyzer : : analyzeAggregation ( )
{
2017-04-02 17:37:49 +00:00
/** Find aggregation keys (aggregation_keys), information about aggregate functions (aggregate_descriptions),
* as well as a set of columns obtained after the aggregation , if any ,
* or after all the actions that are usually performed before aggregation ( aggregated_columns ) .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* Everything below ( compiling temporary ExpressionActions ) - only for the purpose of query analysis ( type output ) .
2017-04-01 07:20:54 +00:00
*/
if ( select_query & & ( select_query - > group_expression_list | | select_query - > having_expression ) )
has_aggregation = true ;
2018-02-26 09:05:06 +00:00
ExpressionActionsPtr temp_actions = std : : make_shared < ExpressionActions > ( source_columns , settings ) ;
2017-04-01 07:20:54 +00:00
if ( select_query & & select_query - > array_join_expression_list ( ) )
{
getRootActions ( select_query - > array_join_expression_list ( ) , true , false , temp_actions ) ;
addMultipleArrayJoinAction ( temp_actions ) ;
2018-04-04 18:56:30 +00:00
array_join_columns = temp_actions - > getSampleBlock ( ) . getNamesAndTypesList ( ) ;
2017-04-01 07:20:54 +00:00
}
if ( select_query )
{
const ASTTablesInSelectQueryElement * join = select_query - > join ( ) ;
if ( join )
{
2018-07-30 13:57:50 +00:00
const auto table_join = static_cast < const ASTTableJoin & > ( * join - > table_join ) ;
if ( table_join . using_expression_list )
getRootActions ( table_join . using_expression_list , true , false , temp_actions ) ;
if ( table_join . on_expression )
for ( const auto & key_ast : analyzed_join . key_asts_left )
getRootActions ( key_ast , true , false , temp_actions ) ;
2017-04-01 07:20:54 +00:00
addJoinAction ( temp_actions , true ) ;
}
}
2018-08-27 17:58:43 +00:00
getAggregates ( query , temp_actions ) ;
2017-04-01 07:20:54 +00:00
if ( has_aggregation )
{
assertSelect ( ) ;
/// Find out aggregation keys.
if ( select_query - > group_expression_list )
{
NameSet unique_keys ;
ASTs & group_asts = select_query - > group_expression_list - > children ;
for ( ssize_t i = 0 ; i < ssize_t ( group_asts . size ( ) ) ; + + i )
{
ssize_t size = group_asts . size ( ) ;
getRootActions ( group_asts [ i ] , true , false , temp_actions ) ;
const auto & column_name = group_asts [ i ] - > getColumnName ( ) ;
const auto & block = temp_actions - > getSampleBlock ( ) ;
if ( ! block . has ( column_name ) )
throw Exception ( " Unknown identifier (in GROUP BY) : " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER) ;
const auto & col = block . getByName ( column_name ) ;
/// Constant expressions have non-null column pointer at this stage.
2018-07-27 18:14:58 +00:00
if ( col . column & & col . column - > isColumnConst ( ) )
2017-04-01 07:20:54 +00:00
{
/// But don't remove last key column if no aggregate functions, otherwise aggregation will not work.
if ( ! aggregate_descriptions . empty ( ) | | size > 1 )
{
if ( i + 1 < static_cast < ssize_t > ( size ) )
group_asts [ i ] = std : : move ( group_asts . back ( ) ) ;
group_asts . pop_back ( ) ;
- - i ;
continue ;
}
}
2017-12-25 21:57:29 +00:00
NameAndTypePair key { column_name , col . type } ;
2017-04-01 07:20:54 +00:00
/// Aggregation keys are uniqued.
if ( ! unique_keys . count ( key . name ) )
{
unique_keys . insert ( key . name ) ;
2018-04-04 18:56:30 +00:00
aggregation_keys . push_back ( key ) ;
2017-04-01 07:20:54 +00:00
/// Key is no longer needed, therefore we can save a little by moving it.
2018-04-04 18:56:30 +00:00
aggregated_columns . push_back ( std : : move ( key ) ) ;
2017-04-01 07:20:54 +00:00
}
}
if ( group_asts . empty ( ) )
{
select_query - > group_expression_list = nullptr ;
2018-04-04 18:56:30 +00:00
has_aggregation = select_query - > having_expression | | aggregate_descriptions . size ( ) ;
2017-04-01 07:20:54 +00:00
}
}
for ( size_t i = 0 ; i < aggregate_descriptions . size ( ) ; + + i )
{
AggregateDescription & desc = aggregate_descriptions [ i ] ;
2018-04-04 18:56:30 +00:00
aggregated_columns . emplace_back ( desc . column_name , desc . function - > getReturnType ( ) ) ;
2017-04-01 07:20:54 +00:00
}
}
2018-04-04 18:56:30 +00:00
else
{
aggregated_columns = temp_actions - > getSampleBlock ( ) . getNamesAndTypesList ( ) ;
}
2013-05-24 10:49:19 +00:00
}
2014-07-04 20:30:06 +00:00
void ExpressionAnalyzer : : initGlobalSubqueriesAndExternalTables ( )
{
2017-04-02 17:37:49 +00:00
/// Adds existing external tables (not subqueries) to the external_tables dictionary.
2018-08-27 17:58:43 +00:00
findExternalTables ( query ) ;
2015-11-04 22:02:52 +00:00
2017-04-02 17:37:49 +00:00
/// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr.
2018-08-27 17:58:43 +00:00
initGlobalSubqueries ( query ) ;
2014-07-04 20:30:06 +00:00
}
void ExpressionAnalyzer : : initGlobalSubqueries ( ASTPtr & ast )
{
2017-04-02 17:37:49 +00:00
/// Recursive calls. We do not go into subqueries.
2017-04-01 07:20:54 +00:00
for ( auto & child : ast - > children )
if ( ! typeid_cast < ASTSelectQuery * > ( child . get ( ) ) )
initGlobalSubqueries ( child ) ;
2017-04-02 17:37:49 +00:00
/// Bottom-up actions.
2017-04-01 07:20:54 +00:00
2018-08-27 17:58:43 +00:00
if ( ASTFunction * func = typeid_cast < ASTFunction * > ( ast . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// For GLOBAL IN.
2018-08-27 17:58:43 +00:00
if ( do_global & & ( func - > name = = " globalIn " | | func - > name = = " globalNotIn " ) )
addExternalStorage ( func - > arguments - > children . at ( 1 ) ) ;
2017-04-01 07:20:54 +00:00
}
2018-08-27 17:58:43 +00:00
else if ( ASTTablesInSelectQueryElement * table_elem = typeid_cast < ASTTablesInSelectQueryElement * > ( ast . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// For GLOBAL JOIN.
2018-08-27 17:58:43 +00:00
if ( do_global & & table_elem - > table_join
& & static_cast < const ASTTableJoin & > ( * table_elem - > table_join ) . locality = = ASTTableJoin : : Locality : : Global )
addExternalStorage ( table_elem - > table_expression ) ;
2017-04-01 07:20:54 +00:00
}
2014-07-04 20:30:06 +00:00
}
void ExpressionAnalyzer : : findExternalTables ( ASTPtr & ast )
{
2017-04-02 17:37:49 +00:00
/// Traverse from the bottom. Intentionally go into subqueries.
2017-04-01 07:20:54 +00:00
for ( auto & child : ast - > children )
findExternalTables ( child ) ;
2014-07-04 20:30:06 +00:00
2017-04-02 17:37:49 +00:00
/// If table type identifier
2017-04-01 07:20:54 +00:00
StoragePtr external_storage ;
2014-07-04 20:30:06 +00:00
2017-04-01 07:20:54 +00:00
if ( ASTIdentifier * node = typeid_cast < ASTIdentifier * > ( ast . get ( ) ) )
if ( node - > kind = = ASTIdentifier : : Table )
if ( ( external_storage = context . tryGetExternalTable ( node - > name ) ) )
external_tables [ node - > name ] = external_storage ;
2014-07-04 20:30:06 +00:00
}
2018-02-26 09:05:06 +00:00
static std : : shared_ptr < InterpreterSelectWithUnionQuery > interpretSubquery (
2018-08-14 09:44:56 +00:00
const ASTPtr & table_expression , const Context & context , size_t subquery_depth , const Names & required_source_columns )
2017-07-15 03:48:36 +00:00
{
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
2018-08-14 09:44:56 +00:00
const ASTSubquery * subquery = typeid_cast < const ASTSubquery * > ( table_expression . get ( ) ) ;
const ASTFunction * function = typeid_cast < const ASTFunction * > ( table_expression . get ( ) ) ;
const ASTIdentifier * table = typeid_cast < const ASTIdentifier * > ( table_expression . get ( ) ) ;
2017-07-15 03:48:36 +00:00
2018-08-14 09:44:56 +00:00
if ( ! subquery & & ! table & & ! function )
throw Exception ( " Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery. " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-07-15 03:48:36 +00:00
/** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result.
* Because the result of this query is not the result of the entire query .
* Constraints work instead
* max_rows_in_set , max_bytes_in_set , set_overflow_mode ,
* max_rows_in_join , max_bytes_in_join , join_overflow_mode ,
* which are checked separately ( in the Set , Join objects ) .
*/
Context subquery_context = context ;
Settings subquery_settings = context . getSettings ( ) ;
2018-03-11 00:15:26 +00:00
subquery_settings . max_result_rows = 0 ;
subquery_settings . max_result_bytes = 0 ;
2017-07-15 03:48:36 +00:00
/// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query).
subquery_settings . extremes = 0 ;
subquery_context . setSettings ( subquery_settings ) ;
ASTPtr query ;
2018-08-14 09:44:56 +00:00
if ( table | | function )
2017-07-15 03:48:36 +00:00
{
/// create ASTSelectQuery for "SELECT * FROM table" as if written by hand
2018-02-26 09:05:06 +00:00
const auto select_with_union_query = std : : make_shared < ASTSelectWithUnionQuery > ( ) ;
query = select_with_union_query ;
select_with_union_query - > list_of_selects = std : : make_shared < ASTExpressionList > ( ) ;
2017-07-15 03:48:36 +00:00
const auto select_query = std : : make_shared < ASTSelectQuery > ( ) ;
2018-02-26 09:05:06 +00:00
select_with_union_query - > list_of_selects - > children . push_back ( select_query ) ;
2017-07-15 03:48:36 +00:00
const auto select_expression_list = std : : make_shared < ASTExpressionList > ( ) ;
select_query - > select_expression_list = select_expression_list ;
select_query - > children . emplace_back ( select_query - > select_expression_list ) ;
2018-08-14 09:44:56 +00:00
NamesAndTypesList columns ;
2017-07-15 03:48:36 +00:00
/// get columns list for target table
2018-08-14 09:44:56 +00:00
if ( function )
{
2018-08-24 07:40:09 +00:00
auto query_context = const_cast < Context * > ( & context . getQueryContext ( ) ) ;
const auto & storage = query_context - > executeTableFunction ( table_expression ) ;
2018-08-14 09:44:56 +00:00
columns = storage - > getColumns ( ) . ordinary ;
select_query - > addTableFunction ( * const_cast < ASTPtr * > ( & table_expression ) ) ;
}
else
{
auto database_table = getDatabaseAndTableNameFromIdentifier ( * table ) ;
const auto & storage = context . getTable ( database_table . first , database_table . second ) ;
columns = storage - > getColumns ( ) . ordinary ;
select_query - > replaceDatabaseAndTable ( database_table . first , database_table . second ) ;
}
2017-07-15 03:48:36 +00:00
2018-08-14 09:44:56 +00:00
select_expression_list - > children . reserve ( columns . size ( ) ) ;
2017-07-15 03:48:36 +00:00
/// manually substitute column names in place of asterisk
for ( const auto & column : columns )
2018-02-26 03:40:20 +00:00
select_expression_list - > children . emplace_back ( std : : make_shared < ASTIdentifier > ( column . name ) ) ;
2017-07-15 03:48:36 +00:00
}
else
{
query = subquery - > children . at ( 0 ) ;
/** Columns with the same name can be specified in a subquery. For example, SELECT x, x FROM t
* This is bad , because the result of such a query can not be saved to the table , because the table can not have the same name columns .
* Saving to the table is required for GLOBAL subqueries .
*
* To avoid this situation , we will rename the same columns .
*/
std : : set < std : : string > all_column_names ;
std : : set < std : : string > assigned_column_names ;
2018-02-26 10:00:56 +00:00
if ( ASTSelectWithUnionQuery * select_with_union = typeid_cast < ASTSelectWithUnionQuery * > ( query . get ( ) ) )
2017-07-15 03:48:36 +00:00
{
2018-02-26 10:00:56 +00:00
if ( ASTSelectQuery * select = typeid_cast < ASTSelectQuery * > ( select_with_union - > list_of_selects - > children . at ( 0 ) . get ( ) ) )
2017-07-15 03:48:36 +00:00
{
2018-02-26 10:00:56 +00:00
for ( auto & expr : select - > select_expression_list - > children )
all_column_names . insert ( expr - > getAliasOrColumnName ( ) ) ;
2017-07-15 03:48:36 +00:00
2018-02-26 10:00:56 +00:00
for ( auto & expr : select - > select_expression_list - > children )
2017-07-15 03:48:36 +00:00
{
2018-02-26 10:00:56 +00:00
auto name = expr - > getAliasOrColumnName ( ) ;
if ( ! assigned_column_names . insert ( name ) . second )
{
size_t i = 1 ;
while ( all_column_names . end ( ) ! = all_column_names . find ( name + " _ " + toString ( i ) ) )
+ + i ;
2017-07-15 03:48:36 +00:00
2018-02-26 10:00:56 +00:00
name = name + " _ " + toString ( i ) ;
expr = expr - > clone ( ) ; /// Cancels fuse of the same expressions in the tree.
expr - > setAlias ( name ) ;
2017-07-15 03:48:36 +00:00
2018-02-26 10:00:56 +00:00
all_column_names . insert ( name ) ;
assigned_column_names . insert ( name ) ;
}
2017-07-15 03:48:36 +00:00
}
}
}
}
2018-02-26 09:05:06 +00:00
return std : : make_shared < InterpreterSelectWithUnionQuery > (
query , subquery_context , required_source_columns , QueryProcessingStage : : Complete , subquery_depth + 1 ) ;
2017-07-15 03:48:36 +00:00
}
2015-11-04 22:02:52 +00:00
2016-07-26 19:43:58 +00:00
void ExpressionAnalyzer : : addExternalStorage ( ASTPtr & subquery_or_table_name_or_table_expression )
2015-11-04 22:02:52 +00:00
{
2017-04-02 17:37:49 +00:00
/// With nondistributed queries, creating temporary tables does not make sense.
2017-04-01 07:20:54 +00:00
if ( ! ( storage & & storage - > isRemote ( ) ) )
return ;
ASTPtr subquery ;
ASTPtr table_name ;
ASTPtr subquery_or_table_name ;
if ( typeid_cast < const ASTIdentifier * > ( subquery_or_table_name_or_table_expression . get ( ) ) )
{
table_name = subquery_or_table_name_or_table_expression ;
subquery_or_table_name = table_name ;
}
else if ( auto ast_table_expr = typeid_cast < const ASTTableExpression * > ( subquery_or_table_name_or_table_expression . get ( ) ) )
{
if ( ast_table_expr - > database_and_table_name )
{
table_name = ast_table_expr - > database_and_table_name ;
subquery_or_table_name = table_name ;
}
else if ( ast_table_expr - > subquery )
{
subquery = ast_table_expr - > subquery ;
subquery_or_table_name = subquery ;
}
}
else if ( typeid_cast < const ASTSubquery * > ( subquery_or_table_name_or_table_expression . get ( ) ) )
{
subquery = subquery_or_table_name_or_table_expression ;
subquery_or_table_name = subquery ;
}
if ( ! subquery_or_table_name )
throw Exception ( " Logical error: unknown AST element passed to ExpressionAnalyzer::addExternalStorage method " , ErrorCodes : : LOGICAL_ERROR ) ;
if ( table_name )
{
2017-04-02 17:37:49 +00:00
/// If this is already an external table, you do not need to add anything. Just remember its presence.
2017-04-01 07:20:54 +00:00
if ( external_tables . end ( ) ! = external_tables . find ( static_cast < const ASTIdentifier & > ( * table_name ) . name ) )
return ;
}
2017-04-02 17:37:49 +00:00
/// Generate the name for the external table.
2017-04-01 07:20:54 +00:00
String external_table_name = " _data " + toString ( external_table_id ) ;
while ( external_tables . count ( external_table_name ) )
{
+ + external_table_id ;
external_table_name = " _data " + toString ( external_table_id ) ;
}
auto interpreter = interpretSubquery ( subquery_or_table_name , context , subquery_depth , { } ) ;
Block sample = interpreter - > getSampleBlock ( ) ;
2017-12-25 21:57:29 +00:00
NamesAndTypesList columns = sample . getNamesAndTypesList ( ) ;
2017-04-01 07:20:54 +00:00
2018-03-10 17:03:57 +00:00
StoragePtr external_storage = StorageMemory : : create ( external_table_name , ColumnsDescription { columns } ) ;
2017-06-06 17:06:14 +00:00
external_storage - > startup ( ) ;
2017-04-01 07:20:54 +00:00
2018-02-20 01:38:42 +00:00
/** We replace the subquery with the name of the temporary table.
* It is in this form , the request will go to the remote server .
* This temporary table will go to the remote server , and on its side ,
* instead of doing a subquery , you just need to read it .
*/
2017-04-01 07:20:54 +00:00
2018-02-26 03:40:20 +00:00
auto database_and_table_name = std : : make_shared < ASTIdentifier > ( external_table_name , ASTIdentifier : : Table ) ;
2017-04-01 07:20:54 +00:00
2018-02-20 01:38:42 +00:00
if ( auto ast_table_expr = typeid_cast < ASTTableExpression * > ( subquery_or_table_name_or_table_expression . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2018-02-20 01:38:42 +00:00
ast_table_expr - > subquery . reset ( ) ;
ast_table_expr - > database_and_table_name = database_and_table_name ;
2017-04-01 07:20:54 +00:00
2018-02-20 01:38:42 +00:00
ast_table_expr - > children . clear ( ) ;
ast_table_expr - > children . emplace_back ( database_and_table_name ) ;
2017-04-01 07:20:54 +00:00
}
else
2018-02-20 01:38:42 +00:00
subquery_or_table_name_or_table_expression = database_and_table_name ;
2017-04-01 07:20:54 +00:00
external_tables [ external_table_name ] = external_storage ;
subqueries_for_sets [ external_table_name ] . source = interpreter - > execute ( ) . in ;
subqueries_for_sets [ external_table_name ] . table = external_storage ;
2017-04-02 17:37:49 +00:00
/** NOTE If it was written IN tmp_table - the existing temporary (but not external) table,
* then a new temporary table will be created ( for example , _data1 ) ,
* and the data will then be copied to it .
* Maybe this can be avoided .
2017-04-01 07:20:54 +00:00
*/
2015-11-04 22:02:52 +00:00
}
2018-03-08 02:16:03 +00:00
static NamesAndTypesList : : iterator findColumn ( const String & name , NamesAndTypesList & cols )
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
return std : : find_if ( cols . begin ( ) , cols . end ( ) ,
2017-12-25 21:57:29 +00:00
[ & ] ( const NamesAndTypesList : : value_type & val ) { return val . name = = name ; } ) ;
2013-05-24 10:49:19 +00:00
}
2017-04-02 17:37:49 +00:00
/// ignore_levels - aliases in how many upper levels of the subtree should be ignored.
/// For example, with ignore_levels=1 ast can not be put in the dictionary, but its children can.
2014-10-03 15:30:10 +00:00
void ExpressionAnalyzer : : addASTAliases ( ASTPtr & ast , int ignore_levels )
2013-05-24 10:49:19 +00:00
{
2017-04-02 17:37:49 +00:00
/// Bottom-up traversal. We do not go into subqueries.
2017-04-01 07:20:54 +00:00
for ( auto & child : ast - > children )
{
int new_ignore_levels = std : : max ( 0 , ignore_levels - 1 ) ;
2017-04-02 17:37:49 +00:00
/// The top-level aliases in the ARRAY JOIN section have a special meaning, we will not add them
/// (skip the expression list itself and its children).
2017-04-01 07:20:54 +00:00
if ( typeid_cast < ASTArrayJoin * > ( ast . get ( ) ) )
new_ignore_levels = 3 ;
2018-02-26 09:05:06 +00:00
/// Don't descent into table functions and subqueries.
2017-04-01 07:20:54 +00:00
if ( ! typeid_cast < ASTTableExpression * > ( child . get ( ) )
2018-02-26 09:05:06 +00:00
& & ! typeid_cast < ASTSelectWithUnionQuery * > ( child . get ( ) ) )
2017-04-01 07:20:54 +00:00
addASTAliases ( child , new_ignore_levels ) ;
}
if ( ignore_levels > 0 )
return ;
String alias = ast - > tryGetAlias ( ) ;
if ( ! alias . empty ( ) )
{
if ( aliases . count ( alias ) & & ast - > getTreeHash ( ) ! = aliases [ alias ] - > getTreeHash ( ) )
2018-02-05 14:03:01 +00:00
{
std : : stringstream message ;
message < < " Different expressions with the same alias " < < backQuoteIfNeed ( alias ) < < " : \n " ;
formatAST ( * ast , message , false , true ) ;
message < < " \n and \n " ;
formatAST ( * aliases [ alias ] , message , false , true ) ;
message < < " \n " ;
throw Exception ( message . str ( ) , ErrorCodes : : MULTIPLE_EXPRESSIONS_FOR_ALIAS ) ;
}
2017-04-01 07:20:54 +00:00
aliases [ alias ] = ast ;
}
2018-02-23 08:05:21 +00:00
else if ( auto subquery = typeid_cast < ASTSubquery * > ( ast . get ( ) ) )
2017-07-26 01:21:19 +00:00
{
/// Set unique aliases for all subqueries. This is needed, because content of subqueries could change after recursive analysis,
/// and auto-generated column names could become incorrect.
size_t subquery_index = 1 ;
while ( true )
{
alias = " _subquery " + toString ( subquery_index ) ;
if ( ! aliases . count ( " _subquery " + toString ( subquery_index ) ) )
break ;
+ + subquery_index ;
}
2018-02-23 08:05:21 +00:00
subquery - > setAlias ( alias ) ;
subquery - > prefer_alias_to_column_name = true ;
2017-07-26 01:21:19 +00:00
aliases [ alias ] = ast ;
}
2013-05-24 10:49:19 +00:00
}
void ExpressionAnalyzer : : normalizeTree ( )
{
2017-04-01 07:20:54 +00:00
SetOfASTs tmp_set ;
MapOfASTs tmp_map ;
2018-08-27 17:58:43 +00:00
normalizeTreeImpl ( query , tmp_map , tmp_set , " " , 0 ) ;
2018-03-02 06:33:17 +00:00
try
{
2018-08-27 17:58:43 +00:00
query - > checkSize ( settings . max_expanded_ast_elements ) ;
2018-03-02 06:33:17 +00:00
}
catch ( Exception & e )
{
e . addMessage ( " (after expansion of aliases) " ) ;
throw ;
}
2013-05-24 10:49:19 +00:00
}
2017-04-02 17:37:49 +00:00
/// finished_asts - already processed vertices (and by what they replaced)
/// current_asts - vertices in the current call stack of this method
/// current_alias - the alias referencing to the ancestor of ast (the deepest ancestor with aliases)
2015-05-12 20:09:18 +00:00
void ExpressionAnalyzer : : normalizeTreeImpl (
2017-04-01 07:20:54 +00:00
ASTPtr & ast , MapOfASTs & finished_asts , SetOfASTs & current_asts , std : : string current_alias , size_t level )
{
2018-03-11 00:15:26 +00:00
if ( level > settings . max_ast_depth )
2018-03-02 06:33:17 +00:00
throw Exception ( " Normalized AST is too deep. Maximum: "
2018-03-11 00:15:26 +00:00
+ settings . max_ast_depth . toString ( ) , ErrorCodes : : TOO_DEEP_AST ) ;
2017-04-01 07:20:54 +00:00
if ( finished_asts . count ( ast ) )
{
ast = finished_asts [ ast ] ;
return ;
}
ASTPtr initial_ast = ast ;
current_asts . insert ( initial_ast . get ( ) ) ;
String my_alias = ast - > tryGetAlias ( ) ;
if ( ! my_alias . empty ( ) )
current_alias = my_alias ;
2017-04-02 17:37:49 +00:00
/// rewrite rules that act when you go from top to bottom.
2017-04-01 07:20:54 +00:00
bool replaced = false ;
2017-08-16 20:10:25 +00:00
ASTIdentifier * identifier_node = nullptr ;
ASTFunction * func_node = nullptr ;
if ( ( func_node = typeid_cast < ASTFunction * > ( ast . get ( ) ) ) )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
2017-04-01 07:20:54 +00:00
if ( functionIsInOrGlobalInOperator ( func_node - > name ) )
if ( ASTIdentifier * right = typeid_cast < ASTIdentifier * > ( func_node - > arguments - > children . at ( 1 ) . get ( ) ) )
2018-03-15 16:57:10 +00:00
if ( ! aliases . count ( right - > name ) )
right - > kind = ASTIdentifier : : Table ;
2017-04-01 07:20:54 +00:00
/// Special cases for count function.
String func_name_lowercase = Poco : : toLower ( func_node - > name ) ;
if ( startsWith ( func_name_lowercase , " count " ) )
{
/// Select implementation of countDistinct based on settings.
/// Important that it is done as query rewrite. It means rewritten query
/// will be sent to remote servers during distributed query execution,
/// and on all remote servers, function implementation will be same.
if ( endsWith ( func_node - > name , " Distinct " ) & & func_name_lowercase = = " countdistinct " )
func_node - > name = settings . count_distinct_implementation ;
/// As special case, treat count(*) as count(), not as count(list of all columns).
if ( func_name_lowercase = = " count " & & func_node - > arguments - > children . size ( ) = = 1
& & typeid_cast < const ASTAsterisk * > ( func_node - > arguments - > children [ 0 ] . get ( ) ) )
{
func_node - > arguments - > children . clear ( ) ;
}
}
}
2017-08-16 20:10:25 +00:00
else if ( ( identifier_node = typeid_cast < ASTIdentifier * > ( ast . get ( ) ) ) )
2017-04-01 07:20:54 +00:00
{
2017-08-16 20:10:25 +00:00
if ( identifier_node - > kind = = ASTIdentifier : : Column )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column").
2017-10-11 13:33:27 +00:00
auto it_alias = aliases . find ( identifier_node - > name ) ;
if ( it_alias ! = aliases . end ( ) & & current_alias ! = identifier_node - > name )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// Let's replace it with the corresponding tree node.
2017-10-11 13:33:27 +00:00
if ( current_asts . count ( it_alias - > second . get ( ) ) )
2017-04-01 07:20:54 +00:00
throw Exception ( " Cyclic aliases " , ErrorCodes : : CYCLIC_ALIASES ) ;
2017-10-11 13:33:27 +00:00
if ( ! my_alias . empty ( ) & & my_alias ! = it_alias - > second - > getAliasOrColumnName ( ) )
2017-04-01 07:20:54 +00:00
{
2017-10-11 13:33:27 +00:00
/// Avoid infinite recursion here
auto replace_to_identifier = typeid_cast < ASTIdentifier * > ( it_alias - > second . get ( ) ) ;
bool is_cycle = replace_to_identifier & &
replace_to_identifier - > kind = = ASTIdentifier : : Column & &
replace_to_identifier - > name = = identifier_node - > name ;
if ( ! is_cycle )
{
/// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a.
ast = it_alias - > second - > clone ( ) ;
ast - > setAlias ( my_alias ) ;
replaced = true ;
}
2017-04-01 07:20:54 +00:00
}
else
{
2017-10-11 13:33:27 +00:00
ast = it_alias - > second ;
replaced = true ;
2017-04-01 07:20:54 +00:00
}
}
}
}
2018-08-27 17:58:43 +00:00
else if ( ASTExpressionList * expr_list = typeid_cast < ASTExpressionList * > ( ast . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// Replace * with a list of columns.
2018-08-27 17:58:43 +00:00
ASTs & asts = expr_list - > children ;
2017-04-01 07:20:54 +00:00
for ( int i = static_cast < int > ( asts . size ( ) ) - 1 ; i > = 0 ; - - i )
{
2018-02-26 03:45:46 +00:00
if ( typeid_cast < ASTAsterisk * > ( asts [ i ] . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2018-08-02 04:15:39 +00:00
Names all_columns_name ;
2018-02-27 19:00:55 +00:00
2018-08-02 04:15:39 +00:00
auto columns_name = storage ? storage - > getColumns ( ) . ordinary . getNames ( ) : source_columns . getNames ( ) ;
all_columns_name . insert ( all_columns_name . begin ( ) , columns_name . begin ( ) , columns_name . end ( ) ) ;
2018-08-04 10:53:50 +00:00
if ( ! settings . asterisk_left_columns_only )
2018-02-27 19:00:55 +00:00
{
2018-08-04 10:53:50 +00:00
auto columns_from_joined_table = analyzed_join . getColumnsFromJoinedTable ( context , select_query ) . getNames ( ) ;
all_columns_name . insert ( all_columns_name . end ( ) , columns_from_joined_table . begin ( ) , columns_from_joined_table . end ( ) ) ;
2018-02-27 19:00:55 +00:00
}
2017-04-01 07:20:54 +00:00
asts . erase ( asts . begin ( ) + i ) ;
2018-08-02 04:15:39 +00:00
for ( size_t idx = 0 ; idx < all_columns_name . size ( ) ; idx + + )
asts . insert ( asts . begin ( ) + idx + i , std : : make_shared < ASTIdentifier > ( all_columns_name [ idx ] ) ) ;
2017-04-01 07:20:54 +00:00
}
}
}
2018-08-27 17:58:43 +00:00
else if ( ASTTablesInSelectQueryElement * tables_elem = typeid_cast < ASTTablesInSelectQueryElement * > ( ast . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2018-08-27 17:58:43 +00:00
if ( tables_elem - > table_expression )
2017-04-01 07:20:54 +00:00
{
2018-08-27 17:58:43 +00:00
auto & database_and_table_name = static_cast < ASTTableExpression & > ( * tables_elem - > table_expression ) . database_and_table_name ;
2017-04-01 07:20:54 +00:00
if ( database_and_table_name )
{
if ( ASTIdentifier * right = typeid_cast < ASTIdentifier * > ( database_and_table_name . get ( ) ) )
{
right - > kind = ASTIdentifier : : Table ;
}
}
}
}
2017-04-02 17:37:49 +00:00
/// If we replace the root of the subtree, we will be called again for the new root, in case the alias is replaced by an alias.
2017-04-01 07:20:54 +00:00
if ( replaced )
{
normalizeTreeImpl ( ast , finished_asts , current_asts , current_alias , level + 1 ) ;
current_asts . erase ( initial_ast . get ( ) ) ;
current_asts . erase ( ast . get ( ) ) ;
finished_asts [ initial_ast ] = ast ;
return ;
}
2017-08-16 20:10:25 +00:00
/// Recurring calls. Don't go into subqueries. Don't go into components of compound identifiers.
2017-04-02 17:37:49 +00:00
/// We also do not go to the left argument of lambda expressions, so as not to replace the formal parameters
/// on aliases in expressions of the form 123 AS x, arrayMap(x -> 1, [2]).
2017-04-01 07:20:54 +00:00
if ( func_node & & func_node - > name = = " lambda " )
{
2017-04-02 17:37:49 +00:00
/// We skip the first argument. We also assume that the lambda function can not have parameters.
2017-04-01 07:20:54 +00:00
for ( size_t i = 1 , size = func_node - > arguments - > children . size ( ) ; i < size ; + + i )
{
auto & child = func_node - > arguments - > children [ i ] ;
if ( typeid_cast < const ASTSelectQuery * > ( child . get ( ) )
| | typeid_cast < const ASTTableExpression * > ( child . get ( ) ) )
continue ;
normalizeTreeImpl ( child , finished_asts , current_asts , current_alias , level + 1 ) ;
}
}
2017-08-16 20:10:25 +00:00
else if ( identifier_node )
{
}
2017-04-01 07:20:54 +00:00
else
{
for ( auto & child : ast - > children )
{
if ( typeid_cast < const ASTSelectQuery * > ( child . get ( ) )
| | typeid_cast < const ASTTableExpression * > ( child . get ( ) ) )
continue ;
normalizeTreeImpl ( child , finished_asts , current_asts , current_alias , level + 1 ) ;
}
}
2017-04-02 17:37:49 +00:00
/// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children, but also in where_expression and having_expression.
2017-04-01 07:20:54 +00:00
if ( ASTSelectQuery * select = typeid_cast < ASTSelectQuery * > ( ast . get ( ) ) )
{
if ( select - > prewhere_expression )
normalizeTreeImpl ( select - > prewhere_expression , finished_asts , current_asts , current_alias , level + 1 ) ;
if ( select - > where_expression )
normalizeTreeImpl ( select - > where_expression , finished_asts , current_asts , current_alias , level + 1 ) ;
if ( select - > having_expression )
normalizeTreeImpl ( select - > having_expression , finished_asts , current_asts , current_alias , level + 1 ) ;
}
current_asts . erase ( initial_ast . get ( ) ) ;
current_asts . erase ( ast . get ( ) ) ;
finished_asts [ initial_ast ] = ast ;
2013-05-24 10:49:19 +00:00
}
2015-07-25 11:47:50 +00:00
2018-02-27 19:00:55 +00:00
void ExpressionAnalyzer : : addAliasColumns ( )
2015-10-29 15:14:19 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! select_query )
return ;
2015-10-29 15:14:19 +00:00
2017-04-01 07:20:54 +00:00
if ( ! storage )
return ;
2015-10-29 15:14:19 +00:00
2018-08-27 17:58:43 +00:00
const auto & storage_aliases = storage - > getColumns ( ) . aliases ;
source_columns . insert ( std : : end ( source_columns ) , std : : begin ( storage_aliases ) , std : : end ( storage_aliases ) ) ;
2015-10-29 15:14:19 +00:00
}
2015-07-25 11:47:50 +00:00
void ExpressionAnalyzer : : executeScalarSubqueries ( )
{
2017-04-01 07:20:54 +00:00
if ( ! select_query )
2018-08-27 17:58:43 +00:00
executeScalarSubqueriesImpl ( query ) ;
2017-04-01 07:20:54 +00:00
else
{
2018-08-27 17:58:43 +00:00
for ( auto & child : query - > children )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// Do not go to FROM, JOIN, UNION.
2017-04-01 07:20:54 +00:00
if ( ! typeid_cast < const ASTTableExpression * > ( child . get ( ) )
2018-02-24 01:55:43 +00:00
& & ! typeid_cast < const ASTSelectQuery * > ( child . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
executeScalarSubqueriesImpl ( child ) ;
}
}
}
2015-07-25 11:47:50 +00:00
}
2015-07-26 07:55:48 +00:00
2016-05-30 20:31:03 +00:00
static ASTPtr addTypeConversion ( std : : unique_ptr < ASTLiteral > & & ast , const String & type_name )
2015-07-26 07:55:48 +00:00
{
2018-02-26 03:40:20 +00:00
auto func = std : : make_shared < ASTFunction > ( ) ;
2017-04-01 07:20:54 +00:00
ASTPtr res = func ;
func - > alias = ast - > alias ;
2017-08-10 14:46:46 +00:00
func - > prefer_alias_to_column_name = ast - > prefer_alias_to_column_name ;
2017-04-01 07:20:54 +00:00
ast - > alias . clear ( ) ;
func - > name = " CAST " ;
2018-02-26 03:40:20 +00:00
auto exp_list = std : : make_shared < ASTExpressionList > ( ) ;
2017-04-01 07:20:54 +00:00
func - > arguments = exp_list ;
func - > children . push_back ( func - > arguments ) ;
exp_list - > children . emplace_back ( ast . release ( ) ) ;
2018-02-26 03:40:20 +00:00
exp_list - > children . emplace_back ( std : : make_shared < ASTLiteral > ( type_name ) ) ;
2017-04-01 07:20:54 +00:00
return res ;
2015-07-26 07:55:48 +00:00
}
2015-07-25 11:47:50 +00:00
void ExpressionAnalyzer : : executeScalarSubqueriesImpl ( ASTPtr & ast )
{
2017-04-02 17:37:49 +00:00
/** Replace subqueries that return exactly one row
* ( " scalar " subqueries ) to the corresponding constants .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* If the subquery returns more than one column , it is replaced by a tuple of constants .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* Features
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* A replacement occurs during query analysis , and not during the main runtime .
* This means that the progress indicator will not work during the execution of these requests ,
* and also such queries can not be aborted .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* But the query result can be used for the index in the table .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* Scalar subqueries are executed on the request - initializer server .
* The request is sent to remote servers with already substituted constants .
2017-04-01 07:20:54 +00:00
*/
if ( ASTSubquery * subquery = typeid_cast < ASTSubquery * > ( ast . get ( ) ) )
{
Context subquery_context = context ;
Settings subquery_settings = context . getSettings ( ) ;
2018-03-11 00:15:26 +00:00
subquery_settings . max_result_rows = 1 ;
2017-04-01 07:20:54 +00:00
subquery_settings . extremes = 0 ;
subquery_context . setSettings ( subquery_settings ) ;
2018-08-27 17:58:43 +00:00
ASTPtr subquery_select = subquery - > children . at ( 0 ) ;
BlockIO res = InterpreterSelectWithUnionQuery ( subquery_select , subquery_context , { } , QueryProcessingStage : : Complete , subquery_depth + 1 ) . execute ( ) ;
2017-04-01 07:20:54 +00:00
Block block ;
try
{
block = res . in - > read ( ) ;
if ( ! block )
{
/// Interpret subquery with empty result as Null literal
2018-02-26 03:40:20 +00:00
auto ast_new = std : : make_unique < ASTLiteral > ( Null ( ) ) ;
2017-06-19 16:22:54 +00:00
ast_new - > setAlias ( ast - > tryGetAlias ( ) ) ;
ast = std : : move ( ast_new ) ;
2017-04-01 07:20:54 +00:00
return ;
}
if ( block . rows ( ) ! = 1 | | res . in - > read ( ) )
throw Exception ( " Scalar subquery returned more than one row " , ErrorCodes : : INCORRECT_RESULT_OF_SCALAR_SUBQUERY ) ;
}
catch ( const Exception & e )
{
2018-03-09 23:23:15 +00:00
if ( e . code ( ) = = ErrorCodes : : TOO_MANY_ROWS )
2017-04-01 07:20:54 +00:00
throw Exception ( " Scalar subquery returned more than one row " , ErrorCodes : : INCORRECT_RESULT_OF_SCALAR_SUBQUERY ) ;
else
throw ;
}
size_t columns = block . columns ( ) ;
if ( columns = = 1 )
{
2018-02-26 03:40:20 +00:00
auto lit = std : : make_unique < ASTLiteral > ( ( * block . safeGetByPosition ( 0 ) . column ) [ 0 ] ) ;
2017-04-01 07:20:54 +00:00
lit - > alias = subquery - > alias ;
2017-08-10 14:46:46 +00:00
lit - > prefer_alias_to_column_name = subquery - > prefer_alias_to_column_name ;
2017-04-01 07:20:54 +00:00
ast = addTypeConversion ( std : : move ( lit ) , block . safeGetByPosition ( 0 ) . type - > getName ( ) ) ;
}
else
{
2018-02-26 03:40:20 +00:00
auto tuple = std : : make_shared < ASTFunction > ( ) ;
2017-04-01 07:20:54 +00:00
tuple - > alias = subquery - > alias ;
ast = tuple ;
tuple - > name = " tuple " ;
2018-02-26 03:40:20 +00:00
auto exp_list = std : : make_shared < ASTExpressionList > ( ) ;
2017-04-01 07:20:54 +00:00
tuple - > arguments = exp_list ;
tuple - > children . push_back ( tuple - > arguments ) ;
exp_list - > children . resize ( columns ) ;
for ( size_t i = 0 ; i < columns ; + + i )
{
exp_list - > children [ i ] = addTypeConversion (
2018-02-26 03:40:20 +00:00
std : : make_unique < ASTLiteral > ( ( * block . safeGetByPosition ( i ) . column ) [ 0 ] ) ,
2017-04-01 07:20:54 +00:00
block . safeGetByPosition ( i ) . type - > getName ( ) ) ;
}
}
}
else
{
/** Don't descend into subqueries in FROM section.
*/
if ( ! typeid_cast < ASTTableExpression * > ( ast . get ( ) ) )
{
/** Don't descend into subqueries in arguments of IN operator.
* But if an argument is not subquery , than deeper may be scalar subqueries and we need to descend in them .
*/
ASTFunction * func = typeid_cast < ASTFunction * > ( ast . get ( ) ) ;
2018-02-24 01:31:42 +00:00
if ( func & & functionIsInOrGlobalInOperator ( func - > name ) )
2017-04-01 07:20:54 +00:00
{
for ( auto & child : ast - > children )
{
if ( child ! = func - > arguments )
executeScalarSubqueriesImpl ( child ) ;
else
for ( size_t i = 0 , size = func - > arguments - > children . size ( ) ; i < size ; + + i )
if ( i ! = 1 | | ! typeid_cast < ASTSubquery * > ( func - > arguments - > children [ i ] . get ( ) ) )
executeScalarSubqueriesImpl ( func - > arguments - > children [ i ] ) ;
}
}
else
for ( auto & child : ast - > children )
executeScalarSubqueriesImpl ( child ) ;
}
}
2015-07-25 11:47:50 +00:00
}
2014-08-04 15:25:38 +00:00
void ExpressionAnalyzer : : optimizeGroupBy ( )
2014-07-31 07:24:07 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! ( select_query & & select_query - > group_expression_list ) )
return ;
2018-01-10 00:04:08 +00:00
const auto is_literal = [ ] ( const ASTPtr & ast )
{
return typeid_cast < const ASTLiteral * > ( ast . get ( ) ) ;
2017-04-01 07:20:54 +00:00
} ;
auto & group_exprs = select_query - > group_expression_list - > children ;
/// removes expression at index idx by making it last one and calling .pop_back()
const auto remove_expr_at_index = [ & group_exprs ] ( const size_t idx )
{
if ( idx < group_exprs . size ( ) - 1 )
std : : swap ( group_exprs [ idx ] , group_exprs . back ( ) ) ;
group_exprs . pop_back ( ) ;
} ;
/// iterate over each GROUP BY expression, eliminate injective function calls and literals
for ( size_t i = 0 ; i < group_exprs . size ( ) ; )
{
if ( const auto function = typeid_cast < ASTFunction * > ( group_exprs [ i ] . get ( ) ) )
{
/// assert function is injective
if ( possibly_injective_function_names . count ( function - > name ) )
{
/// do not handle semantic errors here
if ( function - > arguments - > children . size ( ) < 2 )
{
+ + i ;
continue ;
}
const auto & dict_name = typeid_cast < const ASTLiteral & > ( * function - > arguments - > children [ 0 ] )
. value . safeGet < String > ( ) ;
const auto & dict_ptr = context . getExternalDictionaries ( ) . getDictionary ( dict_name ) ;
const auto & attr_name = typeid_cast < const ASTLiteral & > ( * function - > arguments - > children [ 1 ] )
. value . safeGet < String > ( ) ;
if ( ! dict_ptr - > isInjective ( attr_name ) )
{
+ + i ;
continue ;
}
}
else if ( ! injective_function_names . count ( function - > name ) )
{
+ + i ;
continue ;
}
/// copy shared pointer to args in order to ensure lifetime
auto args_ast = function - > arguments ;
/** remove function call and take a step back to ensure
* next iteration does not skip not yet processed data
*/
remove_expr_at_index ( i ) ;
/// copy non-literal arguments
std : : remove_copy_if (
std : : begin ( args_ast - > children ) , std : : end ( args_ast - > children ) ,
std : : back_inserter ( group_exprs ) , is_literal
) ;
}
else if ( is_literal ( group_exprs [ i ] ) )
{
remove_expr_at_index ( i ) ;
}
else
{
/// if neither a function nor literal - advance to next expression
+ + i ;
}
}
if ( group_exprs . empty ( ) )
{
2017-04-02 17:37:49 +00:00
/** You can not completely remove GROUP BY. Because if there were no aggregate functions, then it turns out that there will be no aggregation.
* Instead , leave ` GROUP BY const ` .
* Next , see deleting the constants in the analyzeAggregation method .
2017-04-01 07:20:54 +00:00
*/
2017-04-02 17:37:49 +00:00
/// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens.
2017-04-01 07:20:54 +00:00
UInt64 unused_column = 0 ;
String unused_column_name = toString ( unused_column ) ;
2018-02-26 09:05:06 +00:00
while ( source_columns . end ( ) ! = std : : find_if ( source_columns . begin ( ) , source_columns . end ( ) ,
2017-12-25 21:57:29 +00:00
[ & unused_column_name ] ( const NameAndTypePair & name_type ) { return name_type . name = = unused_column_name ; } ) )
2017-04-01 07:20:54 +00:00
{
+ + unused_column ;
unused_column_name = toString ( unused_column ) ;
}
select_query - > group_expression_list = std : : make_shared < ASTExpressionList > ( ) ;
2018-02-26 03:40:20 +00:00
select_query - > group_expression_list - > children . emplace_back ( std : : make_shared < ASTLiteral > ( UInt64 ( unused_column ) ) ) ;
2017-04-01 07:20:54 +00:00
}
2014-07-31 07:24:07 +00:00
}
2015-03-12 03:31:15 +00:00
void ExpressionAnalyzer : : optimizeOrderBy ( )
{
2017-04-01 07:20:54 +00:00
if ( ! ( select_query & & select_query - > order_expression_list ) )
return ;
2015-03-12 03:31:15 +00:00
2017-04-02 17:37:49 +00:00
/// Make unique sorting conditions.
2017-04-01 07:20:54 +00:00
using NameAndLocale = std : : pair < String , String > ;
std : : set < NameAndLocale > elems_set ;
2015-03-12 03:31:15 +00:00
2017-04-01 07:20:54 +00:00
ASTs & elems = select_query - > order_expression_list - > children ;
ASTs unique_elems ;
unique_elems . reserve ( elems . size ( ) ) ;
2015-03-12 03:31:15 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & elem : elems )
{
String name = elem - > children . front ( ) - > getColumnName ( ) ;
const ASTOrderByElement & order_by_elem = typeid_cast < const ASTOrderByElement & > ( * elem ) ;
2015-03-12 03:31:15 +00:00
2017-04-01 07:20:54 +00:00
if ( elems_set . emplace ( name , order_by_elem . collation ? order_by_elem . collation - > getColumnName ( ) : " " ) . second )
unique_elems . emplace_back ( elem ) ;
}
2015-03-12 03:31:15 +00:00
2017-04-01 07:20:54 +00:00
if ( unique_elems . size ( ) < elems . size ( ) )
elems = unique_elems ;
2015-03-12 03:31:15 +00:00
}
2016-12-28 15:12:54 +00:00
void ExpressionAnalyzer : : optimizeLimitBy ( )
{
2017-04-01 07:20:54 +00:00
if ( ! ( select_query & & select_query - > limit_by_expression_list ) )
return ;
2016-12-28 15:12:54 +00:00
2017-04-01 07:20:54 +00:00
std : : set < String > elems_set ;
2016-12-28 15:12:54 +00:00
2017-04-01 07:20:54 +00:00
ASTs & elems = select_query - > limit_by_expression_list - > children ;
ASTs unique_elems ;
unique_elems . reserve ( elems . size ( ) ) ;
2016-12-28 15:12:54 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & elem : elems )
{
if ( elems_set . emplace ( elem - > getColumnName ( ) ) . second )
unique_elems . emplace_back ( elem ) ;
}
2016-12-28 15:12:54 +00:00
2017-04-01 07:20:54 +00:00
if ( unique_elems . size ( ) < elems . size ( ) )
elems = unique_elems ;
2016-12-28 15:12:54 +00:00
}
2018-08-31 10:00:46 +00:00
void ExpressionAnalyzer : : optimizeUsing ( )
{
if ( ! select_query )
return ;
auto node = const_cast < ASTTablesInSelectQueryElement * > ( select_query - > join ( ) ) ;
if ( ! node )
return ;
auto table_join = static_cast < ASTTableJoin * > ( & * node - > table_join ) ;
if ( ! ( table_join & & table_join - > using_expression_list ) )
return ;
ASTs & expression_list = table_join - > using_expression_list - > children ;
ASTs uniq_expressions_list ;
std : : set < String > expressions_names ;
for ( const auto & expression : expression_list )
{
auto expression_name = expression - > getAliasOrColumnName ( ) ;
if ( expressions_names . find ( expression_name ) = = expressions_names . end ( ) )
{
uniq_expressions_list . push_back ( expression ) ;
expressions_names . insert ( expression_name ) ;
}
}
if ( uniq_expressions_list . size ( ) < expression_list . size ( ) )
expression_list = uniq_expressions_list ;
}
2016-12-28 15:12:54 +00:00
2014-04-10 08:57:01 +00:00
void ExpressionAnalyzer : : makeSetsForIndex ( )
2014-03-31 14:49:43 +00:00
{
2018-03-16 06:33:23 +00:00
if ( storage & & select_query & & storage - > supportsIndexForIn ( ) )
{
if ( select_query - > where_expression )
makeSetsForIndexImpl ( select_query - > where_expression , storage - > getSampleBlock ( ) ) ;
if ( select_query - > prewhere_expression )
makeSetsForIndexImpl ( select_query - > prewhere_expression , storage - > getSampleBlock ( ) ) ;
}
2014-03-31 14:49:43 +00:00
}
2018-01-21 07:30:07 +00:00
2018-07-02 19:37:04 +00:00
void ExpressionAnalyzer : : tryMakeSetForIndexFromSubquery ( const ASTPtr & subquery_or_table_name )
2018-01-21 07:30:07 +00:00
{
BlockIO res = interpretSubquery ( subquery_or_table_name , context , subquery_depth + 1 , { } ) - > execute ( ) ;
2018-07-19 20:22:49 +00:00
SizeLimits set_for_index_size_limits = SizeLimits ( settings . max_rows_in_set , settings . max_bytes_in_set , settings . set_overflow_mode ) ;
2018-07-02 19:37:04 +00:00
SetPtr set = std : : make_shared < Set > ( set_for_index_size_limits , true ) ;
2018-01-21 07:30:07 +00:00
2018-04-19 21:34:04 +00:00
set - > setHeader ( res . in - > getHeader ( ) ) ;
2018-01-21 07:30:07 +00:00
while ( Block block = res . in - > read ( ) )
{
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
2018-07-02 18:57:14 +00:00
if ( ! set - > insertFromBlock ( block ) )
2018-01-21 07:30:07 +00:00
return ;
}
2018-05-16 10:25:10 +00:00
prepared_sets [ subquery_or_table_name - > range ] = std : : move ( set ) ;
2018-01-21 07:30:07 +00:00
}
2017-07-15 03:48:36 +00:00
void ExpressionAnalyzer : : makeSetsForIndexImpl ( const ASTPtr & node , const Block & sample_block )
2014-03-31 14:49:43 +00:00
{
2017-04-01 07:20:54 +00:00
for ( auto & child : node - > children )
2018-02-14 17:39:16 +00:00
{
2018-03-16 14:01:00 +00:00
/// Don't descent into subqueries.
2018-03-16 06:33:23 +00:00
if ( typeid_cast < ASTSubquery * > ( child . get ( ) ) )
continue ;
/// Don't dive into lambda functions
const ASTFunction * func = typeid_cast < const ASTFunction * > ( child . get ( ) ) ;
if ( func & & func - > name = = " lambda " )
continue ;
makeSetsForIndexImpl ( child , sample_block ) ;
2018-02-14 17:39:16 +00:00
}
2017-04-01 07:20:54 +00:00
2017-07-14 00:33:37 +00:00
const ASTFunction * func = typeid_cast < const ASTFunction * > ( node . get ( ) ) ;
2018-02-24 01:31:42 +00:00
if ( func & & functionIsInOperator ( func - > name ) )
2017-04-01 07:20:54 +00:00
{
2017-07-14 00:33:37 +00:00
const IAST & args = * func - > arguments ;
2017-04-01 07:20:54 +00:00
2018-03-16 06:39:32 +00:00
if ( storage & & storage - > mayBenefitFromIndexForIn ( args . children . at ( 0 ) ) )
2017-04-01 07:20:54 +00:00
{
2018-03-16 06:39:32 +00:00
const ASTPtr & arg = args . children . at ( 1 ) ;
2018-05-16 10:25:10 +00:00
if ( ! prepared_sets . count ( arg - > range ) ) /// Not already prepared.
2017-04-01 07:20:54 +00:00
{
2018-03-16 06:39:32 +00:00
if ( typeid_cast < ASTSubquery * > ( arg . get ( ) ) | | typeid_cast < ASTIdentifier * > ( arg . get ( ) ) )
{
if ( settings . use_index_for_in_with_subqueries )
2018-07-02 19:37:04 +00:00
tryMakeSetForIndexFromSubquery ( arg ) ;
2018-03-16 06:39:32 +00:00
}
else
{
2018-04-03 19:16:24 +00:00
NamesAndTypesList temp_columns = source_columns ;
2018-04-04 18:56:30 +00:00
temp_columns . insert ( temp_columns . end ( ) , array_join_columns . begin ( ) , array_join_columns . end ( ) ) ;
2018-07-31 11:31:18 +00:00
for ( const auto & joined_column : analyzed_join . columns_added_by_join )
temp_columns . push_back ( joined_column . name_and_type ) ;
2018-04-03 19:16:24 +00:00
ExpressionActionsPtr temp_actions = std : : make_shared < ExpressionActions > ( temp_columns , settings ) ;
2018-03-16 06:39:32 +00:00
getRootActions ( func - > arguments - > children . at ( 0 ) , true , false , temp_actions ) ;
2018-03-16 04:37:21 +00:00
2018-03-16 06:39:32 +00:00
Block sample_block_with_calculated_columns = temp_actions - > getSampleBlock ( ) ;
if ( sample_block_with_calculated_columns . has ( args . children . at ( 0 ) - > getColumnName ( ) ) )
makeExplicitSet ( func , sample_block_with_calculated_columns , true ) ;
}
2017-04-01 07:20:54 +00:00
}
}
}
2014-03-31 14:49:43 +00:00
}
2013-05-24 10:49:19 +00:00
2014-03-14 14:52:48 +00:00
2017-07-14 00:33:37 +00:00
void ExpressionAnalyzer : : makeSet ( const ASTFunction * node , const Block & sample_block )
2013-05-24 10:49:19 +00:00
{
2017-04-02 17:37:49 +00:00
/** You need to convert the right argument to a set.
* This can be a table name , a value , a value enumeration , or a subquery .
* The enumeration of values is parsed as a function ` tuple ` .
2017-04-01 07:20:54 +00:00
*/
2017-07-14 00:33:37 +00:00
const IAST & args = * node - > arguments ;
const ASTPtr & arg = args . children . at ( 1 ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Already converted.
2018-05-16 10:25:10 +00:00
if ( prepared_sets . count ( arg - > range ) )
2017-04-01 07:20:54 +00:00
return ;
2017-04-02 17:37:49 +00:00
/// If the subquery or table name for SELECT.
2017-07-14 00:33:37 +00:00
const ASTIdentifier * identifier = typeid_cast < const ASTIdentifier * > ( arg . get ( ) ) ;
if ( typeid_cast < const ASTSubquery * > ( arg . get ( ) ) | | identifier )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
2017-04-01 07:20:54 +00:00
String set_id = arg - > getColumnName ( ) ;
2017-07-14 01:00:29 +00:00
/// A special case is if the name of the table is specified on the right side of the IN statement,
/// and the table has the type Set (a previously prepared set).
2017-04-01 07:20:54 +00:00
if ( identifier )
{
2017-07-14 01:00:29 +00:00
auto database_table = getDatabaseAndTableNameFromIdentifier ( * identifier ) ;
StoragePtr table = context . tryGetTable ( database_table . first , database_table . second ) ;
2017-04-01 07:20:54 +00:00
if ( table )
{
2017-11-04 16:46:14 +00:00
StorageSet * storage_set = dynamic_cast < StorageSet * > ( table . get ( ) ) ;
2017-04-01 07:20:54 +00:00
if ( storage_set )
{
2018-05-16 10:25:10 +00:00
prepared_sets [ arg - > range ] = storage_set - > getSet ( ) ;
2017-04-01 07:20:54 +00:00
return ;
}
}
}
SubqueryForSet & subquery_for_set = subqueries_for_sets [ set_id ] ;
2017-04-02 17:37:49 +00:00
/// If you already created a Set with the same subquery / table.
2017-04-01 07:20:54 +00:00
if ( subquery_for_set . set )
{
2018-05-16 10:25:10 +00:00
prepared_sets [ arg - > range ] = subquery_for_set . set ;
2017-04-01 07:20:54 +00:00
return ;
}
2018-07-02 18:57:14 +00:00
SetPtr set = std : : make_shared < Set > ( SizeLimits ( settings . max_rows_in_set , settings . max_bytes_in_set , settings . set_overflow_mode ) , false ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/** The following happens for GLOBAL INs:
* - in the addExternalStorage function , the IN ( SELECT . . . ) subquery is replaced with IN _data1 ,
* in the subquery_for_set object , this subquery is set as source and the temporary table _data1 as the table .
* - this function shows the expression IN_data1 .
2017-04-01 07:20:54 +00:00
*/
2018-03-15 21:44:09 +00:00
if ( ! subquery_for_set . source & & ( ! storage | | ! storage - > isRemote ( ) ) )
2017-04-01 07:20:54 +00:00
{
auto interpreter = interpretSubquery ( arg , context , subquery_depth , { } ) ;
subquery_for_set . source = std : : make_shared < LazyBlockInputStream > (
2018-02-15 19:48:41 +00:00
interpreter - > getSampleBlock ( ) , [ interpreter ] ( ) mutable { return interpreter - > execute ( ) . in ; } ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/** Why is LazyBlockInputStream used?
2017-04-01 07:20:54 +00:00
*
2018-01-10 00:04:08 +00:00
* The fact is that when processing a query of the form
2017-04-01 07:20:54 +00:00
* SELECT . . . FROM remote_test WHERE column GLOBAL IN ( subquery ) ,
2017-04-02 17:37:49 +00:00
* if the distributed remote_test table contains localhost as one of the servers ,
2018-01-10 00:04:08 +00:00
* the query will be interpreted locally again ( and not sent over TCP , as in the case of a remote server ) .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* The query execution pipeline will be :
2017-04-01 07:20:54 +00:00
* CreatingSets
2017-04-02 17:37:49 +00:00
* subquery execution , filling the temporary table with _data1 ( 1 )
2017-04-01 07:20:54 +00:00
* CreatingSets
2017-04-02 17:37:49 +00:00
* reading from the table _data1 , creating the set ( 2 )
* read from the table subordinate to remote_test .
2017-04-01 07:20:54 +00:00
*
2018-01-10 00:04:08 +00:00
* ( The second part of the pipeline under CreateSets is a reinterpretation of the query inside StorageDistributed ,
2017-04-02 17:37:49 +00:00
* the query differs in that the database name and tables are replaced with subordinates , and the subquery is replaced with _data1 . )
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* But when creating the pipeline , when creating the source ( 2 ) , it will be found that the _data1 table is empty
* ( because the query has not started yet ) , and empty source will be returned as the source .
* And then , when the query is executed , an empty set will be created in step ( 2 ) .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* Therefore , we make the initialization of step ( 2 ) lazy
* - so that it does not occur until step ( 1 ) is completed , on which the table will be populated .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* Note : this solution is not very good , you need to think better .
2017-04-01 07:20:54 +00:00
*/
}
2017-07-14 00:33:37 +00:00
subquery_for_set . set = set ;
2018-05-16 10:25:10 +00:00
prepared_sets [ arg - > range ] = set ;
2017-04-01 07:20:54 +00:00
}
else
{
2017-04-02 17:37:49 +00:00
/// An explicit enumeration of values in parentheses.
2017-04-01 07:20:54 +00:00
makeExplicitSet ( node , sample_block , false ) ;
}
2014-03-31 14:49:43 +00:00
}
2017-04-02 17:37:49 +00:00
/// The case of an explicit enumeration of values.
2017-07-14 00:33:37 +00:00
void ExpressionAnalyzer : : makeExplicitSet ( const ASTFunction * node , const Block & sample_block , bool create_ordered_set )
2014-03-31 14:49:43 +00:00
{
2017-07-14 00:33:37 +00:00
const IAST & args = * node - > arguments ;
2017-04-01 07:20:54 +00:00
if ( args . children . size ( ) ! = 2 )
throw Exception ( " Wrong number of arguments passed to function in " , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2017-07-14 00:33:37 +00:00
const ASTPtr & left_arg = args . children . at ( 0 ) ;
2018-05-14 15:05:45 +00:00
const ASTPtr & right_arg = args . children . at ( 1 ) ;
2018-05-14 15:47:38 +00:00
2018-08-27 17:58:43 +00:00
auto getTupleTypeFromAst = [ this ] ( const ASTPtr & tuple_ast ) - > DataTypePtr
2018-05-14 15:47:38 +00:00
{
2018-08-27 17:58:43 +00:00
auto ast_function = typeid_cast < const ASTFunction * > ( tuple_ast . get ( ) ) ;
2018-05-14 15:47:38 +00:00
if ( ast_function & & ast_function - > name = = " tuple " & & ! ast_function - > arguments - > children . empty ( ) )
{
/// Won't parse all values of outer tuple.
auto element = ast_function - > arguments - > children . at ( 0 ) ;
std : : pair < Field , DataTypePtr > value_raw = evaluateConstantExpression ( element , context ) ;
return std : : make_shared < DataTypeTuple > ( DataTypes ( { value_raw . second } ) ) ;
}
2018-08-27 17:58:43 +00:00
return evaluateConstantExpression ( tuple_ast , context ) . second ;
2018-05-14 15:47:38 +00:00
} ;
2018-05-14 15:05:45 +00:00
const DataTypePtr & left_arg_type = sample_block . getByName ( left_arg - > getColumnName ( ) ) . type ;
2018-05-14 15:47:38 +00:00
const DataTypePtr & right_arg_type = getTupleTypeFromAst ( right_arg ) ;
2017-04-01 07:20:54 +00:00
2018-05-14 15:05:45 +00:00
std : : function < size_t ( const DataTypePtr & ) > getTupleDepth ;
getTupleDepth = [ & getTupleDepth ] ( const DataTypePtr & type ) - > size_t
2017-04-01 07:20:54 +00:00
{
2018-05-14 15:05:45 +00:00
if ( auto tuple_type = typeid_cast < const DataTypeTuple * > ( type . get ( ) ) )
return 1 + ( tuple_type - > getElements ( ) . empty ( ) ? 0 : getTupleDepth ( tuple_type - > getElements ( ) . at ( 0 ) ) ) ;
2017-04-01 07:20:54 +00:00
2018-05-14 15:05:45 +00:00
return 0 ;
} ;
2017-04-01 07:20:54 +00:00
2018-05-14 15:05:45 +00:00
size_t left_tuple_depth = getTupleDepth ( left_arg_type ) ;
2018-05-14 15:47:38 +00:00
size_t right_tuple_depth = getTupleDepth ( right_arg_type ) ;
2018-05-14 15:05:45 +00:00
2018-05-14 15:47:38 +00:00
DataTypes set_element_types = { left_arg_type } ;
auto left_tuple_type = typeid_cast < const DataTypeTuple * > ( left_arg_type . get ( ) ) ;
if ( left_tuple_type & & left_tuple_type - > getElements ( ) . size ( ) ! = 1 )
2018-05-14 15:05:45 +00:00
set_element_types = left_tuple_type - > getElements ( ) ;
2018-05-14 15:47:38 +00:00
ASTPtr elements_ast = nullptr ;
2017-04-01 07:20:54 +00:00
2018-05-14 15:05:45 +00:00
/// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc.
if ( left_tuple_depth = = right_tuple_depth )
2017-04-01 07:20:54 +00:00
{
ASTPtr exp_list = std : : make_shared < ASTExpressionList > ( ) ;
2018-05-14 15:05:45 +00:00
exp_list - > children . push_back ( right_arg ) ;
2017-04-01 07:20:54 +00:00
elements_ast = exp_list ;
}
2018-05-14 15:05:45 +00:00
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
else if ( left_tuple_depth + 1 = = right_tuple_depth )
{
ASTFunction * set_func = typeid_cast < ASTFunction * > ( right_arg . get ( ) ) ;
if ( ! set_func | | set_func - > name ! = " tuple " )
throw Exception ( " Incorrect type of 2nd argument for function " + node - > name
+ " . Must be subquery or set of elements with type " + left_arg_type - > getName ( ) + " . " ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
elements_ast = set_func - > arguments ;
}
else
throw Exception ( " Invalid types for IN function: "
+ left_arg_type - > getName ( ) + " and " + right_arg_type - > getName ( ) + " . " ,
ErrorCodes : : ILLEGAL_TYPE_OF_ARGUMENT ) ;
2017-04-01 07:20:54 +00:00
2018-07-02 18:57:14 +00:00
SetPtr set = std : : make_shared < Set > ( SizeLimits ( settings . max_rows_in_set , settings . max_bytes_in_set , settings . set_overflow_mode ) , create_ordered_set ) ;
set - > createFromAST ( set_element_types , elements_ast , context ) ;
2018-05-16 10:25:10 +00:00
prepared_sets [ right_arg - > range ] = std : : move ( set ) ;
2013-05-24 10:49:19 +00:00
}
2014-07-06 00:19:49 +00:00
static String getUniqueName ( const Block & block , const String & prefix )
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
int i = 1 ;
while ( block . has ( prefix + toString ( i ) ) )
+ + i ;
return prefix + toString ( i ) ;
2013-05-24 10:49:19 +00:00
}
2017-04-02 17:37:49 +00:00
/** For getActionsImpl.
* A stack of ExpressionActions corresponding to nested lambda expressions .
* The new action should be added to the highest possible level .
* For example , in the expression " select arrayMap(x -> x + column1 * column2, array1) "
* calculation of the product must be done outside the lambda expression ( it does not depend on x ) , and the calculation of the sum is inside ( depends on x ) .
2014-06-12 19:53:36 +00:00
*/
2018-04-24 07:16:39 +00:00
ScopeStack : : ScopeStack ( const ExpressionActionsPtr & actions , const Settings & settings_ )
: settings ( settings_ )
2014-06-12 19:53:36 +00:00
{
2018-04-24 07:16:39 +00:00
stack . emplace_back ( ) ;
stack . back ( ) . actions = actions ;
const Block & sample_block = actions - > getSampleBlock ( ) ;
for ( size_t i = 0 , size = sample_block . columns ( ) ; i < size ; + + i )
stack . back ( ) . new_columns . insert ( sample_block . getByPosition ( i ) . name ) ;
}
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
void ScopeStack : : pushLevel ( const NamesAndTypesList & input_columns )
{
stack . emplace_back ( ) ;
Level & prev = stack [ stack . size ( ) - 2 ] ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
ColumnsWithTypeAndName all_columns ;
NameSet new_names ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
for ( NamesAndTypesList : : const_iterator it = input_columns . begin ( ) ; it ! = input_columns . end ( ) ; + + it )
2017-04-01 07:20:54 +00:00
{
2018-04-24 07:16:39 +00:00
all_columns . emplace_back ( nullptr , it - > type , it - > name ) ;
new_names . insert ( it - > name ) ;
stack . back ( ) . new_columns . insert ( it - > name ) ;
2017-04-01 07:20:54 +00:00
}
2018-04-24 07:16:39 +00:00
const Block & prev_sample_block = prev . actions - > getSampleBlock ( ) ;
for ( size_t i = 0 , size = prev_sample_block . columns ( ) ; i < size ; + + i )
2017-04-01 07:20:54 +00:00
{
2018-04-24 07:16:39 +00:00
const ColumnWithTypeAndName & col = prev_sample_block . getByPosition ( i ) ;
if ( ! new_names . count ( col . name ) )
all_columns . push_back ( col ) ;
2017-04-01 07:20:54 +00:00
}
2018-04-24 07:16:39 +00:00
stack . back ( ) . actions = std : : make_shared < ExpressionActions > ( all_columns , settings ) ;
}
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
size_t ScopeStack : : getColumnLevel ( const std : : string & name )
{
for ( int i = static_cast < int > ( stack . size ( ) ) - 1 ; i > = 0 ; - - i )
if ( stack [ i ] . new_columns . count ( name ) )
return i ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
throw Exception ( " Unknown identifier: " + name , ErrorCodes : : UNKNOWN_IDENTIFIER ) ;
}
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
void ScopeStack : : addAction ( const ExpressionAction & action )
{
size_t level = 0 ;
Names required = action . getNeededColumns ( ) ;
for ( size_t i = 0 ; i < required . size ( ) ; + + i )
level = std : : max ( level , getColumnLevel ( required [ i ] ) ) ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
Names added ;
stack [ level ] . actions - > add ( action , added ) ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
stack [ level ] . new_columns . insert ( added . begin ( ) , added . end ( ) ) ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
for ( size_t i = 0 ; i < added . size ( ) ; + + i )
2017-04-01 07:20:54 +00:00
{
2018-04-24 07:16:39 +00:00
const ColumnWithTypeAndName & col = stack [ level ] . actions - > getSampleBlock ( ) . getByName ( added [ i ] ) ;
for ( size_t j = level + 1 ; j < stack . size ( ) ; + + j )
stack [ j ] . actions - > addInput ( col ) ;
2017-04-01 07:20:54 +00:00
}
2018-04-24 07:16:39 +00:00
}
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
ExpressionActionsPtr ScopeStack : : popLevel ( )
{
ExpressionActionsPtr res = stack . back ( ) . actions ;
stack . pop_back ( ) ;
return res ;
}
2014-06-12 19:53:36 +00:00
2018-04-24 07:16:39 +00:00
const Block & ScopeStack : : getSampleBlock ( ) const
{
return stack . back ( ) . actions - > getSampleBlock ( ) ;
}
2014-06-12 19:53:36 +00:00
2017-09-08 03:47:27 +00:00
void ExpressionAnalyzer : : getRootActions ( const ASTPtr & ast , bool no_subqueries , bool only_consts , ExpressionActionsPtr & actions )
2013-06-11 16:21:25 +00:00
{
2017-04-01 07:20:54 +00:00
ScopeStack scopes ( actions , settings ) ;
2018-07-24 12:41:35 +00:00
2018-04-24 07:16:39 +00:00
ProjectionManipulatorPtr projection_manipulator ;
if ( ! isThereArrayJoin ( ast ) & & settings . enable_conditional_computation & & ! only_consts )
projection_manipulator = std : : make_shared < ConditionalTree > ( scopes , context ) ;
else
projection_manipulator = std : : make_shared < DefaultProjectionManipulator > ( scopes ) ;
2018-07-24 12:41:35 +00:00
2018-04-24 07:16:39 +00:00
getActionsImpl ( ast , no_subqueries , only_consts , scopes , projection_manipulator ) ;
2018-07-24 12:41:35 +00:00
2017-04-01 07:20:54 +00:00
actions = scopes . popLevel ( ) ;
2013-06-11 16:21:25 +00:00
}
2013-08-01 13:29:32 +00:00
void ExpressionAnalyzer : : getArrayJoinedColumns ( )
{
2017-04-01 07:20:54 +00:00
if ( select_query & & select_query - > array_join_expression_list ( ) )
{
ASTs & array_join_asts = select_query - > array_join_expression_list ( ) - > children ;
for ( const auto & ast : array_join_asts )
{
const String nested_table_name = ast - > getColumnName ( ) ;
const String nested_table_alias = ast - > getAliasOrColumnName ( ) ;
if ( nested_table_alias = = nested_table_name & & ! typeid_cast < const ASTIdentifier * > ( ast . get ( ) ) )
throw Exception ( " No alias for non-trivial value in ARRAY JOIN: " + nested_table_name , ErrorCodes : : ALIAS_REQUIRED ) ;
if ( array_join_alias_to_name . count ( nested_table_alias ) | | aliases . count ( nested_table_alias ) )
throw Exception ( " Duplicate alias in ARRAY JOIN: " + nested_table_alias , ErrorCodes : : MULTIPLE_EXPRESSIONS_FOR_ALIAS ) ;
array_join_alias_to_name [ nested_table_alias ] = nested_table_name ;
array_join_name_to_alias [ nested_table_name ] = nested_table_alias ;
}
2018-08-27 17:58:43 +00:00
getArrayJoinedColumnsImpl ( query ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column,
/// to get the correct number of rows.
2017-04-01 07:20:54 +00:00
if ( array_join_result_to_source . empty ( ) )
{
ASTPtr expr = select_query - > array_join_expression_list ( ) - > children . at ( 0 ) ;
String source_name = expr - > getColumnName ( ) ;
String result_name = expr - > getAliasOrColumnName ( ) ;
2017-04-02 17:37:49 +00:00
/// This is an array.
2018-02-26 09:05:06 +00:00
if ( ! typeid_cast < ASTIdentifier * > ( expr . get ( ) ) | | findColumn ( source_name , source_columns ) ! = source_columns . end ( ) )
2017-04-01 07:20:54 +00:00
{
array_join_result_to_source [ result_name ] = source_name ;
}
2017-04-02 17:37:49 +00:00
else /// This is a nested table.
2017-04-01 07:20:54 +00:00
{
bool found = false ;
2018-02-26 09:05:06 +00:00
for ( const auto & column_name_type : source_columns )
2017-04-01 07:20:54 +00:00
{
2018-01-02 05:54:34 +00:00
auto splitted = Nested : : splitName ( column_name_type . name ) ;
if ( splitted . first = = source_name & & ! splitted . second . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2018-01-02 05:54:34 +00:00
array_join_result_to_source [ Nested : : concatenateName ( result_name , splitted . second ) ] = column_name_type . name ;
2017-04-01 07:20:54 +00:00
found = true ;
break ;
}
}
if ( ! found )
throw Exception ( " No columns in nested table " + source_name , ErrorCodes : : EMPTY_NESTED_TABLE ) ;
}
}
}
2013-08-01 13:29:32 +00:00
}
2017-04-02 17:37:49 +00:00
/// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that.
2017-09-08 03:47:27 +00:00
void ExpressionAnalyzer : : getArrayJoinedColumnsImpl ( const ASTPtr & ast )
2013-07-26 16:11:31 +00:00
{
2017-04-01 07:20:54 +00:00
if ( typeid_cast < ASTTablesInSelectQuery * > ( ast . get ( ) ) )
return ;
if ( ASTIdentifier * node = typeid_cast < ASTIdentifier * > ( ast . get ( ) ) )
{
if ( node - > kind = = ASTIdentifier : : Column )
{
2018-01-02 05:54:34 +00:00
auto splitted = Nested : : splitName ( node - > name ) ; /// ParsedParams, Key1
2017-04-01 07:20:54 +00:00
if ( array_join_alias_to_name . count ( node - > name ) )
{
2017-04-02 17:37:49 +00:00
/// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1
2017-04-01 07:20:54 +00:00
array_join_result_to_source [ node - > name ] = array_join_alias_to_name [ node - > name ] ; /// K1 -> ParsedParams.Key1
}
2018-01-02 05:54:34 +00:00
else if ( array_join_alias_to_name . count ( splitted . first ) & & ! splitted . second . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP
2017-04-01 07:20:54 +00:00
array_join_result_to_source [ node - > name ] /// PP.Key1 -> ParsedParams.Key1
2018-01-02 05:54:34 +00:00
= Nested : : concatenateName ( array_join_alias_to_name [ splitted . first ] , splitted . second ) ;
2017-04-01 07:20:54 +00:00
}
2017-06-21 18:21:37 +00:00
else if ( array_join_name_to_alias . count ( node - > name ) )
2017-04-01 07:20:54 +00:00
{
2017-06-21 18:21:37 +00:00
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams.Key1 AS PP.Key1.
2017-04-02 17:37:49 +00:00
* That is , the query uses the original array , replicated by itself .
2017-04-01 07:20:54 +00:00
*/
array_join_result_to_source [ /// PP.Key1 -> ParsedParams.Key1
2017-06-21 18:21:37 +00:00
array_join_name_to_alias [ node - > name ] ] = node - > name ;
}
2018-01-02 05:54:34 +00:00
else if ( array_join_name_to_alias . count ( splitted . first ) & & ! splitted . second . empty ( ) )
2017-06-21 18:21:37 +00:00
{
/** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP.
*/
array_join_result_to_source [ /// PP.Key1 -> ParsedParams.Key1
2018-01-02 05:54:34 +00:00
Nested : : concatenateName ( array_join_name_to_alias [ splitted . first ] , splitted . second ) ] = node - > name ;
2017-04-01 07:20:54 +00:00
}
}
}
else
{
for ( auto & child : ast - > children )
if ( ! typeid_cast < const ASTSubquery * > ( child . get ( ) )
& & ! typeid_cast < const ASTSelectQuery * > ( child . get ( ) ) )
getArrayJoinedColumnsImpl ( child ) ;
}
2013-07-26 16:11:31 +00:00
}
2018-05-01 13:31:38 +00:00
bool ExpressionAnalyzer : : isThereArrayJoin ( const ASTPtr & ast )
{
2018-04-24 07:16:39 +00:00
if ( typeid_cast < ASTIdentifier * > ( ast . get ( ) ) )
{
return false ;
}
else if ( ASTFunction * node = typeid_cast < ASTFunction * > ( ast . get ( ) ) )
{
if ( node - > name = = " arrayJoin " )
{
return true ;
}
if ( functionIsInOrGlobalInOperator ( node - > name ) )
{
return isThereArrayJoin ( node - > arguments - > children . at ( 0 ) ) ;
}
if ( node - > name = = " indexHint " )
{
return false ;
}
if ( AggregateFunctionFactory : : instance ( ) . isAggregateFunctionName ( node - > name ) )
{
return false ;
}
for ( auto & child : node - > arguments - > children )
{
2018-05-01 13:31:38 +00:00
if ( isThereArrayJoin ( child ) )
{
2018-04-24 07:16:39 +00:00
return true ;
}
}
return false ;
}
else if ( typeid_cast < ASTLiteral * > ( ast . get ( ) ) )
{
return false ;
}
else
{
for ( auto & child : ast - > children )
{
2018-05-01 13:31:38 +00:00
if ( isThereArrayJoin ( child ) )
{
2018-04-24 07:16:39 +00:00
return true ;
}
}
return false ;
}
}
2013-07-26 16:11:31 +00:00
2018-07-24 12:41:35 +00:00
void ExpressionAnalyzer : : getActionsFromJoinKeys ( const ASTTableJoin & table_join , bool no_subqueries , bool only_consts ,
ExpressionActionsPtr & actions )
{
ScopeStack scopes ( actions , settings ) ;
ProjectionManipulatorPtr projection_manipulator ;
2018-08-27 17:58:43 +00:00
if ( ! isThereArrayJoin ( query ) & & settings . enable_conditional_computation & & ! only_consts )
2018-07-24 12:41:35 +00:00
projection_manipulator = std : : make_shared < ConditionalTree > ( scopes , context ) ;
else
projection_manipulator = std : : make_shared < DefaultProjectionManipulator > ( scopes ) ;
if ( table_join . using_expression_list )
getActionsImpl ( table_join . using_expression_list , no_subqueries , only_consts , scopes , projection_manipulator ) ;
else if ( table_join . on_expression )
{
2018-07-30 13:57:50 +00:00
for ( const auto & ast : analyzed_join . key_asts_left )
2018-07-26 15:11:53 +00:00
getActionsImpl ( ast , no_subqueries , only_consts , scopes , projection_manipulator ) ;
2018-07-24 12:41:35 +00:00
}
actions = scopes . popLevel ( ) ;
}
2018-04-24 07:16:39 +00:00
void ExpressionAnalyzer : : getActionsImpl ( const ASTPtr & ast , bool no_subqueries , bool only_consts , ScopeStack & actions_stack ,
ProjectionManipulatorPtr projection_manipulator )
2013-05-24 10:49:19 +00:00
{
2018-06-27 13:16:12 +00:00
String ast_column_name ;
auto getColumnName = [ & ast , & ast_column_name ] ( )
{
if ( ast_column_name . empty ( ) )
ast_column_name = ast - > getColumnName ( ) ;
return ast_column_name ;
} ;
2017-04-02 17:37:49 +00:00
/// If the result of the calculation already exists in the block.
2017-04-01 07:20:54 +00:00
if ( ( typeid_cast < ASTFunction * > ( ast . get ( ) ) | | typeid_cast < ASTLiteral * > ( ast . get ( ) ) )
2018-06-27 13:16:12 +00:00
& & projection_manipulator - > tryToGetFromUpperProjection ( getColumnName ( ) ) )
2017-04-01 07:20:54 +00:00
return ;
2018-06-27 13:16:12 +00:00
if ( typeid_cast < ASTIdentifier * > ( ast . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2018-06-27 13:16:12 +00:00
if ( ! only_consts & & ! projection_manipulator - > tryToGetFromUpperProjection ( getColumnName ( ) ) )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// The requested column is not in the block.
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
2017-04-01 07:20:54 +00:00
bool found = false ;
2018-02-26 09:05:06 +00:00
for ( const auto & column_name_type : source_columns )
2018-06-27 13:16:12 +00:00
if ( column_name_type . name = = getColumnName ( ) )
2017-04-01 07:20:54 +00:00
found = true ;
if ( found )
2018-06-27 13:16:12 +00:00
throw Exception ( " Column " + getColumnName ( ) + " is not under aggregate function and not in GROUP BY. " ,
2017-04-01 07:20:54 +00:00
ErrorCodes : : NOT_AN_AGGREGATE ) ;
}
}
else if ( ASTFunction * node = typeid_cast < ASTFunction * > ( ast . get ( ) ) )
{
2018-02-24 01:31:42 +00:00
if ( node - > name = = " lambda " )
2017-04-01 07:20:54 +00:00
throw Exception ( " Unexpected lambda expression " , ErrorCodes : : UNEXPECTED_EXPRESSION ) ;
2017-04-02 17:37:49 +00:00
/// Function arrayJoin.
2018-02-24 01:31:42 +00:00
if ( node - > name = = " arrayJoin " )
2017-04-01 07:20:54 +00:00
{
if ( node - > arguments - > children . size ( ) ! = 1 )
throw Exception ( " arrayJoin requires exactly 1 argument " , ErrorCodes : : TYPE_MISMATCH ) ;
ASTPtr arg = node - > arguments - > children . at ( 0 ) ;
2018-04-24 07:16:39 +00:00
getActionsImpl ( arg , no_subqueries , only_consts , actions_stack , projection_manipulator ) ;
2017-04-01 07:20:54 +00:00
if ( ! only_consts )
{
2018-06-27 13:16:12 +00:00
String result_name = projection_manipulator - > getColumnName ( getColumnName ( ) ) ;
2018-04-24 07:16:39 +00:00
actions_stack . addAction ( ExpressionAction : : copyColumn ( projection_manipulator - > getColumnName ( arg - > getColumnName ( ) ) , result_name ) ) ;
2017-04-01 07:20:54 +00:00
NameSet joined_columns ;
joined_columns . insert ( result_name ) ;
actions_stack . addAction ( ExpressionAction : : arrayJoin ( joined_columns , false , context ) ) ;
}
return ;
}
2018-02-24 01:31:42 +00:00
if ( functionIsInOrGlobalInOperator ( node - > name ) )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
if ( ! no_subqueries )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
2018-04-24 07:16:39 +00:00
getActionsImpl ( node - > arguments - > children . at ( 0 ) , no_subqueries , only_consts , actions_stack ,
projection_manipulator ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
/// Transform tuple or subquery into a set.
makeSet ( node , actions_stack . getSampleBlock ( ) ) ;
2017-04-01 07:20:54 +00:00
}
2018-02-24 01:31:42 +00:00
else
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
if ( ! only_consts )
{
/// We are in the part of the tree that we are not going to compute. You just need to define types.
/// Do not subquery and create sets. We insert an arbitrary column of the correct type.
ColumnWithTypeAndName fake_column ;
2018-06-27 13:16:12 +00:00
fake_column . name = projection_manipulator - > getColumnName ( getColumnName ( ) ) ;
2018-02-24 01:31:42 +00:00
fake_column . type = std : : make_shared < DataTypeUInt8 > ( ) ;
2018-07-19 13:36:21 +00:00
fake_column . column = fake_column . type - > createColumn ( ) ;
2018-04-30 19:33:32 +00:00
actions_stack . addAction ( ExpressionAction : : addColumn ( fake_column , projection_manipulator - > getProjectionSourceColumn ( ) , false ) ) ;
2018-04-24 07:16:39 +00:00
getActionsImpl ( node - > arguments - > children . at ( 0 ) , no_subqueries , only_consts , actions_stack ,
projection_manipulator ) ;
2018-02-24 01:31:42 +00:00
}
2017-04-01 07:20:54 +00:00
return ;
}
2018-02-24 01:31:42 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
/// A special function `indexHint`. Everything that is inside it is not calculated
2018-04-20 00:20:36 +00:00
/// (and is used only for index analysis, see KeyCondition).
2018-02-24 01:31:42 +00:00
if ( node - > name = = " indexHint " )
{
actions_stack . addAction ( ExpressionAction : : addColumn ( ColumnWithTypeAndName (
2018-04-24 07:16:39 +00:00
ColumnConst : : create ( ColumnUInt8 : : create ( 1 , 1 ) , 1 ) , std : : make_shared < DataTypeUInt8 > ( ) ,
2018-06-27 13:16:12 +00:00
projection_manipulator - > getColumnName ( getColumnName ( ) ) ) , projection_manipulator - > getProjectionSourceColumn ( ) , false ) ) ;
2018-02-24 01:31:42 +00:00
return ;
}
if ( AggregateFunctionFactory : : instance ( ) . isAggregateFunctionName ( node - > name ) )
return ;
2017-04-01 07:20:54 +00:00
2018-08-29 15:36:24 +00:00
/// Context object that we pass to function should live during query.
const Context & function_context = context . hasQueryContext ( )
? context . getQueryContext ( )
: context ;
const FunctionBuilderPtr & function_builder = FunctionFactory : : instance ( ) . get ( node - > name , function_context ) ;
auto projection_action = getProjectionAction ( node - > name , actions_stack , projection_manipulator , getColumnName ( ) , function_context ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
Names argument_names ;
DataTypes argument_types ;
bool arguments_present = true ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
bool has_lambda_arguments = false ;
2018-05-15 09:32:54 +00:00
for ( size_t arg = 0 ; arg < node - > arguments - > children . size ( ) ; + + arg )
2018-02-24 01:31:42 +00:00
{
2018-05-15 09:32:54 +00:00
auto & child = node - > arguments - > children [ arg ] ;
2018-06-27 13:16:12 +00:00
auto child_column_name = child - > getColumnName ( ) ;
2018-05-15 09:32:54 +00:00
2018-02-24 01:31:42 +00:00
ASTFunction * lambda = typeid_cast < ASTFunction * > ( child . get ( ) ) ;
if ( lambda & & lambda - > name = = " lambda " )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
/// If the argument is a lambda expression, just remember its approximate type.
if ( lambda - > arguments - > children . size ( ) ! = 2 )
throw Exception ( " lambda requires two arguments " , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
ASTFunction * lambda_args_tuple = typeid_cast < ASTFunction * > ( lambda - > arguments - > children . at ( 0 ) . get ( ) ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
if ( ! lambda_args_tuple | | lambda_args_tuple - > name ! = " tuple " )
throw Exception ( " First argument of lambda must be a tuple " , ErrorCodes : : TYPE_MISMATCH ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
has_lambda_arguments = true ;
argument_types . emplace_back ( std : : make_shared < DataTypeFunction > ( DataTypes ( lambda_args_tuple - > arguments - > children . size ( ) ) ) ) ;
/// Select the name in the next cycle.
argument_names . emplace_back ( ) ;
}
2018-05-16 10:25:10 +00:00
else if ( prepared_sets . count ( child - > range ) & & functionIsInOrGlobalInOperator ( node - > name ) & & arg = = 1 )
2018-02-24 01:31:42 +00:00
{
ColumnWithTypeAndName column ;
column . type = std : : make_shared < DataTypeSet > ( ) ;
2017-04-01 07:20:54 +00:00
2018-05-16 10:25:10 +00:00
const SetPtr & set = prepared_sets [ child - > range ] ;
2017-07-14 00:33:37 +00:00
2018-02-24 01:31:42 +00:00
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
2018-04-19 21:34:04 +00:00
/// so that sets with the same literal representation do not fuse together (they can have different types).
2018-02-24 01:31:42 +00:00
if ( ! set - > empty ( ) )
column . name = getUniqueName ( actions_stack . getSampleBlock ( ) , " __set " ) ;
else
2018-06-27 13:16:12 +00:00
column . name = child_column_name ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
column . name = projection_manipulator - > getColumnName ( column . name ) ;
2018-02-24 01:31:42 +00:00
if ( ! actions_stack . getSampleBlock ( ) . has ( column . name ) )
{
column . column = ColumnSet : : create ( 1 , set ) ;
2017-04-01 07:20:54 +00:00
2018-04-30 19:33:32 +00:00
actions_stack . addAction ( ExpressionAction : : addColumn ( column , projection_manipulator - > getProjectionSourceColumn ( ) , false ) ) ;
2018-02-24 01:31:42 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
argument_types . push_back ( column . type ) ;
argument_names . push_back ( column . name ) ;
}
else
{
/// If the argument is not a lambda expression, call it recursively and find out its type.
2018-04-24 07:16:39 +00:00
projection_action - > preArgumentAction ( ) ;
getActionsImpl ( child , no_subqueries , only_consts , actions_stack ,
projection_manipulator ) ;
2018-06-27 13:16:12 +00:00
std : : string name = projection_manipulator - > getColumnName ( child_column_name ) ;
projection_action - > postArgumentAction ( child_column_name ) ;
2018-02-24 01:31:42 +00:00
if ( actions_stack . getSampleBlock ( ) . has ( name ) )
{
argument_types . push_back ( actions_stack . getSampleBlock ( ) . getByName ( name ) . type ) ;
argument_names . push_back ( name ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2018-02-24 01:31:42 +00:00
if ( only_consts )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
arguments_present = false ;
2017-04-01 07:20:54 +00:00
}
else
{
2018-04-24 07:16:39 +00:00
throw Exception ( " Unknown identifier: " + name + " , projection layer " + projection_manipulator - > getProjectionExpression ( ) , ErrorCodes : : UNKNOWN_IDENTIFIER ) ;
2017-04-01 07:20:54 +00:00
}
}
}
2018-02-24 01:31:42 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
if ( only_consts & & ! arguments_present )
return ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
if ( has_lambda_arguments & & ! only_consts )
{
function_builder - > getLambdaArgumentTypes ( argument_types ) ;
/// Call recursively for lambda expressions.
for ( size_t i = 0 ; i < node - > arguments - > children . size ( ) ; + + i )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
ASTPtr child = node - > arguments - > children [ i ] ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
ASTFunction * lambda = typeid_cast < ASTFunction * > ( child . get ( ) ) ;
if ( lambda & & lambda - > name = = " lambda " )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
const DataTypeFunction * lambda_type = typeid_cast < const DataTypeFunction * > ( argument_types [ i ] . get ( ) ) ;
ASTFunction * lambda_args_tuple = typeid_cast < ASTFunction * > ( lambda - > arguments - > children . at ( 0 ) . get ( ) ) ;
ASTs lambda_arg_asts = lambda_args_tuple - > arguments - > children ;
NamesAndTypesList lambda_arguments ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
for ( size_t j = 0 ; j < lambda_arg_asts . size ( ) ; + + j )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
ASTIdentifier * identifier = typeid_cast < ASTIdentifier * > ( lambda_arg_asts [ j ] . get ( ) ) ;
if ( ! identifier )
throw Exception ( " lambda argument declarations must be identifiers " , ErrorCodes : : TYPE_MISMATCH ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
String arg_name = identifier - > name ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
lambda_arguments . emplace_back ( arg_name , lambda_type - > getArgumentTypes ( ) [ j ] ) ;
}
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
projection_action - > preArgumentAction ( ) ;
2018-02-24 01:31:42 +00:00
actions_stack . pushLevel ( lambda_arguments ) ;
2018-04-24 07:16:39 +00:00
getActionsImpl ( lambda - > arguments - > children . at ( 1 ) , no_subqueries , only_consts , actions_stack ,
projection_manipulator ) ;
2018-02-24 01:31:42 +00:00
ExpressionActionsPtr lambda_actions = actions_stack . popLevel ( ) ;
2017-04-01 07:20:54 +00:00
2018-04-24 07:16:39 +00:00
String result_name = projection_manipulator - > getColumnName ( lambda - > arguments - > children . at ( 1 ) - > getColumnName ( ) ) ;
2018-02-24 01:31:42 +00:00
lambda_actions - > finalize ( Names ( 1 , result_name ) ) ;
DataTypePtr result_type = lambda_actions - > getSampleBlock ( ) . getByName ( result_name ) . type ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
Names captured ;
Names required = lambda_actions - > getRequiredColumns ( ) ;
2018-05-15 09:32:54 +00:00
for ( const auto & required_arg : required )
if ( findColumn ( required_arg , lambda_arguments ) = = lambda_arguments . end ( ) )
captured . push_back ( required_arg ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
/// We can not name `getColumnName()`,
/// because it does not uniquely define the expression (the types of arguments can be different).
String lambda_name = getUniqueName ( actions_stack . getSampleBlock ( ) , " __lambda " ) ;
2018-02-06 19:34:53 +00:00
2018-02-24 01:31:42 +00:00
auto function_capture = std : : make_shared < FunctionCapture > (
lambda_actions , captured , lambda_arguments , result_type , result_name ) ;
2018-04-24 07:16:39 +00:00
actions_stack . addAction ( ExpressionAction : : applyFunction ( function_capture , captured , lambda_name ,
2018-04-30 19:33:32 +00:00
projection_manipulator - > getProjectionSourceColumn ( ) ) ) ;
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
argument_types [ i ] = std : : make_shared < DataTypeFunction > ( lambda_type - > getArgumentTypes ( ) , result_type ) ;
argument_names [ i ] = lambda_name ;
2018-04-24 07:16:39 +00:00
projection_action - > postArgumentAction ( lambda_name ) ;
2017-04-01 07:20:54 +00:00
}
}
2018-02-24 01:31:42 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-24 01:31:42 +00:00
if ( only_consts )
{
2018-05-15 09:32:54 +00:00
for ( const auto & argument_name : argument_names )
2017-04-01 07:20:54 +00:00
{
2018-05-15 09:32:54 +00:00
if ( ! actions_stack . getSampleBlock ( ) . has ( argument_name ) )
2017-04-01 07:20:54 +00:00
{
2018-02-24 01:31:42 +00:00
arguments_present = false ;
break ;
2017-04-01 07:20:54 +00:00
}
}
}
2018-02-24 01:31:42 +00:00
if ( arguments_present )
2018-04-24 07:16:39 +00:00
{
projection_action - > preCalculation ( ) ;
if ( projection_action - > isCalculationRequired ( ) )
{
actions_stack . addAction (
ExpressionAction : : applyFunction ( function_builder ,
argument_names ,
2018-06-27 13:16:12 +00:00
projection_manipulator - > getColumnName ( getColumnName ( ) ) ,
2018-04-30 19:33:32 +00:00
projection_manipulator - > getProjectionSourceColumn ( ) ) ) ;
2018-04-24 07:16:39 +00:00
}
}
2017-04-01 07:20:54 +00:00
}
2018-08-27 17:58:43 +00:00
else if ( ASTLiteral * literal = typeid_cast < ASTLiteral * > ( ast . get ( ) ) )
2017-04-01 07:20:54 +00:00
{
2018-08-27 17:58:43 +00:00
DataTypePtr type = applyVisitor ( FieldToDataType ( ) , literal - > value ) ;
2017-04-01 07:20:54 +00:00
ColumnWithTypeAndName column ;
2018-08-27 17:58:43 +00:00
column . column = type - > createColumnConst ( 1 , convertFieldToType ( literal - > value , * type ) ) ;
2017-04-01 07:20:54 +00:00
column . type = type ;
2018-06-27 13:16:12 +00:00
column . name = getColumnName ( ) ;
2017-04-01 07:20:54 +00:00
2018-04-29 13:49:58 +00:00
actions_stack . addAction ( ExpressionAction : : addColumn ( column , " " , false ) ) ;
2018-05-01 13:24:43 +00:00
projection_manipulator - > tryToGetFromUpperProjection ( column . name ) ;
2017-04-01 07:20:54 +00:00
}
else
{
for ( auto & child : ast - > children )
2018-02-24 01:55:43 +00:00
{
/// Do not go to FROM, JOIN, UNION.
if ( ! typeid_cast < const ASTTableExpression * > ( child . get ( ) )
& & ! typeid_cast < const ASTSelectQuery * > ( child . get ( ) ) )
2018-04-24 07:16:39 +00:00
getActionsImpl ( child , no_subqueries , only_consts , actions_stack , projection_manipulator ) ;
2018-02-24 01:55:43 +00:00
}
2017-04-01 07:20:54 +00:00
}
2013-05-24 10:49:19 +00:00
}
2016-03-05 02:30:20 +00:00
void ExpressionAnalyzer : : getAggregates ( const ASTPtr & ast , ExpressionActionsPtr & actions )
2013-05-24 10:49:19 +00:00
{
2017-04-02 17:37:49 +00:00
/// There can not be aggregate functions inside the WHERE and PREWHERE.
2017-04-01 07:20:54 +00:00
if ( select_query & & ( ast . get ( ) = = select_query - > where_expression . get ( ) | | ast . get ( ) = = select_query - > prewhere_expression . get ( ) ) )
{
assertNoAggregates ( ast , " in WHERE or PREWHERE " ) ;
return ;
}
2017-04-02 17:37:49 +00:00
/// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it.
2017-04-01 07:20:54 +00:00
if ( ! select_query )
{
assertNoAggregates ( ast , " in wrong place " ) ;
return ;
}
const ASTFunction * node = typeid_cast < const ASTFunction * > ( ast . get ( ) ) ;
2018-02-24 01:31:42 +00:00
if ( node & & AggregateFunctionFactory : : instance ( ) . isAggregateFunctionName ( node - > name ) )
2017-04-01 07:20:54 +00:00
{
has_aggregation = true ;
AggregateDescription aggregate ;
aggregate . column_name = node - > getColumnName ( ) ;
2017-04-02 17:37:49 +00:00
/// Make unique aggregate functions.
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < aggregate_descriptions . size ( ) ; + + i )
if ( aggregate_descriptions [ i ] . column_name = = aggregate . column_name )
return ;
const ASTs & arguments = node - > arguments - > children ;
aggregate . argument_names . resize ( arguments . size ( ) ) ;
DataTypes types ( arguments . size ( ) ) ;
for ( size_t i = 0 ; i < arguments . size ( ) ; + + i )
{
2017-04-02 17:37:49 +00:00
/// There can not be other aggregate functions within the aggregate functions.
2017-04-01 07:20:54 +00:00
assertNoAggregates ( arguments [ i ] , " inside another aggregate function " ) ;
getRootActions ( arguments [ i ] , true , false , actions ) ;
const std : : string & name = arguments [ i ] - > getColumnName ( ) ;
types [ i ] = actions - > getSampleBlock ( ) . getByName ( name ) . type ;
aggregate . argument_names [ i ] = name ;
}
2017-07-10 23:30:17 +00:00
aggregate . parameters = ( node - > parameters ) ? getAggregateFunctionParametersArray ( node - > parameters ) : Array ( ) ;
aggregate . function = AggregateFunctionFactory : : instance ( ) . get ( node - > name , types , aggregate . parameters ) ;
2017-04-01 07:20:54 +00:00
aggregate_descriptions . push_back ( aggregate ) ;
}
else
{
for ( const auto & child : ast - > children )
if ( ! typeid_cast < const ASTSubquery * > ( child . get ( ) )
& & ! typeid_cast < const ASTSelectQuery * > ( child . get ( ) ) )
getAggregates ( child , actions ) ;
}
2013-05-24 10:49:19 +00:00
}
2016-03-05 02:30:20 +00:00
void ExpressionAnalyzer : : assertNoAggregates ( const ASTPtr & ast , const char * description )
{
2017-04-01 07:20:54 +00:00
const ASTFunction * node = typeid_cast < const ASTFunction * > ( ast . get ( ) ) ;
2016-03-05 02:30:20 +00:00
2018-02-24 01:31:42 +00:00
if ( node & & AggregateFunctionFactory : : instance ( ) . isAggregateFunctionName ( node - > name ) )
2017-04-01 07:20:54 +00:00
throw Exception ( " Aggregate function " + node - > getColumnName ( )
+ " is found " + String ( description ) + " in query " , ErrorCodes : : ILLEGAL_AGGREGATION ) ;
2016-03-05 02:30:20 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & child : ast - > children )
if ( ! typeid_cast < const ASTSubquery * > ( child . get ( ) )
& & ! typeid_cast < const ASTSelectQuery * > ( child . get ( ) ) )
assertNoAggregates ( child , description ) ;
2016-03-05 02:30:20 +00:00
}
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer : : assertSelect ( ) const
2013-05-28 11:54:37 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! select_query )
throw Exception ( " Not a select query " , ErrorCodes : : LOGICAL_ERROR ) ;
2013-05-28 11:54:37 +00:00
}
2013-06-14 16:38:54 +00:00
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer : : assertAggregation ( ) const
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! has_aggregation )
throw Exception ( " No aggregation " , ErrorCodes : : LOGICAL_ERROR ) ;
2013-05-28 11:54:37 +00:00
}
2013-06-14 16:38:54 +00:00
2017-12-25 21:57:29 +00:00
void ExpressionAnalyzer : : initChain ( ExpressionActionsChain & chain , const NamesAndTypesList & columns ) const
2013-05-28 14:24:20 +00:00
{
2017-04-01 07:20:54 +00:00
if ( chain . steps . empty ( ) )
{
chain . settings = settings ;
chain . steps . emplace_back ( std : : make_shared < ExpressionActions > ( columns , settings ) ) ;
}
2013-05-28 14:24:20 +00:00
}
2013-05-28 11:54:37 +00:00
2016-07-23 02:25:09 +00:00
/// "Big" ARRAY JOIN.
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer : : addMultipleArrayJoinAction ( ExpressionActionsPtr & actions ) const
2013-10-17 13:32:32 +00:00
{
2017-04-01 07:20:54 +00:00
NameSet result_columns ;
for ( const auto & result_source : array_join_result_to_source )
{
/// Assign new names to columns, if needed.
if ( result_source . first ! = result_source . second )
actions - > add ( ExpressionAction : : copyColumn ( result_source . second , result_source . first ) ) ;
2015-10-22 20:56:52 +00:00
2017-04-02 17:37:49 +00:00
/// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names.
2017-04-01 07:20:54 +00:00
result_columns . insert ( result_source . first ) ;
}
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
actions - > add ( ExpressionAction : : arrayJoin ( result_columns , select_query - > array_join_is_left ( ) , context ) ) ;
2013-10-17 13:32:32 +00:00
}
2014-03-28 12:13:58 +00:00
bool ExpressionAnalyzer : : appendArrayJoin ( ExpressionActionsChain & chain , bool only_types )
2013-07-26 16:33:05 +00:00
{
2017-04-01 07:20:54 +00:00
assertSelect ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
if ( ! select_query - > array_join_expression_list ( ) )
return false ;
2013-10-17 13:32:32 +00:00
2018-02-26 09:05:06 +00:00
initChain ( chain , source_columns ) ;
2017-04-01 07:20:54 +00:00
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
getRootActions ( select_query - > array_join_expression_list ( ) , only_types , false , step . actions ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
addMultipleArrayJoinAction ( step . actions ) ;
2013-10-21 11:33:25 +00:00
2017-04-01 07:20:54 +00:00
return true ;
2013-07-26 16:33:05 +00:00
}
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer : : addJoinAction ( ExpressionActionsPtr & actions , bool only_types ) const
2014-06-13 02:05:05 +00:00
{
2017-04-01 07:20:54 +00:00
if ( only_types )
2018-07-30 13:57:50 +00:00
actions - > add ( ExpressionAction : : ordinaryJoin ( nullptr , analyzed_join . key_names_left ,
2018-07-31 11:31:18 +00:00
analyzed_join . getColumnsAddedByJoin ( ) ) ) ;
2017-04-01 07:20:54 +00:00
else
for ( auto & subquery_for_set : subqueries_for_sets )
if ( subquery_for_set . second . join )
2018-07-30 13:57:50 +00:00
actions - > add ( ExpressionAction : : ordinaryJoin ( subquery_for_set . second . join , analyzed_join . key_names_left ,
2018-07-31 11:31:18 +00:00
analyzed_join . getColumnsAddedByJoin ( ) ) ) ;
2014-06-13 02:05:05 +00:00
}
2018-07-30 13:57:50 +00:00
2018-08-27 17:58:43 +00:00
void ExpressionAnalyzer : : AnalyzedJoin : : createJoinedBlockActions ( const ASTSelectQuery * select_query_with_join ,
const Context & query_context )
2018-07-30 13:57:50 +00:00
{
2018-08-27 17:58:43 +00:00
if ( ! select_query_with_join )
2018-07-31 11:31:18 +00:00
return ;
2018-08-27 17:58:43 +00:00
const ASTTablesInSelectQueryElement * join = select_query_with_join - > join ( ) ;
2018-07-31 11:31:18 +00:00
if ( ! join )
return ;
const auto & join_params = static_cast < const ASTTableJoin & > ( * join - > table_join ) ;
2018-07-30 13:57:50 +00:00
/// Create custom expression list with join keys from right table.
auto expression_list = std : : make_shared < ASTExpressionList > ( ) ;
ASTs & children = expression_list - > children ;
if ( join_params . on_expression )
for ( const auto & join_right_key : key_asts_right )
children . emplace_back ( join_right_key ) ;
2018-07-31 11:31:18 +00:00
NameSet required_columns_set ( key_names_right . begin ( ) , key_names_right . end ( ) ) ;
for ( const auto & joined_column : columns_added_by_join )
required_columns_set . insert ( joined_column . original_name ) ;
required_columns_set . insert ( key_names_right . begin ( ) , key_names_right . end ( ) ) ;
required_columns_from_joined_table . insert ( required_columns_from_joined_table . end ( ) ,
required_columns_set . begin ( ) , required_columns_set . end ( ) ) ;
2018-07-30 13:57:50 +00:00
2018-08-27 17:58:43 +00:00
const auto & source_columns_name = getColumnsFromJoinedTable ( query_context , select_query_with_join ) ;
ExpressionAnalyzer analyzer ( expression_list , query_context , nullptr , source_columns_name , required_columns_from_joined_table ) ;
2018-07-30 13:57:50 +00:00
joined_block_actions = analyzer . getActions ( false ) ;
2018-07-31 11:31:18 +00:00
for ( const auto & column_required_from_actions : joined_block_actions - > getRequiredColumns ( ) )
if ( ! required_columns_set . count ( column_required_from_actions ) )
required_columns_from_joined_table . push_back ( column_required_from_actions ) ;
}
NamesAndTypesList ExpressionAnalyzer : : AnalyzedJoin : : getColumnsAddedByJoin ( ) const
{
NamesAndTypesList result ;
for ( const auto & joined_column : columns_added_by_join )
result . push_back ( joined_column . name_and_type ) ;
return result ;
2014-06-13 02:05:05 +00:00
}
2018-08-27 17:58:43 +00:00
NamesAndTypesList ExpressionAnalyzer : : AnalyzedJoin : : getColumnsFromJoinedTable ( const Context & query_context , const ASTSelectQuery * select_query_with_join )
2018-08-04 10:53:50 +00:00
{
2018-08-27 17:58:43 +00:00
if ( select_query_with_join & & ! columns_from_joined_table . size ( ) )
2018-08-04 10:53:50 +00:00
{
2018-08-27 17:58:43 +00:00
if ( const ASTTablesInSelectQueryElement * node = select_query_with_join - > join ( ) )
2018-08-04 10:53:50 +00:00
{
Block nested_result_sample ;
const auto & table_expression = static_cast < const ASTTableExpression & > ( * node - > table_expression ) ;
if ( table_expression . subquery )
{
const auto & subquery = table_expression . subquery - > children . at ( 0 ) ;
2018-08-27 17:58:43 +00:00
nested_result_sample = InterpreterSelectWithUnionQuery : : getSampleBlock ( subquery , query_context ) ;
2018-08-04 10:53:50 +00:00
}
2018-08-14 09:44:56 +00:00
else if ( table_expression . table_function )
{
const auto table_function = table_expression . table_function ;
2018-08-24 07:40:09 +00:00
auto query_context = const_cast < Context * > ( & context . getQueryContext ( ) ) ;
const auto & join_storage = query_context - > executeTableFunction ( table_function ) ;
2018-08-14 09:44:56 +00:00
nested_result_sample = join_storage - > getSampleBlockNonMaterialized ( ) ;
}
2018-08-04 10:53:50 +00:00
else if ( table_expression . database_and_table_name )
{
const auto & identifier = static_cast < const ASTIdentifier & > ( * table_expression . database_and_table_name ) ;
auto database_table = getDatabaseAndTableNameFromIdentifier ( identifier ) ;
2018-08-27 17:58:43 +00:00
const auto & table = query_context . getTable ( database_table . first , database_table . second ) ;
2018-08-04 10:53:50 +00:00
nested_result_sample = table - > getSampleBlockNonMaterialized ( ) ;
}
columns_from_joined_table = nested_result_sample . getNamesAndTypesList ( ) ;
}
}
return columns_from_joined_table ;
}
2018-07-31 11:31:18 +00:00
2014-06-13 02:05:05 +00:00
bool ExpressionAnalyzer : : appendJoin ( ExpressionActionsChain & chain , bool only_types )
{
2017-04-01 07:20:54 +00:00
assertSelect ( ) ;
if ( ! select_query - > join ( ) )
return false ;
2018-02-26 09:05:06 +00:00
initChain ( chain , source_columns ) ;
2017-04-01 07:20:54 +00:00
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2018-07-26 15:11:53 +00:00
const auto & join_element = static_cast < const ASTTablesInSelectQueryElement & > ( * select_query - > join ( ) ) ;
2018-08-28 13:57:31 +00:00
auto & join_params = static_cast < ASTTableJoin & > ( * join_element . table_join ) ;
2018-08-29 13:51:07 +00:00
if ( join_params . strictness = = ASTTableJoin : : Strictness : : Unspecified & & join_params . kind ! = ASTTableJoin : : Kind : : Cross )
2018-08-28 13:57:31 +00:00
{
if ( settings . join_default_strictness . toString ( ) = = " ANY " )
join_params . strictness = ASTTableJoin : : Strictness : : Any ;
else if ( settings . join_default_strictness . toString ( ) = = " ALL " )
join_params . strictness = ASTTableJoin : : Strictness : : All ;
else
2018-08-29 15:15:42 +00:00
throw Exception ( " Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty " , DB::ErrorCodes::EXPECTED_ALL_OR_ANY) ;
2018-08-28 13:57:31 +00:00
}
2018-07-26 15:11:53 +00:00
const auto & table_to_join = static_cast < const ASTTableExpression & > ( * join_element . table_expression ) ;
2017-04-01 07:20:54 +00:00
2018-07-24 12:41:35 +00:00
getActionsFromJoinKeys ( join_params , only_types , false , step . actions ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Two JOINs are not supported with the same subquery, but different USINGs.
2018-02-23 08:05:21 +00:00
auto join_hash = join_element . getTreeHash ( ) ;
2017-04-01 07:20:54 +00:00
2018-02-23 08:05:21 +00:00
SubqueryForSet & subquery_for_set = subqueries_for_sets [ toString ( join_hash . first ) + " _ " + toString ( join_hash . second ) ] ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Special case - if table name is specified on the right of JOIN, then the table has the type Join (the previously prepared mapping).
/// TODO This syntax does not support specifying a database name.
2017-04-01 07:20:54 +00:00
if ( table_to_join . database_and_table_name )
{
2018-07-27 20:01:54 +00:00
const auto & identifier = static_cast < const ASTIdentifier & > ( * table_to_join . database_and_table_name ) ;
auto database_table = getDatabaseAndTableNameFromIdentifier ( identifier ) ;
2017-07-14 01:00:29 +00:00
StoragePtr table = context . tryGetTable ( database_table . first , database_table . second ) ;
2017-04-01 07:20:54 +00:00
if ( table )
{
2017-11-04 16:46:14 +00:00
StorageJoin * storage_join = dynamic_cast < StorageJoin * > ( table . get ( ) ) ;
2017-04-01 07:20:54 +00:00
if ( storage_join )
{
storage_join - > assertCompatible ( join_params . kind , join_params . strictness ) ;
2017-04-02 17:37:49 +00:00
/// TODO Check the set of keys.
2017-04-01 07:20:54 +00:00
JoinPtr & join = storage_join - > getJoin ( ) ;
subquery_for_set . join = join ;
}
}
}
if ( ! subquery_for_set . join )
{
JoinPtr join = std : : make_shared < Join > (
2018-07-30 13:57:50 +00:00
analyzed_join . key_names_left , analyzed_join . key_names_right , analyzed_join . columns_added_by_join_from_right_keys ,
2018-03-11 00:15:26 +00:00
settings . join_use_nulls , SizeLimits ( settings . max_rows_in_join , settings . max_bytes_in_join , settings . join_overflow_mode ) ,
2017-04-01 07:20:54 +00:00
join_params . kind , join_params . strictness ) ;
2017-04-02 17:37:49 +00:00
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
* - in the addExternalStorage function , the JOIN ( SELECT . . . ) subquery is replaced with JOIN _data1 ,
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the ` table ` .
* - this function shows the expression JOIN _data1 .
2017-04-01 07:20:54 +00:00
*/
if ( ! subquery_for_set . source )
{
ASTPtr table ;
2018-07-30 13:57:50 +00:00
2018-08-14 09:44:56 +00:00
if ( table_to_join . subquery )
2017-04-01 07:20:54 +00:00
table = table_to_join . subquery ;
2018-08-14 09:44:56 +00:00
else if ( table_to_join . table_function )
table = table_to_join . table_function ;
else if ( table_to_join . database_and_table_name )
table = table_to_join . database_and_table_name ;
2017-04-01 07:20:54 +00:00
2018-07-31 11:31:18 +00:00
auto interpreter = interpretSubquery ( table , context , subquery_depth , analyzed_join . required_columns_from_joined_table ) ;
2018-02-15 19:48:41 +00:00
subquery_for_set . source = std : : make_shared < LazyBlockInputStream > (
interpreter - > getSampleBlock ( ) ,
[ interpreter ] ( ) mutable { return interpreter - > execute ( ) . in ; } ) ;
2017-04-01 07:20:54 +00:00
}
2018-07-31 11:31:18 +00:00
/// Alias duplicating columns.
for ( const auto & joined_column : analyzed_join . columns_added_by_join )
{
const auto & qualified_name = joined_column . name_and_type . name ;
if ( joined_column . original_name ! = qualified_name )
subquery_for_set . joined_block_aliases . emplace_back ( joined_column . original_name , qualified_name ) ;
}
2018-07-30 13:57:50 +00:00
auto sample_block = subquery_for_set . source - > getHeader ( ) ;
analyzed_join . joined_block_actions - > execute ( sample_block ) ;
2018-07-31 11:31:18 +00:00
for ( const auto & name_with_alias : subquery_for_set . joined_block_aliases )
{
if ( sample_block . has ( name_with_alias . first ) )
{
auto pos = sample_block . getPositionByName ( name_with_alias . first ) ;
auto column = sample_block . getByPosition ( pos ) ;
sample_block . erase ( pos ) ;
column . name = name_with_alias . second ;
sample_block . insert ( std : : move ( column ) ) ;
}
}
2017-04-02 17:37:49 +00:00
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
2017-04-01 07:20:54 +00:00
subquery_for_set . join = join ;
2018-07-30 13:57:50 +00:00
subquery_for_set . join - > setSampleBlock ( sample_block ) ;
subquery_for_set . joined_block_actions = analyzed_join . joined_block_actions ;
2017-04-01 07:20:54 +00:00
}
addJoinAction ( step . actions , false ) ;
return true ;
2014-06-13 02:05:05 +00:00
}
2018-06-29 11:42:44 +00:00
bool ExpressionAnalyzer : : appendPrewhere ( ExpressionActionsChain & chain , bool only_types )
2018-04-06 13:58:06 +00:00
{
assertSelect ( ) ;
if ( ! select_query - > prewhere_expression )
return false ;
initChain ( chain , source_columns ) ;
2018-04-12 09:45:24 +00:00
auto & step = chain . getLastStep ( ) ;
2018-04-06 13:58:06 +00:00
getRootActions ( select_query - > prewhere_expression , only_types , false , step . actions ) ;
2018-04-12 09:45:24 +00:00
String prewhere_column_name = select_query - > prewhere_expression - > getColumnName ( ) ;
step . required_output . push_back ( prewhere_column_name ) ;
2018-06-29 11:42:44 +00:00
step . can_remove_required_output . push_back ( true ) ;
2018-04-12 09:45:24 +00:00
{
/// Remove unused source_columns from prewhere actions.
auto tmp_actions = std : : make_shared < ExpressionActions > ( source_columns , settings ) ;
getRootActions ( select_query - > prewhere_expression , only_types , false , tmp_actions ) ;
tmp_actions - > finalize ( { prewhere_column_name } ) ;
auto required_columns = tmp_actions - > getRequiredColumns ( ) ;
NameSet required_source_columns ( required_columns . begin ( ) , required_columns . end ( ) ) ;
auto names = step . actions - > getSampleBlock ( ) . getNames ( ) ;
NameSet name_set ( names . begin ( ) , names . end ( ) ) ;
for ( const auto & column : source_columns )
if ( required_source_columns . count ( column . name ) = = 0 )
name_set . erase ( column . name ) ;
Names required_output ( name_set . begin ( ) , name_set . end ( ) ) ;
step . actions - > finalize ( required_output ) ;
}
{
/// Add empty action with input = {prewhere actions output} + {unused source columns}
/// Reasons:
/// 1. Remove remove source columns which are used only in prewhere actions during prewhere actions execution.
/// Example: select A prewhere B > 0. B can be removed at prewhere step.
/// 2. Store side columns which were calculated during prewhere actions execution if they are used.
/// Example: select F(A) prewhere F(A) > 0. F(A) can be saved from prewhere step.
2018-06-25 13:08:35 +00:00
/// 3. Check if we can remove filter column at prewhere step. If we can, action will store single REMOVE_COLUMN.
2018-04-12 09:45:24 +00:00
ColumnsWithTypeAndName columns = step . actions - > getSampleBlock ( ) . getColumnsWithTypeAndName ( ) ;
auto required_columns = step . actions - > getRequiredColumns ( ) ;
NameSet prewhere_input_names ( required_columns . begin ( ) , required_columns . end ( ) ) ;
NameSet unused_source_columns ;
for ( const auto & column : source_columns )
{
if ( prewhere_input_names . count ( column . name ) = = 0 )
{
columns . emplace_back ( column . type , column . name ) ;
unused_source_columns . emplace ( column . name ) ;
}
}
chain . steps . emplace_back ( std : : make_shared < ExpressionActions > ( std : : move ( columns ) , settings ) ) ;
chain . steps . back ( ) . additional_input = std : : move ( unused_source_columns ) ;
}
2018-04-06 13:58:06 +00:00
return true ;
}
2016-07-22 20:39:28 +00:00
2014-03-28 12:13:58 +00:00
bool ExpressionAnalyzer : : appendWhere ( ExpressionActionsChain & chain , bool only_types )
2013-05-28 11:54:37 +00:00
{
2017-04-01 07:20:54 +00:00
assertSelect ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
if ( ! select_query - > where_expression )
return false ;
2013-10-17 13:32:32 +00:00
2018-02-26 09:05:06 +00:00
initChain ( chain , source_columns ) ;
2017-04-01 07:20:54 +00:00
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
step . required_output . push_back ( select_query - > where_expression - > getColumnName ( ) ) ;
2018-06-29 11:42:44 +00:00
step . can_remove_required_output = { true } ;
2018-04-23 19:05:46 +00:00
2017-04-01 07:20:54 +00:00
getRootActions ( select_query - > where_expression , only_types , false , step . actions ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
return true ;
2013-05-28 11:54:37 +00:00
}
2014-03-28 12:13:58 +00:00
bool ExpressionAnalyzer : : appendGroupBy ( ExpressionActionsChain & chain , bool only_types )
2013-05-28 11:54:37 +00:00
{
2017-04-01 07:20:54 +00:00
assertAggregation ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
if ( ! select_query - > group_expression_list )
return false ;
2013-10-17 13:32:32 +00:00
2018-02-26 09:05:06 +00:00
initChain ( chain , source_columns ) ;
2017-04-01 07:20:54 +00:00
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
ASTs asts = select_query - > group_expression_list - > children ;
for ( size_t i = 0 ; i < asts . size ( ) ; + + i )
{
step . required_output . push_back ( asts [ i ] - > getColumnName ( ) ) ;
getRootActions ( asts [ i ] , only_types , false , step . actions ) ;
}
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
return true ;
2013-05-28 11:54:37 +00:00
}
2014-03-28 12:13:58 +00:00
void ExpressionAnalyzer : : appendAggregateFunctionsArguments ( ExpressionActionsChain & chain , bool only_types )
2013-05-28 11:54:37 +00:00
{
2017-04-01 07:20:54 +00:00
assertAggregation ( ) ;
2013-10-17 13:32:32 +00:00
2018-02-26 09:05:06 +00:00
initChain ( chain , source_columns ) ;
2017-04-01 07:20:54 +00:00
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < aggregate_descriptions . size ( ) ; + + i )
{
for ( size_t j = 0 ; j < aggregate_descriptions [ i ] . argument_names . size ( ) ; + + j )
{
step . required_output . push_back ( aggregate_descriptions [ i ] . argument_names [ j ] ) ;
}
}
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
getActionsBeforeAggregation ( select_query - > select_expression_list , step . actions , only_types ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
if ( select_query - > having_expression )
getActionsBeforeAggregation ( select_query - > having_expression , step . actions , only_types ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
if ( select_query - > order_expression_list )
getActionsBeforeAggregation ( select_query - > order_expression_list , step . actions , only_types ) ;
2013-05-28 11:54:37 +00:00
}
2014-03-28 12:13:58 +00:00
bool ExpressionAnalyzer : : appendHaving ( ExpressionActionsChain & chain , bool only_types )
2013-05-28 11:54:37 +00:00
{
2017-04-01 07:20:54 +00:00
assertAggregation ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
if ( ! select_query - > having_expression )
return false ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
initChain ( chain , aggregated_columns ) ;
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
step . required_output . push_back ( select_query - > having_expression - > getColumnName ( ) ) ;
getRootActions ( select_query - > having_expression , only_types , false , step . actions ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
return true ;
2013-05-24 10:49:19 +00:00
}
2014-03-28 12:13:58 +00:00
void ExpressionAnalyzer : : appendSelect ( ExpressionActionsChain & chain , bool only_types )
2013-05-28 11:54:37 +00:00
{
2017-04-01 07:20:54 +00:00
assertSelect ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
initChain ( chain , aggregated_columns ) ;
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
getRootActions ( select_query - > select_expression_list , only_types , false , step . actions ) ;
2013-10-17 13:32:32 +00:00
2018-02-26 09:55:14 +00:00
for ( const auto & child : select_query - > select_expression_list - > children )
2018-03-01 01:25:06 +00:00
step . required_output . push_back ( child - > getColumnName ( ) ) ;
2013-05-28 11:54:37 +00:00
}
2013-05-24 10:49:19 +00:00
2014-03-28 12:13:58 +00:00
bool ExpressionAnalyzer : : appendOrderBy ( ExpressionActionsChain & chain , bool only_types )
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
assertSelect ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
if ( ! select_query - > order_expression_list )
return false ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
initChain ( chain , aggregated_columns ) ;
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
getRootActions ( select_query - > order_expression_list , only_types , false , step . actions ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
ASTs asts = select_query - > order_expression_list - > children ;
for ( size_t i = 0 ; i < asts . size ( ) ; + + i )
{
ASTOrderByElement * ast = typeid_cast < ASTOrderByElement * > ( asts [ i ] . get ( ) ) ;
if ( ! ast | | ast - > children . size ( ) < 1 )
throw Exception ( " Bad order expression AST " , ErrorCodes : : UNKNOWN_TYPE_OF_AST_NODE ) ;
ASTPtr order_expression = ast - > children . at ( 0 ) ;
step . required_output . push_back ( order_expression - > getColumnName ( ) ) ;
}
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
return true ;
2013-05-28 11:54:37 +00:00
}
2018-03-01 05:24:56 +00:00
bool ExpressionAnalyzer : : appendLimitBy ( ExpressionActionsChain & chain , bool only_types )
{
assertSelect ( ) ;
if ( ! select_query - > limit_by_expression_list )
return false ;
initChain ( chain , aggregated_columns ) ;
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
getRootActions ( select_query - > limit_by_expression_list , only_types , false , step . actions ) ;
for ( const auto & child : select_query - > limit_by_expression_list - > children )
step . required_output . push_back ( child - > getColumnName ( ) ) ;
return true ;
}
2017-12-01 21:13:25 +00:00
void ExpressionAnalyzer : : appendProjectResult ( ExpressionActionsChain & chain ) const
2013-05-28 11:54:37 +00:00
{
2017-04-01 07:20:54 +00:00
assertSelect ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
initChain ( chain , aggregated_columns ) ;
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
NamesWithAliases result_columns ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
ASTs asts = select_query - > select_expression_list - > children ;
for ( size_t i = 0 ; i < asts . size ( ) ; + + i )
{
2018-02-26 09:05:06 +00:00
String result_name = asts [ i ] - > getAliasOrColumnName ( ) ;
2018-08-05 03:28:31 +00:00
if ( required_result_columns . empty ( )
| | std : : find ( required_result_columns . begin ( ) , required_result_columns . end ( ) , result_name ) ! = required_result_columns . end ( ) )
2018-02-26 09:05:06 +00:00
{
result_columns . emplace_back ( asts [ i ] - > getColumnName ( ) , result_name ) ;
step . required_output . push_back ( result_columns . back ( ) . second ) ;
}
2017-04-01 07:20:54 +00:00
}
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
step . actions - > add ( ExpressionAction : : project ( result_columns ) ) ;
2013-05-24 10:49:19 +00:00
}
2017-09-08 03:47:27 +00:00
void ExpressionAnalyzer : : getActionsBeforeAggregation ( const ASTPtr & ast , ExpressionActionsPtr & actions , bool no_subqueries )
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
ASTFunction * node = typeid_cast < ASTFunction * > ( ast . get ( ) ) ;
2013-10-17 13:32:32 +00:00
2018-02-24 01:31:42 +00:00
if ( node & & AggregateFunctionFactory : : instance ( ) . isAggregateFunctionName ( node - > name ) )
2017-04-01 07:20:54 +00:00
for ( auto & argument : node - > arguments - > children )
getRootActions ( argument , no_subqueries , false , actions ) ;
else
for ( auto & child : ast - > children )
getActionsBeforeAggregation ( child , actions , no_subqueries ) ;
2013-05-29 11:46:51 +00:00
}
2013-06-03 13:17:17 +00:00
ExpressionActionsPtr ExpressionAnalyzer : : getActions ( bool project_result )
2013-05-24 10:49:19 +00:00
{
2018-02-26 09:05:06 +00:00
ExpressionActionsPtr actions = std : : make_shared < ExpressionActions > ( source_columns , settings ) ;
2017-04-01 07:20:54 +00:00
NamesWithAliases result_columns ;
Names result_names ;
ASTs asts ;
2018-08-27 17:58:43 +00:00
if ( auto node = typeid_cast < const ASTExpressionList * > ( query . get ( ) ) )
2017-04-01 07:20:54 +00:00
asts = node - > children ;
else
2018-08-27 17:58:43 +00:00
asts = ASTs ( 1 , query ) ;
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < asts . size ( ) ; + + i )
{
std : : string name = asts [ i ] - > getColumnName ( ) ;
std : : string alias ;
if ( project_result )
alias = asts [ i ] - > getAliasOrColumnName ( ) ;
else
alias = name ;
result_columns . emplace_back ( name , alias ) ;
result_names . push_back ( alias ) ;
getRootActions ( asts [ i ] , false , false , actions ) ;
}
if ( project_result )
{
actions - > add ( ExpressionAction : : project ( result_columns ) ) ;
}
else
{
2017-04-02 17:37:49 +00:00
/// We will not delete the original columns.
2018-02-26 09:05:06 +00:00
for ( const auto & column_name_type : source_columns )
2017-04-01 07:20:54 +00:00
result_names . push_back ( column_name_type . name ) ;
}
actions - > finalize ( result_names ) ;
return actions ;
2013-05-24 10:49:19 +00:00
}
ExpressionActionsPtr ExpressionAnalyzer : : getConstActions ( )
{
2017-12-25 21:57:29 +00:00
ExpressionActionsPtr actions = std : : make_shared < ExpressionActions > ( NamesAndTypesList ( ) , settings ) ;
2013-10-17 13:32:32 +00:00
2018-08-27 17:58:43 +00:00
getRootActions ( query , true , true , actions ) ;
2013-10-17 13:32:32 +00:00
2017-04-01 07:20:54 +00:00
return actions ;
2013-05-24 10:49:19 +00:00
}
2015-04-18 22:30:43 +00:00
void ExpressionAnalyzer : : getAggregateInfo ( Names & key_names , AggregateDescriptions & aggregates ) const
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
for ( const auto & name_and_type : aggregation_keys )
key_names . emplace_back ( name_and_type . name ) ;
2015-03-12 02:22:55 +00:00
2017-04-01 07:20:54 +00:00
aggregates = aggregate_descriptions ;
2013-05-24 10:49:19 +00:00
}
2014-07-04 19:13:38 +00:00
void ExpressionAnalyzer : : collectUsedColumns ( )
2013-05-30 16:52:21 +00:00
{
2017-04-02 17:37:49 +00:00
/** Calculate which columns are required to execute the expression.
* Then , delete all other columns from the list of available columns .
* After execution , columns will only contain the list of columns needed to read from the table .
2017-04-01 07:20:54 +00:00
*/
NameSet required ;
NameSet ignored ;
2017-12-14 12:21:01 +00:00
NameSet available_columns ;
2018-02-26 09:05:06 +00:00
for ( const auto & column : source_columns )
2017-12-14 12:21:01 +00:00
available_columns . insert ( column . name ) ;
2017-04-01 07:20:54 +00:00
if ( select_query & & select_query - > array_join_expression_list ( ) )
{
ASTs & expressions = select_query - > array_join_expression_list ( ) - > children ;
for ( size_t i = 0 ; i < expressions . size ( ) ; + + i )
{
2017-04-02 17:37:49 +00:00
/// Ignore the top-level identifiers from the ARRAY JOIN section.
/// Then add them separately.
2017-04-01 07:20:54 +00:00
if ( typeid_cast < ASTIdentifier * > ( expressions [ i ] . get ( ) ) )
{
ignored . insert ( expressions [ i ] - > getColumnName ( ) ) ;
}
else
{
2017-04-02 17:37:49 +00:00
/// Nothing needs to be ignored for expressions in ARRAY JOIN.
2017-04-01 07:20:54 +00:00
NameSet empty ;
2018-02-26 09:05:06 +00:00
getRequiredSourceColumnsImpl ( expressions [ i ] , available_columns , required , empty , empty , empty ) ;
2017-04-01 07:20:54 +00:00
}
ignored . insert ( expressions [ i ] - > getAliasOrColumnName ( ) ) ;
}
}
2017-04-02 17:37:49 +00:00
/** You also need to ignore the identifiers of the columns that are obtained by JOIN.
* ( Do not assume that they are required for reading from the " left " table ) .
2017-04-01 07:20:54 +00:00
*/
NameSet available_joined_columns ;
2018-07-30 13:57:50 +00:00
collectJoinedColumns ( available_joined_columns ) ;
2017-04-01 07:20:54 +00:00
NameSet required_joined_columns ;
2018-07-25 16:11:20 +00:00
2018-07-30 13:57:50 +00:00
for ( const auto & left_key_ast : analyzed_join . key_asts_left )
2018-07-25 16:11:20 +00:00
getRequiredSourceColumnsImpl ( left_key_ast , available_columns , required , ignored , { } , required_joined_columns ) ;
2018-08-27 18:05:28 +00:00
getRequiredSourceColumnsImpl ( query , available_columns , required , ignored , available_joined_columns , required_joined_columns ) ;
2017-04-01 07:20:54 +00:00
2018-07-30 13:57:50 +00:00
for ( auto it = analyzed_join . columns_added_by_join . begin ( ) ; it ! = analyzed_join . columns_added_by_join . end ( ) ; )
2017-12-25 21:57:19 +00:00
{
2018-07-31 11:31:18 +00:00
if ( required_joined_columns . count ( it - > name_and_type . name ) )
2017-12-25 21:57:19 +00:00
+ + it ;
else
2018-07-30 13:57:50 +00:00
analyzed_join . columns_added_by_join . erase ( it + + ) ;
2017-12-25 21:57:19 +00:00
}
2017-04-01 07:20:54 +00:00
2018-07-31 11:31:18 +00:00
analyzed_join . createJoinedBlockActions ( select_query , context ) ;
2018-07-30 13:57:50 +00:00
/// Some columns from right join key may be used in query. This columns will be appended to block during join.
for ( const auto & right_key_name : analyzed_join . key_names_right )
2018-07-27 20:01:54 +00:00
if ( required_joined_columns . count ( right_key_name ) )
2018-07-30 13:57:50 +00:00
analyzed_join . columns_added_by_join_from_right_keys . insert ( right_key_name ) ;
2018-07-27 20:01:54 +00:00
2017-04-02 17:37:49 +00:00
/// Insert the columns required for the ARRAY JOIN calculation into the required columns list.
2017-04-01 07:20:54 +00:00
NameSet array_join_sources ;
for ( const auto & result_source : array_join_result_to_source )
array_join_sources . insert ( result_source . second ) ;
2018-02-26 09:05:06 +00:00
for ( const auto & column_name_type : source_columns )
2017-04-01 07:20:54 +00:00
if ( array_join_sources . count ( column_name_type . name ) )
required . insert ( column_name_type . name ) ;
2017-04-02 17:37:49 +00:00
/// You need to read at least one column to find the number of rows.
2018-02-22 10:54:28 +00:00
if ( select_query & & required . empty ( ) )
2018-02-26 09:05:06 +00:00
required . insert ( ExpressionActions : : getSmallestColumn ( source_columns ) ) ;
2017-04-01 07:20:54 +00:00
2018-02-28 01:29:55 +00:00
NameSet unknown_required_source_columns = required ;
2017-04-01 07:20:54 +00:00
2018-02-26 09:05:06 +00:00
for ( NamesAndTypesList : : iterator it = source_columns . begin ( ) ; it ! = source_columns . end ( ) ; )
2017-04-01 07:20:54 +00:00
{
2018-02-26 09:05:06 +00:00
unknown_required_source_columns . erase ( it - > name ) ;
2017-12-25 21:57:19 +00:00
if ( ! required . count ( it - > name ) )
2018-02-26 09:05:06 +00:00
source_columns . erase ( it + + ) ;
2017-12-25 21:57:19 +00:00
else
+ + it ;
2017-04-01 07:20:54 +00:00
}
2018-02-26 21:00:42 +00:00
/// If there are virtual columns among the unknown columns. Remove them from the list of unknown and add
/// in columns list, so that when further processing they are also considered.
2017-04-01 07:20:54 +00:00
if ( storage )
{
2018-02-26 09:05:06 +00:00
for ( auto it = unknown_required_source_columns . begin ( ) ; it ! = unknown_required_source_columns . end ( ) ; )
2017-04-01 07:20:54 +00:00
{
if ( storage - > hasColumn ( * it ) )
{
2018-02-26 09:05:06 +00:00
source_columns . push_back ( storage - > getColumn ( * it ) ) ;
unknown_required_source_columns . erase ( it + + ) ;
2017-04-01 07:20:54 +00:00
}
else
+ + it ;
}
}
2018-02-28 01:29:55 +00:00
if ( ! unknown_required_source_columns . empty ( ) )
throw Exception ( " Unknown identifier: " + * unknown_required_source_columns . begin ( ) , ErrorCodes : : UNKNOWN_IDENTIFIER ) ;
2013-06-20 13:50:55 +00:00
}
2018-07-25 16:11:20 +00:00
void ExpressionAnalyzer : : collectJoinedColumnsFromJoinOnExpr ( )
{
const auto & tables = static_cast < const ASTTablesInSelectQuery & > ( * select_query - > tables ) ;
const auto * left_tables_element = static_cast < const ASTTablesInSelectQueryElement * > ( tables . children . at ( 0 ) . get ( ) ) ;
const auto * right_tables_element = select_query - > join ( ) ;
if ( ! left_tables_element | | ! right_tables_element )
return ;
const auto & table_join = static_cast < const ASTTableJoin & > ( * right_tables_element - > table_join ) ;
if ( ! table_join . on_expression )
return ;
const auto & left_table_expression = static_cast < const ASTTableExpression & > ( * left_tables_element - > table_expression ) ;
const auto & right_table_expression = static_cast < const ASTTableExpression & > ( * right_tables_element - > table_expression ) ;
auto left_source_names = getTableNameWithAliasFromTableExpression ( left_table_expression , context ) ;
auto right_source_names = getTableNameWithAliasFromTableExpression ( right_table_expression , context ) ;
/// Stores examples of columns which are only from one table.
struct TableBelonging
{
const ASTIdentifier * example_only_from_left = nullptr ;
const ASTIdentifier * example_only_from_right = nullptr ;
} ;
/// Check all identifiers in ast and decide their possible table belonging.
/// Throws if there are two identifiers definitely from different tables.
std : : function < TableBelonging ( const ASTPtr & ) > get_table_belonging ;
get_table_belonging = [ & ] ( const ASTPtr & ast ) - > TableBelonging
{
auto * identifier = typeid_cast < const ASTIdentifier * > ( ast . get ( ) ) ;
if ( identifier )
{
if ( identifier - > kind = = ASTIdentifier : : Column )
{
auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName ( * identifier , left_source_names ) ;
auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName ( * identifier , right_source_names ) ;
2018-07-27 17:11:56 +00:00
/// Assume that component from definite table if num_components is greater than for the other table.
2018-07-25 16:11:20 +00:00
if ( left_num_components > right_num_components )
return { identifier , nullptr } ;
if ( left_num_components < right_num_components )
return { nullptr , identifier } ;
}
return { } ;
}
TableBelonging table_belonging ;
for ( const auto & child : ast - > children )
{
auto children_belonging = get_table_belonging ( child ) ;
if ( ! table_belonging . example_only_from_left )
table_belonging . example_only_from_left = children_belonging . example_only_from_left ;
if ( ! table_belonging . example_only_from_right )
table_belonging . example_only_from_right = children_belonging . example_only_from_right ;
}
if ( table_belonging . example_only_from_left & & table_belonging . example_only_from_right )
throw Exception ( " Invalid columns in JOIN ON section. Columns "
+ table_belonging . example_only_from_left - > getAliasOrColumnName ( ) + " and "
+ table_belonging . example_only_from_right - > getAliasOrColumnName ( )
+ " are from different tables. " , ErrorCodes : : INVALID_JOIN_ON_EXPRESSION ) ;
return table_belonging ;
} ;
std : : function < void ( ASTPtr & , const DatabaseAndTableWithAlias & ) > translate_qualified_names ;
translate_qualified_names = [ & ] ( ASTPtr & ast , const DatabaseAndTableWithAlias & source_names )
{
auto * identifier = typeid_cast < const ASTIdentifier * > ( ast . get ( ) ) ;
if ( identifier )
{
if ( identifier - > kind = = ASTIdentifier : : Column )
{
auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName ( * identifier , source_names ) ;
stripIdentifier ( ast , num_components ) ;
}
return ;
}
for ( auto & child : ast - > children )
translate_qualified_names ( child , source_names ) ;
} ;
const auto supported_syntax = " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) "
" [AND Expr([table.]column, ...) = Expr([table.]column, ...) ...] " ;
auto throwSyntaxException = [ & ] ( const String & msg )
{
throw Exception ( " Invalid expression for JOIN ON. " + msg + supported_syntax , ErrorCodes : : INVALID_JOIN_ON_EXPRESSION ) ;
} ;
/// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys.
auto add_columns_from_equals_expr = [ & ] ( const ASTPtr & expr )
{
auto * func_equals = typeid_cast < const ASTFunction * > ( expr . get ( ) ) ;
if ( ! func_equals | | func_equals - > name ! = " equals " )
throwSyntaxException ( " Expected equals expression, got " + queryToString ( expr ) + " . " ) ;
ASTPtr left_ast = func_equals - > arguments - > children . at ( 0 ) - > clone ( ) ;
ASTPtr right_ast = func_equals - > arguments - > children . at ( 1 ) - > clone ( ) ;
auto left_table_belonging = get_table_belonging ( left_ast ) ;
auto right_table_belonging = get_table_belonging ( right_ast ) ;
bool can_be_left_part_from_left_table = left_table_belonging . example_only_from_right = = nullptr ;
bool can_be_left_part_from_right_table = left_table_belonging . example_only_from_left = = nullptr ;
bool can_be_right_part_from_left_table = right_table_belonging . example_only_from_right = = nullptr ;
bool can_be_right_part_from_right_table = right_table_belonging . example_only_from_left = = nullptr ;
auto add_join_keys = [ & ] ( ASTPtr & ast_to_left_table , ASTPtr & ast_to_right_table )
{
translate_qualified_names ( ast_to_left_table , left_source_names ) ;
translate_qualified_names ( ast_to_right_table , right_source_names ) ;
2018-07-30 13:57:50 +00:00
analyzed_join . key_asts_left . push_back ( ast_to_left_table ) ;
analyzed_join . key_names_left . push_back ( ast_to_left_table - > getColumnName ( ) ) ;
analyzed_join . key_asts_right . push_back ( ast_to_right_table ) ;
analyzed_join . key_names_right . push_back ( ast_to_right_table - > getAliasOrColumnName ( ) ) ;
2018-07-25 16:11:20 +00:00
} ;
/// Default variant when all identifiers may be from any table.
if ( can_be_left_part_from_left_table & & can_be_right_part_from_right_table )
add_join_keys ( left_ast , right_ast ) ;
else if ( can_be_left_part_from_right_table & & can_be_right_part_from_left_table )
add_join_keys ( right_ast , left_ast ) ;
else
{
auto * left_example = left_table_belonging . example_only_from_left ?
left_table_belonging . example_only_from_left :
left_table_belonging . example_only_from_right ;
auto * right_example = right_table_belonging . example_only_from_left ?
right_table_belonging . example_only_from_left :
right_table_belonging . example_only_from_right ;
auto left_name = queryToString ( * left_example ) ;
auto right_name = queryToString ( * right_example ) ;
auto expr_name = queryToString ( expr ) ;
throwSyntaxException ( " In expression " + expr_name + " columns " + left_name + " and " + right_name
+ " are from the same table but from different arguments of equal function. " ) ;
}
} ;
auto * func = typeid_cast < const ASTFunction * > ( table_join . on_expression . get ( ) ) ;
if ( func & & func - > name = = " and " )
{
for ( const auto & expr : func - > arguments - > children )
add_columns_from_equals_expr ( expr ) ;
}
else
add_columns_from_equals_expr ( table_join . on_expression ) ;
}
2018-07-30 13:57:50 +00:00
void ExpressionAnalyzer : : collectJoinedColumns ( NameSet & joined_columns )
2014-06-12 23:21:38 +00:00
{
2017-04-01 07:20:54 +00:00
if ( ! select_query )
return ;
const ASTTablesInSelectQueryElement * node = select_query - > join ( ) ;
if ( ! node )
return ;
2018-07-26 15:11:53 +00:00
const auto & table_join = static_cast < const ASTTableJoin & > ( * node - > table_join ) ;
const auto & table_expression = static_cast < const ASTTableExpression & > ( * node - > table_expression ) ;
auto joined_table_name = getTableNameWithAliasFromTableExpression ( table_expression , context ) ;
2017-04-01 07:20:54 +00:00
2018-07-25 16:11:20 +00:00
auto add_name_to_join_keys = [ ] ( Names & join_keys , ASTs & join_asts , const String & name , const ASTPtr & ast )
2018-07-23 15:54:10 +00:00
{
2018-08-31 10:00:46 +00:00
join_keys . push_back ( name ) ;
join_asts . push_back ( ast ) ;
2018-07-23 15:54:10 +00:00
} ;
2017-04-01 07:20:54 +00:00
if ( table_join . using_expression_list )
{
auto & keys = typeid_cast < ASTExpressionList & > ( * table_join . using_expression_list ) ;
for ( const auto & key : keys . children )
{
2018-07-30 13:57:50 +00:00
add_name_to_join_keys ( analyzed_join . key_names_left , analyzed_join . key_asts_left , key - > getColumnName ( ) , key ) ;
add_name_to_join_keys ( analyzed_join . key_names_right , analyzed_join . key_asts_right , key - > getAliasOrColumnName ( ) , key ) ;
2018-07-23 15:54:10 +00:00
}
}
else if ( table_join . on_expression )
2018-07-25 16:11:20 +00:00
collectJoinedColumnsFromJoinOnExpr ( ) ;
2018-07-23 15:54:10 +00:00
2018-07-27 20:01:54 +00:00
/// When we use JOIN ON syntax, non_joined_columns are columns from join_key_names_left,
/// because even if a column from join_key_names_right, we may need to join it if it has different name.
/// If we use USING syntax, join_key_names_left and join_key_names_right are almost the same, but we need to use
/// join_key_names_right in order to support aliases in USING list. Example:
/// SELECT x FROM tab1 ANY LEFT JOIN tab2 USING (x as y) - will join column x from tab1 with column y from tab2.
2018-07-30 13:57:50 +00:00
auto & not_joined_columns = table_join . using_expression_list ? analyzed_join . key_names_right : analyzed_join . key_names_left ;
2018-08-04 10:53:50 +00:00
auto columns_from_joined_table = analyzed_join . getColumnsFromJoinedTable ( context , select_query ) ;
2017-04-01 07:20:54 +00:00
2018-08-04 10:53:50 +00:00
for ( auto & column_name_and_type : columns_from_joined_table )
2017-04-01 07:20:54 +00:00
{
2018-08-04 10:53:50 +00:00
auto & column_name = column_name_and_type . name ;
auto & column_type = column_name_and_type . type ;
if ( not_joined_columns . end ( ) = = std : : find ( not_joined_columns . begin ( ) , not_joined_columns . end ( ) , column_name ) )
2017-04-01 07:20:54 +00:00
{
2018-08-04 10:53:50 +00:00
auto qualified_name = column_name ;
2018-07-31 11:31:18 +00:00
/// Change name for duplicate column form joined table.
2018-08-04 10:53:50 +00:00
if ( source_columns . contains ( qualified_name ) )
qualified_name = joined_table_name . getQualifiedNamePrefix ( ) + qualified_name ;
2018-07-31 11:31:18 +00:00
2018-08-04 10:53:50 +00:00
if ( joined_columns . count ( qualified_name ) ) /// Duplicate columns in the subquery for JOIN do not make sense.
2018-07-26 15:11:53 +00:00
continue ;
2017-12-25 00:38:45 +00:00
2018-08-04 10:53:50 +00:00
joined_columns . insert ( qualified_name ) ;
2017-12-25 00:38:45 +00:00
2018-07-26 15:11:53 +00:00
bool make_nullable = settings . join_use_nulls & & ( table_join . kind = = ASTTableJoin : : Kind : : Left | |
table_join . kind = = ASTTableJoin : : Kind : : Full ) ;
2018-08-04 10:53:50 +00:00
auto type = make_nullable ? makeNullable ( column_type ) : column_type ;
analyzed_join . columns_added_by_join . emplace_back ( NameAndTypePair ( qualified_name , std : : move ( type ) ) , column_name ) ;
2017-04-01 07:20:54 +00:00
}
}
2014-06-12 23:21:38 +00:00
}
2016-07-23 02:25:09 +00:00
2018-02-26 09:05:06 +00:00
Names ExpressionAnalyzer : : getRequiredSourceColumns ( ) const
2013-06-20 13:50:55 +00:00
{
2018-02-28 01:29:55 +00:00
return source_columns . getNames ( ) ;
2013-05-30 16:52:21 +00:00
}
2016-07-23 02:25:09 +00:00
2018-02-26 09:05:06 +00:00
void ExpressionAnalyzer : : getRequiredSourceColumnsImpl ( const ASTPtr & ast ,
const NameSet & available_columns , NameSet & required_source_columns , NameSet & ignored_names ,
2017-04-01 07:20:54 +00:00
const NameSet & available_joined_columns , NameSet & required_joined_columns )
{
2017-04-02 17:37:49 +00:00
/** Find all the identifiers in the query.
2018-02-26 09:05:06 +00:00
* We will use depth first search in AST .
2017-04-02 17:37:49 +00:00
* In this case
* - for lambda functions we will not take formal parameters ;
2018-02-26 09:05:06 +00:00
* - do not go into subqueries ( they have their own identifiers ) ;
* - there is some exception for the ARRAY JOIN clause ( it has a slightly different identifiers ) ;
* - we put identifiers available from JOIN in required_joined_columns .
2017-04-01 07:20:54 +00:00
*/
if ( ASTIdentifier * node = typeid_cast < ASTIdentifier * > ( ast . get ( ) ) )
{
if ( node - > kind = = ASTIdentifier : : Column
& & ! ignored_names . count ( node - > name )
2017-12-25 18:58:39 +00:00
& & ! ignored_names . count ( Nested : : extractTableName ( node - > name ) ) )
2017-04-01 07:20:54 +00:00
{
2017-12-14 12:21:01 +00:00
if ( ! available_joined_columns . count ( node - > name )
| | available_columns . count ( node - > name ) ) /// Read column from left table if has.
2018-02-26 09:05:06 +00:00
required_source_columns . insert ( node - > name ) ;
2017-04-01 07:20:54 +00:00
else
required_joined_columns . insert ( node - > name ) ;
}
return ;
}
if ( ASTFunction * node = typeid_cast < ASTFunction * > ( ast . get ( ) ) )
{
2018-02-24 01:31:42 +00:00
if ( node - > name = = " lambda " )
2017-04-01 07:20:54 +00:00
{
if ( node - > arguments - > children . size ( ) ! = 2 )
throw Exception ( " lambda requires two arguments " , ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ) ;
ASTFunction * lambda_args_tuple = typeid_cast < ASTFunction * > ( node - > arguments - > children . at ( 0 ) . get ( ) ) ;
if ( ! lambda_args_tuple | | lambda_args_tuple - > name ! = " tuple " )
throw Exception ( " First argument of lambda must be a tuple " , ErrorCodes : : TYPE_MISMATCH ) ;
2018-02-26 09:05:06 +00:00
/// You do not need to add formal parameters of the lambda expression in required_source_columns.
2017-04-01 07:20:54 +00:00
Names added_ignored ;
for ( auto & child : lambda_args_tuple - > arguments - > children )
{
ASTIdentifier * identifier = typeid_cast < ASTIdentifier * > ( child . get ( ) ) ;
if ( ! identifier )
throw Exception ( " lambda argument declarations must be identifiers " , ErrorCodes : : TYPE_MISMATCH ) ;
String & name = identifier - > name ;
if ( ! ignored_names . count ( name ) )
{
ignored_names . insert ( name ) ;
added_ignored . push_back ( name ) ;
}
}
2018-02-26 09:05:06 +00:00
getRequiredSourceColumnsImpl ( node - > arguments - > children . at ( 1 ) ,
available_columns , required_source_columns , ignored_names ,
2017-04-01 07:20:54 +00:00
available_joined_columns , required_joined_columns ) ;
for ( size_t i = 0 ; i < added_ignored . size ( ) ; + + i )
ignored_names . erase ( added_ignored [ i ] ) ;
return ;
}
2017-04-02 17:37:49 +00:00
/// A special function `indexHint`. Everything that is inside it is not calculated
2018-04-20 00:20:36 +00:00
/// (and is used only for index analysis, see KeyCondition).
2017-04-01 07:20:54 +00:00
if ( node - > name = = " indexHint " )
return ;
}
2017-04-02 17:37:49 +00:00
/// Recursively traverses an expression.
2017-04-01 07:20:54 +00:00
for ( auto & child : ast - > children )
{
2017-04-02 17:37:49 +00:00
/** We will not go to the ARRAY JOIN section, because we need to look at the names of non-ARRAY-JOIN columns.
* There , ` collectUsedColumns ` will send us separately .
2017-04-01 07:20:54 +00:00
*/
2018-03-01 04:57:51 +00:00
if ( ! typeid_cast < const ASTSelectQuery * > ( child . get ( ) )
& & ! typeid_cast < const ASTArrayJoin * > ( child . get ( ) )
2018-07-25 16:11:20 +00:00
& & ! typeid_cast < const ASTTableExpression * > ( child . get ( ) )
& & ! typeid_cast < const ASTTableJoin * > ( child . get ( ) ) )
2018-02-26 09:05:06 +00:00
getRequiredSourceColumnsImpl ( child , available_columns , required_source_columns ,
ignored_names , available_joined_columns , required_joined_columns ) ;
2017-04-01 07:20:54 +00:00
}
2013-05-30 16:52:21 +00:00
}
2018-02-26 09:55:14 +00:00
static bool hasArrayJoin ( const ASTPtr & ast )
{
if ( const ASTFunction * function = typeid_cast < const ASTFunction * > ( & * ast ) )
if ( function - > name = = " arrayJoin " )
return true ;
for ( const auto & child : ast - > children )
if ( ! typeid_cast < ASTSelectQuery * > ( child . get ( ) ) & & hasArrayJoin ( child ) )
return true ;
return false ;
}
2018-03-01 01:25:06 +00:00
void ExpressionAnalyzer : : removeUnneededColumnsFromSelectClause ( )
2018-02-26 09:55:14 +00:00
{
if ( ! select_query )
return ;
2018-08-05 03:28:31 +00:00
if ( required_result_columns . empty ( ) )
2018-03-01 01:25:06 +00:00
return ;
ASTs & elements = select_query - > select_expression_list - > children ;
2018-08-05 03:28:31 +00:00
ASTs new_elements ;
new_elements . reserve ( elements . size ( ) ) ;
/// Some columns may be queried multiple times, like SELECT x, y, y FROM table.
/// In that case we keep them exactly same number of times.
std : : map < String , size_t > required_columns_with_duplicate_count ;
for ( const auto & name : required_result_columns )
+ + required_columns_with_duplicate_count [ name ] ;
for ( const auto & elem : elements )
2018-03-01 01:25:06 +00:00
{
2018-08-05 03:28:31 +00:00
String name = elem - > getAliasOrColumnName ( ) ;
auto it = required_columns_with_duplicate_count . find ( name ) ;
if ( required_columns_with_duplicate_count . end ( ) ! = it & & it - > second )
{
new_elements . push_back ( elem ) ;
- - it - > second ;
}
else if ( select_query - > distinct | | hasArrayJoin ( elem ) )
{
new_elements . push_back ( elem ) ;
}
}
elements = std : : move ( new_elements ) ;
2018-02-26 09:55:14 +00:00
}
2013-05-24 10:49:19 +00:00
}