ClickHouse/src/Interpreters/InterpreterExplainQuery.cpp

149 lines
3.8 KiB
C++
Raw Normal View History

#include <Interpreters/InterpreterExplainQuery.h>
2018-09-21 15:20:23 +00:00
#include <DataStreams/BlockIO.h>
#include <DataStreams/OneBlockInputStream.h>
2018-09-21 15:20:23 +00:00
#include <DataTypes/DataTypeString.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/Context.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
2020-06-22 14:37:42 +00:00
#include <IO/WriteBufferFromOStream.h>
2020-02-06 00:30:14 +00:00
#include <Storages/StorageView.h>
#include <sstream>
2020-06-22 14:37:42 +00:00
#include <Processors/QueryPlan/QueryPlan.h>
2018-09-21 15:20:23 +00:00
namespace DB
{
2020-06-22 14:37:42 +00:00
namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
}
namespace
{
struct ExplainAnalyzedSyntaxMatcher
{
struct Data
{
const Context & context;
};
static bool needChildVisit(ASTPtr & node, ASTPtr &)
{
return !node->as<ASTSelectQuery>();
}
static void visit(ASTPtr & ast, Data & data)
{
if (auto * select = ast->as<ASTSelectQuery>())
visit(*select, ast, data);
}
static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data)
{
InterpreterSelectQuery interpreter(
node, data.context, SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify());
const SelectQueryInfo & query_info = interpreter.getQueryInfo();
if (query_info.view_query)
{
ASTPtr tmp;
StorageView::replaceWithSubquery(select, query_info.view_query->clone(), tmp);
}
}
};
using ExplainAnalyzedSyntaxVisitor = InDepthNodeVisitor<ExplainAnalyzedSyntaxMatcher, true>;
}
2018-09-21 15:20:23 +00:00
BlockIO InterpreterExplainQuery::execute()
{
BlockIO res;
res.in = executeImpl();
return res;
}
Block InterpreterExplainQuery::getSampleBlock()
{
Block block;
ColumnWithTypeAndName col;
col.name = "explain";
2018-09-21 15:20:23 +00:00
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();
block.insert(col);
return block;
}
2020-06-22 14:37:42 +00:00
static void fillColumn(IColumn & column, const std::string & str)
{
size_t start = 0;
size_t end = 0;
size_t size = str.size();
while (end < size)
{
if (str[end] == '\n')
{
column.insertData(str.data() + start, end - start);
start = end + 1;
}
++end;
}
if (start < end)
column.insertData(str.data() + start, end - start);
}
2018-09-21 15:20:23 +00:00
BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
{
const auto & ast = query->as<ASTExplainQuery &>();
2020-06-22 14:37:42 +00:00
Block sample_block = getSampleBlock();
MutableColumns res_columns = sample_block.cloneEmptyColumns();
2018-09-21 15:20:23 +00:00
std::stringstream ss;
if (ast.getKind() == ASTExplainQuery::ParsedAST)
{
dumpAST(ast, ss);
}
else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax)
{
ExplainAnalyzedSyntaxVisitor::Data data{.context = context};
ExplainAnalyzedSyntaxVisitor(data).visit(query);
ast.children.at(0)->format(IAST::FormatSettings(ss, false));
}
2020-06-22 14:37:42 +00:00
else if (ast.getKind() == ASTExplainQuery::QueryPlan)
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY);
QueryPlan plan;
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
interpreter.buildQueryPlan(plan);
WriteBufferFromOStream buffer(ss);
plan.explain(buffer);
}
2020-06-22 14:37:42 +00:00
fillColumn(*res_columns[0], ss.str());
2018-09-21 15:20:23 +00:00
return std::make_shared<OneBlockInputStream>(sample_block.cloneWithColumns(std::move(res_columns)));
}
}