ClickHouse/src/Interpreters/InterpreterExplainQuery.cpp

287 lines
8.2 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/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>
2020-06-26 15:07:49 +00:00
#include <Processors/printPipeline.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-06-23 11:26:06 +00:00
extern const int INVALID_SETTING_VALUE;
extern const int UNKNOWN_SETTING;
2020-06-23 13:52:36 +00:00
extern const int LOGICAL_ERROR;
2020-06-22 14:37:42 +00:00
}
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-27 09:14:38 +00:00
/// Split str by line feed and write as separate row to ColumnString.
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
2020-06-23 13:52:36 +00:00
namespace
{
2020-06-23 11:26:06 +00:00
2020-06-27 09:14:38 +00:00
/// Settings. Different for each explain type.
2020-06-26 13:58:28 +00:00
struct QueryPlanSettings
2020-06-23 11:26:06 +00:00
{
2020-06-26 13:58:28 +00:00
QueryPlan::ExplainPlanOptions query_plan_options;
2020-06-26 15:07:49 +00:00
2020-06-26 13:58:28 +00:00
constexpr static char name[] = "PLAN";
2020-06-23 13:52:36 +00:00
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"header", query_plan_options.header},
2020-06-23 14:11:15 +00:00
{"description", query_plan_options.description},
{"actions", query_plan_options.actions}
2020-06-23 13:52:36 +00:00
};
2020-06-26 13:58:28 +00:00
};
struct QueryPipelineSettings
{
QueryPlan::ExplainPipelineOptions query_pipeline_options;
2020-06-26 15:07:49 +00:00
bool graph = false;
2020-06-26 17:56:33 +00:00
bool compact = true;
2020-06-26 15:07:49 +00:00
2020-06-26 13:58:28 +00:00
constexpr static char name[] = "PIPELINE";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"header", query_pipeline_options.header},
2020-06-26 15:07:49 +00:00
{"graph", graph},
2020-06-26 17:56:33 +00:00
{"compact", compact},
2020-06-26 13:58:28 +00:00
};
};
template <typename Settings>
struct ExplainSettings : public Settings
{
using Settings::boolean_settings;
2020-06-23 13:52:36 +00:00
bool has(const std::string & name) const
{
return boolean_settings.count(name) > 0;
}
void setBooleanSetting(const std::string & name, bool value)
{
auto it = boolean_settings.find(name);
if (it == boolean_settings.end())
throw Exception("Unknown setting for ExplainSettings: " + name, ErrorCodes::LOGICAL_ERROR);
it->second.get() = value;
}
2020-06-23 11:26:06 +00:00
2020-06-23 13:52:36 +00:00
std::string getSettingsList() const
2020-06-23 11:26:06 +00:00
{
std::string res;
2020-06-23 13:52:36 +00:00
for (const auto & setting : boolean_settings)
2020-06-23 11:26:06 +00:00
{
if (!res.empty())
res += ", ";
2020-06-23 13:52:36 +00:00
res += setting.first;
2020-06-23 11:26:06 +00:00
}
return res;
2020-06-23 13:52:36 +00:00
}
};
2020-06-26 13:58:28 +00:00
template <typename Settings>
ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
2020-06-23 13:52:36 +00:00
{
if (!ast_settings)
return {};
2020-06-26 13:58:28 +00:00
ExplainSettings<Settings> settings;
2020-06-23 11:26:06 +00:00
const auto & set_query = ast_settings->as<ASTSetQuery &>();
for (const auto & change : set_query.changes)
{
2020-06-23 13:52:36 +00:00
if (!settings.has(change.name))
2020-06-26 13:58:28 +00:00
throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN " + Settings::name + " query. "
"Supported settings: " + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING);
2020-06-23 11:26:06 +00:00
if (change.value.getType() != Field::Types::UInt64)
throw Exception("Invalid type " + std::string(change.value.getTypeName()) + " for setting \"" + change.name +
"\" only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE);
auto value = change.value.get<UInt64>();
if (value > 1)
throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name +
"\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE);
2020-06-23 13:52:36 +00:00
settings.setBooleanSetting(change.name, value);
2020-06-23 11:26:06 +00:00
}
return settings;
}
2020-06-26 13:58:28 +00:00
}
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)
{
2020-06-26 15:07:49 +00:00
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING);
dumpAST(ast, ss);
}
else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax)
{
2020-06-26 15:07:49 +00:00
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING);
ExplainAnalyzedSyntaxVisitor::Data data{.context = context};
ExplainAnalyzedSyntaxVisitor(data).visit(query);
2020-06-23 11:26:06 +00:00
ast.getExplainedQuery()->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);
2020-06-26 13:58:28 +00:00
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
2020-06-22 14:37:42 +00:00
QueryPlan plan;
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
interpreter.buildQueryPlan(plan);
WriteBufferFromOStream buffer(ss);
2020-06-26 13:58:28 +00:00
plan.explainPlan(buffer, settings.query_plan_options);
2020-06-22 14:37:42 +00:00
}
2020-06-25 09:39:17 +00:00
else if (ast.getKind() == ASTExplainQuery::QueryPipeline)
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY);
2020-06-26 13:58:28 +00:00
auto settings = checkAndGetSettings<QueryPipelineSettings>(ast.getSettings());
2020-06-25 09:39:17 +00:00
QueryPlan plan;
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions());
interpreter.buildQueryPlan(plan);
2020-06-26 15:07:49 +00:00
auto pipeline = plan.buildQueryPipeline();
2020-06-25 09:39:17 +00:00
WriteBufferFromOStream buffer(ss);
2020-06-26 15:07:49 +00:00
if (settings.graph)
{
2020-06-26 17:56:33 +00:00
if (settings.compact)
printPipelineCompact(pipeline->getProcessors(), buffer, settings.query_pipeline_options.header);
else
printPipeline(pipeline->getProcessors(), buffer);
2020-06-26 15:07:49 +00:00
}
else
{
plan.explainPipeline(buffer, settings.query_pipeline_options);
}
2020-06-25 09:39:17 +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)));
}
}