2022-08-13 13:03:16 +00:00
# include <Access/AccessControl.h>
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>
2022-03-17 05:51:35 +00:00
# include <Parsers/ASTInterpolateElement.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>
2022-08-02 10:45:06 +00:00
# include <Interpreters/QueryLog.h>
2020-12-12 16:42:15 +00:00
# include <Interpreters/replaceAliasColumnsInQuery.h>
2022-04-28 12:55:37 +00:00
# include <Interpreters/RewriteCountDistinctVisitor.h>
2023-01-19 10:26:38 +00:00
# include <Interpreters/getCustomKeyFilterForParallelReplicas.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>
2022-08-03 11:28:28 +00:00
# include <Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.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/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>
2022-05-20 19:49:31 +00:00
# include <QueryPipeline/QueryPipelineBuilder.h>
2019-03-26 18:28:37 +00:00
2020-06-17 20:19:55 +00:00
# include <Storages/IStorage.h>
2022-01-19 17:26:37 +00:00
# include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
2022-11-27 23:41:31 +00:00
# include <Storages/StorageDistributed.h>
2022-01-19 17:26:37 +00:00
# include <Storages/StorageValues.h>
2020-06-17 20:19:55 +00:00
# include <Storages/StorageView.h>
# include <Columns/Collator.h>
2022-04-21 01:18:40 +00:00
# include <Core/ColumnNumbers.h>
2022-11-27 23:41:31 +00:00
# include <Core/Field.h>
# include <Core/ProtocolDefines.h>
# include <Functions/IFunction.h>
2022-04-21 01:18:40 +00:00
# include <Interpreters/Aggregator.h>
2022-11-27 23:41:31 +00:00
# include <Interpreters/Cluster.h>
2022-03-30 10:07:09 +00:00
# include <Interpreters/IJoin.h>
# include <QueryPipeline/SizeLimits.h>
2021-10-02 07:13:14 +00:00
# include <base/map.h>
2022-11-27 23:41:31 +00:00
# include <base/sort.h>
# include <base/types.h>
# include <Common/FieldVisitorToString.h>
# include <Common/FieldVisitorsAccurateComparison.h>
# include <Common/checkStackSize.h>
2022-04-27 15:05:45 +00:00
# include <Common/scope_guard_safe.h>
2022-10-19 16:30:03 +00:00
# include <Parsers/FunctionParameterValuesVisitor.h>
2022-11-27 23:41:31 +00:00
# include <Common/typeid_cast.h>
2020-06-17 20:19:55 +00:00
2023-01-19 20:50:27 +00:00
# include "config_version.h"
2023-01-19 20:54:04 +00:00
2011-08-28 05:13:24 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
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 ;
2022-03-19 18:12:29 +00:00
extern const int UNKNOWN_IDENTIFIER ;
2023-01-19 10:26:38 +00:00
extern const int BAD_ARGUMENTS ;
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.
2022-06-21 11:24:46 +00:00
FilterDAGInfoPtr generateFilterActions (
const StorageID & table_id ,
2022-08-02 10:45:06 +00:00
const ASTPtr & row_policy_filter_expression ,
2022-06-21 11:24:46 +00:00
const ContextPtr & context ,
const StoragePtr & storage ,
const StorageSnapshotPtr & storage_snapshot ,
const StorageMetadataPtr & metadata_snapshot ,
2022-10-21 12:53:23 +00:00
Names & prerequisite_columns ,
PreparedSetsPtr prepared_sets )
2019-03-29 20:31:06 +00:00
{
2022-06-21 11:24:46 +00:00
auto filter_info = std : : make_shared < FilterDAGInfo > ( ) ;
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
2021-12-06 15:23:17 +00:00
/// The first column is our filter expression.
2022-08-02 10:45:06 +00:00
/// the row_policy_filter_expression should be cloned, because it may be changed by TreeRewriter.
2021-12-06 15:23:17 +00:00
/// which make it possible an invalid expression, although it may be valid in whole select.
2022-08-02 10:45:06 +00:00
expr_list - > children . push_back ( row_policy_filter_expression - > clone ( ) ) ;
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 ;
2023-04-20 12:28:39 +00:00
/// We should add back quotes around column name as it can contain dots.
expr_list - > children . push_back ( parseQuery ( expr_parser , backQuoteIfNeed ( 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-07-09 03:15:41 +00:00
auto syntax_result = TreeRewriter ( context ) . analyzeSelect ( query_ast , TreeRewriterResult ( { } , storage , storage_snapshot ) ) ;
2022-10-21 12:53:23 +00:00
SelectQueryExpressionAnalyzer analyzer ( query_ast , syntax_result , context , metadata_snapshot , { } , false , { } , prepared_sets ) ;
2022-06-21 11:24:46 +00:00
filter_info - > actions = analyzer . simpleSelectActions ( ) ;
2019-03-29 20:31:06 +00:00
2022-06-21 11:24:46 +00:00
filter_info - > column_name = expr_list - > children . at ( 0 ) - > getColumnName ( ) ;
filter_info - > actions - > removeUnusedActions ( NameSet { filter_info - > column_name } ) ;
filter_info - > actions - > projectInput ( false ) ;
2021-02-20 11:00:16 +00:00
2022-06-21 11:24:46 +00:00
for ( const auto * node : filter_info - > actions - > getInputs ( ) )
2022-08-08 15:54:51 +00:00
filter_info - > actions - > getOutputs ( ) . push_back ( node ) ;
2021-02-20 11:00:16 +00:00
2022-06-21 11:24:46 +00:00
auto required_columns_from_filter = filter_info - > actions - > getRequiredColumns ( ) ;
2021-02-20 11:00:16 +00:00
2022-06-21 11:24:46 +00:00
for ( const auto & column : required_columns_from_filter )
{
if ( prerequisite_columns . end ( ) = = std : : find ( prerequisite_columns . begin ( ) , prerequisite_columns . end ( ) , column . name ) )
prerequisite_columns . push_back ( column . name ) ;
}
return filter_info ;
2019-03-29 20:31:06 +00:00
}
2018-02-26 21:00:42 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2022-07-05 22:47:00 +00:00
const 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_ )
2022-03-26 02:17:23 +00:00
{ }
2020-01-29 15:04:46 +00:00
2022-05-30 08:57:51 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2022-07-05 22:47:00 +00:00
const ContextMutablePtr & context_ ,
2022-05-30 08:57:51 +00:00
const SelectQueryOptions & options_ ,
const Names & required_result_column_names_ )
: InterpreterSelectQuery ( query_ptr_ , context_ , std : : nullopt , nullptr , options_ , required_result_column_names_ )
{ }
2020-01-29 15:04:46 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2022-07-05 22:47:00 +00:00
const 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_ ,
2022-07-05 22:47:00 +00:00
const 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
2022-03-26 02:17:23 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2022-07-05 22:47:00 +00:00
const ContextPtr & context_ ,
2022-03-26 02:17:23 +00:00
const SelectQueryOptions & options_ ,
2022-07-18 15:53:30 +00:00
PreparedSetsPtr prepared_sets_ )
2022-03-26 02:17:23 +00:00
: InterpreterSelectQuery (
2022-07-18 15:53:30 +00:00
query_ptr_ , context_ , std : : nullopt , nullptr , options_ , { } , { } , prepared_sets_ )
2022-03-26 02:17:23 +00:00
{ }
2018-07-19 13:36:21 +00:00
InterpreterSelectQuery : : ~ InterpreterSelectQuery ( ) = default ;
2023-01-19 10:26:38 +00:00
namespace
{
2018-07-19 13:36:21 +00:00
/** 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 .
*/
2023-01-19 10:26:38 +00:00
ContextPtr getSubqueryContext ( const 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
2023-01-19 10:26:38 +00:00
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 } ;
2022-05-10 15:12:17 +00:00
cross_to_inner . cross_to_inner_join_rewrite = static_cast < UInt8 > ( std : : min < UInt64 > ( settings . cross_to_inner_join_rewrite , 2 ) ) ;
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 } ;
2022-06-16 15:50:03 +00:00
join_to_subs_data . try_to_keep_original_names = settings . multiple_joins_try_to_keep_original_names ;
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.
2023-01-19 10:26:38 +00:00
void checkAccessRightsForSelect (
2022-07-06 12:59:25 +00:00
const 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
}
2023-01-19 10:26:38 +00:00
ASTPtr parseAdditionalFilterConditionForTable (
2022-07-05 13:10:09 +00:00
const Map & setting ,
2022-06-13 12:18:45 +00:00
const DatabaseAndTableWithAlias & target ,
const Context & context )
{
2022-07-05 13:10:09 +00:00
for ( size_t i = 0 ; i < setting . size ( ) ; + + i )
2022-06-13 12:18:45 +00:00
{
2022-07-05 13:10:09 +00:00
const auto & tuple = setting [ i ] . safeGet < const Tuple & > ( ) ;
auto & table = tuple . at ( 0 ) . safeGet < String > ( ) ;
auto & filter = tuple . at ( 1 ) . safeGet < String > ( ) ;
2022-06-13 12:18:45 +00:00
2022-07-21 11:23:09 +00:00
if ( table = = target . alias | |
( table = = target . table & & context . getCurrentDatabase ( ) = = target . database ) | |
2022-07-05 13:10:09 +00:00
( table = = target . database + ' . ' + target . table ) )
2022-06-13 12:18:45 +00:00
{
2022-07-05 13:10:09 +00:00
/// Try to parse expression
ParserExpression parser ;
const auto & settings = context . getSettingsRef ( ) ;
return parseQuery (
parser , filter . data ( ) , filter . data ( ) + filter . size ( ) ,
" additional filter " , settings . max_query_size , settings . max_parser_depth ) ;
2022-06-13 12:18:45 +00:00
}
}
2022-06-21 11:24:46 +00:00
return nullptr ;
2022-06-13 12:18:45 +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.
2023-01-19 10:26:38 +00:00
bool shouldIgnoreQuotaAndLimits ( const StorageID & table_id )
2020-12-21 23:58:54 +00:00
{
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 ;
}
2023-01-19 10:26:38 +00:00
}
2018-07-17 13:09:33 +00:00
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2022-07-05 22:47:00 +00:00
const 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_ ,
2022-07-18 15:53:30 +00:00
PreparedSetsPtr prepared_sets_ )
2022-05-30 08:57:51 +00:00
: InterpreterSelectQuery (
query_ptr_ ,
Context : : createCopy ( context_ ) ,
std : : move ( input_pipe_ ) ,
storage_ ,
options_ ,
required_result_column_names ,
metadata_snapshot_ ,
2022-07-18 15:53:30 +00:00
prepared_sets_ )
2022-05-30 08:57:51 +00:00
{ }
InterpreterSelectQuery : : InterpreterSelectQuery (
const ASTPtr & query_ptr_ ,
2022-07-05 22:47:00 +00:00
const ContextMutablePtr & context_ ,
2022-05-30 08:57:51 +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_ ,
2022-07-18 15:53:30 +00:00
PreparedSetsPtr 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_ )
2022-07-18 15:53:30 +00:00
, prepared_sets ( prepared_sets_ )
2014-12-26 10:35:03 +00:00
{
2019-08-10 17:51:47 +00:00
checkStackSize ( ) ;
2022-07-20 14:44:26 +00:00
if ( ! prepared_sets )
prepared_sets = std : : make_shared < PreparedSets > ( ) ;
2019-08-10 17:51:47 +00:00
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-04-21 16:00:27 +00:00
2015-06-05 21:28:04 +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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : TOO_DEEP_SUBQUERIES , " Too deep subqueries. Maximum: {} " ,
settings . max_subquery_depth . toString ( ) ) ;
2017-04-01 07:20:54 +00:00
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
2022-02-16 14:26:57 +00:00
query_info . original_query = query_ptr - > clone ( ) ;
2022-04-28 12:55:37 +00:00
if ( settings . count_distinct_optimization )
{
RewriteCountDistinctFunctionMatcher : : Data data_rewrite_countdistinct ;
RewriteCountDistinctFunctionVisitor ( data_rewrite_countdistinct ) . visit ( query_ptr ) ;
}
2023-03-19 18:41:19 +00:00
JoinedTables joined_tables ( getSubqueryContext ( context ) , getSelectQuery ( ) , options . with_all_cols , options_ . is_create_parameterized_view ) ;
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 ( ) ;
2022-01-28 13:47:49 +00:00
// Mark uses_view_source if the returned storage is the same as the one saved in viewSource
2022-01-19 17:26:37 +00:00
uses_view_source | = storage & & storage = = context - > getViewSource ( ) ;
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 ( ) ;
2021-07-09 03:15:41 +00:00
2022-03-17 17:26:18 +00:00
storage_snapshot = storage - > getStorageSnapshotForQuery ( metadata_snapshot , query_ptr , context ) ;
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
2022-01-31 22:27:55 +00:00
if ( context - > getCurrentTransaction ( ) & & context - > getSettingsRef ( ) . throw_on_unsupported_query_inside_transaction )
{
if ( storage )
2023-02-21 12:36:23 +00:00
checkStorageSupportsTransactionsIfNeeded ( storage , context , /* is_readonly_query */ true ) ;
2022-01-31 22:27:55 +00:00
for ( const auto & table : joined_tables . tablesWithColumns ( ) )
{
if ( table . table . table . empty ( ) )
continue ;
auto maybe_storage = DatabaseCatalog : : instance ( ) . tryGetTable ( { table . table . database , table . table . table } , context ) ;
if ( ! maybe_storage )
continue ;
2023-02-21 12:36:23 +00:00
checkStorageSupportsTransactionsIfNeeded ( storage , context , /* is_readonly_query */ true ) ;
2022-01-31 22:27:55 +00:00
}
}
2023-03-07 11:24:46 +00:00
if ( joined_tables . tablesCount ( ) > 1 & & ( ! settings . parallel_replicas_custom_key . value . empty ( ) | | settings . allow_experimental_parallel_reading_from_replicas ) )
2023-02-03 13:34:18 +00:00
{
LOG_WARNING ( log , " Joins are not supported with parallel replicas. Query will be executed without using them. " ) ;
context - > setSetting ( " allow_experimental_parallel_reading_from_replicas " , false ) ;
2023-03-07 11:24:46 +00:00
context - > setSetting ( " parallel_replicas_custom_key " , String { " " } ) ;
2023-02-03 13:34:18 +00:00
}
2023-04-05 15:19:45 +00:00
/// Try to execute query without parallel replicas if we find that there is a FINAL modifier there.
bool is_query_with_final = false ;
if ( query_info . table_expression_modifiers )
is_query_with_final = query_info . table_expression_modifiers - > hasFinal ( ) ;
else if ( query_info . query )
is_query_with_final = query_info . query - > as < ASTSelectQuery & > ( ) . final ( ) ;
if ( is_query_with_final & & ( ! settings . parallel_replicas_custom_key . value . empty ( ) | | settings . allow_experimental_parallel_reading_from_replicas ) )
{
LOG_WARNING ( log , " FINAL modifier is supported with parallel replicas. Will try to execute the query without using them. " ) ;
context - > setSetting ( " allow_experimental_parallel_reading_from_replicas " , false ) ;
context - > setSetting ( " parallel_replicas_custom_key " , String { " " } ) ;
}
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 ( ) ;
2022-01-19 17:26:37 +00:00
if ( auto view_source = context - > getViewSource ( ) )
{
2022-01-28 13:47:49 +00:00
// If we are using a virtual block view to replace a table and that table is used
// inside the JOIN then we need to update uses_view_source accordingly so we avoid propagating scalars that we can't cache
2022-01-19 17:26:37 +00:00
const auto & storage_values = static_cast < const StorageValues & > ( * view_source ) ;
auto tmp_table_id = storage_values . getStorageID ( ) ;
2022-01-24 19:21:42 +00:00
for ( const auto & t : joined_tables . tablesWithColumns ( ) )
2022-01-19 17:26:37 +00:00
uses_view_source | = ( t . table . database = = tmp_table_id . database_name & & t . table . table = = tmp_table_id . table_name ) ;
}
2020-03-08 11:07:05 +00:00
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 ;
2022-03-23 01:25:48 +00:00
storage_snapshot = nullptr ;
2018-07-18 12:17:48 +00:00
}
2014-12-19 12:48:09 +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 )
2022-01-19 17:26:37 +00:00
{
2020-03-08 11:07:05 +00:00
source_header = interpreter_subquery - > getSampleBlock ( ) ;
2022-01-19 17:26:37 +00:00
uses_view_source | = interpreter_subquery - > usesViewSource ( ) ;
}
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 )
2021-11-18 13:04:42 +00:00
row_policy_filter = context - > getRowPolicyFilter ( table_id . getDatabaseName ( ) , table_id . getTableName ( ) , RowPolicyFilterType : : SELECT_FILTER ) ;
2020-05-20 09:29:23 +00:00
2020-06-15 12:36:10 +00:00
StorageView * view = nullptr ;
if ( storage )
view = dynamic_cast < StorageView * > ( storage . get ( ) ) ;
2022-07-05 13:10:09 +00:00
if ( ! settings . additional_table_filters . value . empty ( ) & & storage & & ! joined_tables . tablesWithColumns ( ) . empty ( ) )
2022-06-21 11:24:46 +00:00
query_info . additional_filter_ast = parseAdditionalFilterConditionForTable (
2022-07-05 13:10:09 +00:00
settings . additional_table_filters , joined_tables . tablesWithColumns ( ) . front ( ) . table , * context ) ;
2022-06-21 11:24:46 +00:00
2023-01-19 10:26:38 +00:00
ASTPtr parallel_replicas_custom_filter_ast = nullptr ;
2023-03-27 13:07:24 +00:00
if ( storage & & context - > getParallelReplicasMode ( ) = = Context : : ParallelReplicasMode : : CUSTOM_KEY & & ! joined_tables . tablesWithColumns ( ) . empty ( ) )
2023-01-19 10:26:38 +00:00
{
2023-03-03 15:14:49 +00:00
if ( settings . parallel_replicas_count > 1 )
2023-01-24 10:46:47 +00:00
{
2023-03-03 15:14:49 +00:00
if ( auto custom_key_ast = parseCustomKeyForTable ( settings . parallel_replicas_custom_key , * context ) )
{
2023-03-07 10:56:11 +00:00
LOG_TRACE ( log , " Processing query on a replica using custom_key '{}' " , settings . parallel_replicas_custom_key . value ) ;
2023-03-03 15:14:49 +00:00
parallel_replicas_custom_filter_ast = getCustomKeyFilterForParallelReplica (
settings . parallel_replicas_count ,
settings . parallel_replica_offset ,
std : : move ( custom_key_ast ) ,
settings . parallel_replicas_custom_key_filter_type ,
* storage ,
context ) ;
}
else if ( settings . parallel_replica_offset > 0 )
{
2023-03-07 10:56:11 +00:00
throw Exception (
ErrorCodes : : BAD_ARGUMENTS ,
" Parallel replicas processing with custom_key has been requested "
" (setting 'max_parallel_replicas') but the table does not have custom_key defined for it "
" or it's invalid (settings `parallel_replicas_custom_key`) " ) ;
2023-03-03 15:14:49 +00:00
}
2023-01-24 10:46:47 +00:00
}
2023-03-03 15:14:49 +00:00
else if ( auto * distributed = dynamic_cast < StorageDistributed * > ( storage . get ( ) ) ;
distributed & & canUseCustomKey ( settings , * distributed - > getCluster ( ) , * context ) )
2023-01-24 13:58:42 +00:00
{
2023-03-03 15:14:49 +00:00
query_info . use_custom_key = true ;
context - > setSetting ( " distributed_group_by_no_merge " , 2 ) ;
2023-01-24 13:58:42 +00:00
}
2023-01-19 10:26:38 +00:00
}
2023-02-03 13:03:39 +00:00
if ( autoFinalOnQuery ( query ) )
{
query . setFinal ( ) ;
}
2022-09-02 14:06:32 +00:00
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 ;
2022-12-19 14:05:38 +00:00
NameToNameMap parameter_types ;
2020-06-15 12:36:10 +00:00
if ( view )
2022-09-23 11:35:22 +00:00
{
2022-10-19 16:30:03 +00:00
query_info . is_parameterized_view = view - > isParameterizedView ( ) ;
2022-12-23 16:57:17 +00:00
/// We need to fetch the parameters set for SELECT ... FROM parameterized_view(<params>) before the query is replaced.
2022-12-16 11:19:11 +00:00
/// replaceWithSubquery replaces the function child and adds the subquery in its place.
2022-12-23 16:57:17 +00:00
/// the parameters are children of function child, if function (which corresponds to parametrised view and has
/// parameters in its arguments: `parametrised_view(<params>)`) is replaced the parameters are also gone from tree
2022-12-16 11:19:11 +00:00
/// So we need to get the parameters before they are removed from the tree
2022-12-16 10:11:19 +00:00
/// and after query is replaced, we use these parameters to substitute in the parameterized view query
2022-11-16 12:01:02 +00:00
if ( query_info . is_parameterized_view )
2022-11-28 18:05:01 +00:00
{
2023-03-12 10:00:17 +00:00
query_info . parameterized_view_values = analyzeFunctionParamValues ( query_ptr ) ;
2023-03-11 19:39:43 +00:00
parameter_types = view - > getParameterTypes ( ) ;
2022-11-28 18:05:01 +00:00
}
2022-10-19 16:30:03 +00:00
view - > replaceWithSubquery ( getSelectQuery ( ) , view_table , metadata_snapshot , view - > isParameterizedView ( ) ) ;
2022-11-16 12:01:02 +00:00
if ( query_info . is_parameterized_view )
2022-11-28 18:05:01 +00:00
{
2023-03-12 10:00:17 +00:00
view - > replaceQueryParametersIfParametrizedView ( query_ptr , query_info . parameterized_view_values ) ;
2022-11-28 18:05:01 +00:00
}
2022-09-23 11:35:22 +00:00
}
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 ,
2021-07-09 03:15:41 +00:00
TreeRewriterResult ( source_header . getNamesAndTypesList ( ) , storage , storage_snapshot ) ,
2022-11-28 18:05:01 +00:00
options ,
joined_tables . tablesWithColumns ( ) ,
required_result_column_names ,
table_join ,
query_info . is_parameterized_view ,
2023-03-12 10:00:17 +00:00
query_info . parameterized_view_values ,
2022-12-19 14:05:38 +00:00
parameter_types ) ;
2019-10-19 20:36:35 +00:00
2023-02-03 13:34:18 +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
2022-01-17 18:32:55 +00:00
/// Note that we are only saving scalars and not local_scalars since the latter can't be safely shared across contexts
if ( ! options . only_analyze & & context - > hasQueryContext ( ) )
2020-06-18 16:11:23 +00:00
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 ;
}
2023-01-18 12:17:30 +00:00
if ( try_move_to_prewhere
& & storage & & storage - > canMoveConditionsToPrewhere ( )
& & query . where ( ) & & ! query . prewhere ( )
& & ! query . hasJoin ( ) ) /// Join may produce rows with nulls or default values, it's difficult to analyze if they affected or not.
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 ;
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
Names queried_columns = syntax_analyzer_result - > requiredSourceColumns ( ) ;
const auto & supported_prewhere_columns = storage - > supportedPrewhereColumns ( ) ;
2023-03-20 10:29:27 +00:00
MergeTreeWhereOptimizer where_optimizer {
2021-04-10 23:33:54 +00:00
std : : move ( column_compressed_sizes ) ,
metadata_snapshot ,
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
queried_columns ,
2023-03-02 17:45:59 +00:00
supported_prewhere_columns ,
2021-04-10 23:33:54 +00:00
log } ;
2023-03-20 10:29:27 +00:00
where_optimizer . optimize ( current_info , context ) ;
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 ,
2023-01-04 19:59:15 +00:00
required_result_column_names ,
2021-06-03 14:05:37 +00:00
! options . only_analyze ,
options ,
2022-07-18 15:53:30 +00:00
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 ( ) ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : SAMPLING_NOT_SUPPORTED , " Illegal SAMPLE: table doesn't support sampling " ) ;
2017-11-05 17:48:50 +00:00
2021-10-13 18:22:02 +00:00
if ( query . final ( ) & & ( input_pipe | | ! storage | | ! storage - > supportsFinal ( ) ) )
2023-01-23 21:13:58 +00:00
{
if ( ! input_pipe & & storage )
throw Exception ( ErrorCodes : : ILLEGAL_FINAL , " Storage {} doesn't support FINAL " , storage - > getName ( ) ) ;
else
throw Exception ( ErrorCodes : : ILLEGAL_FINAL , " 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 ( ) ) )
2023-01-23 21:13:58 +00:00
{
if ( ! input_pipe & & storage )
throw Exception ( ErrorCodes : : ILLEGAL_PREWHERE , " Storage {} doesn't support PREWHERE " , storage - > getName ( ) ) ;
else
throw Exception ( ErrorCodes : : ILLEGAL_PREWHERE , " 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 ( ) ;
2022-01-28 13:47:49 +00:00
uses_view_source | = interpreter_subquery - > usesViewSource ( ) ;
2019-10-27 18:12:40 +00:00
}
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
{
2022-07-28 09:40:09 +00:00
query_info . filter_asts . clear ( ) ;
2019-10-27 18:12:40 +00:00
/// Fix source_header for filter actions.
2022-10-24 07:58:14 +00:00
if ( row_policy_filter & & ! row_policy_filter - > empty ( ) )
2019-10-27 18:12:40 +00:00
{
2022-06-21 11:24:46 +00:00
filter_info = generateFilterActions (
2022-10-24 07:58:14 +00:00
table_id , row_policy_filter - > expression , context , storage , storage_snapshot , metadata_snapshot , required_columns ,
2022-10-21 12:53:23 +00:00
prepared_sets ) ;
2022-07-28 09:40:09 +00:00
2022-10-24 07:58:14 +00:00
query_info . filter_asts . push_back ( row_policy_filter - > expression ) ;
2022-06-21 11:24:46 +00:00
}
2021-02-20 11:00:16 +00:00
2022-06-21 11:24:46 +00:00
if ( query_info . additional_filter_ast )
{
additional_filter_info = generateFilterActions (
2022-10-21 12:53:23 +00:00
table_id , query_info . additional_filter_ast , context , storage , storage_snapshot , metadata_snapshot , required_columns ,
prepared_sets ) ;
2021-02-20 11:00:16 +00:00
2022-06-21 11:24:46 +00:00
additional_filter_info - > do_remove_column = true ;
2022-07-28 09:40:09 +00:00
query_info . filter_asts . push_back ( query_info . additional_filter_ast ) ;
2019-10-27 18:12:40 +00:00
}
2019-10-03 11:58:52 +00:00
2023-01-19 10:26:38 +00:00
if ( parallel_replicas_custom_filter_ast )
{
parallel_replicas_custom_filter_info = generateFilterActions (
table_id , parallel_replicas_custom_filter_ast , context , storage , storage_snapshot , metadata_snapshot , required_columns ,
prepared_sets ) ;
parallel_replicas_custom_filter_info - > do_remove_column = true ;
query_info . filter_asts . push_back ( parallel_replicas_custom_filter_ast ) ;
}
2023-03-12 10:00:17 +00:00
source_header = storage_snapshot - > getSampleBlockForColumns ( required_columns , query_info . parameterized_view_values ) ;
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 ;
2022-08-26 14:20:54 +00:00
bool can_analyze_again = false ;
if ( context - > hasQueryContext ( ) )
{
/// Check number of calls of 'analyze' function.
/// If it is too big, we will not analyze the query again not to have exponential blowup.
std : : atomic < size_t > & current_query_analyze_count = context - > getQueryContext ( ) - > kitchen_sink . analyze_counter ;
+ + current_query_analyze_count ;
can_analyze_again = settings . max_analyze_depth = = 0 | | current_query_analyze_count < settings . max_analyze_depth ;
}
2022-08-18 09:13:47 +00:00
2022-08-26 14:20:54 +00:00
if ( can_analyze_again & & ( 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 ;
}
2022-08-18 09:13:47 +00:00
2022-08-26 14:20:54 +00:00
if ( can_analyze_again & & ( analysis_result . where_constant_filter_description . always_false | |
analysis_result . where_constant_filter_description . always_true ) )
2019-10-27 18:12:40 +00:00
{
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
{
2022-08-26 14:20:54 +00:00
size_t current_query_analyze_count = context - > getQueryContext ( ) - > kitchen_sink . analyze_counter . load ( ) ;
LOG_TRACE ( log , " Running 'analyze' second time (current analyze depth: {}) " , current_query_analyze_count ) ;
2021-06-03 14:05:37 +00:00
/// Reuse already built sets for multiple passes of analysis
2022-07-18 15:53:30 +00:00
prepared_sets = query_analyzer - > getPreparedSets ( ) ;
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 ;
}
2022-01-28 14:23:51 +00:00
if ( query_info . projection )
storage_snapshot - > addProjection ( query_info . projection - > desc ) ;
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 ) ) ;
}
2022-05-13 20:02:28 +00:00
/// Extend lifetime of context, table lock, storage.
query_plan . addInterpreterContext ( context ) ;
if ( table_lock )
query_plan . addTableLock ( std : : move ( table_lock ) ) ;
if ( storage )
query_plan . addStorageHolder ( storage ) ;
2020-06-18 17:45:00 +00:00
}
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
2022-05-23 13:46:57 +00:00
auto builder = query_plan . buildQueryPipeline (
2022-05-20 19:49:31 +00:00
QueryPlanOptimizationSettings : : fromContext ( context ) , BuildQueryPipelineSettings : : fromContext ( context ) ) ;
2022-05-09 10:28:05 +00:00
2022-05-24 20:06:08 +00:00
res . pipeline = QueryPipelineBuilder : : getPipeline ( std : : move ( * builder ) ) ;
2022-05-09 10:28:05 +00:00
2022-06-02 10:34:40 +00:00
setQuota ( res . pipeline ) ;
2022-05-09 10:28:05 +00:00
2015-06-18 02:11:05 +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
{
2022-07-01 13:36:20 +00:00
auto & select_query = getSelectQuery ( ) ;
2020-09-10 19:55:36 +00:00
query_info . query = query_ptr ;
2022-07-01 13:36:20 +00:00
/// NOTE: this is required for getQueryProcessingStage(), so should be initialized before ExpressionAnalysisResult.
2021-05-05 21:26:14 +00:00
query_info . has_window = query_analyzer - > hasWindow ( ) ;
2022-07-01 13:36:20 +00:00
/// NOTE: this is required only for IStorage::read(), and to be precise MergeTreeData::read(), in case of projections.
query_info . has_order_by = select_query . orderBy ( ) ! = nullptr ;
query_info . need_aggregate = query_analyzer - > hasAggregation ( ) ;
2019-10-03 15:47:42 +00:00
if ( storage & & ! options . only_analyze )
2021-02-10 14:12:49 +00:00
{
2022-07-01 13:36:20 +00:00
query_analyzer - > makeSetsForIndex ( select_query . where ( ) ) ;
query_analyzer - > makeSetsForIndex ( select_query . prewhere ( ) ) ;
2022-07-18 15:53:30 +00:00
query_info . prepared_sets = query_analyzer - > getPreparedSets ( ) ;
2019-10-03 15:47:42 +00:00
2021-07-20 15:20:21 +00:00
from_stage = storage - > getQueryProcessingStage ( context , options . to_stage , storage_snapshot , query_info ) ;
2022-01-28 14:23:51 +00:00
}
2021-04-21 16:00:27 +00:00
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 (
2022-06-27 17:41:55 +00:00
* query_analyzer , metadata_snapshot , first_stage , second_stage , options . only_analyze , filter_info , additional_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 ;
2022-05-05 17:14:52 +00:00
if ( analysis_result . use_grouping_set_key )
2022-05-05 13:56:16 +00:00
res . insert ( { nullptr , std : : make_shared < DataTypeUInt64 > ( ) , " __grouping_set " } ) ;
2022-07-11 18:11:58 +00:00
if ( context - > getSettingsRef ( ) . group_by_use_nulls & & analysis_result . use_grouping_set_key )
2022-07-07 16:40:35 +00:00
{
for ( const auto & key : query_analyzer - > aggregationKeys ( ) )
2022-07-07 18:53:20 +00:00
res . insert ( { nullptr , makeNullableSafe ( header . getByName ( key . name ) . type ) , key . name } ) ;
2022-07-07 16:40:35 +00:00
}
else
{
for ( const auto & key : query_analyzer - > aggregationKeys ( ) )
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
}
2022-07-20 13:10:45 +00:00
2022-06-06 02:39:21 +00:00
static std : : pair < Field , DataTypePtr > getWithFillFieldValue ( const ASTPtr & node , ContextPtr context )
2019-08-14 17:01:47 +00:00
{
2022-06-06 02:39:21 +00:00
auto field_type = evaluateConstantExpression ( node , context ) ;
2019-08-14 17:01:47 +00:00
2022-06-06 02:39:21 +00:00
if ( ! isColumnedAsNumber ( field_type . second ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ,
" Illegal type {} of WITH FILL expression, must be numeric type " , field_type . second - > getName ( ) ) ;
2018-02-23 06:00:48 +00:00
2022-06-06 02:39:21 +00:00
return field_type ;
2019-08-14 17:01:47 +00:00
}
2018-02-23 06:00:48 +00:00
2022-07-06 12:59:25 +00:00
static std : : pair < Field , std : : optional < IntervalKind > > getWithFillStep ( const ASTPtr & node , const ContextPtr & context )
2021-10-31 16:22:20 +00:00
{
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 ) ;
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ,
" Illegal type {} of WITH FILL expression, must be numeric type " , type - > getName ( ) ) ;
2021-10-31 16:22:20 +00:00
}
2022-07-06 12:59:25 +00:00
static FillColumnDescription getWithFillDescription ( const ASTOrderByElement & order_by_elem , const ContextPtr & context )
2019-08-14 17:01:47 +00:00
{
FillColumnDescription descr ;
2022-03-27 17:33:22 +00:00
2019-08-19 20:22:45 +00:00
if ( order_by_elem . fill_from )
2022-06-06 02:39:21 +00:00
std : : tie ( descr . fill_from , descr . fill_from_type ) = getWithFillFieldValue ( order_by_elem . fill_from , context ) ;
2019-08-19 20:22:45 +00:00
if ( order_by_elem . fill_to )
2022-06-06 02:39:21 +00:00
std : : tie ( descr . fill_to , descr . fill_to_type ) = 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 } ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION , " WITH FILL STEP value cannot be zero " ) ;
2019-08-19 20:22:45 +00:00
if ( order_by_elem . direction = = 1 )
{
if ( applyVisitor ( FieldVisitorAccurateLess ( ) , descr . fill_step , Field { 0 } ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION , " WITH FILL STEP value cannot be negative for sorting in ascending direction " ) ;
2019-08-19 20:22:45 +00:00
if ( ! descr . fill_from . isNull ( ) & & ! descr . fill_to . isNull ( ) & &
applyVisitor ( FieldVisitorAccurateLess ( ) , descr . fill_to , descr . fill_from ) )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ,
" WITH FILL TO value cannot be less than FROM value for sorting in ascending direction " ) ;
2019-08-19 20:22:45 +00:00
}
}
else
{
if ( applyVisitor ( FieldVisitorAccurateLess ( ) , Field { 0 } , descr . fill_step ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION , " WITH FILL STEP value cannot be positive for sorting in descending direction " ) ;
2019-08-19 20:22:45 +00:00
if ( ! descr . fill_from . isNull ( ) & & ! descr . fill_to . isNull ( ) & &
applyVisitor ( FieldVisitorAccurateLess ( ) , descr . fill_from , descr . fill_to ) )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ,
" WITH FILL FROM value cannot be less than TO value for sorting in descending direction " ) ;
2019-08-19 20:22:45 +00:00
}
}
2019-08-14 17:01:47 +00:00
return descr ;
}
2018-02-23 06:00:48 +00:00
2022-07-05 15:31:46 +00:00
SortDescription InterpreterSelectQuery : : getSortDescription ( const ASTSelectQuery & query , const ContextPtr & context_ )
2019-07-18 14:41:11 +00:00
{
SortDescription order_descr ;
order_descr . reserve ( query . orderBy ( ) - > children . size ( ) ) ;
2022-07-05 21:05:10 +00:00
for ( const auto & elem : query . orderBy ( ) - > children )
2022-07-05 15:31:46 +00:00
{
2022-07-05 21:05:10 +00:00
const String & column_name = elem - > children . front ( ) - > getColumnName ( ) ;
const auto & order_by_elem = elem - > as < ASTOrderByElement & > ( ) ;
2019-07-18 14:41:11 +00:00
std : : shared_ptr < Collator > collator ;
if ( order_by_elem . collation )
collator = std : : make_shared < Collator > ( order_by_elem . collation - > as < ASTLiteral & > ( ) . value . get < String > ( ) ) ;
2022-07-05 15:31:46 +00:00
2019-08-14 17:01:47 +00:00
if ( order_by_elem . with_fill )
{
2021-11-02 15:40:37 +00:00
FillColumnDescription fill_desc = getWithFillDescription ( order_by_elem , context_ ) ;
2022-07-05 21:05:10 +00:00
order_descr . emplace_back ( column_name , order_by_elem . direction , order_by_elem . nulls_direction , collator , true , fill_desc ) ;
2019-08-14 17:01:47 +00:00
}
else
2022-07-05 21:05:10 +00:00
order_descr . emplace_back ( column_name , order_by_elem . direction , order_by_elem . nulls_direction , collator ) ;
2019-07-18 14:41:11 +00:00
}
2022-05-23 08:18:28 +00:00
order_descr . compile_sort_description = context_ - > getSettingsRef ( ) . compile_sort_description ;
order_descr . min_count_to_compile_sort_description = context_ - > getSettingsRef ( ) . min_count_to_compile_sort_description ;
2022-03-28 21:19:14 +00:00
2019-07-18 14:41:11 +00:00
return order_descr ;
}
2022-04-01 20:18:19 +00:00
static InterpolateDescriptionPtr getInterpolateDescription (
2022-04-05 18:26:49 +00:00
const ASTSelectQuery & query , const Block & source_block , const Block & result_block , const Aliases & aliases , ContextPtr context )
2022-03-17 05:51:35 +00:00
{
2022-03-24 19:29:29 +00:00
InterpolateDescriptionPtr interpolate_descr ;
2022-03-18 20:44:27 +00:00
if ( query . interpolate ( ) )
2022-03-17 05:51:35 +00:00
{
2022-04-07 05:21:24 +00:00
NamesAndTypesList source_columns ;
ColumnsWithTypeAndName result_columns ;
2022-03-28 23:15:53 +00:00
ASTPtr exprs = std : : make_shared < ASTExpressionList > ( ) ;
2022-04-05 18:26:49 +00:00
if ( query . interpolate ( ) - > children . empty ( ) )
2022-03-18 20:44:27 +00:00
{
2022-04-05 18:26:49 +00:00
std : : unordered_map < String , DataTypePtr > column_names ;
for ( const auto & column : result_block . getColumnsWithTypeAndName ( ) )
column_names [ column . name ] = column . type ;
for ( const auto & elem : query . orderBy ( ) - > children )
if ( elem - > as < ASTOrderByElement > ( ) - > with_fill )
column_names . erase ( elem - > as < ASTOrderByElement > ( ) - > children . front ( ) - > getColumnName ( ) ) ;
for ( const auto & [ name , type ] : column_names )
2022-04-01 20:18:19 +00:00
{
2022-04-07 05:21:24 +00:00
source_columns . emplace_back ( name , type ) ;
result_columns . emplace_back ( type , name ) ;
2022-04-05 18:26:49 +00:00
exprs - > children . emplace_back ( std : : make_shared < ASTIdentifier > ( name ) ) ;
}
}
else
{
2022-04-07 05:21:24 +00:00
NameSet col_set ;
2022-04-05 18:26:49 +00:00
for ( const auto & elem : query . interpolate ( ) - > children )
{
const auto & interpolate = elem - > as < ASTInterpolateElement & > ( ) ;
2022-04-07 05:21:24 +00:00
if ( const ColumnWithTypeAndName * result_block_column = result_block . findByName ( interpolate . column ) )
{
if ( ! col_set . insert ( result_block_column - > name ) . second )
throw Exception ( ErrorCodes : : INVALID_WITH_FILL_EXPRESSION ,
" Duplicate INTERPOLATE column '{}' " , interpolate . column ) ;
result_columns . emplace_back ( result_block_column - > type , result_block_column - > name ) ;
}
else
2022-04-05 18:26:49 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_IDENTIFIER ,
" Missing column '{}' as an INTERPOLATE expression target " , interpolate . column ) ;
exprs - > children . emplace_back ( interpolate . expr - > clone ( ) ) ;
2022-04-01 20:18:19 +00:00
}
2022-04-07 05:21:24 +00:00
col_set . clear ( ) ;
for ( const auto & column : source_block )
{
source_columns . emplace_back ( column . name , column . type ) ;
col_set . insert ( column . name ) ;
}
for ( const auto & column : result_block )
2022-04-18 10:18:43 +00:00
if ( ! col_set . contains ( column . name ) )
2022-04-07 05:21:24 +00:00
source_columns . emplace_back ( column . name , column . type ) ;
2022-03-24 19:29:29 +00:00
}
2022-04-07 05:21:24 +00:00
auto syntax_result = TreeRewriter ( context ) . analyze ( exprs , source_columns ) ;
2022-03-24 19:29:29 +00:00
ExpressionAnalyzer analyzer ( exprs , syntax_result , context ) ;
2022-03-30 20:34:19 +00:00
ActionsDAGPtr actions = analyzer . getActionsDAG ( true ) ;
2022-03-31 12:33:50 +00:00
ActionsDAGPtr conv_dag = ActionsDAG : : makeConvertingActions ( actions - > getResultColumns ( ) ,
2022-04-07 05:21:24 +00:00
result_columns , ActionsDAG : : MatchColumnsMode : : Position , true ) ;
2022-03-31 12:33:50 +00:00
ActionsDAGPtr merge_dag = ActionsDAG : : merge ( std : : move ( * actions - > clone ( ) ) , std : : move ( * conv_dag ) ) ;
2022-03-24 19:29:29 +00:00
2022-03-31 12:33:50 +00:00
interpolate_descr = std : : make_shared < InterpolateDescription > ( merge_dag , aliases ) ;
2022-03-17 05:51:35 +00:00
}
return interpolate_descr ;
}
2021-08-03 18:03:24 +00:00
static SortDescription getSortDescriptionFromGroupBy ( const ASTSelectQuery & query )
2020-02-14 07:12:04 +00:00
{
2022-11-27 23:41:31 +00:00
if ( ! query . groupBy ( ) )
return { } ;
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
}
2022-07-06 12:59:25 +00:00
static UInt64 getLimitUIntValue ( const ASTPtr & node , const ContextPtr & context , const std : : string & expr )
2019-07-18 14:41:11 +00:00
{
const auto & [ field , type ] = evaluateConstantExpression ( node , context ) ;
if ( ! isNativeNumber ( type ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_LIMIT_EXPRESSION , " Illegal type {} of {} expression, must be numeric type " ,
type - > getName ( ) , expr ) ;
2019-07-18 14:41:11 +00:00
Field converted = convertFieldToType ( field , DataTypeUInt64 ( ) ) ;
if ( converted . isNull ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_LIMIT_EXPRESSION , " The value {} of {} expression is not representable as UInt64 " ,
applyVisitor ( FieldVisitorToString ( ) , field ) , expr ) ;
2019-07-18 14:41:11 +00:00
return converted . safeGet < UInt64 > ( ) ;
}
2022-07-06 12:59:25 +00:00
static std : : pair < UInt64 , UInt64 > getLimitLengthAndOffset ( const ASTSelectQuery & query , const 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 } ;
}
2022-07-06 12:59:25 +00:00
UInt64 InterpreterSelectQuery : : getLimitForSorting ( const ASTSelectQuery & query , const 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
{
2021-11-02 15:40:37 +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 > ( ) )
2021-02-17 08:26:52 +00:00
{
2021-11-09 08:16:18 +00:00
if ( hasWithTotalsInAnySubqueryInFromClause ( select_child - > as < ASTSelectQuery & > ( ) ) )
return true ;
2021-02-17 08:26:52 +00:00
}
2021-11-09 08:16:18 +00:00
else if ( const auto * union_child = child_ast - > as < ASTSelectWithUnionQuery > ( ) )
2021-08-16 20:34:39 +00:00
{
2021-11-09 08:16:18 +00:00
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 > ( ) )
2021-02-17 08:26:52 +00:00
{
2021-11-09 08:16:18 +00:00
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-02-17 08:26:52 +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 .
2014-12-19 12:48:09 +00:00
*/
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-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
2022-08-18 10:45:16 +00:00
/// Do I need to aggregate in a separate row that has not passed max_rows_to_group_by?
2021-05-04 10:52:37 +00:00
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 & &
2021-12-25 17:30:31 +00:00
! query . group_by_with_totals & & ! query . group_by_with_rollup & & ! query . group_by_with_cube ;
2021-05-04 10:52:37 +00:00
2022-05-05 17:14:52 +00:00
bool use_grouping_set_key = expressions . use_grouping_set_key ;
2021-05-04 10:52:37 +00:00
2022-04-21 14:11:58 +00:00
if ( query . group_by_with_grouping_sets & & query . group_by_with_totals )
2022-05-19 14:10:04 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " WITH TOTALS and GROUPING SETS are not supported together " ) ;
if ( query . group_by_with_grouping_sets & & ( query . group_by_with_rollup | | query . group_by_with_cube ) )
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " GROUPING SETS are not supported together with ROLLUP and CUBE " ) ;
2021-05-27 19:50:35 +00:00
2022-05-19 16:36:51 +00:00
if ( expressions . hasHaving ( ) & & query . group_by_with_totals & & ( query . group_by_with_rollup | | query . group_by_with_cube ) )
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING " ) ;
2021-05-04 10:52:37 +00:00
if ( query_info . projection & & query_info . projection - > desc - > type = = ProjectionDescription : : Type : : Aggregate )
2021-02-15 19:48:06 +00:00
{
2021-05-04 10:52:37 +00:00
query_info . projection - > aggregate_overflow_row = aggregate_overflow_row ;
query_info . projection - > aggregate_final = aggregate_final ;
2021-02-15 19:48:06 +00:00
}
2018-07-19 13:36:21 +00:00
2019-08-09 13:42:58 +00:00
if ( options . only_analyze )
2018-07-19 13:36:21 +00:00
{
2020-06-18 13:38:45 +00:00
auto read_nothing = std : : make_unique < ReadNothingStep > ( source_header ) ;
query_plan . addStep ( std : : move ( read_nothing ) ) ;
2019-04-03 11:21:38 +00:00
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 ,
2022-06-13 09:57:55 +00:00
true ) ;
2021-02-15 19:48:06 +00:00
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
{
2022-05-20 19:49:31 +00:00
auto prepared_source_step = std : : make_unique < ReadFromPreparedSource > ( std : : move ( * prepared_pipe ) ) ;
2020-06-18 13:38:45 +00:00
query_plan . addStep ( std : : move ( prepared_source_step ) ) ;
2022-05-20 19:49:31 +00:00
query_plan . addInterpreterContext ( context ) ;
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
2022-05-19 05:00:38 +00:00
if ( query_info . projection & & query_info . projection - > input_order_info & & query_info . input_order_info )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " InputOrderInfo is set for projection and for query " ) ;
2022-05-19 05:00:38 +00:00
InputOrderInfoPtr input_order_info_for_order ;
if ( ! expressions . need_aggregate )
input_order_info_for_order = query_info . projection ? query_info . projection - > input_order_info : query_info . input_order_info ;
2019-03-18 12:05:51 +00:00
if ( options . to_stage > QueryProcessingStage : : FetchColumns )
2014-12-19 12:48:09 +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 )
2022-05-19 05:00:38 +00:00
executeOrder ( query_plan , input_order_info_for_order ) ;
2020-03-30 18:50:42 +00:00
2022-06-24 02:29:10 +00:00
/// pre_distinct = false, because if we have limit and distinct,
/// we need to merge streams to one and calculate overall distinct.
/// Otherwise we can take several equal values from different streams
/// according to limit and skip some distinct values.
if ( query . limitLength ( ) )
executeDistinct ( query_plan , false , expressions . selected_columns , false ) ;
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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Query with intermediate stage cannot have any other stages " ) ;
2020-03-30 18:50:42 +00:00
preliminary_sort ( ) ;
if ( expressions . need_aggregate )
2022-05-05 17:14:52 +00:00
executeMergeAggregated ( query_plan , aggregate_overflow_row , aggregate_final , use_grouping_set_key ) ;
2020-03-30 18:50:42 +00:00
}
2022-06-03 15:26:06 +00:00
2020-08-15 13:57:17 +00:00
if ( from_aggregation_stage )
{
if ( intermediate_stage | | expressions . first_stage | | expressions . second_stage )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Query with after aggregation stage cannot have any other stages " ) ;
2020-08-15 13:57:17 +00:00
}
2018-02-23 06:00:48 +00:00
if ( expressions . first_stage )
2014-12-19 12:48:09 +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
}
2023-01-19 10:26:38 +00:00
const auto add_filter_step = [ & ] ( const auto & new_filter_info , const std : : string & description )
2022-06-21 11:24:46 +00:00
{
2023-01-19 10:26:38 +00:00
auto filter_step = std : : make_unique < FilterStep > (
2022-06-21 11:24:46 +00:00
query_plan . getCurrentDataStream ( ) ,
2023-01-19 10:26:38 +00:00
new_filter_info - > actions ,
new_filter_info - > column_name ,
new_filter_info - > do_remove_column ) ;
2022-06-21 11:24:46 +00:00
2023-01-19 10:26:38 +00:00
filter_step - > setStepDescription ( description ) ;
query_plan . addStep ( std : : move ( filter_step ) ) ;
} ;
if ( additional_filter_info )
add_filter_step ( additional_filter_info , " Additional filter " ) ;
if ( parallel_replicas_custom_filter_info )
add_filter_step ( parallel_replicas_custom_filter_info , " Parallel replica custom key filter " ) ;
2022-06-21 11:24:46 +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 ) ;
2017-04-01 07:20:54 +00:00
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 ( ) ;
2017-04-01 07:20:54 +00:00
2021-04-28 17:32:12 +00:00
if ( ! joined_plan )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " There is no joined plan for query " ) ;
2020-06-18 13:38:45 +00:00
2022-08-03 11:08:25 +00:00
auto add_sorting = [ & settings , this ] ( QueryPlan & plan , const Names & key_names , JoinTableSide join_pos )
2022-03-30 10:07:09 +00:00
{
SortDescription order_descr ;
order_descr . reserve ( key_names . size ( ) ) ;
for ( const auto & key_name : key_names )
order_descr . emplace_back ( key_name ) ;
2022-11-09 16:07:38 +00:00
SortingStep : : Settings sort_settings ( * context ) ;
2022-11-01 19:51:52 +00:00
2022-03-30 10:07:09 +00:00
auto sorting_step = std : : make_unique < SortingStep > (
plan . getCurrentDataStream ( ) ,
2022-08-09 22:19:25 +00:00
std : : move ( order_descr ) ,
2022-11-01 19:51:52 +00:00
0 /* LIMIT */ , sort_settings ,
2022-08-16 16:27:41 +00:00
settings . optimize_sorting_by_input_stream_properties ) ;
2022-08-03 11:08:25 +00:00
sorting_step - > setStepDescription ( fmt : : format ( " Sort {} before JOIN " , join_pos ) ) ;
2022-03-30 10:07:09 +00:00
plan . addStep ( std : : move ( sorting_step ) ) ;
} ;
2022-08-03 11:28:28 +00:00
auto crosswise_connection = CreateSetAndFilterOnTheFlyStep : : createCrossConnection ( ) ;
2022-08-09 11:54:15 +00:00
auto add_create_set = [ & settings , crosswise_connection ] ( QueryPlan & plan , const Names & key_names , JoinTableSide join_pos )
2022-07-19 17:43:18 +00:00
{
2022-08-03 11:28:28 +00:00
auto creating_set_step = std : : make_unique < CreateSetAndFilterOnTheFlyStep > (
2022-08-09 12:39:39 +00:00
plan . getCurrentDataStream ( ) , key_names , settings . max_rows_in_set_to_optimize_join , crosswise_connection , join_pos ) ;
2022-08-03 11:08:25 +00:00
creating_set_step - > setStepDescription ( fmt : : format ( " Create set and filter {} joined stream " , join_pos ) ) ;
auto * step_raw_ptr = creating_set_step . get ( ) ;
plan . addStep ( std : : move ( creating_set_step ) ) ;
return step_raw_ptr ;
2022-07-19 17:43:18 +00:00
} ;
2022-03-30 10:07:09 +00:00
if ( expressions . join - > pipelineType ( ) = = JoinPipelineType : : YShaped )
{
2022-08-03 11:15:04 +00:00
const auto & table_join = expressions . join - > getTableJoin ( ) ;
const auto & join_clause = table_join . getOnlyClause ( ) ;
2022-07-19 17:43:18 +00:00
2022-08-03 11:15:04 +00:00
auto join_kind = table_join . kind ( ) ;
bool kind_allows_filtering = isInner ( join_kind ) | | isLeft ( join_kind ) | | isRight ( join_kind ) ;
2023-02-08 16:55:34 +00:00
auto has_non_const = [ ] ( const Block & block , const auto & keys )
{
for ( const auto & key : keys )
{
const auto & column = block . getByName ( key ) . column ;
if ( column & & ! isColumnConst ( * column ) )
return true ;
}
return false ;
} ;
/// This optimization relies on the sorting that should buffer the whole stream before emitting any rows.
/// It doesn't hold such a guarantee for streams with const keys.
2023-02-09 14:03:10 +00:00
/// Note: it's also doesn't work with the read-in-order optimization.
/// No checks here because read in order is not applied if we have `CreateSetAndFilterOnTheFlyStep` in the pipeline between the reading and sorting steps.
2023-02-08 16:55:34 +00:00
bool has_non_const_keys = has_non_const ( query_plan . getCurrentDataStream ( ) . header , join_clause . key_names_left )
& & has_non_const ( joined_plan - > getCurrentDataStream ( ) . header , join_clause . key_names_right ) ;
if ( settings . max_rows_in_set_to_optimize_join > 0 & & kind_allows_filtering & & has_non_const_keys )
2022-07-19 17:43:18 +00:00
{
2022-08-09 11:54:15 +00:00
auto * left_set = add_create_set ( query_plan , join_clause . key_names_left , JoinTableSide : : Left ) ;
auto * right_set = add_create_set ( * joined_plan , join_clause . key_names_right , JoinTableSide : : Right ) ;
2022-08-03 11:15:04 +00:00
if ( isInnerOrLeft ( join_kind ) )
right_set - > setFiltering ( left_set - > getSet ( ) ) ;
if ( isInnerOrRight ( join_kind ) )
left_set - > setFiltering ( right_set - > getSet ( ) ) ;
2022-07-19 17:43:18 +00:00
}
2022-08-03 11:08:25 +00:00
add_sorting ( query_plan , join_clause . key_names_left , JoinTableSide : : Left ) ;
add_sorting ( * joined_plan , join_clause . key_names_right , JoinTableSide : : Right ) ;
2022-03-30 10:07:09 +00:00
}
2021-04-28 17:32:12 +00:00
QueryPlanStepPtr join_step = std : : make_unique < JoinStep > (
query_plan . getCurrentDataStream ( ) ,
joined_plan - > getCurrentDataStream ( ) ,
expressions . join ,
2022-04-27 02:08:22 +00:00
settings . max_block_size ,
max_streams ,
analysis_result . optimize_read_in_order ) ;
2021-04-28 17:32:12 +00:00
2022-03-30 10:07:09 +00:00
join_step - > setStepDescription ( fmt : : format ( " JOIN {} " , expressions . join - > pipelineType ( ) ) ) ;
2021-04-28 17:32:12 +00:00
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 ) } ) ;
2019-03-26 18:28:37 +00:00
}
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 )
2021-05-05 07:31:27 +00:00
executeAggregation (
query_plan , expressions . before_aggregation , aggregate_overflow_row , aggregate_final , query_info . input_order_info ) ;
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".
}
2014-12-19 12:48:09 +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 " ) ;
2022-06-21 15:06:47 +00:00
executeDistinct ( query_plan , true , expressions . selected_columns , true ) ;
2021-04-23 10:58:54 +00:00
}
2014-12-19 12:48:09 +00:00
}
2017-04-01 07:20:54 +00:00
2020-03-30 18:50:42 +00:00
preliminary_sort ( ) ;
2014-12-19 12:48:09 +00:00
}
2017-04-01 07:20:54 +00:00
2020-08-15 13:57:17 +00:00
if ( expressions . second_stage | | from_aggregation_stage )
2014-12-19 12:48:09 +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 )
2014-12-19 12:48:09 +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 )
2022-05-05 17:14:52 +00:00
executeMergeAggregated ( query_plan , aggregate_overflow_row , aggregate_final , use_grouping_set_key ) ;
2017-04-01 07:20:54 +00:00
2014-12-19 12:48:09 +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
{
2021-12-25 17:30:31 +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
2021-02-09 18:40:10 +00:00
if ( ( query . group_by_with_rollup | | query . group_by_with_cube | | query . group_by_with_grouping_sets ) & & expressions . hasHaving ( ) )
2021-09-28 10:01:47 +00:00
executeHaving ( query_plan , expressions . before_having , expressions . remove_having_filter ) ;
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
}
2021-02-09 18:40:10 +00:00
else if ( query . group_by_with_totals | | query . group_by_with_rollup | | query . group_by_with_cube | | query . group_by_with_grouping_sets )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " WITH TOTALS, ROLLUP, CUBE or GROUPING SETS are not supported without aggregation " ) ;
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 ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Window functions does not support processing from WithMergeableStateAfterAggregation " ) ;
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 " ) ;
2022-06-21 15:06:47 +00:00
executeDistinct ( query_plan , true , expressions . selected_columns , true ) ;
2014-12-19 12:48:09 +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 " ) ;
2022-06-21 15:06:47 +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.
}
}
2017-04-01 07:20:54 +00:00
2018-02-23 06:00:48 +00:00
if ( expressions . has_order_by )
2015-01-18 08:27:28 +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 .
2015-01-18 08:27:28 +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.
2022-05-19 05:00:38 +00:00
executeOrder ( query_plan , input_order_info_for_order ) ;
2015-01-18 08:27:28 +00:00
}
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 )
2022-06-21 15:06:47 +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 ) ;
2014-12-19 12:48:09 +00:00
}
}
2017-04-01 07:20:54 +00:00
2022-07-18 15:53:30 +00:00
executeSubqueriesInSetsAndJoins ( query_plan ) ;
2012-05-09 13:12:38 +00:00
}
2021-05-04 10:52:37 +00:00
static void executeMergeAggregatedImpl (
QueryPlan & query_plan ,
bool overflow_row ,
bool final ,
bool is_remote_storage ,
2022-05-05 13:56:16 +00:00
bool has_grouping_sets ,
2021-05-04 10:52:37 +00:00
const Settings & settings ,
const NamesAndTypesList & aggregation_keys ,
2022-06-28 23:16:30 +00:00
const AggregateDescriptions & aggregates ,
2022-11-27 23:41:31 +00:00
bool should_produce_results_in_order_of_bucket_number ,
SortDescription group_by_sort_description )
2021-05-04 10:52:37 +00:00
{
2022-06-27 11:16:52 +00:00
auto keys = aggregation_keys . getNames ( ) ;
2022-05-05 13:56:16 +00:00
if ( has_grouping_sets )
2022-06-27 11:16:52 +00:00
keys . insert ( keys . begin ( ) , " __grouping_set " ) ;
2021-05-04 10:52:37 +00:00
/** There are two modes of distributed aggregation.
*
* 1. In different threads read from the remote servers blocks .
* Save all the blocks in the RAM . Merge blocks .
* If the aggregation is two - level - parallelize to the number of buckets .
*
* 2. In one thread , read blocks from different servers in order .
* RAM stores only one block from each server .
* If the aggregation is a two - level aggregation , we consistently merge the blocks of each next level .
*
* The second option consumes less memory ( up to 256 times less )
* in the case of two - level aggregation , which is used for large results after GROUP BY ,
* but it can work more slowly .
*/
2022-07-29 14:10:53 +00:00
Aggregator : : Params params ( keys , aggregates , overflow_row , settings . max_threads , settings . max_block_size ) ;
2021-05-04 10:52:37 +00:00
auto merging_aggregated = std : : make_unique < MergingAggregatedStep > (
2021-06-03 14:05:37 +00:00
query_plan . getCurrentDataStream ( ) ,
2022-06-27 11:16:52 +00:00
params ,
final ,
2023-01-09 11:57:37 +00:00
/// Grouping sets don't work with distributed_aggregation_memory_efficient enabled (#43989)
settings . distributed_aggregation_memory_efficient & & is_remote_storage & & ! has_grouping_sets ,
2021-06-03 14:05:37 +00:00
settings . max_threads ,
2022-06-28 23:16:30 +00:00
settings . aggregation_memory_efficient_merge_threads ,
2022-11-27 23:41:31 +00:00
should_produce_results_in_order_of_bucket_number ,
settings . max_block_size ,
settings . aggregation_in_order_max_block_bytes ,
std : : move ( group_by_sort_description ) ,
settings . enable_memory_bound_merging_of_aggregation_results ) ;
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 (
2022-07-06 12:59:25 +00:00
QueryPlan & query_plan , const Block & source_header , const SelectQueryInfo & query_info , const 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 )
2020-09-18 14:16:53 +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 . 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
} ) ;
}
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-04 12:40:34 +00:00
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
}
2022-06-28 23:16:30 +00:00
// Let's just choose the safe option since we don't know the value of `to_stage` here.
const bool should_produce_results_in_order_of_bucket_number = true ;
2022-11-27 23:41:31 +00:00
// It is used to determine if we should use memory bound merging strategy. Maybe it makes sense for projections, but so far this case is just left untouched.
SortDescription group_by_sort_description ;
2021-05-04 12:40:34 +00:00
executeMergeAggregatedImpl (
query_plan ,
query_info . projection - > aggregate_overflow_row ,
query_info . projection - > aggregate_final ,
false ,
2022-05-05 13:56:16 +00:00
false ,
2021-05-04 12:40:34 +00:00
context_ - > getSettingsRef ( ) ,
query_info . projection - > aggregation_keys ,
2022-06-28 23:16:30 +00:00
query_info . projection - > aggregate_descriptions ,
2022-11-27 23:41:31 +00:00
should_produce_results_in_order_of_bucket_number ,
std : : move ( group_by_sort_description ) ) ;
2021-05-04 12:40:34 +00:00
}
2021-05-04 10:52:37 +00:00
}
2020-09-18 14:16:53 +00:00
}
2022-10-24 07:58:14 +00:00
RowPolicyFilterPtr InterpreterSelectQuery : : getRowPolicyFilter ( ) const
2022-08-13 13:03:16 +00:00
{
2022-10-24 07:58:14 +00:00
return row_policy_filter ;
2022-08-13 13:03:16 +00:00
}
2022-10-21 16:13:36 +00:00
void InterpreterSelectQuery : : extendQueryLogElemImpl ( QueryLogElement & elem , const ASTPtr & /*ast*/ , ContextPtr /*context_*/ ) const
2022-08-02 10:45:06 +00:00
{
2022-10-24 07:58:14 +00:00
for ( const auto & row_policy : row_policy_filter - > policies )
2022-08-02 10:45:06 +00:00
{
2022-08-13 13:03:16 +00:00
auto name = row_policy - > getFullName ( ) . toString ( ) ;
2022-10-21 16:13:36 +00:00
elem . used_row_policies . emplace ( std : : move ( name ) ) ;
2022-08-02 10:45:06 +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 ;
}
}
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
/// Set of all (including ALIAS) required columns for PREWHERE
auto get_prewhere_columns = [ & ] ( )
2021-05-02 13:38:19 +00:00
{
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
NameSet columns ;
2021-05-02 13:38:19 +00:00
if ( prewhere_info )
{
/// Get some columns directly from PREWHERE expression actions
auto prewhere_required_columns = prewhere_info - > prewhere_actions - > getRequiredColumns ( ) . getNames ( ) ;
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
columns . insert ( prewhere_required_columns . begin ( ) , prewhere_required_columns . end ( ) ) ;
2021-05-02 13:38:19 +00:00
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 ( ) ;
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
columns . insert ( row_level_required_columns . begin ( ) , row_level_required_columns . end ( ) ) ;
2021-05-02 13:38:19 +00:00
}
}
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
return columns ;
} ;
/// 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 = get_prewhere_columns ( ) ;
NameSet required_aliases_from_prewhere ; /// Set of ALIAS required columns for PREWHERE
2021-05-02 13:38:19 +00:00
/// 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 ) ;
2022-04-18 10:18:43 +00:00
if ( required_columns_from_prewhere . contains ( column ) )
2021-05-02 13:38:19 +00:00
{
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 ;
2022-04-18 10:18:43 +00:00
if ( columns_to_remove . contains ( column . name ) )
2021-05-02 13:38:19 +00:00
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
2022-10-21 12:45:28 +00:00
= TreeRewriter ( context ) . analyze ( required_columns_all_expr , required_columns_after_prewhere , storage , storage_snapshot , options . is_create_parameterized_view ) ;
2021-05-02 13:38:19 +00:00
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.
2022-04-18 10:18:43 +00:00
std : : erase_if ( required_columns , [ & ] ( const String & name ) { return required_columns_after_prewhere_set . contains ( name ) ; } ) ;
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 ) ;
/// Add physical columns required by prewhere actions.
for ( const auto & column : required_columns_from_prewhere )
2022-04-18 10:18:43 +00:00
if ( ! required_aliases_from_prewhere . contains ( column ) )
2021-05-02 13:38:19 +00:00
if ( required_columns . end ( ) = = std : : find ( required_columns . begin ( ) , required_columns . end ( ) , column ) )
required_columns . push_back ( column ) ;
}
}
Fix PREWHERE for Merge with different default types
In case of underlying table has an ALIAS for this column, while in Merge
table it is not marked as an alias, there will NOT_FOUND_COLUMN_IN_BLOCK
error.
Further more, when underlying tables has different default type for the
column, i.e. one has ALIAS and another has real column, then you will
also get NOT_FOUND_COLUMN_IN_BLOCK, because Merge engine should take
care of this.
Also this patch reworks how PREWHERE is handled for Merge table, and now
if you use PREWHERE on the column that has the same type and default
type (ALIAS, ...) then it will be possible, and only if the type
differs, it will be prohibited and throw ILLEGAL_PREWHERE error.
And last, but not least, also respect this restrictions for
optimize_move_to_prewhere.
v2: introduce IStorage::supportedPrewhereColumns()
v3: Remove excessive condition for PREWHERE in StorageMerge::read()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-02-15 19:03:40 +00:00
const auto & supported_prewhere_columns = storage - > supportedPrewhereColumns ( ) ;
if ( supported_prewhere_columns . has_value ( ) )
{
NameSet required_columns_from_prewhere = get_prewhere_columns ( ) ;
for ( const auto & column_name : required_columns_from_prewhere )
{
if ( ! supported_prewhere_columns - > contains ( column_name ) )
throw Exception ( ErrorCodes : : ILLEGAL_PREWHERE , " Storage {} doesn't support PREWHERE for {} " , storage - > getName ( ) , column_name ) ;
}
}
2020-09-18 14:16:53 +00:00
}
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 )
2022-02-07 18:26:09 +00:00
& & ! settings . allow_experimental_query_deduplication
2022-07-13 04:54:52 +00:00
& & ! settings . empty_result_for_aggregation_by_empty_set
2020-06-13 20:59:20 +00:00
& & storage
2021-07-26 18:17:28 +00:00
& & storage - > getName ( ) ! = " MaterializedMySQL "
2022-07-06 10:29:29 +00:00
& & ! storage - > hasLightweightDeletedMask ( )
2022-07-28 09:40:09 +00:00
& & query_info . filter_asts . empty ( )
2020-06-13 20:59:20 +00:00
& & processing_stage = = QueryProcessingStage : : FetchColumns
& & query_analyzer - > hasAggregation ( )
& & ( query_analyzer - > aggregates ( ) . size ( ) = = 1 )
2022-01-28 17:47:37 +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
2022-01-28 17:47:37 +00:00
if ( ! query . prewhere ( ) & & ! query . where ( ) & & ! context - > getCurrentTransaction ( ) )
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 ;
2022-07-20 14:45:07 +00:00
temp_query_info . prepared_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 ) ;
2022-05-13 20:02:28 +00:00
auto prepared_count = std : : make_unique < ReadFromPreparedSource > ( Pipe ( std : : move ( source ) ) ) ;
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 .
2014-12-19 12:48:09 +00:00
*/
2015-05-11 00:11:22 +00:00
bool is_remote = false ;
2014-12-19 12:48:09 +00:00
if ( storage & & storage - > isRemote ( ) )
2015-05-11 00:11:22 +00:00
{
is_remote = true ;
2020-10-02 12:38:33 +00:00
max_threads_execute_query = max_streams = settings . max_distributed_connections ;
2015-05-11 00:11:22 +00:00
}
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
2022-10-22 16:27:24 +00:00
auto local_limits = getStorageLimits ( * context , options ) ;
2022-03-30 10:07:09 +00:00
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, JOIN, 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.
2014-12-19 12:48:09 +00:00
*/
2015-03-15 08:05:28 +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 ( )
2022-07-28 09:40:09 +00:00
& & query_info . filter_asts . empty ( )
2019-04-09 14:22:35 +00:00
& & ! query . groupBy ( )
& & ! query . having ( )
& & ! query . orderBy ( )
& & ! query . limitBy ( )
2022-03-30 10:07:09 +00:00
& & ! query . join ( )
2015-03-15 08:05:28 +00:00
& & ! query_analyzer - > hasAggregation ( )
2021-04-06 11:43:27 +00:00
& & ! query_analyzer - > hasWindow ( )
2022-03-30 10:07:09 +00:00
& & query . limitLength ( )
2022-10-22 16:27:24 +00:00
& & limit_length < = std : : numeric_limits < UInt64 > : : max ( ) - limit_offset )
2014-12-19 12:48:09 +00:00
{
2022-10-22 16:27:24 +00:00
if ( limit_length + limit_offset < max_block_size )
{
max_block_size = std : : max < UInt64 > ( 1 , limit_length + limit_offset ) ;
max_threads_execute_query = max_streams = 1 ;
}
if ( limit_length + limit_offset < local_limits . local_limits . size_limits . max_rows )
{
query_info . limit = limit_length + limit_offset ;
}
2014-12-19 12:48:09 +00:00
}
2017-04-01 07:20:54 +00:00
2019-02-10 16:55:12 +00:00
if ( ! max_block_size )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : PARAMETER_OUT_OF_BOUND , " Setting 'max_block_size' cannot be zero " ) ;
2019-02-10 16:55:12 +00:00
2022-05-31 14:43:38 +00:00
storage_limits . emplace_back ( local_limits ) ;
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 )
2014-12-19 12:48:09 +00:00
{
2018-02-21 06:25:21 +00:00
/// Subquery.
2022-03-07 13:51:34 +00:00
ASTPtr subquery = extractTableExpression ( query , 0 ) ;
if ( ! subquery )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Subquery expected " ) ;
2018-10-29 19:04:28 +00:00
2022-03-07 13:51:34 +00:00
interpreter_subquery = std : : make_unique < InterpreterSelectWithUnionQuery > (
subquery , getSubqueryContext ( context ) ,
options . copy ( ) . subquery ( ) . noModify ( ) , required_columns ) ;
2018-07-19 13:36:21 +00:00
2022-05-31 14:43:38 +00:00
interpreter_subquery - > addStorageLimits ( storage_limits ) ;
2022-03-07 13:51:34 +00:00
if ( query_analyzer - > hasAggregation ( ) )
interpreter_subquery - > ignoreWithTotals ( ) ;
2018-07-19 13:36:21 +00:00
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.
2016-06-30 20:39:35 +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.
2015-05-11 00:11:22 +00:00
if ( max_streams > 1 & & ! is_remote )
2022-09-11 01:21:34 +00:00
max_streams = static_cast < size_t > ( max_streams * settings . max_streams_to_max_threads_ratio ) ;
2017-04-01 07:20:54 +00:00
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
2022-05-25 18:59:39 +00:00
bool optimize_read_in_order = analysis_result . optimize_read_in_order ;
bool optimize_aggregation_in_order = analysis_result . optimize_aggregation_in_order & & ! query_analyzer - > useGroupingSetKey ( ) ;
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.
2022-05-25 18:59:39 +00:00
if ( ( optimize_read_in_order | | optimize_aggregation_in_order )
2021-05-02 13:38:19 +00:00
& & ( ! query_info . projection | | query_info . projection - > complete ) )
2019-12-10 23:18:24 +00:00
{
2022-05-25 18:59:39 +00:00
if ( 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?
2021-12-14 12:54:20 +00:00
query ,
2021-05-02 13:38:19 +00:00
analysis_result . order_by_elements_actions ,
getSortDescription ( query , context ) ,
query_info . syntax_analyzer_result ) ;
}
else
{
query_info . order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
2021-12-14 12:54:20 +00:00
query ,
analysis_result . order_by_elements_actions ,
getSortDescription ( query , context ) ,
query_info . syntax_analyzer_result ) ;
2021-05-02 13:38:19 +00:00
}
}
2022-08-15 16:42:50 +00:00
else if ( optimize_aggregation_in_order )
2021-05-02 13:38:19 +00:00
{
if ( query_info . projection )
{
query_info . projection - > order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
2021-12-14 12:54:20 +00:00
query ,
2021-05-02 13:38:19 +00:00
query_info . projection - > group_by_elements_actions ,
2022-02-03 12:47:27 +00:00
query_info . projection - > group_by_elements_order_descr ,
2021-05-02 13:38:19 +00:00
query_info . syntax_analyzer_result ) ;
}
else
{
query_info . order_optimizer = std : : make_shared < ReadInOrderOptimizer > (
2021-12-14 12:54:20 +00:00
query ,
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
}
2022-05-31 14:43:38 +00:00
query_info . storage_limits = std : : make_shared < StorageLimitsList > ( storage_limits ) ;
2022-01-11 12:19:41 +00:00
query_info . settings_limit_offset_done = options . settings_limit_offset_done ;
2021-07-09 03:15:41 +00:00
storage - > read ( query_plan , required_columns , storage_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-07-09 03:15:41 +00:00
auto header = storage_snapshot - > getSampleBlockForColumns ( required_columns ) ;
2021-02-10 14:12:49 +00:00
addEmptySourceToQueryPlan ( query_plan , header , query_info , context ) ;
2020-09-18 14:16:53 +00:00
}
2014-12-19 12:48:09 +00:00
}
2018-02-21 06:25:21 +00:00
else
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Logical error in InterpreterSelectQuery: nowhere to read " ) ;
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
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
2022-05-11 12:16:15 +00:00
static Aggregator : : Params getAggregatorParams (
const ASTPtr & query_ptr ,
const SelectQueryExpressionAnalyzer & query_analyzer ,
const Context & context ,
2022-06-27 11:16:52 +00:00
const Names & keys ,
2021-09-11 03:08:22 +00:00
const AggregateDescriptions & aggregates ,
2022-06-27 11:16:52 +00:00
bool overflow_row ,
const Settings & settings ,
size_t group_by_two_level_threshold ,
size_t group_by_two_level_threshold_bytes )
2019-03-26 18:28:37 +00:00
{
2022-04-14 17:29:51 +00:00
const auto stats_collecting_params = Aggregator : : Params : : StatsCollectingParams (
query_ptr ,
settings . collect_hash_table_stats_during_aggregation ,
settings . max_entries_for_hash_table_stats ,
settings . max_size_to_preallocate_for_aggregation ) ;
2022-06-27 11:16:52 +00:00
return Aggregator : : Params
{
2022-04-21 01:18:40 +00:00
keys ,
aggregates ,
overflow_row ,
settings . max_rows_to_group_by ,
settings . group_by_overflow_mode ,
group_by_two_level_threshold ,
group_by_two_level_threshold_bytes ,
settings . max_bytes_before_external_group_by ,
settings . empty_result_for_aggregation_by_empty_set
| | ( settings . empty_result_for_aggregation_by_constant_keys_on_empty_set & & keys . empty ( )
2022-05-11 12:16:15 +00:00
& & query_analyzer . hasConstAggregationKeys ( ) ) ,
2022-08-31 17:17:31 +00:00
context . getTempDataOnDisk ( ) ,
2022-04-21 01:18:40 +00:00
settings . max_threads ,
settings . min_free_disk_space_for_temporary_data ,
settings . compile_aggregate_expressions ,
settings . min_count_to_compile_aggregate_expression ,
2022-07-29 14:10:53 +00:00
settings . max_block_size ,
2022-09-21 16:59:07 +00:00
settings . enable_software_prefetch_in_aggregation ,
2022-06-27 11:16:52 +00:00
/* only_merge */ false ,
2022-05-10 16:00:00 +00:00
stats_collecting_params
2022-04-21 01:18:40 +00:00
} ;
}
2022-06-27 11:16:52 +00:00
static GroupingSetsParamsList getAggregatorGroupingSetsParams ( const SelectQueryExpressionAnalyzer & query_analyzer , const Names & all_keys )
2022-04-21 01:18:40 +00:00
{
2022-05-05 13:56:16 +00:00
GroupingSetsParamsList result ;
2022-05-11 12:16:15 +00:00
if ( query_analyzer . useGroupingSetKey ( ) )
2021-05-15 16:41:22 +00:00
{
2022-05-11 12:16:15 +00:00
auto const & aggregation_keys_list = query_analyzer . aggregationKeysList ( ) ;
2022-04-21 01:18:40 +00:00
for ( const auto & aggregation_keys : aggregation_keys_list )
2021-05-15 16:41:22 +00:00
{
2022-06-27 11:16:52 +00:00
NameSet keys ;
2021-05-15 16:41:22 +00:00
for ( const auto & key : aggregation_keys )
2022-06-27 11:16:52 +00:00
keys . insert ( key . name ) ;
2021-05-16 20:14:21 +00:00
2022-06-27 11:16:52 +00:00
Names missing_keys ;
for ( const auto & key : all_keys )
if ( ! keys . contains ( key ) )
missing_keys . push_back ( key ) ;
result . emplace_back ( aggregation_keys . getNames ( ) , std : : move ( missing_keys ) ) ;
2022-04-21 01:18:40 +00:00
}
2021-05-15 16:41:22 +00:00
}
2022-05-05 13:56:16 +00:00
return result ;
2021-09-11 03:08:22 +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-02-10 19:55:13 +00:00
AggregateDescriptions aggregates = query_analyzer - > aggregates ( ) ;
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
2022-06-27 11:16:52 +00:00
const auto & keys = query_analyzer - > aggregationKeys ( ) . getNames ( ) ;
2022-03-30 20:47:51 +00:00
2022-06-27 11:16:52 +00:00
auto aggregator_params = getAggregatorParams (
query_ptr ,
* query_analyzer ,
* context ,
keys ,
aggregates ,
overflow_row ,
settings ,
settings . group_by_two_level_threshold ,
settings . group_by_two_level_threshold_bytes ) ;
2021-09-11 03:08:22 +00:00
2022-06-27 11:16:52 +00:00
auto grouping_sets_params = getAggregatorGroupingSetsParams ( * query_analyzer , keys ) ;
2019-03-26 18:28:37 +00:00
2020-06-17 14:21:48 +00:00
SortDescription group_by_sort_description ;
2022-11-23 18:38:12 +00:00
SortDescription sort_description_for_merging ;
2019-04-17 15:35:22 +00:00
2022-05-25 18:59:39 +00:00
if ( group_by_info & & settings . optimize_aggregation_in_order & & ! query_analyzer - > useGroupingSetKey ( ) )
2022-11-23 18:38:12 +00:00
{
2021-08-03 18:03:24 +00:00
group_by_sort_description = getSortDescriptionFromGroupBy ( getSelectQuery ( ) ) ;
2022-11-23 18:38:12 +00:00
sort_description_for_merging = group_by_info - > sort_description_for_merging ;
}
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
2022-11-27 23:41:31 +00:00
if ( ! group_by_info & & settings . force_aggregation_in_order )
{
group_by_sort_description = getSortDescriptionFromGroupBy ( getSelectQuery ( ) ) ;
2023-02-14 13:02:54 +00:00
sort_description_for_merging = group_by_sort_description ;
2022-11-27 23:41:31 +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 ( ) ;
2022-11-27 23:41:31 +00:00
const bool should_produce_results_in_order_of_bucket_number = options . to_stage = = QueryProcessingStage : : WithMergeableState
& & ( settings . distributed_aggregation_memory_efficient | | settings . enable_memory_bound_merging_of_aggregation_results ) ;
2022-06-28 23:16:30 +00:00
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 ( ) ,
2022-04-21 01:18:40 +00:00
std : : move ( aggregator_params ) ,
2022-05-05 13:56:16 +00:00
std : : move ( grouping_sets_params ) ,
2021-06-03 14:05:37 +00:00
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 ,
2022-06-30 15:14:26 +00:00
settings . group_by_use_nulls ,
2022-11-23 18:38:12 +00:00
std : : move ( sort_description_for_merging ) ,
2022-06-28 23:16:30 +00:00
std : : move ( group_by_sort_description ) ,
2022-11-27 23:41:31 +00:00
should_produce_results_in_order_of_bucket_number ,
2023-02-14 13:02:54 +00:00
settings . enable_memory_bound_merging_of_aggregation_results ,
! group_by_info & & settings . force_aggregation_in_order ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( aggregating_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2022-05-05 13:56:16 +00:00
void InterpreterSelectQuery : : executeMergeAggregated ( QueryPlan & query_plan , bool overflow_row , bool final , bool has_grouping_sets )
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 ;
2019-03-26 18:28:37 +00:00
2022-11-27 23:41:31 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
/// Used to determine if we should use memory bound merging strategy.
auto group_by_sort_description
= ! query_analyzer - > useGroupingSetKey ( ) ? getSortDescriptionFromGroupBy ( getSelectQuery ( ) ) : SortDescription { } ;
2022-06-28 23:16:30 +00:00
const bool should_produce_results_in_order_of_bucket_number = options . to_stage = = QueryProcessingStage : : WithMergeableState
2022-11-27 23:41:31 +00:00
& & ( settings . distributed_aggregation_memory_efficient | | settings . enable_memory_bound_merging_of_aggregation_results ) ;
2023-02-03 13:34:18 +00:00
const bool parallel_replicas_from_merge_tree = storage - > isMergeTree ( ) & & context - > canUseParallelReplicasOnInitiator ( ) ;
2022-06-28 23:16:30 +00:00
2021-05-04 10:52:37 +00:00
executeMergeAggregatedImpl (
query_plan ,
overflow_row ,
final ,
2023-02-03 13:34:18 +00:00
storage & & ( storage - > isRemote ( ) | | parallel_replicas_from_merge_tree ) ,
2022-05-05 13:56:16 +00:00
has_grouping_sets ,
2021-05-04 10:52:37 +00:00
context - > getSettingsRef ( ) ,
query_analyzer - > aggregationKeys ( ) ,
2022-06-28 23:16:30 +00:00
query_analyzer - > aggregates ( ) ,
2022-11-27 23:41:31 +00:00
should_produce_results_in_order_of_bucket_number ,
std : : move ( group_by_sort_description ) ) ;
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 ( ) ,
2022-06-27 11:16:52 +00:00
query_analyzer - > aggregates ( ) ,
2021-06-03 14:05:37 +00:00
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
{
2021-09-06 03:05:05 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2021-12-25 17:30:31 +00:00
2022-06-27 11:16:52 +00:00
const auto & keys = query_analyzer - > aggregationKeys ( ) . getNames ( ) ;
// Arguments will not be present in Rollup / Cube input header and they don't actually needed 'cause these steps will work with AggregateFunctionState-s anyway.
auto aggregates = query_analyzer - > aggregates ( ) ;
for ( auto & aggregate : aggregates )
aggregate . argument_names . clear ( ) ;
2021-12-25 17:30:31 +00:00
2022-06-27 11:16:52 +00:00
auto params = getAggregatorParams ( query_ptr , * query_analyzer , * context , keys , aggregates , false , settings , 0 , 0 ) ;
const bool final = true ;
2019-04-05 10:52:07 +00:00
2020-06-18 17:45:00 +00:00
QueryPlanStepPtr step ;
2020-06-17 17:15:24 +00:00
if ( modificator = = Modificator : : ROLLUP )
2022-07-01 16:24:05 +00:00
step = std : : make_unique < RollupStep > ( query_plan . getCurrentDataStream ( ) , std : : move ( params ) , final , settings . group_by_use_nulls ) ;
2021-08-23 06:28:49 +00:00
else if ( modificator = = Modificator : : CUBE )
2022-07-01 16:24:05 +00:00
step = std : : make_unique < CubeStep > ( query_plan . getCurrentDataStream ( ) , std : : move ( params ) , final , settings . group_by_use_nulls ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( step ) ) ;
2019-03-26 18:28:37 +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 ] . 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 ;
2022-03-28 21:19:14 +00:00
for ( const auto & [ _ , window ] : query_analyzer - > windowDescriptions ( ) )
windows_sorted . push_back ( & window ) ;
2020-12-10 19:06:52 +00:00
2022-01-30 19:49:48 +00:00
: : 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 )
{
2022-03-28 21:19:14 +00:00
const auto & window = * 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 ()`.
2022-03-28 21:19:14 +00:00
if ( ! window . full_sort_description . empty ( ) & & ( i = = 0 | | ! sortIsPrefix ( window , * windows_sorted [ i - 1 ] ) ) )
2021-02-10 12:37:05 +00:00
{
2022-11-09 16:07:38 +00:00
SortingStep : : Settings sort_settings ( * context ) ;
2022-11-01 19:51:52 +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 ( ) ,
2022-03-28 21:19:14 +00:00
window . full_sort_description ,
2021-02-10 12:37:05 +00:00
0 /* LIMIT */ ,
2022-11-01 19:51:52 +00:00
sort_settings ,
2022-08-16 16:27:41 +00:00
settings . optimize_sorting_by_input_stream_properties ) ;
2022-03-28 21:19:14 +00:00
sorting_step - > setStepDescription ( " Sorting for window ' " + window . window_name + " ' " ) ;
2021-10-18 14:37:46 +00:00
query_plan . addStep ( std : : move ( sorting_step ) ) ;
2021-02-10 12:37:05 +00:00
}
2020-12-10 19:06:52 +00:00
2022-03-28 21:19:14 +00:00
auto window_step = std : : make_unique < WindowStep > ( query_plan . getCurrentDataStream ( ) , window , window . window_functions ) ;
window_step - > setStepDescription ( " Window step for window ' " + window . 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 ( ) ,
2022-07-01 16:43:40 +00:00
input_sorting_info - > sort_description_for_merging ,
2021-08-04 15:18:03 +00:00
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
2022-11-09 16:07:38 +00:00
SortingStep : : Settings sort_settings ( * context ) ;
2022-11-01 19:51:52 +00:00
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 ,
limit ,
2022-11-01 19:51:52 +00:00
sort_settings ,
2022-08-16 16:27:41 +00:00
settings . optimize_sorting_by_input_stream_properties ) ;
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
{
2022-08-09 22:19:25 +00:00
const auto & query = getSelectQuery ( ) ;
SortDescription sort_description = getSortDescription ( query , context ) ;
const UInt64 limit = getLimitForSorting ( query , context ) ;
const auto max_block_size = context - > getSettingsRef ( ) . max_block_size ;
2023-03-22 03:25:31 +00:00
const auto exact_rows_before_limit = context - > getSettingsRef ( ) . exact_rows_before_limit ;
2019-03-26 18:28:37 +00:00
2023-03-22 03:25:31 +00:00
auto merging_sorted = std : : make_unique < SortingStep > (
query_plan . getCurrentDataStream ( ) , std : : move ( sort_description ) , max_block_size , limit , exact_rows_before_limit ) ;
2020-06-18 17:45:00 +00:00
merging_sorted - > setStepDescription ( " Merge sorted streams " + description ) ;
query_plan . addStep ( std : : move ( merging_sorted ) ) ;
2019-03-26 18:28:37 +00:00
}
2015-01-18 08:27:28 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery : : executeProjection ( QueryPlan & query_plan , const ActionsDAGPtr & expression )
2019-03-26 18:28:37 +00:00
{
2020-06-18 17:45:00 +00:00
auto projection_step = std : : make_unique < ExpressionStep > ( query_plan . getCurrentDataStream ( ) , expression ) ;
projection_step - > setStepDescription ( " Projection " ) ;
query_plan . addStep ( std : : move ( projection_step ) ) ;
2019-03-26 18:28:37 +00:00
}
2012-02-27 06:28:20 +00:00
2022-06-21 15:06:47 +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
UInt64 limit_for_distinct = 0 ;
2022-11-20 22:48:16 +00:00
/// If after this stage of DISTINCT,
/// (1) ORDER BY is not executed
/// (2) there is no LIMIT BY (todo: we can check if DISTINCT and LIMIT BY expressions are match)
2020-07-12 05:18:01 +00:00
/// then you can get no more than limit_length + limit_offset of different rows.
2022-11-20 22:48:16 +00:00
if ( ( ! query . orderBy ( ) | | ! before_order ) & & ! query . limitBy ( ) )
{
auto [ limit_length , limit_offset ] = getLimitLengthAndOffset ( query , context ) ;
if ( limit_length < = std : : numeric_limits < UInt64 > : : max ( ) - limit_offset )
limit_for_distinct = limit_length + limit_offset ;
}
2019-03-26 18:28:37 +00:00
SizeLimits limits ( settings . max_rows_in_distinct , settings . max_bytes_in_distinct , settings . distinct_overflow_mode ) ;
2022-05-30 22:17:46 +00:00
auto distinct_step = std : : make_unique < DistinctStep > (
2022-06-29 23:24:20 +00:00
query_plan . getCurrentDataStream ( ) ,
limits ,
limit_for_distinct ,
columns ,
pre_distinct ,
2022-06-30 13:03:39 +00:00
settings . optimize_distinct_in_order ) ;
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
2021-06-16 12:59:02 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
2021-12-23 08:35:23 +00:00
2021-06-16 12:59:02 +00:00
auto limit = std : : make_unique < LimitStep > ( query_plan . getCurrentDataStream ( ) , limit_length , limit_offset , settings . exact_rows_before_limit ) ;
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) " ) ;
2021-12-23 08:35:23 +00:00
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 ;
2022-04-01 20:18:19 +00:00
InterpolateDescriptionPtr interpolate_descr =
2022-04-05 18:26:49 +00:00
getInterpolateDescription ( query , source_header , result_header , syntax_analyzer_result - > aliases , context ) ;
2022-03-24 19:29:29 +00:00
auto filling_step = std : : make_unique < FillingStep > ( query_plan . getCurrentDataStream ( ) , std : : move ( fill_descr ) , interpolate_descr ) ;
2020-06-18 17:45:00 +00:00
query_plan . addStep ( std : : move ( filling_step ) ) ;
2014-12-19 12:48:09 +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 .
*/
2021-06-16 12:59:02 +00:00
const Settings & settings = context - > getSettingsRef ( ) ;
bool always_read_till_end = settings . exact_rows_before_limit ;
2019-03-26 18:28:37 +00:00
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 ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " LIMIT WITH TIES without ORDER BY " ) ;
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
}
2022-07-18 15:53:30 +00:00
void InterpreterSelectQuery : : executeSubqueriesInSetsAndJoins ( QueryPlan & query_plan )
2019-03-26 18:28:37 +00:00
{
2022-07-27 11:22:16 +00:00
addCreatingSetsStep ( query_plan , prepared_sets , 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
}
2023-01-31 18:21:00 +00:00
bool InterpreterSelectQuery : : autoFinalOnQuery ( ASTSelectQuery & query )
2022-12-05 13:18:44 +00:00
{
// query.tables() is required because not all queries have tables in it, it could be a function.
2023-01-31 18:21:00 +00:00
bool is_auto_final_setting_on = context - > getSettingsRef ( ) . final ;
2023-01-31 12:13:57 +00:00
bool is_final_supported = storage & & storage - > supportsFinal ( ) & & ! storage - > isRemote ( ) & & query . tables ( ) ;
bool is_query_already_final = query . final ( ) ;
2022-12-05 13:18:44 +00:00
2023-01-31 18:21:00 +00:00
return is_auto_final_setting_on & & ! is_query_already_final & & is_final_supported ;
2022-12-05 13:18:44 +00:00
}
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
}