mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Fix review comments
This commit is contained in:
parent
86c5280db3
commit
15bdeba074
@ -44,7 +44,10 @@ void SortColumnDescription::explain(JSONBuilder::JSONMap & map) const
|
||||
|
||||
bool SortDescription::hasPrefix(const SortDescription & prefix) const
|
||||
{
|
||||
if (prefix.empty() || prefix.size() > size())
|
||||
if (prefix.empty())
|
||||
return true;
|
||||
|
||||
if (prefix.size() > size())
|
||||
return false;
|
||||
|
||||
for (size_t i = 0; i < prefix.size(); ++i)
|
||||
|
@ -12,12 +12,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static Poco::Logger * getLogger()
|
||||
{
|
||||
static Poco::Logger & logger = Poco::Logger::get("ExpressionStep");
|
||||
return &logger;
|
||||
}
|
||||
|
||||
static ITransformingStep::Traits getTraits(const ActionsDAGPtr & actions, const Block & header, const SortDescription & sort_description)
|
||||
{
|
||||
return ITransformingStep::Traits
|
||||
@ -41,8 +35,6 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, const ActionsDA
|
||||
getTraits(actions_dag_, input_stream_.header, input_stream_.sort_description))
|
||||
, actions_dag(actions_dag_)
|
||||
{
|
||||
LOG_DEBUG(getLogger(), "ActionsDAG:\n{}", actions_dag->dumpDAG());
|
||||
|
||||
/// Some columns may be removed by expression.
|
||||
updateDistinctColumns(output_stream->header, output_stream->distinct_columns);
|
||||
}
|
||||
|
@ -41,7 +41,7 @@ SortingStep::SortingStep(
|
||||
size_t min_free_disk_space_,
|
||||
bool optimize_sorting_for_input_stream_)
|
||||
: ITransformingStep(input_stream, input_stream.header, getTraits(limit_))
|
||||
, type(Type::Auto)
|
||||
, type(Type::Full)
|
||||
, result_description(description_)
|
||||
, max_block_size(max_block_size_)
|
||||
, limit(limit_)
|
||||
@ -233,38 +233,10 @@ void SortingStep::fullSort(QueryPipelineBuilder & pipeline, const SortDescriptio
|
||||
}
|
||||
}
|
||||
|
||||
static Poco::Logger * getLogger()
|
||||
{
|
||||
static Poco::Logger & logger = Poco::Logger::get("SortingStep");
|
||||
return &logger;
|
||||
}
|
||||
|
||||
void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
const auto input_sort_mode = input_streams.front().sort_mode;
|
||||
const SortDescription & input_sort_desc = input_streams.front().sort_description;
|
||||
|
||||
// dump sorting info
|
||||
LOG_DEBUG(getLogger(), "Sort type : {}", type);
|
||||
LOG_DEBUG(getLogger(), "Sort mode : {}", input_sort_mode);
|
||||
LOG_DEBUG(getLogger(), "Input ({}): {}", input_sort_desc.size(), dumpSortDescription(input_sort_desc));
|
||||
LOG_DEBUG(getLogger(), "Prefix({}): {}", prefix_description.size(), dumpSortDescription(prefix_description));
|
||||
LOG_DEBUG(getLogger(), "Result({}): {}", result_description.size(), dumpSortDescription(result_description));
|
||||
|
||||
if (optimize_sorting_for_input_stream)
|
||||
{
|
||||
if (input_sort_mode == DataStream::SortMode::Stream && input_sort_desc.hasPrefix(result_description))
|
||||
return;
|
||||
|
||||
/// merge sorted
|
||||
if (input_sort_mode == DataStream::SortMode::Port && input_sort_desc.hasPrefix(result_description))
|
||||
{
|
||||
LOG_DEBUG(getLogger(), "MergingSorted, SortMode::Port");
|
||||
mergingSorted(pipeline, result_description, limit);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
/// we consider that a caller has more information what sorting to apply (depends on what constructor was used)
|
||||
/// so we'll try to infer what sorting to use only in case of Full sorting
|
||||
if (type == Type::MergingSorted)
|
||||
{
|
||||
mergingSorted(pipeline, result_description, limit);
|
||||
@ -282,18 +254,31 @@ void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const Build
|
||||
return;
|
||||
}
|
||||
|
||||
if (optimize_sorting_for_input_stream && input_sort_mode == DataStream::SortMode::Chunk)
|
||||
const auto input_sort_mode = input_streams.front().sort_mode;
|
||||
const SortDescription & input_sort_desc = input_streams.front().sort_description;
|
||||
if (optimize_sorting_for_input_stream)
|
||||
{
|
||||
if (input_sort_desc.hasPrefix(result_description))
|
||||
/// skip sorting if stream is already sorted
|
||||
if (input_sort_mode == DataStream::SortMode::Stream && input_sort_desc.hasPrefix(result_description))
|
||||
return;
|
||||
|
||||
/// merge sorted
|
||||
if (input_sort_mode == DataStream::SortMode::Port && input_sort_desc.hasPrefix(result_description))
|
||||
{
|
||||
type = Type::MergingSorted;
|
||||
mergingSorted(pipeline, result_description, limit);
|
||||
return;
|
||||
}
|
||||
|
||||
/// if chunks already sorted according to result_sort_desc, then we can skip chunk sorting
|
||||
if (input_sort_mode == DataStream::SortMode::Chunk && input_sort_desc.hasPrefix(result_description))
|
||||
{
|
||||
LOG_DEBUG(getLogger(), "Almost FullSort");
|
||||
const bool skip_partial_sort = true;
|
||||
fullSort(pipeline, result_description, limit, skip_partial_sort);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
LOG_DEBUG(getLogger(), "FullSort");
|
||||
fullSort(pipeline, result_description, limit);
|
||||
}
|
||||
|
||||
|
@ -65,11 +65,11 @@ private:
|
||||
QueryPipelineBuilder & pipeline,
|
||||
const SortDescription & result_sort_desc,
|
||||
UInt64 limit_,
|
||||
bool skip_partial_sort = false); /// if chunks already sorted according to result_sort_desc, then skip chunk sorting
|
||||
bool skip_partial_sort = false);
|
||||
|
||||
enum class Type
|
||||
{
|
||||
Auto,
|
||||
Full,
|
||||
FinishSorting,
|
||||
MergingSorted,
|
||||
};
|
||||
|
Loading…
Reference in New Issue
Block a user