From 9b86c19836d64d08db831c0745a7795e19c0278e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Apr 2021 17:07:56 +0300 Subject: [PATCH] Use JSONBuilder for explain json result. --- src/Common/JSONBuilder.cpp | 71 ++++++++++++------ src/Common/JSONBuilder.h | 45 ++++++----- .../Serializations/SerializationNumber.cpp | 46 +----------- src/IO/WriteHelpers.h | 41 ++++++++++ src/Interpreters/ActionsDAG.cpp | 21 +++--- src/Interpreters/ActionsDAG.h | 12 ++- src/Interpreters/AggregateDescription.cpp | 48 ++++++++++++ src/Interpreters/AggregateDescription.h | 4 +- src/Interpreters/Aggregator.cpp | 35 +++++++++ src/Interpreters/Aggregator.h | 1 + src/Interpreters/ExpressionActions.cpp | 74 +++++++++---------- src/Interpreters/ExpressionActions.h | 4 +- src/Interpreters/InterpreterExplainQuery.cpp | 21 ++++-- src/Processors/QueryPlan/AggregatingStep.cpp | 5 ++ src/Processors/QueryPlan/AggregatingStep.h | 2 + src/Processors/QueryPlan/ArrayJoinStep.cpp | 13 +++- src/Processors/QueryPlan/ArrayJoinStep.h | 1 + src/Processors/QueryPlan/ExpressionStep.cpp | 6 +- src/Processors/QueryPlan/ExpressionStep.h | 2 +- src/Processors/QueryPlan/IQueryPlanStep.h | 6 +- src/Processors/QueryPlan/QueryPlan.cpp | 49 ++++++------ src/Processors/QueryPlan/QueryPlan.h | 9 ++- 22 files changed, 336 insertions(+), 180 deletions(-) diff --git a/src/Common/JSONBuilder.cpp b/src/Common/JSONBuilder.cpp index da5ac487d51..30bec88003e 100644 --- a/src/Common/JSONBuilder.cpp +++ b/src/Common/JSONBuilder.cpp @@ -1,45 +1,74 @@ #include #include +#include namespace DB::JSONBuilder { +static bool isArrayOrMap(const IItem & item) +{ + return typeid_cast(&item) || typeid_cast(&item); +} + +static bool isSimpleArray(const std::vector & 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); } } diff --git a/src/Common/JSONBuilder.h b/src/Common/JSONBuilder.h index d41a06b9dae..f12e7567986 100644 --- a/src/Common/JSONBuilder.h +++ b/src/Common/JSONBuilder.h @@ -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 && (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(std::move(value))); } + void add(const char * value) { add(std::make_unique(value)); } + void add(bool value) { add(std::make_unique(std::move(value))); } + + template ::value, bool> = true> + void add(T value) { add(std::make_unique>(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(std::move(value))); } + void add(std::string key, const char * value) { add(std::move(key), std::make_unique(value)); } + void add(std::string key, bool value) { add(std::move(key), std::make_unique(std::move(value))); } + + template ::value, bool> = true> + void add(std::string key, T value) { add(std::move(key), std::make_unique>(value)); } + void format(const FormatSettings & settings, FormatContext & context) override; private: diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index b0a91b11716..36cbd6a5ef8 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -33,55 +33,11 @@ void SerializationNumber::deserializeText(IColumn & column, ReadBuffer & istr assert_cast &>(column).getData().push_back(x); } -template -static inline void writeDenormalNumber(T x, WriteBuffer & ostr) -{ - if constexpr (std::is_floating_point_v) - { - 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 void SerializationNumber::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { auto x = assert_cast &>(column).getData()[row_num]; - bool is_finite = isFinite(x); - - const bool need_quote = (is_integer_v && (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 diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index b9497b6f87e..0f8b5adee1a 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -446,6 +447,46 @@ inline void writeJSONString(const String & s, WriteBuffer & buf, const FormatSet writeJSONString(StringRef{s}, buf, settings); } +template +void writeJSONNumber(T x, WriteBuffer & ostr, const FormatSettings & settings) +{ + bool is_finite = isFinite(x); + + const bool need_quote = (is_integer_v && (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) + { + 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 void writeAnyEscapedString(const String & s, WriteBuffer & buf) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index b091a2e7373..fb15a44ad5c 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -13,7 +13,7 @@ #include #include -#include +#include 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); } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 9edeb7d9ada..732b3bfcbc5 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -8,8 +8,6 @@ # include "config_core.h" #endif -#include - namespace DB { @@ -30,6 +28,14 @@ using DataTypePtr = std::shared_ptr; class CompiledExpressionCache; +namespace JSONBuilder +{ + class JSONMap; + + class IItem; + using ItemPtr = std::unique_ptr; +} + /// 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. diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp index e483eb1b7a1..ca10878a4ae 100644 --- a/src/Interpreters/AggregateDescription.cpp +++ b/src/Interpreters/AggregateDescription.cpp @@ -2,6 +2,8 @@ #include #include +#include + 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(); + + function_map->add("Name", function->getName()); + + const auto & params = function->getParameters(); + if (!params.empty()) + { + auto params_array = std::make_unique(); + 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(); + 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(); + 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(); + for (auto pos : arguments) + args_pos_array->add(pos); + + map.add("Argument Positions", std::move(args_pos_array)); + } +} + } diff --git a/src/Interpreters/AggregateDescription.h b/src/Interpreters/AggregateDescription.h index 3af0dc38586..12c14f7a57c 100644 --- a/src/Interpreters/AggregateDescription.h +++ b/src/Interpreters/AggregateDescription.h @@ -5,10 +5,11 @@ #include #include - 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; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 0f8b647096d..94a4e5fe6eb 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -17,10 +17,13 @@ #include #include #include +#include #include #include #include +#include + 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(); + + 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(); + + for (const auto & aggregate : aggregates) + { + auto aggregate_map = std::make_unique(); + 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; }) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 3415382d485..43eab5f8e16 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -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_); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 7232297c315..30457cef91d 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #if defined(MEMORY_SANITIZER) #include @@ -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(); if (node) - tree = node->toTree(); + node->toTree(*map); - boost::property_tree::ptree args; - boost::property_tree::ptree dropped_args; + auto args = std::make_unique(); + auto dropped_args = std::make_unique(); 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(); for (const auto & input_column : required_columns) { - boost::property_tree::ptree tree; - tree.add("Name", input_column.name); + auto map = std::make_unique(); + 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(); for (const auto & output_column : sample_block) { - boost::property_tree::ptree tree; - tree.add("Name", output_column.name); + auto map = std::make_unique(); + 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(); 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(); 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(); + 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 diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 2edd97abb5e..10636faa4d7 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -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; @@ -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); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 129be8bab9a..8dd33a0d139 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -18,8 +19,7 @@ #include #include -#include -#include +#include 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(); + plan_map->add("Plan", plan.explainPlan(settings.query_plan_options)); + auto plan_array = std::make_unique(); + 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); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index daa6e4981bb..772390acb32 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -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()) diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index c29a3cc2836..696aabd4de7 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -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; diff --git a/src/Processors/QueryPlan/ArrayJoinStep.cpp b/src/Processors/QueryPlan/ArrayJoinStep.cpp index 7e82ec87353..9089bb8e5a2 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.cpp +++ b/src/Processors/QueryPlan/ArrayJoinStep.cpp @@ -5,7 +5,7 @@ #include #include #include - +#include 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(); + for (const auto & column : array_join->columns) + columns_array->add(column); + + map.add("Columns", std::move(columns_array)); +} + } diff --git a/src/Processors/QueryPlan/ArrayJoinStep.h b/src/Processors/QueryPlan/ArrayJoinStep.h index db1b4187549..b3e08c2023c 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.h +++ b/src/Processors/QueryPlan/ArrayJoinStep.h @@ -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); diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index bdb2ce07f5a..e6a0475a7c2 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include 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(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_) diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index fa03fd9441a..b7c4c0974f3 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -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; diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index 5b235833ad0..ad088b22dc9 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -3,7 +3,7 @@ #include #include -#include +namespace JSONBuilder { class JSONMap; } namespace DB { @@ -16,6 +16,8 @@ class IProcessor; using ProcessorPtr = std::shared_ptr; using Processors = std::vector; +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 {} diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index e4002f5dba9..b4624b5156e 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB { @@ -201,42 +201,39 @@ void QueryPlan::addInterpreterContext(std::shared_ptr 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(); 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(); + 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 node_map = {}; + std::unique_ptr children_array = {}; }; std::stack stack; stack.push(Frame{.node = root}); - boost::property_tree::ptree tree; + std::unique_ptr 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(); + + frame.node_map = std::make_unique(); + 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)); } } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 4ca7d38f087..e4bb4e9b62d 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -3,7 +3,6 @@ #include #include #include -#include #include @@ -29,6 +28,12 @@ class Pipe; struct QueryPlanOptimizationSettings; struct BuildQueryPipelineSettings; +namespace JSONBuilder +{ + class IItem; + using ItemPtr = std::unique_ptr; +} + /// 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);