ClickHouse/dbms/src/Interpreters/InterpreterSelectQuery.h

239 lines
8.6 KiB
C++
Raw Normal View History

2011-08-28 05:13:24 +00:00
#pragma once
#include <memory>
#include <Core/QueryProcessingStage.h>
#include <Interpreters/Context.h>
#include <Interpreters/IInterpreter.h>
2018-02-23 06:00:48 +00:00
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/SelectQueryOptions.h>
#include <DataStreams/IBlockInputStream.h>
#include <Storages/SelectQueryInfo.h>
2011-08-28 05:13:24 +00:00
namespace Poco { class Logger; }
2011-08-28 05:13:24 +00:00
namespace DB
{
2015-05-06 23:35:37 +00:00
class ASTSelectQuery;
2016-03-07 04:35:06 +00:00
struct SubqueryForSet;
class InterpreterSelectWithUnionQuery;
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
*/
class InterpreterSelectQuery : public IInterpreter, private SelectQueryOptions
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 &,
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 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;
2018-02-25 00:50:53 +00:00
/// Execute the query and return multuple streams for parallel processing.
BlockInputStreams executeWithMultipleStreams();
Block getSampleBlock();
2018-02-25 06:34:20 +00:00
void ignoreWithTotals();
ASTPtr getQuery() const { return query_ptr; }
2011-08-28 05:13:24 +00:00
private:
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
const Context & context_,
const BlockInputStreamPtr & input_,
const StoragePtr & storage_,
const SelectQueryOptions &,
const Names & required_result_column_names = {});
2018-02-21 03:26:06 +00:00
struct Pipeline
{
/** Streams of data.
* The source data streams are produced in the executeFetchColumns function.
* Then they are converted (wrapped in other streams) using the `execute*` functions,
* to get the whole pipeline running the query.
*/
BlockInputStreams streams;
/** When executing FULL or RIGHT JOIN, there will be a data stream from which you can read "not joined" rows.
* It has a special meaning, since reading from it should be done after reading from the main streams.
2018-03-02 04:05:20 +00:00
* It is appended to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream.
2018-02-21 03:26:06 +00:00
*/
BlockInputStreamPtr stream_with_non_joined_data;
BlockInputStreamPtr & firstStream() { return streams.at(0); }
template <typename Transform>
void transform(Transform && transform)
{
for (auto & stream : streams)
transform(stream);
if (stream_with_non_joined_data)
transform(stream_with_non_joined_data);
}
bool hasMoreThanOneStream() const
{
return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1;
}
};
ASTSelectQuery & selectQuery();
2018-08-27 17:42:13 +00:00
void executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run);
2018-02-23 06:00:48 +00:00
struct AnalysisResult
{
bool hasJoin() const { return before_join.get(); }
2018-02-23 06:00:48 +00:00
bool has_where = false;
bool need_aggregate = false;
bool has_having = false;
bool has_order_by = false;
2018-03-01 06:07:04 +00:00
bool has_limit_by = false;
2018-02-23 06:00:48 +00:00
bool remove_where_filter = false;
2018-02-23 06:00:48 +00:00
ExpressionActionsPtr before_join; /// including JOIN
ExpressionActionsPtr before_where;
ExpressionActionsPtr before_aggregation;
ExpressionActionsPtr before_having;
ExpressionActionsPtr before_order_and_select;
ExpressionActionsPtr before_limit_by;
2018-02-23 06:00:48 +00:00
ExpressionActionsPtr final_projection;
/// Columns from the SELECT list, before renaming them to aliases.
Names selected_columns;
/// Columns will be removed after prewhere actions execution.
Names columns_to_remove_after_prewhere;
2018-02-23 06:00:48 +00:00
/// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
bool first_stage = false;
/// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing.
bool second_stage = false;
SubqueriesForSets subqueries_for_sets;
PrewhereInfoPtr prewhere_info;
2018-02-23 06:00:48 +00:00
};
AnalysisResult analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run);
2018-02-23 06:00:48 +00:00
2017-06-02 21:37:28 +00:00
/** From which table to read. With JOIN, the "left" table is returned.
*/
void getDatabaseAndTableNames(String & database_name, String & table_name);
2017-06-02 21:37:28 +00:00
/// Different stages of query execution.
2018-02-28 04:51:09 +00:00
/// dry_run - don't read from table, use empty header block instead.
2018-02-28 01:29:55 +00:00
void executeWithMultipleStreamsImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run);
2018-02-21 03:26:06 +00:00
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere);
2018-02-26 21:00:42 +00:00
void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter);
2018-02-21 03:26:06 +00:00
void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final);
2018-08-24 15:00:00 +00:00
void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
2018-02-21 03:26:06 +00:00
void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeOrder(Pipeline & pipeline);
void executeMergeSorted(Pipeline & pipeline);
void executePreLimit(Pipeline & pipeline);
void executeUnion(Pipeline & pipeline);
void executeLimitBy(Pipeline & pipeline);
void executeLimit(Pipeline & pipeline);
void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeDistinct(Pipeline & pipeline, bool before_order, Names columns);
2018-02-28 02:32:34 +00:00
void executeExtremes(Pipeline & pipeline);
2018-02-21 03:26:06 +00:00
void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
2018-09-20 17:51:42 +00:00
/// If pipeline has several streams with different headers, add ConvertingBlockInputStream to first header.
void unifyStreams(Pipeline & pipeline);
2018-09-20 17:51:42 +00:00
enum class Modificator
{
ROLLUP = 0,
CUBE = 1
};
2018-09-20 20:57:06 +00:00
2018-09-20 17:51:42 +00:00
void executeRollupOrCube(Pipeline & 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();
ASTPtr query_ptr;
Context context;
NamesAndTypesList source_columns;
SyntaxAnalyzerResultPtr syntax_analyzer_result;
std::unique_ptr<ExpressionAnalyzer> query_analyzer;
/// 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;
TableStructureReadLockHolder table_lock;
2018-02-21 03:26:06 +00:00
/// Used when we read from prepared input, not table or subquery.
BlockInputStreamPtr input;
Poco::Logger * log;
2011-08-28 05:13:24 +00:00
};
}