2019-11-05 20:22:20 +00:00
# include <DataStreams/OneBlockInputStream.h>
2020-06-17 20:19:55 +00:00
# include <DataStreams/materializeBlock.h>
# include <DataTypes/DataTypeAggregateFunction.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-05-03 02:00:57 +00:00
# include <Parsers/ExpressionListParsers.h>
# include <Parsers/parseQuery.h>
2017-04-01 09:19:00 +00:00
2020-01-26 09:49:53 +00:00
# include <Access/AccessFlags.h>
2019-11-17 11:57:02 +00:00
2020-09-29 16:32:41 +00:00
# include <Interpreters/ApplyWithAliasVisitor.h>
2020-09-12 17:00:04 +00:00
# include <Interpreters/ApplyWithSubqueryVisitor.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>
2020-04-28 19:41:23 +00:00
# include <Interpreters/addTypeConversionToAST.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ExpressionAnalyzer.h>
2019-12-12 15:28:24 +00:00
# include <Interpreters/getTableExpressions.h>
2019-03-11 12:20:55 +00:00
# include <Interpreters/JoinToSubqueryTransformVisitor.h>
# include <Interpreters/CrossToInnerJoinVisitor.h>
2020-04-07 09:48:47 +00:00
# include <Interpreters/TableJoin.h>
2020-09-18 16:25:20 +00:00
# include <Interpreters/JoinSwitcher.h>
2020-02-28 15:23:32 +00:00
# include <Interpreters/JoinedTables.h>
2020-03-18 21:38:27 +00:00
# include <Interpreters/QueryAliasesVisitor.h>
2017-04-01 09:19:00 +00:00
2020-06-17 20:19:55 +00:00
# include <Processors/Pipe.h>
2019-03-26 18:28:37 +00:00
# include <Processors/Sources/SourceFromInputStream.h>
2020-09-18 14:16:53 +00:00
# include <Processors/Sources/NullSource.h>
2019-03-26 18:28:37 +00:00
# include <Processors/Transforms/ExpressionTransform.h>
2020-09-08 10:40:53 +00:00
# include <Processors/Transforms/JoiningTransform.h>
2019-03-26 18:28:37 +00:00
# include <Processors/Transforms/AggregatingTransform.h>
2020-09-18 14:16:53 +00:00
# include <Processors/Transforms/FilterTransform.h>
2020-08-11 12:03:18 +00:00
# include <Processors/QueryPlan/ArrayJoinStep.h>
2020-09-18 11:39:07 +00:00
# include <Processors/QueryPlan/SettingQuotaAndLimitsStep.h>
2020-06-16 09:42:25 +00:00
# include <Processors/QueryPlan/ExpressionStep.h>
# include <Processors/QueryPlan/FilterStep.h>
2020-06-16 14:11:19 +00:00
# include <Processors/QueryPlan/ReadNothingStep.h>
# include <Processors/QueryPlan/ReadFromPreparedSource.h>
2020-06-16 14:45:52 +00:00
# include <Processors/QueryPlan/PartialSortingStep.h>
2020-06-16 15:08:01 +00:00
# include <Processors/QueryPlan/MergeSortingStep.h>
2020-06-16 15:47:40 +00:00
# include <Processors/QueryPlan/MergingSortedStep.h>
2020-06-16 16:13:07 +00:00
# include <Processors/QueryPlan/DistinctStep.h>
2020-06-16 16:49:49 +00:00
# include <Processors/QueryPlan/LimitByStep.h>
2020-06-16 18:23:01 +00:00
# include <Processors/QueryPlan/LimitStep.h>
2020-06-16 18:57:21 +00:00
# include <Processors/QueryPlan/MergingAggregatedStep.h>
2020-06-22 10:21:13 +00:00
# include <Processors/QueryPlan/AddingDelayedSourceStep.h>
2020-06-17 14:21:48 +00:00
# include <Processors/QueryPlan/AggregatingStep.h>
2020-06-17 15:50:21 +00:00
# include <Processors/QueryPlan/CreatingSetsStep.h>
2020-06-17 16:54:51 +00:00
# include <Processors/QueryPlan/TotalsHavingStep.h>
2020-06-17 17:15:24 +00:00
# include <Processors/QueryPlan/RollupStep.h>
# include <Processors/QueryPlan/CubeStep.h>
2020-06-17 19:20:30 +00:00
# include <Processors/QueryPlan/FillingStep.h>
2020-06-17 19:30:36 +00:00
# include <Processors/QueryPlan/ExtremesStep.h>
2020-06-24 12:09:01 +00:00
# include <Processors/QueryPlan/OffsetStep.h>
2020-06-17 19:57:13 +00:00
# include <Processors/QueryPlan/FinishSortingStep.h>
2020-06-18 13:38:45 +00:00
# include <Processors/QueryPlan/QueryPlan.h>
2019-03-26 18:28:37 +00:00
2020-06-17 20:19:55 +00:00
# include <Storages/MergeTree/MergeTreeData.h>
# include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
# include <Storages/IStorage.h>
# include <Storages/StorageView.h>
# include <Functions/IFunction.h>
# include <Core/Field.h>
2020-09-15 09:55:57 +00:00
# include <common/types.h>
2020-06-17 20:19:55 +00:00
# include <Columns/Collator.h>
# include <Common/FieldVisitorsAccurateComparison.h>
# include <Common/typeid_cast.h>
# include <Common/checkStackSize.h>
# include <ext/map.h>
# include <ext/scope_guard.h>
# include <memory>
2020-06-18 17:45:00 +00:00
# include <Processors/QueryPlan/ConvertingStep.h>
2020-06-17 20:19:55 +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 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 ;
2019-02-03 18:31:17 +00:00
extern const int INVALID_LIMIT_EXPRESSION ;
2019-04-21 03:36:59 +00:00
extern const int INVALID_WITH_FILL_EXPRESSION ;
2016-01-11 21:46:36 +00:00
}
2019-10-05 19:34:25 +00:00
/// Assumes `storage` is set and the table filter (row-level security) is not empty.
2020-06-13 20:59:20 +00:00
String InterpreterSelectQuery : : generateFilterActions (
2020-11-03 11:28:28 +00:00
ActionsDAGPtr & actions , const ASTPtr & row_policy_filter , const Names & prerequisite_columns ) const
2019-03-29 20:31:06 +00:00
{
2019-12-30 18:20:43 +00:00
const auto & db_name = table_id . getDatabaseName ( ) ;
const auto & table_name = table_id . getTableName ( ) ;
2019-03-29 20:31:06 +00:00
/// 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
// The first column is our filter expression.
2019-11-17 11:57:02 +00:00
expr_list - > children . push_back ( row_policy_filter ) ;
2019-03-29 20:31:06 +00:00
/// Keep columns that are required after the filter actions.
for ( const auto & column_str : prerequisite_columns )
2019-11-17 11:57:02 +00:00
{
ParserExpression expr_parser ;
2020-04-15 20:28:05 +00:00
expr_list - > children . push_back ( parseQuery ( expr_parser , column_str , 0 , context - > getSettingsRef ( ) . max_parser_depth ) ) ;
2019-11-17 11:57:02 +00:00
}
2019-03-29 20:31:06 +00:00
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
2020-07-22 17:13:05 +00:00
auto syntax_result = TreeRewriter ( * context ) . analyzeSelect ( query_ast , TreeRewriterResult ( { } , storage , metadata_snapshot ) ) ;
2020-06-17 09:38:47 +00:00
SelectQueryExpressionAnalyzer analyzer ( query_ast , syntax_result , * context , metadata_snapshot ) ;
2020-02-10 15:50:12 +00:00
actions = analyzer . simpleSelectActions ( ) ;
2019-03-29 20:31:06 +00:00
return expr_list - > children . at ( 0 ) - > getColumnName ( ) ;
}
2018-02-26 21:00:42 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
const Context & context_ ,
2019-08-03 11:02:40 +00:00
const SelectQueryOptions & options_ ,
const Names & required_result_column_names_ )
2020-01-29 15:04:46 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , nullptr , std : : nullopt , 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-08-03 11:02:40 +00:00
const SelectQueryOptions & options_ )
2020-01-29 15:04:46 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , input_ , std : : nullopt , nullptr , options_ . copy ( ) . noSubquery ( ) )
{ }
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
const Context & context_ ,
Pipe input_pipe_ ,
const SelectQueryOptions & options_ )
: InterpreterSelectQuery ( query_ptr_ , context_ , nullptr , std : : move ( input_pipe_ ) , 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_ ,
2020-06-17 11:52:19 +00:00
const StorageMetadataPtr & metadata_snapshot_ ,
2019-08-03 11:02:40 +00:00
const SelectQueryOptions & options_ )
2020-06-17 11:52:19 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , nullptr , std : : nullopt , storage_ , options_ . copy ( ) . noSubquery ( ) , { } , metadata_snapshot_ )
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).
2020-03-09 00:08:02 +00:00
subquery_settings . extremes = false ;
2018-07-19 13:36:21 +00:00
subquery_context . setSettings ( subquery_settings ) ;
return subquery_context ;
2018-02-26 21:00:42 +00:00
}
2016-01-11 21:46:36 +00:00
2020-09-30 20:11:49 +00:00
static void rewriteMultipleJoins ( ASTPtr & query , const TablesWithColumns & tables , const String & database )
2020-06-15 12:36:10 +00:00
{
ASTSelectQuery & select = query - > as < ASTSelectQuery & > ( ) ;
Aliases aliases ;
if ( ASTPtr with = select . with ( ) )
QueryAliasesNoSubqueriesVisitor ( aliases ) . visit ( with ) ;
QueryAliasesNoSubqueriesVisitor ( aliases ) . visit ( select . select ( ) ) ;
CrossToInnerJoinVisitor : : Data cross_to_inner { tables , aliases , database } ;
CrossToInnerJoinVisitor ( cross_to_inner ) . visit ( query ) ;
2020-09-30 20:11:49 +00:00
JoinToSubqueryTransformVisitor : : Data join_to_subs_data { tables , aliases } ;
2020-06-15 12:36:10 +00:00
JoinToSubqueryTransformVisitor ( join_to_subs_data ) . visit ( query ) ;
}
2018-07-17 13:09:33 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
const Context & context_ ,
const BlockInputStreamPtr & input_ ,
2020-01-29 15:04:46 +00:00
std : : optional < Pipe > input_pipe_ ,
2018-07-17 13:09:33 +00:00
const StoragePtr & storage_ ,
2019-03-18 12:05:51 +00:00
const SelectQueryOptions & options_ ,
2020-06-17 11:52:19 +00:00
const Names & required_result_column_names ,
const StorageMetadataPtr & metadata_snapshot_ )
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 ( ) )
2019-11-15 18:41:18 +00:00
, context ( std : : make_shared < Context > ( context_ ) )
2018-07-17 13:09:33 +00:00
, storage ( storage_ )
, input ( input_ )
2020-01-29 15:04:46 +00:00
, input_pipe ( std : : move ( input_pipe_ ) )
2020-05-30 21:57:37 +00:00
, log ( & Poco : : Logger : : get ( " InterpreterSelectQuery " ) )
2020-06-17 11:52:19 +00:00
, metadata_snapshot ( metadata_snapshot_ )
2014-12-26 10:35:03 +00:00
{
2019-08-10 17:51:47 +00:00
checkStackSize ( ) ;
2017-04-01 07:20:54 +00:00
initSettings ( ) ;
2019-11-15 18:41:18 +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 ) ;
2020-03-08 11:07:05 +00:00
bool has_input = input | | input_pipe ;
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
}
2020-01-29 15:04:46 +00:00
else if ( input_pipe )
{
/// Read from prepared input.
2020-02-28 15:11:18 +00:00
source_header = input_pipe - > getHeader ( ) ;
2020-01-29 15:04:46 +00:00
}
2018-07-19 13:36:21 +00:00
2020-10-08 09:06:04 +00:00
if ( context - > getSettingsRef ( ) . enable_global_with_statement )
2020-09-29 16:32:41 +00:00
ApplyWithAliasVisitor ( ) . visit ( query_ptr ) ;
2020-09-12 17:00:04 +00:00
ApplyWithSubqueryVisitor ( ) . visit ( query_ptr ) ;
2020-03-08 11:07:05 +00:00
JoinedTables joined_tables ( getSubqueryContext ( * context ) , getSelectQuery ( ) ) ;
if ( ! has_input & & ! storage )
storage = joined_tables . getLeftTableStorage ( ) ;
if ( storage )
{
2020-06-18 16:10:47 +00:00
table_lock = storage - > lockForShare ( context - > getInitialQueryId ( ) , context - > getSettingsRef ( ) . lock_acquire_timeout ) ;
2020-03-08 11:07:05 +00:00
table_id = storage - > getStorageID ( ) ;
2020-06-17 11:52:19 +00:00
if ( metadata_snapshot = = nullptr )
metadata_snapshot = storage - > getInMemoryMetadataPtr ( ) ;
2018-02-28 01:29:55 +00:00
}
2020-03-08 11:07:05 +00:00
if ( has_input | | ! joined_tables . resolveTables ( ) )
2020-06-17 16:39:58 +00:00
joined_tables . makeFakeTable ( storage , metadata_snapshot , source_header ) ;
2020-03-08 11:07:05 +00:00
/// Rewrite JOINs
if ( ! has_input & & joined_tables . tablesCount ( ) > 1 )
2018-02-28 01:29:55 +00:00
{
2020-09-30 20:11:49 +00:00
rewriteMultipleJoins ( query_ptr , joined_tables . tablesWithColumns ( ) , context - > getCurrentDatabase ( ) ) ;
2020-03-18 21:38:27 +00:00
2020-06-15 12:36:10 +00:00
joined_tables . reset ( getSelectQuery ( ) ) ;
2020-03-08 11:07:05 +00:00
joined_tables . resolveTables ( ) ;
if ( storage & & joined_tables . isLeftTableSubquery ( ) )
2018-07-18 12:17:48 +00:00
{
2020-06-15 12:36:10 +00:00
/// Rewritten with subquery. Free storage locks here.
2020-03-08 11:07:05 +00:00
storage = { } ;
2020-06-18 16:10:47 +00:00
table_lock . reset ( ) ;
2020-03-08 11:07:05 +00:00
table_id = StorageID : : createEmpty ( ) ;
2018-07-18 12:17:48 +00:00
}
2017-04-01 07:20:54 +00:00
}
2020-03-08 11:07:05 +00:00
if ( ! has_input )
2019-12-05 11:42:13 +00:00
{
2020-03-08 11:07:05 +00:00
interpreter_subquery = joined_tables . makeLeftTableSubquery ( options . subquery ( ) ) ;
if ( interpreter_subquery )
source_header = interpreter_subquery - > getSampleBlock ( ) ;
2020-03-02 19:39:39 +00:00
}
2020-03-08 11:07:05 +00:00
2020-03-27 20:12:14 +00:00
joined_tables . rewriteDistributedInAndJoins ( query_ptr ) ;
2020-03-08 11:07:05 +00:00
max_streams = settings . max_threads ;
ASTSelectQuery & query = getSelectQuery ( ) ;
2020-04-08 18:59:52 +00:00
std : : shared_ptr < TableJoin > table_join = joined_tables . makeTableJoin ( query ) ;
2020-02-29 11:45:06 +00:00
2020-05-20 09:29:23 +00:00
ASTPtr row_policy_filter ;
if ( storage )
row_policy_filter = context - > getRowPolicyCondition ( table_id . getDatabaseName ( ) , table_id . getTableName ( ) , RowPolicy : : SELECT_FILTER ) ;
2020-06-15 12:36:10 +00:00
StorageView * view = nullptr ;
if ( storage )
view = dynamic_cast < StorageView * > ( storage . get ( ) ) ;
2020-10-13 14:33:34 +00:00
SubqueriesForSets subquery_for_sets ;
2020-05-20 09:29:23 +00:00
auto analyze = [ & ] ( bool try_move_to_prewhere )
2019-10-27 18:12:40 +00:00
{
2020-06-15 12:36:10 +00:00
/// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it.
ASTPtr view_table ;
if ( view )
2020-06-17 14:06:22 +00:00
view - > replaceWithSubquery ( getSelectQuery ( ) , view_table , metadata_snapshot ) ;
2020-06-15 12:36:10 +00:00
2020-07-22 17:13:05 +00:00
syntax_analyzer_result = TreeRewriter ( * context ) . analyzeSelect (
2020-06-17 16:39:58 +00:00
query_ptr ,
2020-07-22 17:13:05 +00:00
TreeRewriterResult ( source_header . getNamesAndTypesList ( ) , storage , metadata_snapshot ) ,
2020-06-17 16:39:58 +00:00
options , joined_tables . tablesWithColumns ( ) , required_result_column_names , table_join ) ;
2019-10-19 20:36:35 +00:00
2020-06-18 16:11:23 +00:00
/// Save scalar sub queries's results in the query context
if ( ! options . only_analyze & & context - > hasQueryContext ( ) )
for ( const auto & it : syntax_analyzer_result - > getScalars ( ) )
context - > getQueryContext ( ) . addScalar ( it . first , it . second ) ;
2020-06-15 12:36:10 +00:00
if ( view )
{
/// Restore original view name. Save rewritten subquery for future usage in StorageView.
query_info . view_query = view - > restoreViewName ( getSelectQuery ( ) , view_table ) ;
view = nullptr ;
}
2020-05-20 09:29:23 +00:00
if ( try_move_to_prewhere & & storage & & ! row_policy_filter & & query . where ( ) & & ! query . prewhere ( ) & & ! query . final ( ) )
{
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable
if ( const auto * merge_tree = dynamic_cast < const MergeTreeData * > ( storage . get ( ) ) )
{
SelectQueryInfo current_info ;
current_info . query = query_ptr ;
current_info . syntax_analyzer_result = syntax_analyzer_result ;
2020-06-17 12:39:20 +00:00
MergeTreeWhereOptimizer { current_info , * context , * merge_tree , metadata_snapshot , syntax_analyzer_result - > requiredSourceColumns ( ) , log } ;
2020-05-20 09:29:23 +00:00
}
}
2019-10-27 18:12:40 +00:00
query_analyzer = std : : make_unique < SelectQueryExpressionAnalyzer > (
2020-06-17 09:38:47 +00:00
query_ptr , syntax_analyzer_result , * context , metadata_snapshot ,
2019-10-27 18:12:40 +00:00
NameSet ( required_result_column_names . begin ( ) , required_result_column_names . end ( ) ) ,
2020-10-13 14:55:22 +00:00
! options . only_analyze , options , std : : move ( subquery_for_sets ) ) ;
2017-04-01 07:20:54 +00:00
2019-10-27 18:12:40 +00:00
if ( ! options . only_analyze )
{
2020-03-23 02:12:31 +00:00
if ( query . sampleSize ( ) & & ( input | | input_pipe | | ! storage | | ! storage - > supportsSampling ( ) ) )
2019-10-27 18:12:40 +00:00
throw Exception ( " Illegal SAMPLE: table doesn't support sampling " , ErrorCodes : : SAMPLING_NOT_SUPPORTED ) ;
2017-11-05 17:48:50 +00:00
2020-01-29 15:04:46 +00:00
if ( query . final ( ) & & ( input | | input_pipe | | ! storage | | ! storage - > supportsFinal ( ) ) )
throw Exception ( ( ! input & & ! input_pipe & & storage ) ? " Storage " + storage - > getName ( ) + " doesn't support FINAL " : " Illegal FINAL " , ErrorCodes : : ILLEGAL_FINAL ) ;
2017-11-05 17:48:50 +00:00
2020-01-29 15:04:46 +00:00
if ( query . prewhere ( ) & & ( input | | input_pipe | | ! storage | | ! storage - > supportsPrewhere ( ) ) )
throw Exception ( ( ! input & & ! input_pipe & & storage ) ? " Storage " + storage - > getName ( ) + " doesn't support PREWHERE " : " Illegal PREWHERE " , ErrorCodes : : ILLEGAL_PREWHERE ) ;
2017-11-05 17:48:50 +00:00
2019-10-27 18:12:40 +00:00
/// Save the new temporary tables in the query context
for ( const auto & it : query_analyzer - > getExternalTables ( ) )
2020-03-12 12:16:16 +00:00
if ( ! context - > tryResolveStorageID ( { " " , it . first } , Context : : ResolveExternal ) )
2020-03-10 19:36:17 +00:00
context - > addExternalTable ( it . first , std : : move ( * it . second ) ) ;
2019-10-27 18:12:40 +00:00
}
2018-03-04 16:15:31 +00:00
2019-10-27 18:12:40 +00:00
if ( ! options . only_analyze | | options . modify_inplace )
2018-09-29 11:29:23 +00:00
{
2019-10-27 18:12:40 +00:00
if ( syntax_analyzer_result - > rewrite_subqueries )
{
/// remake interpreter_subquery when PredicateOptimizer rewrites subqueries and main table is subquery
2020-03-08 11:07:05 +00:00
interpreter_subquery = joined_tables . makeLeftTableSubquery ( options . subquery ( ) ) ;
2019-10-27 18:12:40 +00:00
}
2018-09-29 11:29:23 +00:00
}
2018-07-19 13:36:21 +00:00
2019-10-27 18:12:40 +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 ( ) ;
}
2018-07-19 13:36:21 +00:00
2019-10-27 18:12:40 +00:00
required_columns = syntax_analyzer_result - > requiredSourceColumns ( ) ;
2018-07-19 13:36:21 +00:00
2019-10-27 18:12:40 +00:00
if ( storage )
2019-10-03 11:58:52 +00:00
{
2020-06-19 17:17:13 +00:00
source_header = metadata_snapshot - > getSampleBlockForColumns ( required_columns , storage - > getVirtuals ( ) , storage - > getStorageID ( ) ) ;
2019-10-27 18:12:40 +00:00
/// Fix source_header for filter actions.
2019-11-17 11:57:02 +00:00
if ( row_policy_filter )
2019-10-27 18:12:40 +00:00
{
filter_info = std : : make_shared < FilterInfo > ( ) ;
2019-12-27 12:47:29 +00:00
filter_info - > column_name = generateFilterActions ( filter_info - > actions , row_policy_filter , required_columns ) ;
2020-06-19 17:17:13 +00:00
source_header = metadata_snapshot - > getSampleBlockForColumns (
2020-11-03 11:28:28 +00:00
filter_info - > actions - > getRequiredColumns ( ) . getNames ( ) , storage - > getVirtuals ( ) , storage - > getStorageID ( ) ) ;
2019-10-27 18:12:40 +00:00
}
2019-10-03 11:58:52 +00:00
}
2019-11-03 07:27:35 +00:00
if ( ! options . only_analyze & & storage & & filter_info & & query . prewhere ( ) )
throw Exception ( " PREWHERE is not supported if the table is filtered by row-level security expression " , ErrorCodes : : ILLEGAL_PREWHERE ) ;
2019-10-27 18:12:40 +00:00
/// Calculate structure of the result.
2020-05-20 09:29:23 +00:00
result_header = getSampleBlockImpl ( ) ;
2019-10-27 18:12:40 +00:00
} ;
2020-05-20 09:29:23 +00:00
analyze ( settings . optimize_move_to_prewhere ) ;
2019-10-27 18:12:40 +00:00
bool need_analyze_again = false ;
if ( analysis_result . prewhere_constant_filter_description . always_false | | analysis_result . prewhere_constant_filter_description . always_true )
2019-08-12 16:20:02 +00:00
{
2019-10-27 18:12:40 +00:00
if ( analysis_result . prewhere_constant_filter_description . always_true )
2019-11-03 07:27:35 +00:00
query . setExpression ( ASTSelectQuery : : Expression : : PREWHERE , { } ) ;
else
query . setExpression ( ASTSelectQuery : : Expression : : PREWHERE , std : : make_shared < ASTLiteral > ( 0u ) ) ;
2019-10-27 18:12:40 +00:00
need_analyze_again = true ;
}
if ( analysis_result . where_constant_filter_description . always_false | | analysis_result . where_constant_filter_description . always_true )
{
if ( analysis_result . where_constant_filter_description . always_true )
2019-11-03 07:27:35 +00:00
query . setExpression ( ASTSelectQuery : : Expression : : WHERE , { } ) ;
else
query . setExpression ( ASTSelectQuery : : Expression : : WHERE , std : : make_shared < ASTLiteral > ( 0u ) ) ;
2019-10-27 18:12:40 +00:00
need_analyze_again = true ;
2019-08-12 16:20:02 +00:00
}
2019-11-15 03:38:35 +00:00
if ( query . prewhere ( ) & & query . where ( ) )
{
/// Filter block in WHERE instead to get better performance
query . setExpression ( ASTSelectQuery : : Expression : : WHERE , makeASTFunction ( " and " , query . prewhere ( ) - > clone ( ) , query . where ( ) - > clone ( ) ) ) ;
need_analyze_again = true ;
}
2020-02-05 16:42:27 +00:00
2019-10-27 18:12:40 +00:00
if ( need_analyze_again )
2020-02-05 16:42:27 +00:00
{
2020-10-13 14:33:34 +00:00
subquery_for_sets = std : : move ( query_analyzer - > getSubqueriesForSets ( ) ) ;
2020-02-05 16:42:27 +00:00
/// Do not try move conditions to PREWHERE for the second time.
/// Otherwise, we won't be able to fallback from inefficient PREWHERE to WHERE later.
analyze ( /* try_move_to_prewhere = */ false ) ;
}
2019-10-27 18:12:40 +00:00
2019-11-15 03:38:35 +00:00
/// If there is no WHERE, filter blocks as usual
if ( query . prewhere ( ) & & ! query . where ( ) )
analysis_result . prewhere_info - > need_filter = true ;
2020-03-12 12:16:16 +00:00
const StorageID & left_table_id = joined_tables . leftTableID ( ) ;
2020-03-02 19:39:39 +00:00
2020-03-12 12:16:16 +00:00
if ( left_table_id )
2020-03-13 15:41:36 +00:00
context - > checkAccess ( AccessType : : SELECT , left_table_id , required_columns ) ;
2019-11-11 01:11:32 +00:00
/// Remove limits for some tables in the `system` database.
2020-05-08 12:50:45 +00:00
if ( left_table_id . database_name = = " system " )
2019-11-11 01:11:32 +00:00
{
2020-05-08 12:50:45 +00:00
static const boost : : container : : flat_set < String > system_tables_ignoring_quota { " quotas " , " quota_limits " , " quota_usage " , " quotas_usage " , " one " } ;
if ( system_tables_ignoring_quota . count ( left_table_id . table_name ) )
{
options . ignore_quota = true ;
options . ignore_limits = true ;
}
2019-11-11 01:11:32 +00:00
}
2020-01-24 16:20:36 +00:00
/// Blocks used in expression analysis contains size 1 const columns for constant folding and
/// null non-const columns to avoid useless memory allocations. However, a valid block sample
/// requires all columns to be of size 0, thus we need to sanitize the block here.
2020-06-20 13:48:21 +00:00
sanitizeBlock ( result_header , true ) ;
2014-12-24 14:51:02 +00:00
}
2017-12-22 18:30:42 +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
}
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : buildQueryPlan ( QueryPlan & query_plan )
{
executeImpl ( query_plan , input , std : : move ( input_pipe ) ) ;
/// We must guarantee that result structure is the same as in getSampleBlock()
if ( ! blocksHaveEqualStructure ( query_plan . getCurrentDataStream ( ) . header , result_header ) )
{
2020-07-29 14:27:37 +00:00
auto converting = std : : make_unique < ConvertingStep > ( query_plan . getCurrentDataStream ( ) , result_header , true ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( converting ) ) ;
}
}
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
{
2019-11-15 18:41:18 +00:00
BlockIO res ;
2020-06-18 13:38:45 +00:00
QueryPlan query_plan ;
2020-06-18 17:45:00 +00:00
buildQueryPlan ( query_plan ) ;
2020-06-18 13:38:45 +00:00
res . pipeline = std : : move ( * query_plan . buildQueryPipeline ( ) ) ;
2017-04-01 07:20:54 +00:00
return res ;
2014-12-16 10:39:02 +00:00
}
2020-05-20 09:29:23 +00:00
Block InterpreterSelectQuery : : getSampleBlockImpl ( )
2019-08-07 13:41:36 +00:00
{
2020-09-10 19:55:36 +00:00
query_info . query = query_ptr ;
2019-10-03 15:47:42 +00:00
if ( storage & & ! options . only_analyze )
2020-09-10 19:55:36 +00:00
from_stage = storage - > getQueryProcessingStage ( * context , options . to_stage , query_info ) ;
2019-10-03 15:47:42 +00:00
2020-02-10 15:50:12 +00:00
/// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
bool first_stage = from_stage < QueryProcessingStage : : WithMergeableState
& & options . to_stage > = QueryProcessingStage : : WithMergeableState ;
/// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing.
bool second_stage = from_stage < = QueryProcessingStage : : WithMergeableState
& & options . to_stage > QueryProcessingStage : : WithMergeableState ;
analysis_result = ExpressionAnalysisResult (
2019-08-08 15:18:28 +00:00
* query_analyzer ,
2020-06-17 11:05:11 +00:00
metadata_snapshot ,
2020-02-10 15:50:12 +00:00
first_stage ,
second_stage ,
2019-10-03 11:58:52 +00:00
options . only_analyze ,
2019-10-27 18:12:40 +00:00
filter_info ,
2020-06-13 20:59:20 +00:00
source_header ) ;
2019-08-07 13:41:36 +00:00
2019-08-08 15:18:28 +00:00
if ( options . to_stage = = QueryProcessingStage : : Enum : : FetchColumns )
2019-08-07 13:41:36 +00:00
{
2019-08-08 15:18:28 +00:00
auto header = source_header ;
2019-08-07 13:41:36 +00:00
2019-08-08 15:18:28 +00:00
if ( analysis_result . prewhere_info )
2019-08-07 13:41:36 +00:00
{
2020-11-10 16:27:55 +00:00
ExpressionActions ( analysis_result . prewhere_info - > prewhere_actions ) . execute ( header ) ;
2019-08-07 13:41:36 +00:00
header = materializeBlock ( header ) ;
2019-08-08 15:18:28 +00:00
if ( analysis_result . prewhere_info - > remove_prewhere_column )
header . erase ( analysis_result . prewhere_info - > prewhere_column_name ) ;
2019-08-07 13:41:36 +00:00
}
return header ;
}
2019-08-08 15:18:28 +00:00
if ( options . to_stage = = QueryProcessingStage : : Enum : : WithMergeableState )
2019-08-07 13:41:36 +00:00
{
if ( ! analysis_result . need_aggregate )
2020-11-03 11:28:28 +00:00
return analysis_result . before_order_and_select - > getResultColumns ( ) ;
2019-08-07 13:41:36 +00:00
2020-11-03 11:28:28 +00:00
Block header = analysis_result . before_aggregation - > getResultColumns ( ) ;
2019-08-07 13:41:36 +00:00
Block res ;
2020-04-22 06:01:33 +00:00
for ( const auto & key : query_analyzer - > aggregationKeys ( ) )
2020-02-10 19:55:13 +00:00
res . insert ( { nullptr , header . getByName ( key . name ) . type , key . name } ) ;
2019-08-07 13:41:36 +00:00
2020-04-22 06:01:33 +00:00
for ( const auto & aggregate : query_analyzer - > aggregates ( ) )
2019-08-07 13:41:36 +00:00
{
size_t arguments_size = aggregate . argument_names . size ( ) ;
DataTypes argument_types ( arguments_size ) ;
for ( size_t j = 0 ; j < arguments_size ; + + j )
argument_types [ j ] = header . getByName ( aggregate . argument_names [ j ] ) . type ;
DataTypePtr type = std : : make_shared < DataTypeAggregateFunction > ( aggregate . function , argument_types , aggregate . parameters ) ;
res . insert ( { nullptr , type , aggregate . column_name } ) ;
}
return res ;
}
2020-08-15 13:57:17 +00:00
if ( options . to_stage = = QueryProcessingStage : : Enum : : WithMergeableStateAfterAggregation )
{
2020-11-03 11:28:28 +00:00
return analysis_result . before_order_and_select - > getResultColumns ( ) ;
2020-08-15 13:57:17 +00:00
}
2020-11-03 11:28:28 +00:00
return analysis_result . final_projection - > getResultColumns ( ) ;
2019-08-07 13:41:36 +00:00
}
2019-08-14 17:01:47 +00:00
static Field getWithFillFieldValue ( const ASTPtr & node , const Context & context )
{
const auto & [ field , type ] = evaluateConstantExpression ( node , context ) ;
2019-08-21 16:18:43 +00:00
if ( ! isColumnedAsNumber ( type ) )
2019-08-14 17:01:47 +00:00
throw Exception ( " Illegal type " + type - > getName ( ) + " of WITH FILL expression, must be numeric type " , ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ) ;
2018-02-23 06:00:48 +00:00
2019-08-14 17:01:47 +00:00
return field ;
}
2018-02-23 06:00:48 +00:00
2019-08-29 22:24:33 +00:00
static FillColumnDescription getWithFillDescription ( const ASTOrderByElement & order_by_elem , const Context & context )
2019-08-14 17:01:47 +00:00
{
FillColumnDescription descr ;
2019-08-19 20:22:45 +00:00
if ( order_by_elem . fill_from )
descr . fill_from = getWithFillFieldValue ( order_by_elem . fill_from , context ) ;
if ( order_by_elem . fill_to )
descr . fill_to = getWithFillFieldValue ( order_by_elem . fill_to , context ) ;
if ( order_by_elem . fill_step )
descr . fill_step = getWithFillFieldValue ( order_by_elem . fill_step , context ) ;
2019-08-14 17:01:47 +00:00
else
2019-08-19 20:22:45 +00:00
descr . fill_step = order_by_elem . direction ;
2019-08-14 17:01:47 +00:00
2019-08-19 20:22:45 +00:00
if ( applyVisitor ( FieldVisitorAccurateEquals ( ) , descr . fill_step , Field { 0 } ) )
throw Exception ( " WITH FILL STEP value cannot be zero " , ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ) ;
if ( order_by_elem . direction = = 1 )
{
if ( applyVisitor ( FieldVisitorAccurateLess ( ) , descr . fill_step , Field { 0 } ) )
throw Exception ( " WITH FILL STEP value cannot be negative for sorting in ascending direction " ,
ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ) ;
if ( ! descr . fill_from . isNull ( ) & & ! descr . fill_to . isNull ( ) & &
applyVisitor ( FieldVisitorAccurateLess ( ) , descr . fill_to , descr . fill_from ) )
{
throw Exception ( " WITH FILL TO value cannot be less than FROM value for sorting in ascending direction " ,
ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ) ;
}
}
else
{
if ( applyVisitor ( FieldVisitorAccurateLess ( ) , Field { 0 } , descr . fill_step ) )
throw Exception ( " WITH FILL STEP value cannot be positive for sorting in descending direction " ,
ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ) ;
if ( ! descr . fill_from . isNull ( ) & & ! descr . fill_to . isNull ( ) & &
applyVisitor ( FieldVisitorAccurateLess ( ) , descr . fill_from , descr . fill_to ) )
{
throw Exception ( " WITH FILL FROM value cannot be less than TO value for sorting in descending direction " ,
ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ) ;
}
}
2019-08-14 17:01:47 +00:00
return descr ;
}
2018-02-23 06:00:48 +00:00
2019-08-14 17:01:47 +00:00
static SortDescription getSortDescription ( const ASTSelectQuery & query , const Context & context )
2019-07-18 14:41:11 +00:00
{
SortDescription order_descr ;
order_descr . reserve ( query . orderBy ( ) - > children . size ( ) ) ;
for ( const auto & elem : query . orderBy ( ) - > children )
{
String name = elem - > children . front ( ) - > getColumnName ( ) ;
const auto & order_by_elem = elem - > as < ASTOrderByElement & > ( ) ;
std : : shared_ptr < Collator > collator ;
if ( order_by_elem . collation )
collator = std : : make_shared < Collator > ( order_by_elem . collation - > as < ASTLiteral & > ( ) . value . get < String > ( ) ) ;
2019-08-14 17:01:47 +00:00
if ( order_by_elem . with_fill )
{
FillColumnDescription fill_desc = getWithFillDescription ( order_by_elem , context ) ;
2019-08-19 20:22:45 +00:00
order_descr . emplace_back ( name , order_by_elem . direction ,
2020-09-19 14:20:00 +00:00
order_by_elem . nulls_direction , collator , true , fill_desc ) ;
2019-08-14 17:01:47 +00:00
}
else
2020-09-19 14:20:00 +00:00
order_descr . emplace_back ( name , order_by_elem . direction , order_by_elem . nulls_direction , collator ) ;
2019-07-18 14:41:11 +00:00
}
return order_descr ;
}
2020-05-14 14:20:49 +00:00
static SortDescription getSortDescriptionFromGroupBy ( const ASTSelectQuery & query )
2020-02-14 07:12:04 +00:00
{
2020-02-16 19:46:45 +00:00
SortDescription order_descr ;
order_descr . reserve ( query . groupBy ( ) - > children . size ( ) ) ;
2020-02-14 07:12:04 +00:00
for ( const auto & elem : query . groupBy ( ) - > children )
{
String name = elem - > getColumnName ( ) ;
2020-02-16 19:46:45 +00:00
order_descr . emplace_back ( name , 1 , 1 ) ;
2020-02-14 07:12:04 +00:00
}
2020-02-16 19:46:45 +00:00
return order_descr ;
2020-02-14 07:12:04 +00:00
}
2020-05-07 13:40:50 +00:00
static UInt64 getLimitUIntValue ( const ASTPtr & node , const Context & context , const std : : string & expr )
2019-07-18 14:41:11 +00:00
{
const auto & [ field , type ] = evaluateConstantExpression ( node , context ) ;
if ( ! isNativeNumber ( type ) )
2020-05-07 13:40:50 +00:00
throw Exception ( " Illegal type " + type - > getName ( ) + " of " + expr + " expression, must be numeric type " , ErrorCodes : : INVALID_LIMIT_EXPRESSION ) ;
2019-07-18 14:41:11 +00:00
Field converted = convertFieldToType ( field , DataTypeUInt64 ( ) ) ;
if ( converted . isNull ( ) )
2020-05-07 13:40:50 +00:00
throw Exception ( " The value " + applyVisitor ( FieldVisitorToString ( ) , field ) + " of " + expr + " expression is not representable as UInt64 " , ErrorCodes : : INVALID_LIMIT_EXPRESSION ) ;
2019-07-18 14:41:11 +00:00
return converted . safeGet < UInt64 > ( ) ;
}
static std : : pair < UInt64 , UInt64 > getLimitLengthAndOffset ( const ASTSelectQuery & query , const Context & context )
{
UInt64 length = 0 ;
UInt64 offset = 0 ;
if ( query . limitLength ( ) )
{
2020-05-07 13:40:50 +00:00
length = getLimitUIntValue ( query . limitLength ( ) , context , " LIMIT " ) ;
2019-09-17 18:55:59 +00:00
if ( query . limitOffset ( ) & & length )
2020-05-11 12:18:07 +00:00
offset = getLimitUIntValue ( query . limitOffset ( ) , context , " OFFSET " ) ;
2019-07-18 14:41:11 +00:00
}
2020-05-11 12:18:07 +00:00
else if ( query . limitOffset ( ) )
2020-05-07 13:40:50 +00:00
offset = getLimitUIntValue ( query . limitOffset ( ) , context , " OFFSET " ) ;
2019-07-18 14:41:11 +00:00
return { length , offset } ;
}
static UInt64 getLimitForSorting ( const ASTSelectQuery & query , const Context & context )
{
2020-04-22 04:32:05 +00:00
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY, neither ARRAY JOIN.
2020-05-11 02:44:43 +00:00
if ( ! query . distinct & & ! query . limitBy ( ) & & ! query . limit_with_ties & & ! query . arrayJoinExpressionList ( ) & & query . limitLength ( ) )
2019-07-18 14:41:11 +00:00
{
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , context ) ;
2020-07-12 05:18:01 +00:00
if ( limit_length > std : : numeric_limits < UInt64 > : : max ( ) - limit_offset )
return 0 ;
2019-07-18 14:41:11 +00:00
return limit_length + limit_offset ;
}
return 0 ;
}
2020-06-19 16:51:44 +00:00
static 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 .
*/
if ( auto query_table = extractTableExpression ( query , 0 ) )
{
if ( const auto * ast_union = query_table - > as < ASTSelectWithUnionQuery > ( ) )
{
for ( const auto & elem : ast_union - > list_of_selects - > children )
if ( hasWithTotalsInAnySubqueryInFromClause ( elem - > as < ASTSelectQuery & > ( ) ) )
return true ;
}
}
return false ;
}
2020-06-18 13:38:45 +00:00
void InterpreterSelectQuery : : executeImpl ( QueryPlan & query_plan , const BlockInputStreamPtr & prepared_input , std : : optional < Pipe > prepared_pipe )
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
2020-06-07 21:05:36 +00:00
* if there is an ORDER BY , then glue the streams using ResizeProcessor , and then MergeSorting transforms ,
* if not , then glue it using ResizeProcessor ,
2017-04-02 17:37:49 +00:00
* 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 ( ) ;
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-10-03 11:58:52 +00:00
auto & expressions = analysis_result ;
2020-09-02 13:04:54 +00:00
auto & subqueries_for_sets = query_analyzer - > getSubqueriesForSets ( ) ;
2020-03-30 18:50:42 +00:00
bool intermediate_stage = false ;
2020-08-15 13:57:17 +00:00
bool to_aggregation_stage = false ;
bool from_aggregation_stage = false ;
2018-07-19 13:36:21 +00:00
2019-08-09 13:42:58 +00:00
if ( options . only_analyze )
2018-07-19 13:36:21 +00:00
{
2020-06-18 13:38:45 +00:00
auto read_nothing = std : : make_unique < ReadNothingStep > ( source_header ) ;
query_plan . addStep ( std : : move ( read_nothing ) ) ;
2019-04-03 11:21:38 +00:00
2018-08-20 12:57:31 +00:00
if ( expressions . prewhere_info )
2019-04-03 11:21:38 +00:00
{
2020-06-18 13:38:45 +00:00
auto prewhere_step = std : : make_unique < FilterStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-16 09:42:25 +00:00
expressions . prewhere_info - > prewhere_actions ,
expressions . prewhere_info - > prewhere_column_name ,
expressions . prewhere_info - > remove_prewhere_column ) ;
2020-06-18 13:38:45 +00:00
prewhere_step - > setStepDescription ( " PREWHERE " ) ;
query_plan . addStep ( std : : move ( prewhere_step ) ) ;
2019-07-26 10:48:42 +00:00
// To remove additional columns in dry run
// For example, sample column which can be removed in this stage
if ( expressions . prewhere_info - > remove_columns_actions )
{
2020-06-18 13:38:45 +00:00
auto remove_columns = std : : make_unique < ExpressionStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-16 09:42:25 +00:00
expressions . prewhere_info - > remove_columns_actions ) ;
2020-06-18 13:38:45 +00:00
remove_columns - > setStepDescription ( " Remove unnecessary columns after PREWHERE " ) ;
query_plan . addStep ( std : : move ( remove_columns ) ) ;
2019-07-26 10:48:42 +00:00
}
2019-04-03 11:21:38 +00:00
}
2018-07-19 13:36:21 +00:00
}
else
{
2018-08-27 17:42:13 +00:00
if ( prepared_input )
2019-04-03 11:21:38 +00:00
{
2020-06-18 13:38:45 +00:00
auto prepared_source_step = std : : make_unique < ReadFromPreparedSource > (
2020-06-19 13:34:40 +00:00
Pipe ( std : : make_shared < SourceFromInputStream > ( prepared_input ) ) , context ) ;
2020-06-18 13:38:45 +00:00
query_plan . addStep ( std : : move ( prepared_source_step ) ) ;
2019-04-03 11:21:38 +00:00
}
2020-01-29 15:04:46 +00:00
else if ( prepared_pipe )
{
2020-06-19 13:34:40 +00:00
auto prepared_source_step = std : : make_unique < ReadFromPreparedSource > ( std : : move ( * prepared_pipe ) , context ) ;
2020-06-18 13:38:45 +00:00
query_plan . addStep ( std : : move ( prepared_source_step ) ) ;
2020-01-29 15:04:46 +00:00
}
2018-08-08 03:09:59 +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 )
2020-03-30 18:50:42 +00:00
intermediate_stage = true ;
2017-04-01 07:20:54 +00:00
2020-04-22 21:44:22 +00:00
/// Support optimize_distributed_group_by_sharding_key
2020-08-15 13:57:17 +00:00
/// Is running on the initiating server during distributed processing?
if ( from_stage = = QueryProcessingStage : : WithMergeableStateAfterAggregation )
from_aggregation_stage = true ;
/// Is running on remote servers during distributed processing?
if ( options . to_stage = = QueryProcessingStage : : WithMergeableStateAfterAggregation )
to_aggregation_stage = true ;
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. */
2020-06-18 13:38:45 +00:00
executeFetchColumns ( from_stage , query_plan , expressions . prewhere_info , expressions . columns_to_remove_after_prewhere ) ;
2017-04-01 07:20:54 +00:00
2020-05-23 22:24:01 +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
2020-03-30 18:50:42 +00:00
auto preliminary_sort = [ & ] ( )
{
/** 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 .
*/
if ( ! expressions . second_stage & & ! expressions . need_aggregate & & ! expressions . hasHaving ( ) )
{
if ( expressions . has_order_by )
2020-06-18 13:38:45 +00:00
executeOrder ( query_plan , query_info . input_order_info ) ;
2020-03-30 18:50:42 +00:00
if ( expressions . has_order_by & & query . limitLength ( ) )
2020-06-18 13:38:45 +00:00
executeDistinct ( query_plan , false , expressions . selected_columns , true ) ;
2020-03-30 18:50:42 +00:00
if ( expressions . hasLimitBy ( ) )
{
2020-06-18 13:38:45 +00:00
executeExpression ( query_plan , expressions . before_limit_by , " Before LIMIT BY " ) ;
executeLimitBy ( query_plan ) ;
2020-03-30 18:50:42 +00:00
}
if ( query . limitLength ( ) )
2020-06-18 13:38:45 +00:00
executePreLimit ( query_plan , true ) ;
2020-03-30 18:50:42 +00:00
}
} ;
if ( intermediate_stage )
{
if ( expressions . first_stage | | expressions . second_stage )
throw Exception ( " Query with intermediate stage cannot have any other stages " , ErrorCodes : : LOGICAL_ERROR ) ;
preliminary_sort ( ) ;
if ( expressions . need_aggregate )
2020-06-18 13:38:45 +00:00
executeMergeAggregated ( query_plan , aggregate_overflow_row , aggregate_final ) ;
2020-03-30 18:50:42 +00:00
}
2020-08-15 13:57:17 +00:00
if ( from_aggregation_stage )
{
if ( intermediate_stage | | expressions . first_stage | | expressions . second_stage )
throw Exception ( " Query with after aggregation stage cannot have any other stages " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2020-03-30 18:50:42 +00:00
2018-02-23 06:00:48 +00:00
if ( expressions . first_stage )
2017-04-01 07:20:54 +00:00
{
2020-02-10 15:50:12 +00:00
if ( expressions . hasFilter ( ) )
2019-03-29 20:31:06 +00:00
{
2020-06-18 13:38:45 +00:00
auto row_level_security_step = std : : make_unique < FilterStep > (
query_plan . getCurrentDataStream ( ) ,
2020-05-28 08:24:59 +00:00
expressions . filter_info - > actions ,
expressions . filter_info - > column_name ,
2020-06-16 12:02:10 +00:00
expressions . filter_info - > do_remove_column ) ;
2020-06-18 13:38:45 +00:00
row_level_security_step - > setStepDescription ( " Row-level security filter " ) ;
query_plan . addStep ( std : : move ( row_level_security_step ) ) ;
2019-03-29 20:31:06 +00:00
}
2020-08-11 12:03:18 +00:00
if ( expressions . before_array_join )
{
QueryPlanStepPtr before_array_join_step = std : : make_unique < ExpressionStep > (
query_plan . getCurrentDataStream ( ) ,
expressions . before_array_join ) ;
before_array_join_step - > setStepDescription ( " Before ARRAY JOIN " ) ;
query_plan . addStep ( std : : move ( before_array_join_step ) ) ;
}
if ( expressions . array_join )
{
QueryPlanStepPtr array_join_step = std : : make_unique < ArrayJoinStep > (
query_plan . getCurrentDataStream ( ) ,
expressions . array_join ) ;
array_join_step - > setStepDescription ( " ARRAY JOIN " ) ;
query_plan . addStep ( std : : move ( array_join_step ) ) ;
}
2020-06-25 17:40:25 +00:00
if ( expressions . before_join )
{
QueryPlanStepPtr before_join_step = std : : make_unique < ExpressionStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-26 00:23:11 +00:00
expressions . before_join ) ;
2020-06-25 18:18:27 +00:00
before_join_step - > setStepDescription ( " Before JOIN " ) ;
2020-06-25 17:40:25 +00:00
query_plan . addStep ( std : : move ( before_join_step ) ) ;
}
2019-01-30 12:01:00 +00:00
if ( expressions . hasJoin ( ) )
2018-02-21 08:16:01 +00:00
{
2020-03-13 08:15:43 +00:00
Block join_result_sample ;
2020-09-08 10:40:53 +00:00
JoinPtr join = expressions . join ;
2017-04-01 07:20:54 +00:00
2020-09-08 10:40:53 +00:00
join_result_sample = JoiningTransform : : transformHeader (
2020-06-25 20:28:41 +00:00
query_plan . getCurrentDataStream ( ) . header , expressions . join ) ;
2020-06-16 12:02:10 +00:00
2020-09-08 10:40:53 +00:00
QueryPlanStepPtr join_step = std : : make_unique < JoinStep > (
2020-06-25 18:18:27 +00:00
query_plan . getCurrentDataStream ( ) ,
2020-06-26 00:23:11 +00:00
expressions . join ) ;
2017-04-01 07:20:54 +00:00
2020-06-25 17:40:25 +00:00
join_step - > setStepDescription ( " JOIN " ) ;
query_plan . addStep ( std : : move ( join_step ) ) ;
2020-06-18 13:38:45 +00:00
2020-09-18 16:25:20 +00:00
if ( expressions . join_has_delayed_stream )
2019-03-26 18:28:37 +00:00
{
2020-09-18 16:25:20 +00:00
auto stream = std : : make_shared < LazyNonJoinedBlockInputStream > ( * join , join_result_sample , settings . max_block_size ) ;
2020-06-25 18:18:27 +00:00
auto source = std : : make_shared < SourceFromInputStream > ( std : : move ( stream ) ) ;
2020-06-29 08:14:05 +00:00
auto add_non_joined_rows_step = std : : make_unique < AddingDelayedSourceStep > (
2020-06-25 18:18:27 +00:00
query_plan . getCurrentDataStream ( ) , std : : move ( source ) ) ;
2020-06-17 13:38:17 +00:00
2020-06-25 18:18:27 +00:00
add_non_joined_rows_step - > setStepDescription ( " Add non-joined rows after JOIN " ) ;
query_plan . addStep ( std : : move ( add_non_joined_rows_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2018-02-21 08:16:01 +00:00
}
2017-04-01 07:20:54 +00:00
2020-02-10 15:50:12 +00:00
if ( expressions . hasWhere ( ) )
2020-06-18 13:38:45 +00:00
executeWhere ( query_plan , 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 )
2020-05-14 13:56:17 +00:00
{
2020-06-18 13:38:45 +00:00
executeAggregation ( query_plan , expressions . before_aggregation , aggregate_overflow_row , aggregate_final , query_info . input_order_info ) ;
2020-05-14 13:56:17 +00:00
/// We need to reset input order info, so that executeOrder can't use it
query_info . input_order_info . reset ( ) ;
}
2017-04-01 07:20:54 +00:00
else
{
2020-06-18 13:38:45 +00:00
executeExpression ( query_plan , expressions . before_order_and_select , " Before ORDER BY and SELECT " ) ;
executeDistinct ( query_plan , true , expressions . selected_columns , true ) ;
2017-04-01 07:20:54 +00:00
}
2020-03-30 18:50:42 +00:00
preliminary_sort ( ) ;
2017-04-01 07:20:54 +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.
2020-02-10 19:55:13 +00:00
if ( ! query_analyzer - > hasGlobalSubqueries ( ) & & ! subqueries_for_sets . empty ( ) )
2020-06-18 13:38:45 +00:00
executeSubqueriesInSetsAndJoins ( query_plan , subqueries_for_sets ) ;
2017-04-01 07:20:54 +00:00
}
2020-08-15 13:57:17 +00:00
if ( expressions . second_stage | | from_aggregation_stage )
2017-04-01 07:20:54 +00:00
{
2020-08-15 13:57:17 +00:00
if ( from_aggregation_stage )
{
/// No need to aggregate anything, since this was done on remote shards.
}
else 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 )
2020-06-18 13:38:45 +00:00
executeMergeAggregated ( query_plan , 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 ;
2020-06-18 13:38:45 +00:00
executeTotalsAndHaving ( query_plan , expressions . hasHaving ( ) , expressions . before_having , aggregate_overflow_row , final ) ;
2018-09-17 19:16:51 +00:00
}
2018-09-06 01:06:30 +00:00
2018-09-17 18:01:04 +00:00
if ( query . group_by_with_rollup )
2020-06-18 13:38:45 +00:00
executeRollupOrCube ( query_plan , Modificator : : ROLLUP ) ;
2018-11-23 18:52:00 +00:00
else if ( query . group_by_with_cube )
2020-06-18 13:38:45 +00:00
executeRollupOrCube ( query_plan , Modificator : : CUBE ) ;
2019-01-16 00:26:15 +00:00
2020-02-10 15:50:12 +00:00
if ( ( query . group_by_with_rollup | | query . group_by_with_cube ) & & expressions . hasHaving ( ) )
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 ) ;
2020-06-18 13:38:45 +00:00
executeHaving ( query_plan , expressions . before_having ) ;
2019-01-16 00:26:15 +00:00
}
2018-08-21 16:08:45 +00:00
}
2020-02-10 15:50:12 +00:00
else if ( expressions . hasHaving ( ) )
2020-06-18 13:38:45 +00:00
executeHaving ( query_plan , expressions . before_having ) ;
2017-04-01 07:20:54 +00:00
2020-06-18 13:38:45 +00:00
executeExpression ( query_plan , expressions . before_order_and_select , " Before ORDER BY and SELECT " ) ;
executeDistinct ( query_plan , true , expressions . selected_columns , true ) ;
2017-04-01 07:20:54 +00:00
}
2019-09-18 13:08:51 +00:00
else if ( query . group_by_with_totals | | query . group_by_with_rollup | | query . group_by_with_cube )
2019-12-30 14:46:02 +00:00
throw Exception ( " WITH TOTALS, ROLLUP or CUBE are not supported without aggregation " , ErrorCodes : : NOT_IMPLEMENTED ) ;
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
*/
2019-07-24 14:23:57 +00:00
2018-02-23 06:00:48 +00:00
if ( ! expressions . first_stage & & ! expressions . need_aggregate & & ! ( query . group_by_with_totals & & ! aggregate_final ) )
2020-06-22 14:37:42 +00:00
executeMergeSorted ( query_plan , " for ORDER BY " ) ;
2017-04-02 17:37:49 +00:00
else /// Otherwise, just sort.
2020-06-18 13:38:45 +00:00
executeOrder ( query_plan , query_info . input_order_info ) ;
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 ` .
*/
2020-03-11 13:39:48 +00:00
bool has_prelimit = false ;
2020-08-15 13:57:17 +00:00
if ( ! to_aggregation_stage & &
query . limitLength ( ) & & ! query . limit_with_ties & & ! hasWithTotalsInAnySubqueryInFromClause ( query ) & &
2020-06-19 16:58:27 +00:00
! query . arrayJoinExpressionList ( ) & & ! query . distinct & & ! expressions . hasLimitBy ( ) & & ! settings . extremes )
2018-02-25 06:34:20 +00:00
{
2020-06-18 13:38:45 +00:00
executePreLimit ( query_plan , false ) ;
2020-03-11 13:39:48 +00:00
has_prelimit = true ;
2018-02-25 06:34:20 +00:00
}
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 .
*/
2020-06-18 13:38:45 +00:00
if ( query . distinct )
executeDistinct ( query_plan , false , expressions . selected_columns , false ) ;
2017-04-01 07:20:54 +00:00
2020-02-10 15:50:12 +00:00
if ( expressions . hasLimitBy ( ) )
2018-03-01 05:24:56 +00:00
{
2020-06-18 13:38:45 +00:00
executeExpression ( query_plan , expressions . before_limit_by , " Before LIMIT BY " ) ;
executeLimitBy ( query_plan ) ;
2018-03-01 05:24:56 +00:00
}
2020-06-18 13:38:45 +00:00
executeWithFill ( query_plan ) ;
2019-08-19 20:22:45 +00:00
2020-06-15 22:24:00 +00:00
/// If we have 'WITH TIES', we need execute limit before projection,
/// because in that case columns from 'ORDER BY' are used.
if ( query . limit_with_ties )
{
2020-06-19 08:30:21 +00:00
executeLimit ( query_plan ) ;
2020-06-15 22:24:00 +00:00
has_prelimit = true ;
}
2020-08-15 13:57:17 +00:00
/// Projection not be done on the shards, since then initiator will not find column in blocks.
/// (significant only for WithMergeableStateAfterAggregation).
if ( ! to_aggregation_stage )
{
/// We must do projection after DISTINCT because projection may remove some columns.
executeProjection ( query_plan , expressions . final_projection ) ;
}
2018-03-01 01:25:06 +00:00
2020-08-15 13:57:17 +00:00
/// Extremes are calculated before LIMIT, but after LIMIT BY. This is Ok.
2020-06-18 13:38:45 +00:00
executeExtremes ( query_plan ) ;
2018-03-01 01:25:06 +00:00
2020-08-15 13:57:17 +00:00
/// Limit is no longer needed if there is prelimit.
if ( ! to_aggregation_stage & & ! has_prelimit )
2020-06-18 13:38:45 +00:00
executeLimit ( query_plan ) ;
2020-05-06 06:50:55 +00:00
2020-08-15 13:57:17 +00:00
if ( ! to_aggregation_stage )
executeOffset ( query_plan ) ;
2017-04-01 07:20:54 +00:00
}
}
2020-11-02 12:07:01 +00:00
if ( ! subqueries_for_sets . empty ( ) & & ( expressions . hasHaving ( ) | | query_analyzer - > hasGlobalSubqueries ( ) ) )
2020-06-18 13:38:45 +00:00
executeSubqueriesInSetsAndJoins ( query_plan , subqueries_for_sets ) ;
2012-05-09 13:12:38 +00:00
}
2020-09-15 10:40:39 +00:00
static StreamLocalLimits getLimitsForStorage ( const Settings & settings , const SelectQueryOptions & options )
{
StreamLocalLimits limits ;
limits . mode = LimitsMode : : LIMITS_TOTAL ;
limits . size_limits = SizeLimits ( settings . max_rows_to_read , settings . max_bytes_to_read ,
settings . read_overflow_mode ) ;
limits . speed_limits . max_execution_time = settings . max_execution_time ;
limits . timeout_overflow_mode = settings . timeout_overflow_mode ;
/** 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 .
*
* The limits to throttle maximum execution speed is also checked on all servers .
*/
if ( options . to_stage = = QueryProcessingStage : : Complete )
{
limits . speed_limits . min_execution_rps = settings . min_execution_speed ;
limits . speed_limits . min_execution_bps = settings . min_execution_speed_bytes ;
}
limits . speed_limits . max_execution_rps = settings . max_execution_speed ;
limits . speed_limits . max_execution_bps = settings . max_execution_speed_bytes ;
limits . speed_limits . timeout_before_checking_execution_speed = settings . timeout_before_checking_execution_speed ;
return limits ;
}
2020-09-25 13:19:26 +00:00
void InterpreterSelectQuery : : addEmptySourceToQueryPlan ( QueryPlan & query_plan , const Block & source_header , const SelectQueryInfo & query_info )
2020-09-18 14:16:53 +00:00
{
2020-09-25 13:19:26 +00:00
Pipe pipe ( std : : make_shared < NullSource > ( source_header ) ) ;
2020-09-18 14:16:53 +00:00
if ( query_info . prewhere_info )
{
if ( query_info . prewhere_info - > alias_actions )
{
pipe . addSimpleTransform ( [ & ] ( const Block & header )
{
return std : : make_shared < ExpressionTransform > ( header , query_info . prewhere_info - > alias_actions ) ;
} ) ;
}
pipe . addSimpleTransform ( [ & ] ( const Block & header )
{
return std : : make_shared < FilterTransform > (
header ,
query_info . prewhere_info - > prewhere_actions ,
query_info . prewhere_info - > prewhere_column_name ,
query_info . prewhere_info - > remove_prewhere_column ) ;
} ) ;
// To remove additional columns
// In some cases, we did not read any marks so that the pipeline.streams is empty
// Thus, some columns in prewhere are not removed as expected
// This leads to mismatched header in distributed table
if ( query_info . prewhere_info - > remove_columns_actions )
{
pipe . addSimpleTransform ( [ & ] ( const Block & header )
{
return std : : make_shared < ExpressionTransform > (
header , query_info . prewhere_info - > remove_columns_actions ) ;
} ) ;
}
}
auto read_from_pipe = std : : make_unique < ReadFromPreparedSource > ( std : : move ( pipe ) ) ;
read_from_pipe - > setStepDescription ( " Read from NullSource " ) ;
query_plan . addStep ( std : : move ( read_from_pipe ) ) ;
}
2018-08-05 07:05:36 +00:00
void InterpreterSelectQuery : : executeFetchColumns (
2020-06-18 13:38:45 +00:00
QueryProcessingStage : : Enum processing_stage , QueryPlan & query_plan ,
2020-11-03 19:05:47 +00:00
const PrewhereDAGInfoPtr & prewhere_info , const NameSet & columns_to_remove_after_prewhere )
2012-05-09 13:12:38 +00:00
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery ( ) ;
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2018-09-03 17:24:46 +00:00
2019-10-28 17:27:43 +00:00
/// Optimization for trivial query like SELECT count() FROM table.
2020-06-04 22:01:40 +00:00
bool optimize_trivial_count =
2020-06-13 20:59:20 +00:00
syntax_analyzer_result - > optimize_trivial_count
2020-09-21 10:13:01 +00:00
& & ( settings . max_parallel_replicas < = 1 )
2020-06-13 20:59:20 +00:00
& & storage
2020-10-14 13:15:15 +00:00
& & storage - > getName ( ) ! = " MaterializeMySQL "
2020-06-13 20:59:20 +00:00
& & ! filter_info
& & processing_stage = = QueryProcessingStage : : FetchColumns
& & query_analyzer - > hasAggregation ( )
& & ( query_analyzer - > aggregates ( ) . size ( ) = = 1 )
& & typeid_cast < AggregateFunctionCount * > ( query_analyzer - > aggregates ( ) [ 0 ] . function . get ( ) ) ;
2019-10-28 17:27:43 +00:00
2020-06-04 22:01:40 +00:00
if ( optimize_trivial_count )
2019-10-28 17:27:43 +00:00
{
2020-06-05 10:27:54 +00:00
const auto & desc = query_analyzer - > aggregates ( ) [ 0 ] ;
const auto & func = desc . function ;
2020-09-21 10:13:01 +00:00
std : : optional < UInt64 > num_rows { } ;
if ( ! query . prewhere ( ) & & ! query . where ( ) )
num_rows = storage - > totalRows ( ) ;
else // It's possible to optimize count() given only partition predicates
{
SelectQueryInfo temp_query_info ;
temp_query_info . query = query_ptr ;
temp_query_info . syntax_analyzer_result = syntax_analyzer_result ;
temp_query_info . sets = query_analyzer - > getPreparedSets ( ) ;
num_rows = storage - > totalRowsByPartitionPredicate ( temp_query_info , * context ) ;
}
2019-10-28 17:27:43 +00:00
if ( num_rows )
{
AggregateFunctionCount & agg_count = static_cast < AggregateFunctionCount & > ( * func ) ;
/// We will process it up to "WithMergeableState".
std : : vector < char > state ( agg_count . sizeOfData ( ) ) ;
AggregateDataPtr place = state . data ( ) ;
agg_count . create ( place ) ;
SCOPE_EXIT ( agg_count . destroy ( place ) ) ;
2019-10-30 19:15:55 +00:00
agg_count . set ( place , * num_rows ) ;
2019-10-28 17:27:43 +00:00
auto column = ColumnAggregateFunction : : create ( func ) ;
column - > insertFrom ( place ) ;
2020-11-03 11:28:28 +00:00
Block header = analysis_result . before_aggregation - > getResultColumns ( ) ;
2020-06-04 22:01:40 +00:00
size_t arguments_size = desc . argument_names . size ( ) ;
2019-12-12 10:21:08 +00:00
DataTypes argument_types ( arguments_size ) ;
for ( size_t j = 0 ; j < arguments_size ; + + j )
2020-06-04 22:01:40 +00:00
argument_types [ j ] = header . getByName ( desc . argument_names [ j ] ) . type ;
2019-12-12 10:21:08 +00:00
2019-10-28 17:27:43 +00:00
Block block_with_count {
2020-06-04 22:01:40 +00:00
{ std : : move ( column ) , std : : make_shared < DataTypeAggregateFunction > ( func , argument_types , desc . parameters ) , desc . column_name } } ;
2019-10-28 17:27:43 +00:00
2020-05-28 16:01:12 +00:00
auto istream = std : : make_shared < OneBlockInputStream > ( block_with_count ) ;
2020-06-19 14:00:11 +00:00
auto prepared_count = std : : make_unique < ReadFromPreparedSource > ( Pipe ( std : : make_shared < SourceFromInputStream > ( istream ) ) , context ) ;
2020-06-18 17:45:00 +00:00
prepared_count - > setStepDescription ( " Optimized trivial count " ) ;
query_plan . addStep ( std : : move ( prepared_count ) ) ;
2019-10-28 17:27:43 +00:00
from_stage = QueryProcessingStage : : WithMergeableState ;
analysis_result . first_stage = false ;
return ;
}
}
2017-04-02 17:37:49 +00:00
/// Actions to calculate ALIAS if required.
2020-11-03 11:28:28 +00:00
ActionsDAGPtr alias_actions ;
2017-04-01 07:20:54 +00:00
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
2020-03-07 17:37:38 +00:00
auto row_policy_filter = context - > getRowPolicyCondition ( table_id . getDatabaseName ( ) , table_id . getTableName ( ) , RowPolicy : : SELECT_FILTER ) ;
2019-11-17 11:57:02 +00:00
if ( row_policy_filter )
2019-03-29 20:31:06 +00:00
{
auto initial_required_columns = required_columns ;
2020-11-03 11:28:28 +00:00
ActionsDAGPtr actions ;
2019-12-27 12:47:29 +00:00
generateFilterActions ( actions , row_policy_filter , initial_required_columns ) ;
2019-03-29 20:31:06 +00:00
auto required_columns_from_filter = actions - > getRequiredColumns ( ) ;
for ( const auto & column : required_columns_from_filter )
{
2020-11-03 11:28:28 +00:00
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column . name ) )
required_columns . push_back ( column . name ) ;
2019-03-29 20:31:06 +00:00
}
}
/// Detect, if ALIAS columns are required for query execution
auto alias_columns_required = false ;
2020-06-17 16:39:58 +00:00
const ColumnsDescription & storage_columns = metadata_snapshot - > getColumns ( ) ;
2019-03-14 15:20:51 +00:00
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
2020-11-03 11:28:28 +00:00
auto prewhere_required_columns = prewhere_info - > prewhere_actions - > getRequiredColumns ( ) . getNames ( ) ;
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 )
2020-04-28 19:41:23 +00:00
{
auto column_decl = storage_columns . get ( column ) ;
/// TODO: can make CAST only if the type is different (but requires SyntaxAnalyzer).
auto cast_column_default = addTypeConversionToAST ( column_default - > expression - > clone ( ) , column_decl . type - > getName ( ) ) ;
column_expr = setAlias ( cast_column_default - > 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 ( ) ) ;
2020-11-03 11:28:28 +00:00
Block prewhere_actions_result = prewhere_info - > prewhere_actions - > getResultColumns ( ) ;
2019-03-29 20:31:06 +00:00
/// 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
}
2020-07-22 17:13:05 +00:00
auto syntax_result = TreeRewriter ( * context ) . analyze ( required_columns_all_expr , required_columns_after_prewhere , storage , metadata_snapshot ) ;
2020-11-03 11:28:28 +00:00
alias_actions = ExpressionAnalyzer ( required_columns_all_expr , syntax_result , * context ) . getActionsDAG ( 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.
2020-11-03 11:28:28 +00:00
required_columns = alias_actions - > getRequiredColumns ( ) . getNames ( ) ;
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
{
2020-05-28 08:24:59 +00:00
return required_columns_after_prewhere_set . count ( name ) ! = 0 ;
2019-03-29 20:31:06 +00:00
} ) , 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.
2020-11-03 11:28:28 +00:00
for ( const auto & name : required_columns )
prewhere_info - > prewhere_actions - > tryRestoreColumn ( name ) ;
2018-09-03 17:24:46 +00:00
2019-03-29 20:31:06 +00:00
auto analyzed_result
2020-07-22 17:13:05 +00:00
= TreeRewriter ( * context ) . analyze ( required_columns_from_prewhere_expr , metadata_snapshot - > getColumns ( ) . getAllPhysical ( ) ) ;
2019-03-29 20:31:06 +00:00
prewhere_info - > alias_actions
2020-11-03 11:28:28 +00:00
= ExpressionAnalyzer ( required_columns_from_prewhere_expr , analyzed_result , * context ) . getActionsDAG ( 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 ( ) ;
2020-11-03 11:28:28 +00:00
Block prewhere_actions_result = prewhere_info - > prewhere_actions - > getResultColumns ( ) ;
2019-03-29 20:31:06 +00:00
for ( auto & column : required_columns_from_alias )
2020-11-03 11:28:28 +00:00
if ( ! prewhere_actions_result . has ( column . name ) )
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column . name ) )
required_columns . push_back ( column . name ) ;
2018-09-03 17:24:46 +00:00
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
2020-06-19 11:28:48 +00:00
/// General limit for the number of threads.
2020-10-02 12:38:33 +00:00
size_t max_threads_execute_query = settings . max_threads ;
2020-06-19 10:57:44 +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 ;
2020-10-02 12:38:33 +00:00
max_threads_execute_query = 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-11-15 18:41:18 +00:00
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , * context ) ;
2019-02-10 15:17:45 +00:00
2019-04-19 13:38:25 +00:00
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size,
2017-04-02 17:37:49 +00:00
* 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-19 13:38:25 +00:00
& & ! query . limit_with_ties
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 ( )
2020-07-12 05:18:01 +00:00
& & limit_length < = std : : numeric_limits < UInt64 > : : max ( ) - limit_offset
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 ) ;
2020-10-02 12:38:33 +00:00
max_threads_execute_query = 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?
2020-06-18 17:45:00 +00:00
if ( query_plan . isInitialized ( ) )
2018-02-21 06:25:21 +00:00
{
/// 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-11-15 18:41:18 +00:00
subquery , getSubqueryContext ( * context ) ,
2019-03-18 12:05:51 +00:00
options . copy ( ) . subquery ( ) . noModify ( ) , required_columns ) ;
2018-07-19 13:36:21 +00:00
if ( query_analyzer - > hasAggregation ( ) )
interpreter_subquery - > ignoreWithTotals ( ) ;
}
2020-06-18 17:45:00 +00:00
interpreter_subquery - > buildQueryPlan ( query_plan ) ;
2020-06-19 14:42:01 +00:00
query_plan . addInterpreterContext ( context ) ;
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 ;
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 ( ) ;
2020-11-03 19:05:47 +00:00
if ( prewhere_info )
{
query_info . prewhere_info = std : : make_shared < PrewhereInfo > (
2020-11-10 16:27:55 +00:00
std : : make_shared < ExpressionActions > ( prewhere_info - > prewhere_actions ) ,
2020-11-03 19:05:47 +00:00
prewhere_info - > prewhere_column_name ) ;
if ( prewhere_info - > alias_actions )
2020-11-10 16:27:55 +00:00
query_info . prewhere_info - > alias_actions = std : : make_shared < ExpressionActions > ( prewhere_info - > alias_actions ) ;
2020-11-03 19:05:47 +00:00
if ( prewhere_info - > remove_columns_actions )
2020-11-10 16:27:55 +00:00
query_info . prewhere_info - > remove_columns_actions = std : : make_shared < ExpressionActions > ( prewhere_info - > remove_columns_actions ) ;
2020-11-03 19:05:47 +00:00
query_info . prewhere_info - > remove_prewhere_column = prewhere_info - > remove_prewhere_column ;
query_info . prewhere_info - > need_filter = prewhere_info - > need_filter ;
}
2019-12-10 23:18:24 +00:00
/// Create optimizer with prepared actions.
2020-05-13 13:49:10 +00:00
/// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge.
if ( analysis_result . optimize_read_in_order | | analysis_result . optimize_aggregation_in_order )
2019-12-10 23:18:24 +00:00
{
2020-05-13 13:49:10 +00:00
if ( analysis_result . optimize_read_in_order )
query_info . order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
analysis_result . order_by_elements_actions ,
getSortDescription ( query , * context ) ,
query_info . syntax_analyzer_result ) ;
else
query_info . order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
2020-02-16 19:46:45 +00:00
analysis_result . group_by_elements_actions ,
2020-05-14 14:20:49 +00:00
getSortDescriptionFromGroupBy ( query ) ,
2020-02-16 19:46:45 +00:00
query_info . syntax_analyzer_result ) ;
2019-12-10 23:18:24 +00:00
2020-11-03 18:22:46 +00:00
query_info . input_order_info = query_info . order_optimizer - > getInputOrder ( metadata_snapshot ) ;
2019-12-10 23:18:24 +00:00
}
2020-09-15 10:40:39 +00:00
StreamLocalLimits limits ;
2020-08-28 14:50:25 +00:00
SizeLimits leaf_limits ;
2020-09-14 14:13:58 +00:00
std : : shared_ptr < const EnabledQuota > quota ;
2017-07-15 03:48:36 +00:00
2020-08-28 14:50:25 +00:00
2020-09-14 14:13:58 +00:00
/// Set the limits and quota for reading data, the speed and time of the query.
2020-09-17 09:57:48 +00:00
if ( ! options . ignore_limits )
{
2020-09-15 10:40:39 +00:00
limits = getLimitsForStorage ( settings , options ) ;
2020-08-28 14:50:25 +00:00
leaf_limits = SizeLimits ( settings . max_rows_to_read_leaf , settings . max_bytes_to_read_leaf ,
settings . read_overflow_mode_leaf ) ;
}
2020-09-14 14:13:58 +00:00
if ( ! options . ignore_quota & & ( options . to_stage = = QueryProcessingStage : : Complete ) )
quota = context - > getQuota ( ) ;
2017-07-15 03:48:36 +00:00
2020-09-18 11:39:07 +00:00
storage - > read ( query_plan , required_columns , metadata_snapshot ,
query_info , * context , processing_stage , max_block_size , max_streams ) ;
2020-09-18 14:16:53 +00:00
/// Create step which reads from empty source if storage has no data.
if ( ! query_plan . isInitialized ( ) )
{
auto header = metadata_snapshot - > getSampleBlockForColumns (
required_columns , storage - > getVirtuals ( ) , storage - > getStorageID ( ) ) ;
2020-09-25 13:19:26 +00:00
addEmptySourceToQueryPlan ( query_plan , header , query_info ) ;
2020-09-18 14:16:53 +00:00
}
2020-09-18 11:39:07 +00:00
/// Extend lifetime of context, table lock, storage. Set limits and quota.
auto adding_limits_and_quota = std : : make_unique < SettingQuotaAndLimitsStep > (
query_plan . getCurrentDataStream ( ) ,
storage ,
std : : move ( table_lock ) ,
limits ,
leaf_limits ,
std : : move ( quota ) ,
2020-09-18 12:10:59 +00:00
context ) ;
2020-09-18 11:39:07 +00:00
adding_limits_and_quota - > setStepDescription ( " Set limits and quota after reading from storage " ) ;
query_plan . addStep ( std : : move ( adding_limits_and_quota ) ) ;
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
2020-10-22 11:08:12 +00:00
/// Specify the number of threads only if it wasn't specified in storage.
if ( ! query_plan . getMaxThreads ( ) )
query_plan . setMaxThreads ( max_threads_execute_query ) ;
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
{
2020-06-18 17:45:00 +00:00
auto table_aliases = std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , alias_actions ) ;
table_aliases - > setStepDescription ( " Add table aliases " ) ;
query_plan . addStep ( std : : move ( table_aliases ) ) ;
2018-02-26 21:00:42 +00:00
}
2012-05-09 13:12:38 +00:00
}
2011-08-28 05:13:24 +00:00
2012-05-09 13:12:38 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery : : executeWhere ( QueryPlan & query_plan , const ActionsDAGPtr & expression , bool remove_filter )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
auto where_step = std : : make_unique < FilterStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-16 12:02:10 +00:00
expression ,
getSelectQuery ( ) . where ( ) - > getColumnName ( ) ,
remove_filter ) ;
2020-06-18 17:45:00 +00:00
where_step - > setStepDescription ( " WHERE " ) ;
query_plan . addStep ( std : : move ( where_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2012-08-27 05:13:14 +00:00
2012-05-09 13:12:38 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery : : executeAggregation ( QueryPlan & query_plan , const ActionsDAGPtr & expression , bool overflow_row , bool final , InputOrderInfoPtr group_by_info )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
auto expression_before_aggregation = std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , expression ) ;
expression_before_aggregation - > setStepDescription ( " Before GROUP BY " ) ;
query_plan . addStep ( std : : move ( expression_before_aggregation ) ) ;
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
const auto & header_before_aggregation = query_plan . getCurrentDataStream ( ) . header ;
2019-03-26 18:28:37 +00:00
ColumnNumbers keys ;
2020-02-10 19:55:13 +00:00
for ( const auto & key : query_analyzer - > aggregationKeys ( ) )
keys . push_back ( header_before_aggregation . getPositionByName ( key . name ) ) ;
AggregateDescriptions aggregates = query_analyzer - > aggregates ( ) ;
2019-03-26 18:28:37 +00:00
for ( auto & descr : aggregates )
if ( descr . arguments . empty ( ) )
for ( const auto & name : descr . argument_names )
2019-04-05 11:27:08 +00:00
descr . arguments . push_back ( header_before_aggregation . getPositionByName ( name ) ) ;
2019-03-26 18:28:37 +00:00
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2019-04-05 11:27:08 +00:00
Aggregator : : Params params ( header_before_aggregation , keys , aggregates ,
2019-03-26 18:28:37 +00:00
overflow_row , settings . max_rows_to_group_by , settings . group_by_overflow_mode ,
2020-06-18 18:29:33 +00:00
settings . group_by_two_level_threshold ,
settings . group_by_two_level_threshold_bytes ,
settings . max_bytes_before_external_group_by ,
settings . empty_result_for_aggregation_by_empty_set ,
context - > getTemporaryVolume ( ) ,
settings . max_threads ,
settings . min_free_disk_space_for_temporary_data ) ;
2019-03-26 18:28:37 +00:00
2020-06-17 14:21:48 +00:00
SortDescription group_by_sort_description ;
2019-04-17 15:35:22 +00:00
2020-04-18 09:51:21 +00:00
if ( group_by_info & & settings . optimize_aggregation_in_order )
2020-06-17 14:21:48 +00:00
group_by_sort_description = getSortDescriptionFromGroupBy ( getSelectQuery ( ) ) ;
2019-03-26 18:28:37 +00:00
else
2020-06-17 14:21:48 +00:00
group_by_info = nullptr ;
2019-03-26 18:28:37 +00:00
2020-06-17 14:21:48 +00:00
auto merge_threads = max_streams ;
auto temporary_data_merge_threads = settings . aggregation_memory_efficient_merge_threads
? static_cast < size_t > ( settings . aggregation_memory_efficient_merge_threads )
: static_cast < size_t > ( settings . max_threads ) ;
2019-12-26 16:52:15 +00:00
2020-06-17 14:21:48 +00:00
bool storage_has_evenly_distributed_read = storage & & storage - > hasEvenlyDistributedRead ( ) ;
2020-06-18 17:45:00 +00:00
auto aggregating_step = std : : make_unique < AggregatingStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-18 18:29:33 +00:00
params , final ,
2020-06-17 14:21:48 +00:00
settings . max_block_size ,
merge_threads ,
temporary_data_merge_threads ,
storage_has_evenly_distributed_read ,
std : : move ( group_by_info ) ,
std : : move ( group_by_sort_description ) ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( aggregating_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeMergeAggregated ( QueryPlan & query_plan , bool overflow_row , bool final )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
const auto & header_before_merge = query_plan . getCurrentDataStream ( ) . header ;
2019-03-26 18:28:37 +00:00
ColumnNumbers keys ;
2020-02-10 19:55:13 +00:00
for ( const auto & key : query_analyzer - > aggregationKeys ( ) )
keys . push_back ( header_before_merge . getPositionByName ( key . name ) ) ;
2019-03-26 18:28:37 +00:00
/** There are two modes of distributed aggregation.
*
* 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 .
*
* 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 .
*
* 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 .
*/
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2020-02-10 19:55:13 +00:00
Aggregator : : Params params ( header_before_merge , keys , query_analyzer - > aggregates ( ) , overflow_row , settings . max_threads ) ;
2019-03-26 18:28:37 +00:00
auto transform_params = std : : make_shared < AggregatingTransformParams > ( params , final ) ;
2020-06-18 17:45:00 +00:00
auto merging_aggregated = std : : make_unique < MergingAggregatedStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-17 20:19:55 +00:00
std : : move ( transform_params ) ,
2020-06-16 18:57:21 +00:00
settings . distributed_aggregation_memory_efficient ,
settings . max_threads ,
settings . aggregation_memory_efficient_merge_threads ) ;
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( merging_aggregated ) ) ;
2019-03-26 18:28:37 +00:00
}
2012-05-09 13:12:38 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery : : executeHaving ( QueryPlan & query_plan , const ActionsDAGPtr & expression )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
auto having_step = std : : make_unique < FilterStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-17 19:02:46 +00:00
expression , getSelectQuery ( ) . having ( ) - > getColumnName ( ) , false ) ;
2019-04-29 10:00:17 +00:00
2020-06-18 17:45:00 +00:00
having_step - > setStepDescription ( " HAVING " ) ;
query_plan . addStep ( std : : move ( having_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2011-09-25 03:37:09 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery : : executeTotalsAndHaving ( QueryPlan & query_plan , bool has_having , const ActionsDAGPtr & expression , bool overflow_row , bool final )
2019-03-26 18:28:37 +00:00
{
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
auto totals_having_step = std : : make_unique < TotalsHavingStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-17 16:54:51 +00:00
overflow_row , expression ,
2019-05-14 11:04:11 +00:00
has_having ? getSelectQuery ( ) . having ( ) - > getColumnName ( ) : " " ,
2019-03-26 18:28:37 +00:00
settings . totals_mode , settings . totals_auto_threshold , final ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( totals_having_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeRollupOrCube ( QueryPlan & query_plan , Modificator modificator )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
const auto & header_before_transform = query_plan . getCurrentDataStream ( ) . header ;
2019-03-26 18:28:37 +00:00
ColumnNumbers keys ;
2020-02-10 19:55:13 +00:00
for ( const auto & key : query_analyzer - > aggregationKeys ( ) )
keys . push_back ( header_before_transform . getPositionByName ( key . name ) ) ;
2019-03-26 18:28:37 +00:00
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2020-02-10 19:55:13 +00:00
Aggregator : : Params params ( header_before_transform , keys , query_analyzer - > aggregates ( ) ,
2020-07-16 22:08:44 +00:00
false , settings . max_rows_to_group_by , settings . group_by_overflow_mode , 0 , 0 ,
2019-03-26 18:28:37 +00:00
settings . max_bytes_before_external_group_by , settings . empty_result_for_aggregation_by_empty_set ,
2020-01-19 14:26:28 +00:00
context - > getTemporaryVolume ( ) , settings . max_threads , settings . min_free_disk_space_for_temporary_data ) ;
2019-03-26 18:28:37 +00:00
2019-04-05 10:52:07 +00:00
auto transform_params = std : : make_shared < AggregatingTransformParams > ( params , true ) ;
2020-06-18 17:45:00 +00:00
QueryPlanStepPtr step ;
2020-06-17 17:15:24 +00:00
if ( modificator = = Modificator : : ROLLUP )
2020-06-18 17:45:00 +00:00
step = std : : make_unique < RollupStep > ( query_plan . getCurrentDataStream ( ) , std : : move ( transform_params ) ) ;
2020-06-17 17:15:24 +00:00
else
2020-06-18 17:45:00 +00:00
step = std : : make_unique < CubeStep > ( query_plan . getCurrentDataStream ( ) , std : : move ( transform_params ) ) ;
query_plan . addStep ( std : : move ( step ) ) ;
2019-03-26 18:28:37 +00:00
}
2014-02-27 12:49:21 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery : : executeExpression ( QueryPlan & query_plan , const ActionsDAGPtr & expression , const std : : string & description )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
auto expression_step = std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , expression ) ;
2020-06-16 12:02:10 +00:00
2020-06-18 17:45:00 +00:00
expression_step - > setStepDescription ( description ) ;
query_plan . addStep ( std : : move ( expression_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2012-05-09 13:12:38 +00:00
2012-02-27 06:28:20 +00:00
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeOrderOptimized ( QueryPlan & query_plan , InputOrderInfoPtr input_sorting_info , UInt64 limit , SortDescription & output_order_descr )
2019-03-26 18:28:37 +00:00
{
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
auto finish_sorting_step = std : : make_unique < FinishSortingStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-17 19:57:13 +00:00
input_sorting_info - > order_key_prefix_descr ,
output_order_descr ,
settings . max_block_size ,
limit ) ;
2020-05-17 07:27:55 +00:00
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( finish_sorting_step ) ) ;
2020-04-18 09:51:21 +00:00
}
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeOrder ( QueryPlan & query_plan , InputOrderInfoPtr input_sorting_info )
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery ( ) ;
2019-11-17 13:48:13 +00:00
SortDescription output_order_descr = getSortDescription ( query , * context ) ;
2019-11-15 18:41:18 +00:00
UInt64 limit = getLimitForSorting ( query , * context ) ;
2019-08-26 14:54:29 +00:00
2019-11-15 14:03:42 +00:00
if ( input_sorting_info )
2019-08-23 11:39:22 +00:00
{
/* Case of sorting with optimization using sorting key.
* We have several threads , each of them reads batch of parts in direct
* or reverse order of sorting key using one input stream per part
* and then merge them into one sorted stream .
* At this stage we merge per - thread streams into one .
*/
2020-06-18 17:45:00 +00:00
executeOrderOptimized ( query_plan , input_sorting_info , limit , output_order_descr ) ;
2019-08-26 14:54:29 +00:00
return ;
2019-08-23 11:39:22 +00:00
}
2019-03-26 18:28:37 +00:00
2020-06-16 14:45:52 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
auto partial_sorting = std : : make_unique < PartialSortingStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-16 14:45:52 +00:00
output_order_descr ,
limit ,
SizeLimits ( settings . max_rows_to_sort , settings . max_bytes_to_sort , settings . sort_overflow_mode ) ) ;
2020-05-17 07:27:55 +00:00
2020-06-22 14:37:42 +00:00
partial_sorting - > setStepDescription ( " Sort each block for ORDER BY " ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( partial_sorting ) ) ;
2020-05-17 07:27:55 +00:00
2019-03-26 18:28:37 +00:00
/// Merge the sorted blocks.
2020-06-18 17:45:00 +00:00
auto merge_sorting_step = std : : make_unique < MergeSortingStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-16 15:08:01 +00:00
output_order_descr , settings . max_block_size , limit ,
2020-06-18 17:45:00 +00:00
settings . max_bytes_before_remerge_sort ,
2020-06-16 15:08:01 +00:00
settings . max_bytes_before_external_sort , context - > getTemporaryVolume ( ) ,
settings . min_free_disk_space_for_temporary_data ) ;
2019-04-30 17:01:12 +00:00
2020-06-22 14:37:42 +00:00
merge_sorting_step - > setStepDescription ( " Merge sorted blocks for ORDER BY " ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( merge_sorting_step ) ) ;
2020-01-07 05:41:18 +00:00
/// If there are several streams, we merge them into one
2020-06-22 14:37:42 +00:00
executeMergeSorted ( query_plan , output_order_descr , limit , " for ORDER BY " ) ;
2019-03-26 18:28:37 +00:00
}
2012-07-25 20:29:22 +00:00
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeMergeSorted ( QueryPlan & query_plan , const std : : string & description )
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery ( ) ;
2019-11-15 18:41:18 +00:00
SortDescription order_descr = getSortDescription ( query , * context ) ;
UInt64 limit = getLimitForSorting ( query , * context ) ;
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
executeMergeSorted ( query_plan , order_descr , limit , description ) ;
2019-10-17 11:56:05 +00:00
}
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeMergeSorted ( QueryPlan & query_plan , const SortDescription & sort_description , UInt64 limit , const std : : string & description )
2019-10-17 11:56:05 +00:00
{
2020-06-16 15:47:40 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-10-17 11:56:05 +00:00
2020-06-18 17:45:00 +00:00
auto merging_sorted = std : : make_unique < MergingSortedStep > (
query_plan . getCurrentDataStream ( ) ,
2019-10-17 11:56:05 +00:00
sort_description ,
2019-03-26 18:28:37 +00:00
settings . max_block_size , limit ) ;
2020-06-18 17:45:00 +00:00
merging_sorted - > setStepDescription ( " Merge sorted streams " + description ) ;
query_plan . addStep ( std : : move ( merging_sorted ) ) ;
2019-03-26 18:28:37 +00:00
}
2015-01-18 08:27:28 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery : : executeProjection ( QueryPlan & query_plan , const ActionsDAGPtr & expression )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
auto projection_step = std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , expression ) ;
projection_step - > setStepDescription ( " Projection " ) ;
query_plan . addStep ( std : : move ( projection_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2012-02-27 06:28:20 +00:00
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeDistinct ( QueryPlan & query_plan , bool before_order , Names columns , bool pre_distinct )
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery ( ) ;
2019-03-26 18:28:37 +00:00
if ( query . distinct )
{
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2019-11-15 18:41:18 +00:00
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , * context ) ;
2019-03-26 18:28:37 +00:00
UInt64 limit_for_distinct = 0 ;
2020-07-12 05:18:01 +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.
if ( ( ! query . orderBy ( ) | | ! before_order ) & & limit_length < = std : : numeric_limits < UInt64 > : : max ( ) - limit_offset )
2019-03-26 18:28:37 +00:00
limit_for_distinct = limit_length + limit_offset ;
SizeLimits limits ( settings . max_rows_in_distinct , settings . max_bytes_in_distinct , settings . distinct_overflow_mode ) ;
2020-06-18 17:45:00 +00:00
auto distinct_step = std : : make_unique < DistinctStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-18 13:00:16 +00:00
limits , limit_for_distinct , columns , pre_distinct ) ;
if ( pre_distinct )
2020-06-18 17:45:00 +00:00
distinct_step - > setStepDescription ( " Preliminary DISTINCT " ) ;
2019-04-30 17:01:12 +00:00
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( distinct_step ) ) ;
2019-03-26 18:28:37 +00:00
}
}
2013-06-01 07:43:57 +00:00
2019-03-26 18:28:37 +00:00
/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined.
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executePreLimit ( QueryPlan & query_plan , bool do_not_skip_offset )
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery ( ) ;
2019-03-26 18:28:37 +00:00
/// If there is LIMIT
2019-05-14 11:04:11 +00:00
if ( query . limitLength ( ) )
2019-03-26 18:28:37 +00:00
{
2019-11-15 18:41:18 +00:00
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , * context ) ;
2020-03-13 13:44:08 +00:00
if ( do_not_skip_offset )
2019-03-26 18:28:37 +00:00
{
2020-07-12 05:18:01 +00:00
if ( limit_length > std : : numeric_limits < UInt64 > : : max ( ) - limit_offset )
return ;
2020-03-13 13:44:08 +00:00
limit_length + = limit_offset ;
limit_offset = 0 ;
}
2019-04-30 17:06:41 +00:00
2020-06-18 17:45:00 +00:00
auto limit = std : : make_unique < LimitStep > ( query_plan . getCurrentDataStream ( ) , limit_length , limit_offset ) ;
limit - > setStepDescription ( " preliminary LIMIT " ) ;
query_plan . addStep ( std : : move ( limit ) ) ;
2019-03-26 18:28:37 +00:00
}
}
2012-06-24 23:17:06 +00:00
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeLimitBy ( QueryPlan & query_plan )
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery ( ) ;
2019-05-14 11:04:11 +00:00
if ( ! query . limitByLength ( ) | | ! query . limitBy ( ) )
2019-03-26 18:28:37 +00:00
return ;
Names columns ;
2019-05-14 11:04:11 +00:00
for ( const auto & elem : query . limitBy ( ) - > children )
2019-03-26 18:28:37 +00:00
columns . emplace_back ( elem - > getColumnName ( ) ) ;
2020-05-07 13:40:50 +00:00
UInt64 length = getLimitUIntValue ( query . limitByLength ( ) , * context , " LIMIT " ) ;
UInt64 offset = ( query . limitByOffset ( ) ? getLimitUIntValue ( query . limitByOffset ( ) , * context , " OFFSET " ) : 0 ) ;
2019-03-26 18:28:37 +00:00
2020-06-18 17:45:00 +00:00
auto limit_by = std : : make_unique < LimitByStep > ( query_plan . getCurrentDataStream ( ) , length , offset , columns ) ;
query_plan . addStep ( std : : move ( limit_by ) ) ;
2019-03-26 18:28:37 +00:00
}
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeWithFill ( QueryPlan & query_plan )
2019-08-14 17:01:47 +00:00
{
2019-08-23 11:20:53 +00:00
auto & query = getSelectQuery ( ) ;
if ( query . orderBy ( ) )
{
2019-11-15 18:41:18 +00:00
SortDescription order_descr = getSortDescription ( query , * context ) ;
2019-08-23 11:20:53 +00:00
SortDescription fill_descr ;
for ( auto & desc : order_descr )
{
if ( desc . with_fill )
fill_descr . push_back ( desc ) ;
}
if ( fill_descr . empty ( ) )
return ;
2020-06-18 17:45:00 +00:00
auto filling_step = std : : make_unique < FillingStep > ( query_plan . getCurrentDataStream ( ) , std : : move ( fill_descr ) ) ;
query_plan . addStep ( std : : move ( filling_step ) ) ;
2017-04-01 07:20:54 +00:00
}
2011-08-28 05:13:24 +00:00
}
2019-08-14 17:01:47 +00:00
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeLimit ( QueryPlan & query_plan )
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery ( ) ;
2019-03-26 18:28:37 +00:00
/// If there is LIMIT
2019-05-14 11:04:11 +00:00
if ( query . limitLength ( ) )
2019-03-26 18:28:37 +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 .
*
* 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 .
*/
bool always_read_till_end = false ;
2019-05-14 11:04:11 +00:00
if ( query . group_by_with_totals & & ! query . orderBy ( ) )
2019-03-26 18:28:37 +00:00
always_read_till_end = true ;
if ( ! query . group_by_with_totals & & hasWithTotalsInAnySubqueryInFromClause ( query ) )
always_read_till_end = true ;
UInt64 limit_length ;
UInt64 limit_offset ;
2019-11-15 18:41:18 +00:00
std : : tie ( limit_length , limit_offset ) = getLimitLengthAndOffset ( query , * context ) ;
2019-03-26 18:28:37 +00:00
2019-08-27 17:48:42 +00:00
SortDescription order_descr ;
if ( query . limit_with_ties )
{
if ( ! query . orderBy ( ) )
throw Exception ( " LIMIT WITH TIES without ORDER BY " , ErrorCodes : : LOGICAL_ERROR ) ;
2019-11-15 18:41:18 +00:00
order_descr = getSortDescription ( query , * context ) ;
2019-08-27 17:48:42 +00:00
}
2020-06-18 17:45:00 +00:00
auto limit = std : : make_unique < LimitStep > (
query_plan . getCurrentDataStream ( ) ,
2020-06-16 18:23:01 +00:00
limit_length , limit_offset , always_read_till_end , query . limit_with_ties , order_descr ) ;
2019-04-30 17:01:12 +00:00
2020-06-16 18:23:01 +00:00
if ( query . limit_with_ties )
2020-06-18 17:45:00 +00:00
limit - > setStepDescription ( " LIMIT WITH TIES " ) ;
2020-06-16 18:23:01 +00:00
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( limit ) ) ;
2019-03-26 18:28:37 +00:00
}
}
2011-08-28 05:13:24 +00:00
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeOffset ( QueryPlan & query_plan )
2020-05-06 06:50:55 +00:00
{
auto & query = getSelectQuery ( ) ;
2020-05-11 04:47:31 +00:00
/// If there is not a LIMIT but an offset
2020-05-06 06:50:55 +00:00
if ( ! query . limitLength ( ) & & query . limitOffset ( ) )
{
UInt64 limit_length ;
UInt64 limit_offset ;
std : : tie ( limit_length , limit_offset ) = getLimitLengthAndOffset ( query , * context ) ;
2020-06-24 12:09:01 +00:00
auto offsets_step = std : : make_unique < OffsetStep > ( query_plan . getCurrentDataStream ( ) , limit_offset ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( offsets_step ) ) ;
2020-05-06 06:50:55 +00:00
}
}
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : executeExtremes ( QueryPlan & query_plan )
2019-03-26 18:28:37 +00:00
{
2019-11-15 18:41:18 +00:00
if ( ! context - > getSettingsRef ( ) . extremes )
2019-03-26 18:28:37 +00:00
return ;
2020-06-18 17:45:00 +00:00
auto extremes_step = std : : make_unique < ExtremesStep > ( query_plan . getCurrentDataStream ( ) ) ;
query_plan . addStep ( std : : move ( extremes_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2020-09-02 13:04:54 +00:00
void InterpreterSelectQuery : : executeSubqueriesInSetsAndJoins ( QueryPlan & query_plan , SubqueriesForSets & subqueries_for_sets )
2019-03-26 18:28:37 +00:00
{
2020-05-13 13:49:10 +00:00
if ( query_info . input_order_info )
2020-06-18 17:45:00 +00:00
executeMergeSorted ( query_plan , query_info . input_order_info - > order_key_prefix_descr , 0 , " before creating sets for subqueries and joins " ) ;
2019-10-17 11:56:05 +00:00
2019-11-15 18:41:18 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2019-03-26 18:28:37 +00:00
2020-09-15 13:25:14 +00:00
SizeLimits limits ( settings . max_rows_to_transfer , settings . max_bytes_to_transfer , settings . transfer_overflow_mode ) ;
2020-09-15 17:13:13 +00:00
addCreatingSetsStep ( query_plan , std : : move ( subqueries_for_sets ) , limits , * context ) ;
2019-03-26 18:28:37 +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 ( ) )
2019-11-15 18:41:18 +00:00
InterpreterSetQuery ( query . settings ( ) , * context ) . executeForCurrentContext ( ) ;
2015-06-05 21:28:04 +00:00
}
2011-08-28 05:13:24 +00:00
}