Graph for pipeline.

This commit is contained in:
Nikolai Kochetov 2020-06-26 18:07:49 +03:00
parent 79bcc8309c
commit f4e944cd35
2 changed files with 25 additions and 2 deletions

View File

@ -16,6 +16,7 @@
#include <Storages/StorageView.h> #include <Storages/StorageView.h>
#include <sstream> #include <sstream>
#include <Processors/QueryPlan/QueryPlan.h> #include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/printPipeline.h>
namespace DB namespace DB
{ {
@ -114,6 +115,7 @@ namespace
struct QueryPlanSettings struct QueryPlanSettings
{ {
QueryPlan::ExplainPlanOptions query_plan_options; QueryPlan::ExplainPlanOptions query_plan_options;
constexpr static char name[] = "PLAN"; constexpr static char name[] = "PLAN";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings = std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
@ -127,11 +129,14 @@ struct QueryPlanSettings
struct QueryPipelineSettings struct QueryPipelineSettings
{ {
QueryPlan::ExplainPipelineOptions query_pipeline_options; QueryPlan::ExplainPipelineOptions query_pipeline_options;
bool graph = false;
constexpr static char name[] = "PIPELINE"; constexpr static char name[] = "PIPELINE";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings = std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{ {
{"header", query_pipeline_options.header}, {"header", query_pipeline_options.header},
{"graph", graph},
}; };
}; };
@ -212,10 +217,16 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
if (ast.getKind() == ASTExplainQuery::ParsedAST) if (ast.getKind() == ASTExplainQuery::ParsedAST)
{ {
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING);
dumpAST(ast, ss); dumpAST(ast, ss);
} }
else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax) else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax)
{ {
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING);
ExplainAnalyzedSyntaxVisitor::Data data{.context = context}; ExplainAnalyzedSyntaxVisitor::Data data{.context = context};
ExplainAnalyzedSyntaxVisitor(data).visit(query); ExplainAnalyzedSyntaxVisitor(data).visit(query);
@ -245,10 +256,18 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions()); InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
interpreter.buildQueryPlan(plan); interpreter.buildQueryPlan(plan);
plan.buildQueryPipeline(); auto pipeline = plan.buildQueryPipeline();
WriteBufferFromOStream buffer(ss); WriteBufferFromOStream buffer(ss);
plan.explainPipeline(buffer, settings.query_pipeline_options);
if (settings.graph)
{
printPipeline(pipeline->getProcessors(), buffer);
}
else
{
plan.explainPipeline(buffer, settings.query_pipeline_options);
}
} }
fillColumn(*res_columns[0], ss.str()); fillColumn(*res_columns[0], ss.str());

View File

@ -82,6 +82,7 @@ public:
Processors * getCollectedProcessors() const { return collected_processors; } Processors * getCollectedProcessors() const { return collected_processors; }
Processors * setCollectedProcessors(Processors * collected_processors); Processors * setCollectedProcessors(Processors * collected_processors);
Processors & get() { return processors; } Processors & get() { return processors; }
const Processors & get() const { return processors; }
Processors detach() { return std::move(processors); } Processors detach() { return std::move(processors); }
private: private:
/// All added processors. /// All added processors.
@ -196,6 +197,9 @@ public:
Pipe getPipe() &&; Pipe getPipe() &&;
Pipes getPipes() &&; Pipes getPipes() &&;
/// Get internal processors.
const Processors & getProcessors() const { return processors.get(); }
private: private:
/// Destruction order: processors, header, locks, temporary storages, local contexts /// Destruction order: processors, header, locks, temporary storages, local contexts