ClickHouse/src/Interpreters/InterpreterSelectQuery.h

194 lines
7.3 KiB
C++
Raw Normal View History

2011-08-28 05:13:24 +00:00
#pragma once
#include <memory>
#include <Core/QueryProcessingStage.h>
2019-03-18 10:36:13 +00:00
#include <Parsers/ASTSelectQuery.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <Interpreters/ExpressionActions.h>
2018-02-23 06:00:48 +00:00
#include <Interpreters/ExpressionAnalyzer.h>
2019-03-18 12:05:51 +00:00
#include <Interpreters/IInterpreter.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/TableStructureLockHolder.h>
2019-12-10 23:18:24 +00:00
#include <Storages/ReadInOrderOptimizer.h>
2020-03-13 10:30:55 +00:00
#include <Interpreters/StorageID.h>
2011-08-28 05:13:24 +00:00
2019-03-26 18:28:37 +00:00
#include <Processors/QueryPipeline.h>
2019-10-27 18:12:40 +00:00
#include <Columns/FilterDescription.h>
namespace Poco { class Logger; }
2011-08-28 05:13:24 +00:00
namespace DB
{
2016-03-07 04:35:06 +00:00
struct SubqueryForSet;
class InterpreterSelectWithUnionQuery;
class Context;
2015-05-06 23:35:37 +00:00
struct SyntaxAnalyzerResult;
using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
2015-05-06 23:35:37 +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
*/
2019-03-18 12:05:51 +00:00
class InterpreterSelectQuery : public IInterpreter
2011-08-28 05:13:24 +00:00
{
public:
/**
* 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.
*/
InterpreterSelectQuery(
2017-05-23 18:24:43 +00:00
const ASTPtr & query_ptr_,
const Context & context_,
const SelectQueryOptions &,
2019-08-03 11:02:40 +00:00
const Names & required_result_column_names_ = Names{});
/// 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_,
const SelectQueryOptions & = {});
/// 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 & = {});
/// 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_,
const SelectQueryOptions & = {});
~InterpreterSelectQuery() override;
2018-02-25 00:50:53 +00:00
/// Execute a query. Get the stream of blocks to read.
BlockIO execute() override;
bool ignoreLimits() const override { return options.ignore_limits; }
bool ignoreQuota() const override { return options.ignore_quota; }
Block getSampleBlock();
2018-02-25 06:34:20 +00:00
void ignoreWithTotals();
ASTPtr getQuery() const { return query_ptr; }
size_t getMaxStreams() const { return max_streams; }
2011-08-28 05:13:24 +00:00
private:
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
const BlockInputStreamPtr & input_,
std::optional<Pipe> input_pipe,
const StoragePtr & storage_,
const SelectQueryOptions &,
const Names & required_result_column_names = {});
ASTSelectQuery & getSelectQuery() { return query_ptr->as<ASTSelectQuery &>(); }
2019-03-12 14:07:02 +00:00
Block getSampleBlockImpl();
2020-05-28 08:24:59 +00:00
void executeImpl(QueryPipeline & pipeline, 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.
2020-05-28 08:24:59 +00:00
void executeFetchColumns(
QueryProcessingStage::Enum processing_stage,
QueryPipeline & pipeline,
2019-12-10 23:18:24 +00:00
const PrewhereInfoPtr & prewhere_info,
2020-05-28 08:24:59 +00:00
const Names & columns_to_remove_after_prewhere);
2018-09-20 17:51:42 +00:00
2020-05-12 14:50:13 +00:00
void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter);
2020-05-13 13:49:10 +00:00
void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info);
2019-03-26 18:28:37 +00:00
void executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final);
void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
2020-03-18 00:57:00 +00:00
static void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
2020-05-13 13:49:10 +00:00
void executeOrder(QueryPipeline & pipeline, InputOrderInfoPtr sorting_info);
void executeOrderOptimized(QueryPipeline & pipeline, InputOrderInfoPtr sorting_info, UInt64 limit, SortDescription & output_order_descr);
void executeWithFill(QueryPipeline & pipeline);
2019-03-26 18:28:37 +00:00
void executeMergeSorted(QueryPipeline & pipeline);
2020-03-13 13:44:08 +00:00
void executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset);
2019-03-26 18:28:37 +00:00
void executeLimitBy(QueryPipeline & pipeline);
void executeLimit(QueryPipeline & pipeline);
2020-05-06 06:50:55 +00:00
void executeOffset(QueryPipeline & pipeline);
2020-03-18 00:57:00 +00:00
static void executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
2019-03-26 18:28:37 +00:00
void executeDistinct(QueryPipeline & pipeline, bool before_order, Names columns);
void executeExtremes(QueryPipeline & pipeline);
void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
void executeMergeSorted(QueryPipeline & pipeline, const SortDescription & sort_description, UInt64 limit);
2019-03-26 18:28:37 +00:00
String generateFilterActions(
ExpressionActionsPtr & 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
2019-03-26 18:28:37 +00:00
void executeRollupOrCube(QueryPipeline & pipeline, Modificator modificator);
2017-06-02 21:37:28 +00:00
/** If there is a SETTINGS section in the SELECT query, then apply settings from it.
*
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.
*/
void initSettings();
SelectQueryOptions options;
ASTPtr query_ptr;
std::shared_ptr<Context> context;
SyntaxAnalyzerResultPtr syntax_analyzer_result;
std::unique_ptr<SelectQueryExpressionAnalyzer> query_analyzer;
2019-07-19 10:14:27 +00:00
SelectQueryInfo query_info;
/// Is calculated in getSampleBlock. Is used later in readImpl.
ExpressionAnalysisResult analysis_result;
/// For row-level security.
FilterInfoPtr filter_info;
2019-10-03 15:47:42 +00:00
QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns;
/// How many streams we ask for storage to produce, and in how many threads we will do further processing.
size_t max_streams = 1;
/// List of columns to read to execute the query.
Names required_columns;
/// Structure of query source (table, subquery, etc).
Block source_header;
/// Structure of query result.
Block result_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.
StoragePtr storage;
2019-12-30 18:20:43 +00:00
StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr
TableStructureReadLockHolder table_lock;
2018-02-21 03:26:06 +00:00
/// Used when we read from prepared input, not table or subquery.
BlockInputStreamPtr input;
std::optional<Pipe> input_pipe;
2018-02-21 03:26:06 +00:00
Poco::Logger * log;
2011-08-28 05:13:24 +00:00
};
}