mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-18 21:51:57 +00:00
Limit push down.
This commit is contained in:
parent
e6508db4f0
commit
9c6627a7eb
@ -42,6 +42,15 @@ FinishSortingStep::FinishSortingStep(
|
|||||||
output_stream->sort_mode = DataStream::SortMode::Stream;
|
output_stream->sort_mode = DataStream::SortMode::Stream;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void FinishSortingStep::updateLimit(size_t limit_)
|
||||||
|
{
|
||||||
|
if (limit_ && (limit == 0 || limit_ < limit))
|
||||||
|
{
|
||||||
|
limit = limit_;
|
||||||
|
transform_traits.preserves_number_of_rows = limit == 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void FinishSortingStep::transformPipeline(QueryPipeline & pipeline)
|
void FinishSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||||
{
|
{
|
||||||
bool need_finish_sorting = (prefix_description.size() < result_description.size());
|
bool need_finish_sorting = (prefix_description.size() < result_description.size());
|
||||||
|
@ -22,6 +22,9 @@ public:
|
|||||||
|
|
||||||
void describeActions(FormatSettings & settings) const override;
|
void describeActions(FormatSettings & settings) const override;
|
||||||
|
|
||||||
|
/// Add limit or change it to lower value.
|
||||||
|
void updateLimit(size_t limit_);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
SortDescription prefix_description;
|
SortDescription prefix_description;
|
||||||
SortDescription result_description;
|
SortDescription result_description;
|
||||||
|
@ -23,6 +23,8 @@ public:
|
|||||||
|
|
||||||
void describeActions(FormatSettings & settings) const override;
|
void describeActions(FormatSettings & settings) const override;
|
||||||
|
|
||||||
|
size_t limitPlusOffset() const { return limit + offset; }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
size_t limit;
|
size_t limit;
|
||||||
size_t offset;
|
size_t offset;
|
||||||
|
@ -45,6 +45,15 @@ MergeSortingStep::MergeSortingStep(
|
|||||||
: DataStream::SortMode::Port;
|
: DataStream::SortMode::Port;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void MergeSortingStep::updateLimit(size_t limit_)
|
||||||
|
{
|
||||||
|
if (limit_ && (limit == 0 || limit_ < limit))
|
||||||
|
{
|
||||||
|
limit = limit_;
|
||||||
|
transform_traits.preserves_number_of_rows = limit == 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void MergeSortingStep::transformPipeline(QueryPipeline & pipeline)
|
void MergeSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||||
{
|
{
|
||||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
||||||
|
@ -27,6 +27,9 @@ public:
|
|||||||
|
|
||||||
void describeActions(FormatSettings & settings) const override;
|
void describeActions(FormatSettings & settings) const override;
|
||||||
|
|
||||||
|
/// Add limit or change it to lower value.
|
||||||
|
void updateLimit(size_t limit_);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
SortDescription description;
|
SortDescription description;
|
||||||
size_t max_merged_block_size;
|
size_t max_merged_block_size;
|
||||||
|
@ -37,6 +37,15 @@ MergingSortedStep::MergingSortedStep(
|
|||||||
output_stream->sort_mode = DataStream::SortMode::Stream;
|
output_stream->sort_mode = DataStream::SortMode::Stream;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void MergingSortedStep::updateLimit(size_t limit_)
|
||||||
|
{
|
||||||
|
if (limit_ && (limit == 0 || limit_ < limit))
|
||||||
|
{
|
||||||
|
limit = limit_;
|
||||||
|
transform_traits.preserves_number_of_rows = limit == 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void MergingSortedStep::transformPipeline(QueryPipeline & pipeline)
|
void MergingSortedStep::transformPipeline(QueryPipeline & pipeline)
|
||||||
{
|
{
|
||||||
/// If there are several streams, then we merge them into one
|
/// If there are several streams, then we merge them into one
|
||||||
|
@ -23,6 +23,9 @@ public:
|
|||||||
|
|
||||||
void describeActions(FormatSettings & settings) const override;
|
void describeActions(FormatSettings & settings) const override;
|
||||||
|
|
||||||
|
/// Add limit or change it to lower value.
|
||||||
|
void updateLimit(size_t limit_);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
SortDescription sort_description;
|
SortDescription sort_description;
|
||||||
size_t max_block_size;
|
size_t max_block_size;
|
||||||
|
@ -37,6 +37,15 @@ PartialSortingStep::PartialSortingStep(
|
|||||||
output_stream->sort_mode = DataStream::SortMode::Chunk;
|
output_stream->sort_mode = DataStream::SortMode::Chunk;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void PartialSortingStep::updateLimit(size_t limit_)
|
||||||
|
{
|
||||||
|
if (limit_ && (limit == 0 || limit_ < limit))
|
||||||
|
{
|
||||||
|
limit = limit_;
|
||||||
|
transform_traits.preserves_number_of_rows = limit == 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void PartialSortingStep::transformPipeline(QueryPipeline & pipeline)
|
void PartialSortingStep::transformPipeline(QueryPipeline & pipeline)
|
||||||
{
|
{
|
||||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
||||||
|
@ -22,6 +22,9 @@ public:
|
|||||||
|
|
||||||
void describeActions(FormatSettings & settings) const override;
|
void describeActions(FormatSettings & settings) const override;
|
||||||
|
|
||||||
|
/// Add limit or change it to lower value.
|
||||||
|
void updateLimit(size_t limit_);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
SortDescription sort_description;
|
SortDescription sort_description;
|
||||||
UInt64 limit;
|
UInt64 limit;
|
||||||
|
@ -5,6 +5,10 @@
|
|||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <stack>
|
#include <stack>
|
||||||
#include <Processors/QueryPlan/LimitStep.h>
|
#include <Processors/QueryPlan/LimitStep.h>
|
||||||
|
#include "MergingSortedStep.h"
|
||||||
|
#include "FinishSortingStep.h"
|
||||||
|
#include "MergeSortingStep.h"
|
||||||
|
#include "PartialSortingStep.h"
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -129,6 +133,7 @@ void QueryPlan::addStep(QueryPlanStepPtr step)
|
|||||||
QueryPipelinePtr QueryPlan::buildQueryPipeline()
|
QueryPipelinePtr QueryPlan::buildQueryPipeline()
|
||||||
{
|
{
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
|
optimize();
|
||||||
|
|
||||||
struct Frame
|
struct Frame
|
||||||
{
|
{
|
||||||
@ -308,8 +313,50 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static void tryPushDownLimit(QueryPlanStepPtr & parent, QueryPlanStepPtr & child)
|
/// If plan looks like Limit -> Sorting, update limit for Sorting
|
||||||
|
bool tryUpdateLimitForSortingSteps(QueryPlan::Node * node, size_t limit)
|
||||||
{
|
{
|
||||||
|
QueryPlanStepPtr & step = node->step;
|
||||||
|
QueryPlan::Node * child = nullptr;
|
||||||
|
bool updated = false;
|
||||||
|
|
||||||
|
if (auto * merging_sorted = typeid_cast<MergingSortedStep *>(step.get()))
|
||||||
|
{
|
||||||
|
/// TODO: remove LimitStep here.
|
||||||
|
merging_sorted->updateLimit(limit);
|
||||||
|
updated = true;
|
||||||
|
child = node->children.front();
|
||||||
|
}
|
||||||
|
else if (auto * finish_sorting = typeid_cast<FinishSortingStep *>(step.get()))
|
||||||
|
{
|
||||||
|
/// TODO: remove LimitStep here.
|
||||||
|
finish_sorting->updateLimit(limit);
|
||||||
|
updated = true;
|
||||||
|
}
|
||||||
|
else if (auto * merge_sorting = typeid_cast<MergeSortingStep *>(step.get()))
|
||||||
|
{
|
||||||
|
merge_sorting->updateLimit(limit);
|
||||||
|
updated = true;
|
||||||
|
child = node->children.front();
|
||||||
|
}
|
||||||
|
else if (auto * partial_sorting = typeid_cast<PartialSortingStep *>(step.get()))
|
||||||
|
{
|
||||||
|
partial_sorting->updateLimit(limit);
|
||||||
|
updated = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// We often have chain PartialSorting -> MergeSorting -> MergingSorted
|
||||||
|
/// Try update limit for them also if possible.
|
||||||
|
if (child)
|
||||||
|
tryUpdateLimitForSortingSteps(child, limit);
|
||||||
|
|
||||||
|
return updated;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Move LimitStep down if possible.
|
||||||
|
static void tryPushDownLimit(QueryPlanStepPtr & parent, QueryPlan::Node * child_node)
|
||||||
|
{
|
||||||
|
auto & child = child_node->step;
|
||||||
const auto * limit = typeid_cast<const LimitStep *>(parent.get());
|
const auto * limit = typeid_cast<const LimitStep *>(parent.get());
|
||||||
|
|
||||||
if (!limit)
|
if (!limit)
|
||||||
@ -321,6 +368,10 @@ static void tryPushDownLimit(QueryPlanStepPtr & parent, QueryPlanStepPtr & child
|
|||||||
if (!transforming)
|
if (!transforming)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
|
/// Special cases for sorting steps.
|
||||||
|
if (tryUpdateLimitForSortingSteps(child_node, limit->limitPlusOffset()))
|
||||||
|
return;
|
||||||
|
|
||||||
/// Now we should decide if pushing down limit possible for this step.
|
/// Now we should decide if pushing down limit possible for this step.
|
||||||
|
|
||||||
const auto & transform_traits = transforming->getTransformTraits();
|
const auto & transform_traits = transforming->getTransformTraits();
|
||||||
@ -356,7 +407,7 @@ void QueryPlan::optimize()
|
|||||||
{
|
{
|
||||||
/// First entrance, try push down.
|
/// First entrance, try push down.
|
||||||
if (frame.node->children.size() == 1)
|
if (frame.node->children.size() == 1)
|
||||||
tryPushDownLimit(frame.node->step, frame.node->children.front()->step);
|
tryPushDownLimit(frame.node->step, frame.node->children.front());
|
||||||
}
|
}
|
||||||
|
|
||||||
if (frame.next_child < frame.node->children.size())
|
if (frame.next_child < frame.node->children.size())
|
||||||
|
@ -62,7 +62,6 @@ public:
|
|||||||
|
|
||||||
void addInterpreterContext(std::shared_ptr<Context> context);
|
void addInterpreterContext(std::shared_ptr<Context> context);
|
||||||
|
|
||||||
private:
|
|
||||||
/// Tree node. Step and it's children.
|
/// Tree node. Step and it's children.
|
||||||
struct Node
|
struct Node
|
||||||
{
|
{
|
||||||
@ -70,6 +69,8 @@ private:
|
|||||||
std::vector<Node *> children = {};
|
std::vector<Node *> children = {};
|
||||||
};
|
};
|
||||||
|
|
||||||
|
private:
|
||||||
|
|
||||||
using Nodes = std::list<Node>;
|
using Nodes = std::list<Node>;
|
||||||
Nodes nodes;
|
Nodes nodes;
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user