mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
Refactor some code.
This commit is contained in:
parent
893a75bcf7
commit
5b1596cdb2
@ -28,12 +28,9 @@ bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) cons
|
||||
}
|
||||
|
||||
|
||||
void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out, bool escaped) const
|
||||
void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const
|
||||
{
|
||||
if (escaped)
|
||||
writeEscapedString(name, out);
|
||||
else
|
||||
out << name;
|
||||
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, bool escaped = false) const;
|
||||
void dumpStructure(WriteBuffer & out) const;
|
||||
String dumpStructure() const;
|
||||
};
|
||||
#pragma GCC diagnostic pop
|
||||
|
@ -1,40 +1,40 @@
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Core/Block.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
String dumpSortDescription(const SortDescription & description, const Block & header)
|
||||
void dumpSortDescription(const SortDescription & description, const Block & header, WriteBuffer & out)
|
||||
{
|
||||
String res;
|
||||
bool first = true;
|
||||
|
||||
for (const auto & desc : description)
|
||||
{
|
||||
if (!res.empty())
|
||||
res += ", ";
|
||||
if (!first)
|
||||
out << ", ";
|
||||
first = false;
|
||||
|
||||
if (!desc.column_name.empty())
|
||||
res += desc.column_name;
|
||||
out << desc.column_name;
|
||||
else
|
||||
{
|
||||
if (desc.column_number < header.columns())
|
||||
res += header.getByPosition(desc.column_number).name;
|
||||
out << header.getByPosition(desc.column_number).name;
|
||||
else
|
||||
res += "?";
|
||||
out << "?";
|
||||
|
||||
res += " (pos " + std::to_string(desc.column_number) + ")";
|
||||
out << " (pos " << desc.column_number << ")";
|
||||
}
|
||||
|
||||
if (desc.direction > 0)
|
||||
res += " ASC";
|
||||
out << " ASC";
|
||||
else
|
||||
res += " DESC";
|
||||
out << " DESC";
|
||||
|
||||
if (desc.with_fill)
|
||||
res += " WITH FILL";
|
||||
out << " WITH FILL";
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -72,6 +72,6 @@ struct SortColumnDescription
|
||||
using SortDescription = std::vector<SortColumnDescription>;
|
||||
|
||||
class Block;
|
||||
String dumpSortDescription(const SortDescription & description, const Block & header);
|
||||
void dumpSortDescription(const SortDescription & description, const Block & header, WriteBuffer & out);
|
||||
|
||||
}
|
||||
|
@ -1,82 +1,101 @@
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Strings AggregateDescription::explain() const
|
||||
void AggregateDescription::explain(WriteBuffer & out, size_t ident) const
|
||||
{
|
||||
Strings res;
|
||||
String prefix(ident, ' ');
|
||||
|
||||
res.emplace_back(column_name);
|
||||
out << prefix << column_name << '\n';
|
||||
|
||||
auto get_params_string = [](const Array & arr)
|
||||
auto dump_params = [&](const Array & arr)
|
||||
{
|
||||
String params_str;
|
||||
bool first = true;
|
||||
for (const auto & param : arr)
|
||||
{
|
||||
if (!params_str.empty())
|
||||
params_str += ", ";
|
||||
if (!first)
|
||||
out << ", ";
|
||||
|
||||
params_str += applyVisitor(FieldVisitorToString(), param);
|
||||
first = false;
|
||||
|
||||
out << applyVisitor(FieldVisitorToString(), param);
|
||||
}
|
||||
|
||||
return params_str;
|
||||
};
|
||||
|
||||
if (function)
|
||||
{
|
||||
String types_str;
|
||||
for (const auto & type : function->getArgumentTypes())
|
||||
{
|
||||
if (!types_str.empty())
|
||||
types_str += ", ";
|
||||
out << prefix << " Function: " << function->getName();
|
||||
|
||||
types_str += type->getName();
|
||||
const auto & params = function->getParameters();
|
||||
if (!params.empty())
|
||||
{
|
||||
out << "(";
|
||||
dump_params(params);
|
||||
out << ")";
|
||||
}
|
||||
|
||||
auto params_str = get_params_string(function->getParameters());
|
||||
if (!params_str.empty())
|
||||
params_str = "(" + params_str + ")";
|
||||
out << "(";
|
||||
|
||||
res.emplace_back(" Function: " + function->getName() + params_str + '(' + types_str + ") -> " +
|
||||
function->getReturnType()->getName());
|
||||
bool first = true;
|
||||
for (const auto & type : function->getArgumentTypes())
|
||||
{
|
||||
if (!first)
|
||||
out << ", ";
|
||||
first = false;
|
||||
|
||||
out << type->getName();
|
||||
}
|
||||
|
||||
out << ")\n";
|
||||
}
|
||||
else
|
||||
res.emplace_back(" Function: nullptr");
|
||||
out << prefix << " Function: nullptr\n";
|
||||
|
||||
if (!parameters.empty())
|
||||
res.emplace_back(" Parameters: " + get_params_string(parameters));
|
||||
|
||||
String arguments_names_str;
|
||||
for (const auto & arg : argument_names)
|
||||
{
|
||||
if (!arguments_names_str.empty())
|
||||
arguments_names_str += ", ";
|
||||
|
||||
arguments_names_str += arg;
|
||||
out << prefix << " Parameters: ";
|
||||
dump_params(parameters);
|
||||
out << '\n';
|
||||
}
|
||||
|
||||
if (arguments_names_str.empty())
|
||||
arguments_names_str = "none";
|
||||
out << prefix << " Arguments: ";
|
||||
|
||||
res.emplace_back(" Arguments: " + arguments_names_str);
|
||||
|
||||
String arguments_pos_str;
|
||||
for (auto arg : arguments)
|
||||
if (argument_names.empty())
|
||||
out << "none\n";
|
||||
else
|
||||
{
|
||||
if (!arguments_pos_str.empty())
|
||||
arguments_pos_str += ", ";
|
||||
bool first = true;
|
||||
for (const auto & arg : argument_names)
|
||||
{
|
||||
if (!first)
|
||||
out << ", ";
|
||||
first = false;
|
||||
|
||||
arguments_pos_str += std::to_string(arg);
|
||||
out << arg;
|
||||
}
|
||||
out << "\n";
|
||||
}
|
||||
|
||||
if (arguments_pos_str.empty())
|
||||
arguments_pos_str = "none";
|
||||
out << prefix << " Argument positions: ";
|
||||
|
||||
res.emplace_back(" Argument positions: " + arguments_pos_str);
|
||||
if (arguments.empty())
|
||||
out << "none\n";
|
||||
else
|
||||
{
|
||||
bool first = true;
|
||||
for (auto arg : arguments)
|
||||
{
|
||||
if (!first)
|
||||
out << ", ";
|
||||
first = false;
|
||||
|
||||
return res;
|
||||
out << arg;
|
||||
}
|
||||
out << '\n';
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,7 +16,7 @@ struct AggregateDescription
|
||||
Names argument_names; /// used if no `arguments` are specified.
|
||||
String column_name; /// What name to use for a column with aggregate function values
|
||||
|
||||
Strings explain() const; /// Get description for EXPLAIN query.
|
||||
void explain(WriteBuffer & out, size_t ident) const; /// Get description for EXPLAIN query.
|
||||
};
|
||||
|
||||
using AggregateDescriptions = std::vector<AggregateDescription>;
|
||||
|
@ -29,6 +29,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionArray.h>
|
||||
#include <AggregateFunctions/AggregateFunctionState.h>
|
||||
#include <Disks/StoragePolicy.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -150,44 +151,41 @@ Block Aggregator::Params::getHeader(
|
||||
return materializeBlock(res);
|
||||
}
|
||||
|
||||
Strings Aggregator::Params::explain() const
|
||||
void Aggregator::Params::explain(WriteBuffer & out, size_t ident) const
|
||||
{
|
||||
Strings res;
|
||||
const auto & header = src_header ? src_header
|
||||
: intermediate_header;
|
||||
String keys_str;
|
||||
for (auto key : keys)
|
||||
|
||||
String prefix(ident, ' ');
|
||||
|
||||
{
|
||||
if (!keys_str.empty())
|
||||
keys_str += ", ";
|
||||
/// Dump keys.
|
||||
out << prefix << "Keys: ";
|
||||
|
||||
if (key >= header.columns())
|
||||
keys_str += "unknown position " + std::to_string(key);
|
||||
else
|
||||
keys_str += header.getByPosition(key).name;
|
||||
bool first = true;
|
||||
for (auto key : keys)
|
||||
{
|
||||
if (!first)
|
||||
out << ", ";
|
||||
first = false;
|
||||
|
||||
if (key >= header.columns())
|
||||
out << "unknown position " << key;
|
||||
else
|
||||
out << header.getByPosition(key).name;
|
||||
}
|
||||
|
||||
out << '\n';
|
||||
}
|
||||
|
||||
res.emplace_back("Keys: " + std::move(keys_str));
|
||||
|
||||
if (!aggregates.empty())
|
||||
{
|
||||
bool first = true;
|
||||
out << prefix << "Aggregates:\n";
|
||||
|
||||
for (const auto & aggregate : aggregates)
|
||||
{
|
||||
auto aggregate_strings = aggregate.explain();
|
||||
for (const auto & aggregate_str : aggregate_strings)
|
||||
{
|
||||
if (first)
|
||||
res.emplace_back("Aggregates: " + aggregate_str);
|
||||
else
|
||||
res.emplace_back(" " + aggregate_str);
|
||||
|
||||
first = false;
|
||||
}
|
||||
}
|
||||
aggregate.explain(out, ident + 4);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
Aggregator::Aggregator(const Params & params_)
|
||||
|
@ -925,7 +925,7 @@ public:
|
||||
}
|
||||
|
||||
/// Returns keys and aggregated for EXPLAIN query
|
||||
Strings explain() const;
|
||||
void explain(WriteBuffer & out, size_t ident) const;
|
||||
};
|
||||
|
||||
Aggregator(const Params & params_);
|
||||
|
@ -152,9 +152,9 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
pipeline.enableQuotaForCurrentStreams();
|
||||
}
|
||||
|
||||
Strings AggregatingStep::describeActions() const
|
||||
void AggregatingStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
return params.explain();
|
||||
params.explain(settings.out, settings.offset);
|
||||
}
|
||||
|
||||
void AggregatingStep::describePipeline(FormatSettings & settings) const
|
||||
|
@ -28,7 +28,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings &) const override;
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/ConvertingStep.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/ConvertingTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -46,19 +47,19 @@ void ConvertingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings ConvertingStep::describeActions() const
|
||||
void ConvertingStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
const auto & header = input_streams[0].header;
|
||||
auto conversion = ConvertingTransform(header, result_header, ConvertingTransform::MatchColumnsMode::Name)
|
||||
.getConversion();
|
||||
|
||||
Strings res;
|
||||
|
||||
auto get_description = [](const ColumnWithTypeAndName & elem, bool is_const)
|
||||
auto dump_description = [&](const ColumnWithTypeAndName & elem, bool is_const)
|
||||
{
|
||||
return elem.name + " " + elem.type->getName() + (is_const ? " Const" : "");
|
||||
settings.out << elem.name << ' ' << elem.type->getName() << (is_const ? " Const" : "") << '\n';
|
||||
};
|
||||
|
||||
String prefix(settings.offset, ' ');
|
||||
|
||||
for (size_t i = 0; i < conversion.size(); ++i)
|
||||
{
|
||||
const auto & from = header.getByPosition(conversion[i]);
|
||||
@ -67,13 +68,19 @@ Strings ConvertingStep::describeActions() const
|
||||
bool from_const = from.column && isColumnConst(*from.column);
|
||||
bool to_const = to.column && isColumnConst(*to.column);
|
||||
|
||||
settings.out << prefix;
|
||||
|
||||
if (from.name == to.name && from.type->equals(*to.type) && from_const == to_const)
|
||||
res.emplace_back(get_description(from, from_const));
|
||||
dump_description(from, from_const);
|
||||
else
|
||||
res.emplace_back(get_description(to, to_const) + " <- " + get_description(from, from_const));
|
||||
{
|
||||
dump_description(to, to_const);
|
||||
settings.out << " <- ";
|
||||
dump_description(from, from_const);
|
||||
}
|
||||
|
||||
settings.out << '\n';
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,7 +13,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
Block result_header;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/CreatingSetsStep.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/CreatingSetsTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -35,21 +36,20 @@ void CreatingSetsStep::transformPipeline(QueryPipeline & pipeline)
|
||||
pipeline.addCreatingSetsTransform(std::move(creating_sets));
|
||||
}
|
||||
|
||||
Strings CreatingSetsStep::describeActions() const
|
||||
void CreatingSetsStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res;
|
||||
String prefix(settings.offset, ' ');
|
||||
|
||||
for (const auto & set : subqueries_for_sets)
|
||||
{
|
||||
String str;
|
||||
settings.out << prefix;
|
||||
if (set.second.set)
|
||||
str += "Set: ";
|
||||
settings.out << "Set: ";
|
||||
else if (set.second.join)
|
||||
str += "Join: ";
|
||||
settings.out << "Join: ";
|
||||
|
||||
str += set.first;
|
||||
settings.out << set.first << '\n';
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
SubqueriesForSets subqueries_for_sets;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/DistinctStep.h>
|
||||
#include <Processors/Transforms/DistinctTransform.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -65,18 +66,27 @@ void DistinctStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings DistinctStep::describeActions() const
|
||||
void DistinctStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
String res;
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (!res.empty())
|
||||
res += ", ";
|
||||
String prefix(settings.offset, ' ');
|
||||
settings.out << prefix << "Columns: ";
|
||||
|
||||
res += column;
|
||||
if (columns.empty())
|
||||
settings.out << "none";
|
||||
else
|
||||
{
|
||||
bool first = true;
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (!first)
|
||||
settings.out << ", ";
|
||||
first = false;
|
||||
|
||||
settings.out << column;
|
||||
}
|
||||
}
|
||||
|
||||
return {"Columns: " + res};
|
||||
settings.out << '\n';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
SizeLimits set_size_limits;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/InflatingExpressionTransform.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -50,19 +51,23 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
static Strings getActionsDescription(const ExpressionActionsPtr & expression)
|
||||
static void doDescribeActions(const ExpressionActionsPtr & expression, IQueryPlanStep::FormatSettings & settings)
|
||||
{
|
||||
Strings res;
|
||||
for (const auto & action : expression->getActions())
|
||||
res.emplace_back((res.empty() ? "Actions: "
|
||||
: " ") + action.toString());
|
||||
String prefix(settings.offset, ' ');
|
||||
bool first = true;
|
||||
|
||||
return res;
|
||||
for (const auto & action : expression->getActions())
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
: " ");
|
||||
first = false;
|
||||
settings.out << action.toString() << '\n';
|
||||
}
|
||||
}
|
||||
|
||||
Strings ExpressionStep::describeActions() const
|
||||
void ExpressionStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
return getActionsDescription(expression);
|
||||
doDescribeActions(expression, settings);
|
||||
}
|
||||
|
||||
InflatingExpressionStep::InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_)
|
||||
@ -93,9 +98,9 @@ void InflatingExpressionStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings InflatingExpressionStep::describeActions() const
|
||||
void InflatingExpressionStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
return getActionsDescription(expression);
|
||||
doDescribeActions(expression, settings);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
ExpressionActionsPtr expression;
|
||||
@ -37,7 +37,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
ExpressionActionsPtr expression;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/FillingStep.h>
|
||||
#include <Processors/Transforms/FillingTransform.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -27,9 +28,11 @@ void FillingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings FillingStep::describeActions() const
|
||||
void FillingStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
return {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)};
|
||||
settings.out << String(settings.offset, ' ');
|
||||
dumpSortDescription(sort_description, input_streams.front().header, settings.out);
|
||||
settings.out << '\n';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
SortDescription sort_description;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Processors/Transforms/FilterTransform.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -55,16 +56,19 @@ void FilterStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings FilterStep::describeActions() const
|
||||
void FilterStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res;
|
||||
res.emplace_back("Filter column: " + filter_column_name);
|
||||
String prefix(settings.offset, ' ');
|
||||
settings.out << prefix << "Filter column: " << filter_column_name << '\n';
|
||||
|
||||
bool first = true;
|
||||
for (const auto & action : expression->getActions())
|
||||
res.emplace_back((res.size() == 1 ? "Actions: "
|
||||
: " ") + action.toString());
|
||||
|
||||
return res;
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
: " ");
|
||||
first = false;
|
||||
settings.out << action.toString() << '\n';
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
String getName() const override { return "Filter"; }
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
ExpressionActionsPtr expression;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <Processors/Transforms/PartialSortingTransform.h>
|
||||
#include <Processors/Transforms/FinishSortingTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -69,17 +70,20 @@ void FinishSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
}
|
||||
}
|
||||
|
||||
Strings FinishSortingStep::describeActions() const
|
||||
void FinishSortingStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res = {
|
||||
"Prefix sort description: " + dumpSortDescription(prefix_description, input_streams.front().header),
|
||||
"Result sort description: " + dumpSortDescription(result_description, input_streams.front().header)
|
||||
};
|
||||
String prefix(settings.offset, ' ');
|
||||
|
||||
settings.out << prefix << "Prefix sort description: ";
|
||||
dumpSortDescription(prefix_description, input_streams.front().header, settings.out);
|
||||
settings.out << '\n';
|
||||
|
||||
settings.out << prefix << "Result sort description: ";
|
||||
dumpSortDescription(result_description, input_streams.front().header, settings.out);
|
||||
settings.out << '\n';
|
||||
|
||||
if (limit)
|
||||
res.emplace_back("Limit " + std::to_string(limit));
|
||||
|
||||
return res;
|
||||
settings.out << prefix << "Limit " << limit << '\n';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
SortDescription prefix_description;
|
||||
|
@ -45,7 +45,7 @@ static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep:
|
||||
settings.out << prefix;
|
||||
|
||||
first = false;
|
||||
elem.dumpStructure(settings.out, true);
|
||||
elem.dumpStructure(settings.out);
|
||||
settings.out << '\n';
|
||||
}
|
||||
}
|
||||
|
@ -65,7 +65,7 @@ public:
|
||||
};
|
||||
|
||||
/// Get detailed description of step actions. This is shown in EXPLAIN query with options `actions = 1`.
|
||||
virtual Strings describeActions() const { return {}; }
|
||||
virtual void describeActions(FormatSettings & /*settings*/) const {}
|
||||
|
||||
/// Get description of processors added in current step. Should be called after updatePipeline().
|
||||
virtual void describePipeline(FormatSettings & /*settings*/) const {}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/LimitByStep.h>
|
||||
#include <Processors/Transforms/LimitByTransform.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -37,23 +38,30 @@ void LimitByStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings LimitByStep::describeActions() const
|
||||
void LimitByStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res;
|
||||
String columns_str;
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (!columns_str.empty())
|
||||
columns_str += ", ";
|
||||
String prefix(settings.offset, ' ');
|
||||
|
||||
columns_str += column;
|
||||
settings.out << prefix << "Columns: ";
|
||||
|
||||
if (columns.empty())
|
||||
settings.out << "none\n";
|
||||
else
|
||||
{
|
||||
bool first = true;
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
if (!first)
|
||||
settings.out << ", ";
|
||||
first = false;
|
||||
|
||||
settings.out << column;
|
||||
}
|
||||
settings.out << '\n';
|
||||
}
|
||||
|
||||
return {
|
||||
"Columns: " + columns_str,
|
||||
"Length " + std::to_string(group_length),
|
||||
"Offset " + std::to_string(group_offset),
|
||||
};
|
||||
settings.out << prefix << "Length " << group_length << '\n';
|
||||
settings.out << prefix << "Offset " << group_offset << '\n';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -15,7 +15,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
size_t group_length;
|
||||
@ -24,5 +24,3 @@ private:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/LimitStep.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/LimitTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -35,30 +36,30 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline)
|
||||
pipeline.addPipe({std::move(transform)});
|
||||
}
|
||||
|
||||
Strings LimitStep::describeActions() const
|
||||
void LimitStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res;
|
||||
res.emplace_back("Limit " + std::to_string(limit));
|
||||
res.emplace_back("Offset " + std::to_string(offset));
|
||||
String prefix(settings.offset, ' ');
|
||||
settings.out << prefix << "Limit " << limit << '\n';
|
||||
settings.out << prefix << "Offset " << offset << '\n';
|
||||
|
||||
if (with_ties || always_read_till_end)
|
||||
{
|
||||
settings.out << prefix;
|
||||
|
||||
String str;
|
||||
if (with_ties)
|
||||
str += "WITH TIES";
|
||||
settings.out << "WITH TIES";
|
||||
|
||||
if (always_read_till_end)
|
||||
{
|
||||
if (!str.empty())
|
||||
str += ", ";
|
||||
if (!with_ties)
|
||||
settings.out << ", ";
|
||||
|
||||
str += "Reads all data";
|
||||
settings.out << "Reads all data";
|
||||
}
|
||||
|
||||
res.emplace_back(str);
|
||||
settings.out << '\n';
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
size_t limit;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/MergeSortingStep.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/MergeSortingTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -48,15 +49,15 @@ void MergeSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings MergeSortingStep::describeActions() const
|
||||
void MergeSortingStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res = {"Sort description: " + dumpSortDescription(description, input_streams.front().header)};
|
||||
String prefix(settings.offset, ' ');
|
||||
settings.out << prefix << "Sort description: ";
|
||||
dumpSortDescription(description, input_streams.front().header, settings.out);
|
||||
settings.out << '\n';
|
||||
|
||||
if (limit)
|
||||
res.emplace_back("Limit " + std::to_string(limit));
|
||||
|
||||
return res;
|
||||
settings.out << prefix << "Limit " << limit << '\n';
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
SortDescription description;
|
||||
|
@ -63,9 +63,9 @@ void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline)
|
||||
pipeline.enableQuotaForCurrentStreams();
|
||||
}
|
||||
|
||||
Strings MergingAggregatedStep::describeActions() const
|
||||
void MergingAggregatedStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
return params->params.explain();
|
||||
return params->params.explain(settings.out, settings.offset);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
AggregatingTransformParamsPtr params;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/MergingSortedStep.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Merges/MergingSortedTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -46,14 +47,12 @@ void MergingSortedStep::transformPipeline(QueryPipeline & pipeline)
|
||||
}
|
||||
}
|
||||
|
||||
Strings MergingSortedStep::describeActions() const
|
||||
void MergingSortedStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)};
|
||||
|
||||
if (limit)
|
||||
res.emplace_back("Limit " + std::to_string(limit));
|
||||
|
||||
return res;
|
||||
String prefix(settings.offset, ' ');
|
||||
settings.out << prefix << "Sort description: ";
|
||||
dumpSortDescription(sort_description, input_streams.front().header, settings.out);
|
||||
settings.out << '\n';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
SortDescription sort_description;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Processors/QueryPlan/OffsetStep.h>
|
||||
#include <Processors/OffsetTransform.h>
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -30,9 +31,9 @@ void OffsetStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings OffsetStep::describeActions() const
|
||||
void OffsetStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
return {"Offset " + std::to_string(offset)};
|
||||
settings.out << String(settings.offset, ' ') << "Offset " << offset << '\n';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
size_t offset;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/PartialSortingTransform.h>
|
||||
#include <Processors/Transforms/LimitsCheckingTransform.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -50,14 +51,12 @@ void PartialSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||
});
|
||||
}
|
||||
|
||||
Strings PartialSortingStep::describeActions() const
|
||||
void PartialSortingStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)};
|
||||
|
||||
if (limit)
|
||||
res.emplace_back("Limit " + std::to_string(limit));
|
||||
|
||||
return res;
|
||||
String prefix(settings.offset, ' ');
|
||||
settings.out << prefix << "Sort description: ";
|
||||
dumpSortDescription(sort_description, input_streams.front().header, settings.out);
|
||||
settings.out << '\n';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
SortDescription sort_description;
|
||||
|
@ -178,60 +178,55 @@ void QueryPlan::addInterpreterContext(std::shared_ptr<Context> context)
|
||||
|
||||
|
||||
static void explainStep(
|
||||
WriteBuffer & buffer, IQueryPlanStep & step, size_t ident, const QueryPlan::ExplainPlanOptions & options)
|
||||
const IQueryPlanStep & step,
|
||||
IQueryPlanStep::FormatSettings & settings,
|
||||
const QueryPlan::ExplainPlanOptions & options)
|
||||
{
|
||||
std::string prefix(ident, ' ');
|
||||
buffer << prefix;
|
||||
buffer << step.getName();
|
||||
std::string prefix(settings.offset, ' ');
|
||||
settings.out << prefix;
|
||||
settings.out << step.getName();
|
||||
|
||||
const auto & description = step.getStepDescription();
|
||||
if (options.description && !description.empty())
|
||||
buffer <<" (" << description << ')';
|
||||
settings.out <<" (" << description << ')';
|
||||
|
||||
buffer.write('\n');
|
||||
settings.out.write('\n');
|
||||
|
||||
if (options.header)
|
||||
{
|
||||
buffer << prefix;
|
||||
settings.out << prefix;
|
||||
|
||||
if (!step.hasOutputStream())
|
||||
buffer << "No header";
|
||||
settings.out << "No header";
|
||||
else if (!step.getOutputStream().header)
|
||||
buffer << "Empty header";
|
||||
settings.out << "Empty header";
|
||||
else
|
||||
{
|
||||
buffer << "Header: ";
|
||||
settings.out << "Header: ";
|
||||
bool first = true;
|
||||
|
||||
for (const auto & elem : step.getOutputStream().header)
|
||||
{
|
||||
if (!first)
|
||||
buffer << "\n" << prefix << " ";
|
||||
settings.out << "\n" << prefix << " ";
|
||||
|
||||
first = false;
|
||||
elem.dumpStructure(buffer, true);
|
||||
elem.dumpStructure(settings.out);
|
||||
}
|
||||
}
|
||||
|
||||
buffer.write('\n');
|
||||
settings.out.write('\n');
|
||||
}
|
||||
|
||||
if (options.actions)
|
||||
{
|
||||
auto actions = step.describeActions();
|
||||
if (!actions.empty())
|
||||
{
|
||||
for (auto & action : actions)
|
||||
buffer << prefix << action << '\n';
|
||||
}
|
||||
}
|
||||
step.describeActions(settings);
|
||||
}
|
||||
|
||||
void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options)
|
||||
{
|
||||
checkInitialized();
|
||||
|
||||
size_t ident = 2;
|
||||
IQueryPlanStep::FormatSettings settings{.out = buffer, .write_header = options.header};
|
||||
|
||||
struct Frame
|
||||
{
|
||||
@ -249,7 +244,8 @@ void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & opt
|
||||
|
||||
if (!frame.is_description_printed)
|
||||
{
|
||||
explainStep(buffer, *frame.node->step, (stack.size() - 1) * ident, options);
|
||||
settings.offset = (stack.size() - 1) * settings.ident;
|
||||
explainStep(*frame.node->step, settings, options);
|
||||
frame.is_description_printed = true;
|
||||
}
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Processors/QueryPipeline.h>
|
||||
#include <Processors/Transforms/TotalsHavingTransform.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -62,17 +63,20 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
Strings TotalsHavingStep::describeActions() const
|
||||
void TotalsHavingStep::describeActions(FormatSettings & settings) const
|
||||
{
|
||||
Strings res;
|
||||
res.emplace_back("Filter column: " + filter_column_name);
|
||||
res.emplace_back("Mode: " + totalsModeToString(totals_mode, auto_include_threshold));
|
||||
String prefix(settings.offset, ' ');
|
||||
settings.out << prefix << "Filter column: " << filter_column_name << '\n';
|
||||
settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n';
|
||||
|
||||
bool first = true;
|
||||
for (const auto & action : expression->getActions())
|
||||
res.emplace_back((res.size() == 2 ? "Actions: "
|
||||
: " ") + action.toString());
|
||||
|
||||
return res;
|
||||
{
|
||||
settings.out << prefix << (first ? "Actions: "
|
||||
: " ");
|
||||
first = false;
|
||||
settings.out << action.toString() << '\n';
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
|
||||
void transformPipeline(QueryPipeline & pipeline) override;
|
||||
|
||||
Strings describeActions() const override;
|
||||
void describeActions(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
bool overflow_row;
|
||||
|
Loading…
Reference in New Issue
Block a user