Header for explain.

This commit is contained in:
Nikolai Kochetov 2020-06-23 14:26:06 +03:00
parent e1481ccfc2
commit 47d9f4af9c
7 changed files with 155 additions and 27 deletions

View File

@ -28,9 +28,12 @@ bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) cons
}
void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const
void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out, bool escaped) const
{
out << name;
if (escaped)
writeEscapedString(name, out);
else
out << name;
if (type)
out << ' ' << type->getName();

View File

@ -33,7 +33,7 @@ struct ColumnWithTypeAndName
ColumnWithTypeAndName cloneEmpty() const;
bool operator==(const ColumnWithTypeAndName & other) const;
void dumpStructure(WriteBuffer & out) const;
void dumpStructure(WriteBuffer & out, bool escaped = false) const;
String dumpStructure() const;
};
#pragma GCC diagnostic pop

View File

@ -24,6 +24,8 @@ namespace DB
namespace ErrorCodes
{
extern const int INCORRECT_QUERY;
extern const int INVALID_SETTING_VALUE;
extern const int UNKNOWN_SETTING;
}
namespace
@ -106,9 +108,67 @@ static void fillColumn(IColumn & column, const std::string & str)
column.insertData(str.data() + start, end - start);
}
using BinarySettings = std::unordered_map<std::string, bool>;
BinarySettings checkAndGetSettings(const ASTPtr & ast_settings)
{
if (!ast_settings)
return {};
NameSet supported_settings = {"header"};
auto get_supported_settings_string = [&supported_settings]()
{
std::string res;
for (const auto & setting : supported_settings)
{
if (!res.empty())
res += ", ";
res += setting;
}
return res;
};
BinarySettings settings;
const auto & set_query = ast_settings->as<ASTSetQuery &>();
for (const auto & change : set_query.changes)
{
if (supported_settings.count(change.name) == 0)
throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN query. Supported settings: " +
get_supported_settings_string(), ErrorCodes::UNKNOWN_SETTING);
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);
settings[change.name] = value;
}
return settings;
}
static QueryPlan::ExplainOptions getExplainOptions(const BinarySettings & settings)
{
QueryPlan::ExplainOptions options;
auto it = settings.find("header");
if (it != settings.end())
options.header = it->second;
return options;
}
BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
{
const auto & ast = query->as<ASTExplainQuery &>();
auto settings = checkAndGetSettings(ast.getSettings());
Block sample_block = getSampleBlock();
MutableColumns res_columns = sample_block.cloneEmptyColumns();
@ -124,7 +184,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
ExplainAnalyzedSyntaxVisitor::Data data{.context = context};
ExplainAnalyzedSyntaxVisitor(data).visit(query);
ast.children.at(0)->format(IAST::FormatSettings(ss, false));
ast.getExplainedQuery()->format(IAST::FormatSettings(ss, false));
}
else if (ast.getKind() == ASTExplainQuery::QueryPlan)
{
@ -137,7 +197,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
interpreter.buildQueryPlan(plan);
WriteBufferFromOStream buffer(ss);
plan.explain(buffer);
plan.explain(buffer, getExplainOptions(settings));
}
fillColumn(*res_columns[0], ss.str());

View File

@ -21,7 +21,6 @@ public:
ASTExplainQuery(ExplainKind kind_, bool old_syntax_)
: kind(kind_), old_syntax(old_syntax_)
{
children.emplace_back(); /// explained query
}
String getID(char delim) const override { return "Explain" + (delim + toString(kind, old_syntax)); }
@ -35,19 +34,37 @@ public:
return res;
}
const ASTPtr & getExplainedQuery() const { return children.at(0); }
void setExplainedQuery(ASTPtr query_)
{
children.emplace_back(query_);
query = std::move(query_);
}
void setSettings(ASTPtr settings_)
{
children.emplace_back(settings_);
ast_settings = std::move(settings_);
}
const ASTPtr & getExplainedQuery() const { return query; }
const ASTPtr & getSettings() const { return ast_settings; }
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind, old_syntax) << (settings.hilite ? hilite_none : "") << " ";
children.at(0)->formatImpl(settings, state, frame);
ast_settings->formatImpl(settings, state, frame);
settings.ostr << settings.nl_or_ws;
query->formatImpl(settings, state, frame);
}
private:
ExplainKind kind;
bool old_syntax; /// "EXPLAIN AST" -> "AST", "EXPLAIN SYNTAX" -> "ANALYZE"
ASTPtr query;
ASTPtr ast_settings;
static String toString(ExplainKind kind, bool old_syntax)
{
switch (kind)

View File

@ -2,6 +2,7 @@
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
namespace DB
{
@ -40,10 +41,24 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
auto explain_query = std::make_shared<ASTExplainQuery>(kind, old_syntax);
{
ASTPtr settings;
ParserSetQuery parser_settings(true);
auto begin = pos;
if (parser_settings.parse(pos, settings, expected))
explain_query->setSettings(std::move(settings));
else
pos = begin;
}
ParserSelectWithUnionQuery select_p;
if (!select_p.parse(pos, explain_query->children.at(0), expected))
ASTPtr query;
if (!select_p.parse(pos, query, expected))
return false;
explain_query->setExplainedQuery(std::move(query));
node = std::move(explain_query);
return true;
}

View File

@ -2,6 +2,8 @@
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Processors/QueryPipeline.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <stack>
namespace DB
@ -175,7 +177,48 @@ void QueryPlan::addInterpreterContext(std::shared_ptr<Context> context)
interpreter_context.emplace_back(std::move(context));
}
void QueryPlan::explain(WriteBuffer & buffer)
static void explainStep(
WriteBuffer & buffer, IQueryPlanStep & step, size_t ident, const QueryPlan::ExplainOptions & options)
{
std::string prefix(ident, ' ');
buffer << prefix;
buffer << step.getName();
const auto & description = step.getStepDescription();
if (!description.empty())
buffer <<" (" << description << ')';
buffer.write('\n');
if (options.header)
{
buffer << prefix;
if (!step.hasOutputStream())
buffer << "No header";
else if (!step.getOutputStream().header)
buffer << "Empty header";
else
{
buffer << "Header: ";
bool first = true;
for (const auto & elem : step.getOutputStream().header)
{
if (!first)
buffer << ",\n" << prefix << " ";
first = false;
elem.dumpStructure(buffer, true);
}
}
buffer.write('\n');
}
}
void QueryPlan::explain(WriteBuffer & buffer, const ExplainOptions & options)
{
checkInitialized();
@ -197,22 +240,7 @@ void QueryPlan::explain(WriteBuffer & buffer)
if (!frame.is_description_printed)
{
std::string prefix((stack.size() - 1) * ident, ' ');
buffer.write(prefix.data(), prefix.size());
auto name = frame.node->step->getName();
buffer.write(name.data(), name.size());
auto description = frame.node->step->getStepDescription();
if (!description.empty())
{
buffer.write(" (", 2);
buffer.write(description.data(), description.size());
buffer.write(')');
}
buffer.write('\n');
explainStep(buffer, *frame.node->step, (stack.size() - 1) * ident, options);
frame.is_description_printed = true;
}

View File

@ -33,7 +33,12 @@ public:
QueryPipelinePtr buildQueryPipeline();
void explain(WriteBuffer & buffer);
struct ExplainOptions
{
bool header = false;
};
void explain(WriteBuffer & buffer, const ExplainOptions & options);
/// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines.
/// TODO: make it in a better way.