Use JSONBuilder for explain json result.

This commit is contained in:
Nikolai Kochetov 2021-04-14 17:07:56 +03:00
parent 5692de5e02
commit 9b86c19836
22 changed files with 336 additions and 180 deletions

View File

@ -1,45 +1,74 @@
#include <Common/JSONBuilder.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB::JSONBuilder
{
static bool isArrayOrMap(const IItem & item)
{
return typeid_cast<const JSONArray *>(&item) || typeid_cast<const JSONMap *>(&item);
}
static bool isSimpleArray(const std::vector<ItemPtr> & values)
{
for (const auto & value : values)
if (isArrayOrMap(*value))
return false;
return true;
}
void JSONString::format(const FormatSettings & settings, FormatContext & context)
{
writeJSONString(value, context.out, settings.settings);
}
void JSONBool::format(const FormatSettings &, FormatContext & context)
{
writeString(value ? "true" : "false", context.out);
}
void JSONArray::format(const FormatSettings & settings, FormatContext & context)
{
writeChar('[');
writeChar('\n');
writeChar('[', context.out);
context.offset += settings.indent;
bool single_row = settings.print_simple_arrays_in_single_row && isSimpleArray(values);
bool first = true;
for (const auto & value : values)
{
if (first)
writeChar(',');
if (!first)
writeChar(',', context.out);
if (!single_row)
{
writeChar('\n', context.out);
writeChar(' ', context.offset, context.out);
}
else if (!first)
writeChar(' ', context.out);
first = false;
writeChar(' ', context.indent, context.out);
value->format(settings, context);
writeChar('\n');
}
context.offset -= settings.indent;
writeChar(' ', context.indent, context.out);
writeChar(']');
if (!single_row)
{
writeChar('\n', context.out);
writeChar(' ', context.offset, context.out);
}
writeChar(']', context.out);
}
void JSONMap::format(const FormatSettings & settings, FormatContext & context)
{
writeChar('{');
writeChar('\n');
writeChar('{', context.out);
context.offset += settings.indent;
@ -47,24 +76,24 @@ void JSONMap::format(const FormatSettings & settings, FormatContext & context)
for (const auto & value : values)
{
if (first)
writeChar(',');
if (!first)
writeChar(',', context.out);
first = false;
writeChar(' ', context.indent, context.out);
writeChar('\n', context.out);
writeChar(' ', context.offset, context.out);
writeJSONString(value.key, context.out, settings.settings);
writeChar(':');
writeChar(' ');
writeChar(':', context.out);
writeChar(' ', context.out);
value.value->format(settings, context);
writeChar('\n');
}
context.offset -= settings.indent;
writeChar(' ', context.indent, context.out);
writeChar('}');
writeChar('\n', context.out);
writeChar(' ', context.offset, context.out);
writeChar('}', context.out);
}
}

View File

