提交 ca01094f 编写于 作者: N Nikolai Kochetov

More actions description.

上级 d88e19b6
#include <Core/SortDescription.h>
#include <Core/Block.h>
namespace DB
{
String dumpSortDescription(const SortDescription & description, const Block & header)
{
String res;
for (const auto & desc : description)
{
if (!res.empty())
res += ", ";
if (!desc.column_name.empty())
res += desc.column_name;
else
{
if (desc.column_number < header.columns())
res += header.getByPosition(desc.column_number).name;
else
res += "?";
res += " (pos " + std::to_string(desc.column_number) + ")";
}
if (desc.direction > 0)
res += " ASC";
else
res += " DESC";
if (desc.with_fill)
res += " WITH FILL";
}
return res;
}
}
......@@ -71,4 +71,7 @@ struct SortColumnDescription
/// Description of the sorting rule for several columns.
using SortDescription = std::vector<SortColumnDescription>;
class Block;
String dumpSortDescription(const SortDescription & description, const Block & header);
}
......@@ -20,6 +20,7 @@ SRCS(
NamesAndTypes.cpp
Settings.cpp
SettingsCollection.cpp
SortDescription.cpp
)
END()
......@@ -7,34 +7,8 @@ namespace DB
Strings AggregateDescription::explain() const
{
Strings res;
String arguments_pos_str;
for (auto arg : arguments)
{
if (!arguments_pos_str.empty())
arguments_pos_str += ", ";
arguments_pos_str += std::to_string(arg);
}
if (arguments_pos_str.empty())
arguments_pos_str = "none";
res.emplace_back("argument positions: " + arguments_pos_str);
String arguments_names_str;
for (const auto & arg : argument_names)
{
if (!arguments_names_str.empty())
arguments_names_str += ", ";
arguments_names_str += arg;
}
if (arguments_names_str.empty())
arguments_names_str = "none";
res.emplace_back("arguments: " + arguments_names_str);
res.emplace_back("column_name: " + column_name);
res.emplace_back(column_name);
auto get_params_string = [](const Array & arr)
{
......@@ -65,14 +39,42 @@ Strings AggregateDescription::explain() const
if (!params_str.empty())
params_str = "(" + params_str + ")";
res.emplace_back("function: " + function->getName() + params_str + '(' + types_str + ") -> " +
res.emplace_back(" Function: " + function->getName() + params_str + '(' + types_str + ") -> " +
function->getReturnType()->getName());
}
else
res.emplace_back("function: nullptr");
res.emplace_back(" Function: nullptr");
if (!parameters.empty())
res.emplace_back("parameters: " + get_params_string(parameters));
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;
}
if (arguments_names_str.empty())
arguments_names_str = "none";
res.emplace_back(" Arguments: " + arguments_names_str);
String arguments_pos_str;
for (auto arg : arguments)
{
if (!arguments_pos_str.empty())
arguments_pos_str += ", ";
arguments_pos_str += std::to_string(arg);
}
if (arguments_pos_str.empty())
arguments_pos_str = "none";
res.emplace_back(" Argument positions: " + arguments_pos_str);
return res;
}
......
......@@ -158,21 +158,33 @@ Strings Aggregator::Params::explain() const
String keys_str;
for (auto key : keys)
{
if (keys_str.empty())
if (!keys_str.empty())
keys_str += ", ";
if (key >= header.columns())
keys_str += "unknown position " + std::to_string(key);
else
keys_str += src_header.getByPosition(key).name;
keys_str += header.getByPosition(key).name;
}
res.emplace_back("keys: " + std::move(keys_str));
res.emplace_back("Keys: " + std::move(keys_str));
for (const auto & aggregate : aggregates)
if (!aggregates.empty())
{
auto aggregate_strings = aggregate.explain();
res.insert(res.end(), aggregate_strings.begin(), aggregate_strings.end());
bool first = true;
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;
}
}
}
return res;
......
......@@ -53,7 +53,7 @@
#include <Processors/QueryPlan/CubeStep.h>
#include <Processors/QueryPlan/FillingStep.h>
#include <Processors/QueryPlan/ExtremesStep.h>
#include <Processors/QueryPlan/OffsetsStep.h>
#include <Processors/QueryPlan/OffsetStep.h>
#include <Processors/QueryPlan/FinishSortingStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
......@@ -1792,7 +1792,7 @@ void InterpreterSelectQuery::executeOffset(QueryPlan & query_plan)
UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context);
auto offsets_step = std::make_unique<OffsetsStep>(query_plan.getCurrentDataStream(), limit_offset);
auto offsets_step = std::make_unique<OffsetStep>(query_plan.getCurrentDataStream(), limit_offset);
query_plan.addStep(std::move(offsets_step));
}
}
......
......@@ -46,4 +46,34 @@ void ConvertingStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings ConvertingStep::describeActions() 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)
{
return elem.name + " " + elem.type->getName() + (is_const ? " Const" : "");
};
for (size_t i = 0; i < conversion.size(); ++i)
{
const auto & from = header.getByPosition(conversion[i]);
const auto & to = result_header.getByPosition(i);
bool from_const = from.column && isColumnConst(*from.column);
bool to_const = to.column && isColumnConst(*to.column);
if (from.name == to.name && from.type->equals(*to.type) && from_const == to_const)
res.emplace_back(get_description(from, from_const));
else
res.emplace_back(get_description(to, to_const) + " <- " + get_description(from, from_const));
}
return res;
}
}
......@@ -13,6 +13,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
Block result_header;
};
......
......@@ -35,4 +35,21 @@ void CreatingSetsStep::transformPipeline(QueryPipeline & pipeline)
pipeline.addCreatingSetsTransform(std::move(creating_sets));
}
Strings CreatingSetsStep::describeActions() const
{
Strings res;
for (const auto & set : subqueries_for_sets)
{
String str;
if (set.second.set)
str += "Set: ";
else if (set.second.join)
str += "Join: ";
str += set.first;
}
return res;
}
}
......@@ -19,6 +19,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
SubqueriesForSets subqueries_for_sets;
SizeLimits network_transfer_limits;
......
......@@ -65,4 +65,18 @@ void DistinctStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings DistinctStep::describeActions() const
{
String res;
for (const auto & column : columns)
{
if (!res.empty())
res += ", ";
res += column;
}
return {"Columns: " + res};
}
}
......@@ -19,6 +19,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
SizeLimits set_size_limits;
UInt64 limit_hint;
......
......@@ -59,6 +59,21 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline)
});
}
static Strings getActionsDescription(const ExpressionActionsPtr & expression)
{
Strings res;
for (const auto & action : expression->getActions())
res.emplace_back((res.empty() ? "Actions: "
: " ") + action.toString());
return res;
}
Strings ExpressionStep::describeActions() const
{
return getActionsDescription(expression);
}
InflatingExpressionStep::InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_)
: ITransformingStep(
input_stream_,
......@@ -88,4 +103,9 @@ void InflatingExpressionStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings InflatingExpressionStep::describeActions() const
{
return getActionsDescription(expression);
}
}
......@@ -15,6 +15,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
ExpressionActionsPtr expression;
bool default_totals; /// See ExpressionTransform
......@@ -29,6 +31,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
ExpressionActionsPtr expression;
bool default_totals; /// See ExpressionTransform
......
......@@ -27,4 +27,9 @@ void FillingStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings FillingStep::describeActions() const
{
return {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)};
}
}
......@@ -14,6 +14,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
SortDescription sort_description;
};
......
......@@ -55,4 +55,16 @@ void FilterStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings FilterStep::describeActions() const
{
Strings res;
res.emplace_back("Filter column: " + filter_column_name);
for (const auto & action : expression->getActions())
res.emplace_back((res.size() == 1 ? "Actions: "
: " ") + action.toString());
return res;
}
}
......@@ -19,6 +19,8 @@ public:
String getName() const override { return "Filter"; }
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
ExpressionActionsPtr expression;
String filter_column_name;
......
......@@ -69,4 +69,17 @@ void FinishSortingStep::transformPipeline(QueryPipeline & pipeline)
}
}
Strings FinishSortingStep::describeActions() const
{
Strings res = {
"Prefix sort description: " + dumpSortDescription(prefix_description, input_streams.front().header),
"Result sort description: " + dumpSortDescription(result_description, input_streams.front().header)
};
if (limit)
res.emplace_back("Limit " + std::to_string(limit));
return res;
}
}
......@@ -19,6 +19,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
SortDescription prefix_description;
SortDescription result_description;
......
......@@ -37,4 +37,23 @@ void LimitByStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings LimitByStep::describeActions() const
{
Strings res;
String columns_str;
for (const auto & column : columns)
{
if (!columns_str.empty())
columns_str += ", ";
columns_str += column;
}
return {
"Columns: " + columns_str,
"Length " + std::to_string(group_length),
"Offset " + std::to_string(group_offset),
};
}
}
......@@ -15,6 +15,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
size_t group_length;
size_t group_offset;
......
......@@ -35,4 +35,30 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline)
pipeline.addPipe({std::move(transform)});
}
Strings LimitStep::describeActions() const
{
Strings res;
res.emplace_back("Limit " + std::to_string(limit));
res.emplace_back("Offset " + std::to_string(offset));
if (with_ties || always_read_till_end)
{
String str;
if (with_ties)
str += "WITH TIES";
if (always_read_till_end)
{
if (!str.empty())
str += ", ";
str += "Reads all data";
}
res.emplace_back(str);
}
return res;
}
}
......@@ -20,6 +20,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
size_t limit;
size_t offset;
......
......@@ -48,4 +48,15 @@ void MergeSortingStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings MergeSortingStep::describeActions() const
{
Strings res = {"Sort description: " + dumpSortDescription(description, input_streams.front().header)};
if (limit)
res.emplace_back("Limit " + std::to_string(limit));
return res;
}
}
......@@ -24,6 +24,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
SortDescription description;
size_t max_merged_block_size;
......
......@@ -63,4 +63,9 @@ void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline)
pipeline.enableQuotaForCurrentStreams();
}
Strings MergingAggregatedStep::describeActions() const
{
return params->params.explain();
}
}
......@@ -22,6 +22,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
AggregatingTransformParamsPtr params;
bool memory_efficient_aggregation;
......
......@@ -46,4 +46,14 @@ void MergingSortedStep::transformPipeline(QueryPipeline & pipeline)
}
}
Strings MergingSortedStep::describeActions() const
{
Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)};
if (limit)
res.emplace_back("Limit " + std::to_string(limit));
return res;
}
}
......@@ -20,6 +20,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
SortDescription sort_description;
size_t max_block_size;
......
#include <Processors/QueryPlan/OffsetsStep.h>
#include <Processors/QueryPlan/OffsetStep.h>
#include <Processors/OffsetTransform.h>
#include <Processors/QueryPipeline.h>
......@@ -13,13 +13,13 @@ static ITransformingStep::DataStreamTraits getTraits()
};
}
OffsetsStep::OffsetsStep(const DataStream & input_stream_, size_t offset_)
OffsetStep::OffsetStep(const DataStream & input_stream_, size_t offset_)
: ITransformingStep(input_stream_, input_stream_.header, getTraits())
, offset(offset_)
{
}
void OffsetsStep::transformPipeline(QueryPipeline & pipeline)
void OffsetStep::transformPipeline(QueryPipeline & pipeline)
{
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
......@@ -30,4 +30,9 @@ void OffsetsStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings OffsetStep::describeActions() const
{
return {"Offset " + std::to_string(offset)};
}
}
......@@ -5,15 +5,17 @@
namespace DB
{
class OffsetsStep : public ITransformingStep
class OffsetStep : public ITransformingStep
{
public:
OffsetsStep(const DataStream & input_stream_, size_t offset_);
OffsetStep(const DataStream & input_stream_, size_t offset_);
String getName() const override { return "Offsets"; }
String getName() const override { return "Offset"; }
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
size_t offset;
};
......
......@@ -50,4 +50,14 @@ void PartialSortingStep::transformPipeline(QueryPipeline & pipeline)
});
}
Strings PartialSortingStep::describeActions() const
{
Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)};
if (limit)
res.emplace_back("Limit " + std::to_string(limit));
return res;
}
}
......@@ -19,6 +19,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
SortDescription sort_description;
UInt64 limit;
......
......@@ -207,7 +207,7 @@ static void explainStep(
for (const auto & elem : step.getOutputStream().header)
{
if (!first)
buffer << ",\n" << prefix << " ";
buffer << "\n" << prefix << " ";
first = false;
elem.dumpStructure(buffer, true);
......@@ -222,17 +222,8 @@ static void explainStep(
auto actions = step.describeActions();
if (!actions.empty())
{
buffer << "Actions: ";
bool first = true;
for (auto & action : actions)
{
if (!first)
buffer << ",\n" << prefix << " ";
first = false;
buffer << action;
}
buffer << prefix << action << '\n';
}
}
}
......
......@@ -2,6 +2,7 @@
#include <Processors/Transforms/DistinctTransform.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Transforms/TotalsHavingTransform.h>
#include <Interpreters/ExpressionActions.h>
namespace DB
{
......@@ -44,4 +45,34 @@ void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline)
pipeline.addTotalsHavingTransform(std::move(totals_having));
}
static String totalsModeToString(TotalsMode totals_mode, double auto_include_threshold)
{
switch (totals_mode)
{
case TotalsMode::BEFORE_HAVING:
return "before_having";
case TotalsMode::AFTER_HAVING_INCLUSIVE:
return "after_having_inclusive";
case TotalsMode::AFTER_HAVING_EXCLUSIVE:
return "after_having_exclusive";
case TotalsMode::AFTER_HAVING_AUTO:
return "after_having_auto threshold " + std::to_string(auto_include_threshold);
}
__builtin_unreachable();
}
Strings TotalsHavingStep::describeActions() const
{
Strings res;
res.emplace_back("Filter column: " + filter_column_name);
res.emplace_back("Mode: " + totalsModeToString(totals_mode, auto_include_threshold));
for (const auto & action : expression->getActions())
res.emplace_back((res.size() == 2 ? "Actions: "
: " ") + action.toString());
return res;
}
}
......@@ -25,6 +25,8 @@ public:
void transformPipeline(QueryPipeline & pipeline) override;
Strings describeActions() const override;
private:
bool overflow_row;
ExpressionActionsPtr expression;
......
......@@ -35,6 +35,8 @@ public:
String getName() const override { return "Converting"; }
const ColumnNumbers & getConversion() const { return conversion; }
protected:
void transform(Chunk & chunk) override;
......
......@@ -157,7 +157,7 @@ SRCS(
QueryPlan/MergeSortingStep.cpp
QueryPlan/MergingAggregatedStep.cpp
QueryPlan/MergingSortedStep.cpp
QueryPlan/OffsetsStep.cpp
QueryPlan/OffsetStep.cpp
QueryPlan/PartialSortingStep.cpp
QueryPlan/UnionStep.cpp
QueryPlan/ReadFromPreparedSource.cpp
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册