ClickHouse/src/Interpreters/InterpreterExplainQuery.cpp

448 lines
15 KiB
C++
Raw Normal View History

#include <Interpreters/InterpreterExplainQuery.h>
2018-09-21 15:20:23 +00:00
2021-10-15 20:18:20 +00:00
#include <QueryPipeline/BlockIO.h>
2022-05-20 19:49:31 +00:00
#include <QueryPipeline/QueryPipelineBuilder.h>
2021-09-15 19:35:48 +00:00
#include <Processors/Sources/SourceFromSingleChunk.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/InterpreterInsertQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/TableOverrideUtils.h>
2022-02-14 19:47:17 +00:00
#include <Interpreters/MergeTreeTransaction.h>
#include <Formats/FormatFactory.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
2020-02-06 00:30:14 +00:00
#include <Storages/StorageView.h>
2020-06-22 14:37:42 +00:00
#include <Processors/QueryPlan/QueryPlan.h>
2021-03-04 17:38:12 +00:00
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/printPipeline.h>
2018-09-21 15:20:23 +00:00
#include <Common/JSONBuilder.h>
2021-04-09 14:44:58 +00:00
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 : public WithContext
{
explicit Data(ContextPtr context_) : WithContext(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.getContext(), 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;
2021-09-15 19:35:48 +00:00
res.pipeline = executeImpl();
2018-09-21 15:20:23 +00:00
return res;
}
2021-07-09 10:29:45 +00:00
Block InterpreterExplainQuery::getSampleBlock(const ASTExplainQuery::ExplainKind kind)
2018-09-21 15:20:23 +00:00
{
2021-07-09 10:29:45 +00:00
if (kind == ASTExplainQuery::ExplainKind::QueryEstimates)
{
auto cols = NamesAndTypes{
{"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"parts", std::make_shared<DataTypeUInt64>()},
{"rows", std::make_shared<DataTypeUInt64>()},
{"marks", std::make_shared<DataTypeUInt64>()},
2021-07-09 10:29:45 +00:00
};
return Block({
{cols[0].type->createColumn(), cols[0].type, cols[0].name},
{cols[1].type->createColumn(), cols[1].type, cols[1].name},
{cols[2].type->createColumn(), cols[2].type, cols[2].name},
{cols[3].type->createColumn(), cols[3].type, cols[3].name},
{cols[4].type->createColumn(), cols[4].type, cols[4].name},
});
}
else
{
Block res;
ColumnWithTypeAndName col;
col.name = "explain";
col.type = std::make_shared<DataTypeString>();
col.column = col.type->createColumn();
res.insert(col);
return res;
}
2018-09-21 15:20:23 +00:00
}
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.
2022-03-10 10:04:35 +00:00
struct QueryASTSettings
{
bool graph = false;
2022-07-19 12:40:50 +00:00
bool optimize = false;
2022-03-10 10:04:35 +00:00
constexpr static char name[] = "AST";
std::unordered_map<std::string, std::reference_wrapper<bool>> boolean_settings =
{
{"graph", graph},
2022-07-19 12:40:50 +00:00
{"optimize", optimize}
2022-03-10 10:04:35 +00:00
};
};
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
/// Apply query plan optimizations.
2020-09-23 17:42:46 +00:00
bool optimize = true;
bool json = false;
2020-09-23 17:42:46 +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},
2020-09-23 17:42:46 +00:00
{"actions", query_plan_options.actions},
2021-04-16 11:11:45 +00:00
{"indexes", query_plan_options.indexes},
{"optimize", optimize},
{"json", json},
{"sortmode", query_plan_options.sort_mode},
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
2020-06-27 15:30:19 +00:00
bool has(const std::string & name_) const
2020-06-23 13:52:36 +00:00
{
2020-06-27 15:30:19 +00:00
return boolean_settings.count(name_) > 0;
2020-06-23 13:52:36 +00:00
}
2020-06-27 15:30:19 +00:00
void setBooleanSetting(const std::string & name_, bool value)
2020-06-23 13:52:36 +00:00
{
2020-06-27 15:30:19 +00:00
auto it = boolean_settings.find(name_);
2020-06-23 13:52:36 +00:00
if (it == boolean_settings.end())
2020-06-27 15:30:19 +00:00
throw Exception("Unknown setting for ExplainSettings: " + name_, ErrorCodes::LOGICAL_ERROR);
2020-06-23 13:52:36 +00:00
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)
2021-09-06 15:59:46 +00:00
throw Exception(ErrorCodes::INVALID_SETTING_VALUE,
"Invalid type {} for setting \"{}\" only boolean settings are supported",
change.value.getTypeName(), change.name);
2020-06-23 11:26:06 +00:00
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
}
2021-09-15 19:35:48 +00:00
QueryPipeline InterpreterExplainQuery::executeImpl()
2018-09-21 15:20:23 +00:00
{
2021-07-10 23:14:03 +00:00
const auto & ast = query->as<const ASTExplainQuery &>();
2020-06-22 14:37:42 +00:00
2021-07-09 10:29:45 +00:00
Block sample_block = getSampleBlock(ast.getKind());
MutableColumns res_columns = sample_block.cloneEmptyColumns();
2018-09-21 15:20:23 +00:00
2020-11-09 16:05:40 +00:00
WriteBufferFromOwnString buf;
bool single_line = false;
bool insert_buf = true;
2018-09-21 15:20:23 +00:00
SelectQueryOptions options;
options.setExplain();
switch (ast.getKind())
{
case ASTExplainQuery::ParsedAST:
{
2022-03-10 10:04:35 +00:00
auto settings = checkAndGetSettings<QueryASTSettings>(ast.getSettings());
2022-07-19 12:40:50 +00:00
if (settings.optimize)
2022-07-06 14:21:59 +00:00
{
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
ExplainAnalyzedSyntaxVisitor(data).visit(query);
}
2022-03-10 10:04:35 +00:00
if (settings.graph)
dumpASTInDotFormat(*ast.getExplainedQuery(), buf);
else
dumpAST(*ast.getExplainedQuery(), buf);
break;
}
case ASTExplainQuery::AnalyzedSyntax:
{
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING);
2020-06-26 15:07:49 +00:00
ExplainAnalyzedSyntaxVisitor::Data data(getContext());
ExplainAnalyzedSyntaxVisitor(data).visit(query);
ast.getExplainedQuery()->format(IAST::FormatSettings(buf, false));
break;
}
case ASTExplainQuery::QueryPlan:
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY);
2020-06-22 14:37:42 +00:00
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
QueryPlan plan;
2020-06-22 14:37:42 +00:00
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options);
interpreter.buildQueryPlan(plan);
2020-06-22 14:37:42 +00:00
if (settings.optimize)
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
2020-07-28 14:53:02 +00:00
if (settings.json)
{
/// Add extra layers to make plan look more like from postgres.
auto plan_map = std::make_unique<JSONBuilder::JSONMap>();
plan_map->add("Plan", plan.explainPlan(settings.query_plan_options));
auto plan_array = std::make_unique<JSONBuilder::JSONArray>();
plan_array->add(std::move(plan_map));
auto format_settings = getFormatSettings(getContext());
format_settings.json.quote_64bit_integers = false;
JSONBuilder::FormatSettings json_format_settings{.settings = format_settings};
JSONBuilder::FormatContext format_context{.out = buf};
plan_array->format(json_format_settings, format_context);
single_line = true;
}
else
plan.explainPlan(buf, settings.query_plan_options);
break;
2021-04-09 14:44:58 +00:00
}
case ASTExplainQuery::QueryPipeline:
{
if (dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
{
auto settings = checkAndGetSettings<QueryPipelineSettings>(ast.getSettings());
QueryPlan plan;
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options);
interpreter.buildQueryPlan(plan);
auto pipeline = plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(getContext()),
BuildQueryPipelineSettings::fromContext(getContext()));
if (settings.graph)
{
/// Pipe holds QueryPlan, should not go out-of-scope
2022-05-23 13:46:57 +00:00
QueryPlanResourceHolder resources;
auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline), resources);
const auto & processors = pipe.getProcessors();
if (settings.compact)
printPipelineCompact(processors, buf, settings.query_pipeline_options.header);
else
printPipeline(processors, buf);
}
else
{
plan.explainPipeline(buf, settings.query_pipeline_options);
}
}
else if (dynamic_cast<const ASTInsertQuery *>(ast.getExplainedQuery().get()))
{
InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext());
auto io = insert.execute();
printPipeline(io.pipeline.getProcessors(), buf);
}
else
throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY);
break;
}
case ASTExplainQuery::QueryEstimates:
{
if (!dynamic_cast<const ASTSelectWithUnionQuery *>(ast.getExplainedQuery().get()))
throw Exception("Only SELECT is supported for EXPLAIN ESTIMATE query", ErrorCodes::INCORRECT_QUERY);
auto settings = checkAndGetSettings<QueryPlanSettings>(ast.getSettings());
QueryPlan plan;
2020-06-25 09:39:17 +00:00
InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions());
interpreter.buildQueryPlan(plan);
// collect the selected marks, rows, parts during build query pipeline.
plan.buildQueryPipeline(
QueryPlanOptimizationSettings::fromContext(getContext()),
BuildQueryPipelineSettings::fromContext(getContext()));
2020-06-25 09:39:17 +00:00
if (settings.optimize)
plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext()));
plan.explainEstimate(res_columns);
insert_buf = false;
break;
}
case ASTExplainQuery::TableOverride:
2020-06-26 15:07:49 +00:00
{
if (auto * table_function = ast.getTableFunction()->as<ASTFunction>(); !table_function || table_function->name != "mysql")
{
throw Exception(ErrorCodes::INCORRECT_QUERY, "EXPLAIN TABLE OVERRIDE is not supported for the {}() table function", table_function->name);
}
auto storage = getContext()->getQueryContext()->executeTableFunction(ast.getTableFunction());
auto metadata_snapshot = storage->getInMemoryMetadata();
TableOverrideAnalyzer::Result override_info;
TableOverrideAnalyzer override_analyzer(ast.getTableOverride());
override_analyzer.analyze(metadata_snapshot, override_info);
override_info.appendTo(buf);
break;
2020-06-26 15:07:49 +00:00
}
2022-02-14 19:47:17 +00:00
case ASTExplainQuery::CurrentTransaction:
{
if (ast.getSettings())
throw Exception("Settings are not supported for EXPLAIN CURRENT TRANSACTION query.", ErrorCodes::UNKNOWN_SETTING);
if (auto txn = getContext()->getCurrentTransaction())
{
String dump = txn->dumpDescription();
buf.write(dump.data(), dump.size());
}
else
{
writeCString("<no current transaction>", buf);
}
break;
}
2021-07-09 10:29:45 +00:00
}
if (insert_buf)
2021-07-09 10:29:45 +00:00
{
if (single_line)
res_columns[0]->insertData(buf.str().data(), buf.str().size());
else
fillColumn(*res_columns[0], buf.str());
}
2018-09-21 15:20:23 +00:00
2021-09-15 19:35:48 +00:00
return QueryPipeline(std::make_shared<SourceFromSingleChunk>(sample_block.cloneWithColumns(std::move(res_columns))));
2018-09-21 15:20:23 +00:00
}
}