@ -12,12 +12,13 @@ struct FormatSettings
{
const DB::FormatSettings & settings;
size_t indent = 2;
bool print_simple_arrays_in_single_row = true;
};
struct FormatContext
{
WriteBuffer & out;
size_t offset;
size_t offset = 0;
};
class IItem
@ -50,33 +51,34 @@ public:
void format(const FormatSettings & settings, FormatContext & context) override
{
bool is_finite = isFinite(value);
const bool need_quote = (std::is_integral_v<T> && (sizeof(T) >= 8) && settings.settings.json.quote_64bit_integers)
|| (settings.settings.json.quote_denormals && !is_finite);
if (need_quote)
writeChar('"', context.out);
if (is_finite)
writeText(value, context.out);
else if (!settings.settings.json.quote_denormals)
writeCString("null", context.out);
else
writeDenormalNumber(value, context.out);
if (need_quote)
writeChar('"', context.out);
writeJSONNumber(value, context.out, settings.settings);
}
private:
T value;
};
class JSONBool : public IItem
{
public:
explicit JSONBool(bool value_) : value(std::move(value_)) {}
void format(const FormatSettings & settings, FormatContext & context) override;
private:
bool value;
};
class JSONArray : public IItem
{
public:
void add(ItemPtr value) { values.push_back(std::move(value)); }
void add(std::string value) { add(std::make_unique<JSONString>(std::move(value))); }
void add(const char * value) { add(std::make_unique<JSONString>(value)); }
void add(bool value) { add(std::make_unique<JSONBool>(std::move(value))); }
template <typename T, std::enable_if_t<std::is_arithmetic<T>::value, bool> = true>
void add(T value) { add(std::make_unique<JSONNumber<T>>(value)); }
void format(const FormatSettings & settings, FormatContext & context) override;
private:
@ -93,6 +95,13 @@ class JSONMap : public IItem
public:
void add(std::string key, ItemPtr value) { values.emplace_back(Pair{.key = std::move(key), .value = std::move(value)}); }
void add(std::string key, std::string value) { add(std::move(key), std::make_unique<JSONString>(std::move(value))); }
void add(std::string key, const char * value) { add(std::move(key), std::make_unique<JSONString>(value)); }
void add(std::string key, bool value) { add(std::move(key), std::make_unique<JSONBool>(std::move(value))); }
template <typename T, std::enable_if_t<std::is_arithmetic<T>::value, bool> = true>
void add(std::string key, T value) { add(std::move(key), std::make_unique<JSONNumber<T>>(value)); }
void format(const FormatSettings & settings, FormatContext & context) override;
private:

View File

@ -33,55 +33,11 @@ void SerializationNumber<T>::deserializeText(IColumn & column, ReadBuffer & istr
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
}
template <typename T>
static inline void writeDenormalNumber(T x, WriteBuffer & ostr)
{
if constexpr (std::is_floating_point_v<T>)
{
if (std::signbit(x))
{
if (isNaN(x))
writeCString("-nan", ostr);
else
writeCString("-inf", ostr);
}
else
{
if (isNaN(x))
writeCString("nan", ostr);
else
writeCString("inf", ostr);
}
}
else
{
/// This function is not called for non floating point numbers.
(void)x;
}
}
template <typename T>
void SerializationNumber<T>::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
auto x = assert_cast<const ColumnVector<T> &>(column).getData()[row_num];
bool is_finite = isFinite(x);
const bool need_quote = (is_integer_v<T> && (sizeof(T) >= 8) && settings.json.quote_64bit_integers)
|| (settings.json.quote_denormals && !is_finite);
if (need_quote)
writeChar('"', ostr);
if (is_finite)
writeText(x, ostr);
else if (!settings.json.quote_denormals)
writeCString("null", ostr);
else
writeDenormalNumber(x, ostr);
if (need_quote)
writeChar('"', ostr);
writeJSONNumber(x, ostr, settings);
}
template <typename T>

View File

@ -20,6 +20,7 @@
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/UInt128.h>
#include <Common/NaNUtils.h>
#include <IO/CompressionMethod.h>
#include <IO/WriteBuffer.h>
@ -446,6 +447,46 @@ inline void writeJSONString(const String & s, WriteBuffer & buf, const FormatSet
writeJSONString(StringRef{s}, buf, settings);
}
template <typename T>
void writeJSONNumber(T x, WriteBuffer & ostr, const FormatSettings & settings)
{
bool is_finite = isFinite(x);
const bool need_quote = (is_integer_v<T> && (sizeof(T) >= 8) && settings.json.quote_64bit_integers)
|| (settings.json.quote_denormals && !is_finite);
if (need_quote)
writeChar('"', ostr);
if (is_finite)
writeText(x, ostr);
else if (!settings.json.quote_denormals)
writeCString("null", ostr);
else
{
if constexpr (std::is_floating_point_v<T>)
{
if (std::signbit(x))
{
if (isNaN(x))
writeCString("-nan", ostr);
else
writeCString("-inf", ostr);
}
else
{
if (isNaN(x))
writeCString("nan", ostr);
else
writeCString("inf", ostr);
}
}
}
if (need_quote)
writeChar('"', ostr);
}
template <char c>
void writeAnyEscapedString(const String & s, WriteBuffer & buf)

View File

@ -13,7 +13,7 @@
#include <IO/Operators.h>
#include <stack>
#include <boost/property_tree/ptree.hpp>
#include <Common/JSONBuilder.h>
namespace DB
{
@ -47,29 +47,26 @@ const char * ActionsDAG::typeToString(ActionsDAG::ActionType type)
__builtin_unreachable();
}
boost::property_tree::ptree ActionsDAG::Node::toTree() const
void ActionsDAG::Node::toTree(JSONBuilder::JSONMap & map) const
{
boost::property_tree::ptree tree;
tree.add("NodeType", ActionsDAG::typeToString(type));
map.add("Node Type", ActionsDAG::typeToString(type));
if (result_type)
tree.add("ResultType", result_type->getName());
map.add("Result Type", result_type->getName());
if (!result_name.empty())
tree.add("ResultType", ActionsDAG::typeToString(type));
map.add("Result Type", ActionsDAG::typeToString(type));
if (column)
tree.add("Column", column->getName());
map.add("Column", column->getName());
if (function_base)
tree.add("Function", function_base->getName());
map.add("Function", function_base->getName());
else if (function_builder)
tree.add("Function", function_base->getName());
map.add("Function", function_base->getName());
if (type == ActionType::FUNCTION)
tree.add("Compiled", is_function_compiled);
return tree;
map.add("Compiled", is_function_compiled);
}

