ClickHouse/src/Processors/QueryPlan/QueryPlan.h

122 lines
3.4 KiB
C++
Raw Normal View History

2020-06-08 09:14:58 +00:00
#pragma once
#include <Core/Names.h>
#include <Interpreters/Context_fwd.h>
2021-07-09 10:29:45 +00:00
#include <Columns/IColumn.h>
#include <list>
#include <memory>
#include <set>
#include <vector>
2020-06-08 09:14:58 +00:00
namespace DB
{
class DataStream;
class IQueryPlanStep;
using QueryPlanStepPtr = std::unique_ptr<IQueryPlanStep>;
class QueryPipeline;
using QueryPipelinePtr = std::unique_ptr<QueryPipeline>;
2020-06-22 14:37:42 +00:00
class WriteBuffer;
class QueryPlan;
using QueryPlanPtr = std::unique_ptr<QueryPlan>;
2020-10-07 11:26:29 +00:00
class Pipe;
2021-03-04 17:38:12 +00:00
struct QueryPlanOptimizationSettings;
struct BuildQueryPipelineSettings;
namespace JSONBuilder
{
class IItem;
using ItemPtr = std::unique_ptr<IItem>;
}
2020-06-08 09:14:58 +00:00
/// A tree of query steps.
/// The goal of QueryPlan is to build QueryPipeline.
/// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimizations.
2020-06-08 09:14:58 +00:00
class QueryPlan
{
public:
2020-06-22 14:37:42 +00:00
QueryPlan();
~QueryPlan();
2020-09-15 17:13:13 +00:00
QueryPlan(QueryPlan &&);
QueryPlan & operator=(QueryPlan &&);
void unitePlans(QueryPlanStepPtr step, std::vector<QueryPlanPtr> plans);
2020-06-08 09:14:58 +00:00
void addStep(QueryPlanStepPtr step);
bool isInitialized() const { return root != nullptr; } /// Tree is not empty
bool isCompleted() const; /// Tree is not empty and root hasOutputStream()
const DataStream & getCurrentDataStream() const; /// Checks that (isInitialized() && !isCompleted())
void optimize(const QueryPlanOptimizationSettings & optimization_settings);
2020-07-23 10:20:38 +00:00
2021-03-04 17:38:12 +00:00
QueryPipelinePtr buildQueryPipeline(
const QueryPlanOptimizationSettings & optimization_settings,
const BuildQueryPipelineSettings & build_pipeline_settings);
2020-06-08 09:14:58 +00:00
2020-10-07 11:26:29 +00:00
/// If initialized, build pipeline and convert to pipe. Otherwise, return empty pipe.
2021-03-04 17:38:12 +00:00
Pipe convertToPipe(
const QueryPlanOptimizationSettings & optimization_settings,
const BuildQueryPipelineSettings & build_pipeline_settings);
2020-10-07 11:26:29 +00:00
2020-06-26 13:58:28 +00:00
struct ExplainPlanOptions
2020-06-23 11:26:06 +00:00
{
2020-06-26 13:58:28 +00:00
/// Add output header to step.
2020-06-23 11:26:06 +00:00
bool header = false;
2020-06-26 13:58:28 +00:00
/// Add description of step.
2020-06-23 14:11:15 +00:00
bool description = true;
2020-06-26 13:58:28 +00:00
/// Add detailed information about step actions.
bool actions = false;
2021-04-16 11:11:45 +00:00
/// Add information about indexes actions.
bool indexes = false;
2020-06-23 11:26:06 +00:00
};
2020-06-26 13:58:28 +00:00
struct ExplainPipelineOptions
{
/// Show header of output ports.
bool header = false;
};
JSONBuilder::ItemPtr explainPlan(const ExplainPlanOptions & options);
2020-06-26 13:58:28 +00:00
void explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options);
void explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptions & options);
2021-07-10 23:14:03 +00:00
void explainEstimate(MutableColumns & columns);
2020-06-22 14:37:42 +00:00
/// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines.
/// TODO: make it in a better way.
void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; }
2020-10-22 11:08:12 +00:00
size_t getMaxThreads() const { return max_threads; }
void addInterpreterContext(std::shared_ptr<Context> context);
/// Tree node. Step and it's children.
2020-06-08 09:14:58 +00:00
struct Node
{
QueryPlanStepPtr step;
2020-06-16 14:11:19 +00:00
std::vector<Node *> children = {};
2020-06-08 09:14:58 +00:00
};
using Nodes = std::list<Node>;
2020-08-12 13:30:02 +00:00
private:
Nodes nodes;
2020-06-08 09:14:58 +00:00
Node * root = nullptr;
void checkInitialized() const;
void checkNotCompleted() const;
/// Those fields are passed to QueryPipeline.
size_t max_threads = 0;
std::vector<std::shared_ptr<Context>> interpreter_context;
2020-06-08 09:14:58 +00:00
};
2020-12-24 09:06:12 +00:00
std::string debugExplainStep(const IQueryPlanStep & step);
2020-12-24 04:03:33 +00:00
2020-06-08 09:14:58 +00:00
}