mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Cleanup: related to #38719 (SortingStep: deduce way to sort based on input stream sort)
This commit is contained in:
parent
61ce5161f3
commit
b7f46d954e
@ -10,7 +10,7 @@ namespace DB
|
||||
MergingSortedAlgorithm::MergingSortedAlgorithm(
|
||||
Block header_,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size,
|
||||
SortingQueueStrategy sorting_queue_strategy_,
|
||||
UInt64 limit_,
|
||||
@ -18,7 +18,7 @@ MergingSortedAlgorithm::MergingSortedAlgorithm(
|
||||
bool use_average_block_sizes)
|
||||
: header(std::move(header_))
|
||||
, merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size)
|
||||
, description(std::move(description_))
|
||||
, description(description_)
|
||||
, limit(limit_)
|
||||
, out_row_sources_buf(out_row_sources_buf_)
|
||||
, current_inputs(num_inputs)
|
||||
@ -29,7 +29,7 @@ MergingSortedAlgorithm::MergingSortedAlgorithm(
|
||||
sort_description_types.reserve(description.size());
|
||||
|
||||
/// Replace column names in description to positions.
|
||||
for (auto & column_description : description)
|
||||
for (const auto & column_description : description)
|
||||
{
|
||||
has_collation |= column_description.collator != nullptr;
|
||||
sort_description_types.emplace_back(header.getByName(column_description.column_name).type);
|
||||
|
@ -16,7 +16,7 @@ public:
|
||||
MergingSortedAlgorithm(
|
||||
Block header_,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size,
|
||||
SortingQueueStrategy sorting_queue_strategy_,
|
||||
UInt64 limit_ = 0,
|
||||
@ -38,7 +38,7 @@ private:
|
||||
|
||||
/// Settings
|
||||
SortDescription description;
|
||||
UInt64 limit;
|
||||
const UInt64 limit;
|
||||
bool has_collation = false;
|
||||
|
||||
/// Used in Vertical merge algorithm to gather non-PK/non-index columns (on next step)
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
MergingSortedTransform::MergingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description_,
|
||||
const SortDescription & description_,
|
||||
size_t max_block_size,
|
||||
SortingQueueStrategy sorting_queue_strategy,
|
||||
UInt64 limit_,
|
||||
@ -19,10 +19,14 @@ MergingSortedTransform::MergingSortedTransform(
|
||||
bool use_average_block_sizes,
|
||||
bool have_all_inputs_)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, have_all_inputs_, limit_,
|
||||
num_inputs,
|
||||
header,
|
||||
header,
|
||||
have_all_inputs_,
|
||||
limit_,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
description_,
|
||||
max_block_size,
|
||||
sorting_queue_strategy,
|
||||
limit_,
|
||||
|
@ -14,7 +14,7 @@ public:
|
||||
MergingSortedTransform(
|
||||
const Block & header,
|
||||
size_t num_inputs,
|
||||
SortDescription description,
|
||||
const SortDescription & description,
|
||||
size_t max_block_size,
|
||||
SortingQueueStrategy sorting_queue_strategy,
|
||||
UInt64 limit_ = 0,
|
||||
|
@ -36,9 +36,8 @@ FinishSortingTransform::FinishSortingTransform(
|
||||
|
||||
/// The target description is modified in SortingTransform constructor.
|
||||
/// To avoid doing the same actions with description_sorted just copy it from prefix of target description.
|
||||
size_t prefix_size = description_sorted_.size();
|
||||
for (size_t i = 0; i < prefix_size; ++i)
|
||||
description_with_positions.emplace_back(description[i], header_without_constants.getPositionByName(description[i].column_name));
|
||||
for (const auto & column_sort_desc : description_sorted_)
|
||||
description_with_positions.emplace_back(column_sort_desc, header_without_constants.getPositionByName(column_sort_desc.column_name));
|
||||
}
|
||||
|
||||
void FinishSortingTransform::consume(Chunk chunk)
|
||||
|
@ -85,7 +85,7 @@ bool compareWithThreshold(const ColumnRawPtrs & raw_block_columns, size_t min_bl
|
||||
}
|
||||
|
||||
PartialSortingTransform::PartialSortingTransform(
|
||||
const Block & header_, SortDescription & description_, UInt64 limit_)
|
||||
const Block & header_, const SortDescription & description_, UInt64 limit_)
|
||||
: ISimpleTransform(header_, header_, false)
|
||||
, description(description_)
|
||||
, limit(limit_)
|
||||
@ -93,9 +93,8 @@ PartialSortingTransform::PartialSortingTransform(
|
||||
// Sorting by no columns doesn't make sense.
|
||||
assert(!description_.empty());
|
||||
|
||||
size_t description_size = description.size();
|
||||
for (size_t i = 0; i < description_size; ++i)
|
||||
description_with_positions.emplace_back(description[i], header_.getPositionByName(description[i].column_name));
|
||||
for (const auto & column_sort_desc : description)
|
||||
description_with_positions.emplace_back(column_sort_desc, header_.getPositionByName(column_sort_desc.column_name));
|
||||
}
|
||||
|
||||
void PartialSortingTransform::transform(Chunk & chunk)
|
||||
|
@ -15,7 +15,7 @@ public:
|
||||
/// limit - if not 0, then you can sort each block not completely, but only `limit` first rows by order.
|
||||
PartialSortingTransform(
|
||||
const Block & header_,
|
||||
SortDescription & description_,
|
||||
const SortDescription & description_,
|
||||
UInt64 limit_ = 0);
|
||||
|
||||
String getName() const override { return "PartialSortingTransform"; }
|
||||
@ -26,9 +26,9 @@ protected:
|
||||
void transform(Chunk & chunk) override;
|
||||
|
||||
private:
|
||||
SortDescription description;
|
||||
const SortDescription description;
|
||||
SortDescriptionWithPositions description_with_positions;
|
||||
UInt64 limit;
|
||||
const UInt64 limit;
|
||||
RowsBeforeLimitCounterPtr read_rows;
|
||||
|
||||
Columns sort_description_threshold_columns;
|
||||
|
@ -73,8 +73,8 @@ public:
|
||||
~SortingTransform() override;
|
||||
|
||||
protected:
|
||||
Status prepare() override final;
|
||||
void work() override final;
|
||||
Status prepare() final;
|
||||
void work() final;
|
||||
|
||||
virtual void consume(Chunk chunk) = 0;
|
||||
virtual void generate() = 0;
|
||||
@ -82,7 +82,7 @@ protected:
|
||||
|
||||
SortDescription description;
|
||||
size_t max_merged_block_size;
|
||||
UInt64 limit;
|
||||
const UInt64 limit;
|
||||
|
||||
/// Before operation, will remove constant columns from blocks. And after, place constant columns back.
|
||||
/// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files)
|
||||
|
Loading…
Reference in New Issue
Block a user