View File

@ -8,8 +8,6 @@
# include "config_core.h"
#endif
#include <boost/property_tree/ptree_fwd.hpp>
namespace DB
{
@ -30,6 +28,14 @@ using DataTypePtr = std::shared_ptr<const IDataType>;
class CompiledExpressionCache;
namespace JSONBuilder
{
class JSONMap;
class IItem;
using ItemPtr = std::unique_ptr<IItem>;
}
/// Directed acyclic graph of expressions.
/// This is an intermediate representation of actions which is usually built from expression list AST.
/// Node of DAG describe calculation of a single column with known type, name, and constant value (if applicable).
@ -85,7 +91,7 @@ public:
/// We calculate such constants in order to avoid unnecessary materialization, but prohibit it's folding.
bool allow_constant_folding = true;
boost::property_tree::ptree toTree() const;
void toTree(JSONBuilder::JSONMap & map) const;
};
/// NOTE: std::list is an implementation detail.

View File

@ -2,6 +2,8 @@
#include <Common/FieldVisitors.h>
#include <IO/Operators.h>
#include <Common/JSONBuilder.h>
namespace DB
{
@ -99,4 +101,50 @@ void AggregateDescription::explain(WriteBuffer & out, size_t indent) const
}
}
void AggregateDescription::explain(JSONBuilder::JSONMap & map) const
{
map.add("Name", column_name);
if (function)
{
auto function_map = std::make_unique<JSONBuilder::JSONMap>();
function_map->add("Name", function->getName());
const auto & params = function->getParameters();
if (!params.empty())
{
auto params_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & param : params)
params_array->add(applyVisitor(FieldVisitorToString(), param));
function_map->add("Parameters", std::move(params_array));
}
auto args_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & type : function->getArgumentTypes())
args_array->add(type->getName());
function_map->add("Argument Types", std::move(args_array));
function_map->add("Result Type", function->getReturnType()->getName());
map.add("Function", std::move(function_map));
}
auto args_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & name : argument_names)
args_array->add(name);
map.add("Arguments", std::move(args_array));
if (!arguments.empty())
{
auto args_pos_array = std::make_unique<JSONBuilder::JSONArray>();
for (auto pos : arguments)
args_pos_array->add(pos);
map.add("Argument Positions", std::move(args_pos_array));
}
}
}

View File

@ -5,10 +5,11 @@
#include <Core/Names.h>
#include <Core/Types.h>
namespace DB
{
namespace JSONBuilder { class JSONMap; }
struct AggregateDescription
{
AggregateFunctionPtr function;
@ -18,6 +19,7 @@ struct AggregateDescription
String column_name; /// What name to use for a column with aggregate function values
void explain(WriteBuffer & out, size_t indent) const; /// Get description for EXPLAIN query.
void explain(JSONBuilder::JSONMap & map) const;
};
using AggregateDescriptions = std::vector<AggregateDescription>;

View File

@ -17,10 +17,13 @@
#include <Common/CurrentThread.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/JSONBuilder.h>
#include <AggregateFunctions/AggregateFunctionArray.h>
#include <AggregateFunctions/AggregateFunctionState.h>
#include <IO/Operators.h>
#include <Common/JSONBuilder.h>
namespace ProfileEvents
{
@ -178,6 +181,38 @@ void Aggregator::Params::explain(WriteBuffer & out, size_t indent) const
}
}
void Aggregator::Params::explain(JSONBuilder::JSONMap & map) const
{
const auto & header = src_header ? src_header
: intermediate_header;
auto keys_array = std::make_unique<JSONBuilder::JSONArray>();
for (auto key : keys)
{
if (key >= header.columns())
keys_array->add("");
else
keys_array->add(header.getByPosition(key).name);
}
map.add("Keys", std::move(keys_array));
if (!aggregates.empty())
{
auto aggregates_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & aggregate : aggregates)
{
auto aggregate_map = std::make_unique<JSONBuilder::JSONMap>();
aggregate.explain(*aggregate_map);
aggregates_array->add(std::move(aggregate_map));
}
map.add("Aggregates", std::move(aggregates_array));
}
}
Aggregator::Aggregator(const Params & params_)
: params(params_),
isCancelled([]() { return false; })

