提交 45adacf0 编写于 作者: N Nikolai Kochetov

Use QueryPlan in InterpreterSelectQuery [part 2].

上级 e0841360
......@@ -77,6 +77,9 @@ public:
/// Execute a query. Get the stream of blocks to read.
BlockIO execute() override;
/// Builds QueryPlan for current query.
void buildQueryPlan(QueryPlan & query_plan);
bool ignoreLimits() const override { return options.ignore_limits; }
bool ignoreQuota() const override { return options.ignore_quota; }
......
......@@ -6,10 +6,9 @@
#include <Columns/getLeastSuperColumn.h>
#include <Common/typeid_cast.h>
#include <Parsers/queryToString.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Pipe.h>
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/UnionStep.h>
namespace DB
......@@ -173,47 +172,35 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(
return cache[key] = InterpreterSelectWithUnionQuery(query_ptr, context, SelectQueryOptions().analyze()).getSampleBlock();
}
BlockIO InterpreterSelectWithUnionQuery::execute()
void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
{
BlockIO res;
QueryPipeline & main_pipeline = res.pipeline;
std::vector<QueryPipeline> pipelines;
bool has_main_pipeline = false;
Blocks headers;
headers.reserve(nested_interpreters.size());
size_t num_plans = nested_interpreters.size();
std::vector<QueryPlan> plans(num_plans);
DataStreams data_streams(num_plans);
for (auto & interpreter : nested_interpreters)
for (size_t i = 0; i < num_plans; ++i)
{
if (!has_main_pipeline)
{
has_main_pipeline = true;
main_pipeline = interpreter->execute().pipeline;
headers.emplace_back(main_pipeline.getHeader());
}
else
{
pipelines.emplace_back(interpreter->execute().pipeline);
headers.emplace_back(pipelines.back().getHeader());
}
nested_interpreters[i]->buildQueryPlan(plans[i]);
data_streams[i] = plans[i].getCurrentDataStream();
}
if (!has_main_pipeline)
main_pipeline.init(Pipe(std::make_shared<NullSource>(getSampleBlock())));
auto max_threads = context->getSettingsRef().max_threads;
auto union_step = std::make_unique<UnionStep>(std::move(data_streams), result_header, max_threads);
if (!pipelines.empty())
{
auto common_header = getCommonHeaderForUnion(headers);
main_pipeline.unitePipelines(std::move(pipelines), common_header);
query_plan.unitePlans(std::move(union_step), std::move(plans));
}
// nested queries can force 1 thread (due to simplicity)
// but in case of union this cannot be done.
UInt64 max_threads = context->getSettingsRef().max_threads;
main_pipeline.setMaxThreads(std::min<UInt64>(nested_interpreters.size(), max_threads));
}
BlockIO InterpreterSelectWithUnionQuery::execute()
{
BlockIO res;
QueryPlan query_plan;
buildQueryPlan(query_plan);
auto pipeline = query_plan.buildQueryPipeline();
main_pipeline.addInterpreterContext(context);
res.pipeline = std::move(*pipeline);
res.pipeline.addInterpreterContext(context);
return res;
}
......
......@@ -5,14 +5,12 @@
#include <Interpreters/SelectQueryOptions.h>
#include <Parsers/IAST_fwd.h>
#include <Processors/QueryPipeline.h>
namespace DB
{
class Context;
class InterpreterSelectQuery;
class QueryPlan;
/** Interprets one or multiple SELECT queries inside UNION ALL chain.
*/
......@@ -27,6 +25,9 @@ public:
~InterpreterSelectWithUnionQuery() override;
/// Builds QueryPlan for current query.
void buildQueryPlan(QueryPlan & query_plan);
BlockIO execute() override;
bool ignoreLimits() const override { return options.ignore_limits; }
......
......@@ -563,7 +563,7 @@ void QueryPipeline::setOutputFormat(ProcessorPtr output)
}
void QueryPipeline::unitePipelines(
std::vector<QueryPipeline> && pipelines, const Block & common_header)
std::vector<std::unique_ptr<QueryPipeline>> pipelines, const Block & common_header)
{
if (initialized())
{
......@@ -583,8 +583,9 @@ void QueryPipeline::unitePipelines(
if (totals_having_port)
totals.push_back(totals_having_port);
for (auto & pipeline : pipelines)
for (auto & pipeline_ptr : pipelines)
{
auto & pipeline = *pipeline_ptr;
pipeline.checkInitialized();
if (!pipeline.isCompleted())
......
......@@ -135,7 +135,7 @@ public:
void enableQuotaForCurrentStreams();
void unitePipelines(std::vector<QueryPipeline> && pipelines, const Block & common_header);
void unitePipelines(std::vector<std::unique_ptr<QueryPipeline>> pipelines, const Block & common_header);
PipelineExecutorPtr execute();
......
#include <Processors/QueryPlan/ConvertingStep.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Transforms/ConvertingTransform.h>
namespace DB
{
static ITransformingStep::DataStreamTraits getTraits()
{
return ITransformingStep::DataStreamTraits{
.preserves_distinct_columns = true
};
}
static void filterDistinctColumns(const Block & res_header, NameSet & distinct_columns)
{
if (distinct_columns.empty())
return;
NameSet new_distinct_columns;
for (const auto & column : res_header)
if (distinct_columns.count(column.name))
new_distinct_columns.insert(column.name);
distinct_columns.swap(new_distinct_columns);
}
ConvertingStep::ConvertingStep(const DataStream & input_stream_, Block result_header_)
: ITransformingStep(
input_stream_,
result_header_,
getTraits())
, result_header(std::move(result_header_))
{
/// Some columns may be removed
filterDistinctColumns(output_stream->header, output_stream->distinct_columns);
filterDistinctColumns(output_stream->header, output_stream->local_distinct_columns);
}
void ConvertingStep::transformPipeline(QueryPipeline & pipeline)
{
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ConvertingTransform>(header, result_header, ConvertingTransform::MatchColumnsMode::Name);
});
}
}
#pragma once
#include <Processors/QueryPlan/ITransformingStep.h>
namespace DB
{
class ConvertingStep : public ITransformingStep
{
public:
ConvertingStep(const DataStream & input_stream_, Block result_header_);
String getName() const override { return "Converting"; }
void transformPipeline(QueryPipeline & pipeline) override;
private:
Block result_header;
};
}
......@@ -40,7 +40,7 @@ void MergeSortingStep::transformPipeline(QueryPipeline & pipeline)
return std::make_shared<MergeSortingTransform>(
header, description, max_merged_block_size, limit,
max_bytes_before_remerge,
max_bytes_before_remerge / pipeline.getNumStreams(),
max_bytes_before_external_sort,
tmp_volume,
min_free_disk_space);
......
......@@ -35,6 +35,43 @@ const DataStream & QueryPlan::getCurrentDataStream() const
return root->step->getOutputStream();
}
void QueryPlan::unitePlans(QueryPlanStepPtr step, std::vector<QueryPlan> plans)
{
if (isInitialized())
throw Exception("Cannot unite plans because current QueryPlan is already initialized",
ErrorCodes::LOGICAL_ERROR);
const auto & inputs = step->getInputStreams();
size_t num_inputs = step->getInputStreams().size();
if (num_inputs != plans.size())
{
throw Exception("Cannot unite QueryPlans using " + step->getName() +
" because step has different number of inputs. "
"Has " + std::to_string(plans.size()) + " plans "
"and " + std::to_string(num_inputs) + " inputs", ErrorCodes::LOGICAL_ERROR);
}
for (size_t i = 0; i < num_inputs; ++i)
{
const auto & step_header = inputs[i].header;
const auto & plan_header = plans[i].getCurrentDataStream().header;
if (!blocksHaveEqualStructure(step_header, plan_header))
throw Exception("Cannot unite QueryPlans using " + step->getName() + " because "
"it has incompatible header with plan " + root->step->getName() + " "
"plan header: " + plan_header.dumpStructure() +
"step header: " + step_header.dumpStructure(), ErrorCodes::LOGICAL_ERROR);
}
for (auto & plan : plans)
nodes.insert(nodes.end(), plan.nodes.begin(), plan.nodes.end());
nodes.emplace_back(Node{.step = std::move(step)});
root = &nodes.back();
for (auto & plan : plans)
root->children.emplace_back(plan.root);
}
void QueryPlan::addStep(QueryPlanStepPtr step)
{
checkNotCompleted();
......@@ -48,6 +85,7 @@ void QueryPlan::addStep(QueryPlanStepPtr step)
"step has no inputs, but QueryPlan is already initialised", ErrorCodes::LOGICAL_ERROR);
nodes.emplace_back(Node{.step = std::move(step)});
root = &nodes.back();
return;
}
......@@ -100,7 +138,12 @@ QueryPipelinePtr QueryPlan::buildQueryPipeline()
size_t next_child = frame.pipelines.size();
if (next_child == frame.node->children.size())
{
bool limit_max_threads = frame.pipelines.empty();
last_pipeline = frame.node->step->updatePipeline(std::move(frame.pipelines));
if (limit_max_threads)
last_pipeline->setMaxThreads(max_threads);
stack.pop();
}
else
......
......@@ -18,6 +18,7 @@ using QueryPipelinePtr = std::unique_ptr<QueryPipeline>;
class QueryPlan
{
public:
void unitePlans(QueryPlanStepPtr step, std::vector<QueryPlan> plans);
void addStep(QueryPlanStepPtr step);
bool isInitialized() const { return root != nullptr; } /// Tree is not empty
......@@ -26,6 +27,10 @@ public:
QueryPipelinePtr buildQueryPipeline();
/// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines.
/// TODO: make it in a better way.
void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; }
private:
struct Node
{
......@@ -40,6 +45,8 @@ private:
void checkInitialized() const;
void checkNotCompleted() const;
size_t max_threads = 0;
};
}
#pragma once
#include <Processors/QueryPlan/ITransformingStep.h>
#include <DataStreams/SizeLimits.h>
namespace DB
{
......
#include <Processors/QueryPlan/UnionStep.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Sources/NullSource.h>
#include <Interpreters/Context.h>
namespace DB
{
UnionStep::UnionStep(DataStreams input_streams_, Block result_header, size_t max_threads_)
: header(std::move(result_header))
, max_threads(max_threads_)
{
input_streams = std::move(input_streams_);
/// TODO: update traits
output_stream = DataStream{.header = header};
}
QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines)
{
auto pipeline = std::make_unique<QueryPipeline>();
if (pipelines.empty())
{
pipeline->init(Pipe(std::make_shared<NullSource>(output_stream->header)));
return pipeline;
}
size_t num_pipelines = pipelines.size();
pipeline->unitePipelines(std::move(pipelines), output_stream->header);
if (num_pipelines > 1)
{
// nested queries can force 1 thread (due to simplicity)
// but in case of union this cannot be done.
pipeline->setMaxThreads(std::min<UInt64>(num_pipelines, max_threads));
}
}
}
#pragma once
#include <Processors/QueryPlan/IQueryPlanStep.h>
namespace DB
{
class UnionStep : public IQueryPlanStep
{
public:
/// max_threads is used to limit the number of threads for result pipeline.
UnionStep(DataStreams input_streams_, Block result_header, size_t max_threads_);
String getName() const override { return "Union"; }
QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override;
private:
Block header;
size_t max_threads;
};
}
......@@ -139,6 +139,7 @@ SRCS(
Transforms/AggregatingInOrderTransform.cpp
QueryPlan/AddingDelayedStreamStep.cpp
QueryPlan/AggregatingStep.cpp
QueryPlan/ConvertingStep.cpp
QueryPlan/CreatingSetsStep.cpp
QueryPlan/CubeStep.cpp
QueryPlan/DistinctStep.cpp
......@@ -157,6 +158,7 @@ SRCS(
QueryPlan/MergingSortedStep.cpp
QueryPlan/OffsetsStep.cpp
QueryPlan/PartialSortingStep.cpp
QueryPlan/UnionStep.cpp
QueryPlan/ReadFromPreparedSource.cpp
QueryPlan/ReadFromStorageStep.cpp
QueryPlan/ReadNothingStep.cpp
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册