2020-06-17 20:19:55 +00:00
# include <DataTypes/DataTypeAggregateFunction.h>
2021-10-31 16:22:20 +00:00
# include <DataTypes/DataTypeInterval.h>
2020-06-17 20:19:55 +00:00
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>
2021-08-16 20:34:39 +00:00
# include <Parsers/ASTSelectIntersectExceptQuery.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
2021-10-31 08:51:20 +00:00
# include <Access/Common/AccessFlags.h>
2020-12-18 20:09:39 +00:00
# include <Access/ContextAccess.h>
2019-11-17 11:57:02 +00:00
2020-12-10 23:56:57 +00:00
# include <AggregateFunctions/AggregateFunctionCount.h>
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-02-28 15:23:32 +00:00
# include <Interpreters/JoinedTables.h>
2020-11-19 15:52:11 +00:00
# include <Interpreters/OpenTelemetrySpanLog.h>
2020-03-18 21:38:27 +00:00
# include <Interpreters/QueryAliasesVisitor.h>
2020-12-12 16:42:15 +00:00
# include <Interpreters/replaceAliasColumnsInQuery.h>
2017-04-01 09:19:00 +00:00
2021-10-16 14:03:50 +00:00
# include <QueryPipeline/Pipe.h>
2020-12-10 19:06:52 +00:00
# include <Processors/QueryPlan/AggregatingStep.h>
2020-08-11 12:03:18 +00:00
# include <Processors/QueryPlan/ArrayJoinStep.h>
2020-12-10 19:06:52 +00:00
# include <Processors/QueryPlan/CreatingSetsStep.h>
# include <Processors/QueryPlan/CubeStep.h>
# include <Processors/QueryPlan/DistinctStep.h>
2020-06-16 09:42:25 +00:00
# include <Processors/QueryPlan/ExpressionStep.h>
2020-12-10 19:06:52 +00:00
# include <Processors/QueryPlan/ExtremesStep.h>
# include <Processors/QueryPlan/FillingStep.h>
2020-06-16 09:42:25 +00:00
# include <Processors/QueryPlan/FilterStep.h>
2021-04-28 17:32:12 +00:00
# include <Processors/QueryPlan/JoinStep.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>
2021-10-18 14:37:46 +00:00
# include <Processors/QueryPlan/SortingStep.h>
2020-06-16 18:57:21 +00:00
# include <Processors/QueryPlan/MergingAggregatedStep.h>
2020-06-24 12:09:01 +00:00
# include <Processors/QueryPlan/OffsetStep.h>
2020-06-18 13:38:45 +00:00
# include <Processors/QueryPlan/QueryPlan.h>
2020-12-10 19:06:52 +00:00
# include <Processors/QueryPlan/ReadFromPreparedSource.h>
# include <Processors/QueryPlan/ReadNothingStep.h>
# include <Processors/QueryPlan/RollupStep.h>
# include <Processors/QueryPlan/SettingQuotaAndLimitsStep.h>
# include <Processors/QueryPlan/TotalsHavingStep.h>
# include <Processors/QueryPlan/WindowStep.h>
2021-03-04 17:38:12 +00:00
# include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
2020-12-10 19:06:52 +00:00
# include <Processors/Sources/NullSource.h>
2021-10-08 14:03:54 +00:00
# include <Processors/Sources/SourceFromSingleChunk.h>
2020-12-10 19:06:52 +00:00
# include <Processors/Transforms/AggregatingTransform.h>
# include <Processors/Transforms/ExpressionTransform.h>
# include <Processors/Transforms/FilterTransform.h>
2019-03-26 18:28:37 +00:00
2020-06-17 20:19:55 +00:00
# include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
# include <Storages/IStorage.h>
# include <Storages/StorageView.h>
# include <Functions/IFunction.h>
# include <Core/Field.h>
2021-10-12 11:41:23 +00:00
# include <Core/ProtocolDefines.h>
2021-10-02 07:13:14 +00:00
# include <base/types.h>
2020-06-17 20:19:55 +00:00
# include <Columns/Collator.h>
# include <Common/FieldVisitorsAccurateComparison.h>
2021-06-14 04:13:35 +00:00
# include <Common/FieldVisitorToString.h>
2020-06-17 20:19:55 +00:00
# include <Common/typeid_cast.h>
# include <Common/checkStackSize.h>
2021-10-02 07:13:14 +00:00
# include <base/map.h>
# include <base/scope_guard_safe.h>
2020-06-17 20:19:55 +00:00
# include <memory>
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 ;
2020-12-18 20:09:39 +00:00
extern const int ACCESS_DENIED ;
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.
2021-02-16 09:37:19 +00:00
String InterpreterSelectQuery : : generateFilterActions ( ActionsDAGPtr & actions , 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 ) ;
2020-10-26 15:49:00 +00:00
table_expr - > database_and_table_name = std : : make_shared < ASTTableIdentifier > ( db_name , table_name ) ;
2019-03-29 20:31:06 +00:00
table_expr - > children . push_back ( table_expr - > database_and_table_name ) ;
/// Using separate expression analyzer to prevent any possible alias injection
2021-04-10 23:33:54 +00:00
auto syntax_result = TreeRewriter ( context ) . analyzeSelect ( query_ast , TreeRewriterResult ( { } , storage , metadata_snapshot ) ) ;
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
2021-08-03 18:03:24 +00:00
auto column_name = expr_list - > children . at ( 0 ) - > getColumnName ( ) ;
2021-03-10 08:41:24 +00:00
actions - > removeUnusedActions ( NameSet { column_name } ) ;
2021-02-20 11:00:16 +00:00
actions - > projectInput ( false ) ;
for ( const auto * node : actions - > getInputs ( ) )
2021-03-05 15:21:14 +00:00
actions - > getIndex ( ) . push_back ( node ) ;
2021-02-20 11:00:16 +00:00
return column_name ;
2019-03-29 20:31:06 +00:00
}
2018-02-26 21:00:42 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2021-04-10 23:33:54 +00:00
ContextPtr context_ ,
2019-08-03 11:02:40 +00:00
const SelectQueryOptions & options_ ,
const Names & required_result_column_names_ )
2021-10-13 18:22:02 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , std : : nullopt , nullptr , options_ , required_result_column_names_ )
2018-02-26 21:00:42 +00:00
{
}
2021-10-29 11:51:41 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
ContextPtr context_ ,
const SelectQueryOptions & options_ ,
PreparedSets prepared_sets_ )
: InterpreterSelectQuery ( query_ptr_ , context_ , std : : nullopt , nullptr , options_ , { } , { } , std : : move ( prepared_sets_ ) )
{
}
2020-01-29 15:04:46 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2021-04-10 23:33:54 +00:00
ContextPtr context_ ,
2020-01-29 15:04:46 +00:00
Pipe input_pipe_ ,
const SelectQueryOptions & options_ )
2021-10-13 18:22:02 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , 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_ ,
2021-04-10 23:33:54 +00:00
ContextPtr context_ ,
2018-07-18 12:17:48 +00:00
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_ )
2021-10-13 18:22:02 +00:00
: InterpreterSelectQuery ( query_ptr_ , context_ , 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 .
*/
2021-04-10 23:33:54 +00:00
static ContextPtr getSubqueryContext ( ContextPtr context )
2018-02-26 21:00:42 +00:00
{
2021-04-10 23:33:54 +00:00
auto subquery_context = Context : : createCopy ( context ) ;
Settings subquery_settings = context - > getSettings ( ) ;
2018-07-19 13:36:21 +00:00
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 ;
2021-04-10 23:33:54 +00:00
subquery_context - > setSettings ( subquery_settings ) ;
2018-07-19 13:36:21 +00:00
return subquery_context ;
2018-02-26 21:00:42 +00:00
}
2016-01-11 21:46:36 +00:00
2021-02-15 19:40:32 +00:00
static void rewriteMultipleJoins ( ASTPtr & query , const TablesWithColumns & tables , const String & database , const Settings & settings )
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 } ;
2021-02-15 19:40:32 +00:00
cross_to_inner . cross_to_inner_join_rewrite = settings . cross_to_inner_join_rewrite ;
2020-06-15 12:36:10 +00:00
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 ) ;
}
2020-12-18 20:09:39 +00:00
/// Checks that the current user has the SELECT privilege.
static void checkAccessRightsForSelect (
2021-04-10 23:33:54 +00:00
ContextPtr context ,
2020-12-18 20:09:39 +00:00
const StorageID & table_id ,
const StorageMetadataPtr & table_metadata ,
const TreeRewriterResult & syntax_analyzer_result )
{
if ( ! syntax_analyzer_result . has_explicit_columns & & table_metadata & & ! table_metadata - > getColumns ( ) . empty ( ) )
{
/// For a trivial query like "SELECT count() FROM table" access is granted if at least
/// one column is accessible.
/// In this case just checking access for `required_columns` doesn't work correctly
/// because `required_columns` will contain the name of a column of minimum size (see TreeRewriterResult::collectUsedColumns())
/// which is probably not the same column as the column the current user has access to.
2021-04-10 23:33:54 +00:00
auto access = context - > getAccess ( ) ;
2020-12-18 20:09:39 +00:00
for ( const auto & column : table_metadata - > getColumns ( ) )
{
if ( access - > isGranted ( AccessType : : SELECT , table_id . database_name , table_id . table_name , column . name ) )
return ;
}
2021-06-03 14:05:37 +00:00
throw Exception (
ErrorCodes : : ACCESS_DENIED ,
" {}: Not enough privileges. To execute this query it's necessary to have grant SELECT for at least one column on {} " ,
context - > getUserName ( ) ,
table_id . getFullTableName ( ) ) ;
2020-12-18 20:09:39 +00:00
}
/// General check.
2021-06-17 15:00:03 +00:00
context - > checkAccess ( AccessType : : SELECT , table_id , syntax_analyzer_result . requiredSourceColumnsForAccessCheck ( ) ) ;
2020-12-18 20:09:39 +00:00
}
2020-12-21 23:58:54 +00:00
/// Returns true if we should ignore quotas and limits for a specified table in the system database.
static bool shouldIgnoreQuotaAndLimits ( const StorageID & table_id )
{
if ( table_id . database_name = = DatabaseCatalog : : SYSTEM_DATABASE )
{
static const boost : : container : : flat_set < String > tables_ignoring_quota { " quotas " , " quota_limits " , " quota_usage " , " quotas_usage " , " one " } ;
if ( tables_ignoring_quota . count ( table_id . table_name ) )
return true ;
}
return false ;
}
2018-07-17 13:09:33 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2021-04-10 23:33:54 +00:00
ContextPtr context_ ,
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 ,
2021-10-29 11:51:41 +00:00
const StorageMetadataPtr & metadata_snapshot_ ,
PreparedSets prepared_sets_ )
2019-02-11 19:53:55 +00:00
/// NOTE: the query almost always should be cloned because it will be modified during analysis.
2020-11-01 13:54:07 +00:00
: IInterpreterUnionOrSelectQuery ( options_ . modify_inplace ? query_ptr_ : query_ptr_ - > clone ( ) , context_ , options_ )
2018-07-17 13:09:33 +00:00
, storage ( storage_ )
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_ )
2021-10-29 11:51:41 +00:00
, prepared_sets ( std : : move ( prepared_sets_ ) )
2014-12-26 10:35:03 +00:00
{
2019-08-10 17:51:47 +00:00
checkStackSize ( ) ;
2021-04-21 16:00:27 +00:00
query_info . ignore_projections = options . ignore_projections ;
2021-07-07 05:01:30 +00:00
query_info . is_projection_query = options . is_projection_query ;
2021-10-19 14:09:43 +00:00
query_info . original_query = query_ptr - > clone ( ) ;
2021-04-21 16:00:27 +00:00
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 ) ;
2021-10-13 18:22:02 +00:00
bool has_input = input_pipe ! = std : : nullopt ;
if ( input_pipe )
2020-01-29 15:04:46 +00:00
{
/// 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
2021-01-13 14:28:36 +00:00
// Only propagate WITH elements to subqueries if we're not a subquery
2021-02-08 16:25:24 +00:00
if ( ! options . is_subquery )
2021-01-13 14:28:36 +00:00
{
if ( context - > getSettingsRef ( ) . enable_global_with_statement )
ApplyWithAliasVisitor ( ) . visit ( query_ptr ) ;
ApplyWithSubqueryVisitor ( ) . visit ( query_ptr ) ;
}
2020-09-12 17:00:04 +00:00
2021-07-08 10:49:13 +00:00
JoinedTables joined_tables ( getSubqueryContext ( context ) , getSelectQuery ( ) , options . with_all_cols ) ;
2020-03-08 11:07:05 +00:00
2020-12-21 23:58:54 +00:00
bool got_storage_from_query = false ;
2020-03-08 11:07:05 +00:00
if ( ! has_input & & ! storage )
2020-12-21 23:58:54 +00:00
{
2020-03-08 11:07:05 +00:00
storage = joined_tables . getLeftTableStorage ( ) ;
2020-12-21 23:58:54 +00:00
got_storage_from_query = true ;
}
2020-03-08 11:07:05 +00:00
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-12-21 23:58:54 +00:00
if ( ! metadata_snapshot )
2020-06-17 11:52:19 +00:00
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
{
2021-02-15 19:40:32 +00:00
rewriteMultipleJoins ( query_ptr , joined_tables . tablesWithColumns ( ) , context - > getCurrentDatabase ( ) , context - > getSettingsRef ( ) ) ;
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-12-21 23:58:54 +00:00
storage = nullptr ;
2020-06-18 16:10:47 +00:00
table_lock . reset ( ) ;
2020-03-08 11:07:05 +00:00
table_id = StorageID : : createEmpty ( ) ;
2020-12-21 23:58:54 +00:00
metadata_snapshot = nullptr ;
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
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 ( ) ) ;
2021-06-03 14:05:37 +00:00
/// Reuse already built sets for multiple passes of analysis
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
2021-04-10 23:33:54 +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
2021-05-03 13:00:59 +00:00
query_info . syntax_analyzer_result = syntax_analyzer_result ;
2021-07-30 09:23:49 +00:00
context - > setDistributed ( syntax_analyzer_result - > is_remote_storage ) ;
2021-05-03 13:00:59 +00:00
2021-06-28 10:35:55 +00:00
if ( storage & & ! query . final ( ) & & storage - > needRewriteQueryWithFinal ( syntax_analyzer_result - > requiredSourceColumns ( ) ) )
query . setFinal ( ) ;
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 ( ) )
2021-04-10 23:33:54 +00:00
context - > getQueryContext ( ) - > addScalar ( it . first , it . second ) ;
2020-06-18 16:11:23 +00:00
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 ;
}
2021-07-12 15:19:49 +00:00
if ( try_move_to_prewhere & & storage & & storage - > supportsPrewhere ( ) & & query . where ( ) & & ! query . prewhere ( ) )
2020-05-20 09:29:23 +00:00
{
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable
2021-02-06 15:08:42 +00:00
if ( const auto & column_sizes = storage - > getColumnSizes ( ) ; ! column_sizes . empty ( ) )
2020-05-20 09:29:23 +00:00
{
2021-02-06 15:08:42 +00:00
/// Extract column compressed sizes.
std : : unordered_map < std : : string , UInt64 > column_compressed_sizes ;
for ( const auto & [ name , sizes ] : column_sizes )
column_compressed_sizes [ name ] = sizes . data_compressed ;
2020-05-20 09:29:23 +00:00
SelectQueryInfo current_info ;
current_info . query = query_ptr ;
current_info . syntax_analyzer_result = syntax_analyzer_result ;
2021-04-10 23:33:54 +00:00
MergeTreeWhereOptimizer {
current_info ,
context ,
std : : move ( column_compressed_sizes ) ,
metadata_snapshot ,
syntax_analyzer_result - > requiredSourceColumns ( ) ,
log } ;
2020-05-20 09:29:23 +00:00
}
}
2021-05-04 12:40:34 +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 ( ) ) ) ;
}
2019-10-27 18:12:40 +00:00
query_analyzer = std : : make_unique < SelectQueryExpressionAnalyzer > (
2021-06-03 14:05:37 +00:00
query_ptr ,
syntax_analyzer_result ,
context ,
metadata_snapshot ,
NameSet ( required_result_column_names . begin ( ) , required_result_column_names . end ( ) ) ,
! options . only_analyze ,
options ,
std : : move ( subquery_for_sets ) ,
std : : move ( prepared_sets ) ) ;
2017-04-01 07:20:54 +00:00
2019-10-27 18:12:40 +00:00
if ( ! options . only_analyze )
{
2021-10-13 18:22:02 +00:00
if ( query . sampleSize ( ) & & ( 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
2021-10-13 18:22:02 +00:00
if ( query . final ( ) & & ( input_pipe | | ! storage | | ! storage - > supportsFinal ( ) ) )
2021-06-03 14:05:37 +00:00
throw Exception (
2021-10-13 18:22:02 +00:00
( ! input_pipe & & storage ) ? " Storage " + storage - > getName ( ) + " doesn't support FINAL " : " Illegal FINAL " ,
2021-06-03 14:05:37 +00:00
ErrorCodes : : ILLEGAL_FINAL ) ;
2017-11-05 17:48:50 +00:00
2021-10-13 18:22:02 +00:00
if ( query . prewhere ( ) & & ( input_pipe | | ! storage | | ! storage - > supportsPrewhere ( ) ) )
2021-06-03 14:05:37 +00:00
throw Exception (
2021-10-13 18:22:02 +00:00
( ! input_pipe & & storage ) ? " Storage " + storage - > getName ( ) + " doesn't support PREWHERE " : " Illegal PREWHERE " ,
2021-06-03 14:05:37 +00:00
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
{
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
{
2021-02-13 22:07:13 +00:00
filter_info = std : : make_shared < FilterDAGInfo > ( ) ;
2021-02-16 09:37:19 +00:00
filter_info - > column_name = generateFilterActions ( filter_info - > actions , required_columns ) ;
2021-02-20 11:00:16 +00:00
auto required_columns_from_filter = filter_info - > actions - > getRequiredColumns ( ) ;
for ( const auto & column : required_columns_from_filter )
{
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column . name ) )
required_columns . push_back ( column . name ) ;
}
2019-10-27 18:12:40 +00:00
}
2019-10-03 11:58:52 +00:00
2021-02-20 11:00:16 +00:00
source_header = metadata_snapshot - > getSampleBlockForColumns ( required_columns , storage - > getVirtuals ( ) , storage - > getStorageID ( ) ) ;
2019-10-03 11:58:52 +00:00
}
2019-11-03 07:27:35 +00:00
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
} ;
2021-07-03 22:53:51 +00:00
analyze ( shouldMoveToPrewhere ( ) ) ;
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
}
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
{
2021-03-08 06:33:30 +00:00
LOG_TRACE ( log , " Running 'analyze' second time " ) ;
2021-06-03 14:05:37 +00:00
/// Reuse already built sets for multiple passes of analysis
subquery_for_sets = std : : move ( query_analyzer - > getSubqueriesForSets ( ) ) ;
2021-10-29 11:51:41 +00:00
prepared_sets = query_info . sets . empty ( ) ? std : : move ( query_analyzer - > getPreparedSets ( ) ) : std : : move ( query_info . sets ) ;
2021-03-05 14:43:26 +00:00
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-12-21 23:58:54 +00:00
if ( table_id & & got_storage_from_query & & ! joined_tables . isLeftTableFunction ( ) )
2019-11-11 01:11:32 +00:00
{
2021-06-15 10:28:13 +00:00
/// The current user should have the SELECT privilege. If this table_id is for a table
/// function we don't check access rights here because in this case they have been already
/// checked in ITableFunction::execute().
2021-06-17 15:00:03 +00:00
checkAccessRightsForSelect ( context , table_id , metadata_snapshot , * syntax_analyzer_result ) ;
2020-12-21 23:58:54 +00:00
/// Remove limits for some tables in the `system` database.
if ( shouldIgnoreQuotaAndLimits ( table_id ) & & ( joined_tables . tablesCount ( ) < = 1 ) )
2020-05-08 12:50:45 +00:00
{
options . ignore_quota = true ;
options . ignore_limits = true ;
}
2019-11-11 01:11:32 +00:00
}
2020-01-24 16:20:36 +00:00
2021-05-05 18:52:20 +00:00
/// Add prewhere actions with alias columns and record needed columns from storage.
if ( storage )
{
addPrewhereAliasActions ( ) ;
analysis_result . required_columns = required_columns ;
}
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
}
2020-06-18 17:45:00 +00:00
void InterpreterSelectQuery : : buildQueryPlan ( QueryPlan & query_plan )
{
2021-10-13 18:22:02 +00:00
executeImpl ( query_plan , std : : move ( input_pipe ) ) ;
2020-06-18 17:45:00 +00:00
/// We must guarantee that result structure is the same as in getSampleBlock()
2021-04-22 08:43:35 +00:00
///
2021-07-07 05:01:30 +00:00
/// But if it's a projection query, plan header does not match result_header.
2021-04-22 08:43:35 +00:00
/// TODO: add special stage for InterpreterSelectQuery?
2021-07-07 05:01:30 +00:00
if ( ! options . is_projection_query & & ! blocksHaveEqualStructure ( query_plan . getCurrentDataStream ( ) . header , result_header ) )
2020-06-18 17:45:00 +00:00
{
2020-11-17 17:16:55 +00:00
auto convert_actions_dag = ActionsDAG : : makeConvertingActions (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) . header . getColumnsWithTypeAndName ( ) ,
result_header . getColumnsWithTypeAndName ( ) ,
ActionsDAG : : MatchColumnsMode : : Name ,
true ) ;
2020-11-17 17:16:55 +00:00
auto converting = std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , convert_actions_dag ) ;
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
2021-09-15 19:35:48 +00:00
res . pipeline = QueryPipelineBuilder : : getPipeline ( std : : move ( * query_plan . buildQueryPipeline (
QueryPlanOptimizationSettings : : fromContext ( context ) , BuildQueryPipelineSettings : : fromContext ( context ) ) ) ) ;
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-11-19 15:52:11 +00:00
OpenTelemetrySpanHolder span ( __PRETTY_FUNCTION__ ) ;
2020-09-10 19:55:36 +00:00
query_info . query = query_ptr ;
2021-05-05 21:26:14 +00:00
query_info . has_window = query_analyzer - > hasWindow ( ) ;
2019-10-03 15:47:42 +00:00
if ( storage & & ! options . only_analyze )
2021-02-10 14:12:49 +00:00
{
2021-06-30 10:39:58 +00:00
auto & query = getSelectQuery ( ) ;
query_analyzer - > makeSetsForIndex ( query . where ( ) ) ;
query_analyzer - > makeSetsForIndex ( query . prewhere ( ) ) ;
query_info . sets = query_analyzer - > getPreparedSets ( ) ;
2021-02-10 14:12:49 +00:00
}
2019-10-03 15:47:42 +00:00
if ( storage & & ! options . only_analyze )
2021-04-21 16:00:27 +00:00
from_stage = storage - > getQueryProcessingStage ( context , options . to_stage , metadata_snapshot , query_info ) ;
2021-02-18 18:41:50 +00:00
/// Do I need to perform the first part of the pipeline?
/// Running on remote servers during distributed processing or if query is not distributed.
///
/// Also note that with distributed_group_by_no_merge=1 or when there is
/// only one remote server, it is equal to local query in terms of query
/// stages (or when due to optimize_distributed_group_by_sharding_key the query was processed up to Complete stage).
2020-02-10 15:50:12 +00:00
bool first_stage = from_stage < QueryProcessingStage : : WithMergeableState
& & options . to_stage > = QueryProcessingStage : : WithMergeableState ;
2021-02-18 18:41:50 +00:00
/// Do I need to execute the second part of the pipeline?
/// Running on the initiating server during distributed processing or if query is not distributed.
///
/// Also note that with distributed_group_by_no_merge=2 (i.e. when optimize_distributed_group_by_sharding_key takes place)
2021-06-04 06:43:56 +00:00
/// the query on the remote server will be processed up to WithMergeableStateAfterAggregationAndLimit,
2021-02-18 18:41:50 +00:00
/// So it will do partial second stage (second_stage=true), and initiator will do the final part.
2020-02-10 15:50:12 +00:00
bool second_stage = from_stage < = QueryProcessingStage : : WithMergeableState
& & options . to_stage > QueryProcessingStage : : WithMergeableState ;
analysis_result = ExpressionAnalysisResult (
2021-06-03 14:05:37 +00:00
* query_analyzer , metadata_snapshot , first_stage , second_stage , options . only_analyze , filter_info , 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
{
2021-05-19 08:43:16 +00:00
header = analysis_result . prewhere_info - > prewhere_actions - > updateHeader ( 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-12-25 03:13:30 +00:00
{
// What's the difference with selected_columns?
2021-04-23 15:45:45 +00:00
// Here we calculate the header we want from remote server after it
// executes query up to WithMergeableState. When there is an ORDER BY,
// it is executed on remote server firstly, then we execute merge
// sort on initiator. To execute ORDER BY, we need to calculate the
// ORDER BY keys. These keys might be not present among the final
// SELECT columns given by the `selected_column`. This is why we have
// to use proper keys given by the result columns of the
// `before_order_by` expression actions.
// Another complication is window functions -- if we have them, they
// are calculated on initiator, before ORDER BY columns. In this case,
// the shard has to return columns required for window function
// calculation and further steps, given by the `before_window`
// expression actions.
// As of 21.6 this is broken: the actions in `before_window` might
// not contain everything required for the ORDER BY step, but this
// is a responsibility of ExpressionAnalyzer and is not a problem
// with this code. See
// https://github.com/ClickHouse/ClickHouse/issues/19857 for details.
2021-04-23 10:58:54 +00:00
if ( analysis_result . before_window )
return analysis_result . before_window - > getResultColumns ( ) ;
2021-06-03 14:05:37 +00:00
2020-12-25 03:13:30 +00:00
return analysis_result . before_order_by - > 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 ;
}
2021-06-04 06:43:56 +00:00
if ( options . to_stage > = QueryProcessingStage : : Enum : : WithMergeableStateAfterAggregation )
2020-08-15 13:57:17 +00:00
{
2021-04-23 15:45:45 +00:00
// It's different from selected_columns, see the comment above for
// WithMergeableState stage.
2021-04-23 10:58:54 +00:00
if ( analysis_result . before_window )
return analysis_result . before_window - > getResultColumns ( ) ;
2020-12-25 03:13:30 +00:00
return analysis_result . before_order_by - > 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
}
2021-04-10 23:33:54 +00:00
static Field getWithFillFieldValue ( const ASTPtr & node , ContextPtr context )
2019-08-14 17:01:47 +00:00
{
2021-10-31 16:22:20 +00:00
auto [ field , type ] = evaluateConstantExpression ( node , context ) ;
2019-08-14 17:01:47 +00:00
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
2021-10-31 16:22:20 +00:00
static std : : pair < Field , std : : optional < IntervalKind > > getWithFillStep ( const ASTPtr & node , ContextPtr context )
{
auto [ field , type ] = evaluateConstantExpression ( node , context ) ;
if ( const auto * type_interval = typeid_cast < const DataTypeInterval * > ( type . get ( ) ) )
return std : : make_pair ( std : : move ( field ) , type_interval - > getKind ( ) ) ;
if ( isColumnedAsNumber ( type ) )
return std : : make_pair ( std : : move ( field ) , std : : nullopt ) ;
throw Exception ( " Illegal type " + type - > getName ( ) + " of WITH FILL expression, must be numeric type " , ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ) ;
}
2021-04-10 23:33:54 +00:00
static FillColumnDescription getWithFillDescription ( const ASTOrderByElement & order_by_elem , ContextPtr 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 ) ;
2021-10-31 16:22:20 +00:00
2019-08-19 20:22:45 +00:00
if ( order_by_elem . fill_step )
2021-10-31 16:22:20 +00:00
std : : tie ( descr . fill_step , descr . step_kind ) = getWithFillStep ( 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
2021-04-10 23:33:54 +00:00
static SortDescription getSortDescription ( const ASTSelectQuery & query , ContextPtr 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 )
{
2021-08-03 18:03:24 +00:00
String name = elem - > children . front ( ) - > getColumnName ( ) ;
2019-07-18 14:41:11 +00:00
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 ) ;
2021-06-03 14:05:37 +00:00
order_descr . emplace_back ( name , order_by_elem . direction , 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 ;
}
2021-08-03 18:03:24 +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 )
{
2021-08-03 18:03:24 +00:00
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
}
2021-04-10 23:33:54 +00:00
static UInt64 getLimitUIntValue ( const ASTPtr & node , ContextPtr context , const std : : string & expr )
2019-07-18 14:41:11 +00:00
{
const auto & [ field , type ] = evaluateConstantExpression ( node , context ) ;
if ( ! isNativeNumber ( type ) )
2021-06-03 14:05:37 +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 ( ) )
2021-06-03 14:05:37 +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 > ( ) ;
}
2021-04-10 23:33:54 +00:00
static std : : pair < UInt64 , UInt64 > getLimitLengthAndOffset ( const ASTSelectQuery & query , ContextPtr context )
2019-07-18 14:41:11 +00:00
{
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 } ;
}
2021-04-10 23:33:54 +00:00
static UInt64 getLimitForSorting ( const ASTSelectQuery & query , ContextPtr context )
2019-07-18 14:41:11 +00:00
{
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.
2021-06-28 09:53:54 +00:00
if ( ! query . distinct & & ! query . limitBy ( ) & & ! query . limit_with_ties & & ! query . arrayJoinExpressionList ( ) . first & & 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.
2021-11-09 08:16:18 +00:00
* 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 .
*/
2020-06-19 16:51:44 +00:00
if ( auto query_table = extractTableExpression ( query , 0 ) )
{
if ( const auto * ast_union = query_table - > as < ASTSelectWithUnionQuery > ( ) )
{
2021-11-01 13:19:31 +00:00
/** NOTE
* 1. For ASTSelectWithUnionQuery after normalization for union child node the height of the AST tree is at most 2.
* 2. For ASTSelectIntersectExceptQuery after normalization in case there are intersect or except nodes ,
* the height of the AST tree can have any depth ( each intersect / except adds a level ) , but the
* number of children in those nodes is always 2.
*/
std : : function < bool ( ASTPtr ) > traverse_recursively = [ & ] ( ASTPtr child_ast ) - > bool
2021-02-17 08:26:52 +00:00
{
2021-11-09 08:16:18 +00:00
if ( const auto * select_child = child_ast - > as < ASTSelectQuery > ( ) )
{
if ( hasWithTotalsInAnySubqueryInFromClause ( select_child - > as < ASTSelectQuery & > ( ) ) )
return true ;
}
else if ( const auto * union_child = child_ast - > as < ASTSelectWithUnionQuery > ( ) )
{
for ( const auto & subchild : union_child - > list_of_selects - > children )
2021-11-01 13:19:31 +00:00
if ( traverse_recursively ( subchild ) )
2021-02-17 08:26:52 +00:00
return true ;
2021-11-09 08:16:18 +00:00
}
else if ( const auto * intersect_child = child_ast - > as < ASTSelectIntersectExceptQuery > ( ) )
{
auto selects = intersect_child - > getListOfSelects ( ) ;
for ( const auto & subchild : selects )
2021-11-01 13:19:31 +00:00
if ( traverse_recursively ( subchild ) )
return true ;
2021-11-09 08:16:18 +00:00
}
2021-11-01 13:19:31 +00:00
return false ;
} ;
for ( const auto & elem : ast_union - > list_of_selects - > children )
2021-11-09 08:16:18 +00:00
if ( traverse_recursively ( elem ) )
return true ;
2020-06-19 16:51:44 +00:00
}
}
return false ;
}
2021-10-13 18:22:02 +00:00
void InterpreterSelectQuery : : executeImpl ( QueryPlan & query_plan , 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
2021-05-04 10:52:37 +00:00
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
bool aggregate_overflow_row =
expressions . need_aggregate & &
query . group_by_with_totals & &
settings . max_rows_to_group_by & &
settings . group_by_overflow_mode = = OverflowMode : : ANY & &
settings . totals_mode ! = TotalsMode : : AFTER_HAVING_EXCLUSIVE ;
/// Do I need to immediately finalize the aggregate functions after the aggregation?
bool aggregate_final =
expressions . need_aggregate & &
options . to_stage > QueryProcessingStage : : WithMergeableState & &
! query . group_by_with_totals & & ! query . group_by_with_rollup & & ! query . group_by_with_cube ;
if ( query_info . projection & & query_info . projection - > desc - > type = = ProjectionDescription : : Type : : Aggregate )
{
query_info . projection - > aggregate_overflow_row = aggregate_overflow_row ;
query_info . projection - > aggregate_final = aggregate_final ;
}
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
2021-02-15 19:48:06 +00:00
if ( expressions . filter_info )
2021-01-25 18:09:17 +00:00
{
auto row_level_security_step = std : : make_unique < FilterStep > (
query_plan . getCurrentDataStream ( ) ,
2021-02-13 22:07:13 +00:00
expressions . filter_info - > actions ,
2021-01-25 18:09:17 +00:00
expressions . filter_info - > column_name ,
expressions . filter_info - > do_remove_column ) ;
2021-02-15 19:48:06 +00:00
row_level_security_step - > setStepDescription ( " Row-level security filter " ) ;
2021-01-25 18:09:17 +00:00
query_plan . addStep ( std : : move ( row_level_security_step ) ) ;
}
2018-08-20 12:57:31 +00:00
if ( expressions . prewhere_info )
2019-04-03 11:21:38 +00:00
{
2021-06-25 14:49:28 +00:00
if ( expressions . prewhere_info - > row_level_filter )
2021-02-15 19:48:06 +00:00
{
auto row_level_filter_step = std : : make_unique < FilterStep > (
query_plan . getCurrentDataStream ( ) ,
2021-06-25 14:49:28 +00:00
expressions . prewhere_info - > row_level_filter ,
2021-02-15 19:48:06 +00:00
expressions . prewhere_info - > row_level_column_name ,
false ) ;
row_level_filter_step - > setStepDescription ( " Row-level security filter (PREWHERE) " ) ;
query_plan . addStep ( std : : move ( row_level_filter_step ) ) ;
}
2020-06-18 13:38:45 +00:00
auto prewhere_step = std : : make_unique < FilterStep > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) ,
expressions . prewhere_info - > prewhere_actions ,
expressions . prewhere_info - > prewhere_column_name ,
expressions . prewhere_info - > remove_prewhere_column ) ;
2020-06-16 09:42:25 +00:00
2020-06-18 13:38:45 +00:00
prewhere_step - > setStepDescription ( " PREWHERE " ) ;
query_plan . addStep ( std : : move ( prewhere_step ) ) ;
2019-04-03 11:21:38 +00:00
}
2018-07-19 13:36:21 +00:00
}
else
{
2021-10-13 18:22:02 +00:00
if ( prepared_pipe )
2020-01-29 15:04:46 +00:00
{
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?
2021-06-04 06:43:56 +00:00
if ( from_stage > = QueryProcessingStage : : WithMergeableStateAfterAggregation )
2020-08-15 13:57:17 +00:00
from_aggregation_stage = true ;
/// Is running on remote servers during distributed processing?
2021-06-04 06:43:56 +00:00
if ( options . to_stage > = QueryProcessingStage : : WithMergeableStateAfterAggregation )
2020-08-15 13:57:17 +00:00
to_aggregation_stage = true ;
2021-01-25 18:09:17 +00:00
/// Read the data from Storage. from_stage - to what stage the request was completed in Storage.
2021-02-15 19:48:06 +00:00
executeFetchColumns ( from_stage , query_plan ) ;
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
{
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 .
*/
2021-04-23 10:58:54 +00:00
if ( ! expressions . second_stage
& & ! expressions . need_aggregate
& & ! expressions . hasHaving ( )
& & ! expressions . has_window )
2020-03-30 18:50:42 +00:00
{
if ( expressions . has_order_by )
2021-05-02 13:38:19 +00:00
executeOrder (
query_plan ,
query_info . input_order_info ? query_info . input_order_info
: ( query_info . projection ? query_info . projection - > input_order_info : nullptr ) ) ;
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
{
2021-05-07 06:11:24 +00:00
// If there is a storage that supports prewhere, this will always be nullptr
// Thus, we don't actually need to check if projection is active.
if ( ! query_info . projection & & expressions . filter_info )
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 > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) ,
expressions . filter_info - > actions ,
expressions . filter_info - > column_name ,
expressions . filter_info - > do_remove_column ) ;
2020-06-16 12:02:10 +00:00
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 )
{
2021-06-03 14:05:37 +00:00
QueryPlanStepPtr before_array_join_step
= std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , expressions . before_array_join ) ;
2020-08-11 12:03:18 +00:00
before_array_join_step - > setStepDescription ( " Before ARRAY JOIN " ) ;
query_plan . addStep ( std : : move ( before_array_join_step ) ) ;
}
if ( expressions . array_join )
{
2021-06-03 14:05:37 +00:00
QueryPlanStepPtr array_join_step
= std : : make_unique < ArrayJoinStep > ( query_plan . getCurrentDataStream ( ) , expressions . array_join ) ;
2020-08-11 12:03:18 +00:00
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 ) ) ;
}
2021-02-09 13:17:42 +00:00
/// Optional step to convert key columns to common supertype.
if ( expressions . converting_join_columns )
{
QueryPlanStepPtr convert_join_step = std : : make_unique < ExpressionStep > (
query_plan . getCurrentDataStream ( ) ,
expressions . converting_join_columns ) ;
convert_join_step - > setStepDescription ( " Convert JOIN columns " ) ;
query_plan . addStep ( std : : move ( convert_join_step ) ) ;
}
2019-01-30 12:01:00 +00:00
if ( expressions . hasJoin ( ) )
2018-02-21 08:16:01 +00:00
{
2021-04-29 09:08:49 +00:00
if ( expressions . join - > isFilled ( ) )
2021-04-28 17:32:12 +00:00
{
2021-04-29 09:08:49 +00:00
QueryPlanStepPtr filled_join_step = std : : make_unique < FilledJoinStep > (
2021-04-28 17:32:12 +00:00
query_plan . getCurrentDataStream ( ) ,
expressions . join ,
settings . max_block_size ) ;
2021-04-29 09:08:49 +00:00
filled_join_step - > setStepDescription ( " JOIN " ) ;
query_plan . addStep ( std : : move ( filled_join_step ) ) ;
2021-04-28 17:32:12 +00:00
}
else
{
auto joined_plan = query_analyzer - > getJoinedPlan ( ) ;
if ( ! joined_plan )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " There is no joined plan for query " ) ;
QueryPlanStepPtr join_step = std : : make_unique < JoinStep > (
query_plan . getCurrentDataStream ( ) ,
joined_plan - > getCurrentDataStream ( ) ,
expressions . join ,
settings . max_block_size ) ;
join_step - > setStepDescription ( " JOIN " ) ;
std : : vector < QueryPlanPtr > plans ;
plans . emplace_back ( std : : make_unique < QueryPlan > ( std : : move ( query_plan ) ) ) ;
plans . emplace_back ( std : : move ( joined_plan ) ) ;
2017-04-01 07:20:54 +00:00
2021-04-28 17:32:12 +00:00
query_plan = QueryPlan ( ) ;
query_plan . unitePlans ( std : : move ( join_step ) , { std : : move ( plans ) } ) ;
}
2018-02-21 08:16:01 +00:00
}
2017-04-01 07:20:54 +00:00
2021-05-07 06:11:24 +00:00
if ( ! query_info . projection & & 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
{
2021-05-05 07:31:27 +00:00
executeAggregation (
query_plan , expressions . before_aggregation , aggregate_overflow_row , aggregate_final , query_info . input_order_info ) ;
/// We need to reset input order info, so that executeOrder can't use it
query_info . input_order_info . reset ( ) ;
2020-05-14 13:56:17 +00:00
}
2021-04-23 10:58:54 +00:00
// Now we must execute:
// 1) expressions before window functions,
// 2) window functions,
// 3) expressions after window functions,
// 4) preliminary distinct.
// This code decides which part we execute on shard (first_stage)
// and which part on initiator (second_stage). See also the counterpart
// code for "second_stage" that has to execute the rest.
if ( expressions . need_aggregate )
{
// We have aggregation, so we can't execute any later-stage
// expressions on shards, neither "before window functions" nor
// "before ORDER BY".
}
2017-04-01 07:20:54 +00:00
else
{
2021-04-23 10:58:54 +00:00
// We don't have aggregation.
// Window functions must be executed on initiator (second_stage).
// ORDER BY and DISTINCT might depend on them, so if we have
// window functions, we can't execute ORDER BY and DISTINCT
// now, on shard (first_stage).
if ( query_analyzer - > hasWindow ( ) )
{
executeExpression ( query_plan , expressions . before_window , " Before window functions " ) ;
}
else
{
// We don't have window functions, so we can execute the
// expressions before ORDER BY and the preliminary DISTINCT
// now, on shards (first_stage).
assert ( ! expressions . before_window ) ;
executeExpression ( query_plan , expressions . before_order_by , " Before ORDER BY " ) ;
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 ;
2021-06-03 14:05:37 +00:00
executeTotalsAndHaving (
2021-09-28 10:01:47 +00:00
query_plan , expressions . hasHaving ( ) , expressions . before_having , expressions . remove_having_filter , 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 )
2021-06-03 14:05:37 +00:00
throw Exception (
" WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING " ,
ErrorCodes : : NOT_IMPLEMENTED ) ;
2021-09-28 10:01:47 +00:00
executeHaving ( query_plan , expressions . before_having , expressions . remove_having_filter ) ;
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 ( ) )
2021-09-28 10:01:47 +00:00
executeHaving ( query_plan , expressions . before_having , expressions . remove_having_filter ) ;
2021-04-23 10:58:54 +00:00
}
else if ( query . group_by_with_totals | | query . group_by_with_rollup | | query . group_by_with_cube )
throw Exception ( " WITH TOTALS, ROLLUP or CUBE are not supported without aggregation " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2017-04-01 07:20:54 +00:00
2021-04-23 10:58:54 +00:00
// Now we must execute:
2021-04-23 12:44:46 +00:00
// 1) expressions before window functions,
// 2) window functions,
// 3) expressions after window functions,
// 4) preliminary distinct.
2021-04-23 10:58:54 +00:00
// Some of these were already executed at the shards (first_stage),
// see the counterpart code and comments there.
2021-05-05 20:56:14 +00:00
if ( from_aggregation_stage )
{
if ( query_analyzer - > hasWindow ( ) )
2021-06-03 14:05:37 +00:00
throw Exception (
" Window functions does not support processing from WithMergeableStateAfterAggregation " ,
ErrorCodes : : NOT_IMPLEMENTED ) ;
2021-05-05 20:56:14 +00:00
}
else if ( expressions . need_aggregate )
2021-04-23 10:58:54 +00:00
{
2020-12-25 03:13:30 +00:00
executeExpression ( query_plan , expressions . before_window ,
" Before window functions " ) ;
2020-12-10 19:06:52 +00:00
executeWindow ( query_plan ) ;
2020-12-25 03:13:30 +00:00
executeExpression ( query_plan , expressions . before_order_by , " Before ORDER BY " ) ;
2021-04-23 12:44:46 +00:00
executeDistinct ( query_plan , true , expressions . selected_columns , true ) ;
2017-04-01 07:20:54 +00:00
}
2021-04-23 10:58:54 +00:00
else
{
if ( query_analyzer - > hasWindow ( ) )
{
executeWindow ( query_plan ) ;
executeExpression ( query_plan , expressions . before_order_by , " Before ORDER BY " ) ;
2021-04-23 12:44:46 +00:00
executeDistinct ( query_plan , true , expressions . selected_columns , true ) ;
2021-04-23 10:58:54 +00:00
}
else
{
// Neither aggregation nor windows, all expressions before
// ORDER BY executed on shards.
}
}
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 .
2021-02-18 18:41:50 +00:00
*
2021-06-04 06:43:56 +00:00
* Also in case of remote servers was process the query up to WithMergeableStateAfterAggregationAndLimit
2021-02-18 18:41:50 +00:00
* ( distributed_group_by_no_merge = 2 or optimize_distributed_group_by_sharding_key = 1 takes place ) ,
* then merge the sorted streams is enough , since remote servers already did full ORDER BY .
2017-04-01 07:20:54 +00:00
*/
2019-07-24 14:23:57 +00:00
2021-02-18 18:41:50 +00:00
if ( from_aggregation_stage )
2021-05-27 16:05:49 +00:00
executeMergeSorted ( query_plan , " after aggregation stage for ORDER BY " ) ;
else if ( ! expressions . first_stage
& & ! expressions . need_aggregate
& & ! expressions . has_window
& & ! ( query . group_by_with_totals & & ! aggregate_final ) )
executeMergeSorted ( query_plan , " for ORDER BY, without aggregation " ) ;
2017-04-02 17:37:49 +00:00
else /// Otherwise, just sort.
2021-05-02 13:38:19 +00:00
executeOrder (
query_plan ,
query_info . input_order_info ? query_info . input_order_info
: ( query_info . projection ? query_info . projection - > input_order_info : nullptr ) ) ;
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-12-17 09:33:47 +00:00
bool has_withfill = false ;
if ( query . orderBy ( ) )
{
2021-04-10 23:33:54 +00:00
SortDescription order_descr = getSortDescription ( query , context ) ;
2020-12-17 09:33:47 +00:00
for ( auto & desc : order_descr )
if ( desc . with_fill )
{
has_withfill = true ;
break ;
}
}
2021-06-04 06:43:56 +00:00
bool apply_limit = options . to_stage ! = QueryProcessingStage : : WithMergeableStateAfterAggregation ;
2021-08-05 18:05:46 +00:00
bool apply_prelimit = apply_limit & &
query . limitLength ( ) & & ! query . limit_with_ties & &
! hasWithTotalsInAnySubqueryInFromClause ( query ) & &
2021-06-28 09:53:54 +00:00
! query . arrayJoinExpressionList ( ) . first & &
2021-08-05 18:05:46 +00:00
! query . distinct & &
! expressions . hasLimitBy ( ) & &
! settings . extremes & &
! has_withfill ;
2021-06-04 06:43:56 +00:00
bool apply_offset = options . to_stage ! = QueryProcessingStage : : WithMergeableStateAfterAggregationAndLimit ;
2021-08-05 18:05:46 +00:00
if ( apply_prelimit )
2018-02-25 06:34:20 +00:00
{
2021-06-04 06:43:56 +00:00
executePreLimit ( query_plan , /* do_not_skip_offset= */ ! apply_offset ) ;
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 .
*/
2021-07-17 17:21:13 +00:00
if ( ! from_aggregation_stage & & query . distinct )
2020-06-18 13:38:45 +00:00
executeDistinct ( query_plan , false , expressions . selected_columns , false ) ;
2017-04-01 07:20:54 +00:00
2021-07-17 17:21:13 +00:00
if ( ! from_aggregation_stage & & 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.
2021-08-05 18:05:46 +00:00
if ( query . limit_with_ties & & apply_offset )
2020-06-15 22:24:00 +00:00
{
2020-06-19 08:30:21 +00:00
executeLimit ( query_plan ) ;
2020-06-15 22:24:00 +00:00
}
2020-08-15 13:57:17 +00:00
/// Projection not be done on the shards, since then initiator will not find column in blocks.
2021-06-04 06:43:56 +00:00
/// (significant only for WithMergeableStateAfterAggregation/WithMergeableStateAfterAggregationAndLimit).
2020-08-15 13:57:17 +00:00
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
2021-06-28 09:53:54 +00:00
bool limit_applied = apply_prelimit | | ( query . limit_with_ties & & apply_offset ) ;
2020-08-15 13:57:17 +00:00
/// Limit is no longer needed if there is prelimit.
2021-08-05 18:05:46 +00:00
///
2021-08-08 06:58:07 +00:00
/// NOTE: that LIMIT cannot be applied if OFFSET should not be applied,
2021-08-05 18:05:46 +00:00
/// since LIMIT will apply OFFSET too.
/// This is the case for various optimizations for distributed queries,
/// and when LIMIT cannot be applied it will be applied on the initiator anyway.
if ( apply_limit & & ! limit_applied & & apply_offset )
2020-06-18 13:38:45 +00:00
executeLimit ( query_plan ) ;
2020-05-06 06:50:55 +00:00
2021-06-04 06:43:56 +00:00
if ( apply_offset )
2020-08-15 13:57:17 +00:00
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 ;
2021-06-03 14:05:37 +00:00
limits . size_limits = SizeLimits ( settings . max_rows_to_read , settings . max_bytes_to_read , settings . read_overflow_mode ) ;
2020-09-15 10:40:39 +00:00
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 ;
}
2021-05-04 10:52:37 +00:00
static void executeMergeAggregatedImpl (
QueryPlan & query_plan ,
bool overflow_row ,
bool final ,
bool is_remote_storage ,
const Settings & settings ,
const NamesAndTypesList & aggregation_keys ,
const AggregateDescriptions & aggregates )
{
const auto & header_before_merge = query_plan . getCurrentDataStream ( ) . header ;
ColumnNumbers keys ;
for ( const auto & key : aggregation_keys )
keys . push_back ( header_before_merge . getPositionByName ( key . name ) ) ;
/** 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 .
*/
Aggregator : : Params params ( header_before_merge , keys , aggregates , overflow_row , settings . max_threads ) ;
auto transform_params = std : : make_shared < AggregatingTransformParams > ( params , final ) ;
auto merging_aggregated = std : : make_unique < MergingAggregatedStep > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) ,
std : : move ( transform_params ) ,
settings . distributed_aggregation_memory_efficient & & is_remote_storage ,
settings . max_threads ,
settings . aggregation_memory_efficient_merge_threads ) ;
2021-05-04 10:52:37 +00:00
query_plan . addStep ( std : : move ( merging_aggregated ) ) ;
}
2021-02-10 14:12:49 +00:00
void InterpreterSelectQuery : : addEmptySourceToQueryPlan (
QueryPlan & query_plan , const Block & source_header , const SelectQueryInfo & query_info , ContextPtr context_ )
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
2021-05-03 13:00:59 +00:00
PrewhereInfoPtr prewhere_info_ptr = query_info . projection ? query_info . projection - > prewhere_info : query_info . prewhere_info ;
if ( prewhere_info_ptr )
2021-02-10 14:12:49 +00:00
{
2021-05-03 13:00:59 +00:00
auto & prewhere_info = * prewhere_info_ptr ;
2021-02-13 22:07:13 +00:00
2021-02-15 19:48:06 +00:00
if ( prewhere_info . alias_actions )
2020-09-18 14:16:53 +00:00
{
2021-06-25 14:49:28 +00:00
pipe . addSimpleTransform ( [ & ] ( const Block & header )
{
return std : : make_shared < ExpressionTransform > ( header ,
std : : make_shared < ExpressionActions > ( prewhere_info . alias_actions ) ) ;
} ) ;
2021-02-13 22:07:13 +00:00
}
2021-01-25 14:31:59 +00:00
2021-02-15 19:48:06 +00:00
if ( prewhere_info . row_level_filter )
2020-09-18 14:16:53 +00:00
{
pipe . addSimpleTransform ( [ & ] ( const Block & header )
{
2021-06-25 14:49:28 +00:00
return std : : make_shared < FilterTransform > ( header ,
std : : make_shared < ExpressionActions > ( prewhere_info . row_level_filter ) ,
prewhere_info . row_level_column_name , true ) ;
2020-09-18 14:16:53 +00:00
} ) ;
}
pipe . addSimpleTransform ( [ & ] ( const Block & header )
{
return std : : make_shared < FilterTransform > (
2021-06-25 14:49:28 +00:00
header , std : : make_shared < ExpressionActions > ( prewhere_info . prewhere_actions ) ,
prewhere_info . prewhere_column_name , prewhere_info . remove_prewhere_column ) ;
2020-09-18 14:16:53 +00:00
} ) ;
}
2021-05-04 12:40:34 +00:00
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 ) ) ;
2021-05-03 13:00:59 +00:00
if ( query_info . projection )
{
if ( query_info . projection - > before_where )
{
2021-05-04 12:40:34 +00:00
auto where_step = std : : make_unique < FilterStep > (
query_plan . getCurrentDataStream ( ) ,
query_info . projection - > before_where ,
query_info . projection - > where_column_name ,
query_info . projection - > remove_where_filter ) ;
where_step - > setStepDescription ( " WHERE " ) ;
query_plan . addStep ( std : : move ( where_step ) ) ;
2021-05-03 13:00:59 +00:00
}
2021-05-04 10:52:37 +00:00
if ( query_info . projection - > desc - > type = = ProjectionDescription : : Type : : Aggregate )
2021-05-03 13:00:59 +00:00
{
2021-05-04 10:52:37 +00:00
if ( query_info . projection - > before_aggregation )
{
2021-05-04 12:40:34 +00:00
auto expression_before_aggregation
= std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , query_info . projection - > before_aggregation ) ;
expression_before_aggregation - > setStepDescription ( " Before GROUP BY " ) ;
query_plan . addStep ( std : : move ( expression_before_aggregation ) ) ;
2021-05-04 10:52:37 +00:00
}
2021-05-04 12:40:34 +00:00
executeMergeAggregatedImpl (
query_plan ,
query_info . projection - > aggregate_overflow_row ,
query_info . projection - > aggregate_final ,
false ,
context_ - > getSettingsRef ( ) ,
query_info . projection - > aggregation_keys ,
query_info . projection - > aggregate_descriptions ) ;
}
2021-05-04 10:52:37 +00:00
}
2020-09-18 14:16:53 +00:00
}
2021-07-03 22:53:51 +00:00
bool InterpreterSelectQuery : : shouldMoveToPrewhere ( )
2021-05-02 13:38:19 +00:00
{
2021-05-04 12:40:34 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2021-07-03 11:28:24 +00:00
const ASTSelectQuery & query = getSelectQuery ( ) ;
2021-07-03 22:53:51 +00:00
return settings . optimize_move_to_prewhere & & ( ! query . final ( ) | | settings . optimize_move_to_prewhere_if_final ) ;
2021-07-03 11:28:24 +00:00
}
2021-05-02 13:38:19 +00:00
void InterpreterSelectQuery : : addPrewhereAliasActions ( )
{
2021-05-04 12:40:34 +00:00
auto & expressions = analysis_result ;
if ( expressions . filter_info )
{
if ( ! expressions . prewhere_info )
{
2021-10-13 18:22:02 +00:00
const bool does_storage_support_prewhere = ! input_pipe & & storage & & storage - > supportsPrewhere ( ) ;
2021-07-03 22:53:51 +00:00
if ( does_storage_support_prewhere & & shouldMoveToPrewhere ( ) )
2021-05-04 12:40:34 +00:00
{
/// Execute row level filter in prewhere as a part of "move to prewhere" optimization.
2021-06-25 14:49:28 +00:00
expressions . prewhere_info = std : : make_shared < PrewhereInfo > (
2021-05-04 12:40:34 +00:00
std : : move ( expressions . filter_info - > actions ) ,
std : : move ( expressions . filter_info - > column_name ) ) ;
expressions . prewhere_info - > prewhere_actions - > projectInput ( false ) ;
expressions . prewhere_info - > remove_prewhere_column = expressions . filter_info - > do_remove_column ;
expressions . prewhere_info - > need_filter = true ;
expressions . filter_info = nullptr ;
}
}
else
{
/// Add row level security actions to prewhere.
2021-06-25 14:49:28 +00:00
expressions . prewhere_info - > row_level_filter = std : : move ( expressions . filter_info - > actions ) ;
2021-05-04 12:40:34 +00:00
expressions . prewhere_info - > row_level_column_name = std : : move ( expressions . filter_info - > column_name ) ;
2021-06-25 14:49:28 +00:00
expressions . prewhere_info - > row_level_filter - > projectInput ( false ) ;
2021-05-04 12:40:34 +00:00
expressions . filter_info = nullptr ;
}
}
2021-05-02 13:38:19 +00:00
auto & prewhere_info = analysis_result . prewhere_info ;
auto & columns_to_remove_after_prewhere = analysis_result . columns_to_remove_after_prewhere ;
/// Detect, if ALIAS columns are required for query execution
auto alias_columns_required = false ;
const ColumnsDescription & storage_columns = metadata_snapshot - > getColumns ( ) ;
for ( const auto & column_name : required_columns )
{
auto column_default = storage_columns . getDefault ( column_name ) ;
if ( column_default & & column_default - > kind = = ColumnDefaultKind : : Alias )
{
alias_columns_required = true ;
break ;
}
}
/// 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.
if ( alias_columns_required )
{
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
if ( prewhere_info )
{
/// Get some columns directly from PREWHERE expression actions
auto prewhere_required_columns = prewhere_info - > prewhere_actions - > getRequiredColumns ( ) . getNames ( ) ;
required_columns_from_prewhere . insert ( prewhere_required_columns . begin ( ) , prewhere_required_columns . end ( ) ) ;
2021-06-25 14:49:28 +00:00
if ( prewhere_info - > row_level_filter )
2021-05-02 13:38:19 +00:00
{
2021-06-25 14:49:28 +00:00
auto row_level_required_columns = prewhere_info - > row_level_filter - > getRequiredColumns ( ) . getNames ( ) ;
2021-05-02 13:38:19 +00:00
required_columns_from_prewhere . insert ( row_level_required_columns . begin ( ) , row_level_required_columns . end ( ) ) ;
}
}
/// Expression, that contains all raw required columns
ASTPtr required_columns_all_expr = std : : make_shared < ASTExpressionList > ( ) ;
/// 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`.
for ( const auto & column : required_columns )
{
ASTPtr column_expr ;
const auto column_default = storage_columns . getDefault ( column ) ;
bool is_alias = column_default & & column_default - > kind = = ColumnDefaultKind : : Alias ;
if ( is_alias )
{
auto column_decl = storage_columns . get ( column ) ;
column_expr = column_default - > expression - > clone ( ) ;
// recursive visit for alias to alias
replaceAliasColumnsInQuery (
2021-05-21 17:01:21 +00:00
column_expr , metadata_snapshot - > getColumns ( ) , syntax_analyzer_result - > array_join_result_to_source , context ) ;
2021-05-02 13:38:19 +00:00
column_expr = addTypeConversionToAST (
std : : move ( column_expr ) , column_decl . type - > getName ( ) , metadata_snapshot - > getColumns ( ) . getAll ( ) , context ) ;
column_expr = setAlias ( column_expr , column ) ;
}
else
column_expr = std : : make_shared < ASTIdentifier > ( column ) ;
if ( required_columns_from_prewhere . count ( column ) )
{
required_columns_from_prewhere_expr - > children . emplace_back ( std : : move ( column_expr ) ) ;
if ( is_alias )
required_aliases_from_prewhere . insert ( column ) ;
}
else
required_columns_all_expr - > children . emplace_back ( std : : move ( column_expr ) ) ;
}
/// 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 )
{
NameSet columns_to_remove ( columns_to_remove_after_prewhere . begin ( ) , columns_to_remove_after_prewhere . end ( ) ) ;
Block prewhere_actions_result = prewhere_info - > prewhere_actions - > getResultColumns ( ) ;
/// 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 ;
if ( columns_to_remove . count ( column . name ) )
continue ;
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
2021-06-15 19:55:21 +00:00
= collections : : map < NameSet > ( required_columns_after_prewhere , [ ] ( const auto & it ) { return it . name ; } ) ;
2021-05-02 13:38:19 +00:00
}
auto syntax_result
= TreeRewriter ( context ) . analyze ( required_columns_all_expr , required_columns_after_prewhere , storage , metadata_snapshot ) ;
alias_actions = ExpressionAnalyzer ( required_columns_all_expr , syntax_result , context ) . getActionsDAG ( true ) ;
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
required_columns = alias_actions - > getRequiredColumns ( ) . getNames ( ) ;
/// Do not remove prewhere filter if it is a column which is used as alias.
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 ;
/// Remove columns which will be added by prewhere.
2021-06-03 14:05:37 +00:00
std : : erase_if ( required_columns , [ & ] ( const String & name ) { return required_columns_after_prewhere_set . count ( name ) ! = 0 ; } ) ;
2021-05-02 13:38:19 +00:00
if ( prewhere_info )
{
/// Don't remove columns which are needed to be aliased.
for ( const auto & name : required_columns )
prewhere_info - > prewhere_actions - > tryRestoreColumn ( name ) ;
auto analyzed_result
= TreeRewriter ( context ) . analyze ( required_columns_from_prewhere_expr , metadata_snapshot - > getColumns ( ) . getAllPhysical ( ) ) ;
prewhere_info - > alias_actions
= ExpressionAnalyzer ( required_columns_from_prewhere_expr , analyzed_result , context ) . getActionsDAG ( true , false ) ;
/// Add (physical?) columns required by alias actions.
auto required_columns_from_alias = prewhere_info - > alias_actions - > getRequiredColumns ( ) ;
Block prewhere_actions_result = prewhere_info - > prewhere_actions - > getResultColumns ( ) ;
for ( auto & column : required_columns_from_alias )
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 ) ;
/// Add physical columns required by prewhere actions.
for ( const auto & column : required_columns_from_prewhere )
if ( required_aliases_from_prewhere . count ( column ) = = 0 )
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column ) )
required_columns . push_back ( column ) ;
}
}
}
2021-02-15 19:48:06 +00:00
void InterpreterSelectQuery : : executeFetchColumns ( QueryProcessingStage : : Enum processing_stage , QueryPlan & query_plan )
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
2021-07-26 18:17:28 +00:00
& & storage - > getName ( ) ! = " MaterializedMySQL "
2021-02-16 09:37:19 +00:00
& & ! row_policy_filter
2020-06-13 20:59:20 +00:00
& & processing_stage = = QueryProcessingStage : : FetchColumns
& & query_analyzer - > hasAggregation ( )
& & ( query_analyzer - > aggregates ( ) . size ( ) = = 1 )
2021-06-06 21:49:55 +00:00
& & typeid_cast < const 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 { } ;
2021-02-21 20:06:31 +00:00
2020-09-21 10:13:01 +00:00
if ( ! query . prewhere ( ) & & ! query . where ( ) )
2021-02-21 20:06:31 +00:00
{
2020-11-25 13:47:32 +00:00
num_rows = storage - > totalRows ( settings ) ;
2021-02-21 20:06:31 +00:00
}
2020-09-21 10:13:01 +00:00
else // It's possible to optimize count() given only partition predicates
{
SelectQueryInfo temp_query_info ;
2020-12-12 16:42:15 +00:00
temp_query_info . query = query_ptr ;
2020-09-21 10:13:01 +00:00
temp_query_info . syntax_analyzer_result = syntax_analyzer_result ;
temp_query_info . sets = query_analyzer - > getPreparedSets ( ) ;
2020-12-12 16:42:15 +00:00
2021-04-10 23:33:54 +00:00
num_rows = storage - > totalRowsByPartitionPredicate ( temp_query_info , context ) ;
2020-09-21 10:13:01 +00:00
}
2021-02-21 20:06:31 +00:00
2019-10-28 17:27:43 +00:00
if ( num_rows )
{
2021-06-06 21:49:55 +00:00
const AggregateFunctionCount & agg_count = static_cast < const AggregateFunctionCount & > ( * func ) ;
2019-10-28 17:27:43 +00:00
/// We will process it up to "WithMergeableState".
std : : vector < char > state ( agg_count . sizeOfData ( ) ) ;
AggregateDataPtr place = state . data ( ) ;
agg_count . create ( place ) ;
2021-04-04 09:23:40 +00:00
SCOPE_EXIT_MEMORY_SAFE ( agg_count . destroy ( place ) ) ;
2019-10-28 17:27:43 +00:00
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
2021-10-08 14:03:54 +00:00
auto source = std : : make_shared < SourceFromSingleChunk > ( block_with_count ) ;
auto prepared_count = std : : make_unique < ReadFromPreparedSource > ( Pipe ( std : : move ( source ) ) , 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-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 )
2021-06-03 14:05:37 +00:00
throw Exception (
ErrorCodes : : TOO_MANY_COLUMNS ,
" Limit for number of columns to read exceeded. Requested: {}, maximum: {} " ,
required_columns . size ( ) ,
settings . max_columns_to_read ) ;
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
2021-04-10 23:33:54 +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 ( )
2021-04-06 11:43:27 +00:00
& & ! query_analyzer - > hasWindow ( )
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 > (
2021-04-10 23:33:54 +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 )
2021-04-20 06:03:32 +00:00
max_streams = 1 ;
2017-04-01 07:20:54 +00:00
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 ;
2021-05-02 13:38:19 +00:00
auto & prewhere_info = analysis_result . prewhere_info ;
2020-11-03 19:05:47 +00:00
if ( prewhere_info )
2021-06-25 14:49:28 +00:00
query_info . prewhere_info = prewhere_info ;
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.
2021-05-02 13:38:19 +00:00
if ( ( analysis_result . optimize_read_in_order | | analysis_result . optimize_aggregation_in_order )
& & ( ! query_info . projection | | query_info . projection - > complete ) )
2019-12-10 23:18:24 +00:00
{
2020-05-13 13:49:10 +00:00
if ( analysis_result . optimize_read_in_order )
2021-05-02 13:38:19 +00:00
{
if ( query_info . projection )
{
query_info . projection - > order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
// TODO Do we need a projection variant for this field?
analysis_result . order_by_elements_actions ,
getSortDescription ( query , context ) ,
query_info . syntax_analyzer_result ) ;
}
else
{
query_info . order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
analysis_result . order_by_elements_actions , getSortDescription ( query , context ) , query_info . syntax_analyzer_result ) ;
}
}
2020-05-13 13:49:10 +00:00
else
2021-05-02 13:38:19 +00:00
{
if ( query_info . projection )
{
query_info . projection - > order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
query_info . projection - > group_by_elements_actions ,
2021-08-03 18:03:24 +00:00
getSortDescriptionFromGroupBy ( query ) ,
2021-05-02 13:38:19 +00:00
query_info . syntax_analyzer_result ) ;
}
else
{
query_info . order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
2021-08-03 18:03:24 +00:00
analysis_result . group_by_elements_actions , getSortDescriptionFromGroupBy ( query ) , query_info . syntax_analyzer_result ) ;
2021-05-02 13:38:19 +00:00
}
}
2019-12-10 23:18:24 +00:00
2021-07-13 14:24:45 +00:00
/// If we don't have filtration, we can pushdown limit to reading stage for optimizations.
2021-08-04 15:18:03 +00:00
UInt64 limit = ( query . hasFiltration ( ) | | query . groupBy ( ) ) ? 0 : getLimitForSorting ( query , context ) ;
2021-05-02 13:38:19 +00:00
if ( query_info . projection )
query_info . projection - > input_order_info
2021-07-13 14:24:45 +00:00
= query_info . projection - > order_optimizer - > getInputOrder ( query_info . projection - > desc - > metadata , context , limit ) ;
2021-05-02 13:38:19 +00:00
else
2021-07-13 14:24:45 +00:00
query_info . input_order_info = query_info . order_optimizer - > getInputOrder ( metadata_snapshot , context , limit ) ;
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-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 ) ;
2021-06-03 14:05:37 +00:00
leaf_limits = SizeLimits ( settings . max_rows_to_read_leaf , settings . max_bytes_to_read_leaf , settings . read_overflow_mode_leaf ) ;
2020-08-28 14:50:25 +00:00
}
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
2021-06-03 14:05:37 +00:00
storage - > read ( query_plan , required_columns , metadata_snapshot , query_info , context , processing_stage , max_block_size , max_streams ) ;
2020-09-18 11:39:07 +00:00
2020-12-18 06:54:38 +00:00
if ( context - > hasQueryContext ( ) & & ! options . is_internal )
2020-12-02 12:08:03 +00:00
{
2021-06-24 16:54:04 +00:00
const String view_name { } ;
2020-12-02 12:08:03 +00:00
auto local_storage_id = storage - > getStorageID ( ) ;
2021-04-10 23:33:54 +00:00
context - > getQueryContext ( ) - > addQueryAccessInfo (
2021-02-10 14:12:49 +00:00
backQuoteIfNeed ( local_storage_id . getDatabaseName ( ) ) ,
local_storage_id . getFullTableName ( ) ,
required_columns ,
2021-06-24 16:54:04 +00:00
query_info . projection ? query_info . projection - > desc - > name : " " ,
view_name ) ;
2020-12-02 12:08:03 +00:00
}
2020-09-18 14:16:53 +00:00
/// Create step which reads from empty source if storage has no data.
if ( ! query_plan . isInitialized ( ) )
{
2021-05-03 13:00:59 +00:00
auto header = query_info . projection
? query_info . projection - > desc - > metadata - > getSampleBlockForColumns (
query_info . projection - > required_columns , storage - > getVirtuals ( ) , storage - > getStorageID ( ) )
: metadata_snapshot - > getSampleBlockForColumns ( required_columns , storage - > getVirtuals ( ) , storage - > getStorageID ( ) ) ;
2021-02-10 14:12:49 +00:00
addEmptySourceToQueryPlan ( query_plan , header , query_info , context ) ;
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 > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) , storage , std : : move ( table_lock ) , limits , leaf_limits , std : : move ( quota ) , 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.
2020-12-06 22:12:54 +00:00
///
/// But in case of remote query and prefer_localhost_replica=1 (default)
/// The inner local query (that is done in the same process, without
/// network interaction), it will setMaxThreads earlier and distributed
/// query will not update it.
if ( ! query_plan . getMaxThreads ( ) | | is_remote )
2020-10-22 11:08:12 +00:00
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 > (
2021-08-03 18:03:24 +00:00
query_plan . getCurrentDataStream ( ) , expression , getSelectQuery ( ) . where ( ) - > getColumnName ( ) , remove_filter ) ;
2020-06-16 12:02:10 +00:00
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
2021-07-07 05:01:30 +00:00
if ( options . is_projection_query )
2021-04-21 16:00:27 +00:00
return ;
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
2021-06-03 14:05:37 +00:00
Aggregator : : Params params (
header_before_aggregation ,
keys ,
aggregates ,
overflow_row ,
settings . max_rows_to_group_by ,
settings . group_by_overflow_mode ,
settings . group_by_two_level_threshold ,
settings . group_by_two_level_threshold_bytes ,
settings . max_bytes_before_external_group_by ,
2021-08-20 14:27:47 +00:00
settings . empty_result_for_aggregation_by_empty_set
| | ( settings . empty_result_for_aggregation_by_constant_keys_on_empty_set & & keys . empty ( )
& & query_analyzer - > hasConstAggregationKeys ( ) ) ,
2021-06-03 14:05:37 +00:00
context - > getTemporaryVolume ( ) ,
settings . max_threads ,
2021-05-31 08:05:40 +00:00
settings . min_free_disk_space_for_temporary_data ,
settings . compile_aggregate_expressions ,
2021-06-13 12:38:57 +00:00
settings . min_count_to_compile_aggregate_expression ) ;
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 )
2021-08-03 18:03:24 +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
2021-06-03 14:05:37 +00:00
? 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 > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) ,
params ,
final ,
settings . max_block_size ,
2021-10-19 13:36:31 +00:00
settings . aggregation_in_order_max_block_bytes ,
2021-06-03 14:05:37 +00:00
merge_threads ,
temporary_data_merge_threads ,
storage_has_evenly_distributed_read ,
std : : move ( group_by_info ) ,
std : : move ( group_by_sort_description ) ) ;
2020-06-17 14:21:48 +00:00
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
{
2021-04-22 08:43:35 +00:00
/// If aggregate projection was chosen for table, avoid adding MergeAggregated.
/// It is already added by storage (because of performance issues).
/// TODO: We should probably add another one processing stage for storage?
/// WithMergeableStateAfterAggregation is not ok because, e.g., it skips sorting after aggregation.
2021-04-29 07:38:47 +00:00
if ( query_info . projection & & query_info . projection - > desc - > type = = ProjectionDescription : : Type : : Aggregate )
2021-04-21 16:00:27 +00:00
return ;
2021-05-04 10:52:37 +00:00
executeMergeAggregatedImpl (
query_plan ,
overflow_row ,
final ,
storage & & storage - > isRemote ( ) ,
context - > getSettingsRef ( ) ,
query_analyzer - > aggregationKeys ( ) ,
query_analyzer - > aggregates ( ) ) ;
2019-03-26 18:28:37 +00:00
}
2012-05-09 13:12:38 +00:00
2021-09-28 10:01:47 +00:00
void InterpreterSelectQuery : : executeHaving ( QueryPlan & query_plan , const ActionsDAGPtr & expression , bool remove_filter )
2019-03-26 18:28:37 +00:00
{
2021-06-03 14:05:37 +00:00
auto having_step
2021-09-28 10:01:47 +00:00
= std : : make_unique < FilterStep > ( query_plan . getCurrentDataStream ( ) , expression , getSelectQuery ( ) . having ( ) - > getColumnName ( ) , remove_filter ) ;
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
2021-06-03 14:05:37 +00:00
void InterpreterSelectQuery : : executeTotalsAndHaving (
2021-09-28 10:01:47 +00:00
QueryPlan & query_plan , bool has_having , const ActionsDAGPtr & expression , bool remove_filter , 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 > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) ,
overflow_row ,
expression ,
2021-08-03 18:03:24 +00:00
has_having ? getSelectQuery ( ) . having ( ) - > getColumnName ( ) : " " ,
2021-09-28 10:01:47 +00:00
remove_filter ,
2021-06-03 14:05:37 +00:00
settings . totals_mode ,
settings . totals_auto_threshold ,
final ) ;
2019-03-26 18:28:37 +00:00
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
2021-06-03 14:05:37 +00:00
Aggregator : : Params params (
header_before_transform ,
keys ,
query_analyzer - > aggregates ( ) ,
false ,
settings . max_rows_to_group_by ,
settings . group_by_overflow_mode ,
0 ,
0 ,
settings . max_bytes_before_external_group_by ,
settings . empty_result_for_aggregation_by_empty_set ,
context - > getTemporaryVolume ( ) ,
settings . max_threads ,
2021-05-31 08:05:40 +00:00
settings . min_free_disk_space_for_temporary_data ,
settings . compile_aggregate_expressions ,
2021-06-13 12:38:57 +00:00
settings . min_count_to_compile_aggregate_expression ) ;
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-12-25 03:13:30 +00:00
if ( ! expression )
return ;
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
2021-06-03 14:05:37 +00:00
static bool windowDescriptionComparator ( const WindowDescription * _left , const WindowDescription * _right )
2021-02-10 12:37:05 +00:00
{
const auto & left = _left - > full_sort_description ;
const auto & right = _right - > full_sort_description ;
2021-02-10 13:27:22 +00:00
for ( size_t i = 0 ; i < std : : min ( left . size ( ) , right . size ( ) ) ; + + i )
2021-02-10 12:37:05 +00:00
{
if ( left [ i ] . column_name < right [ i ] . column_name )
return true ;
2021-02-11 08:39:39 +00:00
else if ( left [ i ] . column_name > right [ i ] . column_name )
return false ;
else if ( left [ i ] . column_number < right [ i ] . column_number )
2021-02-10 12:37:05 +00:00
return true ;
2021-02-11 08:39:39 +00:00
else if ( left [ i ] . column_number > right [ i ] . column_number )
return false ;
else if ( left [ i ] . direction < right [ i ] . direction )
2021-02-10 12:37:05 +00:00
return true ;
2021-02-11 08:39:39 +00:00
else if ( left [ i ] . direction > right [ i ] . direction )
return false ;
else if ( left [ i ] . nulls_direction < right [ i ] . nulls_direction )
2021-02-10 12:37:05 +00:00
return true ;
2021-02-11 08:39:39 +00:00
else if ( left [ i ] . nulls_direction > right [ i ] . nulls_direction )
2021-02-10 13:27:22 +00:00
return false ;
2021-02-11 08:39:39 +00:00
assert ( left [ i ] = = right [ i ] ) ;
2021-02-10 12:37:05 +00:00
}
2021-02-10 13:27:22 +00:00
// Note that we check the length last, because we want to put together the
// sort orders that have common prefix but different length.
return left . size ( ) > right . size ( ) ;
2021-02-10 12:37:05 +00:00
}
static bool sortIsPrefix ( const WindowDescription & _prefix ,
const WindowDescription & _full )
{
const auto & prefix = _prefix . full_sort_description ;
const auto & full = _full . full_sort_description ;
if ( prefix . size ( ) > full . size ( ) )
return false ;
for ( size_t i = 0 ; i < prefix . size ( ) ; + + i )
{
if ( full [ i ] ! = prefix [ i ] )
return false ;
}
return true ;
}
2012-02-27 06:28:20 +00:00
2020-12-10 19:06:52 +00:00
void InterpreterSelectQuery : : executeWindow ( QueryPlan & query_plan )
{
2021-02-10 13:27:22 +00:00
// Try to sort windows in such an order that the window with the longest
// sort description goes first, and all window that use its prefixes follow.
2021-02-10 12:37:05 +00:00
std : : vector < const WindowDescription * > windows_sorted ;
2020-12-22 19:01:15 +00:00
for ( const auto & [ _ , w ] : query_analyzer - > windowDescriptions ( ) )
2021-02-10 12:37:05 +00:00
windows_sorted . push_back ( & w ) ;
2020-12-10 19:06:52 +00:00
2021-06-03 14:05:37 +00:00
std : : sort ( windows_sorted . begin ( ) , windows_sorted . end ( ) , windowDescriptionComparator ) ;
2020-12-10 19:06:52 +00:00
2021-02-10 12:37:05 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
for ( size_t i = 0 ; i < windows_sorted . size ( ) ; + + i )
{
const auto & w = * windows_sorted [ i ] ;
2021-03-17 15:31:30 +00:00
// We don't need to sort again if the input from previous window already
// has suitable sorting. Also don't create sort steps when there are no
// columns to sort by, because the sort nodes are confused by this. It
// happens in case of `over ()`.
2021-06-03 14:05:37 +00:00
if ( ! w . full_sort_description . empty ( ) & & ( i = = 0 | | ! sortIsPrefix ( w , * windows_sorted [ i - 1 ] ) ) )
2021-02-10 12:37:05 +00:00
{
2021-10-18 14:37:46 +00:00
auto sorting_step = std : : make_unique < SortingStep > (
2021-02-10 12:37:05 +00:00
query_plan . getCurrentDataStream ( ) ,
w . full_sort_description ,
settings . max_block_size ,
0 /* LIMIT */ ,
2021-10-18 14:37:46 +00:00
SizeLimits ( settings . max_rows_to_sort , settings . max_bytes_to_sort , settings . sort_overflow_mode ) ,
2021-02-10 12:37:05 +00:00
settings . max_bytes_before_remerge_sort ,
settings . remerge_sort_lowered_memory_bytes_ratio ,
settings . max_bytes_before_external_sort ,
context - > getTemporaryVolume ( ) ,
settings . min_free_disk_space_for_temporary_data ) ;
2021-10-18 14:37:46 +00:00
sorting_step - > setStepDescription ( " Sorting for window ' " + w . window_name + " ' " ) ;
query_plan . addStep ( std : : move ( sorting_step ) ) ;
2021-02-10 12:37:05 +00:00
}
2020-12-10 19:06:52 +00:00
2021-06-03 14:05:37 +00:00
auto window_step = std : : make_unique < WindowStep > ( query_plan . getCurrentDataStream ( ) , w , w . window_functions ) ;
window_step - > setStepDescription ( " Window step for window ' " + w . window_name + " ' " ) ;
2020-12-10 19:06:52 +00:00
query_plan . addStep ( std : : move ( window_step ) ) ;
}
}
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
2021-10-18 14:37:46 +00:00
auto finish_sorting_step = std : : make_unique < SortingStep > (
2021-08-04 15:18:03 +00:00
query_plan . getCurrentDataStream ( ) ,
input_sorting_info - > order_key_prefix_descr ,
output_order_descr ,
settings . max_block_size ,
2021-10-18 09:54:12 +00:00
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 ( ) ;
2021-04-10 23:33:54 +00:00
SortDescription output_order_descr = getSortDescription ( query , context ) ;
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
/// Merge the sorted blocks.
2021-10-18 14:37:46 +00:00
auto sorting_step = std : : make_unique < SortingStep > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) ,
output_order_descr ,
settings . max_block_size ,
limit ,
2021-10-18 14:37:46 +00:00
SizeLimits ( settings . max_rows_to_sort , settings . max_bytes_to_sort , settings . sort_overflow_mode ) ,
2021-06-03 14:05:37 +00:00
settings . max_bytes_before_remerge_sort ,
settings . remerge_sort_lowered_memory_bytes_ratio ,
settings . max_bytes_before_external_sort ,
context - > getTemporaryVolume ( ) ,
settings . min_free_disk_space_for_temporary_data ) ;
2019-04-30 17:01:12 +00:00
2021-10-18 14:37:46 +00:00
sorting_step - > setStepDescription ( " Sorting for ORDER BY " ) ;
2021-10-18 17:53:30 +00:00
query_plan . addStep ( std : : move ( sorting_step ) ) ;
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 ( ) ;
2021-04-10 23:33:54 +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
2021-06-03 14:05:37 +00:00
auto merging_sorted
2021-10-18 14:37:46 +00:00
= std : : make_unique < SortingStep > ( query_plan . getCurrentDataStream ( ) , sort_description , settings . max_block_size , limit ) ;
2019-03-26 18:28:37 +00:00
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
2021-04-10 23:33:54 +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 ) ;
2021-06-03 14:05:37 +00:00
auto distinct_step
= std : : make_unique < DistinctStep > ( query_plan . getCurrentDataStream ( ) , limits , limit_for_distinct , columns , pre_distinct ) ;
2020-06-18 13:00:16 +00:00
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
{
2021-04-10 23:33:54 +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 ) ;
2021-08-05 18:05:46 +00:00
if ( do_not_skip_offset )
limit - > setStepDescription ( " preliminary LIMIT (with OFFSET) " ) ;
else
limit - > setStepDescription ( " preliminary LIMIT (without OFFSET) " ) ;
2020-06-18 17:45:00 +00:00
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 )
2021-08-03 18:03:24 +00:00
columns . emplace_back ( elem - > getColumnName ( ) ) ;
2019-03-26 18:28:37 +00:00
2021-04-10 23:33:54 +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 ( ) )
{
2021-04-10 23:33:54 +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 ;
2021-04-10 23:33:54 +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 ) ;
2021-04-10 23:33:54 +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 ;
2021-04-10 23:33:54 +00:00
std : : tie ( limit_length , limit_offset ) = getLimitLengthAndOffset ( query , context ) ;
2020-05-06 06:50:55 +00:00
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
{
2021-05-02 13:38:19 +00:00
const auto & input_order_info = query_info . input_order_info
? query_info . input_order_info
: ( query_info . projection ? query_info . projection - > input_order_info : nullptr ) ;
if ( input_order_info )
executeMergeSorted ( query_plan , 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 ) ;
2021-04-10 23:33:54 +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 ( ) )
2021-04-10 23:33:54 +00:00
InterpreterSetQuery ( query . settings ( ) , context ) . executeForCurrentContext ( ) ;
2021-10-04 14:27:44 +00:00
auto & client_info = context - > getClientInfo ( ) ;
auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD ;
auto min_minor = DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD ;
if ( client_info . query_kind = = ClientInfo : : QueryKind : : SECONDARY_QUERY & &
std : : forward_as_tuple ( client_info . connection_client_version_major , client_info . connection_client_version_minor ) < std : : forward_as_tuple ( min_major , min_minor ) )
{
/// Disable two-level aggregation due to version incompatibility.
context - > setSetting ( " group_by_two_level_threshold " , Field ( 0 ) ) ;
context - > setSetting ( " group_by_two_level_threshold_bytes " , Field ( 0 ) ) ;
}
2015-06-05 21:28:04 +00:00
}
2011-08-28 05:13:24 +00:00
}