View File

@ -950,6 +950,7 @@ public:
/// Returns keys and aggregated for EXPLAIN query
void explain(WriteBuffer & out, size_t indent) const;
void explain(JSONBuilder::JSONMap & map) const;
};
Aggregator(const Params & params_);

View File

@ -15,7 +15,7 @@
#include <Columns/ColumnSet.h>
#include <queue>
#include <stack>
#include <boost/property_tree/ptree.hpp>
#include <Common/JSONBuilder.h>
#if defined(MEMORY_SANITIZER)
#include <sanitizer/msan_interface.h>
@ -259,29 +259,27 @@ std::string ExpressionActions::Action::toString() const
return out.str();
}
boost::property_tree::ptree ExpressionActions::Action::toTree() const
JSONBuilder::ItemPtr ExpressionActions::Action::toTree() const
{
boost::property_tree::ptree tree;
auto map = std::make_unique<JSONBuilder::JSONMap>();
if (node)
tree = node->toTree();
node->toTree(*map);
boost::property_tree::ptree args;
boost::property_tree::ptree dropped_args;
auto args = std::make_unique<JSONBuilder::JSONArray>();
auto dropped_args = std::make_unique<JSONBuilder::JSONArray>();
for (auto arg : arguments)
{
boost::property_tree::ptree arg_tree;
arg_tree.put("", arg.pos);
args.add_child("", arg_tree);
args->add(arg.pos);
if (!arg.needed_later)
dropped_args.add_child("", arg_tree);
dropped_args->add(arg.pos);
}
tree.add_child("Arguments", args);
tree.add_child("RemovedArguments", dropped_args);
tree.put("Result", result_position);
map->add("Arguments", std::move(args));
map->add("Removed Arguments", std::move(dropped_args));
map->add("Result", result_position);
return tree;
return map;
}
void ExpressionActions::checkLimits(const ColumnsWithTypeAndName & columns) const
@ -593,52 +591,48 @@ std::string ExpressionActions::dumpActions() const
return ss.str();
}
boost::property_tree::ptree ExpressionActions::toTree() const
JSONBuilder::ItemPtr ExpressionActions::toTree() const
{
boost::property_tree::ptree inputs_tree;
auto inputs_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & input_column : required_columns)
{
boost::property_tree::ptree tree;
tree.add("Name", input_column.name);
auto map = std::make_unique<JSONBuilder::JSONMap>();
map->add("Name", input_column.name);
if (input_column.type)
tree.add("Type", input_column.type->getName());
map->add("Type", input_column.type->getName());
inputs_tree.add_child("", tree);
inputs_array->add(std::move(map));
}
boost::property_tree::ptree outputs_tree;
auto outputs_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & output_column : sample_block)
{
boost::property_tree::ptree tree;
tree.add("Name", output_column.name);
auto map = std::make_unique<JSONBuilder::JSONMap>();
map->add("Name", output_column.name);
if (output_column.type)
tree.add("Type", output_column.type->getName());
map->add("Type", output_column.type->getName());
outputs_tree.add_child("", tree);
outputs_array->add(std::move(map));
}
boost::property_tree::ptree actions_tree;
auto actions_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & action : actions)
actions_tree.add_child("", action.toTree());
actions_array->add(action.toTree());
boost::property_tree::ptree positions_tree;
auto positions_array = std::make_unique<JSONBuilder::JSONArray>();
for (auto pos : result_positions)
{
boost::property_tree::ptree pos_tree;
pos_tree.put("", pos);
positions_tree.add_child("", pos_tree);
}
positions_array->add(pos);
boost::property_tree::ptree tree;
tree.add_child("Inputs", inputs_tree);
tree.add_child("Actions", actions_tree);
tree.add_child("Outputs", outputs_tree);
tree.add_child("Positions", positions_tree);
tree.put("ProjectInput", actions_dag->isInputProjected());
auto map = std::make_unique<JSONBuilder::JSONMap>();
map->add("Inputs", std::move(inputs_array));
map->add("Actions", std::move(actions_array));
map->add("Outputs", std::move(outputs_array));
map->add("Positions", std::move(positions_array));
map->add("Project Input", actions_dag->isInputProjected());
return tree;
return map;
}
bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) const

View File

