mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
Header for explain.
This commit is contained in:
parent
e1481ccfc2
commit
47d9f4af9c
@ -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();
|
||||
|
@ -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
|
||||
|
@ -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());
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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.
|
||||
|
Loading…
Reference in New Issue
Block a user