2019-02-11 19:53:55 +00:00
|
|
|
#include <Interpreters/InterpreterExplainQuery.h>
|
2018-09-21 15:20:23 +00:00
|
|
|
|
|
|
|
#include <DataStreams/BlockIO.h>
|
2019-02-11 19:53:55 +00:00
|
|
|
#include <DataStreams/OneBlockInputStream.h>
|
2018-09-21 15:20:23 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
2020-02-05 08:22:25 +00:00
|
|
|
#include <Interpreters/InDepthNodeVisitor.h>
|
2019-02-11 19:53:55 +00:00
|
|
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
2020-06-15 12:36:10 +00:00
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2018-10-09 14:32:11 +00:00
|
|
|
#include <Parsers/DumpASTNode.h>
|
2019-02-11 19:53:55 +00:00
|
|
|
#include <Parsers/queryToString.h>
|
2020-02-05 08:22:25 +00:00
|
|
|
#include <Parsers/ASTExplainQuery.h>
|
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2020-06-15 12:36:10 +00:00
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
2020-06-22 14:37:42 +00:00
|
|
|
#include <IO/WriteBufferFromOStream.h>
|
2020-02-06 00:30:14 +00:00
|
|
|
|
2020-02-05 08:22:25 +00:00
|
|
|
#include <Storages/StorageView.h>
|
2019-02-11 19:53:55 +00:00
|
|
|
#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;
|
|
|
|
}
|
|
|
|
|
2020-02-05 08:22:25 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
struct ExplainAnalyzedSyntaxMatcher
|
|
|
|
{
|
|
|
|
struct Data
|
|
|
|
{
|
|
|
|
const Context & context;
|
|
|
|
};
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
static bool needChildVisit(ASTPtr & node, ASTPtr &)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
return !node->as<ASTSelectQuery>();
|
2020-02-05 08:22:25 +00:00
|
|
|
}
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
static void visit(ASTPtr & ast, Data & data)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
if (auto * select = ast->as<ASTSelectQuery>())
|
|
|
|
visit(*select, ast, data);
|
2020-02-05 08:22:25 +00:00
|
|
|
}
|
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
InterpreterSelectQuery interpreter(
|
|
|
|
node, data.context, SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze().modify());
|
2020-02-05 08:22:25 +00:00
|
|
|
|
2020-06-15 12:36:10 +00:00
|
|
|
const SelectQueryInfo & query_info = interpreter.getQueryInfo();
|
|
|
|
if (query_info.view_query)
|
2020-02-05 08:22:25 +00:00
|
|
|
{
|
2020-06-15 12:36:10 +00:00
|
|
|
ASTPtr tmp;
|
|
|
|
StorageView::replaceWithSubquery(select, query_info.view_query->clone(), tmp);
|
2020-02-05 08:22:25 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
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;
|
2019-02-11 19:53:55 +00:00
|
|
|
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()
|
|
|
|
{
|
2019-03-15 16:14:13 +00:00
|
|
|
const auto & ast = query->as<ASTExplainQuery &>();
|
2020-06-22 14:37:42 +00:00
|
|
|
|
2019-02-11 19:53:55 +00:00
|
|
|
Block sample_block = getSampleBlock();
|
|
|
|
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
2018-09-21 15:20:23 +00:00
|
|
|
|
|
|
|
std::stringstream ss;
|
|
|
|
|
2019-02-11 19:53:55 +00:00
|
|
|
if (ast.getKind() == ASTExplainQuery::ParsedAST)
|
|
|
|
{
|
|
|
|
dumpAST(ast, ss);
|
|
|
|
}
|
|
|
|
else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax)
|
|
|
|
{
|
2020-02-05 08:22:25 +00:00
|
|
|
ExplainAnalyzedSyntaxVisitor::Data data{.context = context};
|
|
|
|
ExplainAnalyzedSyntaxVisitor(data).visit(query);
|
|
|
|
|
|
|
|
ast.children.at(0)->format(IAST::FormatSettings(ss, false));
|
2019-02-11 19:53:55 +00:00
|
|
|
}
|
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);
|
|
|
|
}
|
2019-02-11 19:53:55 +00:00
|
|
|
|
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)));
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|