2017-04-01 09:19:00 +00:00
# include <DataStreams/ExpressionBlockInputStream.h>
# include <DataStreams/FilterBlockInputStream.h>
# include <DataStreams/LimitBlockInputStream.h>
# include <DataStreams/LimitByBlockInputStream.h>
# include <DataStreams/PartialSortingBlockInputStream.h>
# include <DataStreams/MergeSortingBlockInputStream.h>
# include <DataStreams/MergingSortedBlockInputStream.h>
# include <DataStreams/AggregatingBlockInputStream.h>
# include <DataStreams/MergingAggregatedBlockInputStream.h>
# include <DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
# include <DataStreams/AsynchronousBlockInputStream.h>
# include <DataStreams/UnionBlockInputStream.h>
# include <DataStreams/ParallelAggregatingBlockInputStream.h>
# include <DataStreams/DistinctBlockInputStream.h>
# include <DataStreams/NullBlockInputStream.h>
# include <DataStreams/TotalsHavingBlockInputStream.h>
# include <DataStreams/copyData.h>
# include <DataStreams/CreatingSetsBlockInputStream.h>
# include <DataStreams/MaterializingBlockInputStream.h>
# include <DataStreams/ConcatBlockInputStream.h>
2018-08-21 16:08:45 +00:00
# include <DataStreams/RollupBlockInputStream.h>
2018-09-17 18:01:04 +00:00
# include <DataStreams/CubeBlockInputStream.h>
2018-09-27 15:55:22 +00:00
# include <DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h>
2019-02-10 15:17:45 +00:00
# include <DataStreams/ConvertingBlockInputStream.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTFunction.h>
2019-03-29 20:31:06 +00:00
# include <Parsers/ASTIdentifier.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTLiteral.h>
# include <Parsers/ASTOrderByElement.h>
2019-03-29 20:31:06 +00:00
# include <Parsers/ASTSelectWithUnionQuery.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTTablesInSelectQuery.h>
2019-03-29 20:31:06 +00:00
# include <Parsers/ParserSelectQuery.h>
2019-05-03 02:00:57 +00:00
# include <Parsers/ExpressionListParsers.h>
# include <Parsers/parseQuery.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/InterpreterSelectQuery.h>
2018-02-25 06:34:20 +00:00
# include <Interpreters/InterpreterSelectWithUnionQuery.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/InterpreterSetQuery.h>
2019-02-03 18:31:17 +00:00
# include <Interpreters/evaluateConstantExpression.h>
2019-02-10 15:17:45 +00:00
# include <Interpreters/convertFieldToType.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ExpressionAnalyzer.h>
2018-10-30 16:31:21 +00:00
# include <Interpreters/DatabaseAndTableWithAlias.h>
2019-03-11 12:20:55 +00:00
# include <Interpreters/JoinToSubqueryTransformVisitor.h>
# include <Interpreters/CrossToInnerJoinVisitor.h>
2017-04-01 09:19:00 +00:00
2019-02-10 15:17:45 +00:00
# include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
2017-04-01 09:19:00 +00:00
# include <Storages/IStorage.h>
2019-05-03 02:00:57 +00:00
# include <Storages/MergeTree/MergeTreeData.h>
2017-04-01 09:19:00 +00:00
# include <TableFunctions/ITableFunction.h>
# include <TableFunctions/TableFunctionFactory.h>
# include <Core/Field.h>
2019-02-03 18:31:17 +00:00
# include <Core/Types.h>
2017-11-20 06:01:05 +00:00
# include <Columns/Collator.h>
2017-07-13 20:58:19 +00:00
# include <Common/typeid_cast.h>
2018-03-04 16:15:31 +00:00
# include <Parsers/queryToString.h>
2018-09-06 07:49:42 +00:00
# include <ext/map.h>
2018-09-07 15:13:08 +00:00
# include <memory>
2014-01-28 19:26:39 +00:00
2015-10-29 15:14:19 +00:00
2011-08-28 05:13:24 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2017-04-01 07:20:54 +00:00
extern const int TOO_DEEP_SUBQUERIES ;
extern const int THERE_IS_NO_COLUMN ;
extern const int SAMPLING_NOT_SUPPORTED ;
extern const int ILLEGAL_FINAL ;
extern const int ILLEGAL_PREWHERE ;
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_COLUMNS ;
2018-02-21 06:25:21 +00:00
extern const int LOGICAL_ERROR ;
2018-03-16 02:08:31 +00:00
extern const int NOT_IMPLEMENTED ;
2018-08-25 16:32:44 +00:00
extern const int PARAMETER_OUT_OF_BOUND ;
2018-09-07 15:13:08 +00:00
extern const int ARGUMENT_OUT_OF_BOUND ;
2019-02-03 18:31:17 +00:00
extern const int INVALID_LIMIT_EXPRESSION ;
2016-01-11 21:46:36 +00:00
}
2019-03-29 20:31:06 +00:00
namespace
{
/// Assumes `storage` is set and the table filter is not empty.
String generateFilterActions ( ExpressionActionsPtr & actions , const StoragePtr & storage , const Context & context , const Names & prerequisite_columns = { } )
{
const auto & db_name = storage - > getDatabaseName ( ) ;
const auto & table_name = storage - > getTableName ( ) ;
const auto & filter_str = context . getUserProperty ( db_name , table_name , " filter " ) ;
/// TODO: implement some AST builders for this kind of stuff
ASTPtr query_ast = std : : make_shared < ASTSelectQuery > ( ) ;
auto * select_ast = query_ast - > as < ASTSelectQuery > ( ) ;
2019-04-09 14:22:35 +00:00
select_ast - > setExpression ( ASTSelectQuery : : Expression : : SELECT , std : : make_shared < ASTExpressionList > ( ) ) ;
auto expr_list = select_ast - > select ( ) ;
2019-03-29 20:31:06 +00:00
auto parseExpression = [ ] ( const String & expr )
{
ParserExpression expr_parser ;
return parseQuery ( expr_parser , expr , 0 ) ;
} ;
// The first column is our filter expression.
expr_list - > children . push_back ( parseExpression ( filter_str ) ) ;
/// Keep columns that are required after the filter actions.
for ( const auto & column_str : prerequisite_columns )
expr_list - > children . push_back ( parseExpression ( column_str ) ) ;
2019-04-09 14:22:35 +00:00
select_ast - > setExpression ( ASTSelectQuery : : Expression : : TABLES , std : : make_shared < ASTTablesInSelectQuery > ( ) ) ;
2019-04-09 14:59:06 +00:00
auto tables = select_ast - > tables ( ) ;
2019-03-29 20:31:06 +00:00
auto tables_elem = std : : make_shared < ASTTablesInSelectQueryElement > ( ) ;
auto table_expr = std : : make_shared < ASTTableExpression > ( ) ;
tables - > children . push_back ( tables_elem ) ;
tables_elem - > table_expression = table_expr ;
tables_elem - > children . push_back ( table_expr ) ;
table_expr - > database_and_table_name = createTableIdentifier ( db_name , table_name ) ;
table_expr - > children . push_back ( table_expr - > database_and_table_name ) ;
/// Using separate expression analyzer to prevent any possible alias injection
auto syntax_result = SyntaxAnalyzer ( context ) . analyze ( query_ast , storage - > getColumns ( ) . getAllPhysical ( ) ) ;
ExpressionAnalyzer analyzer ( query_ast , syntax_result , context ) ;
ExpressionActionsChain new_chain ( context ) ;
analyzer . appendSelect ( new_chain , false ) ;
actions = new_chain . getLastActions ( ) ;
return expr_list - > children . at ( 0 ) - > getColumnName ( ) ;
}
} // namespace
2018-02-26 21:00:42 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
const Context & context_ ,
2019-03-15 13:49:58 +00:00
const SelectQueryOptions & options ,
const Names & required_result_column_names )
: InterpreterSelectQuery ( query_ptr_ , context_ , nullptr , nullptr , options , required_result_column_names )
2018-02-26 21:00:42 +00:00
{
}
2018-07-17 13:09:33 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
const Context & context_ ,
const BlockInputStreamPtr & input_ ,
2019-03-15 13:49:58 +00:00
const SelectQueryOptions & options )
2019-03-18 12:05:51 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , input_ , nullptr , options . copy ( ) . noSubquery ( ) )
2019-03-15 15:57:18 +00:00
{ }
2018-02-26 21:00:42 +00:00
2018-07-18 12:17:48 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
const Context & context_ ,
const StoragePtr & storage_ ,
2019-03-15 13:49:58 +00:00
const SelectQueryOptions & options )
2019-03-18 12:05:51 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , nullptr , storage_ , options . copy ( ) . noSubquery ( ) )
2019-03-15 15:57:18 +00:00
{ }
2018-07-18 12:17:48 +00:00
2018-07-19 13:36:21 +00:00
InterpreterSelectQuery : : ~ InterpreterSelectQuery ( ) = default ;
/** There are no limits on the maximum size of the result for the subquery.
* Since the result of the query is not the result of the entire query .
*/
static Context getSubqueryContext ( const Context & context )
2018-02-26 21:00:42 +00:00
{
2018-07-19 13:36:21 +00:00
Context subquery_context = context ;
Settings subquery_settings = context . getSettings ( ) ;
subquery_settings . max_result_rows = 0 ;
subquery_settings . max_result_bytes = 0 ;
/// 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 for whole query).
subquery_settings . extremes = 0 ;
subquery_context . setSettings ( subquery_settings ) ;
return subquery_context ;
2018-02-26 21:00:42 +00:00
}
2016-01-11 21:46:36 +00:00
2018-07-17 13:09:33 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
const Context & context_ ,
const BlockInputStreamPtr & input_ ,
const StoragePtr & storage_ ,
2019-03-18 12:05:51 +00:00
const SelectQueryOptions & options_ ,
2019-03-15 13:49:58 +00:00
const Names & required_result_column_names )
2019-03-18 12:05:51 +00:00
: options ( options_ )
2019-02-11 19:53:55 +00:00
/// NOTE: the query almost always should be cloned because it will be modified during analysis.
2019-03-18 12:05:51 +00:00
, query_ptr ( options . modify_inplace ? query_ptr_ : query_ptr_ - > clone ( ) )
2018-07-17 13:09:33 +00:00
, context ( context_ )
, storage ( storage_ )
, input ( input_ )
, log ( & Logger : : get ( " InterpreterSelectQuery " ) )
2014-12-26 10:35:03 +00:00
{
2017-04-01 07:20:54 +00:00
initSettings ( ) ;
2017-05-24 20:13:04 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2017-04-01 07:20:54 +00:00
2019-03-18 12:05:51 +00:00
if ( settings . max_subquery_depth & & options . subquery_depth > settings . max_subquery_depth )
2018-03-11 00:15:26 +00:00
throw Exception ( " Too deep subqueries. Maximum: " + settings . max_subquery_depth . toString ( ) ,
2017-04-01 07:20:54 +00:00
ErrorCodes : : TOO_DEEP_SUBQUERIES ) ;
2019-03-11 12:20:55 +00:00
if ( settings . allow_experimental_cross_to_join_conversion )
{
CrossToInnerJoinVisitor : : Data cross_to_inner ;
CrossToInnerJoinVisitor ( cross_to_inner ) . visit ( query_ptr ) ;
}
2019-03-11 13:08:51 +00:00
if ( settings . allow_experimental_multiple_joins_emulation )
{
2019-04-03 16:06:05 +00:00
JoinToSubqueryTransformVisitor : : Data join_to_subs_data { context } ;
2019-03-11 13:08:51 +00:00
JoinToSubqueryTransformVisitor ( join_to_subs_data ) . visit ( query_ptr ) ;
}
2017-07-03 21:04:10 +00:00
max_streams = settings . max_threads ;
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2017-07-03 21:04:10 +00:00
2019-01-15 18:29:54 +00:00
ASTPtr table_expression = extractTableExpression ( query , 0 ) ;
bool is_table_func = false ;
bool is_subquery = false ;
if ( table_expression )
{
2019-03-11 13:22:51 +00:00
is_table_func = table_expression - > as < ASTFunction > ( ) ;
is_subquery = table_expression - > as < ASTSelectWithUnionQuery > ( ) ;
2019-01-15 18:29:54 +00:00
}
2018-02-28 01:29:55 +00:00
2018-02-15 18:54:12 +00:00
if ( input )
2017-04-01 07:20:54 +00:00
{
2018-02-28 01:29:55 +00:00
/// Read from prepared input.
2018-07-19 13:36:21 +00:00
source_header = input - > getHeader ( ) ;
2017-04-01 07:20:54 +00:00
}
2019-01-15 18:29:54 +00:00
else if ( is_subquery )
2017-04-01 07:20:54 +00:00
{
2018-02-15 18:54:12 +00:00
/// Read from subquery.
2018-07-19 13:36:21 +00:00
interpreter_subquery = std : : make_unique < InterpreterSelectWithUnionQuery > (
2019-03-18 12:05:51 +00:00
table_expression , getSubqueryContext ( context ) , options . subquery ( ) , required_columns ) ;
2018-07-19 13:36:21 +00:00
source_header = interpreter_subquery - > getSampleBlock ( ) ;
2018-02-28 01:29:55 +00:00
}
2018-07-18 12:17:48 +00:00
else if ( ! storage )
2018-02-28 01:29:55 +00:00
{
2019-01-15 18:29:54 +00:00
if ( is_table_func )
2018-07-18 12:17:48 +00:00
{
/// Read from table function.
storage = context . getQueryContext ( ) . executeTableFunction ( table_expression ) ;
}
else
{
/// Read from table. Even without table expression (implicit SELECT ... FROM system.one).
String database_name ;
String table_name ;
getDatabaseAndTableNames ( database_name , table_name ) ;
storage = context . getTable ( database_name , table_name ) ;
}
2017-04-01 07:20:54 +00:00
}
2018-02-28 01:29:55 +00:00
if ( storage )
2019-03-07 18:04:47 +00:00
table_lock = storage - > lockStructureForShare ( false , context . getCurrentQueryId ( ) ) ;
2017-04-01 07:20:54 +00:00
2019-03-18 14:56:33 +00:00
syntax_analyzer_result = SyntaxAnalyzer ( context , options ) . analyze (
2019-01-09 16:16:59 +00:00
query_ptr , source_header . getNamesAndTypesList ( ) , required_result_column_names , storage ) ;
2018-02-26 09:05:06 +00:00
query_analyzer = std : : make_unique < ExpressionAnalyzer > (
2019-01-30 15:51:39 +00:00
query_ptr , syntax_analyzer_result , context , NamesAndTypesList ( ) ,
2019-03-18 12:05:51 +00:00
NameSet ( required_result_column_names . begin ( ) , required_result_column_names . end ( ) ) ,
options . subquery_depth , ! options . only_analyze ) ;
2017-04-01 07:20:54 +00:00
2019-03-18 12:05:51 +00:00
if ( ! options . only_analyze )
2018-03-12 15:14:26 +00:00
{
if ( query . sample_size ( ) & & ( input | | ! storage | | ! storage - > supportsSampling ( ) ) )
throw Exception ( " Illegal SAMPLE: table doesn't support sampling " , ErrorCodes : : SAMPLING_NOT_SUPPORTED ) ;
2017-11-05 17:48:50 +00:00
2018-03-12 15:14:26 +00:00
if ( query . final ( ) & & ( input | | ! storage | | ! storage - > supportsFinal ( ) ) )
throw Exception ( ( ! input & & storage ) ? " Storage " + storage - > getName ( ) + " doesn't support FINAL " : " Illegal FINAL " , ErrorCodes : : ILLEGAL_FINAL ) ;
2017-11-05 17:48:50 +00:00
2019-04-09 14:22:35 +00:00
if ( query . prewhere ( ) & & ( input | | ! storage | | ! storage - > supportsPrewhere ( ) ) )
2018-03-12 15:14:26 +00:00
throw Exception ( ( ! input & & storage ) ? " Storage " + storage - > getName ( ) + " doesn't support PREWHERE " : " Illegal PREWHERE " , ErrorCodes : : ILLEGAL_PREWHERE ) ;
2017-11-05 17:48:50 +00:00
2018-03-12 15:14:26 +00:00
/// Save the new temporary tables in the query context
for ( const auto & it : query_analyzer - > getExternalTables ( ) )
if ( ! context . tryGetExternalTable ( it . first ) )
context . addExternalTable ( it . first , it . second ) ;
2019-02-20 12:36:46 +00:00
}
2018-03-04 16:15:31 +00:00
2019-03-18 12:05:51 +00:00
if ( ! options . only_analyze | | options . modify_inplace )
2019-02-20 12:36:46 +00:00
{
2018-08-20 15:49:39 +00:00
if ( query_analyzer - > isRewriteSubqueriesPredicate ( ) )
2018-09-29 11:29:23 +00:00
{
2019-04-18 10:39:25 +00:00
/// remake interpreter_subquery when PredicateOptimizer rewrites subqueries and main table is subquery
2019-01-15 18:29:54 +00:00
if ( is_subquery )
2018-09-29 11:29:23 +00:00
interpreter_subquery = std : : make_unique < InterpreterSelectWithUnionQuery > (
2019-02-20 12:36:46 +00:00
table_expression ,
getSubqueryContext ( context ) ,
2019-03-18 12:05:51 +00:00
options . subquery ( ) ,
2019-03-15 13:49:58 +00:00
required_columns ) ;
2018-09-29 11:29:23 +00:00
}
2018-03-12 15:14:26 +00:00
}
2018-07-19 13:36:21 +00:00
if ( interpreter_subquery )
{
/// If there is an aggregation in the outer query, WITH TOTALS is ignored in the subquery.
if ( query_analyzer - > hasAggregation ( ) )
interpreter_subquery - > ignoreWithTotals ( ) ;
}
required_columns = query_analyzer - > getRequiredSourceColumns ( ) ;
if ( storage )
source_header = storage - > getSampleBlockForColumns ( required_columns ) ;
/// Calculate structure of the result.
{
Pipeline pipeline ;
2018-08-08 03:09:59 +00:00
executeImpl ( pipeline , nullptr , true ) ;
2018-07-19 13:36:21 +00:00
result_header = pipeline . firstStream ( ) - > getHeader ( ) ;
}
2014-12-24 14:51:02 +00:00
}
2017-12-22 18:30:42 +00:00
2012-08-20 19:21:04 +00:00
void InterpreterSelectQuery : : getDatabaseAndTableNames ( String & database_name , String & table_name )
2011-08-28 05:13:24 +00:00
{
2019-03-12 14:07:02 +00:00
if ( auto db_and_table = getDatabaseAndTable ( getSelectQuery ( ) , 0 ) )
2018-10-29 19:04:28 +00:00
{
2018-10-30 16:31:21 +00:00
table_name = db_and_table - > table ;
database_name = db_and_table - > database ;
2017-04-01 07:20:54 +00:00
2018-10-29 19:04:28 +00:00
/// If the database is not specified - use the current database.
if ( database_name . empty ( ) & & ! context . tryGetTable ( " " , table_name ) )
database_name = context . getCurrentDatabase ( ) ;
}
else /// If the table is not specified - use the table `system.one`.
2017-04-01 07:20:54 +00:00
{
database_name = " system " ;
table_name = " one " ;
}
2012-08-20 19:21:04 +00:00
}
2011-08-28 05:13:24 +00:00
2011-10-30 05:19:41 +00:00
Block InterpreterSelectQuery : : getSampleBlock ( )
{
2018-07-19 13:36:21 +00:00
return result_header ;
2015-07-13 15:02:29 +00:00
}
2015-06-18 02:11:05 +00:00
BlockIO InterpreterSelectQuery : : execute ( )
2014-12-19 15:56:12 +00:00
{
2018-02-21 03:26:06 +00:00
Pipeline pipeline ;
2019-03-18 12:05:51 +00:00
executeImpl ( pipeline , input , options . only_analyze ) ;
2018-02-21 03:26:06 +00:00
executeUnion ( pipeline ) ;
2017-04-01 07:20:54 +00:00
BlockIO res ;
2018-02-21 03:26:06 +00:00
res . in = pipeline . firstStream ( ) ;
2017-04-01 07:20:54 +00:00
return res ;
2014-12-16 10:39:02 +00:00
}
2018-02-25 00:50:53 +00:00
BlockInputStreams InterpreterSelectQuery : : executeWithMultipleStreams ( )
2018-02-21 03:26:06 +00:00
{
Pipeline pipeline ;
2019-03-18 12:05:51 +00:00
executeImpl ( pipeline , input , options . only_analyze ) ;
2018-02-21 03:26:06 +00:00
return pipeline . streams ;
}
2019-03-29 20:31:06 +00:00
InterpreterSelectQuery : : AnalysisResult
InterpreterSelectQuery : : analyzeExpressions ( QueryProcessingStage : : Enum from_stage , bool dry_run , const FilterInfoPtr & filter_info )
2018-02-23 06:00:48 +00:00
{
AnalysisResult res ;
/// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
res . first_stage = from_stage < QueryProcessingStage : : WithMergeableState
2019-03-18 12:05:51 +00:00
& & options . to_stage > = QueryProcessingStage : : WithMergeableState ;
2018-02-23 06:00:48 +00:00
/// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing.
res . second_stage = from_stage < = QueryProcessingStage : : WithMergeableState
2019-03-18 12:05:51 +00:00
& & options . to_stage > QueryProcessingStage : : WithMergeableState ;
2018-02-23 06:00:48 +00:00
/** First we compose a chain of actions and remember the necessary steps from it.
* Regardless of from_stage and to_stage , we will compose a complete sequence of actions to perform optimization and
* throw out unnecessary columns based on the entire query . In unnecessary parts of the query , we will not execute subqueries .
*/
2019-03-29 20:31:06 +00:00
bool has_filter = false ;
2018-06-29 11:42:44 +00:00
bool has_prewhere = false ;
bool has_where = false ;
size_t where_step_num ;
auto finalizeChain = [ & ] ( ExpressionActionsChain & chain )
{
chain . finalize ( ) ;
if ( has_prewhere )
2018-09-07 15:13:08 +00:00
{
const ExpressionActionsChain : : Step & step = chain . steps . at ( 0 ) ;
res . prewhere_info - > remove_prewhere_column = step . can_remove_required_output . at ( 0 ) ;
2018-10-04 08:58:19 +00:00
Names columns_to_remove ;
2018-09-07 15:13:08 +00:00
for ( size_t i = 1 ; i < step . required_output . size ( ) ; + + i )
{
if ( step . can_remove_required_output [ i ] )
2018-10-04 08:58:19 +00:00
columns_to_remove . push_back ( step . required_output [ i ] ) ;
2018-09-07 15:13:08 +00:00
}
2018-10-04 08:58:19 +00:00
if ( ! columns_to_remove . empty ( ) )
2018-09-07 15:13:08 +00:00
{
auto columns = res . prewhere_info - > prewhere_actions - > getSampleBlock ( ) . getNamesAndTypesList ( ) ;
ExpressionActionsPtr actions = std : : make_shared < ExpressionActions > ( columns , context ) ;
2018-10-04 08:58:19 +00:00
for ( const auto & column : columns_to_remove )
2018-09-07 15:13:08 +00:00
actions - > add ( ExpressionAction : : removeColumn ( column ) ) ;
2018-10-04 08:58:19 +00:00
res . prewhere_info - > remove_columns_actions = std : : move ( actions ) ;
2018-09-07 15:13:08 +00:00
}
2018-11-06 13:55:41 +00:00
res . columns_to_remove_after_prewhere = std : : move ( columns_to_remove ) ;
2018-09-07 15:13:08 +00:00
}
2019-03-29 20:31:06 +00:00
else if ( has_filter )
{
/// Can't have prewhere and filter set simultaneously
res . filter_info - > do_remove_column = chain . steps . at ( 0 ) . can_remove_required_output . at ( 0 ) ;
}
2018-06-29 11:42:44 +00:00
if ( has_where )
res . remove_where_filter = chain . steps . at ( where_step_num ) . can_remove_required_output . at ( 0 ) ;
2019-03-29 20:31:06 +00:00
has_filter = has_prewhere = has_where = false ;
2018-06-29 11:42:44 +00:00
chain . clear ( ) ;
} ;
2018-04-12 09:45:24 +00:00
2018-02-23 06:00:48 +00:00
{
2018-08-30 16:31:20 +00:00
ExpressionActionsChain chain ( context ) ;
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2018-02-23 06:00:48 +00:00
2018-11-08 16:39:43 +00:00
Names additional_required_columns_after_prewhere ;
if ( storage & & query . sample_size ( ) )
{
Names columns_for_sampling = storage - > getColumnsRequiredForSampling ( ) ;
additional_required_columns_after_prewhere . insert ( additional_required_columns_after_prewhere . end ( ) ,
columns_for_sampling . begin ( ) , columns_for_sampling . end ( ) ) ;
}
if ( storage & & query . final ( ) )
{
Names columns_for_final = storage - > getColumnsRequiredForFinal ( ) ;
additional_required_columns_after_prewhere . insert ( additional_required_columns_after_prewhere . end ( ) ,
columns_for_final . begin ( ) , columns_for_final . end ( ) ) ;
}
2019-03-29 20:31:06 +00:00
if ( storage & & context . hasUserProperty ( storage - > getDatabaseName ( ) , storage - > getTableName ( ) , " filter " ) )
{
has_filter = true ;
/// XXX: aggregated copy-paste from ExpressionAnalyzer::appendSmth()
if ( chain . steps . empty ( ) )
{
chain . steps . emplace_back ( std : : make_shared < ExpressionActions > ( source_columns , context ) ) ;
}
ExpressionActionsChain : : Step & step = chain . steps . back ( ) ;
// FIXME: assert(filter_info);
res . filter_info = filter_info ;
step . actions = filter_info - > actions ;
step . required_output . push_back ( res . filter_info - > column_name ) ;
step . can_remove_required_output = { true } ;
chain . addStep ( ) ;
}
2018-11-08 16:39:43 +00:00
if ( query_analyzer - > appendPrewhere ( chain , ! res . first_stage , additional_required_columns_after_prewhere ) )
2018-04-19 14:47:09 +00:00
{
2018-06-29 11:42:44 +00:00
has_prewhere = true ;
2018-04-19 14:47:09 +00:00
res . prewhere_info = std : : make_shared < PrewhereInfo > (
2019-04-09 14:22:35 +00:00
chain . steps . front ( ) . actions , query . prewhere ( ) - > getColumnName ( ) ) ;
2018-06-29 11:42:44 +00:00
2018-04-19 14:47:09 +00:00
chain . addStep ( ) ;
}
2018-02-23 06:00:48 +00:00
res . need_aggregate = query_analyzer - > hasAggregation ( ) ;
2018-07-19 13:36:21 +00:00
query_analyzer - > appendArrayJoin ( chain , dry_run | | ! res . first_stage ) ;
2018-02-23 06:00:48 +00:00
2018-07-19 13:36:21 +00:00
if ( query_analyzer - > appendJoin ( chain , dry_run | | ! res . first_stage ) )
2018-02-23 06:00:48 +00:00
{
res . before_join = chain . getLastActions ( ) ;
2019-01-30 12:01:00 +00:00
if ( ! res . hasJoin ( ) )
throw Exception ( " No expected JOIN " , ErrorCodes : : LOGICAL_ERROR ) ;
2018-02-23 06:00:48 +00:00
chain . addStep ( ) ;
}
2018-07-19 13:36:21 +00:00
if ( query_analyzer - > appendWhere ( chain , dry_run | | ! res . first_stage ) )
2018-02-23 06:00:48 +00:00
{
2018-06-29 11:42:44 +00:00
where_step_num = chain . steps . size ( ) - 1 ;
has_where = res . has_where = true ;
2018-02-23 06:00:48 +00:00
res . before_where = chain . getLastActions ( ) ;
chain . addStep ( ) ;
}
if ( res . need_aggregate )
{
2018-07-19 13:36:21 +00:00
query_analyzer - > appendGroupBy ( chain , dry_run | | ! res . first_stage ) ;
query_analyzer - > appendAggregateFunctionsArguments ( chain , dry_run | | ! res . first_stage ) ;
2018-02-23 06:00:48 +00:00
res . before_aggregation = chain . getLastActions ( ) ;
2018-06-29 11:42:44 +00:00
finalizeChain ( chain ) ;
2018-02-23 06:00:48 +00:00
2018-07-19 13:36:21 +00:00
if ( query_analyzer - > appendHaving ( chain , dry_run | | ! res . second_stage ) )
2018-02-23 06:00:48 +00:00
{
res . has_having = true ;
res . before_having = chain . getLastActions ( ) ;
chain . addStep ( ) ;
}
}
/// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers.
2018-07-19 13:36:21 +00:00
query_analyzer - > appendSelect ( chain , dry_run | | ( res . need_aggregate ? ! res . second_stage : ! res . first_stage ) ) ;
2018-02-23 06:00:48 +00:00
res . selected_columns = chain . getLastStep ( ) . required_output ;
2018-07-19 13:36:21 +00:00
res . has_order_by = query_analyzer - > appendOrderBy ( chain , dry_run | | ( res . need_aggregate ? ! res . second_stage : ! res . first_stage ) ) ;
2018-02-23 06:00:48 +00:00
res . before_order_and_select = chain . getLastActions ( ) ;
chain . addStep ( ) ;
2018-07-19 13:36:21 +00:00
if ( query_analyzer - > appendLimitBy ( chain , dry_run | | ! res . second_stage ) )
2018-03-01 06:07:04 +00:00
{
res . has_limit_by = true ;
res . before_limit_by = chain . getLastActions ( ) ;
chain . addStep ( ) ;
}
2018-03-01 05:24:56 +00:00
2018-02-23 06:00:48 +00:00
query_analyzer - > appendProjectResult ( chain ) ;
res . final_projection = chain . getLastActions ( ) ;
2018-06-29 11:42:44 +00:00
finalizeChain ( chain ) ;
2018-02-23 06:00:48 +00:00
}
/// Before executing WHERE and HAVING, remove the extra columns from the block (mostly the aggregation keys).
2019-03-29 20:31:06 +00:00
if ( res . filter_info )
res . filter_info - > actions - > prependProjectInput ( ) ;
2018-02-23 06:00:48 +00:00
if ( res . has_where )
res . before_where - > prependProjectInput ( ) ;
if ( res . has_having )
res . before_having - > prependProjectInput ( ) ;
res . subqueries_for_sets = query_analyzer - > getSubqueriesForSets ( ) ;
2018-10-11 20:52:25 +00:00
/// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows.
if ( res . prewhere_info )
{
auto check_actions = [ ] ( const ExpressionActionsPtr & actions )
{
if ( actions )
for ( const auto & action : actions - > getActions ( ) )
if ( action . type = = ExpressionAction : : Type : : JOIN | | action . type = = ExpressionAction : : Type : : ARRAY_JOIN )
throw Exception ( " PREWHERE cannot contain ARRAY JOIN or JOIN action " , ErrorCodes : : ILLEGAL_PREWHERE ) ;
} ;
check_actions ( res . prewhere_info - > prewhere_actions ) ;
check_actions ( res . prewhere_info - > alias_actions ) ;
check_actions ( res . prewhere_info - > remove_columns_actions ) ;
}
2018-02-23 06:00:48 +00:00
return res ;
}
2018-08-27 17:42:13 +00:00
void InterpreterSelectQuery : : executeImpl ( Pipeline & pipeline , const BlockInputStreamPtr & prepared_input , bool dry_run )
2011-08-28 05:13:24 +00:00
{
2017-04-02 17:37:49 +00:00
/** Streams of data. When the query is executed in parallel, we have several data streams.
* If there is no GROUP BY , then perform all operations before ORDER BY and LIMIT in parallel , then
* if there is an ORDER BY , then glue the streams using UnionBlockInputStream , and then MergeSortingBlockInputStream ,
* if not , then glue it using UnionBlockInputStream ,
* then apply LIMIT .
* If there is GROUP BY , then we will perform all operations up to GROUP BY , inclusive , in parallel ;
* a parallel GROUP BY will glue streams into one ,
* then perform the remaining operations with one resulting stream .
2017-04-01 07:20:54 +00:00
*/
2019-03-12 14:07:02 +00:00
/// Now we will compose block streams that perform the necessary actions.
auto & query = getSelectQuery ( ) ;
2018-04-19 15:18:26 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2018-04-19 14:47:09 +00:00
2018-04-19 15:18:26 +00:00
QueryProcessingStage : : Enum from_stage = QueryProcessingStage : : FetchColumns ;
2018-03-16 02:08:31 +00:00
2018-04-19 15:18:26 +00:00
/// PREWHERE optimization
2019-03-29 20:31:06 +00:00
/// Turn off, if the table filter is applied.
if ( storage & & ! context . hasUserProperty ( storage - > getDatabaseName ( ) , storage - > getTableName ( ) , " filter " ) )
2018-04-19 15:18:26 +00:00
{
2018-04-06 13:58:06 +00:00
if ( ! dry_run )
2018-04-19 15:18:26 +00:00
from_stage = storage - > getQueryProcessingStage ( context ) ;
2018-04-20 19:51:21 +00:00
query_analyzer - > makeSetsForIndex ( ) ;
2018-04-19 15:18:26 +00:00
auto optimize_prewhere = [ & ] ( auto & merge_tree )
{
SelectQueryInfo query_info ;
query_info . query = query_ptr ;
2018-11-08 15:43:14 +00:00
query_info . syntax_analyzer_result = syntax_analyzer_result ;
2018-04-19 15:18:26 +00:00
query_info . sets = query_analyzer - > getPreparedSets ( ) ;
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
2019-04-09 14:22:35 +00:00
if ( settings . optimize_move_to_prewhere & & query . where ( ) & & ! query . prewhere ( ) & & ! query . final ( ) )
2019-05-03 02:00:57 +00:00
MergeTreeWhereOptimizer { query_info , context , merge_tree , query_analyzer - > getRequiredSourceColumns ( ) , log } ;
2018-04-19 15:18:26 +00:00
} ;
2019-05-03 02:00:57 +00:00
if ( const MergeTreeData * merge_tree_data = dynamic_cast < const MergeTreeData * > ( storage . get ( ) ) )
optimize_prewhere ( * merge_tree_data ) ;
2018-04-06 13:58:06 +00:00
}
2018-02-23 06:00:48 +00:00
2018-07-19 13:36:21 +00:00
AnalysisResult expressions ;
2019-03-29 20:31:06 +00:00
FilterInfoPtr filter_info ;
/// We need proper `source_header` for `NullBlockInputStream` in dry-run.
if ( storage & & context . hasUserProperty ( storage - > getDatabaseName ( ) , storage - > getTableName ( ) , " filter " ) )
{
filter_info = std : : make_shared < FilterInfo > ( ) ;
filter_info - > column_name = generateFilterActions ( filter_info - > actions , storage , context , required_columns ) ;
source_header = storage - > getSampleBlockForColumns ( filter_info - > actions - > getRequiredColumns ( ) ) ;
}
2018-07-19 13:36:21 +00:00
if ( dry_run )
{
pipeline . streams . emplace_back ( std : : make_shared < NullBlockInputStream > ( source_header ) ) ;
2019-03-29 20:31:06 +00:00
expressions = analyzeExpressions ( QueryProcessingStage : : FetchColumns , true , filter_info ) ;
if ( storage & & expressions . filter_info & & expressions . prewhere_info )
throw Exception ( " PREWHERE is not supported if the table is filtered by row-level security expression " , ErrorCodes : : ILLEGAL_PREWHERE ) ;
2018-08-20 12:57:31 +00:00
if ( expressions . prewhere_info )
pipeline . streams . back ( ) = std : : make_shared < FilterBlockInputStream > (
pipeline . streams . back ( ) , expressions . prewhere_info - > prewhere_actions ,
2018-09-10 03:59:48 +00:00
expressions . prewhere_info - > prewhere_column_name , expressions . prewhere_info - > remove_prewhere_column ) ;
2018-07-19 13:36:21 +00:00
}
else
{
2018-08-27 17:42:13 +00:00
if ( prepared_input )
pipeline . streams . push_back ( prepared_input ) ;
2018-08-08 03:09:59 +00:00
2019-03-29 20:31:06 +00:00
expressions = analyzeExpressions ( from_stage , false , filter_info ) ;
2018-03-16 02:08:31 +00:00
2018-08-05 07:05:36 +00:00
if ( from_stage = = QueryProcessingStage : : WithMergeableState & &
2019-03-18 12:05:51 +00:00
options . to_stage = = QueryProcessingStage : : WithMergeableState )
2018-07-19 13:36:21 +00:00
throw Exception ( " Distributed on Distributed is not supported " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2017-04-01 07:20:54 +00:00
2019-03-29 20:31:06 +00:00
if ( storage & & expressions . filter_info & & expressions . prewhere_info )
throw Exception ( " PREWHERE is not supported if the table is filtered by row-level security expression " , ErrorCodes : : ILLEGAL_PREWHERE ) ;
2018-08-05 07:05:36 +00:00
/** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */
2018-11-06 13:55:41 +00:00
executeFetchColumns ( from_stage , pipeline , expressions . prewhere_info , expressions . columns_to_remove_after_prewhere ) ;
2017-04-01 07:20:54 +00:00
2019-03-18 12:05:51 +00:00
LOG_TRACE ( log , QueryProcessingStage : : toString ( from_stage ) < < " -> " < < QueryProcessingStage : : toString ( options . to_stage ) ) ;
2018-07-19 13:36:21 +00:00
}
2018-02-23 06:00:48 +00:00
2019-03-18 12:05:51 +00:00
if ( options . to_stage > QueryProcessingStage : : FetchColumns )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
2017-04-01 07:20:54 +00:00
bool aggregate_overflow_row =
2018-02-23 06:00:48 +00:00
expressions . need_aggregate & &
2017-04-01 07:20:54 +00:00
query . group_by_with_totals & &
2018-03-11 00:15:26 +00:00
settings . max_rows_to_group_by & &
settings . group_by_overflow_mode = = OverflowMode : : ANY & &
2017-04-01 07:20:54 +00:00
settings . totals_mode ! = TotalsMode : : AFTER_HAVING_EXCLUSIVE ;
2017-04-02 17:37:49 +00:00
/// Do I need to immediately finalize the aggregate functions after the aggregation?
2017-04-01 07:20:54 +00:00
bool aggregate_final =
2018-02-23 06:00:48 +00:00
expressions . need_aggregate & &
2019-03-18 12:05:51 +00:00
options . to_stage > QueryProcessingStage : : WithMergeableState & &
2018-09-17 18:01:04 +00:00
! query . group_by_with_totals & & ! query . group_by_with_rollup & & ! query . group_by_with_cube ;
2017-04-01 07:20:54 +00:00
2018-02-23 06:00:48 +00:00
if ( expressions . first_stage )
2017-04-01 07:20:54 +00:00
{
2019-03-29 20:31:06 +00:00
if ( expressions . filter_info )
{
pipeline . transform ( [ & ] ( auto & stream )
{
stream = std : : make_shared < FilterBlockInputStream > (
stream ,
expressions . filter_info - > actions ,
expressions . filter_info - > column_name ,
expressions . filter_info - > do_remove_column ) ;
} ) ;
}
2019-01-30 12:01:00 +00:00
if ( expressions . hasJoin ( ) )
2018-02-21 08:16:01 +00:00
{
2019-03-15 16:14:13 +00:00
const auto & join = query . join ( ) - > table_join - > as < ASTTableJoin & > ( ) ;
2019-03-11 11:14:09 +00:00
if ( isRightOrFull ( join . kind ) )
2018-02-23 06:00:48 +00:00
pipeline . stream_with_non_joined_data = expressions . before_join - > createStreamWithNonJoinedDataIfFullOrRightJoin (
2018-02-21 08:16:01 +00:00
pipeline . firstStream ( ) - > getHeader ( ) , settings . max_block_size ) ;
2018-02-21 03:26:06 +00:00
for ( auto & stream : pipeline . streams ) /// Applies to all sources except stream_with_non_joined_data.
2018-02-23 06:00:48 +00:00
stream = std : : make_shared < ExpressionBlockInputStream > ( stream , expressions . before_join ) ;
2018-02-21 08:16:01 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-23 06:00:48 +00:00
if ( expressions . has_where )
2018-04-12 09:45:24 +00:00
executeWhere ( pipeline , expressions . before_where , expressions . remove_where_filter ) ;
2017-04-01 07:20:54 +00:00
2018-02-23 06:00:48 +00:00
if ( expressions . need_aggregate )
executeAggregation ( pipeline , expressions . before_aggregation , aggregate_overflow_row , aggregate_final ) ;
2017-04-01 07:20:54 +00:00
else
{
2018-02-23 06:00:48 +00:00
executeExpression ( pipeline , expressions . before_order_and_select ) ;
executeDistinct ( pipeline , true , expressions . selected_columns ) ;
2017-04-01 07:20:54 +00:00
}
2017-04-02 17:37:49 +00:00
/** For distributed query processing,
* if no GROUP , HAVING set ,
* but there is an ORDER or LIMIT ,
* then we will perform the preliminary sorting and LIMIT on the remote server .
2017-04-01 07:20:54 +00:00
*/
2018-02-23 06:00:48 +00:00
if ( ! expressions . second_stage & & ! expressions . need_aggregate & & ! expressions . has_having )
2017-04-01 07:20:54 +00:00
{
2018-02-23 06:00:48 +00:00
if ( expressions . has_order_by )
2018-02-21 03:26:06 +00:00
executeOrder ( pipeline ) ;
2017-04-01 07:20:54 +00:00
2019-04-09 14:22:35 +00:00
if ( expressions . has_order_by & & query . limitLength ( ) )
2018-02-23 06:00:48 +00:00
executeDistinct ( pipeline , false , expressions . selected_columns ) ;
2017-04-01 07:20:54 +00:00
2019-03-15 15:22:22 +00:00
if ( expressions . has_limit_by )
{
executeExpression ( pipeline , expressions . before_limit_by ) ;
executeLimitBy ( pipeline ) ;
}
2019-04-09 14:22:35 +00:00
if ( query . limitLength ( ) )
2018-02-21 03:26:06 +00:00
executePreLimit ( pipeline ) ;
2017-04-01 07:20:54 +00:00
}
2018-10-10 17:07:21 +00:00
2019-01-22 19:56:53 +00:00
// If there is no global subqueries, we can run subqueries only when receive them on server.
2018-10-10 17:07:21 +00:00
if ( ! query_analyzer - > hasGlobalSubqueries ( ) & & ! expressions . subqueries_for_sets . empty ( ) )
executeSubqueriesInSetsAndJoins ( pipeline , expressions . subqueries_for_sets ) ;
2017-04-01 07:20:54 +00:00
}
2018-02-23 06:00:48 +00:00
if ( expressions . second_stage )
2017-04-01 07:20:54 +00:00
{
2018-02-25 06:34:20 +00:00
bool need_second_distinct_pass = false ;
bool need_merge_streams = false ;
2017-04-01 07:20:54 +00:00
2018-02-23 06:00:48 +00:00
if ( expressions . need_aggregate )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/// If you need to combine aggregated results from multiple servers
2018-02-23 06:00:48 +00:00
if ( ! expressions . first_stage )
2018-02-21 03:26:06 +00:00
executeMergeAggregated ( pipeline , aggregate_overflow_row , aggregate_final ) ;
2017-04-01 07:20:54 +00:00
if ( ! aggregate_final )
2018-08-21 16:08:45 +00:00
{
if ( query . group_by_with_totals )
2018-09-17 19:16:51 +00:00
{
bool final = ! query . group_by_with_rollup & & ! query . group_by_with_cube ;
executeTotalsAndHaving ( pipeline , expressions . has_having , expressions . before_having , aggregate_overflow_row , final ) ;
}
2018-09-06 01:06:30 +00:00
2018-09-17 18:01:04 +00:00
if ( query . group_by_with_rollup )
2018-09-20 17:51:42 +00:00
executeRollupOrCube ( pipeline , Modificator : : ROLLUP ) ;
2018-11-23 18:52:00 +00:00
else if ( query . group_by_with_cube )
2018-09-20 17:51:42 +00:00
executeRollupOrCube ( pipeline , Modificator : : CUBE ) ;
2019-01-16 00:26:15 +00:00
2018-12-15 06:27:40 +00:00
if ( ( query . group_by_with_rollup | | query . group_by_with_cube ) & & expressions . has_having )
2019-01-16 00:26:15 +00:00
{
if ( query . group_by_with_totals )
throw Exception ( " WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2018-12-15 06:27:40 +00:00
executeHaving ( pipeline , expressions . before_having ) ;
2019-01-16 00:26:15 +00:00
}
2018-08-21 16:08:45 +00:00
}
2018-02-23 06:00:48 +00:00
else if ( expressions . has_having )
executeHaving ( pipeline , expressions . before_having ) ;
2017-04-01 07:20:54 +00:00
2018-02-23 06:00:48 +00:00
executeExpression ( pipeline , expressions . before_order_and_select ) ;
executeDistinct ( pipeline , true , expressions . selected_columns ) ;
2017-04-01 07:20:54 +00:00
2019-04-12 17:04:38 +00:00
need_second_distinct_pass = query . distinct & & pipeline . hasMixedStreams ( ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2019-04-12 17:04:38 +00:00
need_second_distinct_pass = query . distinct & & pipeline . hasMixedStreams ( ) ;
2017-04-01 07:20:54 +00:00
if ( query . group_by_with_totals & & ! aggregate_final )
2018-09-17 19:16:51 +00:00
{
bool final = ! query . group_by_with_rollup & & ! query . group_by_with_cube ;
executeTotalsAndHaving ( pipeline , expressions . has_having , expressions . before_having , aggregate_overflow_row , final ) ;
}
2018-08-24 15:00:00 +00:00
2018-12-15 06:27:40 +00:00
if ( ( query . group_by_with_rollup | | query . group_by_with_cube ) & & ! aggregate_final )
{
if ( query . group_by_with_rollup )
executeRollupOrCube ( pipeline , Modificator : : ROLLUP ) ;
else if ( query . group_by_with_cube )
executeRollupOrCube ( pipeline , Modificator : : CUBE ) ;
2019-01-16 00:26:15 +00:00
2018-12-15 06:27:40 +00:00
if ( expressions . has_having )
2019-01-16 00:26:15 +00:00
{
if ( query . group_by_with_totals )
throw Exception ( " WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2018-12-15 06:27:40 +00:00
executeHaving ( pipeline , expressions . before_having ) ;
2019-01-16 00:26:15 +00:00
}
2018-12-15 06:27:40 +00:00
}
2017-04-01 07:20:54 +00:00
}
2018-02-23 06:00:48 +00:00
if ( expressions . has_order_by )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/** If there is an ORDER BY for distributed query processing,
* but there is no aggregation , then on the remote servers ORDER BY was made
* - therefore , we merge the sorted streams from remote servers .
2017-04-01 07:20:54 +00:00
*/
2018-02-23 06:00:48 +00:00
if ( ! expressions . first_stage & & ! expressions . need_aggregate & & ! ( query . group_by_with_totals & & ! aggregate_final ) )
2018-02-21 03:26:06 +00:00
executeMergeSorted ( pipeline ) ;
2017-04-02 17:37:49 +00:00
else /// Otherwise, just sort.
2018-02-21 03:26:06 +00:00
executeOrder ( pipeline ) ;
2017-04-01 07:20:54 +00:00
}
2017-04-02 17:37:49 +00:00
/** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT,
2018-02-23 06:00:48 +00:00
* limiting the number of rows in each up to ` offset + limit ` .
*/
2019-04-09 14:22:35 +00:00
if ( query . limitLength ( ) & & pipeline . hasMoreThanOneStream ( ) & & ! query . distinct & & ! expressions . has_limit_by & & ! settings . extremes )
2018-02-25 06:34:20 +00:00
{
2018-02-21 03:26:06 +00:00
executePreLimit ( pipeline ) ;
2018-02-25 06:34:20 +00:00
}
2017-04-01 07:20:54 +00:00
2018-02-25 06:34:20 +00:00
if ( need_second_distinct_pass
2019-04-09 14:22:35 +00:00
| | query . limitLength ( )
| | query . limitBy ( )
2018-02-25 06:34:20 +00:00
| | pipeline . stream_with_non_joined_data )
{
need_merge_streams = true ;
}
2017-04-01 07:20:54 +00:00
2018-02-25 06:34:20 +00:00
if ( need_merge_streams )
2018-02-21 03:26:06 +00:00
executeUnion ( pipeline ) ;
2017-04-01 07:20:54 +00:00
2018-03-01 01:25:06 +00:00
/** If there was more than one stream,
* then DISTINCT needs to be performed once again after merging all streams .
*/
if ( need_second_distinct_pass )
2018-03-01 05:42:44 +00:00
executeDistinct ( pipeline , false , expressions . selected_columns ) ;
2017-04-01 07:20:54 +00:00
2018-03-01 06:07:04 +00:00
if ( expressions . has_limit_by )
2018-03-01 05:24:56 +00:00
{
executeExpression ( pipeline , expressions . before_limit_by ) ;
executeLimitBy ( pipeline ) ;
}
2018-03-01 01:25:06 +00:00
/** We must do projection after DISTINCT because projection may remove some columns.
*/
executeProjection ( pipeline , expressions . final_projection ) ;
/** Extremes are calculated before LIMIT, but after LIMIT BY. This is Ok.
*/
executeExtremes ( pipeline ) ;
executeLimit ( pipeline ) ;
2017-04-01 07:20:54 +00:00
}
}
2018-10-10 17:07:21 +00:00
if ( query_analyzer - > hasGlobalSubqueries ( ) & & ! expressions . subqueries_for_sets . empty ( ) )
2018-02-23 06:00:48 +00:00
executeSubqueriesInSetsAndJoins ( pipeline , expressions . subqueries_for_sets ) ;
2012-05-09 13:12:38 +00:00
}
2011-08-28 05:13:24 +00:00
2019-02-10 15:17:45 +00:00
static UInt64 getLimitUIntValue ( const ASTPtr & node , const Context & context )
{
const auto & [ field , type ] = evaluateConstantExpression ( node , context ) ;
2019-05-24 12:11:03 +00:00
if ( ! isNativeNumber ( type ) )
2019-02-10 15:17:45 +00:00
throw Exception ( " Illegal type " + type - > getName ( ) + " of LIMIT expression, must be numeric type " , ErrorCodes : : INVALID_LIMIT_EXPRESSION ) ;
Field converted = convertFieldToType ( field , DataTypeUInt64 ( ) ) ;
if ( converted . isNull ( ) )
throw Exception ( " The value " + applyVisitor ( FieldVisitorToString ( ) , field ) + " of LIMIT expression is not representable as UInt64 " , ErrorCodes : : INVALID_LIMIT_EXPRESSION ) ;
return converted . safeGet < UInt64 > ( ) ;
}
static std : : pair < UInt64 , UInt64 > getLimitLengthAndOffset ( const ASTSelectQuery & query , const Context & context )
2012-05-09 13:12:38 +00:00
{
2019-02-10 15:17:45 +00:00
UInt64 length = 0 ;
UInt64 offset = 0 ;
2019-04-09 14:22:35 +00:00
if ( query . limitLength ( ) )
2017-04-01 07:20:54 +00:00
{
2019-04-09 14:22:35 +00:00
length = getLimitUIntValue ( query . limitLength ( ) , context ) ;
if ( query . limitOffset ( ) )
offset = getLimitUIntValue ( query . limitOffset ( ) , context ) ;
2017-04-01 07:20:54 +00:00
}
2012-05-09 13:12:38 +00:00
2019-02-10 15:17:45 +00:00
return { length , offset } ;
}
2018-10-14 08:23:21 +00:00
2019-03-11 12:49:39 +00:00
static UInt64 getLimitForSorting ( const ASTSelectQuery & query , const Context & context )
2019-02-03 18:31:17 +00:00
{
2019-02-10 15:17:45 +00:00
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY.
2019-04-09 14:22:35 +00:00
if ( ! query . distinct & & ! query . limitBy ( ) )
2019-02-10 15:17:45 +00:00
{
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , context ) ;
return limit_length + limit_offset ;
2019-02-03 18:31:17 +00:00
}
2019-02-10 15:17:45 +00:00
return 0 ;
2019-02-03 18:31:17 +00:00
}
2018-08-05 07:05:36 +00:00
void InterpreterSelectQuery : : executeFetchColumns (
2018-11-06 13:55:41 +00:00
QueryProcessingStage : : Enum processing_stage , Pipeline & pipeline ,
const PrewhereInfoPtr & prewhere_info , const Names & columns_to_remove_after_prewhere )
2012-05-09 13:12:38 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2018-09-03 17:24:46 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2017-04-02 17:37:49 +00:00
/// Actions to calculate ALIAS if required.
2017-04-01 07:20:54 +00:00
ExpressionActionsPtr alias_actions ;
2019-03-14 15:20:51 +00:00
if ( storage )
2017-04-01 07:20:54 +00:00
{
2019-03-29 20:31:06 +00:00
/// Append columns from the table filter to required
if ( context . hasUserProperty ( storage - > getDatabaseName ( ) , storage - > getTableName ( ) , " filter " ) )
{
auto initial_required_columns = required_columns ;
ExpressionActionsPtr actions ;
generateFilterActions ( actions , storage , context , initial_required_columns ) ;
auto required_columns_from_filter = actions - > getRequiredColumns ( ) ;
for ( const auto & column : required_columns_from_filter )
{
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column ) )
required_columns . push_back ( column ) ;
}
}
/// Detect, if ALIAS columns are required for query execution
auto alias_columns_required = false ;
2019-03-14 15:20:51 +00:00
const ColumnsDescription & storage_columns = storage - > getColumns ( ) ;
for ( const auto & column_name : required_columns )
2017-04-01 07:20:54 +00:00
{
2019-03-14 15:20:51 +00:00
auto column_default = storage_columns . getDefault ( column_name ) ;
if ( column_default & & column_default - > kind = = ColumnDefaultKind : : Alias )
2017-04-01 07:20:54 +00:00
{
alias_columns_required = true ;
break ;
}
}
2019-03-29 20:31:06 +00:00
/// There are multiple sources of required columns:
/// - raw required columns,
/// - columns deduced from ALIAS columns,
/// - raw required columns from PREWHERE,
/// - columns deduced from ALIAS columns from PREWHERE.
/// PREWHERE is a special case, since we need to resolve it and pass directly to `IStorage::read()`
/// before any other executions.
2017-04-01 07:20:54 +00:00
if ( alias_columns_required )
{
2019-03-29 20:31:06 +00:00
NameSet required_columns_from_prewhere ; /// Set of all (including ALIAS) required columns for PREWHERE
NameSet required_aliases_from_prewhere ; /// Set of ALIAS required columns for PREWHERE
2018-09-03 17:24:46 +00:00
if ( prewhere_info )
{
2019-03-29 20:31:06 +00:00
/// Get some columns directly from PREWHERE expression actions
2019-01-04 12:10:00 +00:00
auto prewhere_required_columns = prewhere_info - > prewhere_actions - > getRequiredColumns ( ) ;
2019-03-29 20:31:06 +00:00
required_columns_from_prewhere . insert ( prewhere_required_columns . begin ( ) , prewhere_required_columns . end ( ) ) ;
2018-09-03 17:24:46 +00:00
}
2019-03-29 20:31:06 +00:00
/// Expression, that contains all raw required columns
ASTPtr required_columns_all_expr = std : : make_shared < ASTExpressionList > ( ) ;
2018-09-03 17:24:46 +00:00
2019-03-29 20:31:06 +00:00
/// Expression, that contains raw required columns for PREWHERE
ASTPtr required_columns_from_prewhere_expr = std : : make_shared < ASTExpressionList > ( ) ;
/// Sort out already known required columns between expressions,
/// also populate `required_aliases_from_prewhere`.
2017-04-01 07:20:54 +00:00
for ( const auto & column : required_columns )
{
2018-09-03 17:24:46 +00:00
ASTPtr column_expr ;
2019-03-14 15:20:51 +00:00
const auto column_default = storage_columns . getDefault ( column ) ;
bool is_alias = column_default & & column_default - > kind = = ColumnDefaultKind : : Alias ;
2018-09-03 17:24:46 +00:00
if ( is_alias )
2019-03-14 15:20:51 +00:00
column_expr = setAlias ( column_default - > expression - > clone ( ) , column ) ;
2018-09-03 17:24:46 +00:00
else
column_expr = std : : make_shared < ASTIdentifier > ( column ) ;
2019-03-29 20:31:06 +00:00
if ( required_columns_from_prewhere . count ( column ) )
2018-09-03 17:24:46 +00:00
{
2019-03-29 20:31:06 +00:00
required_columns_from_prewhere_expr - > children . emplace_back ( std : : move ( column_expr ) ) ;
2018-09-03 17:24:46 +00:00
if ( is_alias )
2019-03-29 20:31:06 +00:00
required_aliases_from_prewhere . insert ( column ) ;
2018-09-03 17:24:46 +00:00
}
2017-04-01 07:20:54 +00:00
else
2019-03-29 20:31:06 +00:00
required_columns_all_expr - > children . emplace_back ( std : : move ( column_expr ) ) ;
2017-04-01 07:20:54 +00:00
}
2019-03-29 20:31:06 +00:00
/// Columns, which we will get after prewhere and filter executions.
NamesAndTypesList required_columns_after_prewhere ;
NameSet required_columns_after_prewhere_set ;
/// Collect required columns from prewhere expression actions.
if ( prewhere_info )
2018-09-03 17:24:46 +00:00
{
2019-03-29 20:31:06 +00:00
NameSet columns_to_remove ( columns_to_remove_after_prewhere . begin ( ) , columns_to_remove_after_prewhere . end ( ) ) ;
Block prewhere_actions_result = prewhere_info - > prewhere_actions - > getSampleBlock ( ) ;
/// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards.
/// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure.
for ( const auto & column : prewhere_actions_result )
{
if ( prewhere_info - > remove_prewhere_column & & column . name = = prewhere_info - > prewhere_column_name )
continue ;
2018-09-03 17:24:46 +00:00
2019-03-29 20:31:06 +00:00
if ( columns_to_remove . count ( column . name ) )
continue ;
2018-11-06 13:55:41 +00:00
2019-03-29 20:31:06 +00:00
required_columns_all_expr - > children . emplace_back ( std : : make_shared < ASTIdentifier > ( column . name ) ) ;
required_columns_after_prewhere . emplace_back ( column . name , column . type ) ;
}
required_columns_after_prewhere_set
= ext : : map < NameSet > ( required_columns_after_prewhere , [ ] ( const auto & it ) { return it . name ; } ) ;
2018-09-03 17:24:46 +00:00
}
2019-03-29 20:31:06 +00:00
auto syntax_result = SyntaxAnalyzer ( context ) . analyze ( required_columns_all_expr , required_columns_after_prewhere , { } , storage ) ;
alias_actions = ExpressionAnalyzer ( required_columns_all_expr , syntax_result , context ) . getActions ( true ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
2017-04-01 07:20:54 +00:00
required_columns = alias_actions - > getRequiredColumns ( ) ;
2018-09-03 17:24:46 +00:00
2018-09-04 18:50:19 +00:00
/// Do not remove prewhere filter if it is a column which is used as alias.
2018-09-03 17:24:46 +00:00
if ( prewhere_info & & prewhere_info - > remove_prewhere_column )
if ( required_columns . end ( )
! = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , prewhere_info - > prewhere_column_name ) )
prewhere_info - > remove_prewhere_column = false ;
2018-09-04 18:50:19 +00:00
/// Remove columns which will be added by prewhere.
2019-03-29 20:31:06 +00:00
required_columns . erase ( std : : remove_if ( required_columns . begin ( ) , required_columns . end ( ) , [ & ] ( const String & name )
2018-09-03 17:24:46 +00:00
{
2019-03-29 20:31:06 +00:00
return ! ! required_columns_after_prewhere_set . count ( name ) ;
} ) , required_columns . end ( ) ) ;
2018-09-03 17:24:46 +00:00
if ( prewhere_info )
{
2018-09-04 18:50:19 +00:00
/// Don't remove columns which are needed to be aliased.
2018-09-06 09:52:22 +00:00
auto new_actions = std : : make_shared < ExpressionActions > ( prewhere_info - > prewhere_actions - > getRequiredColumnsWithTypes ( ) , context ) ;
2018-09-03 17:24:46 +00:00
for ( const auto & action : prewhere_info - > prewhere_actions - > getActions ( ) )
{
if ( action . type ! = ExpressionAction : : REMOVE_COLUMN
| | required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , action . source_name ) )
new_actions - > add ( action ) ;
}
prewhere_info - > prewhere_actions = std : : move ( new_actions ) ;
2019-03-29 20:31:06 +00:00
auto analyzed_result
= SyntaxAnalyzer ( context ) . analyze ( required_columns_from_prewhere_expr , storage - > getColumns ( ) . getAllPhysical ( ) ) ;
prewhere_info - > alias_actions
= ExpressionAnalyzer ( required_columns_from_prewhere_expr , analyzed_result , context ) . getActions ( true , false ) ;
2018-09-04 18:50:19 +00:00
2019-03-29 20:31:06 +00:00
/// Add (physical?) columns required by alias actions.
auto required_columns_from_alias = prewhere_info - > alias_actions - > getRequiredColumns ( ) ;
Block prewhere_actions_result = prewhere_info - > prewhere_actions - > getSampleBlock ( ) ;
for ( auto & column : required_columns_from_alias )
2018-09-03 17:24:46 +00:00
if ( ! prewhere_actions_result . has ( column ) )
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column ) )
required_columns . push_back ( column ) ;
2019-03-29 20:31:06 +00:00
/// Add physical columns required by prewhere actions.
for ( const auto & column : required_columns_from_prewhere )
if ( required_aliases_from_prewhere . count ( column ) = = 0 )
2018-09-03 17:24:46 +00:00
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column ) )
required_columns . push_back ( column ) ;
}
2017-04-01 07:20:54 +00:00
}
}
2017-05-24 20:25:01 +00:00
/// Limitation on the number of columns to read.
2018-07-19 13:36:21 +00:00
/// It's not applied in 'only_analyze' mode, because the query could be analyzed without removal of unnecessary columns.
2019-03-18 12:05:51 +00:00
if ( ! options . only_analyze & & settings . max_columns_to_read & & required_columns . size ( ) > settings . max_columns_to_read )
2017-05-24 20:25:01 +00:00
throw Exception ( " Limit for number of columns to read exceeded. "
" Requested: " + toString ( required_columns . size ( ) )
2018-03-11 00:15:26 +00:00
+ " , maximum: " + settings . max_columns_to_read . toString ( ) ,
2018-03-09 23:23:15 +00:00
ErrorCodes : : TOO_MANY_COLUMNS ) ;
2017-05-24 20:25:01 +00:00
2017-04-02 17:37:49 +00:00
/** With distributed query processing, almost no computations are done in the threads,
* but wait and receive data from remote servers .
* If we have 20 remote servers , and max_threads = 8 , then it would not be very good
* connect and ask only 8 servers at a time .
* To simultaneously query more remote servers ,
* instead of max_threads , max_distributed_connections is used .
2017-04-01 07:20:54 +00:00
*/
bool is_remote = false ;
if ( storage & & storage - > isRemote ( ) )
{
is_remote = true ;
2017-05-24 20:25:01 +00:00
max_streams = settings . max_distributed_connections ;
2017-04-01 07:20:54 +00:00
}
2019-02-10 16:55:12 +00:00
UInt64 max_block_size = settings . max_block_size ;
2018-08-25 16:32:44 +00:00
2019-02-10 15:17:45 +00:00
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , context ) ;
2017-04-02 17:37:49 +00:00
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size,
* then as the block size we will use limit + offset ( not to read more from the table than requested ) ,
* and also set the number of threads to 1.
2017-04-01 07:20:54 +00:00
*/
if ( ! query . distinct
2019-04-09 14:22:35 +00:00
& & ! query . prewhere ( )
& & ! query . where ( )
& & ! query . groupBy ( )
& & ! query . having ( )
& & ! query . orderBy ( )
& & ! query . limitBy ( )
& & query . limitLength ( )
2017-04-01 07:20:54 +00:00
& & ! query_analyzer - > hasAggregation ( )
2018-02-25 06:34:20 +00:00
& & limit_length + limit_offset < max_block_size )
2017-04-01 07:20:54 +00:00
{
2019-02-10 16:55:12 +00:00
max_block_size = std : : max ( UInt64 ( 1 ) , limit_length + limit_offset ) ;
2017-05-24 20:25:01 +00:00
max_streams = 1 ;
2017-04-01 07:20:54 +00:00
}
2019-02-10 16:55:12 +00:00
if ( ! max_block_size )
throw Exception ( " Setting 'max_block_size' cannot be zero " , ErrorCodes : : PARAMETER_OUT_OF_BOUND ) ;
2018-02-21 06:25:21 +00:00
/// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery or prepared input?
if ( ! pipeline . streams . empty ( ) )
{
/// Prepared input.
}
else if ( interpreter_subquery )
2017-04-01 07:20:54 +00:00
{
2018-02-21 06:25:21 +00:00
/// Subquery.
2018-07-19 13:36:21 +00:00
/// If we need less number of columns that subquery have - update the interpreter.
if ( required_columns . size ( ) < source_header . columns ( ) )
{
2019-01-15 18:29:54 +00:00
ASTPtr subquery = extractTableExpression ( query , 0 ) ;
2018-10-29 19:04:28 +00:00
if ( ! subquery )
throw Exception ( " Subquery expected " , ErrorCodes : : LOGICAL_ERROR ) ;
2018-07-19 13:36:21 +00:00
interpreter_subquery = std : : make_unique < InterpreterSelectWithUnionQuery > (
2019-03-18 12:05:51 +00:00
subquery , getSubqueryContext ( context ) ,
options . copy ( ) . subquery ( ) . noModify ( ) , required_columns ) ;
2018-07-19 13:36:21 +00:00
if ( query_analyzer - > hasAggregation ( ) )
interpreter_subquery - > ignoreWithTotals ( ) ;
}
pipeline . streams = interpreter_subquery - > executeWithMultipleStreams ( ) ;
2018-02-21 06:25:21 +00:00
}
else if ( storage )
{
/// Table.
2017-04-01 07:20:54 +00:00
if ( max_streams = = 0 )
throw Exception ( " Logical error: zero number of streams requested " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-04-02 17:37:49 +00:00
/// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads.
2017-04-01 07:20:54 +00:00
if ( max_streams > 1 & & ! is_remote )
max_streams * = settings . max_streams_to_max_threads_ratio ;
2017-07-15 03:48:36 +00:00
SelectQueryInfo query_info ;
query_info . query = query_ptr ;
2018-11-08 15:43:14 +00:00
query_info . syntax_analyzer_result = syntax_analyzer_result ;
2017-07-15 03:48:36 +00:00
query_info . sets = query_analyzer - > getPreparedSets ( ) ;
2018-04-19 14:47:09 +00:00
query_info . prewhere_info = prewhere_info ;
2017-07-15 03:48:36 +00:00
2018-08-05 07:05:36 +00:00
pipeline . streams = storage - > read ( required_columns , query_info , context , processing_stage , max_block_size , max_streams ) ;
2017-04-01 07:20:54 +00:00
2018-02-21 03:26:06 +00:00
if ( pipeline . streams . empty ( ) )
2018-04-16 12:21:36 +00:00
{
2019-03-29 20:31:06 +00:00
pipeline . streams = { std : : make_shared < NullBlockInputStream > ( storage - > getSampleBlockForColumns ( required_columns ) ) } ;
2018-01-07 00:35:44 +00:00
2018-04-16 12:21:36 +00:00
if ( query_info . prewhere_info )
2019-03-29 20:31:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
{
stream = std : : make_shared < FilterBlockInputStream > (
stream ,
prewhere_info - > prewhere_actions ,
prewhere_info - > prewhere_column_name ,
prewhere_info - > remove_prewhere_column ) ;
} ) ;
2018-04-16 12:21:36 +00:00
}
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
stream - > addTableLock ( table_lock ) ;
} ) ;
2018-04-02 18:01:25 +00:00
/// Set the limits and quota for reading data, the speed and time of the query.
2017-04-01 07:20:54 +00:00
{
2019-01-23 14:48:50 +00:00
IBlockInputStream : : LocalLimits limits ;
limits . mode = IBlockInputStream : : LIMITS_TOTAL ;
2018-03-11 00:15:26 +00:00
limits . size_limits = SizeLimits ( settings . max_rows_to_read , settings . max_bytes_to_read , settings . read_overflow_mode ) ;
limits . max_execution_time = settings . max_execution_time ;
limits . timeout_overflow_mode = settings . timeout_overflow_mode ;
2018-04-02 18:01:25 +00:00
/** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers,
* because the initiating server has a summary of the execution of the request on all servers .
*
* But limits on data size to read and maximum execution time are reasonable to check both on initiator and
* additionally on each remote server , because these limits are checked per block of data processed ,
* and remote servers may process way more blocks of data than are received by initiator .
*/
2019-03-18 12:05:51 +00:00
if ( options . to_stage = = QueryProcessingStage : : Complete )
2018-04-02 18:01:25 +00:00
{
limits . min_execution_speed = settings . min_execution_speed ;
2019-02-18 03:03:44 +00:00
limits . max_execution_speed = settings . max_execution_speed ;
2019-03-04 02:09:44 +00:00
limits . min_execution_speed_bytes = settings . min_execution_speed_bytes ;
limits . max_execution_speed_bytes = settings . max_execution_speed_bytes ;
2018-04-02 18:01:25 +00:00
limits . timeout_before_checking_execution_speed = settings . timeout_before_checking_execution_speed ;
}
2018-02-21 06:25:21 +00:00
QuotaForIntervals & quota = context . getQuota ( ) ;
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
2019-06-14 19:27:53 +00:00
if ( ! options . ignore_limits )
stream - > setLimits ( limits ) ;
2018-04-02 18:01:25 +00:00
2019-03-18 12:05:51 +00:00
if ( options . to_stage = = QueryProcessingStage : : Complete )
2019-01-23 14:48:50 +00:00
stream - > setQuota ( quota ) ;
2018-02-21 06:25:21 +00:00
} ) ;
}
2017-04-01 07:20:54 +00:00
}
2018-02-21 06:25:21 +00:00
else
throw Exception ( " Logical error in InterpreterSelectQuery: nowhere to read " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-04-01 07:20:54 +00:00
2018-02-26 21:00:42 +00:00
/// Aliases in table declaration.
2018-04-19 15:18:26 +00:00
if ( processing_stage = = QueryProcessingStage : : FetchColumns & & alias_actions )
2018-02-26 21:00:42 +00:00
{
pipeline . transform ( [ & ] ( auto & stream )
{
stream = std : : make_shared < ExpressionBlockInputStream > ( stream , alias_actions ) ;
} ) ;
}
2012-05-09 13:12:38 +00:00
}
2011-08-28 05:13:24 +00:00
2012-05-09 13:12:38 +00:00
2018-04-12 09:45:24 +00:00
void InterpreterSelectQuery : : executeWhere ( Pipeline & pipeline , const ExpressionActionsPtr & expression , bool remove_fiter )
2012-08-27 05:13:14 +00:00
{
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
2019-04-09 14:22:35 +00:00
stream = std : : make_shared < FilterBlockInputStream > ( stream , expression , getSelectQuery ( ) . where ( ) - > getColumnName ( ) , remove_fiter ) ;
2017-04-01 07:20:54 +00:00
} ) ;
2012-08-27 05:13:14 +00:00
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeAggregation ( Pipeline & pipeline , const ExpressionActionsPtr & expression , bool overflow_row , bool final )
2012-05-09 13:12:38 +00:00
{
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
2018-09-27 15:55:22 +00:00
stream = std : : make_shared < ExpressionBlockInputStream > ( stream , expression ) ;
2017-04-01 07:20:54 +00:00
} ) ;
Names key_names ;
AggregateDescriptions aggregates ;
query_analyzer - > getAggregateInfo ( key_names , aggregates ) ;
2018-02-21 03:26:06 +00:00
Block header = pipeline . firstStream ( ) - > getHeader ( ) ;
2018-01-06 18:10:44 +00:00
ColumnNumbers keys ;
for ( const auto & name : key_names )
keys . push_back ( header . getPositionByName ( name ) ) ;
for ( auto & descr : aggregates )
if ( descr . arguments . empty ( ) )
for ( const auto & name : descr . argument_names )
descr . arguments . push_back ( header . getPositionByName ( name ) ) ;
2017-05-24 20:13:04 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2017-04-02 17:37:49 +00:00
/** Two-level aggregation is useful in two cases:
2017-07-04 12:46:31 +00:00
* 1. Parallel aggregation is done , and the results should be merged in parallel .
* 2. An aggregation is done with store of temporary data on the disk , and they need to be merged in a memory efficient way .
2017-04-01 07:20:54 +00:00
*/
2018-03-11 00:15:26 +00:00
bool allow_to_use_two_level_group_by = pipeline . streams . size ( ) > 1 | | settings . max_bytes_before_external_group_by ! = 0 ;
2017-04-01 07:20:54 +00:00
2018-01-06 18:10:44 +00:00
Aggregator : : Params params ( header , keys , aggregates ,
2018-03-11 00:15:26 +00:00
overflow_row , settings . max_rows_to_group_by , settings . group_by_overflow_mode ,
2017-04-01 07:20:54 +00:00
settings . compile ? & context . getCompiler ( ) : nullptr , settings . min_count_to_compile ,
allow_to_use_two_level_group_by ? settings . group_by_two_level_threshold : SettingUInt64 ( 0 ) ,
allow_to_use_two_level_group_by ? settings . group_by_two_level_threshold_bytes : SettingUInt64 ( 0 ) ,
2018-03-11 00:15:26 +00:00
settings . max_bytes_before_external_group_by , settings . empty_result_for_aggregation_by_empty_set ,
2018-09-12 13:27:00 +00:00
context . getTemporaryPath ( ) , settings . max_threads ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// If there are several sources, then we perform parallel aggregation
2018-02-21 03:26:06 +00:00
if ( pipeline . streams . size ( ) > 1 )
2017-04-01 07:20:54 +00:00
{
2018-02-21 03:26:06 +00:00
pipeline . firstStream ( ) = std : : make_shared < ParallelAggregatingBlockInputStream > (
pipeline . streams , pipeline . stream_with_non_joined_data , params , final ,
2017-07-03 21:04:10 +00:00
max_streams ,
2017-04-01 07:20:54 +00:00
settings . aggregation_memory_efficient_merge_threads
2017-07-03 21:04:10 +00:00
? static_cast < size_t > ( settings . aggregation_memory_efficient_merge_threads )
2017-07-25 19:43:23 +00:00
: static_cast < size_t > ( settings . max_threads ) ) ;
2017-04-01 07:20:54 +00:00
2018-02-21 03:26:06 +00:00
pipeline . stream_with_non_joined_data = nullptr ;
pipeline . streams . resize ( 1 ) ;
2017-04-01 07:20:54 +00:00
}
else
{
BlockInputStreams inputs ;
2018-02-21 03:26:06 +00:00
if ( ! pipeline . streams . empty ( ) )
inputs . push_back ( pipeline . firstStream ( ) ) ;
2017-04-01 07:20:54 +00:00
else
2018-02-21 03:26:06 +00:00
pipeline . streams . resize ( 1 ) ;
2017-04-01 07:20:54 +00:00
2018-02-21 03:26:06 +00:00
if ( pipeline . stream_with_non_joined_data )
inputs . push_back ( pipeline . stream_with_non_joined_data ) ;
2017-04-01 07:20:54 +00:00
2018-02-21 03:26:06 +00:00
pipeline . firstStream ( ) = std : : make_shared < AggregatingBlockInputStream > ( std : : make_shared < ConcatBlockInputStream > ( inputs ) , params , final ) ;
2017-04-01 07:20:54 +00:00
2018-02-21 03:26:06 +00:00
pipeline . stream_with_non_joined_data = nullptr ;
2017-04-01 07:20:54 +00:00
}
2012-05-09 13:12:38 +00:00
}
2011-11-06 22:00:39 +00:00
2012-05-09 13:12:38 +00:00
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeMergeAggregated ( Pipeline & pipeline , bool overflow_row , bool final )
2012-05-30 01:38:02 +00:00
{
2017-04-01 07:20:54 +00:00
Names key_names ;
AggregateDescriptions aggregates ;
query_analyzer - > getAggregateInfo ( key_names , aggregates ) ;
2018-02-21 03:26:06 +00:00
Block header = pipeline . firstStream ( ) - > getHeader ( ) ;
2018-01-07 00:35:44 +00:00
2018-01-06 18:10:44 +00:00
ColumnNumbers keys ;
for ( const auto & name : key_names )
keys . push_back ( header . getPositionByName ( name ) ) ;
2017-04-02 17:37:49 +00:00
/** There are two modes of distributed aggregation.
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* 1. In different threads read from the remote servers blocks .
* Save all the blocks in the RAM . Merge blocks .
* If the aggregation is two - level - parallelize to the number of buckets .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* 2. In one thread , read blocks from different servers in order .
* RAM stores only one block from each server .
* If the aggregation is a two - level aggregation , we consistently merge the blocks of each next level .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* The second option consumes less memory ( up to 256 times less )
* in the case of two - level aggregation , which is used for large results after GROUP BY ,
* but it can work more slowly .
2017-04-01 07:20:54 +00:00
*/
2017-05-24 20:13:04 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2018-09-12 13:27:00 +00:00
Aggregator : : Params params ( header , keys , aggregates , overflow_row , settings . max_threads ) ;
2017-04-01 07:20:54 +00:00
if ( ! settings . distributed_aggregation_memory_efficient )
{
2017-04-02 17:37:49 +00:00
/// We union several sources into one, parallelizing the work.
2018-02-21 03:26:06 +00:00
executeUnion ( pipeline ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Now merge the aggregated blocks
2018-02-21 03:26:06 +00:00
pipeline . firstStream ( ) = std : : make_shared < MergingAggregatedBlockInputStream > ( pipeline . firstStream ( ) , params , final , settings . max_threads ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2018-02-21 03:26:06 +00:00
pipeline . firstStream ( ) = std : : make_shared < MergingAggregatedMemoryEfficientBlockInputStream > ( pipeline . streams , params , final ,
2017-07-03 21:04:10 +00:00
max_streams ,
2017-04-01 07:20:54 +00:00
settings . aggregation_memory_efficient_merge_threads
2017-07-25 19:43:23 +00:00
? static_cast < size_t > ( settings . aggregation_memory_efficient_merge_threads )
: static_cast < size_t > ( settings . max_threads ) ) ;
2017-04-01 07:20:54 +00:00
2018-02-21 03:26:06 +00:00
pipeline . streams . resize ( 1 ) ;
2017-04-01 07:20:54 +00:00
}
2012-05-09 13:12:38 +00:00
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeHaving ( Pipeline & pipeline , const ExpressionActionsPtr & expression )
2012-05-09 13:12:38 +00:00
{
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
2019-04-09 14:22:35 +00:00
stream = std : : make_shared < FilterBlockInputStream > ( stream , expression , getSelectQuery ( ) . having ( ) - > getColumnName ( ) ) ;
2017-04-01 07:20:54 +00:00
} ) ;
2012-05-09 13:12:38 +00:00
}
2011-09-25 03:37:09 +00:00
2018-08-24 15:00:00 +00:00
void InterpreterSelectQuery : : executeTotalsAndHaving ( Pipeline & pipeline , bool has_having , const ExpressionActionsPtr & expression , bool overflow_row , bool final )
2014-02-27 12:49:21 +00:00
{
2018-02-21 03:26:06 +00:00
executeUnion ( pipeline ) ;
2014-12-19 12:48:09 +00:00
2017-05-24 20:13:04 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2018-02-21 03:26:06 +00:00
pipeline . firstStream ( ) = std : : make_shared < TotalsHavingBlockInputStream > (
2019-03-11 12:49:39 +00:00
pipeline . firstStream ( ) ,
overflow_row ,
expression ,
2019-04-09 14:22:35 +00:00
has_having ? getSelectQuery ( ) . having ( ) - > getColumnName ( ) : " " ,
2019-03-11 12:49:39 +00:00
settings . totals_mode ,
settings . totals_auto_threshold ,
final ) ;
2014-02-27 12:49:21 +00:00
}
2018-09-20 17:51:42 +00:00
void InterpreterSelectQuery : : executeRollupOrCube ( Pipeline & pipeline , Modificator modificator )
2018-08-21 16:08:45 +00:00
{
executeUnion ( pipeline ) ;
Names key_names ;
AggregateDescriptions aggregates ;
query_analyzer - > getAggregateInfo ( key_names , aggregates ) ;
Block header = pipeline . firstStream ( ) - > getHeader ( ) ;
2018-08-24 15:00:00 +00:00
2018-08-21 16:08:45 +00:00
ColumnNumbers keys ;
for ( const auto & name : key_names )
keys . push_back ( header . getPositionByName ( name ) ) ;
2018-09-06 01:06:30 +00:00
2018-08-21 16:08:45 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2018-09-06 01:06:30 +00:00
2018-08-21 16:08:45 +00:00
Aggregator : : Params params ( header , keys , aggregates ,
false , settings . max_rows_to_group_by , settings . group_by_overflow_mode ,
settings . compile ? & context . getCompiler ( ) : nullptr , settings . min_count_to_compile ,
2018-08-24 17:37:08 +00:00
SettingUInt64 ( 0 ) , SettingUInt64 ( 0 ) ,
2018-08-21 16:08:45 +00:00
settings . max_bytes_before_external_group_by , settings . empty_result_for_aggregation_by_empty_set ,
2018-09-14 09:42:38 +00:00
context . getTemporaryPath ( ) , settings . max_threads ) ;
2018-08-21 16:08:45 +00:00
2018-09-20 17:51:42 +00:00
if ( modificator = = Modificator : : ROLLUP )
2018-09-17 18:01:04 +00:00
pipeline . firstStream ( ) = std : : make_shared < RollupBlockInputStream > ( pipeline . firstStream ( ) , params ) ;
2018-09-20 15:46:37 +00:00
else
2018-09-17 18:01:04 +00:00
pipeline . firstStream ( ) = std : : make_shared < CubeBlockInputStream > ( pipeline . firstStream ( ) , params ) ;
2014-02-27 12:49:21 +00:00
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeExpression ( Pipeline & pipeline , const ExpressionActionsPtr & expression )
2012-05-09 13:12:38 +00:00
{
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
stream = std : : make_shared < ExpressionBlockInputStream > ( stream , expression ) ;
} ) ;
2012-05-09 13:12:38 +00:00
}
2019-03-11 12:49:39 +00:00
static SortDescription getSortDescription ( const ASTSelectQuery & query )
2012-05-09 13:12:38 +00:00
{
2017-04-01 07:20:54 +00:00
SortDescription order_descr ;
2019-04-09 14:22:35 +00:00
order_descr . reserve ( query . orderBy ( ) - > children . size ( ) ) ;
for ( const auto & elem : query . orderBy ( ) - > children )
2017-04-01 07:20:54 +00:00
{
String name = elem - > children . front ( ) - > getColumnName ( ) ;
2019-03-15 16:14:13 +00:00
const auto & order_by_elem = elem - > as < ASTOrderByElement & > ( ) ;
2015-01-07 15:30:11 +00:00
2017-04-01 07:20:54 +00:00
std : : shared_ptr < Collator > collator ;
if ( order_by_elem . collation )
2019-03-15 16:14:13 +00:00
collator = std : : make_shared < Collator > ( order_by_elem . collation - > as < ASTLiteral & > ( ) . value . get < String > ( ) ) ;
2016-11-20 12:43:20 +00:00
2017-04-01 07:20:54 +00:00
order_descr . emplace_back ( name , order_by_elem . direction , order_by_elem . nulls_direction , collator ) ;
}
2014-12-19 12:48:09 +00:00
2017-04-01 07:20:54 +00:00
return order_descr ;
2015-01-18 08:27:28 +00:00
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeOrder ( Pipeline & pipeline )
2015-01-18 08:27:28 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2017-04-01 07:20:54 +00:00
SortDescription order_descr = getSortDescription ( query ) ;
2019-02-10 15:17:45 +00:00
UInt64 limit = getLimitForSorting ( query , context ) ;
2017-04-01 07:20:54 +00:00
2017-05-24 20:13:04 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
auto sorting_stream = std : : make_shared < PartialSortingBlockInputStream > ( stream , order_descr , limit ) ;
2017-04-02 17:37:49 +00:00
/// Limits on sorting
2019-01-23 14:48:50 +00:00
IBlockInputStream : : LocalLimits limits ;
limits . mode = IBlockInputStream : : LIMITS_TOTAL ;
2018-03-11 00:15:26 +00:00
limits . size_limits = SizeLimits ( settings . max_rows_to_sort , settings . max_bytes_to_sort , settings . sort_overflow_mode ) ;
2017-04-01 07:20:54 +00:00
sorting_stream - > setLimits ( limits ) ;
stream = sorting_stream ;
} ) ;
2017-04-02 17:37:49 +00:00
/// If there are several streams, we merge them into one
2018-02-21 03:26:06 +00:00
executeUnion ( pipeline ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Merge the sorted blocks.
2018-02-21 03:26:06 +00:00
pipeline . firstStream ( ) = std : : make_shared < MergeSortingBlockInputStream > (
pipeline . firstStream ( ) , order_descr , settings . max_block_size , limit ,
2018-09-24 20:07:30 +00:00
settings . max_bytes_before_remerge_sort ,
2018-03-11 00:15:26 +00:00
settings . max_bytes_before_external_sort , context . getTemporaryPath ( ) ) ;
2013-06-03 10:18:41 +00:00
}
2012-02-27 06:28:20 +00:00
2012-07-25 20:29:22 +00:00
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeMergeSorted ( Pipeline & pipeline )
2015-01-18 08:27:28 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2017-04-01 07:20:54 +00:00
SortDescription order_descr = getSortDescription ( query ) ;
2019-02-10 15:17:45 +00:00
UInt64 limit = getLimitForSorting ( query , context ) ;
2017-04-01 07:20:54 +00:00
2017-05-24 20:13:04 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2017-04-02 17:37:49 +00:00
/// If there are several streams, then we merge them into one
2018-02-21 03:26:06 +00:00
if ( pipeline . hasMoreThanOneStream ( ) )
2017-04-01 07:20:54 +00:00
{
2018-11-06 11:44:45 +00:00
unifyStreams ( pipeline ) ;
2018-11-06 10:11:37 +00:00
2017-04-02 17:37:49 +00:00
/** MergingSortedBlockInputStream reads the sources sequentially.
* To make the data on the remote servers prepared in parallel , we wrap it in AsynchronousBlockInputStream .
2017-04-01 07:20:54 +00:00
*/
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
stream = std : : make_shared < AsynchronousBlockInputStream > ( stream ) ;
} ) ;
2017-04-02 17:37:49 +00:00
/// Merge the sorted sources into one sorted source.
2018-02-21 03:26:06 +00:00
pipeline . firstStream ( ) = std : : make_shared < MergingSortedBlockInputStream > ( pipeline . streams , order_descr , settings . max_block_size , limit ) ;
pipeline . streams . resize ( 1 ) ;
2017-04-01 07:20:54 +00:00
}
2015-01-18 08:27:28 +00:00
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeProjection ( Pipeline & pipeline , const ExpressionActionsPtr & expression )
2013-06-03 10:18:41 +00:00
{
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
stream = std : : make_shared < ExpressionBlockInputStream > ( stream , expression ) ;
} ) ;
2012-05-09 13:12:38 +00:00
}
2012-02-27 06:28:20 +00:00
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeDistinct ( Pipeline & pipeline , bool before_order , Names columns )
2013-06-01 07:43:57 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2017-04-01 07:20:54 +00:00
if ( query . distinct )
{
2017-05-24 20:13:04 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2019-02-10 15:17:45 +00:00
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , context ) ;
UInt64 limit_for_distinct = 0 ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows.
2019-04-09 14:22:35 +00:00
if ( ! query . orderBy ( ) | | ! before_order )
2017-04-01 07:20:54 +00:00
limit_for_distinct = limit_length + limit_offset ;
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
2018-03-11 00:15:26 +00:00
SizeLimits limits ( settings . max_rows_in_distinct , settings . max_bytes_in_distinct , settings . distinct_overflow_mode ) ;
2018-05-31 18:28:04 +00:00
stream = std : : make_shared < DistinctBlockInputStream > ( stream , limits , limit_for_distinct , columns ) ;
2017-04-01 07:20:54 +00:00
} ) ;
}
2013-06-01 07:43:57 +00:00
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeUnion ( Pipeline & pipeline )
2012-05-09 13:12:38 +00:00
{
2017-04-02 17:37:49 +00:00
/// If there are still several streams, then we combine them into one
2018-02-21 03:26:06 +00:00
if ( pipeline . hasMoreThanOneStream ( ) )
2017-04-01 07:20:54 +00:00
{
2018-11-06 11:44:45 +00:00
unifyStreams ( pipeline ) ;
2018-10-23 17:57:27 +00:00
2018-11-28 14:33:40 +00:00
pipeline . firstStream ( ) = std : : make_shared < UnionBlockInputStream > ( pipeline . streams , pipeline . stream_with_non_joined_data , max_streams ) ;
2018-02-21 03:26:06 +00:00
pipeline . stream_with_non_joined_data = nullptr ;
pipeline . streams . resize ( 1 ) ;
2019-04-12 17:04:38 +00:00
pipeline . union_stream = true ;
2017-04-01 07:20:54 +00:00
}
2018-02-21 03:26:06 +00:00
else if ( pipeline . stream_with_non_joined_data )
2017-04-01 07:20:54 +00:00
{
2018-02-21 03:26:06 +00:00
pipeline . streams . push_back ( pipeline . stream_with_non_joined_data ) ;
pipeline . stream_with_non_joined_data = nullptr ;
2017-04-01 07:20:54 +00:00
}
2012-05-09 13:12:38 +00:00
}
2017-04-02 17:37:49 +00:00
/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined.
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executePreLimit ( Pipeline & pipeline )
2012-05-09 13:12:38 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2017-04-02 17:37:49 +00:00
/// If there is LIMIT
2019-04-09 14:22:35 +00:00
if ( query . limitLength ( ) )
2017-04-01 07:20:54 +00:00
{
2019-02-10 15:17:45 +00:00
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , context ) ;
pipeline . transform ( [ & , limit = limit_length + limit_offset ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
2019-02-10 15:17:45 +00:00
stream = std : : make_shared < LimitBlockInputStream > ( stream , limit , 0 , false ) ;
2017-04-01 07:20:54 +00:00
} ) ;
}
2012-06-24 23:17:06 +00:00
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeLimitBy ( Pipeline & pipeline )
2016-12-28 15:12:54 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2019-04-29 13:12:39 +00:00
if ( ! query . limitByLength ( ) | | ! query . limitBy ( ) )
2017-04-01 07:20:54 +00:00
return ;
Names columns ;
2019-04-09 14:22:35 +00:00
for ( const auto & elem : query . limitBy ( ) - > children )
2018-03-01 05:24:56 +00:00
columns . emplace_back ( elem - > getColumnName ( ) ) ;
2019-04-29 13:12:39 +00:00
UInt64 length = getLimitUIntValue ( query . limitByLength ( ) , context ) ;
UInt64 offset = ( query . limitByOffset ( ) ? getLimitUIntValue ( query . limitByOffset ( ) , context ) : 0 ) ;
2017-04-01 07:20:54 +00:00
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
2019-04-29 13:12:39 +00:00
stream = std : : make_shared < LimitByBlockInputStream > ( stream , length , offset , columns ) ;
2017-04-01 07:20:54 +00:00
} ) ;
2016-12-28 15:12:54 +00:00
}
2019-03-29 20:31:06 +00:00
// TODO: move to anonymous namespace
2018-02-26 06:12:59 +00:00
bool hasWithTotalsInAnySubqueryInFromClause ( const ASTSelectQuery & query )
{
if ( query . group_by_with_totals )
return true ;
/** NOTE You can also check that the table in the subquery is distributed, and that it only looks at one shard.
* In other cases , totals will be computed on the initiating server of the query , and it is not necessary to read the data to the end .
*/
2019-01-15 18:29:54 +00:00
if ( auto query_table = extractTableExpression ( query , 0 ) )
2018-02-26 06:12:59 +00:00
{
2019-03-11 13:22:51 +00:00
if ( const auto * ast_union = query_table - > as < ASTSelectWithUnionQuery > ( ) )
2018-02-26 06:12:59 +00:00
{
for ( const auto & elem : ast_union - > list_of_selects - > children )
2019-03-15 16:14:13 +00:00
if ( hasWithTotalsInAnySubqueryInFromClause ( elem - > as < ASTSelectQuery & > ( ) ) )
2018-02-26 06:12:59 +00:00
return true ;
}
}
return false ;
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeLimit ( Pipeline & pipeline )
2012-06-24 23:17:06 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2017-04-02 17:37:49 +00:00
/// If there is LIMIT
2019-04-09 14:22:35 +00:00
if ( query . limitLength ( ) )
2017-04-01 07:20:54 +00:00
{
2017-04-02 17:37:49 +00:00
/** Rare case:
* if there is no WITH TOTALS and there is a subquery in FROM , and there is WITH TOTALS on one of the levels ,
* then when using LIMIT , you should read the data to the end , rather than cancel the query earlier ,
* because if you cancel the query , we will not get ` totals ` data from the remote server .
2017-04-01 07:20:54 +00:00
*
2017-04-02 17:37:49 +00:00
* Another case :
* if there is WITH TOTALS and there is no ORDER BY , then read the data to the end ,
* otherwise TOTALS is counted according to incomplete data .
2017-04-01 07:20:54 +00:00
*/
bool always_read_till_end = false ;
2019-04-09 14:22:35 +00:00
if ( query . group_by_with_totals & & ! query . orderBy ( ) )
2017-04-01 07:20:54 +00:00
always_read_till_end = true ;
2018-02-26 06:12:59 +00:00
if ( ! query . group_by_with_totals & & hasWithTotalsInAnySubqueryInFromClause ( query ) )
always_read_till_end = true ;
2017-04-01 07:20:54 +00:00
2019-02-10 15:17:45 +00:00
UInt64 limit_length ;
UInt64 limit_offset ;
std : : tie ( limit_length , limit_offset ) = getLimitLengthAndOffset ( query , context ) ;
2018-02-21 03:26:06 +00:00
pipeline . transform ( [ & ] ( auto & stream )
2017-04-01 07:20:54 +00:00
{
stream = std : : make_shared < LimitBlockInputStream > ( stream , limit_length , limit_offset , always_read_till_end ) ;
} ) ;
}
2011-08-28 05:13:24 +00:00
}
2018-02-28 02:32:34 +00:00
void InterpreterSelectQuery : : executeExtremes ( Pipeline & pipeline )
{
if ( ! context . getSettingsRef ( ) . extremes )
return ;
pipeline . transform ( [ & ] ( auto & stream )
{
2019-01-23 14:48:50 +00:00
stream - > enableExtremes ( ) ;
2018-02-28 02:32:34 +00:00
} ) ;
}
2018-02-21 03:26:06 +00:00
void InterpreterSelectQuery : : executeSubqueriesInSetsAndJoins ( Pipeline & pipeline , SubqueriesForSets & subqueries_for_sets )
2014-03-04 11:26:55 +00:00
{
2018-02-21 03:26:06 +00:00
executeUnion ( pipeline ) ;
2018-03-11 00:15:26 +00:00
pipeline . firstStream ( ) = std : : make_shared < CreatingSetsBlockInputStream > (
2019-02-27 18:26:24 +00:00
pipeline . firstStream ( ) , subqueries_for_sets , context ) ;
2016-11-12 17:55:40 +00:00
}
2018-11-06 11:44:45 +00:00
void InterpreterSelectQuery : : unifyStreams ( Pipeline & pipeline )
{
if ( pipeline . hasMoreThanOneStream ( ) )
{
/// Unify streams in case they have different headers.
auto first_header = pipeline . streams . at ( 0 ) - > getHeader ( ) ;
for ( size_t i = 1 ; i < pipeline . streams . size ( ) ; + + i )
{
auto & stream = pipeline . streams [ i ] ;
auto header = stream - > getHeader ( ) ;
auto mode = ConvertingBlockInputStream : : MatchColumnsMode : : Name ;
if ( ! blocksHaveEqualStructure ( first_header , header ) )
stream = std : : make_shared < ConvertingBlockInputStream > ( context , stream , first_header , mode ) ;
}
}
}
2014-03-04 11:26:55 +00:00
2015-05-06 23:35:37 +00:00
void InterpreterSelectQuery : : ignoreWithTotals ( )
{
2019-03-12 14:07:02 +00:00
getSelectQuery ( ) . group_by_with_totals = false ;
2015-05-06 23:35:37 +00:00
}
2015-06-05 21:28:04 +00:00
void InterpreterSelectQuery : : initSettings ( )
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2019-04-09 14:22:35 +00:00
if ( query . settings ( ) )
InterpreterSetQuery ( query . settings ( ) , context ) . executeForCurrentContext ( ) ;
2015-06-05 21:28:04 +00:00
}
2011-08-28 05:13:24 +00:00
}