2011-08-28 05:13:24 +00:00
|
|
|
#pragma once
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Core/QueryProcessingStage.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/IInterpreter.h>
|
2018-02-23 06:00:48 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
2011-08-28 05:13:24 +00:00
|
|
|
|
2017-01-21 04:24:28 +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 ExpressionAnalyzer;
|
|
|
|
class ASTSelectQuery;
|
2016-03-07 04:35:06 +00:00
|
|
|
struct SubqueryForSet;
|
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
|
|
|
*/
|
2015-06-18 02:11:05 +00:00
|
|
|
class InterpreterSelectQuery : public IInterpreter
|
2011-08-28 05:13:24 +00:00
|
|
|
{
|
|
|
|
public:
|
2017-11-05 17:48:50 +00:00
|
|
|
/**
|
|
|
|
* query_ptr
|
|
|
|
* - A query AST to interpret.
|
|
|
|
*
|
|
|
|
* to_stage
|
2017-06-02 21:37:28 +00:00
|
|
|
* - the stage to which the query is to be executed. By default - till to the end.
|
|
|
|
* You can perform till the intermediate aggregation state, which are combined from different servers for distributed query processing.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
|
|
|
* subquery_depth
|
2018-02-28 01:29:55 +00:00
|
|
|
* - to control the limit on the depth of nesting of subqueries. For subqueries, a value that is incremented by one is passed;
|
2018-02-23 06:37:53 +00:00
|
|
|
* for INSERT SELECT, a value 1 is passed instead of 0.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
|
|
|
* input
|
2018-02-27 19:00:55 +00:00
|
|
|
* - if given - read not from the table specified in the query, but from prepared source.
|
2017-04-01 07:20:54 +00:00
|
|
|
*
|
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_,
|
2018-02-27 19:00:55 +00:00
|
|
|
const Names & required_result_column_names = Names{},
|
2017-04-01 07:20:54 +00:00
|
|
|
QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete,
|
|
|
|
size_t subquery_depth_ = 0,
|
2018-03-12 15:14:26 +00:00
|
|
|
const BlockInputStreamPtr & input = nullptr,
|
|
|
|
bool only_analyze = false);
|
2017-04-01 07:20:54 +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;
|
|
|
|
|
2018-02-25 00:50:53 +00:00
|
|
|
/// Execute the query and return multuple streams for parallel processing.
|
|
|
|
BlockInputStreams executeWithMultipleStreams();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
Block getSampleBlock();
|
|
|
|
|
|
|
|
static Block getSampleBlock(
|
2017-05-23 18:24:43 +00:00
|
|
|
const ASTPtr & query_ptr_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context_);
|
2015-07-13 15:02:29 +00:00
|
|
|
|
2018-02-25 06:34:20 +00:00
|
|
|
void ignoreWithTotals();
|
|
|
|
|
2011-08-28 05:13:24 +00:00
|
|
|
private:
|
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;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
struct OnlyAnalyzeTag {};
|
|
|
|
InterpreterSelectQuery(
|
|
|
|
OnlyAnalyzeTag,
|
2017-05-23 18:24:43 +00:00
|
|
|
const ASTPtr & query_ptr_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context_);
|
|
|
|
|
2018-02-27 19:00:55 +00:00
|
|
|
void init(const Names & required_result_column_names);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-28 01:29:55 +00:00
|
|
|
void executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-02-23 06:00:48 +00:00
|
|
|
|
|
|
|
struct AnalysisResult
|
|
|
|
{
|
|
|
|
bool has_join = false;
|
|
|
|
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
|
|
|
|
|
|
|
ExpressionActionsPtr before_join; /// including JOIN
|
|
|
|
ExpressionActionsPtr before_where;
|
|
|
|
ExpressionActionsPtr before_aggregation;
|
|
|
|
ExpressionActionsPtr before_having;
|
|
|
|
ExpressionActionsPtr before_order_and_select;
|
2018-03-01 05:24:56 +00:00
|
|
|
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;
|
|
|
|
|
|
|
|
/// 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;
|
|
|
|
};
|
|
|
|
|
|
|
|
AnalysisResult analyzeExpressions(QueryProcessingStage::Enum from_stage);
|
|
|
|
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/** From which table to read. With JOIN, the "left" table is returned.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
|
|
|
void getDatabaseAndTableNames(String & database_name, String & table_name);
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// Different stages of query execution.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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
|
|
|
|
2018-02-28 01:29:55 +00:00
|
|
|
/// Fetch data from the table. Returns the stage to which the query was processed in Storage.
|
|
|
|
QueryProcessingStage::Enum executeFetchColumns(Pipeline & pipeline, bool dry_run);
|
2018-02-26 21:00:42 +00:00
|
|
|
|
2018-02-21 03:26:06 +00:00
|
|
|
void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression);
|
|
|
|
void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
|
|
|
|
void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final);
|
|
|
|
void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row);
|
|
|
|
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);
|
2017-04-01 07:20:54 +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();
|
|
|
|
|
|
|
|
ASTPtr query_ptr;
|
|
|
|
ASTSelectQuery & query;
|
|
|
|
Context context;
|
|
|
|
QueryProcessingStage::Enum to_stage;
|
|
|
|
size_t subquery_depth;
|
|
|
|
std::unique_ptr<ExpressionAnalyzer> query_analyzer;
|
|
|
|
|
2017-07-03 21:04:10 +00:00
|
|
|
/// How many streams we ask for storage to produce, and in how many threads we will do further processing.
|
|
|
|
size_t max_streams = 1;
|
|
|
|
|
2017-06-02 21:37:28 +00:00
|
|
|
/// The object was created only for query analysis.
|
2017-04-01 07:20:54 +00:00
|
|
|
bool only_analyze = false;
|
|
|
|
|
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;
|
|
|
|
TableStructureReadLockPtr table_lock;
|
|
|
|
|
2018-02-21 03:26:06 +00:00
|
|
|
/// Used when we read from prepared input, not table or subquery.
|
|
|
|
BlockInputStreamPtr input;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::Logger * log;
|
2011-08-28 05:13:24 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|