2011-08-28 05:13:24 +00:00
|
|
|
#pragma once
|
|
|
|
|
2018-07-19 13:36:21 +00:00
|
|
|
#include <memory>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/QueryProcessingStage.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <DataStreams/IBlockStream_fwd.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2018-02-23 06:00:48 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
2020-11-01 13:54:07 +00:00
|
|
|
#include <Interpreters/IInterpreterUnionOrSelectQuery.h>
|
|
|
|
#include <Interpreters/StorageID.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Storages/ReadInOrderOptimizer.h>
|
2018-09-05 09:47:57 +00:00
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2020-06-22 09:49:21 +00:00
|
|
|
#include <Storages/TableLockHolder.h>
|
2011-08-28 05:13:24 +00:00
|
|
|
|
2019-10-27 18:12:40 +00:00
|
|
|
#include <Columns/FilterDescription.h>
|
2017-01-21 04:24:28 +00:00
|
|
|
|
|
|
|
namespace Poco { class Logger; }
|
|
|
|
|
2011-08-28 05:13:24 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-03-07 04:35:06 +00:00
|
|
|
struct SubqueryForSet;
|
2018-07-19 13:36:21 +00:00
|
|
|
class InterpreterSelectWithUnionQuery;
|
2020-05-20 20:16:32 +00:00
|
|
|
class Context;
|
2020-06-18 13:38:45 +00:00
|
|
|
class QueryPlan;
|
2015-05-06 23:35:37 +00:00
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
struct TreeRewriterResult;
|
|
|
|
using TreeRewriterResultPtr = std::shared_ptr<const TreeRewriterResult>;
|
2015-05-06 23:35:37 +00:00
|
|
|
|
2019-03-15 13:49:58 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage.
|
2011-08-28 05:13:24 +00:00
|
|
|
*/
|
2020-11-01 13:54:07 +00:00
|
|
|
class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery
|
2011-08-28 05:13:24 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-11-05 17:48:50 +00:00
|
|
|
/**
|
|
|
|
* query_ptr
|
|
|
|
* - A query AST to interpret.
|
|
|
|
*
|
2018-02-27 19:00:55 +00:00
|
|
|
* required_result_column_names
|
2018-03-02 04:05:20 +00:00
|
|
|
* - don't calculate all columns except the specified ones from the query
|
|
|
|
* - it is used to remove calculation (and reading) of unnecessary columns from subqueries.
|
|
|
|
* empty means - use all columns.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
|
|
|
|
InterpreterSelectQuery(
|
2017-05-23 18:24:43 +00:00
|
|
|
const ASTPtr & query_ptr_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context_,
|
2019-03-15 13:49:58 +00:00
|
|
|
const SelectQueryOptions &,
|
2019-08-03 11:02:40 +00:00
|
|
|
const Names & required_result_column_names_ = Names{});
|
2018-07-17 13:09:33 +00:00
|
|
|
|
|
|
|
/// Read data not from the table specified in the query, but from the prepared source `input`.
|
|
|
|
InterpreterSelectQuery(
|
|
|
|
const ASTPtr & query_ptr_,
|
|
|
|
const Context & context_,
|
|
|
|
const BlockInputStreamPtr & input_,
|
2019-03-15 13:49:58 +00:00
|
|
|
const SelectQueryOptions & = {});
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-29 15:04:46 +00:00
|
|
|
/// Read data not from the table specified in the query, but from the prepared pipe `input`.
|
|
|
|
InterpreterSelectQuery(
|
|
|
|
const ASTPtr & query_ptr_,
|
|
|
|
const Context & context_,
|
|
|
|
Pipe input_pipe_,
|
|
|
|
const SelectQueryOptions & = {});
|
|
|
|
|
2018-07-18 12:17:48 +00:00
|
|
|
/// Read data not from the table specified in the query, but from the specified `storage_`.
|
|
|
|
InterpreterSelectQuery(
|
|
|
|
const ASTPtr & query_ptr_,
|
|
|
|
const Context & context_,
|
|
|
|
const StoragePtr & storage_,
|
2020-06-17 11:52:19 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot_ = nullptr,
|
2019-03-15 13:49:58 +00:00
|
|
|
const SelectQueryOptions & = {});
|
2018-07-18 12:17:48 +00:00
|
|
|
|
2018-06-03 20:39:06 +00:00
|
|
|
~InterpreterSelectQuery() override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-25 00:50:53 +00:00
|
|
|
/// Execute a query. Get the stream of blocks to read.
|
2017-04-01 07:20:54 +00:00
|
|
|
BlockIO execute() override;
|
|
|
|
|
2020-06-18 17:45:00 +00:00
|
|
|
/// Builds QueryPlan for current query.
|
2020-11-01 13:54:07 +00:00
|
|
|
virtual void buildQueryPlan(QueryPlan & query_plan) override;
|
2020-06-18 17:45:00 +00:00
|
|
|
|
2019-11-11 01:11:32 +00:00
|
|
|
bool ignoreLimits() const override { return options.ignore_limits; }
|
|
|
|
bool ignoreQuota() const override { return options.ignore_quota; }
|
|
|
|
|
2020-11-01 13:54:07 +00:00
|
|
|
virtual void ignoreWithTotals() override;
|
2020-03-16 18:45:39 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
const SelectQueryInfo & getQueryInfo() const { return query_info; }
|
|
|
|
|
2020-09-25 13:19:26 +00:00
|
|
|
static void addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info);
|
|
|
|
|
2011-08-28 05:13:24 +00:00
|
|
|
private:
|
2018-07-17 13:09:33 +00:00
|
|
|
InterpreterSelectQuery(
|
|
|
|
const ASTPtr & query_ptr_,
|
|
|
|
const Context & context_,
|
|
|
|
const BlockInputStreamPtr & input_,
|
2020-01-29 15:04:46 +00:00
|
|
|
std::optional<Pipe> input_pipe,
|
2018-07-17 13:09:33 +00:00
|
|
|
const StoragePtr & storage_,
|
2019-03-15 13:49:58 +00:00
|
|
|
const SelectQueryOptions &,
|
2020-06-17 11:52:19 +00:00
|
|
|
const Names & required_result_column_names = {},
|
|
|
|
const StorageMetadataPtr & metadata_snapshot_= nullptr);
|
2018-07-17 13:09:33 +00:00
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
|
2019-03-12 14:07:02 +00:00
|
|
|
|
2020-05-20 09:29:23 +00:00
|
|
|
Block getSampleBlockImpl();
|
2018-07-17 13:09:33 +00:00
|
|
|
|
2020-06-18 13:38:45 +00:00
|
|
|
void executeImpl(QueryPlan & query_plan, const BlockInputStreamPtr & prepared_input, std::optional<Pipe> prepared_pipe);
|
2018-02-23 06:00:48 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Different stages of query execution.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-28 08:24:59 +00:00
|
|
|
void executeFetchColumns(
|
|
|
|
QueryProcessingStage::Enum processing_stage,
|
2020-06-18 13:38:45 +00:00
|
|
|
QueryPlan & query_plan,
|
2020-11-03 19:05:47 +00:00
|
|
|
const PrewhereDAGInfoPtr & prewhere_info,
|
2020-11-03 11:28:28 +00:00
|
|
|
const NameSet & columns_to_remove_after_prewhere);
|
2018-09-20 17:51:42 +00:00
|
|
|
|
2020-11-03 11:28:28 +00:00
|
|
|
void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter);
|
|
|
|
void executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info);
|
2020-06-18 13:38:45 +00:00
|
|
|
void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final);
|
2020-11-03 11:28:28 +00:00
|
|
|
void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final);
|
|
|
|
void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression);
|
|
|
|
static void executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description);
|
2020-06-18 13:38:45 +00:00
|
|
|
void executeOrder(QueryPlan & query_plan, InputOrderInfoPtr sorting_info);
|
|
|
|
void executeOrderOptimized(QueryPlan & query_plan, InputOrderInfoPtr sorting_info, UInt64 limit, SortDescription & output_order_descr);
|
|
|
|
void executeWithFill(QueryPlan & query_plan);
|
|
|
|
void executeMergeSorted(QueryPlan & query_plan, const std::string & description);
|
|
|
|
void executePreLimit(QueryPlan & query_plan, bool do_not_skip_offset);
|
|
|
|
void executeLimitBy(QueryPlan & query_plan);
|
|
|
|
void executeLimit(QueryPlan & query_plan);
|
|
|
|
void executeOffset(QueryPlan & query_plan);
|
2020-11-03 11:28:28 +00:00
|
|
|
static void executeProjection(QueryPlan & query_plan, const ActionsDAGPtr & expression);
|
2020-06-18 13:38:45 +00:00
|
|
|
void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct);
|
|
|
|
void executeExtremes(QueryPlan & query_plan);
|
2020-09-02 13:04:54 +00:00
|
|
|
void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
|
2020-06-18 13:38:45 +00:00
|
|
|
void executeMergeSorted(QueryPlan & query_plan, const SortDescription & sort_description, UInt64 limit, const std::string & description);
|
2019-03-26 18:28:37 +00:00
|
|
|
|
2020-06-13 20:59:20 +00:00
|
|
|
String generateFilterActions(
|
2020-11-03 11:28:28 +00:00
|
|
|
ActionsDAGPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const;
|
2019-12-05 11:42:13 +00:00
|
|
|
|
2018-09-20 17:51:42 +00:00
|
|
|
enum class Modificator
|
|
|
|
{
|
|
|
|
ROLLUP = 0,
|
|
|
|
CUBE = 1
|
|
|
|
};
|
2018-09-20 20:57:06 +00:00
|
|
|
|
2020-06-18 13:38:45 +00:00
|
|
|
void executeRollupOrCube(QueryPlan & query_plan, Modificator modificator);
|
2019-03-26 18:28:37 +00:00
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** If there is a SETTINGS section in the SELECT query, then apply settings from it.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
2017-06-02 21:37:28 +00:00
|
|
|
* Section SETTINGS - settings for a specific query.
|
|
|
|
* Normally, the settings can be passed in other ways, not inside the query.
|
|
|
|
* But the use of this section is justified if you need to set the settings for one subquery.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void initSettings();
|
|
|
|
|
2020-07-22 17:13:05 +00:00
|
|
|
TreeRewriterResultPtr syntax_analyzer_result;
|
2019-08-14 19:30:30 +00:00
|
|
|
std::unique_ptr<SelectQueryExpressionAnalyzer> query_analyzer;
|
2019-07-19 10:14:27 +00:00
|
|
|
SelectQueryInfo query_info;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-10-03 11:58:52 +00:00
|
|
|
/// Is calculated in getSampleBlock. Is used later in readImpl.
|
2020-02-10 15:50:12 +00:00
|
|
|
ExpressionAnalysisResult analysis_result;
|
2020-06-13 20:59:20 +00:00
|
|
|
/// For row-level security.
|
2019-10-03 11:58:52 +00:00
|
|
|
FilterInfoPtr filter_info;
|
|
|
|
|
2019-10-03 15:47:42 +00:00
|
|
|
QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns;
|
|
|
|
|
2018-07-19 13:36:21 +00:00
|
|
|
/// List of columns to read to execute the query.
|
|
|
|
Names required_columns;
|
|
|
|
/// Structure of query source (table, subquery, etc).
|
|
|
|
Block source_header;
|
|
|
|
|
|
|
|
/// The subquery interpreter, if the subquery
|
|
|
|
std::unique_ptr<InterpreterSelectWithUnionQuery> interpreter_subquery;
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Table from where to read data, if not subquery.
|
2017-04-01 07:20:54 +00:00
|
|
|
StoragePtr storage;
|
2019-12-30 18:20:43 +00:00
|
|
|
StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr
|
2020-06-18 16:10:47 +00:00
|
|
|
TableLockHolder table_lock;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-21 03:26:06 +00:00
|
|
|
/// Used when we read from prepared input, not table or subquery.
|
|
|
|
BlockInputStreamPtr input;
|
2020-01-29 15:04:46 +00:00
|
|
|
std::optional<Pipe> input_pipe;
|
2018-02-21 03:26:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::Logger * log;
|
2020-06-17 11:52:19 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2011-08-28 05:13:24 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|