mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Update memory optimisation for MergingSorted.
This commit is contained in:
parent
158b4c26b7
commit
6e479b301a
@ -2303,14 +2303,12 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input
|
||||
{
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
const auto & query = getSelectQuery();
|
||||
auto finish_sorting_step = std::make_unique<FinishSortingStep>(
|
||||
query_plan.getCurrentDataStream(),
|
||||
input_sorting_info->order_key_prefix_descr,
|
||||
output_order_descr,
|
||||
settings.max_block_size,
|
||||
limit,
|
||||
query.hasFiltration());
|
||||
limit);
|
||||
|
||||
query_plan.addStep(std::move(finish_sorting_step));
|
||||
}
|
||||
|
@ -16,7 +16,7 @@ public:
|
||||
const Block & header, size_t num_inputs,
|
||||
SortDescription description_, size_t max_block_size)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
SortDescription description,
|
||||
size_t max_block_size)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
header,
|
||||
num_inputs,
|
||||
params,
|
||||
|
@ -15,7 +15,7 @@ public:
|
||||
SortDescription description_, size_t max_block_size,
|
||||
Graphite::Params params_, time_t time_of_merge_)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
|
@ -15,10 +15,10 @@ IMergingTransformBase::IMergingTransformBase(
|
||||
const Block & input_header,
|
||||
const Block & output_header,
|
||||
bool have_all_inputs_,
|
||||
bool has_limit_below_one_block_)
|
||||
size_t limit_hint_)
|
||||
: IProcessor(InputPorts(num_inputs, input_header), {output_header})
|
||||
, have_all_inputs(have_all_inputs_)
|
||||
, has_limit_below_one_block(has_limit_below_one_block_)
|
||||
, limit_hint(limit_hint_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -79,7 +79,10 @@ IProcessor::Status IMergingTransformBase::prepareInitializeInputs()
|
||||
/// setNotNeeded after reading first chunk, because in optimismtic case
|
||||
/// (e.g. with optimized 'ORDER BY primary_key LIMIT n' and small 'n')
|
||||
/// we won't have to read any chunks anymore;
|
||||
auto chunk = input.pull(has_limit_below_one_block);
|
||||
auto chunk = input.pull(limit_hint != 0);
|
||||
if (limit_hint && chunk.getNumRows() < limit_hint)
|
||||
input.setNeeded();
|
||||
|
||||
if (!chunk.hasRows())
|
||||
{
|
||||
if (!input.isFinished())
|
||||
|
@ -17,7 +17,7 @@ public:
|
||||
const Block & input_header,
|
||||
const Block & output_header,
|
||||
bool have_all_inputs_,
|
||||
bool has_limit_below_one_block_);
|
||||
size_t limit_hint_);
|
||||
|
||||
OutputPort & getOutputPort() { return outputs.front(); }
|
||||
|
||||
@ -67,7 +67,7 @@ private:
|
||||
std::vector<InputState> input_states;
|
||||
std::atomic<bool> have_all_inputs;
|
||||
bool is_initialized = false;
|
||||
bool has_limit_below_one_block = false;
|
||||
size_t limit_hint = 0;
|
||||
|
||||
IProcessor::Status prepareInitializeInputs();
|
||||
};
|
||||
@ -83,9 +83,9 @@ public:
|
||||
const Block & input_header,
|
||||
const Block & output_header,
|
||||
bool have_all_inputs_,
|
||||
bool has_limit_below_one_block_,
|
||||
size_t limit_hint_,
|
||||
Args && ... args)
|
||||
: IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_, has_limit_below_one_block_)
|
||||
: IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_, limit_hint_)
|
||||
, algorithm(std::forward<Args>(args) ...)
|
||||
{
|
||||
}
|
||||
|
@ -13,13 +13,12 @@ MergingSortedTransform::MergingSortedTransform(
|
||||
SortDescription description_,
|
||||
size_t max_block_size,
|
||||
UInt64 limit_,
|
||||
bool has_limit_below_one_block_,
|
||||
WriteBuffer * out_row_sources_buf_,
|
||||
bool quiet_,
|
||||
bool use_average_block_sizes,
|
||||
bool have_all_inputs_)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, have_all_inputs_, has_limit_below_one_block_,
|
||||
num_inputs, header, header, have_all_inputs_, limit_,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
|
@ -17,7 +17,6 @@ public:
|
||||
SortDescription description,
|
||||
size_t max_block_size,
|
||||
UInt64 limit_ = 0,
|
||||
bool has_limit_below_one_block_ = false,
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool quiet_ = false,
|
||||
bool use_average_block_sizes = false,
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
const Names & partition_key_columns,
|
||||
size_t max_block_size)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
WriteBuffer * out_row_sources_buf_ = nullptr,
|
||||
bool use_average_block_sizes = false)
|
||||
: IMergingTransform(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
header,
|
||||
num_inputs,
|
||||
std::move(description_),
|
||||
|
@ -31,14 +31,12 @@ FinishSortingStep::FinishSortingStep(
|
||||
SortDescription prefix_description_,
|
||||
SortDescription result_description_,
|
||||
size_t max_block_size_,
|
||||
UInt64 limit_,
|
||||
bool has_filtration_)
|
||||
UInt64 limit_)
|
||||
: ITransformingStep(input_stream_, input_stream_.header, getTraits(limit_))
|
||||
, prefix_description(std::move(prefix_description_))
|
||||
, result_description(std::move(result_description_))
|
||||
, max_block_size(max_block_size_)
|
||||
, limit(limit_)
|
||||
, has_filtration(has_filtration_)
|
||||
{
|
||||
/// TODO: check input_stream is sorted by prefix_description.
|
||||
output_stream->sort_description = result_description;
|
||||
@ -60,14 +58,12 @@ void FinishSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const
|
||||
if (pipeline.getNumStreams() > 1)
|
||||
{
|
||||
UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit);
|
||||
bool has_limit_below_one_block = !has_filtration && limit_for_merging && limit_for_merging < max_block_size;
|
||||
auto transform = std::make_shared<MergingSortedTransform>(
|
||||
pipeline.getHeader(),
|
||||
pipeline.getNumStreams(),
|
||||
prefix_description,
|
||||
max_block_size,
|
||||
limit_for_merging,
|
||||
has_limit_below_one_block);
|
||||
limit_for_merging);
|
||||
|
||||
pipeline.addTransform(std::move(transform));
|
||||
}
|
||||
|
@ -14,8 +14,7 @@ public:
|
||||
SortDescription prefix_description_,
|
||||
SortDescription result_description_,
|
||||
size_t max_block_size_,
|
||||
UInt64 limit_,
|
||||
bool has_filtration_);
|
||||
UInt64 limit_);
|
||||
|
||||
String getName() const override { return "FinishSorting"; }
|
||||
|
||||
@ -32,7 +31,6 @@ private:
|
||||
SortDescription result_description;
|
||||
size_t max_block_size;
|
||||
UInt64 limit;
|
||||
bool has_filtration;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -485,8 +485,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
|
||||
pipe.getHeader(),
|
||||
pipe.numOutputPorts(),
|
||||
sort_description,
|
||||
max_block_size,
|
||||
0, true);
|
||||
max_block_size);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
}
|
||||
|
@ -127,7 +127,7 @@ ColumnGathererTransform::ColumnGathererTransform(
|
||||
ReadBuffer & row_sources_buf_,
|
||||
size_t block_preferred_size_)
|
||||
: IMergingTransform<ColumnGathererStream>(
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false,
|
||||
num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0,
|
||||
num_inputs, row_sources_buf_, block_preferred_size_)
|
||||
, log(&Poco::Logger::get("ColumnGathererStream"))
|
||||
{
|
||||
|
@ -197,7 +197,6 @@ void MergeSortingTransform::consume(Chunk chunk)
|
||||
description,
|
||||
max_merged_block_size,
|
||||
limit,
|
||||
false,
|
||||
nullptr,
|
||||
quiet,
|
||||
use_average_block_sizes,
|
||||
|
@ -83,7 +83,7 @@ TEST(MergingSortedTest, SimpleBlockSizeTest)
|
||||
EXPECT_EQ(pipe.numOutputPorts(), 3);
|
||||
|
||||
auto transform = std::make_shared<MergingSortedTransform>(pipe.getHeader(), pipe.numOutputPorts(), sort_description,
|
||||
DEFAULT_MERGE_BLOCK_SIZE, 0, false, nullptr, false, true);
|
||||
DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
|
||||
@ -130,7 +130,7 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes)
|
||||
EXPECT_EQ(pipe.numOutputPorts(), 3);
|
||||
|
||||
auto transform = std::make_shared<MergingSortedTransform>(pipe.getHeader(), pipe.numOutputPorts(), sort_description,
|
||||
DEFAULT_MERGE_BLOCK_SIZE, 0, false, nullptr, false, true);
|
||||
DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true);
|
||||
|
||||
pipe.addTransform(std::move(transform));
|
||||
|
||||
|
@ -778,7 +778,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
|
||||
{
|
||||
case MergeTreeData::MergingParams::Ordinary:
|
||||
merged_transform = std::make_shared<MergingSortedTransform>(
|
||||
header, pipes.size(), sort_description, merge_block_size, 0, false, ctx->rows_sources_write_buf.get(), true, ctx->blocks_are_granules_size);
|
||||
header, pipes.size(), sort_description, merge_block_size, 0, ctx->rows_sources_write_buf.get(), true, ctx->blocks_are_granules_size);
|
||||
break;
|
||||
|
||||
case MergeTreeData::MergingParams::Collapsing:
|
||||
|
Loading…
Reference in New Issue
Block a user