@ -58,7 +58,7 @@ public:
size_t result_position;
std::string toString() const;
boost::property_tree::ptree toTree() const;
JSONBuilder::ItemPtr toTree() const;
};
using Actions = std::vector<Action>;
@ -109,7 +109,7 @@ public:
const Block & getSampleBlock() const { return sample_block; }
std::string dumpActions() const;
boost::property_tree::ptree toTree() const;
JSONBuilder::ItemPtr toTree() const;
static std::string getSmallestColumn(const NamesAndTypesList & columns);

View File

@ -7,6 +7,7 @@
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/Context.h>
#include <Formats/FormatFactory.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTExplainQuery.h>
@ -18,8 +19,7 @@
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/printPipeline.h>
#include <boost/property_tree/ptree.hpp>
#include <boost/property_tree/json_parser.hpp>
#include <Common/JSONBuilder.h>
namespace DB
{
@ -262,10 +262,19 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl()
if (settings.json)
{
auto tree = plan.explainPlan(settings.query_plan_options);
std::stringstream out;
boost::property_tree::json_parser::write_json(out, tree);
buf.str() = out.str();
/// 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(context);
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);
}
else
plan.explainPlan(buf, settings.query_plan_options);

View File

@ -163,6 +163,11 @@ void AggregatingStep::describeActions(FormatSettings & settings) const
params.explain(settings.out, settings.offset);
}
void AggregatingStep::describeActions(JSONBuilder::JSONMap & map) const
{
params.explain(map);
}
void AggregatingStep::describePipeline(FormatSettings & settings) const
{
if (!aggregating.empty())

View File

@ -29,6 +29,8 @@ public:
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override;
void describeActions(JSONBuilder::JSONMap & map) const override;
void describeActions(FormatSettings &) const override;
void describePipeline(FormatSettings & settings) const override;

View File

@ -5,7 +5,7 @@
#include <Interpreters/ArrayJoinAction.h>
#include <Interpreters/ExpressionActions.h>
#include <IO/Operators.h>
#include <Common/JSONBuilder.h>
namespace DB
{
@ -87,4 +87,15 @@ void ArrayJoinStep::describeActions(FormatSettings & settings) const
settings.out << '\n';
}
void ArrayJoinStep::describeActions(JSONBuilder::JSONMap & map) const
{
map.add("Left", array_join->is_left);
auto columns_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & column : array_join->columns)
columns_array->add(column);
map.add("Columns", std::move(columns_array));
}
}

View File

@ -15,6 +15,7 @@ public:
void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override;
void describeActions(JSONBuilder::JSONMap & map) const override;
void describeActions(FormatSettings & settings) const override;
void updateInputStream(DataStream input_stream, Block result_header);

View File

@ -7,7 +7,7 @@
#include <Processors/Sources/SourceFromInputStream.h>
#include <Interpreters/JoinSwitcher.h>
#include <boost/property_tree/ptree.hpp>
#include <Common/JSONBuilder.h>
namespace DB
{
@ -112,10 +112,10 @@ void ExpressionStep::describeActions(FormatSettings & settings) const
settings.out << '\n';
}
void ExpressionStep::describeActions(boost::property_tree::ptree & tree) const
void ExpressionStep::describeActions(JSONBuilder::JSONMap & map) const
{
auto expression = std::make_shared<ExpressionActions>(actions_dag, ExpressionActionsSettings{});
tree.add_child("Expression", expression->toTree());
map.add("Expression", expression->toTree());
}
JoinStep::JoinStep(const DataStream & input_stream_, JoinPtr join_, bool has_non_joined_rows_, size_t max_block_size_)

View File

@ -30,7 +30,7 @@ public:
const ActionsDAGPtr & getExpression() const { return actions_dag; }
void describeActions(boost::property_tree::ptree & tree) const override;
void describeActions(JSONBuilder::JSONMap & map) const override;
private:
ActionsDAGPtr actions_dag;

View File

@ -3,7 +3,7 @@
#include <Core/SortDescription.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <boost/property_tree/ptree_fwd.hpp>
namespace JSONBuilder { class JSONMap; }
namespace DB
{
@ -16,6 +16,8 @@ class IProcessor;
using ProcessorPtr = std::shared_ptr<IProcessor>;
using Processors = std::vector<ProcessorPtr>;
namespace JSONBuilder { class JSONMap; }
/// Description of data stream.
/// Single logical data stream may relate to many ports of pipeline.
class DataStream
@ -98,7 +100,7 @@ public:
const bool write_header = false;
};
virtual void describeActions(boost::property_tree::ptree & /*tree*/) const {}
virtual void describeActions(JSONBuilder::JSONMap & /*map*/) const {}
/// Get detailed description of step actions. This is shown in EXPLAIN query with options `actions = 1`.
virtual void describeActions(FormatSettings & /*settings*/) const {}

