ClickHouse/src/Interpreters/InterpreterSelectQuery.cpp

2372 lines
100 KiB
C++
Raw Normal View History

2019-11-05 20:22:20 +00:00
#include <DataStreams/OneBlockInputStream.h>
2020-06-17 20:19:55 +00:00
#include <DataStreams/materializeBlock.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
2019-05-03 02:00:57 +00:00
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Access/AccessFlags.h>
#include <Access/ContextAccess.h>
#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>
#include <Interpreters/InterpreterSelectQuery.h>
2018-02-25 06:34:20 +00:00
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/evaluateConstantExpression.h>
2019-02-10 15:17:45 +00:00
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/addTypeConversionToAST.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/TableJoin.h>
2020-09-18 16:25:20 +00:00
#include <Interpreters/JoinSwitcher.h>
2020-02-28 15:23:32 +00:00
#include <Interpreters/JoinedTables.h>
2020-11-19 15:52:11 +00:00
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <Interpreters/QueryAliasesVisitor.h>
2020-12-12 16:42:15 +00:00
#include <Interpreters/replaceAliasColumnsInQuery.h>
2020-06-17 20:19:55 +00:00
#include <Processors/Pipe.h>
#include <Processors/QueryPlan/AggregatingStep.h>
#include <Processors/QueryPlan/ArrayJoinStep.h>
#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>
#include <Processors/QueryPlan/ExtremesStep.h>
#include <Processors/QueryPlan/FillingStep.h>
2020-06-16 09:42:25 +00:00
#include <Processors/QueryPlan/FilterStep.h>
#include <Processors/QueryPlan/FinishSortingStep.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>
#include <Processors/QueryPlan/MergeSortingStep.h>
2020-06-16 18:57:21 +00:00
#include <Processors/QueryPlan/MergingAggregatedStep.h>
#include <Processors/QueryPlan/MergingSortedStep.h>
2020-06-24 12:09:01 +00:00
#include <Processors/QueryPlan/OffsetStep.h>
#include <Processors/QueryPlan/PartialSortingStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/QueryPlan/ReadNothingStep.h>
#include <Processors/QueryPlan/RollupStep.h>
#include <Processors/QueryPlan/SettingQuotaAndLimitsStep.h>
#include <Processors/QueryPlan/TotalsHavingStep.h>
#include <Processors/QueryPlan/WindowStep.h>
2021-03-04 17:38:12 +00:00
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Transforms/AggregatingTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/JoiningTransform.h>
2019-03-26 18:28:37 +00:00
2020-06-17 20:19:55 +00:00
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
#include <Storages/IStorage.h>
#include <Storages/StorageView.h>
#include <Functions/IFunction.h>
#include <Core/Field.h>
2020-09-15 09:55:57 +00:00
#include <common/types.h>
2020-06-17 20:19:55 +00:00
#include <Columns/Collator.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <ext/map.h>
#include <ext/scope_guard_safe.h>
2020-06-17 20:19:55 +00:00
#include <memory>
2011-08-28 05:13:24 +00:00
namespace DB
{
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;
extern const int NOT_IMPLEMENTED;
extern const int PARAMETER_OUT_OF_BOUND;
extern const int INVALID_LIMIT_EXPRESSION;
extern const int INVALID_WITH_FILL_EXPRESSION;
extern const int ACCESS_DENIED;
}
2019-10-05 19:34:25 +00:00
/// Assumes `storage` is set and the table filter (row-level security) is not empty.
2021-02-16 09:37:19 +00:00
String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, const Names & prerequisite_columns) const
{
2019-12-30 18:20:43 +00:00
const auto & db_name = table_id.getDatabaseName();
const auto & table_name = table_id.getTableName();
/// TODO: implement some AST builders for this kind of stuff
ASTPtr query_ast = std::make_shared<ASTSelectQuery>();
auto * select_ast = query_ast->as<ASTSelectQuery>();
select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
auto expr_list = select_ast->select();
// The first column is our filter expression.
expr_list->children.push_back(row_policy_filter);
/// Keep columns that are required after the filter actions.
for (const auto & column_str : prerequisite_columns)
{
ParserExpression expr_parser;
expr_list->children.push_back(parseQuery(expr_parser, column_str, 0, context->getSettingsRef().max_parser_depth));
}
select_ast->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared<ASTTablesInSelectQuery>());
2019-04-09 14:59:06 +00:00
auto tables = select_ast->tables();
auto tables_elem = std::make_shared<ASTTablesInSelectQueryElement>();
auto table_expr = std::make_shared<ASTTableExpression>();
tables->children.push_back(tables_elem);
tables_elem->table_expression = table_expr;
tables_elem->children.push_back(table_expr);
table_expr->database_and_table_name = createTableIdentifier(db_name, table_name);
table_expr->children.push_back(table_expr->database_and_table_name);
/// Using separate expression analyzer to prevent any possible alias injection
auto syntax_result = TreeRewriter(*context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, metadata_snapshot));
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context, metadata_snapshot);
actions = analyzer.simpleSelectActions();
2021-02-20 11:00:16 +00:00
auto column_name = expr_list->children.at(0)->getColumnName();
2021-03-10 08:41:24 +00:00
actions->removeUnusedActions(NameSet{column_name});
2021-02-20 11:00:16 +00:00
actions->projectInput(false);
for (const auto * node : actions->getInputs())
actions->getIndex().push_back(node);
2021-02-20 11:00:16 +00:00
return column_name;
}
2018-02-26 21:00:42 +00:00
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
2019-08-03 11:02:40 +00:00
const SelectQueryOptions & options_,
const Names & required_result_column_names_)
: InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, nullptr, options_, required_result_column_names_)
2018-02-26 21:00:42 +00:00
{
}
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
const BlockInputStreamPtr & input_,
2019-08-03 11:02:40 +00:00
const SelectQueryOptions & options_)
: InterpreterSelectQuery(query_ptr_, context_, input_, std::nullopt, nullptr, options_.copy().noSubquery())
{}
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
Pipe input_pipe_,
const SelectQueryOptions & options_)
: InterpreterSelectQuery(query_ptr_, context_, nullptr, std::move(input_pipe_), nullptr, options_.copy().noSubquery())
2019-03-15 15:57:18 +00:00
{}
2018-02-26 21:00:42 +00:00
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
const StoragePtr & storage_,
2020-06-17 11:52:19 +00:00
const StorageMetadataPtr & metadata_snapshot_,
2019-08-03 11:02:40 +00:00
const SelectQueryOptions & options_)
2020-06-17 11:52:19 +00:00
: InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_)
2019-03-15 15:57:18 +00:00
{}
InterpreterSelectQuery::~InterpreterSelectQuery() = default;
/** There are no limits on the maximum size of the result for the subquery.
* Since the result of the query is not the result of the entire query.
*/
static Context getSubqueryContext(const Context & context)
2018-02-26 21:00:42 +00:00
{
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.max_result_rows = 0;
subquery_settings.max_result_bytes = 0;
/// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query).
2020-03-09 00:08:02 +00:00
subquery_settings.extremes = false;
subquery_context.setSettings(subquery_settings);
return subquery_context;
2018-02-26 21:00:42 +00:00
}
2021-02-15 19:40:32 +00:00
static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings)
{
ASTSelectQuery & select = query->as<ASTSelectQuery &>();
Aliases aliases;
if (ASTPtr with = select.with())
QueryAliasesNoSubqueriesVisitor(aliases).visit(with);
QueryAliasesNoSubqueriesVisitor(aliases).visit(select.select());
CrossToInnerJoinVisitor::Data cross_to_inner{tables, aliases, database};
2021-02-15 19:40:32 +00:00
cross_to_inner.cross_to_inner_join_rewrite = settings.cross_to_inner_join_rewrite;
CrossToInnerJoinVisitor(cross_to_inner).visit(query);
JoinToSubqueryTransformVisitor::Data join_to_subs_data{tables, aliases};
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query);
}
/// Checks that the current user has the SELECT privilege.
static void checkAccessRightsForSelect(
const Context & context,
const StorageID & table_id,
const StorageMetadataPtr & table_metadata,
const Strings & required_columns,
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.
auto access = context.getAccess();
for (const auto & column : table_metadata->getColumns())
{
if (access->isGranted(AccessType::SELECT, table_id.database_name, table_id.table_name, column.name))
return;
}
2021-01-16 21:11:58 +00:00
throw Exception(context.getUserName() + ": Not enough privileges. "
"To execute this query it's necessary to have grant SELECT for at least one column on " + table_id.getFullTableName(),
ErrorCodes::ACCESS_DENIED);
}
/// General check.
context.checkAccess(AccessType::SELECT, table_id, required_columns);
}
/// Returns true if we should ignore quotas and limits for a specified table in the system database.
static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id)
{
if (table_id.database_name == DatabaseCatalog::SYSTEM_DATABASE)
{
static const boost::container::flat_set<String> tables_ignoring_quota{"quotas", "quota_limits", "quota_usage", "quotas_usage", "one"};
if (tables_ignoring_quota.count(table_id.table_name))
return true;
}
return false;
}
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
const BlockInputStreamPtr & input_,
std::optional<Pipe> input_pipe_,
const StoragePtr & storage_,
2019-03-18 12:05:51 +00:00
const SelectQueryOptions & options_,
2020-06-17 11:52:19 +00:00
const Names & required_result_column_names,
const StorageMetadataPtr & metadata_snapshot_)
/// 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_)
, storage(storage_)
, input(input_)
, 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_)
{
2019-08-10 17:51:47 +00:00
checkStackSize();
initSettings();
const Settings & settings = context->getSettingsRef();
2019-03-18 12:05:51 +00:00
if (settings.max_subquery_depth && options.subquery_depth > settings.max_subquery_depth)
throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(),
ErrorCodes::TOO_DEEP_SUBQUERIES);
bool has_input = input || input_pipe;
if (input)
{
2018-02-28 01:29:55 +00:00
/// Read from prepared input.
source_header = input->getHeader();
}
else if (input_pipe)
{
/// Read from prepared input.
2020-02-28 15:11:18 +00:00
source_header = input_pipe->getHeader();
}
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
JoinedTables joined_tables(getSubqueryContext(*context), getSelectQuery());
bool got_storage_from_query = false;
if (!has_input && !storage)
{
storage = joined_tables.getLeftTableStorage();
got_storage_from_query = true;
}
if (storage)
{
2020-06-18 16:10:47 +00:00
table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
table_id = storage->getStorageID();
if (!metadata_snapshot)
2020-06-17 11:52:19 +00:00
metadata_snapshot = storage->getInMemoryMetadataPtr();
2018-02-28 01:29:55 +00:00
}
if (has_input || !joined_tables.resolveTables())
joined_tables.makeFakeTable(storage, metadata_snapshot, source_header);
/// 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());
joined_tables.reset(getSelectQuery());
joined_tables.resolveTables();
if (storage && joined_tables.isLeftTableSubquery())
{
/// Rewritten with subquery. Free storage locks here.
storage = nullptr;
2020-06-18 16:10:47 +00:00
table_lock.reset();
table_id = StorageID::createEmpty();
metadata_snapshot = nullptr;
}
}
if (!has_input)
2019-12-05 11:42:13 +00:00
{
interpreter_subquery = joined_tables.makeLeftTableSubquery(options.subquery());
if (interpreter_subquery)
source_header = interpreter_subquery->getSampleBlock();
2020-03-02 19:39:39 +00:00
}
joined_tables.rewriteDistributedInAndJoins(query_ptr);
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
if (storage)
row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
StorageView * view = nullptr;
if (storage)
view = dynamic_cast<StorageView *>(storage.get());
2020-10-13 14:33:34 +00:00
SubqueriesForSets subquery_for_sets;
auto analyze = [&] (bool try_move_to_prewhere)
2019-10-27 18:12:40 +00:00
{
/// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it.
ASTPtr view_table;
if (view)
2020-06-17 14:06:22 +00:00
view->replaceWithSubquery(getSelectQuery(), view_table, metadata_snapshot);
syntax_analyzer_result = TreeRewriter(*context).analyzeSelect(
query_ptr,
TreeRewriterResult(source_header.getNamesAndTypesList(), storage, metadata_snapshot),
options, joined_tables.tablesWithColumns(), required_result_column_names, table_join);
2019-10-19 20:36:35 +00:00
2020-06-18 16:11:23 +00:00
/// Save scalar sub queries's results in the query context
if (!options.only_analyze && context->hasQueryContext())
for (const auto & it : syntax_analyzer_result->getScalars())
context->getQueryContext().addScalar(it.first, it.second);
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;
}
if (try_move_to_prewhere && storage && query.where() && !query.prewhere())
{
/// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable
if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty())
{
/// 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;
SelectQueryInfo current_info;
current_info.query = query_ptr;
current_info.syntax_analyzer_result = syntax_analyzer_result;
MergeTreeWhereOptimizer{current_info, *context, std::move(column_compressed_sizes), metadata_snapshot, syntax_analyzer_result->requiredSourceColumns(), log};
}
}
2019-10-27 18:12:40 +00:00
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
query_ptr, syntax_analyzer_result, *context, metadata_snapshot,
2019-10-27 18:12:40 +00:00
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
2020-10-13 14:55:22 +00:00
!options.only_analyze, options, std::move(subquery_for_sets));
2019-10-27 18:12:40 +00:00
if (!options.only_analyze)
{
if (query.sampleSize() && (input || input_pipe || !storage || !storage->supportsSampling()))
2019-10-27 18:12:40 +00:00
throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED);
if (query.final() && (input || input_pipe || !storage || !storage->supportsFinal()))
throw Exception((!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL);
if (query.prewhere() && (input || input_pipe || !storage || !storage->supportsPrewhere()))
throw Exception((!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE);
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
}
2019-10-27 18:12:40 +00:00
if (!options.only_analyze || options.modify_inplace)
{
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
interpreter_subquery = joined_tables.makeLeftTableSubquery(options.subquery());
2019-10-27 18:12:40 +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();
}
2019-10-27 18:12:40 +00:00
required_columns = syntax_analyzer_result->requiredSourceColumns();
2019-10-27 18:12:40 +00:00
if (storage)
{
2019-10-27 18:12:40 +00:00
/// Fix source_header for filter actions.
if (row_policy_filter)
2019-10-27 18:12:40 +00:00
{
filter_info = std::make_shared<FilterDAGInfo>();
2021-02-16 09:37:19 +00:00
filter_info->column_name = generateFilterActions(filter_info->actions, required_columns);
2021-02-20 11:00:16 +00:00
auto required_columns_from_filter = filter_info->actions->getRequiredColumns();
for (const auto & column : required_columns_from_filter)
{
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name))
required_columns.push_back(column.name);
}
2019-10-27 18:12:40 +00:00
}
2021-02-20 11:00:16 +00:00
source_header = metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID());
}
2019-11-03 07:27:35 +00:00
2019-10-27 18:12:40 +00:00
/// Calculate structure of the result.
result_header = getSampleBlockImpl();
2019-10-27 18:12:40 +00:00
};
analyze(settings.optimize_move_to_prewhere);
2019-10-27 18:12:40 +00:00
bool need_analyze_again = false;
if (analysis_result.prewhere_constant_filter_description.always_false || analysis_result.prewhere_constant_filter_description.always_true)
{
2019-10-27 18:12:40 +00:00
if (analysis_result.prewhere_constant_filter_description.always_true)
2019-11-03 07:27:35 +00:00
query.setExpression(ASTSelectQuery::Expression::PREWHERE, {});
else
query.setExpression(ASTSelectQuery::Expression::PREWHERE, std::make_shared<ASTLiteral>(0u));
2019-10-27 18:12:40 +00:00
need_analyze_again = true;
}
if (analysis_result.where_constant_filter_description.always_false || analysis_result.where_constant_filter_description.always_true)
{
if (analysis_result.where_constant_filter_description.always_true)
2019-11-03 07:27:35 +00:00
query.setExpression(ASTSelectQuery::Expression::WHERE, {});
else
query.setExpression(ASTSelectQuery::Expression::WHERE, std::make_shared<ASTLiteral>(0u));
2019-10-27 18:12:40 +00:00
need_analyze_again = true;
}
if (query.prewhere() && query.where())
{
/// Filter block in WHERE instead to get better performance
query.setExpression(ASTSelectQuery::Expression::WHERE, makeASTFunction("and", query.prewhere()->clone(), query.where()->clone()));
need_analyze_again = true;
}
2020-02-05 16:42:27 +00:00
2019-10-27 18:12:40 +00:00
if (need_analyze_again)
2020-02-05 16:42:27 +00:00
{
LOG_TRACE(log, "Running 'analyze' second time");
query_analyzer->getSubqueriesForSets().clear();
subquery_for_sets = SubqueriesForSets();
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
/// If there is no WHERE, filter blocks as usual
if (query.prewhere() && !query.where())
analysis_result.prewhere_info->need_filter = true;
if (table_id && got_storage_from_query && !joined_tables.isLeftTableFunction())
{
/// 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().
checkAccessRightsForSelect(*context, table_id, metadata_snapshot, required_columns, *syntax_analyzer_result);
/// Remove limits for some tables in the `system` database.
if (shouldIgnoreQuotaAndLimits(table_id) && (joined_tables.tablesCount() <= 1))
{
options.ignore_quota = true;
options.ignore_limits = true;
}
}
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);
}
void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan)
{
executeImpl(query_plan, input, std::move(input_pipe));
/// We must guarantee that result structure is the same as in getSampleBlock()
if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header))
{
2020-11-17 17:16:55 +00:00
auto convert_actions_dag = ActionsDAG::makeConvertingActions(
query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(),
result_header.getColumnsWithTypeAndName(),
ActionsDAG::MatchColumnsMode::Name,
true);
auto converting = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), convert_actions_dag);
query_plan.addStep(std::move(converting));
}
}
2015-06-18 02:11:05 +00:00
BlockIO InterpreterSelectQuery::execute()
{
BlockIO res;
QueryPlan query_plan;
buildQueryPlan(query_plan);
2021-03-04 17:38:12 +00:00
res.pipeline = std::move(*query_plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(*context),
BuildQueryPipelineSettings::fromContext(*context)));
return res;
2014-12-16 10:39:02 +00:00
}
Block InterpreterSelectQuery::getSampleBlockImpl()
{
2020-11-19 15:52:11 +00:00
OpenTelemetrySpanHolder span(__PRETTY_FUNCTION__);
query_info.query = query_ptr;
2019-10-03 15:47:42 +00:00
if (storage && !options.only_analyze)
from_stage = storage->getQueryProcessingStage(*context, options.to_stage, query_info);
2019-10-03 15:47:42 +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).
bool first_stage = from_stage < QueryProcessingStage::WithMergeableState
&& options.to_stage >= QueryProcessingStage::WithMergeableState;
/// Do I need to execute the second part of the pipeline?
/// Running on the initiating server during distributed processing 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)
/// the query on the remote server will be processed up to WithMergeableStateAfterAggregation,
/// So it will do partial second stage (second_stage=true), and initiator will do the final part.
bool second_stage = from_stage <= QueryProcessingStage::WithMergeableState
&& options.to_stage > QueryProcessingStage::WithMergeableState;
analysis_result = ExpressionAnalysisResult(
*query_analyzer,
2020-06-17 11:05:11 +00:00
metadata_snapshot,
first_stage,
second_stage,
options.only_analyze,
2019-10-27 18:12:40 +00:00
filter_info,
source_header);
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
{
auto header = source_header;
if (analysis_result.prewhere_info)
{
2021-03-04 17:38:12 +00:00
ExpressionActions(
analysis_result.prewhere_info->prewhere_actions,
ExpressionActionsSettings::fromContext(*context)).execute(header);
if (analysis_result.prewhere_info->remove_prewhere_column)
header.erase(analysis_result.prewhere_info->prewhere_column_name);
}
return header;
}
if (options.to_stage == QueryProcessingStage::Enum::WithMergeableState)
{
if (!analysis_result.need_aggregate)
2020-12-25 03:13:30 +00:00
{
// What's the difference with selected_columns?
return analysis_result.before_order_by->getResultColumns();
}
2020-11-03 11:28:28 +00:00
Block header = analysis_result.before_aggregation->getResultColumns();
Block res;
2020-04-22 06:01:33 +00:00
for (const auto & key : query_analyzer->aggregationKeys())
res.insert({nullptr, header.getByName(key.name).type, key.name});
2020-04-22 06:01:33 +00:00
for (const auto & aggregate : query_analyzer->aggregates())
{
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;
}
if (options.to_stage == QueryProcessingStage::Enum::WithMergeableStateAfterAggregation)
{
2020-12-25 03:13:30 +00:00
// What's the difference with selected_columns?
return analysis_result.before_order_by->getResultColumns();
}
2020-11-03 11:28:28 +00:00
return analysis_result.final_projection->getResultColumns();
}
2019-08-14 17:01:47 +00:00
static Field getWithFillFieldValue(const ASTPtr & node, const Context & context)
{
const auto & [field, type] = evaluateConstantExpression(node, context);
if (!isColumnedAsNumber(type))
2019-08-14 17:01:47 +00:00
throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
2018-02-23 06:00:48 +00:00
2019-08-14 17:01:47 +00:00
return field;
}
2018-02-23 06:00:48 +00:00
static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, const Context & context)
2019-08-14 17:01:47 +00:00
{
FillColumnDescription descr;
if (order_by_elem.fill_from)
descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context);
if (order_by_elem.fill_to)
descr.fill_to = getWithFillFieldValue(order_by_elem.fill_to, context);
if (order_by_elem.fill_step)
descr.fill_step = getWithFillFieldValue(order_by_elem.fill_step, context);
2019-08-14 17:01:47 +00:00
else
descr.fill_step = order_by_elem.direction;
2019-08-14 17:01:47 +00:00
if (applyVisitor(FieldVisitorAccurateEquals(), descr.fill_step, Field{0}))
throw Exception("WITH FILL STEP value cannot be zero", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (order_by_elem.direction == 1)
{
if (applyVisitor(FieldVisitorAccurateLess(), descr.fill_step, Field{0}))
throw Exception("WITH FILL STEP value cannot be negative for sorting in ascending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (!descr.fill_from.isNull() && !descr.fill_to.isNull() &&
applyVisitor(FieldVisitorAccurateLess(), descr.fill_to, descr.fill_from))
{
throw Exception("WITH FILL TO value cannot be less than FROM value for sorting in ascending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
}
else
{
if (applyVisitor(FieldVisitorAccurateLess(), Field{0}, descr.fill_step))
throw Exception("WITH FILL STEP value cannot be positive for sorting in descending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (!descr.fill_from.isNull() && !descr.fill_to.isNull() &&
applyVisitor(FieldVisitorAccurateLess(), descr.fill_from, descr.fill_to))
{
throw Exception("WITH FILL FROM value cannot be less than TO value for sorting in descending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
}
2019-08-14 17:01:47 +00:00
return descr;
}
2018-02-23 06:00:48 +00:00
2019-08-14 17:01:47 +00:00
static SortDescription getSortDescription(const ASTSelectQuery & query, const Context & context)
{
SortDescription order_descr;
order_descr.reserve(query.orderBy()->children.size());
for (const auto & elem : query.orderBy()->children)
{
String name = elem->children.front()->getColumnName();
const auto & order_by_elem = elem->as<ASTOrderByElement &>();
std::shared_ptr<Collator> collator;
if (order_by_elem.collation)
collator = std::make_shared<Collator>(order_by_elem.collation->as<ASTLiteral &>().value.get<String>());
2019-08-14 17:01:47 +00:00
if (order_by_elem.with_fill)
{
FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context);
order_descr.emplace_back(name, order_by_elem.direction,
2020-09-19 14:20:00 +00:00
order_by_elem.nulls_direction, collator, true, fill_desc);
2019-08-14 17:01:47 +00:00
}
else
2020-09-19 14:20:00 +00:00
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator);
}
return order_descr;
}
2020-05-14 14:20:49 +00:00
static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query)
2020-02-14 07:12:04 +00:00
{
2020-02-16 19:46:45 +00:00
SortDescription order_descr;
order_descr.reserve(query.groupBy()->children.size());
2020-02-14 07:12:04 +00:00
for (const auto & elem : query.groupBy()->children)
{
String name = elem->getColumnName();
2020-02-16 19:46:45 +00:00
order_descr.emplace_back(name, 1, 1);
2020-02-14 07:12:04 +00:00
}
2020-02-16 19:46:45 +00:00
return order_descr;
2020-02-14 07:12:04 +00:00
}
2020-05-07 13:40:50 +00:00
static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context, const std::string & expr)
{
const auto & [field, type] = evaluateConstantExpression(node, context);
if (!isNativeNumber(type))
2020-05-07 13:40:50 +00:00
throw Exception("Illegal type " + type->getName() + " of " + expr + " expression, must be numeric type", ErrorCodes::INVALID_LIMIT_EXPRESSION);
Field converted = convertFieldToType(field, DataTypeUInt64());
if (converted.isNull())
2020-05-07 13:40:50 +00:00
throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of " + expr + " expression is not representable as UInt64", ErrorCodes::INVALID_LIMIT_EXPRESSION);
return converted.safeGet<UInt64>();
}
static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery & query, const Context & context)
{
UInt64 length = 0;
UInt64 offset = 0;
if (query.limitLength())
{
2020-05-07 13:40:50 +00:00
length = getLimitUIntValue(query.limitLength(), context, "LIMIT");
2019-09-17 18:55:59 +00:00
if (query.limitOffset() && length)
2020-05-11 12:18:07 +00:00
offset = getLimitUIntValue(query.limitOffset(), context, "OFFSET");
}
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");
return {length, offset};
}
static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context)
{
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY, neither ARRAY JOIN.
2020-05-11 02:44:43 +00:00
if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList() && query.limitLength())
{
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;
return limit_length + limit_offset;
}
return 0;
}
2020-06-19 16:51:44 +00:00
static bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query)
{
if (query.group_by_with_totals)
return true;
/** NOTE You can also check that the table in the subquery is distributed, and that it only looks at one shard.
* In other cases, totals will be computed on the initiating server of the query, and it is not necessary to read the data to the end.
*/
if (auto query_table = extractTableExpression(query, 0))
{
if (const auto * ast_union = query_table->as<ASTSelectWithUnionQuery>())
{
2021-02-17 18:23:27 +00:00
/// NOTE: Child of subquery can be ASTSelectWithUnionQuery or ASTSelectQuery,
2021-02-17 08:26:52 +00:00
/// and after normalization, the height of the AST tree is at most 2
2020-06-19 16:51:44 +00:00
for (const auto & elem : ast_union->list_of_selects->children)
2021-02-17 08:26:52 +00:00
{
if (const auto * child_union = elem->as<ASTSelectWithUnionQuery>())
{
for (const auto & child_elem : child_union->list_of_selects->children)
if (hasWithTotalsInAnySubqueryInFromClause(child_elem->as<ASTSelectQuery &>()))
return true;
}
else
{
if (hasWithTotalsInAnySubqueryInFromClause(elem->as<ASTSelectQuery &>()))
return true;
}
}
2020-06-19 16:51:44 +00:00
}
}
return false;
}
void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInputStreamPtr & prepared_input, std::optional<Pipe> prepared_pipe)
2011-08-28 05:13:24 +00:00
{
2017-04-02 17:37:49 +00:00
/** Streams of data. When the query is executed in parallel, we have several data streams.
* If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then
2020-06-07 21:05:36 +00:00
* if there is an ORDER BY, then glue the streams using ResizeProcessor, and then MergeSorting transforms,
* if not, then glue it using ResizeProcessor,
2017-04-02 17:37:49 +00:00
* then apply LIMIT.
* If there is GROUP BY, then we will perform all operations up to GROUP BY, inclusive, in parallel;
* a parallel GROUP BY will glue streams into one,
* then perform the remaining operations with one resulting stream.
*/
2019-03-12 14:07:02 +00:00
/// Now we will compose block streams that perform the necessary actions.
auto & query = getSelectQuery();
const Settings & settings = context->getSettingsRef();
auto & expressions = analysis_result;
auto & subqueries_for_sets = query_analyzer->getSubqueriesForSets();
bool intermediate_stage = false;
bool to_aggregation_stage = false;
bool from_aggregation_stage = false;
2021-02-15 19:48:06 +00:00
if (expressions.filter_info)
{
if (!expressions.prewhere_info)
{
const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere();
if (does_storage_support_prewhere && settings.optimize_move_to_prewhere)
{
/// Execute row level filter in prewhere as a part of "move to prewhere" optimization.
expressions.prewhere_info = std::make_shared<PrewhereDAGInfo>(
std::move(expressions.filter_info->actions),
std::move(expressions.filter_info->column_name));
2021-02-20 11:00:16 +00:00
expressions.prewhere_info->prewhere_actions->projectInput(false);
2021-02-15 19:48:06 +00:00
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.
expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions);
expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name);
expressions.prewhere_info->row_level_filter_actions->projectInput(false);
expressions.filter_info = nullptr;
}
}
if (options.only_analyze)
{
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(),
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-02-15 19:48:06 +00:00
if (expressions.prewhere_info->row_level_filter_actions)
{
auto row_level_filter_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.prewhere_info->row_level_filter_actions,
expressions.prewhere_info->row_level_column_name,
false);
row_level_filter_step->setStepDescription("Row-level security filter (PREWHERE)");
query_plan.addStep(std::move(row_level_filter_step));
}
auto prewhere_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
2020-06-16 09:42:25 +00:00
expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_column_name,
expressions.prewhere_info->remove_prewhere_column);
prewhere_step->setStepDescription("PREWHERE");
query_plan.addStep(std::move(prewhere_step));
// To remove additional columns in dry run
// For example, sample column which can be removed in this stage
if (expressions.prewhere_info->remove_columns_actions)
{
auto remove_columns = std::make_unique<ExpressionStep>(
query_plan.getCurrentDataStream(),
2020-06-16 09:42:25 +00:00
expressions.prewhere_info->remove_columns_actions);
remove_columns->setStepDescription("Remove unnecessary columns after PREWHERE");
query_plan.addStep(std::move(remove_columns));
}
2019-04-03 11:21:38 +00:00
}
}
else
{
2018-08-27 17:42:13 +00:00
if (prepared_input)
2019-04-03 11:21:38 +00:00
{
auto prepared_source_step = std::make_unique<ReadFromPreparedSource>(
Pipe(std::make_shared<SourceFromInputStream>(prepared_input)), context);
query_plan.addStep(std::move(prepared_source_step));
2019-04-03 11:21:38 +00:00
}
else if (prepared_pipe)
{
auto prepared_source_step = std::make_unique<ReadFromPreparedSource>(std::move(*prepared_pipe), context);
query_plan.addStep(std::move(prepared_source_step));
}
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)
intermediate_stage = true;
/// Support optimize_distributed_group_by_sharding_key
/// Is running on the initiating server during distributed processing?
if (from_stage == QueryProcessingStage::WithMergeableStateAfterAggregation)
from_aggregation_stage = true;
/// Is running on remote servers during distributed processing?
if (options.to_stage == QueryProcessingStage::WithMergeableStateAfterAggregation)
to_aggregation_stage = true;
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);
2020-05-23 22:24:01 +00:00
LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage));
}
2018-02-23 06:00:48 +00:00
2019-03-18 12:05:51 +00:00
if (options.to_stage > QueryProcessingStage::FetchColumns)
{
2017-04-02 17:37:49 +00:00
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
bool aggregate_overflow_row =
2018-02-23 06:00:48 +00:00
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;
2017-04-02 17:37:49 +00:00
/// Do I need to immediately finalize the aggregate functions after the aggregation?
bool aggregate_final =
2018-02-23 06:00:48 +00:00
expressions.need_aggregate &&
2019-03-18 12:05:51 +00:00
options.to_stage > QueryProcessingStage::WithMergeableState &&
2018-09-17 18:01:04 +00:00
!query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube;
auto preliminary_sort = [&]()
{
/** For distributed query processing,
* if no GROUP, HAVING set,
* but there is an ORDER or LIMIT,
* then we will perform the preliminary sorting and LIMIT on the remote server.
*/
if (!expressions.second_stage && !expressions.need_aggregate && !expressions.hasHaving())
{
if (expressions.has_order_by)
executeOrder(query_plan, query_info.input_order_info);
if (expressions.has_order_by && query.limitLength())
executeDistinct(query_plan, false, expressions.selected_columns, true);
if (expressions.hasLimitBy())
{
executeExpression(query_plan, expressions.before_limit_by, "Before LIMIT BY");
executeLimitBy(query_plan);
}
if (query.limitLength())
executePreLimit(query_plan, true);
}
};
if (intermediate_stage)
{
if (expressions.first_stage || expressions.second_stage)
throw Exception("Query with intermediate stage cannot have any other stages", ErrorCodes::LOGICAL_ERROR);
preliminary_sort();
if (expressions.need_aggregate)
executeMergeAggregated(query_plan, aggregate_overflow_row, aggregate_final);
}
if (from_aggregation_stage)
{
if (intermediate_stage || expressions.first_stage || expressions.second_stage)
throw Exception("Query with after aggregation stage cannot have any other stages", ErrorCodes::LOGICAL_ERROR);
}
2018-02-23 06:00:48 +00:00
if (expressions.first_stage)
{
2021-02-15 19:48:06 +00:00
if (expressions.filter_info)
{
auto row_level_security_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
expressions.filter_info->actions,
2020-05-28 08:24:59 +00:00
expressions.filter_info->column_name,
2020-06-16 12:02:10 +00:00
expressions.filter_info->do_remove_column);
row_level_security_step->setStepDescription("Row-level security filter");
query_plan.addStep(std::move(row_level_security_step));
}
if (expressions.before_array_join)
{
QueryPlanStepPtr before_array_join_step = std::make_unique<ExpressionStep>(
query_plan.getCurrentDataStream(),
expressions.before_array_join);
before_array_join_step->setStepDescription("Before ARRAY JOIN");
query_plan.addStep(std::move(before_array_join_step));
}
if (expressions.array_join)
{
QueryPlanStepPtr array_join_step = std::make_unique<ArrayJoinStep>(
query_plan.getCurrentDataStream(),
expressions.array_join);
array_join_step->setStepDescription("ARRAY JOIN");
query_plan.addStep(std::move(array_join_step));
}
2020-06-25 17:40:25 +00:00
if (expressions.before_join)
{
QueryPlanStepPtr before_join_step = std::make_unique<ExpressionStep>(
query_plan.getCurrentDataStream(),
2020-06-26 00:23:11 +00:00
expressions.before_join);
2020-06-25 18:18:27 +00:00
before_join_step->setStepDescription("Before JOIN");
2020-06-25 17:40:25 +00:00
query_plan.addStep(std::move(before_join_step));
}
/// Optional step to convert key columns to common supertype.
/// Columns with changed types will be returned to user,
/// so its only suitable for `USING` join.
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));
}
if (expressions.hasJoin())
{
2020-09-08 10:40:53 +00:00
QueryPlanStepPtr join_step = std::make_unique<JoinStep>(
2020-06-25 18:18:27 +00:00
query_plan.getCurrentDataStream(),
2021-03-24 13:22:51 +00:00
expressions.join,
expressions.join_has_delayed_stream,
settings.max_block_size);
2020-06-25 17:40:25 +00:00
join_step->setStepDescription("JOIN");
query_plan.addStep(std::move(join_step));
}
if (expressions.hasWhere())
executeWhere(query_plan, expressions.before_where, expressions.remove_where_filter);
2018-02-23 06:00:48 +00:00
if (expressions.need_aggregate)
2020-05-14 13:56:17 +00:00
{
executeAggregation(query_plan, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.input_order_info);
2020-05-14 13:56:17 +00:00
/// We need to reset input order info, so that executeOrder can't use it
query_info.input_order_info.reset();
}
else
{
2020-12-25 03:13:30 +00:00
executeExpression(query_plan, expressions.before_window, "Before window functions");
executeWindow(query_plan);
2020-12-25 03:13:30 +00:00
executeExpression(query_plan, expressions.before_order_by, "Before ORDER BY");
executeDistinct(query_plan, true, expressions.selected_columns, true);
}
preliminary_sort();
2019-01-22 19:56:53 +00:00
// If there is no global subqueries, we can run subqueries only when receive them on server.
if (!query_analyzer->hasGlobalSubqueries() && !subqueries_for_sets.empty())
executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets);
}
if (expressions.second_stage || from_aggregation_stage)
{
if (from_aggregation_stage)
{
/// No need to aggregate anything, since this was done on remote shards.
}
else if (expressions.need_aggregate)
{
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)
executeMergeAggregated(query_plan, aggregate_overflow_row, aggregate_final);
if (!aggregate_final)
2018-08-21 16:08:45 +00:00
{
if (query.group_by_with_totals)
2018-09-17 19:16:51 +00:00
{
bool final = !query.group_by_with_rollup && !query.group_by_with_cube;
executeTotalsAndHaving(query_plan, expressions.hasHaving(), expressions.before_having, aggregate_overflow_row, final);
2018-09-17 19:16:51 +00:00
}
2018-09-06 01:06:30 +00:00
2018-09-17 18:01:04 +00:00
if (query.group_by_with_rollup)
executeRollupOrCube(query_plan, Modificator::ROLLUP);
else if (query.group_by_with_cube)
executeRollupOrCube(query_plan, Modificator::CUBE);
2019-01-16 00:26:15 +00:00
if ((query.group_by_with_rollup || query.group_by_with_cube) && expressions.hasHaving())
2019-01-16 00:26:15 +00:00
{
if (query.group_by_with_totals)
throw Exception("WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING", ErrorCodes::NOT_IMPLEMENTED);
executeHaving(query_plan, expressions.before_having);
2019-01-16 00:26:15 +00:00
}
2018-08-21 16:08:45 +00:00
}
else if (expressions.hasHaving())
executeHaving(query_plan, expressions.before_having);
2020-12-25 03:13:30 +00:00
executeExpression(query_plan, expressions.before_window,
"Before window functions");
executeWindow(query_plan);
2020-12-25 03:13:30 +00:00
executeExpression(query_plan, expressions.before_order_by, "Before ORDER BY");
executeDistinct(query_plan, true, expressions.selected_columns, true);
}
else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube)
throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::NOT_IMPLEMENTED);
2018-02-23 06:00:48 +00:00
if (expressions.has_order_by)
{
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.
*
* Also in case of remote servers was process the query up to WithMergeableStateAfterAggregation
* (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.
*/
if (from_aggregation_stage)
executeMergeSorted(query_plan, "for ORDER BY");
else if (!expressions.first_stage && !expressions.need_aggregate && !(query.group_by_with_totals && !aggregate_final))
2020-06-22 14:37:42 +00:00
executeMergeSorted(query_plan, "for ORDER BY");
2017-04-02 17:37:49 +00:00
else /// Otherwise, just sort.
executeOrder(query_plan, query_info.input_order_info);
}
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`.
*/
bool has_withfill = false;
if (query.orderBy())
{
SortDescription order_descr = getSortDescription(query, *context);
for (auto & desc : order_descr)
if (desc.with_fill)
{
has_withfill = true;
break;
}
}
bool has_prelimit = false;
if (!to_aggregation_stage &&
query.limitLength() && !query.limit_with_ties && !hasWithTotalsInAnySubqueryInFromClause(query) &&
!query.arrayJoinExpressionList() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes &&
!has_withfill)
2018-02-25 06:34:20 +00:00
{
executePreLimit(query_plan, false);
has_prelimit = true;
2018-02-25 06:34:20 +00:00
}
2018-03-01 01:25:06 +00:00
/** If there was more than one stream,
* then DISTINCT needs to be performed once again after merging all streams.
*/
if (query.distinct)
executeDistinct(query_plan, false, expressions.selected_columns, false);
if (expressions.hasLimitBy())
{
executeExpression(query_plan, expressions.before_limit_by, "Before LIMIT BY");
executeLimitBy(query_plan);
}
executeWithFill(query_plan);
2020-06-15 22:24:00 +00:00
/// If we have 'WITH TIES', we need execute limit before projection,
/// because in that case columns from 'ORDER BY' are used.
if (query.limit_with_ties)
{
2020-06-19 08:30:21 +00:00
executeLimit(query_plan);
2020-06-15 22:24:00 +00:00
has_prelimit = true;
}
/// Projection not be done on the shards, since then initiator will not find column in blocks.
/// (significant only for WithMergeableStateAfterAggregation).
if (!to_aggregation_stage)
{
/// We must do projection after DISTINCT because projection may remove some columns.
executeProjection(query_plan, expressions.final_projection);
}
2018-03-01 01:25:06 +00:00
/// Extremes are calculated before LIMIT, but after LIMIT BY. This is Ok.
executeExtremes(query_plan);
2018-03-01 01:25:06 +00:00
/// Limit is no longer needed if there is prelimit.
if (!to_aggregation_stage && !has_prelimit)
executeLimit(query_plan);
2020-05-06 06:50:55 +00:00
if (!to_aggregation_stage)
executeOffset(query_plan);
}
}
2020-11-02 12:07:01 +00:00
if (!subqueries_for_sets.empty() && (expressions.hasHaving() || query_analyzer->hasGlobalSubqueries()))
executeSubqueriesInSetsAndJoins(query_plan, subqueries_for_sets);
2012-05-09 13:12:38 +00:00
}
2020-09-15 10:40:39 +00:00
static StreamLocalLimits getLimitsForStorage(const Settings & settings, const SelectQueryOptions & options)
{
StreamLocalLimits limits;
limits.mode = LimitsMode::LIMITS_TOTAL;
limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read,
settings.read_overflow_mode);
limits.speed_limits.max_execution_time = settings.max_execution_time;
limits.timeout_overflow_mode = settings.timeout_overflow_mode;
/** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers,
* because the initiating server has a summary of the execution of the request on all servers.
*
* But limits on data size to read and maximum execution time are reasonable to check both on initiator and
* additionally on each remote server, because these limits are checked per block of data processed,
* and remote servers may process way more blocks of data than are received by initiator.
*
* The limits to throttle maximum execution speed is also checked on all servers.
*/
if (options.to_stage == QueryProcessingStage::Complete)
{
limits.speed_limits.min_execution_rps = settings.min_execution_speed;
limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes;
}
limits.speed_limits.max_execution_rps = settings.max_execution_speed;
limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes;
limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed;
return limits;
}
2020-09-25 13:19:26 +00:00
void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info)
{
2020-09-25 13:19:26 +00:00
Pipe pipe(std::make_shared<NullSource>(source_header));
if (query_info.prewhere_info)
{
auto & prewhere_info = *query_info.prewhere_info;
2021-02-15 19:48:06 +00:00
if (prewhere_info.alias_actions)
{
pipe.addSimpleTransform([&](const Block & header)
{
2021-02-15 19:48:06 +00:00
return std::make_shared<ExpressionTransform>(
header,
2021-02-15 19:48:06 +00:00
prewhere_info.alias_actions);
});
}
2021-02-15 19:48:06 +00:00
if (prewhere_info.row_level_filter)
{
pipe.addSimpleTransform([&](const Block & header)
{
2021-02-15 19:48:06 +00:00
return std::make_shared<FilterTransform>(
header,
2021-02-15 19:48:06 +00:00
prewhere_info.row_level_filter,
prewhere_info.row_level_column_name,
2021-02-20 11:00:16 +00:00
true);
});
}
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(
header,
prewhere_info.prewhere_actions,
prewhere_info.prewhere_column_name,
prewhere_info.remove_prewhere_column);
});
// To remove additional columns
// In some cases, we did not read any marks so that the pipeline.streams is empty
// Thus, some columns in prewhere are not removed as expected
// This leads to mismatched header in distributed table
if (prewhere_info.remove_columns_actions)
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(
header,
prewhere_info.remove_columns_actions);
});
}
}
auto read_from_pipe = std::make_unique<ReadFromPreparedSource>(std::move(pipe));
read_from_pipe->setStepDescription("Read from NullSource");
query_plan.addStep(std::move(read_from_pipe));
}
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();
const Settings & settings = context->getSettingsRef();
2021-01-25 18:09:17 +00:00
auto & expressions = analysis_result;
auto & prewhere_info = expressions.prewhere_info;
auto & columns_to_remove_after_prewhere = expressions.columns_to_remove_after_prewhere;
/// Optimization for trivial query like SELECT count() FROM table.
2020-06-04 22:01:40 +00:00
bool optimize_trivial_count =
syntax_analyzer_result->optimize_trivial_count
2020-09-21 10:13:01 +00:00
&& (settings.max_parallel_replicas <= 1)
&& storage
2020-10-14 13:15:15 +00:00
&& storage->getName() != "MaterializeMySQL"
2021-02-16 09:37:19 +00:00
&& !row_policy_filter
&& processing_stage == QueryProcessingStage::FetchColumns
&& query_analyzer->hasAggregation()
&& (query_analyzer->aggregates().size() == 1)
&& typeid_cast<AggregateFunctionCount *>(query_analyzer->aggregates()[0].function.get());
2020-06-04 22:01:40 +00:00
if (optimize_trivial_count)
{
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{};
2020-09-21 10:13:01 +00:00
if (!query.prewhere() && !query.where())
{
2020-11-25 13:47:32 +00:00
num_rows = storage->totalRows(settings);
}
2020-09-21 10:13:01 +00:00
else // It's possible to optimize count() given only partition predicates
{
SelectQueryInfo temp_query_info;
2020-12-12 16:42:15 +00:00
temp_query_info.query = query_ptr;
2020-09-21 10:13:01 +00:00
temp_query_info.syntax_analyzer_result = syntax_analyzer_result;
temp_query_info.sets = query_analyzer->getPreparedSets();
2020-12-12 16:42:15 +00:00
2020-09-21 10:13:01 +00:00
num_rows = storage->totalRowsByPartitionPredicate(temp_query_info, *context);
}
if (num_rows)
{
AggregateFunctionCount & agg_count = static_cast<AggregateFunctionCount &>(*func);
/// We will process it up to "WithMergeableState".
std::vector<char> state(agg_count.sizeOfData());
AggregateDataPtr place = state.data();
agg_count.create(place);
SCOPE_EXIT_MEMORY_SAFE(agg_count.destroy(place));
2019-10-30 19:15:55 +00:00
agg_count.set(place, *num_rows);
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();
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;
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}};
2020-05-28 16:01:12 +00:00
auto istream = std::make_shared<OneBlockInputStream>(block_with_count);
auto prepared_count = std::make_unique<ReadFromPreparedSource>(Pipe(std::make_shared<SourceFromInputStream>(istream)), context);
prepared_count->setStepDescription("Optimized trivial count");
query_plan.addStep(std::move(prepared_count));
from_stage = QueryProcessingStage::WithMergeableState;
analysis_result.first_stage = false;
return;
}
}
2017-04-02 17:37:49 +00:00
/// Actions to calculate ALIAS if required.
2020-11-03 11:28:28 +00:00
ActionsDAGPtr alias_actions;
if (storage)
{
/// Detect, if ALIAS columns are required for query execution
auto alias_columns_required = false;
const ColumnsDescription & storage_columns = metadata_snapshot->getColumns();
for (const auto & column_name : required_columns)
{
auto column_default = storage_columns.getDefault(column_name);
if (column_default && column_default->kind == ColumnDefaultKind::Alias)
{
alias_columns_required = true;
break;
}
}
/// There are multiple sources of required columns:
/// - raw required columns,
/// - columns deduced from ALIAS columns,
/// - raw required columns from PREWHERE,
/// - columns deduced from ALIAS columns from PREWHERE.
/// PREWHERE is a special case, since we need to resolve it and pass directly to `IStorage::read()`
/// before any other executions.
if (alias_columns_required)
{
NameSet required_columns_from_prewhere; /// Set of all (including ALIAS) required columns for PREWHERE
NameSet required_aliases_from_prewhere; /// Set of ALIAS required columns for PREWHERE
if (prewhere_info)
{
/// Get some columns directly from PREWHERE expression actions
2020-11-03 11:28:28 +00:00
auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames();
required_columns_from_prewhere.insert(prewhere_required_columns.begin(), prewhere_required_columns.end());
2021-02-20 14:19:11 +00:00
if (prewhere_info->row_level_filter_actions)
{
auto row_level_required_columns = prewhere_info->row_level_filter_actions->getRequiredColumns().getNames();
required_columns_from_prewhere.insert(row_level_required_columns.begin(), row_level_required_columns.end());
}
}
/// Expression, that contains all raw required columns
ASTPtr required_columns_all_expr = std::make_shared<ASTExpressionList>();
/// Expression, that contains raw required columns for PREWHERE
ASTPtr required_columns_from_prewhere_expr = std::make_shared<ASTExpressionList>();
/// Sort out already known required columns between expressions,
/// also populate `required_aliases_from_prewhere`.
for (const auto & column : required_columns)
{
ASTPtr column_expr;
const auto column_default = storage_columns.getDefault(column);
bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias;
if (is_alias)
{
auto column_decl = storage_columns.get(column);
2020-12-12 16:42:15 +00:00
column_expr = column_default->expression->clone();
// recursive visit for alias to alias
2020-12-12 16:42:15 +00:00
replaceAliasColumnsInQuery(column_expr, metadata_snapshot->getColumns(), syntax_analyzer_result->getArrayJoinSourceNameSet(), *context);
2020-12-13 09:33:02 +00:00
column_expr = addTypeConversionToAST(std::move(column_expr), column_decl.type->getName(), metadata_snapshot->getColumns().getAll(), *context);
column_expr = setAlias(column_expr, column);
}
else
column_expr = std::make_shared<ASTIdentifier>(column);
if (required_columns_from_prewhere.count(column))
{
required_columns_from_prewhere_expr->children.emplace_back(std::move(column_expr));
if (is_alias)
required_aliases_from_prewhere.insert(column);
}
else
required_columns_all_expr->children.emplace_back(std::move(column_expr));
}
/// Columns, which we will get after prewhere and filter executions.
NamesAndTypesList required_columns_after_prewhere;
NameSet required_columns_after_prewhere_set;
/// Collect required columns from prewhere expression actions.
if (prewhere_info)
{
NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end());
2020-11-03 11:28:28 +00:00
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
/// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards.
/// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure.
for (const auto & column : prewhere_actions_result)
{
if (prewhere_info->remove_prewhere_column && column.name == prewhere_info->prewhere_column_name)
continue;
if (columns_to_remove.count(column.name))
continue;
required_columns_all_expr->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
required_columns_after_prewhere.emplace_back(column.name, column.type);
}
required_columns_after_prewhere_set
= ext::map<NameSet>(required_columns_after_prewhere, [](const auto & it) { return it.name; });
}
auto syntax_result = TreeRewriter(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, metadata_snapshot);
2020-11-03 11:28:28 +00:00
alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, *context).getActionsDAG(true);
2017-04-02 17:37:49 +00:00
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
2020-11-03 11:28:28 +00:00
required_columns = alias_actions->getRequiredColumns().getNames();
2018-09-04 18:50:19 +00:00
/// 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;
2018-09-04 18:50:19 +00:00
/// Remove columns which will be added by prewhere.
required_columns.erase(std::remove_if(required_columns.begin(), required_columns.end(), [&](const String & name)
{
2020-05-28 08:24:59 +00:00
return required_columns_after_prewhere_set.count(name) != 0;
}), required_columns.end());
if (prewhere_info)
{
2018-09-04 18:50:19 +00:00
/// Don't remove columns which are needed to be aliased.
2020-11-03 11:28:28 +00:00
for (const auto & name : required_columns)
prewhere_info->prewhere_actions->tryRestoreColumn(name);
auto analyzed_result
= TreeRewriter(*context).analyze(required_columns_from_prewhere_expr, metadata_snapshot->getColumns().getAllPhysical());
prewhere_info->alias_actions
2020-11-03 11:28:28 +00:00
= ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, *context).getActionsDAG(true, false);
2018-09-04 18:50:19 +00:00
/// Add (physical?) columns required by alias actions.
auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns();
2020-11-03 11:28:28 +00:00
Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns();
for (auto & column : required_columns_from_alias)
2020-11-03 11:28:28 +00:00
if (!prewhere_actions_result.has(column.name))
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name))
required_columns.push_back(column.name);
/// Add physical columns required by prewhere actions.
for (const auto & column : required_columns_from_prewhere)
if (required_aliases_from_prewhere.count(column) == 0)
if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column))
required_columns.push_back(column);
}
}
}
/// Limitation on the number of columns to read.
/// 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)
throw Exception("Limit for number of columns to read exceeded. "
"Requested: " + toString(required_columns.size())
+ ", maximum: " + settings.max_columns_to_read.toString(),
2018-03-09 23:23:15 +00:00
ErrorCodes::TOO_MANY_COLUMNS);
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.
*/
bool is_remote = false;
if (storage && storage->isRemote())
{
is_remote = true;
2020-10-02 12:38:33 +00:00
max_threads_execute_query = max_streams = settings.max_distributed_connections;
}
2019-02-10 16:55:12 +00:00
UInt64 max_block_size = settings.max_block_size;
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
2019-02-10 15:17:45 +00:00
2019-04-19 13:38:25 +00:00
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size,
2017-04-02 17:37:49 +00:00
* then as the block size we will use limit + offset (not to read more from the table than requested),
* and also set the number of threads to 1.
*/
if (!query.distinct
2019-04-19 13:38:25 +00:00
&& !query.limit_with_ties
&& !query.prewhere()
&& !query.where()
&& !query.groupBy()
&& !query.having()
&& !query.orderBy()
&& !query.limitBy()
&& query.limitLength()
&& !query_analyzer->hasAggregation()
&& !query_analyzer->hasWindow()
2020-07-12 05:18:01 +00:00
&& limit_length <= std::numeric_limits<UInt64>::max() - limit_offset
2018-02-25 06:34:20 +00:00
&& limit_length + limit_offset < max_block_size)
{
2019-02-10 16:55:12 +00:00
max_block_size = std::max(UInt64(1), limit_length + limit_offset);
2020-10-02 12:38:33 +00:00
max_threads_execute_query = max_streams = 1;
}
2019-02-10 16:55:12 +00:00
if (!max_block_size)
throw Exception("Setting 'max_block_size' cannot be zero", ErrorCodes::PARAMETER_OUT_OF_BOUND);
2018-02-21 06:25:21 +00:00
/// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery or prepared input?
if (query_plan.isInitialized())
2018-02-21 06:25:21 +00:00
{
/// Prepared input.
}
else if (interpreter_subquery)
{
2018-02-21 06:25:21 +00:00
/// Subquery.
/// If we need less number of columns that subquery have - update the interpreter.
if (required_columns.size() < source_header.columns())
{
2019-01-15 18:29:54 +00:00
ASTPtr subquery = extractTableExpression(query, 0);
if (!subquery)
throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR);
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
subquery, getSubqueryContext(*context),
2019-03-18 12:05:51 +00:00
options.copy().subquery().noModify(), required_columns);
if (query_analyzer->hasAggregation())
interpreter_subquery->ignoreWithTotals();
}
interpreter_subquery->buildQueryPlan(query_plan);
query_plan.addInterpreterContext(context);
2018-02-21 06:25:21 +00:00
}
else if (storage)
{
/// Table.
if (max_streams == 0)
throw Exception("Logical error: zero number of streams requested", ErrorCodes::LOGICAL_ERROR);
2017-04-02 17:37:49 +00:00
/// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads.
if (max_streams > 1 && !is_remote)
max_streams *= settings.max_streams_to_max_threads_ratio;
query_info.syntax_analyzer_result = syntax_analyzer_result;
query_info.sets = query_analyzer->getPreparedSets();
2021-03-04 17:38:12 +00:00
auto actions_settings = ExpressionActionsSettings::fromContext(*context);
2020-11-03 19:05:47 +00:00
if (prewhere_info)
{
query_info.prewhere_info = std::make_shared<PrewhereInfo>();
query_info.prewhere_info->prewhere_actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions, actions_settings);
2020-11-03 19:05:47 +00:00
2021-02-15 19:48:06 +00:00
if (prewhere_info->row_level_filter_actions)
query_info.prewhere_info->row_level_filter = std::make_shared<ExpressionActions>(prewhere_info->row_level_filter_actions, actions_settings);
2020-11-03 19:05:47 +00:00
if (prewhere_info->alias_actions)
2021-03-04 17:38:12 +00:00
query_info.prewhere_info->alias_actions = std::make_shared<ExpressionActions>(prewhere_info->alias_actions, actions_settings);
2020-11-03 19:05:47 +00:00
if (prewhere_info->remove_columns_actions)
2021-03-04 17:38:12 +00:00
query_info.prewhere_info->remove_columns_actions = std::make_shared<ExpressionActions>(prewhere_info->remove_columns_actions, actions_settings);
2020-11-03 19:05:47 +00:00
query_info.prewhere_info->prewhere_column_name = prewhere_info->prewhere_column_name;
2020-11-03 19:05:47 +00:00
query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column;
2021-02-15 19:48:06 +00:00
query_info.prewhere_info->row_level_column_name = prewhere_info->row_level_column_name;
2020-11-03 19:05:47 +00:00
query_info.prewhere_info->need_filter = prewhere_info->need_filter;
}
2019-12-10 23:18:24 +00:00
/// Create optimizer with prepared actions.
2020-05-13 13:49:10 +00:00
/// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge.
if (analysis_result.optimize_read_in_order || analysis_result.optimize_aggregation_in_order)
2019-12-10 23:18:24 +00:00
{
2020-05-13 13:49:10 +00:00
if (analysis_result.optimize_read_in_order)
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
analysis_result.order_by_elements_actions,
getSortDescription(query, *context),
query_info.syntax_analyzer_result);
else
query_info.order_optimizer = std::make_shared<ReadInOrderOptimizer>(
2020-02-16 19:46:45 +00:00
analysis_result.group_by_elements_actions,
2020-05-14 14:20:49 +00:00
getSortDescriptionFromGroupBy(query),
2020-02-16 19:46:45 +00:00
query_info.syntax_analyzer_result);
2019-12-10 23:18:24 +00:00
2020-12-12 16:42:15 +00:00
query_info.input_order_info = query_info.order_optimizer->getInputOrder(metadata_snapshot, *context);
2019-12-10 23:18:24 +00:00
}
2020-09-15 10:40:39 +00:00
StreamLocalLimits limits;
SizeLimits leaf_limits;
2020-09-14 14:13:58 +00:00
std::shared_ptr<const EnabledQuota> quota;
2020-09-14 14:13:58 +00:00
/// Set the limits and quota for reading data, the speed and time of the query.
2020-09-17 09:57:48 +00:00
if (!options.ignore_limits)
{
2020-09-15 10:40:39 +00:00
limits = getLimitsForStorage(settings, options);
leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, settings.max_bytes_to_read_leaf,
settings.read_overflow_mode_leaf);
}
2020-09-14 14:13:58 +00:00
if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete))
quota = context->getQuota();
storage->read(query_plan, required_columns, metadata_snapshot,
query_info, *context, processing_stage, max_block_size, max_streams);
2020-12-18 06:54:38 +00:00
if (context->hasQueryContext() && !options.is_internal)
{
auto local_storage_id = storage->getStorageID();
context->getQueryContext().addQueryAccessInfo(
2020-12-13 22:36:04 +00:00
backQuoteIfNeed(local_storage_id.getDatabaseName()), local_storage_id.getFullTableName(), required_columns);
}
/// Create step which reads from empty source if storage has no data.
if (!query_plan.isInitialized())
{
auto header = metadata_snapshot->getSampleBlockForColumns(
required_columns, storage->getVirtuals(), storage->getStorageID());
2020-09-25 13:19:26 +00:00
addEmptySourceToQueryPlan(query_plan, header, query_info);
}
/// Extend lifetime of context, table lock, storage. Set limits and quota.
auto adding_limits_and_quota = std::make_unique<SettingQuotaAndLimitsStep>(
query_plan.getCurrentDataStream(),
storage,
std::move(table_lock),
limits,
leaf_limits,
std::move(quota),
context);
adding_limits_and_quota->setStepDescription("Set limits and quota after reading from storage");
query_plan.addStep(std::move(adding_limits_and_quota));
}
2018-02-21 06:25:21 +00:00
else
throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR);
2020-10-22 11:08:12 +00:00
/// Specify the number of threads only if it wasn't specified in storage.
///
/// 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
{
auto table_aliases = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), alias_actions);
table_aliases->setStepDescription("Add table aliases");
query_plan.addStep(std::move(table_aliases));
2018-02-26 21:00:42 +00:00
}
2012-05-09 13:12:38 +00:00
}
2011-08-28 05:13:24 +00:00
2012-05-09 13:12:38 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
2019-03-26 18:28:37 +00:00
{
auto where_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
2020-06-16 12:02:10 +00:00
expression,
getSelectQuery().where()->getColumnName(),
remove_filter);
where_step->setStepDescription("WHERE");
query_plan.addStep(std::move(where_step));
2019-03-26 18:28:37 +00:00
}
2012-08-27 05:13:14 +00:00
2012-05-09 13:12:38 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info)
2019-03-26 18:28:37 +00:00
{
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
const auto & header_before_aggregation = query_plan.getCurrentDataStream().header;
2019-03-26 18:28:37 +00:00
ColumnNumbers keys;
for (const auto & key : query_analyzer->aggregationKeys())
keys.push_back(header_before_aggregation.getPositionByName(key.name));
AggregateDescriptions aggregates = query_analyzer->aggregates();
2019-03-26 18:28:37 +00:00
for (auto & descr : aggregates)
if (descr.arguments.empty())
for (const auto & name : descr.argument_names)
2019-04-05 11:27:08 +00:00
descr.arguments.push_back(header_before_aggregation.getPositionByName(name));
2019-03-26 18:28:37 +00:00
const Settings & settings = context->getSettingsRef();
2019-03-26 18:28:37 +00:00
2019-04-05 11:27:08 +00:00
Aggregator::Params params(header_before_aggregation, keys, aggregates,
2019-03-26 18:28:37 +00:00
overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
settings.group_by_two_level_threshold,
settings.group_by_two_level_threshold_bytes,
settings.max_bytes_before_external_group_by,
settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(),
settings.max_threads,
settings.min_free_disk_space_for_temporary_data);
2019-03-26 18:28:37 +00:00
2020-06-17 14:21:48 +00:00
SortDescription group_by_sort_description;
2019-04-17 15:35:22 +00:00
2020-04-18 09:51:21 +00:00
if (group_by_info && settings.optimize_aggregation_in_order)
2020-06-17 14:21:48 +00:00
group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery());
2019-03-26 18:28:37 +00:00
else
2020-06-17 14:21:48 +00:00
group_by_info = nullptr;
2019-03-26 18:28:37 +00:00
2020-06-17 14:21:48 +00:00
auto merge_threads = max_streams;
auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
2019-12-26 16:52:15 +00:00
2020-06-17 14:21:48 +00:00
bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead();
auto aggregating_step = std::make_unique<AggregatingStep>(
query_plan.getCurrentDataStream(),
params, final,
2020-06-17 14:21:48 +00:00
settings.max_block_size,
merge_threads,
temporary_data_merge_threads,
storage_has_evenly_distributed_read,
std::move(group_by_info),
std::move(group_by_sort_description));
query_plan.addStep(std::move(aggregating_step));
2019-03-26 18:28:37 +00:00
}
void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final)
2019-03-26 18:28:37 +00:00
{
const auto & header_before_merge = query_plan.getCurrentDataStream().header;
2019-03-26 18:28:37 +00:00
ColumnNumbers keys;
for (const auto & key : query_analyzer->aggregationKeys())
keys.push_back(header_before_merge.getPositionByName(key.name));
2019-03-26 18:28:37 +00:00
/** There are two modes of distributed aggregation.
*
* 1. In different threads read from the remote servers blocks.
* Save all the blocks in the RAM. Merge blocks.
* If the aggregation is two-level - parallelize to the number of buckets.
*
* 2. In one thread, read blocks from different servers in order.
* RAM stores only one block from each server.
* If the aggregation is a two-level aggregation, we consistently merge the blocks of each next level.
*
* The second option consumes less memory (up to 256 times less)
* in the case of two-level aggregation, which is used for large results after GROUP BY,
* but it can work more slowly.
*/
const Settings & settings = context->getSettingsRef();
2019-03-26 18:28:37 +00:00
Aggregator::Params params(header_before_merge, keys, query_analyzer->aggregates(), overflow_row, settings.max_threads);
2019-03-26 18:28:37 +00:00
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
query_plan.getCurrentDataStream(),
2020-06-17 20:19:55 +00:00
std::move(transform_params),
settings.distributed_aggregation_memory_efficient && storage && storage->isRemote(),
2020-06-16 18:57:21 +00:00
settings.max_threads,
settings.aggregation_memory_efficient_merge_threads);
2019-03-26 18:28:37 +00:00
query_plan.addStep(std::move(merging_aggregated));
2019-03-26 18:28:37 +00:00
}
2012-05-09 13:12:38 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression)
2019-03-26 18:28:37 +00:00
{
auto having_step = std::make_unique<FilterStep>(
query_plan.getCurrentDataStream(),
2020-06-17 19:02:46 +00:00
expression, getSelectQuery().having()->getColumnName(), false);
2019-04-29 10:00:17 +00:00
having_step->setStepDescription("HAVING");
query_plan.addStep(std::move(having_step));
2019-03-26 18:28:37 +00:00
}
2011-09-25 03:37:09 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery::executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final)
2019-03-26 18:28:37 +00:00
{
const Settings & settings = context->getSettingsRef();
2019-03-26 18:28:37 +00:00
auto totals_having_step = std::make_unique<TotalsHavingStep>(
query_plan.getCurrentDataStream(),
2020-06-17 16:54:51 +00:00
overflow_row, expression,
2019-05-14 11:04:11 +00:00
has_having ? getSelectQuery().having()->getColumnName() : "",
2019-03-26 18:28:37 +00:00
settings.totals_mode, settings.totals_auto_threshold, final);
query_plan.addStep(std::move(totals_having_step));
2019-03-26 18:28:37 +00:00
}
void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modificator modificator)
2019-03-26 18:28:37 +00:00
{
const auto & header_before_transform = query_plan.getCurrentDataStream().header;
2019-03-26 18:28:37 +00:00
ColumnNumbers keys;
for (const auto & key : query_analyzer->aggregationKeys())
keys.push_back(header_before_transform.getPositionByName(key.name));
2019-03-26 18:28:37 +00:00
const Settings & settings = context->getSettingsRef();
2019-03-26 18:28:37 +00:00
Aggregator::Params params(header_before_transform, keys, query_analyzer->aggregates(),
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0,
2019-03-26 18:28:37 +00:00
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
2019-03-26 18:28:37 +00:00
2019-04-05 10:52:07 +00:00
auto transform_params = std::make_shared<AggregatingTransformParams>(params, true);
QueryPlanStepPtr step;
2020-06-17 17:15:24 +00:00
if (modificator == Modificator::ROLLUP)
step = std::make_unique<RollupStep>(query_plan.getCurrentDataStream(), std::move(transform_params));
2020-06-17 17:15:24 +00:00
else
step = std::make_unique<CubeStep>(query_plan.getCurrentDataStream(), std::move(transform_params));
query_plan.addStep(std::move(step));
2019-03-26 18:28:37 +00:00
}
2014-02-27 12:49:21 +00:00
2020-11-03 11:28:28 +00:00
void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description)
2019-03-26 18:28:37 +00:00
{
2020-12-25 03:13:30 +00:00
if (!expression)
{
return;
}
auto expression_step = std::make_unique<ExpressionStep>(query_plan.getCurrentDataStream(), expression);
2020-06-16 12:02:10 +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-02-10 12:37:05 +00:00
static bool windowDescriptionComparator(const WindowDescription * _left,
const WindowDescription * _right)
{
const auto & left = _left->full_sort_description;
const auto & right = _right->full_sort_description;
2021-02-10 13:27:22 +00:00
for (size_t i = 0; i < std::min(left.size(), right.size()); ++i)
2021-02-10 12:37:05 +00:00
{
if (left[i].column_name < right[i].column_name)
{
return true;
}
2021-02-11 08:39:39 +00:00
else if (left[i].column_name > right[i].column_name)
{
return false;
}
else if (left[i].column_number < right[i].column_number)
2021-02-10 12:37:05 +00:00
{
return true;
}
2021-02-11 08:39:39 +00:00
else if (left[i].column_number > right[i].column_number)
{
return false;
}
else if (left[i].direction < right[i].direction)
2021-02-10 12:37:05 +00:00
{
return true;
}
2021-02-11 08:39:39 +00:00
else if (left[i].direction > right[i].direction)
{
return false;
}
else if (left[i].nulls_direction < right[i].nulls_direction)
2021-02-10 12:37:05 +00:00
{
return true;
}
2021-02-11 08:39:39 +00:00
else if (left[i].nulls_direction > right[i].nulls_direction)
2021-02-10 13:27:22 +00:00
{
return false;
}
2021-02-11 08:39:39 +00:00
assert(left[i] == right[i]);
2021-02-10 12:37:05 +00:00
}
2021-02-10 13:27:22 +00:00
// Note that we check the length last, because we want to put together the
// sort orders that have common prefix but different length.
return left.size() > right.size();
2021-02-10 12:37:05 +00:00
}
static bool sortIsPrefix(const WindowDescription & _prefix,
const WindowDescription & _full)
{
const auto & prefix = _prefix.full_sort_description;
const auto & full = _full.full_sort_description;
if (prefix.size() > full.size())
{
return false;
}
for (size_t i = 0; i < prefix.size(); ++i)
{
if (full[i] != prefix[i])
{
return false;
}
}
return true;
}
2012-02-27 06:28:20 +00:00
void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
{
2021-02-10 13:27:22 +00:00
// Try to sort windows in such an order that the window with the longest
// sort description goes first, and all window that use its prefixes follow.
2021-02-10 12:37:05 +00:00
std::vector<const WindowDescription *> windows_sorted;
2020-12-22 19:01:15 +00:00
for (const auto & [_, w] : query_analyzer->windowDescriptions())
{
2021-02-10 12:37:05 +00:00
windows_sorted.push_back(&w);
}
2021-02-10 12:37:05 +00:00
std::sort(windows_sorted.begin(), windows_sorted.end(),
windowDescriptionComparator);
2021-02-10 12:37:05 +00:00
const Settings & settings = context->getSettingsRef();
for (size_t i = 0; i < windows_sorted.size(); ++i)
{
const auto & w = *windows_sorted[i];
2021-03-17 15:31:30 +00:00
// We don't need to sort again if the input from previous window already
// has suitable sorting. Also don't create sort steps when there are no
// columns to sort by, because the sort nodes are confused by this. It
// happens in case of `over ()`.
if (!w.full_sort_description.empty()
&& (i == 0 || !sortIsPrefix(w, *windows_sorted[i - 1])))
2021-02-10 12:37:05 +00:00
{
auto partial_sorting = std::make_unique<PartialSortingStep>(
query_plan.getCurrentDataStream(),
w.full_sort_description,
0 /* LIMIT */,
SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort,
settings.sort_overflow_mode));
partial_sorting->setStepDescription("Sort each block for window '"
+ w.window_name + "'");
query_plan.addStep(std::move(partial_sorting));
auto merge_sorting_step = std::make_unique<MergeSortingStep>(
query_plan.getCurrentDataStream(),
w.full_sort_description,
settings.max_block_size,
0 /* LIMIT */,
settings.max_bytes_before_remerge_sort,
settings.remerge_sort_lowered_memory_bytes_ratio,
settings.max_bytes_before_external_sort,
context->getTemporaryVolume(),
settings.min_free_disk_space_for_temporary_data);
merge_sorting_step->setStepDescription(
"Merge sorted blocks for window '" + w.window_name + "'");
query_plan.addStep(std::move(merge_sorting_step));
// First MergeSorted, now MergingSorted.
auto merging_sorted = std::make_unique<MergingSortedStep>(
query_plan.getCurrentDataStream(),
w.full_sort_description,
settings.max_block_size,
0 /* LIMIT */);
merging_sorted->setStepDescription(
"Merge sorted streams for window '" + w.window_name + "'");
query_plan.addStep(std::move(merging_sorted));
}
auto window_step = std::make_unique<WindowStep>(
query_plan.getCurrentDataStream(),
2020-12-15 00:36:03 +00:00
w,
2020-12-22 01:37:45 +00:00
w.window_functions);
window_step->setStepDescription("Window step for window '"
+ w.window_name + "'");
query_plan.addStep(std::move(window_step));
}
}
void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, InputOrderInfoPtr input_sorting_info, UInt64 limit, SortDescription & output_order_descr)
2019-03-26 18:28:37 +00:00
{
const Settings & settings = context->getSettingsRef();
2019-03-26 18:28:37 +00:00
auto finish_sorting_step = std::make_unique<FinishSortingStep>(
query_plan.getCurrentDataStream(),
2020-06-17 19:57:13 +00:00
input_sorting_info->order_key_prefix_descr,
output_order_descr,
settings.max_block_size,
limit);
2020-05-17 07:27:55 +00:00
query_plan.addStep(std::move(finish_sorting_step));
2020-04-18 09:51:21 +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();
SortDescription output_order_descr = getSortDescription(query, *context);
UInt64 limit = getLimitForSorting(query, *context);
if (input_sorting_info)
{
/* 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.
*/
executeOrderOptimized(query_plan, input_sorting_info, limit, output_order_descr);
return;
}
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
auto partial_sorting = std::make_unique<PartialSortingStep>(
query_plan.getCurrentDataStream(),
2020-06-16 14:45:52 +00:00
output_order_descr,
limit,
SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode));
2020-05-17 07:27:55 +00:00
2020-06-22 14:37:42 +00:00
partial_sorting->setStepDescription("Sort each block for ORDER BY");
query_plan.addStep(std::move(partial_sorting));
2020-05-17 07:27:55 +00:00
2019-03-26 18:28:37 +00:00
/// Merge the sorted blocks.
auto merge_sorting_step = std::make_unique<MergeSortingStep>(
query_plan.getCurrentDataStream(),
2020-12-15 17:16:01 +00:00
output_order_descr,
settings.max_block_size,
limit,
settings.max_bytes_before_remerge_sort,
settings.remerge_sort_lowered_memory_bytes_ratio,
settings.max_bytes_before_external_sort,
context->getTemporaryVolume(),
2020-06-16 15:08:01 +00:00
settings.min_free_disk_space_for_temporary_data);
2020-06-22 14:37:42 +00:00
merge_sorting_step->setStepDescription("Merge sorted blocks for ORDER BY");
query_plan.addStep(std::move(merge_sorting_step));
2020-01-07 05:41:18 +00:00
/// If there are several streams, we merge them into one
2020-06-22 14:37:42 +00:00
executeMergeSorted(query_plan, output_order_descr, limit, "for ORDER BY");
2019-03-26 18:28:37 +00:00
}
2012-07-25 20:29:22 +00:00
void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const std::string & description)
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query, *context);
UInt64 limit = getLimitForSorting(query, *context);
2019-03-26 18:28:37 +00:00
executeMergeSorted(query_plan, order_descr, limit, description);
}
2019-03-26 18:28:37 +00:00
void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const SortDescription & sort_description, UInt64 limit, const std::string & description)
{
2020-06-16 15:47:40 +00:00
const Settings & settings = context->getSettingsRef();
auto merging_sorted = std::make_unique<MergingSortedStep>(
query_plan.getCurrentDataStream(),
sort_description,
2019-03-26 18:28:37 +00:00
settings.max_block_size, limit);
merging_sorted->setStepDescription("Merge sorted streams " + description);
query_plan.addStep(std::move(merging_sorted));
2019-03-26 18:28:37 +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
{
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
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)
{
const Settings & settings = context->getSettingsRef();
2019-03-26 18:28:37 +00:00
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
2019-03-26 18:28:37 +00:00
UInt64 limit_for_distinct = 0;
2020-07-12 05:18:01 +00:00
/// If after this stage of DISTINCT ORDER BY is not executed,
/// then you can get no more than limit_length + limit_offset of different rows.
if ((!query.orderBy() || !before_order) && limit_length <= std::numeric_limits<UInt64>::max() - limit_offset)
2019-03-26 18:28:37 +00:00
limit_for_distinct = limit_length + limit_offset;
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
auto distinct_step = std::make_unique<DistinctStep>(
query_plan.getCurrentDataStream(),
2020-06-18 13:00:16 +00:00
limits, limit_for_distinct, columns, pre_distinct);
if (pre_distinct)
distinct_step->setStepDescription("Preliminary DISTINCT");
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.
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
{
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;
}
auto limit = std::make_unique<LimitStep>(query_plan.getCurrentDataStream(), limit_length, limit_offset);
limit->setStepDescription("preliminary LIMIT");
query_plan.addStep(std::move(limit));
2019-03-26 18:28:37 +00:00
}
}
2012-06-24 23:17:06 +00:00
void InterpreterSelectQuery::executeLimitBy(QueryPlan & query_plan)
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery();
2019-05-14 11:04:11 +00:00
if (!query.limitByLength() || !query.limitBy())
2019-03-26 18:28:37 +00:00
return;
Names columns;
2019-05-14 11:04:11 +00:00
for (const auto & elem : query.limitBy()->children)
2019-03-26 18:28:37 +00:00
columns.emplace_back(elem->getColumnName());
2020-05-07 13:40:50 +00:00
UInt64 length = getLimitUIntValue(query.limitByLength(), *context, "LIMIT");
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context, "OFFSET") : 0);
2019-03-26 18:28:37 +00:00
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
}
void InterpreterSelectQuery::executeWithFill(QueryPlan & query_plan)
2019-08-14 17:01:47 +00:00
{
auto & query = getSelectQuery();
if (query.orderBy())
{
SortDescription order_descr = getSortDescription(query, *context);
SortDescription fill_descr;
for (auto & desc : order_descr)
{
if (desc.with_fill)
fill_descr.push_back(desc);
}
if (fill_descr.empty())
return;
auto filling_step = std::make_unique<FillingStep>(query_plan.getCurrentDataStream(), std::move(fill_descr));
query_plan.addStep(std::move(filling_step));
}
2011-08-28 05:13:24 +00:00
}
2019-08-14 17:01:47 +00:00
void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan)
2019-03-26 18:28:37 +00:00
{
2019-04-03 11:21:38 +00:00
auto & query = getSelectQuery();
2019-03-26 18:28:37 +00:00
/// If there is LIMIT
2019-05-14 11:04:11 +00:00
if (query.limitLength())
2019-03-26 18:28:37 +00:00
{
/** Rare case:
* if there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels,
* then when using LIMIT, you should read the data to the end, rather than cancel the query earlier,
* because if you cancel the query, we will not get `totals` data from the remote server.
*
* Another case:
* if there is WITH TOTALS and there is no ORDER BY, then read the data to the end,
* otherwise TOTALS is counted according to incomplete data.
*/
bool always_read_till_end = false;
2019-05-14 11:04:11 +00:00
if (query.group_by_with_totals && !query.orderBy())
2019-03-26 18:28:37 +00:00
always_read_till_end = true;
if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query))
always_read_till_end = true;
UInt64 limit_length;
UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context);
2019-03-26 18:28:37 +00:00
2019-08-27 17:48:42 +00:00
SortDescription order_descr;
if (query.limit_with_ties)
{
if (!query.orderBy())
throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
order_descr = getSortDescription(query, *context);
2019-08-27 17:48:42 +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);
2020-06-16 18:23:01 +00:00
if (query.limit_with_ties)
limit->setStepDescription("LIMIT WITH TIES");
2020-06-16 18:23:01 +00:00
query_plan.addStep(std::move(limit));
2019-03-26 18:28:37 +00:00
}
}
2011-08-28 05:13:24 +00:00
void InterpreterSelectQuery::executeOffset(QueryPlan & query_plan)
2020-05-06 06:50:55 +00:00
{
auto & query = getSelectQuery();
2020-05-11 04:47:31 +00:00
/// If there is not a LIMIT but an offset
2020-05-06 06:50:55 +00:00
if (!query.limitLength() && query.limitOffset())
{
UInt64 limit_length;
UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context);
2020-06-24 12:09:01 +00:00
auto offsets_step = std::make_unique<OffsetStep>(query_plan.getCurrentDataStream(), limit_offset);
query_plan.addStep(std::move(offsets_step));
2020-05-06 06:50:55 +00:00
}
}
void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan)
2019-03-26 18:28:37 +00:00
{
if (!context->getSettingsRef().extremes)
2019-03-26 18:28:37 +00:00
return;
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
}
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan, SubqueriesForSets & subqueries_for_sets)
2019-03-26 18:28:37 +00:00
{
2020-05-13 13:49:10 +00:00
if (query_info.input_order_info)
executeMergeSorted(query_plan, query_info.input_order_info->order_key_prefix_descr, 0, "before creating sets for subqueries and joins");
const Settings & settings = context->getSettingsRef();
2019-03-26 18:28:37 +00:00
2020-09-15 13:25:14 +00:00
SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode);
2020-09-15 17:13:13 +00:00
addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), limits, *context);
2019-03-26 18:28:37 +00:00
}
2015-05-06 23:35:37 +00:00
void InterpreterSelectQuery::ignoreWithTotals()
{
2019-03-12 14:07:02 +00:00
getSelectQuery().group_by_with_totals = false;
2015-05-06 23:35:37 +00:00
}
void InterpreterSelectQuery::initSettings()
{
2019-03-12 14:07:02 +00:00
auto & query = getSelectQuery();
if (query.settings())
InterpreterSetQuery(query.settings(), *context).executeForCurrentContext();
}
2011-08-28 05:13:24 +00:00
}