mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 19:42:00 +00:00
More actions description.
This commit is contained in:
parent
d88e19b69d
commit
ca01094f4b
41
src/Core/SortDescription.cpp
Normal file
41
src/Core/SortDescription.cpp
Normal file
@ -0,0 +1,41 @@
|
||||
#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;
|
||||
|
@ -1,4 +1,4 @@
|
||||
#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
|
||||
|
Loading…
Reference in New Issue
Block a user