View File

@ -9,7 +9,7 @@
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <boost/property_tree/ptree.hpp>
#include <Common/JSONBuilder.h>
namespace DB
{
@ -201,42 +201,39 @@ void QueryPlan::addInterpreterContext(std::shared_ptr<Context> context)
}
static boost::property_tree::ptree explainStep(const IQueryPlanStep & step, const QueryPlan::ExplainPlanOptions & options)
static void explainStep(const IQueryPlanStep & step, JSONBuilder::JSONMap & map, const QueryPlan::ExplainPlanOptions & options)
{
boost::property_tree::ptree tree;
tree.put("Node Type", step.getName());
map.add("Node Type", step.getName());
if (options.description)
{
const auto & description = step.getStepDescription();
if (!description.empty())
tree.put("Description", description);
map.add("Description", description);
}
if (options.header && step.hasOutputStream())
{
boost::property_tree::ptree header_tree;
auto header_array = std::make_unique<JSONBuilder::JSONArray>();
for (const auto & output_column : step.getOutputStream().header)
{
boost::property_tree::ptree column_tree;
column_tree.add("Name", output_column.name);
auto column_map = std::make_unique<JSONBuilder::JSONMap>();
column_map->add("Name", output_column.name);
if (output_column.type)
column_tree.add("Type", output_column.type->getName());
column_map->add("Type", output_column.type->getName());
header_tree.add_child("", column_tree);
header_array->add(std::move(column_map));
}
tree.add_child("Header", header_tree);
map.add("Header", std::move(header_array));
}
if (options.actions)
step.describeActions(tree);
return tree;
step.describeActions(map);
}
boost::property_tree::ptree QueryPlan::explainPlan(const ExplainPlanOptions & options)
JSONBuilder::ItemPtr QueryPlan::explainPlan(const ExplainPlanOptions & options)
{
checkInitialized();
@ -244,21 +241,27 @@ boost::property_tree::ptree QueryPlan::explainPlan(const ExplainPlanOptions & op
{
Node * node;
size_t next_child = 0;
boost::property_tree::ptree node_tree = {};
boost::property_tree::ptree children_trees = {};
std::unique_ptr<JSONBuilder::JSONMap> node_map = {};
std::unique_ptr<JSONBuilder::JSONArray> children_array = {};
};
std::stack<Frame> stack;
stack.push(Frame{.node = root});
boost::property_tree::ptree tree;
std::unique_ptr<JSONBuilder::JSONMap> tree;
while (!stack.empty())
{
auto & frame = stack.top();
if (frame.next_child == 0)
frame.node_tree = explainStep(*frame.node->step, options);
{
if (!frame.node->children.empty())
frame.children_array = std::make_unique<JSONBuilder::JSONArray>();
frame.node_map = std::make_unique<JSONBuilder::JSONMap>();
explainStep(*frame.node->step, *frame.node_map, options);
}
if (frame.next_child < frame.node->children.size())
{
@ -267,14 +270,14 @@ boost::property_tree::ptree QueryPlan::explainPlan(const ExplainPlanOptions & op
}
else
{
if (!frame.children_trees.empty())
frame.node_tree.add_child("Plans", frame.children_trees);
if (frame.children_array)
frame.node_map->add("Plans", std::move(frame.children_array));
tree.swap(frame.node_tree);
tree.swap(frame.node_map);
stack.pop();
if (!stack.empty())
stack.top().children_trees.add_child("", tree);
stack.top().children_array->add(std::move(tree));
}
}

View File

@ -3,7 +3,6 @@
#include <list>
#include <vector>
#include <set>
#include <boost/property_tree/ptree_fwd.hpp>
#include <Core/Names.h>
@ -29,6 +28,12 @@ class Pipe;
struct QueryPlanOptimizationSettings;
struct BuildQueryPipelineSettings;
namespace JSONBuilder
{
class IItem;
using ItemPtr = std::unique_ptr<IItem>;
}
/// A tree of query steps.
/// The goal of QueryPlan is to build QueryPipeline.
/// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimizations.
@ -74,7 +79,7 @@ public:
bool header = false;
};
boost::property_tree::ptree explainPlan(const ExplainPlanOptions & options);
JSONBuilder::ItemPtr explainPlan(const ExplainPlanOptions & options);
void explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options);
void explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptions & options);