From 3e5c3fb40b15c9b4b5a70c2597cb30847d420105 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Thu, 2 Mar 2023 16:27:53 +0800 Subject: [PATCH 01/24] update partA --- .../Flash/Coprocessor/FineGrainedShuffle.h | 5 +++ .../Flash/Pipeline/Exec/PipelineExecBuilder.h | 8 ---- .../Exec/tests/gtest_simple_operator.cpp | 6 +-- dbms/src/Flash/Pipeline/Pipeline.cpp | 12 ++---- .../Events/FineGrainedPipelineEvent.cpp | 32 +++++++++++++++ .../Events/FineGrainedPipelineEvent.h | 41 +++++++++++++++++++ dbms/src/Flash/Planner/PhysicalPlan.cpp | 5 +-- dbms/src/Flash/Planner/PhysicalPlanNode.cpp | 17 +++++++- dbms/src/Flash/Planner/PhysicalPlanNode.h | 23 ++++++++++- .../Planner/Plans/PhysicalAggregation.cpp | 4 +- .../Flash/Planner/Plans/PhysicalAggregation.h | 9 ++-- dbms/src/Flash/Planner/Plans/PhysicalBinary.h | 3 +- .../Plans/PhysicalExchangeReceiver.cpp | 23 +++++++---- .../Planner/Plans/PhysicalExchangeReceiver.h | 10 ++++- .../Planner/Plans/PhysicalExchangeSender.cpp | 13 +++--- .../Planner/Plans/PhysicalExchangeSender.h | 13 +++--- .../Flash/Planner/Plans/PhysicalExpand.cpp | 10 +++-- dbms/src/Flash/Planner/Plans/PhysicalExpand.h | 9 +++- .../Flash/Planner/Plans/PhysicalFilter.cpp | 9 +++- dbms/src/Flash/Planner/Plans/PhysicalFilter.h | 9 +++- .../Planner/Plans/PhysicalGetResultSink.cpp | 17 ++++++-- .../Planner/Plans/PhysicalGetResultSink.h | 10 ++++- dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp | 4 +- dbms/src/Flash/Planner/Plans/PhysicalJoin.h | 8 ++-- dbms/src/Flash/Planner/Plans/PhysicalLeaf.h | 3 +- .../src/Flash/Planner/Plans/PhysicalLimit.cpp | 9 +++- dbms/src/Flash/Planner/Plans/PhysicalLimit.h | 9 +++- .../Plans/PhysicalMockExchangeReceiver.cpp | 16 +++++--- .../Plans/PhysicalMockExchangeReceiver.h | 9 +++- .../Plans/PhysicalMockExchangeSender.cpp | 1 + .../Plans/PhysicalMockExchangeSender.h | 10 ++++- .../Planner/Plans/PhysicalMockTableScan.cpp | 12 ++++-- .../Planner/Plans/PhysicalMockTableScan.h | 7 +++- .../Planner/Plans/PhysicalProjection.cpp | 11 ++++- .../Flash/Planner/Plans/PhysicalProjection.h | 9 +++- .../Flash/Planner/Plans/PhysicalTableScan.cpp | 4 +- .../Flash/Planner/Plans/PhysicalTableScan.h | 1 + dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp | 11 +++-- dbms/src/Flash/Planner/Plans/PhysicalTopN.h | 9 +++- dbms/src/Flash/Planner/Plans/PhysicalUnary.h | 3 +- .../Flash/Planner/Plans/PhysicalWindow.cpp | 4 +- dbms/src/Flash/Planner/Plans/PhysicalWindow.h | 8 ++-- .../Planner/Plans/PhysicalWindowSort.cpp | 4 +- .../Flash/Planner/Plans/PhysicalWindowSort.h | 7 ++-- 44 files changed, 328 insertions(+), 119 deletions(-) create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp create mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h diff --git a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h index c6bced12ba4..60debed2fde 100644 --- a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h +++ b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h @@ -28,6 +28,11 @@ inline bool enableFineGrainedShuffle(uint64_t stream_count) struct FineGrainedShuffle { + FineGrainedShuffle() + : stream_count(0) + , batch_size(0) + {} + explicit FineGrainedShuffle(const tipb::Executor * executor) : stream_count(executor ? executor->fine_grained_shuffle_stream_count() : 0) , batch_size(executor ? executor->fine_grained_shuffle_batch_size() : 0) diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h b/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h index 4f0a986a932..1211b387125 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExecBuilder.h @@ -18,8 +18,6 @@ namespace DB { -class PipelineExecutorStatus; - struct PipelineExecBuilder { SourceOpPtr source_op; @@ -41,12 +39,6 @@ struct PipelineExecGroupBuilder using BuilderGroup = std::vector<PipelineExecBuilder>; BuilderGroup group; - explicit PipelineExecGroupBuilder(PipelineExecutorStatus & exec_status_) - : exec_status(exec_status_) - {} - - PipelineExecutorStatus & exec_status; - size_t concurrency = 0; void init(size_t init_concurrency); diff --git a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp index 56c61959d7e..f62dd3aa61c 100644 --- a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp +++ b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp @@ -59,12 +59,12 @@ class SimpleOperatorTestRunner : public DB::tests::ExecutorTest PhysicalPlan physical_plan{context.context, ""}; physical_plan.build(request.get()); assert(!result_handler.isIgnored()); - auto plan_tree = PhysicalGetResultSink::build(std::move(result_handler), physical_plan.outputAndOptimize()); + auto plan_tree = PhysicalGetResultSink::build(std::move(result_handler), Logger::get(), physical_plan.outputAndOptimize()); - PipelineExecGroupBuilder group_builder{exec_status}; + PipelineExecGroupBuilder group_builder; PhysicalPlanVisitor::visitPostOrder(plan_tree, [&](const PhysicalPlanNodePtr & plan) { assert(plan); - plan->buildPipelineExec(group_builder, context.context, /*concurrency=*/1); + plan->buildPipelineExecGroup(exec_status, group_builder, context.context, /*concurrency=*/1); }); auto result = group_builder.build(); assert(result.size() == 1); diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index fa789746c8b..6806ba98829 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -76,16 +76,16 @@ void Pipeline::toTreeString(FmtBuffer & buffer, size_t level) const void Pipeline::addGetResultSink(ResultHandler && result_handler) { assert(!plan_nodes.empty()); - auto get_result_sink = PhysicalGetResultSink::build(std::move(result_handler), plan_nodes.back()); + auto get_result_sink = PhysicalGetResultSink::build(std::move(result_handler), log, plan_nodes.back()); addPlanNode(get_result_sink); } PipelineExecGroup Pipeline::buildExecGroup(PipelineExecutorStatus & exec_status, Context & context, size_t concurrency) { assert(!plan_nodes.empty()); - PipelineExecGroupBuilder builder{exec_status}; + PipelineExecGroupBuilder builder; for (const auto & plan_node : plan_nodes) - plan_node->buildPipelineExec(builder, context, concurrency); + plan_node->buildPipelineExecGroup(exec_status, builder, context, concurrency); return builder.build(); } @@ -128,12 +128,6 @@ bool Pipeline::isSupported(const tipb::DAGRequest & dag_request) traverseExecutors( &dag_request, [&](const tipb::Executor & executor) { - // TODO support fine grained shuffle. - if (FineGrainedShuffle(&executor).enable()) - { - is_supported = false; - return false; - } switch (executor.tp()) { case tipb::ExecType::TypeProjection: diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp new file mode 100644 index 00000000000..f6d45197e29 --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp @@ -0,0 +1,32 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include <Flash/Pipeline/Pipeline.h> +#include <Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h> +#include <Flash/Pipeline/Schedule/Tasks/PipelineTask.h> + +namespace DB +{ +std::vector<TaskPtr> FineGrainedPipelineEvent::scheduleImpl() +{ + assert(pipeline); + auto pipeline_exec_group = pipeline->buildExecGroup(exec_status, context, partition_id); + assert(!pipeline_exec_group.empty()); + std::vector<TaskPtr> tasks; + tasks.reserve(pipeline_exec_group.size()); + for (auto & pipline_exec : pipeline_exec_group) + tasks.push_back(std::make_unique<PipelineTask>(mem_tracker, log->identifier(), exec_status, shared_from_this(), std::move(pipline_exec))); + return tasks; +} +} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h new file mode 100644 index 00000000000..9f92db4f3cd --- /dev/null +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h @@ -0,0 +1,41 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include <Flash/Pipeline/Schedule/Events/PipelineEvent.h> + +namespace DB +{ +class FineGrainedPipelineEvent : public PipelineEvent +{ +public: + FineGrainedPipelineEvent( + PipelineExecutorStatus & exec_status_, + MemoryTrackerPtr mem_tracker_, + const String & req_id, + Context & context_, + const PipelinePtr & pipeline_, + size_t partition_id_) + : PipelineEvent(exec_status_, std::move(mem_tracker_), req_id, context_, pipeline_) + , partition_id(partition_id_) + {} + +protected: + std::vector<TaskPtr> scheduleImpl() override; + +private: + size_t partition_id; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 32b75f2f7a2..8493d1b7df2 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -153,14 +153,13 @@ void PhysicalPlan::build(const String & executor_id, const tipb::Executor * exec GET_METRIC(tiflash_coprocessor_executor_count, type_exchange_receiver).Increment(); if (unlikely(context.isExecutorTest() || context.isInterpreterTest())) { - size_t fine_grained_stream_count = executor->has_fine_grained_shuffle_stream_count() ? executor->fine_grained_shuffle_stream_count() : 0; - pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log, executor->exchange_receiver(), fine_grained_stream_count)); + pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log, executor->exchange_receiver(), FineGrainedShuffle(executor))); } else { // for MPP test, we can use real exchangeReceiver to run an query across different compute nodes // or use one compute node to simulate MPP process. - pushBack(PhysicalExchangeReceiver::build(context, executor_id, log)); + pushBack(PhysicalExchangeReceiver::build(context, executor_id, log, FineGrainedShuffle(executor))); } break; } diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp index cf200be1d6b..9b3ca236128 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp @@ -28,10 +28,12 @@ PhysicalPlanNode::PhysicalPlanNode( const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id) : executor_id(executor_id_) , type(type_) , schema(schema_) + , fine_grained_shuffle(fine_grained_shuffle_) , log(Logger::get(req_id, type_.toString(), executor_id_)) {} @@ -88,7 +90,20 @@ void PhysicalPlanNode::buildBlockInputStream(DAGPipeline & pipeline, Context & c } } -void PhysicalPlanNode::buildPipelineExec(PipelineExecGroupBuilder & /*group_builder*/, Context & /*context*/, size_t /*concurrency*/) +void PhysicalPlanNode::buildPipelineExecGroup( + PipelineExecutorStatus & /*exec_status*/, + PipelineExecGroupBuilder & /*group_builder*/, + Context & /*context*/, + size_t /*concurrency*/) +{ + throw Exception("Unsupport"); +} + +void PhysicalPlanNode::buildPipelineExec( + PipelineExecutorStatus & /*exec_status*/, + PipelineExecBuilder & /*exec_builder*/, + Context & /*context*/, + size_t /*partition_id*/) { throw Exception("Unsupport"); } diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.h b/dbms/src/Flash/Planner/PhysicalPlanNode.h index 4d903c2ba59..0afef4d622a 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.h +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -18,6 +18,7 @@ #include <Core/Block.h> #include <Core/Names.h> #include <Core/NamesAndTypes.h> +#include <Flash/Coprocessor/FineGrainedShuffle.h> #include <Flash/Planner/PlanType.h> #include <memory> @@ -28,6 +29,9 @@ struct DAGPipeline; class Context; class DAGContext; +class PipelineExecutorStatus; + +struct PipelineExecBuilder; struct PipelineExecGroupBuilder; class Pipeline; @@ -44,6 +48,7 @@ class PhysicalPlanNode : public std::enable_shared_from_this<PhysicalPlanNode> const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id); virtual ~PhysicalPlanNode() = default; @@ -60,7 +65,16 @@ class PhysicalPlanNode : public std::enable_shared_from_this<PhysicalPlanNode> virtual void buildBlockInputStream(DAGPipeline & pipeline, Context & context, size_t max_streams); - virtual void buildPipelineExec(PipelineExecGroupBuilder & /*group_builder*/, Context & /*context*/, size_t /*concurrency*/); + virtual void buildPipelineExecGroup( + PipelineExecutorStatus & /*exec_status*/, + PipelineExecGroupBuilder & /*group_builder*/, + Context & /*context*/, + size_t /*concurrency*/); + virtual void buildPipelineExec( + PipelineExecutorStatus & /*exec_status*/, + PipelineExecBuilder & /*exec_builder*/, + Context & /*context*/, + size_t /*partition_id*/); virtual void buildPipeline(PipelineBuilder & builder); @@ -76,6 +90,8 @@ class PhysicalPlanNode : public std::enable_shared_from_this<PhysicalPlanNode> void disableRestoreConcurrency() { is_restore_concurrency = false; } + const FineGrainedShuffle & getFineGrainedShuffle() const { return fine_grained_shuffle; } + String toString(); String toSimpleString(); @@ -89,6 +105,11 @@ class PhysicalPlanNode : public std::enable_shared_from_this<PhysicalPlanNode> PlanType type; NamesAndTypes schema; + //Most operators are not aware of whether they are fine-grained shuffles or not; + //whether they are fine-grained shuffles or not, their execution remains unchanged. + // Only a few operators need to sense fine-grained shuffle, such as exchange sender/receiver, join, aggregate, window and window sort. + FineGrainedShuffle fine_grained_shuffle; + bool is_tidb_operator = true; bool is_restore_concurrency = true; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp index 9aacf49e967..cf7c9fd2a5f 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp @@ -74,6 +74,7 @@ PhysicalPlanNodePtr PhysicalAggregation::build( auto physical_agg = std::make_shared<PhysicalAggregation>( executor_id, schema, + fine_grained_shuffle, log->identifier(), child, before_agg_actions, @@ -81,8 +82,7 @@ PhysicalPlanNodePtr PhysicalAggregation::build( collators, AggregationInterpreterHelper::isFinalAgg(aggregation), aggregate_descriptions, - expr_after_agg_actions, - fine_grained_shuffle); + expr_after_agg_actions); return physical_agg; } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.h b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.h index b97492e286f..81ae6005f4c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.h @@ -35,6 +35,7 @@ class PhysicalAggregation : public PhysicalUnary PhysicalAggregation( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, const ExpressionActionsPtr & before_agg_actions_, @@ -42,16 +43,14 @@ class PhysicalAggregation : public PhysicalUnary const TiDB::TiDBCollators & aggregation_collators_, bool is_final_agg_, const AggregateDescriptions & aggregate_descriptions_, - const ExpressionActionsPtr & expr_after_agg_, - const FineGrainedShuffle & fine_grained_shuffle_) - : PhysicalUnary(executor_id_, PlanType::Aggregation, schema_, req_id, child_) + const ExpressionActionsPtr & expr_after_agg_) + : PhysicalUnary(executor_id_, PlanType::Aggregation, schema_, fine_grained_shuffle_, req_id, child_) , before_agg_actions(before_agg_actions_) , aggregation_keys(aggregation_keys_) , aggregation_collators(aggregation_collators_) , is_final_agg(is_final_agg_) , aggregate_descriptions(aggregate_descriptions_) , expr_after_agg(expr_after_agg_) - , fine_grained_shuffle(fine_grained_shuffle_) {} void buildPipeline(PipelineBuilder & builder) override; @@ -63,12 +62,12 @@ class PhysicalAggregation : public PhysicalUnary private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; +private: ExpressionActionsPtr before_agg_actions; Names aggregation_keys; TiDB::TiDBCollators aggregation_collators; bool is_final_agg; AggregateDescriptions aggregate_descriptions; ExpressionActionsPtr expr_after_agg; - FineGrainedShuffle fine_grained_shuffle; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalBinary.h b/dbms/src/Flash/Planner/Plans/PhysicalBinary.h index 22300e2f087..230489d3a50 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalBinary.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalBinary.h @@ -30,10 +30,11 @@ class PhysicalBinary : public PhysicalPlanNode const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & left_, const PhysicalPlanNodePtr & right_) - : PhysicalPlanNode(executor_id_, type_, schema_, req_id) + : PhysicalPlanNode(executor_id_, type_, schema_, fine_grained_shuffle_, req_id) , left(left_) , right(right_) { diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp index 9a74346cb98..a22e5099200 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp @@ -31,10 +31,11 @@ namespace DB PhysicalExchangeReceiver::PhysicalExchangeReceiver( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, const Block & sample_block_, const std::shared_ptr<ExchangeReceiver> & mpp_exchange_receiver_) - : PhysicalLeaf(executor_id_, PlanType::ExchangeReceiver, schema_, req_id) + : PhysicalLeaf(executor_id_, PlanType::ExchangeReceiver, schema_, fine_grained_shuffle, req_id) , sample_block(sample_block_) , mpp_exchange_receiver(mpp_exchange_receiver_) {} @@ -42,7 +43,8 @@ PhysicalExchangeReceiver::PhysicalExchangeReceiver( PhysicalPlanNodePtr PhysicalExchangeReceiver::build( const Context & context, const String & executor_id, - const LoggerPtr & log) + const LoggerPtr & log, + const FineGrainedShuffle & fine_grained_shuffle) { auto mpp_exchange_receiver = context.getDAGContext()->getMPPExchangeReceiver(executor_id); if (unlikely(mpp_exchange_receiver == nullptr)) @@ -54,6 +56,7 @@ PhysicalPlanNodePtr PhysicalExchangeReceiver::build( auto physical_exchange_receiver = std::make_shared<PhysicalExchangeReceiver>( executor_id, schema, + fine_grained_shuffle, log->identifier(), Block(schema), mpp_exchange_receiver); @@ -68,10 +71,9 @@ void PhysicalExchangeReceiver::buildBlockInputStreamImpl(DAGPipeline & pipeline, // todo choose a more reasonable stream number auto & exchange_receiver_io_input_streams = dag_context.getInBoundIOInputStreamsMap()[executor_id]; - const bool enable_fine_grained_shuffle = enableFineGrainedShuffle(mpp_exchange_receiver->getFineGrainedShuffleStreamCount()); String extra_info = "squashing after exchange receiver"; size_t stream_count = max_streams; - if (enable_fine_grained_shuffle) + if (fine_grained_shuffle.enable()) { extra_info += ", " + String(enableFineGrainedShuffleExtraInfo); stream_count = std::min(max_streams, mpp_exchange_receiver->getFineGrainedShuffleStreamCount()); @@ -83,24 +85,27 @@ void PhysicalExchangeReceiver::buildBlockInputStreamImpl(DAGPipeline & pipeline, mpp_exchange_receiver, log->identifier(), execId(), - /*stream_id=*/enable_fine_grained_shuffle ? i : 0); + /*stream_id=*/fine_grained_shuffle.enable() ? i : 0); exchange_receiver_io_input_streams.push_back(stream); stream->setExtraInfo(extra_info); pipeline.streams.push_back(stream); } } -void PhysicalExchangeReceiver::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t concurrency) +void PhysicalExchangeReceiver::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t concurrency) { // TODO support fine grained shuffle. - const bool enable_fine_grained_shuffle = enableFineGrainedShuffle(mpp_exchange_receiver->getFineGrainedShuffleStreamCount()); - RUNTIME_CHECK(!enable_fine_grained_shuffle); + RUNTIME_CHECK(!fine_grained_shuffle.enable()); // TODO choose a more reasonable concurrency. group_builder.init(concurrency); group_builder.transform([&](auto & builder) { builder.setSourceOp(std::make_unique<ExchangeReceiverSourceOp>( - group_builder.exec_status, + exec_status, log->identifier(), mpp_exchange_receiver, /*stream_id=*/0)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.h b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.h index f73084d4244..9150900e8f1 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.h @@ -27,11 +27,13 @@ class PhysicalExchangeReceiver : public PhysicalLeaf static PhysicalPlanNodePtr build( const Context & context, const String & executor_id, - const LoggerPtr & log); + const LoggerPtr & log, + const FineGrainedShuffle & fine_grained_shuffle); PhysicalExchangeReceiver( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle, const String & req_id, const Block & sample_block_, const std::shared_ptr<ExchangeReceiver> & mpp_exchange_receiver_); @@ -45,7 +47,11 @@ class PhysicalExchangeReceiver : public PhysicalLeaf return mpp_exchange_receiver->getSourceNum(); } - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp index 0e161469fba..1173fb764f2 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp @@ -41,12 +41,12 @@ PhysicalPlanNodePtr PhysicalExchangeSender::build( auto physical_exchange_sender = std::make_shared<PhysicalExchangeSender>( executor_id, child->getSchema(), + fine_grained_shuffle, log->identifier(), child, partition_col_ids, partition_col_collators, exchange_sender.tp(), - fine_grained_shuffle, exchange_sender.compression()); // executeUnion will be call after sender.transform, so don't need to restore concurrency. physical_exchange_sender->disableRestoreConcurrency(); @@ -87,11 +87,12 @@ void PhysicalExchangeSender::buildBlockInputStreamImpl(DAGPipeline & pipeline, C }); } -void PhysicalExchangeSender::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & context, size_t /*concurrency*/) +void PhysicalExchangeSender::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t /*concurrency*/) { - // TODO support fine grained shuffle - RUNTIME_CHECK(!fine_grained_shuffle.enable()); - group_builder.transform([&](auto & builder) { // construct writer std::unique_ptr<DAGResponseWriter> response_writer = newMPPExchangeWriter( @@ -108,7 +109,7 @@ void PhysicalExchangeSender::buildPipelineExec(PipelineExecGroupBuilder & group_ context.getSettingsRef().batch_send_min_limit_compression, log->identifier(), /*is_async=*/true); - builder.setSinkOp(std::make_unique<ExchangeSenderSinkOp>(group_builder.exec_status, log->identifier(), std::move(response_writer))); + builder.setSinkOp(std::make_unique<ExchangeSenderSinkOp>(exec_status, log->identifier(), std::move(response_writer))); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.h b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.h index 121238cc6eb..6f6e5460cc3 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.h @@ -34,26 +34,29 @@ class PhysicalExchangeSender : public PhysicalUnary PhysicalExchangeSender( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, const std::vector<Int64> & partition_col_ids_, const TiDB::TiDBCollators & collators_, const tipb::ExchangeType & exchange_type_, - const FineGrainedShuffle & fine_grained_shuffle_, const tipb::CompressionMode & compression_mode_) - : PhysicalUnary(executor_id_, PlanType::ExchangeSender, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::ExchangeSender, schema_, fine_grained_shuffle_, req_id, child_) , partition_col_ids(partition_col_ids_) , partition_col_collators(collators_) , exchange_type(exchange_type_) , compression_mode(compression_mode_) - , fine_grained_shuffle(fine_grained_shuffle_) {} void finalize(const Names & parent_require) override; const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & context, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t /*concurrency*/) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; @@ -62,7 +65,5 @@ class PhysicalExchangeSender : public PhysicalUnary TiDB::TiDBCollators partition_col_collators; tipb::ExchangeType exchange_type; tipb::CompressionMode compression_mode; - - FineGrainedShuffle fine_grained_shuffle; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExpand.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExpand.cpp index fdba8dd1b8a..5f8574eee4d 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExpand.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExpand.cpp @@ -64,11 +64,11 @@ PhysicalPlanNodePtr PhysicalExpand::build( auto physical_expand = std::make_shared<PhysicalExpand>( executor_id, expand_output_columns, + child->getFineGrainedShuffle(), log->identifier(), child, expand_action.expand, before_expand_actions); - return physical_expand; } @@ -82,10 +82,14 @@ void PhysicalExpand::expandTransform(DAGPipeline & child_pipeline) }); } -void PhysicalExpand::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context &, size_t) +void PhysicalExpand::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(group_builder.exec_status, log->identifier(), expand_actions)); + builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(exec_status, log->identifier(), expand_actions)); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExpand.h b/dbms/src/Flash/Planner/Plans/PhysicalExpand.h index fa668b64114..5289b816e4e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExpand.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalExpand.h @@ -34,11 +34,12 @@ class PhysicalExpand : public PhysicalUnary PhysicalExpand( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, const std::shared_ptr<const Expand> & shared_expand, const ExpressionActionsPtr & expand_actions) - : PhysicalUnary(executor_id_, PlanType::Expand, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::Expand, schema_, fine_grained_shuffle_, req_id, child_) , shared_expand(shared_expand) , expand_actions(expand_actions) {} @@ -49,7 +50,11 @@ class PhysicalExpand : public PhysicalUnary const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/Plans/PhysicalFilter.cpp index a3d353a60a1..1aa8f106dce 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalFilter.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalFilter.cpp @@ -42,6 +42,7 @@ PhysicalPlanNodePtr PhysicalFilter::build( auto physical_filter = std::make_shared<PhysicalFilter>( executor_id, child->getSchema(), + child->getFineGrainedShuffle(), log->identifier(), child, filter_column_name, @@ -57,11 +58,15 @@ void PhysicalFilter::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & pipeline.transform([&](auto & stream) { stream = std::make_shared<FilterBlockInputStream>(stream, before_filter_actions, filter_column, log->identifier()); }); } -void PhysicalFilter::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +void PhysicalFilter::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { auto input_header = group_builder.getCurrentHeader(); group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<FilterTransformOp>(group_builder.exec_status, log->identifier(), input_header, before_filter_actions, filter_column)); + builder.appendTransformOp(std::make_unique<FilterTransformOp>(exec_status, log->identifier(), input_header, before_filter_actions, filter_column)); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalFilter.h b/dbms/src/Flash/Planner/Plans/PhysicalFilter.h index 08f420530e2..f22a57b456e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalFilter.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalFilter.h @@ -33,11 +33,12 @@ class PhysicalFilter : public PhysicalUnary PhysicalFilter( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, const String & filter_column_, const ExpressionActionsPtr & before_filter_actions_) - : PhysicalUnary(executor_id_, PlanType::Filter, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::Filter, schema_, fine_grained_shuffle_, req_id, child_) , filter_column(filter_column_) , before_filter_actions(before_filter_actions_) {} @@ -46,7 +47,11 @@ class PhysicalFilter : public PhysicalUnary const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp index e1ac74f8b32..d7a81162a01 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.cpp @@ -20,16 +20,27 @@ namespace DB { PhysicalPlanNodePtr PhysicalGetResultSink::build( ResultHandler && result_handler, + const LoggerPtr & log, const PhysicalPlanNodePtr & child) { - return std::make_shared<PhysicalGetResultSink>("get_result_sink", child->getSchema(), "", child, std::move(result_handler)); + return std::make_shared<PhysicalGetResultSink>( + "get_result_sink", + child->getSchema(), + child->getFineGrainedShuffle(), + log->identifier(), + child, + std::move(result_handler)); } -void PhysicalGetResultSink::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +void PhysicalGetResultSink::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { auto this_shared_ptr = std::static_pointer_cast<PhysicalGetResultSink>(shared_from_this()); group_builder.transform([&](auto & builder) { - builder.setSinkOp(std::make_unique<GetResultSinkOp>(group_builder.exec_status, log->identifier(), this_shared_ptr)); + builder.setSinkOp(std::make_unique<GetResultSinkOp>(exec_status, log->identifier(), this_shared_ptr)); }); } } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h index 5d3d42df298..5e17d2ce269 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalGetResultSink.h @@ -26,15 +26,17 @@ class PhysicalGetResultSink : public PhysicalUnary public: static PhysicalPlanNodePtr build( ResultHandler && result_handler, + const LoggerPtr & log, const PhysicalPlanNodePtr & child); PhysicalGetResultSink( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, ResultHandler && result_handler_) - : PhysicalUnary(executor_id_, PlanType::GetResult, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::GetResult, schema_, fine_grained_shuffle_, req_id, child_) , result_handler(std::move(result_handler_)) { assert(!result_handler.isIgnored()); @@ -50,7 +52,11 @@ class PhysicalGetResultSink : public PhysicalUnary throw Exception("Unsupport"); } - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; private: friend class GetResultSinkOp; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp index c6eec3eaf6a..b15cfd9f3c0 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp @@ -141,14 +141,14 @@ PhysicalPlanNodePtr PhysicalJoin::build( auto physical_join = std::make_shared<PhysicalJoin>( executor_id, join_output_schema, + fine_grained_shuffle, log->identifier(), probe_plan, build_plan, join_ptr, probe_side_prepare_actions, build_side_prepare_actions, - Block(join_output_schema), - fine_grained_shuffle); + Block(join_output_schema)); return physical_join; } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalJoin.h b/dbms/src/Flash/Planner/Plans/PhysicalJoin.h index 4a1f29fb5e8..e1fc5351c7a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalJoin.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalJoin.h @@ -36,20 +36,19 @@ class PhysicalJoin : public PhysicalBinary PhysicalJoin( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & probe_, const PhysicalPlanNodePtr & build_, const JoinPtr & join_ptr_, const ExpressionActionsPtr & probe_side_prepare_actions_, const ExpressionActionsPtr & build_side_prepare_actions_, - const Block & sample_block_, - const FineGrainedShuffle & fine_grained_shuffle_) - : PhysicalBinary(executor_id_, PlanType::Join, schema_, req_id, probe_, build_) + const Block & sample_block_) + : PhysicalBinary(executor_id_, PlanType::Join, schema_, fine_grained_shuffle_, req_id, probe_, build_) , join_ptr(join_ptr_) , probe_side_prepare_actions(probe_side_prepare_actions_) , build_side_prepare_actions(build_side_prepare_actions_) , sample_block(sample_block_) - , fine_grained_shuffle(fine_grained_shuffle_) {} void buildPipeline(PipelineBuilder & builder) override; @@ -78,6 +77,5 @@ class PhysicalJoin : public PhysicalBinary ExpressionActionsPtr build_side_prepare_actions; Block sample_block; - FineGrainedShuffle fine_grained_shuffle; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/Plans/PhysicalLeaf.h index b2e0ded27d8..ffcb8c47c46 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalLeaf.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalLeaf.h @@ -29,8 +29,9 @@ class PhysicalLeaf : public PhysicalPlanNode const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id) - : PhysicalPlanNode(executor_id_, type_, schema_, req_id) + : PhysicalPlanNode(executor_id_, type_, schema_, fine_grained_shuffle_, req_id) {} PhysicalPlanNodePtr children(size_t) const override diff --git a/dbms/src/Flash/Planner/Plans/PhysicalLimit.cpp b/dbms/src/Flash/Planner/Plans/PhysicalLimit.cpp index 5144ad6c11d..746a7876564 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalLimit.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalLimit.cpp @@ -35,6 +35,7 @@ PhysicalPlanNodePtr PhysicalLimit::build( auto physical_limit = std::make_shared<PhysicalLimit>( executor_id, child->getSchema(), + child->getFineGrainedShuffle(), log->identifier(), child, limit.limit()); @@ -53,12 +54,16 @@ void PhysicalLimit::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & } } -void PhysicalLimit::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +void PhysicalLimit::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { auto input_header = group_builder.getCurrentHeader(); auto global_limit = std::make_shared<GlobalLimitTransformAction>(input_header, limit); group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<LimitTransformOp>(group_builder.exec_status, log->identifier(), global_limit)); + builder.appendTransformOp(std::make_unique<LimitTransformOp>(exec_status, log->identifier(), global_limit)); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalLimit.h b/dbms/src/Flash/Planner/Plans/PhysicalLimit.h index b8e026a6b49..7589b564bb8 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalLimit.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalLimit.h @@ -31,10 +31,11 @@ class PhysicalLimit : public PhysicalUnary PhysicalLimit( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, size_t limit_) - : PhysicalUnary(executor_id_, PlanType::Limit, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::Limit, schema_, fine_grained_shuffle_, req_id, child_) , limit(limit_) {} @@ -42,7 +43,11 @@ class PhysicalLimit : public PhysicalUnary const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.cpp index 7d5dca38f14..d3d2efc11e0 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.cpp @@ -28,11 +28,12 @@ namespace DB PhysicalMockExchangeReceiver::PhysicalMockExchangeReceiver( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const Block & sample_block_, const BlockInputStreams & mock_streams_, size_t source_num_) - : PhysicalLeaf(executor_id_, PlanType::MockExchangeReceiver, schema_, req_id) + : PhysicalLeaf(executor_id_, PlanType::MockExchangeReceiver, schema_, fine_grained_shuffle_, req_id) , sample_block(sample_block_) , mock_streams(mock_streams_) , source_num(source_num_) @@ -43,13 +44,14 @@ PhysicalPlanNodePtr PhysicalMockExchangeReceiver::build( const String & executor_id, const LoggerPtr & log, const tipb::ExchangeReceiver & exchange_receiver, - size_t fine_grained_stream_count) + const FineGrainedShuffle & fine_grained_shuffle) { - auto [schema, mock_streams] = mockSchemaAndStreamsForExchangeReceiver(context, executor_id, log, exchange_receiver, fine_grained_stream_count); + auto [schema, mock_streams] = mockSchemaAndStreamsForExchangeReceiver(context, executor_id, log, exchange_receiver, fine_grained_shuffle.stream_count); auto physical_mock_exchange_receiver = std::make_shared<PhysicalMockExchangeReceiver>( executor_id, schema, + fine_grained_shuffle, log->identifier(), Block(schema), mock_streams, @@ -63,12 +65,16 @@ void PhysicalMockExchangeReceiver::buildBlockInputStreamImpl(DAGPipeline & pipel pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); } -void PhysicalMockExchangeReceiver::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +void PhysicalMockExchangeReceiver::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { group_builder.init(mock_streams.size()); size_t i = 0; group_builder.transform([&](auto & builder) { - builder.setSourceOp(std::make_unique<BlockInputStreamSourceOp>(group_builder.exec_status, log->identifier(), mock_streams[i++])); + builder.setSourceOp(std::make_unique<BlockInputStreamSourceOp>(exec_status, log->identifier(), mock_streams[i++])); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.h b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.h index 26b2c72caed..ac3bf4561ff 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeReceiver.h @@ -33,11 +33,12 @@ class PhysicalMockExchangeReceiver : public PhysicalLeaf const String & executor_id, const LoggerPtr & log, const tipb::ExchangeReceiver & exchange_receiver, - size_t fine_grained_stream_count); + const FineGrainedShuffle & fine_grained_shuffle); PhysicalMockExchangeReceiver( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const Block & sample_block_, const BlockInputStreams & mock_streams, @@ -49,7 +50,11 @@ class PhysicalMockExchangeReceiver : public PhysicalLeaf size_t getSourceNum() const { return source_num; }; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.cpp index 490ec164ae6..78b96d2c5d4 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.cpp @@ -30,6 +30,7 @@ PhysicalPlanNodePtr PhysicalMockExchangeSender::build( auto physical_mock_exchange_sender = std::make_shared<PhysicalMockExchangeSender>( executor_id, child->getSchema(), + FineGrainedShuffle{}, log->identifier(), child); // executeUnion will be call after sender.transform, so don't need to restore concurrency. diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.h b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.h index 445222d094d..cc5a84c8a2c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockExchangeSender.h @@ -31,16 +31,22 @@ class PhysicalMockExchangeSender : public PhysicalUnary PhysicalMockExchangeSender( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_) - : PhysicalUnary(executor_id_, PlanType::MockExchangeSender, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::MockExchangeSender, schema_, fine_grained_shuffle_, req_id, child_) {} void finalize(const Names & parent_require) override; const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & /*group_builder*/, Context & /*context*/, size_t /*concurrency*/) override {} + void buildPipelineExecGroup( + PipelineExecutorStatus & /*exec_status*/, + PipelineExecGroupBuilder & /*group_builder*/, + Context & /*context*/, + size_t /*concurrency*/) override + {} private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index 422af263138..682d8944c1b 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -76,11 +76,12 @@ std::pair<NamesAndTypes, BlockInputStreams> mockSchemaAndStreams( PhysicalMockTableScan::PhysicalMockTableScan( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const Block & sample_block_, const BlockInputStreams & mock_streams_, Int64 table_id_) - : PhysicalLeaf(executor_id_, PlanType::MockTableScan, schema_, req_id) + : PhysicalLeaf(executor_id_, PlanType::MockTableScan, schema_, fine_grained_shuffle_, req_id) , sample_block(sample_block_) , mock_streams(mock_streams_) , table_id(table_id_) @@ -98,6 +99,7 @@ PhysicalPlanNodePtr PhysicalMockTableScan::build( auto physical_mock_table_scan = std::make_shared<PhysicalMockTableScan>( executor_id, schema, + FineGrainedShuffle{}, log->identifier(), Block(schema), mock_streams, @@ -111,12 +113,16 @@ void PhysicalMockTableScan::buildBlockInputStreamImpl(DAGPipeline & pipeline, Co pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); } -void PhysicalMockTableScan::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +void PhysicalMockTableScan::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { group_builder.init(mock_streams.size()); size_t i = 0; group_builder.transform([&](auto & builder) { - builder.setSourceOp(std::make_unique<BlockInputStreamSourceOp>(group_builder.exec_status, log->identifier(), mock_streams[i++])); + builder.setSourceOp(std::make_unique<BlockInputStreamSourceOp>(exec_status, log->identifier(), mock_streams[i++])); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h index 7330b5d16c4..6573e6cb29c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h @@ -39,6 +39,7 @@ class PhysicalMockTableScan : public PhysicalLeaf PhysicalMockTableScan( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const Block & sample_block_, const BlockInputStreams & mock_streams_, @@ -48,7 +49,11 @@ class PhysicalMockTableScan : public PhysicalLeaf const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; void initStreams(Context & context); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp index 9eea80a077c..c175fb00ac7 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.cpp @@ -50,6 +50,7 @@ PhysicalPlanNodePtr PhysicalProjection::build( auto physical_projection = std::make_shared<PhysicalProjection>( executor_id, schema, + child->getFineGrainedShuffle(), log->identifier(), child, "projection", @@ -82,6 +83,7 @@ PhysicalPlanNodePtr PhysicalProjection::buildNonRootFinal( auto physical_projection = std::make_shared<PhysicalProjection>( child->execId(), schema, + child->getFineGrainedShuffle(), log->identifier(), child, "final projection", @@ -127,6 +129,7 @@ PhysicalPlanNodePtr PhysicalProjection::buildRootFinal( auto physical_projection = std::make_shared<PhysicalProjection>( child->execId(), schema, + child->getFineGrainedShuffle(), log->identifier(), child, "final projection", @@ -143,12 +146,16 @@ void PhysicalProjection::buildBlockInputStreamImpl(DAGPipeline & pipeline, Conte executeExpression(pipeline, project_actions, log, extra_info); } -void PhysicalProjection::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +void PhysicalProjection::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { if (project_actions && !project_actions->getActions().empty()) { group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(group_builder.exec_status, log->identifier(), project_actions)); + builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(exec_status, log->identifier(), project_actions)); }); } } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h index ab1e1936a86..c0f22712516 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalProjection.h @@ -53,11 +53,12 @@ class PhysicalProjection : public PhysicalUnary PhysicalProjection( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, const String & extra_info_, const ExpressionActionsPtr & project_actions_) - : PhysicalUnary(executor_id_, PlanType::Projection, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::Projection, schema_, fine_grained_shuffle_, req_id, child_) , extra_info(extra_info_) , project_actions(project_actions_) {} @@ -66,7 +67,11 @@ class PhysicalProjection : public PhysicalUnary const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp index 27e9d8564b9..02de8a19925 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp @@ -28,10 +28,11 @@ namespace DB PhysicalTableScan::PhysicalTableScan( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const TiDBTableScan & tidb_table_scan_, const Block & sample_block_) - : PhysicalLeaf(executor_id_, PlanType::TableScan, schema_, req_id) + : PhysicalLeaf(executor_id_, PlanType::TableScan, schema_, fine_grained_shuffle_, req_id) , tidb_table_scan(tidb_table_scan_) , sample_block(sample_block_) {} @@ -45,6 +46,7 @@ PhysicalPlanNodePtr PhysicalTableScan::build( auto physical_table_scan = std::make_shared<PhysicalTableScan>( executor_id, schema, + FineGrainedShuffle{}, log->identifier(), table_scan, Block(schema)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h index 00c68e783cf..2525f79c052 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h @@ -30,6 +30,7 @@ class PhysicalTableScan : public PhysicalLeaf PhysicalTableScan( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const TiDBTableScan & tidb_table_scan_, const Block & sample_block_); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp index 5fe779b6e46..f23b2d43e77 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalTopN.cpp @@ -51,6 +51,7 @@ PhysicalPlanNodePtr PhysicalTopN::build( auto physical_top_n = std::make_shared<PhysicalTopN>( executor_id, child->getSchema(), + child->getFineGrainedShuffle(), log->identifier(), child, order_descr, @@ -68,16 +69,20 @@ void PhysicalTopN::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & c orderStreams(pipeline, max_streams, order_descr, limit, false, context, log); } -void PhysicalTopN::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & context, size_t /*concurrency*/) +void PhysicalTopN::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t /*concurrency*/) { if (!before_sort_actions->getActions().empty()) { group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(group_builder.exec_status, log->identifier(), before_sort_actions)); + builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(exec_status, log->identifier(), before_sort_actions)); }); } group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<TopNTransformOp>(group_builder.exec_status, log->identifier(), order_descr, limit, context.getSettingsRef().max_block_size)); + builder.appendTransformOp(std::make_unique<TopNTransformOp>(exec_status, log->identifier(), order_descr, limit, context.getSettingsRef().max_block_size)); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalTopN.h b/dbms/src/Flash/Planner/Plans/PhysicalTopN.h index af1482cea7d..4964e1f8534 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalTopN.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalTopN.h @@ -34,12 +34,13 @@ class PhysicalTopN : public PhysicalUnary PhysicalTopN( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, const SortDescription & order_descr_, const ExpressionActionsPtr & before_sort_actions_, size_t limit_) - : PhysicalUnary(executor_id_, PlanType::TopN, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::TopN, schema_, fine_grained_shuffle_, req_id, child_) , order_descr(order_descr_) , before_sort_actions(before_sort_actions_) , limit(limit_) @@ -49,7 +50,11 @@ class PhysicalTopN : public PhysicalUnary const Block & getSampleBlock() const override; - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & context, size_t concurrency) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t concurrency) override; private: void buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalUnary.h b/dbms/src/Flash/Planner/Plans/PhysicalUnary.h index 47f32643be4..96e2551d63d 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalUnary.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalUnary.h @@ -31,9 +31,10 @@ class PhysicalUnary : public PhysicalPlanNode const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_) - : PhysicalPlanNode(executor_id_, type_, schema_, req_id) + : PhysicalPlanNode(executor_id_, type_, schema_, fine_grained_shuffle_, req_id) , child(child_) { RUNTIME_ASSERT(child, log, "children(0) shouldn't be nullptr"); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalWindow.cpp b/dbms/src/Flash/Planner/Plans/PhysicalWindow.cpp index c6134574571..22bae4eb40a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalWindow.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindow.cpp @@ -54,10 +54,10 @@ PhysicalPlanNodePtr PhysicalWindow::build( auto physical_window = std::make_shared<PhysicalWindow>( executor_id, schema, + fine_grained_shuffle, log->identifier(), child, - window_description, - fine_grained_shuffle); + window_description); return physical_window; } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalWindow.h b/dbms/src/Flash/Planner/Plans/PhysicalWindow.h index 05a7b5a6eca..e7b9314b36b 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalWindow.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindow.h @@ -35,13 +35,12 @@ class PhysicalWindow : public PhysicalUnary PhysicalWindow( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, - const WindowDescription & window_description_, - const FineGrainedShuffle & fine_grained_shuffle_) - : PhysicalUnary(executor_id_, PlanType::Window, schema_, req_id, child_) + const WindowDescription & window_description_) + : PhysicalUnary(executor_id_, PlanType::Window, schema_, fine_grained_shuffle_, req_id, child_) , window_description(window_description_) - , fine_grained_shuffle(fine_grained_shuffle_) {} void finalize(const Names & parent_require) override; @@ -53,6 +52,5 @@ class PhysicalWindow : public PhysicalUnary private: WindowDescription window_description; - FineGrainedShuffle fine_grained_shuffle; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.cpp b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.cpp index a35578fa3e2..0a2445d67c6 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.cpp @@ -43,10 +43,10 @@ PhysicalPlanNodePtr PhysicalWindowSort::build( auto physical_window_sort = std::make_shared<PhysicalWindowSort>( executor_id, child->getSchema(), + fine_grained_shuffle, log->identifier(), child, - order_descr, - fine_grained_shuffle); + order_descr); return physical_window_sort; } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h index 3ad4251a19a..d4c66f81e0e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h @@ -35,13 +35,12 @@ class PhysicalWindowSort : public PhysicalUnary PhysicalWindowSort( const String & executor_id_, const NamesAndTypes & schema_, + const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const PhysicalPlanNodePtr & child_, - const SortDescription & order_descr_, - const FineGrainedShuffle & fine_grained_shuffle_) - : PhysicalUnary(executor_id_, PlanType::WindowSort, schema_, req_id, child_) + const SortDescription & order_descr_) + : PhysicalUnary(executor_id_, PlanType::WindowSort, schema_, fine_grained_shuffle_, req_id, child_) , order_descr(order_descr_) - , fine_grained_shuffle(fine_grained_shuffle_) {} void finalize(const Names & parent_require) override; From 60fcb90094eaf3e5d26978f4597d278665636f56 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Thu, 2 Mar 2023 17:03:31 +0800 Subject: [PATCH 02/24] part2 --- dbms/src/Flash/Pipeline/Exec/PipelineExec.h | 2 +- .../Schedule/Events/FineGrainedPipelineEvent.cpp | 8 ++------ .../Schedule/Events/FineGrainedPipelineEvent.h | 12 ++++++++---- .../Pipeline/Schedule/Events/PlainPipelineEvent.cpp | 4 ++-- dbms/src/Flash/Planner/PhysicalPlanNode.cpp | 9 --------- dbms/src/Flash/Planner/PhysicalPlanNode.h | 10 ++-------- .../Flash/Planner/Plans/PhysicalExchangeReceiver.cpp | 10 +++++----- .../Flash/Planner/Plans/PhysicalExchangeSender.cpp | 6 ++++++ 8 files changed, 26 insertions(+), 35 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Exec/PipelineExec.h b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h index 0230cfe10da..7fb845b3df1 100644 --- a/dbms/src/Flash/Pipeline/Exec/PipelineExec.h +++ b/dbms/src/Flash/Pipeline/Exec/PipelineExec.h @@ -23,7 +23,7 @@ namespace DB // The executor for push model operator. // A pipeline will generate multiple pipeline_execs. // data flow: source --> transform --> .. --> transform --> sink -class PipelineExec +class PipelineExec : private boost::noncopyable { public: PipelineExec( diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp index f6d45197e29..bf69617a519 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp @@ -20,13 +20,9 @@ namespace DB { std::vector<TaskPtr> FineGrainedPipelineEvent::scheduleImpl() { - assert(pipeline); - auto pipeline_exec_group = pipeline->buildExecGroup(exec_status, context, partition_id); - assert(!pipeline_exec_group.empty()); std::vector<TaskPtr> tasks; - tasks.reserve(pipeline_exec_group.size()); - for (auto & pipline_exec : pipeline_exec_group) - tasks.push_back(std::make_unique<PipelineTask>(mem_tracker, log->identifier(), exec_status, shared_from_this(), std::move(pipline_exec))); + tasks.reserve(1); + tasks.push_back(std::make_unique<PipelineTask>(mem_tracker, log->identifier(), exec_status, shared_from_this(), std::move(pipeline_exec))); return tasks; } } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h index 9f92db4f3cd..ce3c0a3eedc 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h @@ -14,6 +14,7 @@ #pragma once +#include <Flash/Pipeline/Exec/PipelineExec.h> #include <Flash/Pipeline/Schedule/Events/PipelineEvent.h> namespace DB @@ -27,15 +28,18 @@ class FineGrainedPipelineEvent : public PipelineEvent const String & req_id, Context & context_, const PipelinePtr & pipeline_, - size_t partition_id_) + PipelineExecPtr && pipeline_exec_) : PipelineEvent(exec_status_, std::move(mem_tracker_), req_id, context_, pipeline_) - , partition_id(partition_id_) - {} + , pipeline_exec(std::move(pipeline_exec_)) + { + assert(pipeline_exec); + } protected: std::vector<TaskPtr> scheduleImpl() override; private: - size_t partition_id; + // The pipeline exec for executing the specific fine-grained shuffle partition id. + PipelineExecPtr pipeline_exec; }; } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp index bea061b551c..eca5ad3a8d7 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp @@ -25,8 +25,8 @@ std::vector<TaskPtr> PlainPipelineEvent::scheduleImpl() assert(!pipeline_exec_group.empty()); std::vector<TaskPtr> tasks; tasks.reserve(pipeline_exec_group.size()); - for (auto & pipline_exec : pipeline_exec_group) - tasks.push_back(std::make_unique<PipelineTask>(mem_tracker, log->identifier(), exec_status, shared_from_this(), std::move(pipline_exec))); + for (auto & pipeline_exec : pipeline_exec_group) + tasks.push_back(std::make_unique<PipelineTask>(mem_tracker, log->identifier(), exec_status, shared_from_this(), std::move(pipeline_exec))); return tasks; } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp index 9b3ca236128..7c8c5668207 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp @@ -99,15 +99,6 @@ void PhysicalPlanNode::buildPipelineExecGroup( throw Exception("Unsupport"); } -void PhysicalPlanNode::buildPipelineExec( - PipelineExecutorStatus & /*exec_status*/, - PipelineExecBuilder & /*exec_builder*/, - Context & /*context*/, - size_t /*partition_id*/) -{ - throw Exception("Unsupport"); -} - void PhysicalPlanNode::buildPipeline(PipelineBuilder & builder) { assert(childrenSize() <= 1); diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.h b/dbms/src/Flash/Planner/PhysicalPlanNode.h index 0afef4d622a..d25d978be6b 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.h +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -31,7 +31,6 @@ class DAGContext; class PipelineExecutorStatus; -struct PipelineExecBuilder; struct PipelineExecGroupBuilder; class Pipeline; @@ -70,11 +69,6 @@ class PhysicalPlanNode : public std::enable_shared_from_this<PhysicalPlanNode> PipelineExecGroupBuilder & /*group_builder*/, Context & /*context*/, size_t /*concurrency*/); - virtual void buildPipelineExec( - PipelineExecutorStatus & /*exec_status*/, - PipelineExecBuilder & /*exec_builder*/, - Context & /*context*/, - size_t /*partition_id*/); virtual void buildPipeline(PipelineBuilder & builder); @@ -105,8 +99,8 @@ class PhysicalPlanNode : public std::enable_shared_from_this<PhysicalPlanNode> PlanType type; NamesAndTypes schema; - //Most operators are not aware of whether they are fine-grained shuffles or not; - //whether they are fine-grained shuffles or not, their execution remains unchanged. + // Most operators are not aware of whether they are fine-grained shuffles or not. + // Whether they are fine-grained shuffles or not, their execution remains unchanged. // Only a few operators need to sense fine-grained shuffle, such as exchange sender/receiver, join, aggregate, window and window sort. FineGrainedShuffle fine_grained_shuffle; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp index a22e5099200..63dbffac6fc 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeReceiver.cpp @@ -76,7 +76,7 @@ void PhysicalExchangeReceiver::buildBlockInputStreamImpl(DAGPipeline & pipeline, if (fine_grained_shuffle.enable()) { extra_info += ", " + String(enableFineGrainedShuffleExtraInfo); - stream_count = std::min(max_streams, mpp_exchange_receiver->getFineGrainedShuffleStreamCount()); + stream_count = std::min(max_streams, fine_grained_shuffle.stream_count); } for (size_t i = 0; i < stream_count; ++i) @@ -98,17 +98,17 @@ void PhysicalExchangeReceiver::buildPipelineExecGroup( Context & /*context*/, size_t concurrency) { - // TODO support fine grained shuffle. - RUNTIME_CHECK(!fine_grained_shuffle.enable()); + if (fine_grained_shuffle.enable()) + concurrency = std::min(concurrency, fine_grained_shuffle.stream_count); - // TODO choose a more reasonable concurrency. group_builder.init(concurrency); + size_t partition_id = 0; group_builder.transform([&](auto & builder) { builder.setSourceOp(std::make_unique<ExchangeReceiverSourceOp>( exec_status, log->identifier(), mpp_exchange_receiver, - /*stream_id=*/0)); + /*stream_id=*/fine_grained_shuffle.enable() ? partition_id++ : 0)); }); } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp index 1173fb764f2..ab85b33d47e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp @@ -93,6 +93,12 @@ void PhysicalExchangeSender::buildPipelineExecGroup( Context & context, size_t /*concurrency*/) { + if (fine_grained_shuffle.enable()) + { + RUNTIME_CHECK(exchange_type == tipb::ExchangeType::Hash, ExchangeType_Name(exchange_type)); + RUNTIME_CHECK(fine_grained_shuffle.stream_count <= 1024, fine_grained_shuffle.stream_count); + } + group_builder.transform([&](auto & builder) { // construct writer std::unique_ptr<DAGResponseWriter> response_writer = newMPPExchangeWriter( From 884aefd7c1ce1a915b2318cec5bc83f7d48ae29d Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Thu, 2 Mar 2023 17:46:04 +0800 Subject: [PATCH 03/24] udpate --- dbms/src/Flash/Pipeline/Pipeline.cpp | 110 +++++++++++++++--- dbms/src/Flash/Pipeline/Pipeline.h | 3 +- .../Events/FineGrainedPipelineEvent.cpp | 1 - 3 files changed, 94 insertions(+), 20 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index 6806ba98829..3e2ca5e2738 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -18,6 +18,7 @@ #include <Flash/Pipeline/Exec/PipelineExecBuilder.h> #include <Flash/Pipeline/Pipeline.h> #include <Flash/Pipeline/Schedule/Events/Event.h> +#include <Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h> #include <Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h> #include <Flash/Planner/PhysicalPlanNode.h> #include <Flash/Planner/Plans/PhysicalGetResultSink.h> @@ -32,6 +33,73 @@ FmtBuffer & addPrefix(FmtBuffer & buffer, size_t level) { return buffer.append(String(level, ' ')); } + +void mapEvents(const Events & inputs, const Events & outputs) +{ + assert(!inputs.empty()); + assert(!outputs.empty()); + if (inputs.size() == outputs.size()) + { + /** + * 1. for fine grained inputs and fine grained outputs + * ``` + * FineGrainedPipelineEvent<────FineGrainedPipelineEvent + * FineGrainedPipelineEvent<────FineGrainedPipelineEvent + * FineGrainedPipelineEvent<────FineGrainedPipelineEvent + * FineGrainedPipelineEvent<────FineGrainedPipelineEvent + * ``` + * 2. for non fine grained inputs and non fine grained outputs + * ``` + * PlainPipelineEvent<────PlainPipelineEvent + * ``` + * 3. for non fine grained inputs and fine grained outputs + * ``` + * PlainPipelineEvent<────FineGrainedPipelineEvent + * ``` + * 4. for fine grained inputs and non fine grained outputs + * ``` + * FineGrainedPipelineEvent<────PlainPipelineEvent + * ``` + */ + size_t partition_num = inputs.size(); + for (size_t index = 0; index < partition_num; ++index) + outputs[index]->addInput(inputs[index]); + } + else + { + /** + * 1. for fine grained inputs and fine grained outputs + * If the number of partitions does not match, it is safer to use full mapping. + * ``` + * FineGrainedPipelineEvent◄──┐ ┌──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┼─┼──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┤ ├──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┘ └──FineGrainedPipelineEvent + * ``` + * 2. for non fine grained inputs and non fine grained outputs + * This is not possible, the size of inputs and outputs must be the same and 1. + * 3. for non fine grained inputs and fine grained outputs + * ``` + * FineGrainedPipelineEvent◄──┐ + * FineGrainedPipelineEvent◄──┼──PlainPipelineEvent + * FineGrainedPipelineEvent◄──┤ + * FineGrainedPipelineEvent◄──┘ + * ``` + * 4. for fine grained inputs and non fine grained outputs + * ``` + * ┌──FineGrainedPipelineEvent + * PlainPipelineEvent◄──┼──FineGrainedPipelineEvent + * ├──FineGrainedPipelineEvent + * └──FineGrainedPipelineEvent + * ``` + */ + for (const auto & output : outputs) + { + for (const auto & input : inputs) + output->addInput(input); + } + } +} } // namespace void Pipeline::addPlanNode(const PhysicalPlanNodePtr & plan_node) @@ -92,34 +160,40 @@ PipelineExecGroup Pipeline::buildExecGroup(PipelineExecutorStatus & exec_status, Events Pipeline::toEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency) { Events all_events; - toEvent(status, context, concurrency, all_events); + doToEvents(status, context, concurrency, all_events); assert(!all_events.empty()); return all_events; } -EventPtr Pipeline::toEvent(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events) +Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency) { - // TODO support fine grained shuffle - // - a fine grained partition maps to an event - // - the event flow will be - // ``` - // disable fine grained partition pipeline enable fine grained partition pipeline enable fine grained partition pipeline - // ┌───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent - // PlainPipelineEvent<────────┼───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent - // ├───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent - // └───────────────FineGrainedPipelineEvent<────────────────FineGrainedPipelineEvent - // ``` auto memory_tracker = current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr; + Events self_events; + assert(!plan_nodes.empty()); + if (plan_nodes.front()->getFineGrainedShuffle().enable()) + { + auto fine_grained_exec_group = buildExecGroup(status, context, concurrency); + assert(!fine_grained_exec_group.empty()); + for (auto & pipeline_exec : fine_grained_exec_group) + self_events.push_back(std::make_shared<FineGrainedPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), std::move(pipeline_exec))); + } + else + { + self_events.push_back(std::make_shared<PlainPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), concurrency)); + } + return self_events; +} - auto plain_pipeline_event = std::make_shared<PlainPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), concurrency); +Events Pipeline::doToEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events) +{ + auto self_events = toSelfEvents(status, context, concurrency); for (const auto & child : children) { - auto input = child->toEvent(status, context, concurrency, all_events); - assert(input); - plain_pipeline_event->addInput(input); + auto inputs = child->doToEvents(status, context, concurrency, all_events); + mapEvents(inputs, self_events); } - all_events.push_back(plain_pipeline_event); - return plain_pipeline_event; + all_events.insert(all_events.end(), self_events.cbegin(), self_events.cend()); + return self_events; } bool Pipeline::isSupported(const tipb::DAGRequest & dag_request) diff --git a/dbms/src/Flash/Pipeline/Pipeline.h b/dbms/src/Flash/Pipeline/Pipeline.h index 37504b29b6a..bd75f719ba5 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.h +++ b/dbms/src/Flash/Pipeline/Pipeline.h @@ -71,7 +71,8 @@ class Pipeline : public std::enable_shared_from_this<Pipeline> private: void toSelfString(FmtBuffer & buffer, size_t level) const; - EventPtr toEvent(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events); + Events toSelfEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency); + Events doToEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events); private: const UInt32 id; diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp index bf69617a519..368a73a84d1 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include <Flash/Pipeline/Pipeline.h> #include <Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h> #include <Flash/Pipeline/Schedule/Tasks/PipelineTask.h> From d2520b630ca3687ac670ebab3c374292fe4ae650 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Thu, 2 Mar 2023 17:47:36 +0800 Subject: [PATCH 04/24] fix comment --- dbms/src/Flash/Pipeline/Pipeline.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index 3e2ca5e2738..38631212033 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -73,8 +73,8 @@ void mapEvents(const Events & inputs, const Events & outputs) * ``` * FineGrainedPipelineEvent◄──┐ ┌──FineGrainedPipelineEvent * FineGrainedPipelineEvent◄──┼─┼──FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄──┤ ├──FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄──┘ └──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┤ └──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┘ * ``` * 2. for non fine grained inputs and non fine grained outputs * This is not possible, the size of inputs and outputs must be the same and 1. From c1362bd85534c7db05a3f6e16a40e4b69c8b252c Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Thu, 2 Mar 2023 18:49:44 +0800 Subject: [PATCH 05/24] add ut --- .../MockExecutor/ExchangeSenderBinder.cpp | 29 +++++++++++- .../Debug/MockExecutor/ExchangeSenderBinder.h | 14 +++++- dbms/src/Flash/Pipeline/Pipeline.cpp | 17 ++++--- dbms/src/Flash/tests/gtest_compute_server.cpp | 46 ++++++++++--------- dbms/src/TestUtils/mockExecutor.cpp | 9 +++- dbms/src/TestUtils/mockExecutor.h | 3 +- 6 files changed, 82 insertions(+), 36 deletions(-) diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp index 88bfc19c4fb..33ea7e004be 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp @@ -77,9 +77,34 @@ tipb::ExchangeType ExchangeSenderBinder::getType() const return type; } -ExecutorBinderPtr compileExchangeSender(ExecutorBinderPtr input, size_t & executor_index, tipb::ExchangeType exchange_type) +ExecutorBinderPtr compileExchangeSender( + ExecutorBinderPtr input, + size_t & executor_index, + tipb::ExchangeType exchange_type, + ASTPtr partition_key_list, + uint64_t fine_grained_shuffle_stream_count) { - ExecutorBinderPtr exchange_sender = std::make_shared<mock::ExchangeSenderBinder>(executor_index, input->output_schema, exchange_type); + std::vector<size_t> partition_key_indexes; + for (const auto & partition_key : partition_key_list->children) + { + size_t schema_index = 0; + for (; schema_index < input->output_schema.size(); ++schema_index) + { + if (input->output_schema[schema_index].first == partition_key->getColumnName()) + { + partition_key_indexes.push_back(schema_index); + break; + } + } + if (schema_index == input->output_schema.size()) + throw Exception("Unknown partition key: " + partition_key->getColumnName()); + } + ExecutorBinderPtr exchange_sender = std::make_shared<mock::ExchangeSenderBinder>( + executor_index, + input->output_schema, + exchange_type, + partition_key_indexes, + fine_grained_shuffle_stream_count); exchange_sender->children.push_back(input); return exchange_sender; } diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h index c03899351eb..f6d10599614 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.h @@ -22,7 +22,12 @@ namespace DB::mock class ExchangeSenderBinder : public ExecutorBinder { public: - ExchangeSenderBinder(size_t & index, const DAGSchema & output, tipb::ExchangeType type_, const std::vector<size_t> & partition_keys_ = {}, uint64_t fine_grained_shuffle_stream_count_ = 0) + ExchangeSenderBinder( + size_t & index, + const DAGSchema & output, + tipb::ExchangeType type_, + const std::vector<size_t> & partition_keys_ = {}, + uint64_t fine_grained_shuffle_stream_count_ = 0) : ExecutorBinder(index, "exchange_sender_" + std::to_string(index), output) , type(type_) , partition_keys(partition_keys_) @@ -42,5 +47,10 @@ class ExchangeSenderBinder : public ExecutorBinder uint64_t fine_grained_shuffle_stream_count; }; -ExecutorBinderPtr compileExchangeSender(ExecutorBinderPtr input, size_t & executor_index, tipb::ExchangeType exchange_type); +ExecutorBinderPtr compileExchangeSender( + ExecutorBinderPtr input, + size_t & executor_index, + tipb::ExchangeType exchange_type, + ASTPtr partition_key_list = {}, + uint64_t fine_grained_shuffle_stream_count = 0); } // namespace DB::mock diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index 38631212033..2b59ecd22c7 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -43,22 +43,22 @@ void mapEvents(const Events & inputs, const Events & outputs) /** * 1. for fine grained inputs and fine grained outputs * ``` - * FineGrainedPipelineEvent<────FineGrainedPipelineEvent - * FineGrainedPipelineEvent<────FineGrainedPipelineEvent - * FineGrainedPipelineEvent<────FineGrainedPipelineEvent - * FineGrainedPipelineEvent<────FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent * ``` * 2. for non fine grained inputs and non fine grained outputs * ``` - * PlainPipelineEvent<────PlainPipelineEvent + * PlainPipelineEvent◄────PlainPipelineEvent * ``` * 3. for non fine grained inputs and fine grained outputs * ``` - * PlainPipelineEvent<────FineGrainedPipelineEvent + * PlainPipelineEvent◄────FineGrainedPipelineEvent * ``` * 4. for fine grained inputs and non fine grained outputs * ``` - * FineGrainedPipelineEvent<────PlainPipelineEvent + * FineGrainedPipelineEvent◄────PlainPipelineEvent * ``` */ size_t partition_num = inputs.size(); @@ -170,16 +170,19 @@ Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context auto memory_tracker = current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr; Events self_events; assert(!plan_nodes.empty()); + // The source plan node determines whether the execution mode is fine grained or non-fine grained. if (plan_nodes.front()->getFineGrainedShuffle().enable()) { auto fine_grained_exec_group = buildExecGroup(status, context, concurrency); assert(!fine_grained_exec_group.empty()); for (auto & pipeline_exec : fine_grained_exec_group) self_events.push_back(std::make_shared<FineGrainedPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), std::move(pipeline_exec))); + LOG_DEBUG(log, "generate {} fine grained pipeline event", self_events.size()); } else { self_events.push_back(std::make_shared<PlainPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), concurrency)); + LOG_DEBUG(log, "generate one plain pipeline event"); } return self_events; } diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index bf6d5b15806..70aeec3af75 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -132,9 +132,9 @@ try { std::vector<String> expected_strings = { R"( -exchange_sender_2 | type:PassThrough, {<0, Long>} - project_1 | {<0, Long>} - table_scan_0 | {<0, Long>})"}; + exchange_sender_2 | type:PassThrough, {<0, Long>} + project_1 | {<0, Long>} + table_scan_0 | {<0, Long>})"}; ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( context .scan("test_db", "big_table") @@ -145,12 +145,12 @@ exchange_sender_2 | type:PassThrough, {<0, Long>} { std::vector<String> expected_strings = { R"( -exchange_sender_1 | type:PassThrough, {<0, Long>} - table_scan_0 | {<0, Long>})", + exchange_sender_1 | type:PassThrough, {<0, Long>} + table_scan_0 | {<0, Long>})", R"( -exchange_sender_4 | type:PassThrough, {<0, Long>} - project_3 | {<0, Long>} - exchange_receiver_2 | type:PassThrough, {<0, Long>})"}; + exchange_sender_4 | type:PassThrough, {<0, Long>} + project_3 | {<0, Long>} + exchange_receiver_2 | type:PassThrough, {<0, Long>})"}; ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( context .scan("test_db", "big_table") @@ -163,12 +163,12 @@ exchange_sender_4 | type:PassThrough, {<0, Long>} { std::vector<String> expected_strings = { R"( -exchange_sender_1 | type:Broadcast, {<0, Long>} - table_scan_0 | {<0, Long>})", + exchange_sender_1 | type:Broadcast, {<0, Long>} + table_scan_0 | {<0, Long>})", R"( -exchange_sender_4 | type:PassThrough, {<0, Long>} - project_3 | {<0, Long>} - exchange_receiver_2 | type:Broadcast, {<0, Long>})"}; + exchange_sender_4 | type:PassThrough, {<0, Long>} + project_3 | {<0, Long>} + exchange_receiver_2 | type:Broadcast, {<0, Long>})"}; ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( context .scan("test_db", "big_table") @@ -197,14 +197,18 @@ exchange_sender_4 | type:PassThrough, {<0, Long>} exchange_sender_4 | type:PassThrough, {<0, Long>} project_3 | {<0, Long>} exchange_receiver_2 | type:Hash, {<0, Long>})"}; - ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( - context - .scan("test_db", "big_table") - .exchangeSender(tipb::ExchangeType::Hash) - .exchangeReceiver("recv", {{"s1", TiDB::TP::TypeLong}}) - .project({"s1"}), - expected_strings, - expected_cols); + std::vector<uint64_t> fine_grained_shuffle_stream_count{8, 0}; + for (uint64_t stream_count : fine_grained_shuffle_stream_count) + { + ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( + context + .scan("test_db", "big_table") + .exchangeSender(tipb::ExchangeType::Hash, {"test_db.big_table.s1"}, stream_count) + .exchangeReceiver("recv", {{"s1", TiDB::TP::TypeLong}}, stream_count) + .project({"s1"}), + expected_strings, + expected_cols); + } } WRAP_FOR_SERVER_TEST_END } diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index b40a7450e18..cb2a2ec7aec 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -268,10 +268,15 @@ DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNameVec col_names) return *this; } -DAGRequestBuilder & DAGRequestBuilder::exchangeSender(tipb::ExchangeType exchange_type) +DAGRequestBuilder & DAGRequestBuilder::exchangeSender(tipb::ExchangeType exchange_type, MockColumnNameVec part_keys, uint64_t fine_grained_shuffle_stream_count) { assert(root); - root = mock::compileExchangeSender(root, getExecutorIndex(), exchange_type); + auto partition_key_list = std::make_shared<ASTExpressionList>(); + for (const auto & part_key : part_keys) + { + partition_key_list->children.push_back(col(part_key)); + } + root = mock::compileExchangeSender(root, getExecutorIndex(), exchange_type, partition_key_list, fine_grained_shuffle_stream_count); return *this; } diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 248d303abb2..3498514bd5f 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -114,8 +114,7 @@ class DAGRequestBuilder } DAGRequestBuilder & project(MockColumnNameVec col_names); - - DAGRequestBuilder & exchangeSender(tipb::ExchangeType exchange_type); + DAGRequestBuilder & exchangeSender(tipb::ExchangeType exchange_type, MockColumnNameVec part_keys = {}, uint64_t fine_grained_shuffle_stream_count = 0); /// User should prefer using other simplified join buidler API instead of this one unless he/she have to test /// join conditional expressions and knows how TiDB translates sql's `join on` clause to conditional expressions. From 8aabbe8183dc4b7a01f0dce635fd2ed1f0118948 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Thu, 2 Mar 2023 18:51:18 +0800 Subject: [PATCH 06/24] fix --- dbms/src/Flash/tests/gtest_compute_server.cpp | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/Flash/tests/gtest_compute_server.cpp b/dbms/src/Flash/tests/gtest_compute_server.cpp index 70aeec3af75..3cf4f43ecbf 100644 --- a/dbms/src/Flash/tests/gtest_compute_server.cpp +++ b/dbms/src/Flash/tests/gtest_compute_server.cpp @@ -132,9 +132,9 @@ try { std::vector<String> expected_strings = { R"( - exchange_sender_2 | type:PassThrough, {<0, Long>} - project_1 | {<0, Long>} - table_scan_0 | {<0, Long>})"}; +exchange_sender_2 | type:PassThrough, {<0, Long>} + project_1 | {<0, Long>} + table_scan_0 | {<0, Long>})"}; ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( context .scan("test_db", "big_table") @@ -145,12 +145,12 @@ try { std::vector<String> expected_strings = { R"( - exchange_sender_1 | type:PassThrough, {<0, Long>} - table_scan_0 | {<0, Long>})", +exchange_sender_1 | type:PassThrough, {<0, Long>} + table_scan_0 | {<0, Long>})", R"( - exchange_sender_4 | type:PassThrough, {<0, Long>} - project_3 | {<0, Long>} - exchange_receiver_2 | type:PassThrough, {<0, Long>})"}; +exchange_sender_4 | type:PassThrough, {<0, Long>} + project_3 | {<0, Long>} + exchange_receiver_2 | type:PassThrough, {<0, Long>})"}; ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( context .scan("test_db", "big_table") @@ -163,12 +163,12 @@ try { std::vector<String> expected_strings = { R"( - exchange_sender_1 | type:Broadcast, {<0, Long>} - table_scan_0 | {<0, Long>})", +exchange_sender_1 | type:Broadcast, {<0, Long>} + table_scan_0 | {<0, Long>})", R"( - exchange_sender_4 | type:PassThrough, {<0, Long>} - project_3 | {<0, Long>} - exchange_receiver_2 | type:Broadcast, {<0, Long>})"}; +exchange_sender_4 | type:PassThrough, {<0, Long>} + project_3 | {<0, Long>} + exchange_receiver_2 | type:Broadcast, {<0, Long>})"}; ASSERT_MPPTASK_EQUAL_PLAN_AND_RESULT( context .scan("test_db", "big_table") From 700e04aacbcc4b1eeb8ed3c4ca5c8e19229814d6 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Thu, 2 Mar 2023 18:59:41 +0800 Subject: [PATCH 07/24] fix comment --- .../Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h index ce3c0a3eedc..545b21be7e4 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h @@ -39,7 +39,7 @@ class FineGrainedPipelineEvent : public PipelineEvent std::vector<TaskPtr> scheduleImpl() override; private: - // The pipeline exec for executing the specific fine-grained shuffle partition id. + // The pipeline exec for executing the specific fine-grained partition. PipelineExecPtr pipeline_exec; }; } // namespace DB From f70406978f65cb791a9a02681f6a9ff3a802dc03 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Fri, 3 Mar 2023 11:56:19 +0800 Subject: [PATCH 08/24] fix --- dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h index d4c66f81e0e..2692bb6d387 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalWindowSort.h @@ -52,6 +52,5 @@ class PhysicalWindowSort : public PhysicalUnary private: SortDescription order_descr; - FineGrainedShuffle fine_grained_shuffle; }; } // namespace DB From 60359991325187fc342ad0dfa6c069bf4a40ffee Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Fri, 3 Mar 2023 13:07:35 +0800 Subject: [PATCH 09/24] update --- dbms/src/Flash/Pipeline/Pipeline.cpp | 19 +++++++++++-------- dbms/src/Flash/Pipeline/Pipeline.h | 2 ++ 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index 2b59ecd22c7..137413c87e5 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -79,12 +79,8 @@ void mapEvents(const Events & inputs, const Events & outputs) * 2. for non fine grained inputs and non fine grained outputs * This is not possible, the size of inputs and outputs must be the same and 1. * 3. for non fine grained inputs and fine grained outputs - * ``` - * FineGrainedPipelineEvent◄──┐ - * FineGrainedPipelineEvent◄──┼──PlainPipelineEvent - * FineGrainedPipelineEvent◄──┤ - * FineGrainedPipelineEvent◄──┘ - * ``` + * This is not possible, if fine-grained is enabled in outputs, then inputs must also be enabled. + * Checked in `doToEvents`. * 4. for fine grained inputs and non fine grained outputs * ``` * ┌──FineGrainedPipelineEvent @@ -157,6 +153,12 @@ PipelineExecGroup Pipeline::buildExecGroup(PipelineExecutorStatus & exec_status, return builder.build(); } +bool Pipeline::isFineGrainedPipeline() const +{ + // The source plan node determines whether the execution mode is fine grained or non-fine grained. + return plan_nodes.front()->getFineGrainedShuffle().enable(); +} + Events Pipeline::toEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency) { Events all_events; @@ -170,8 +172,7 @@ Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context auto memory_tracker = current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr; Events self_events; assert(!plan_nodes.empty()); - // The source plan node determines whether the execution mode is fine grained or non-fine grained. - if (plan_nodes.front()->getFineGrainedShuffle().enable()) + if (isFineGrainedPipeline()) { auto fine_grained_exec_group = buildExecGroup(status, context, concurrency); assert(!fine_grained_exec_group.empty()); @@ -192,6 +193,8 @@ Events Pipeline::doToEvents(PipelineExecutorStatus & status, Context & context, auto self_events = toSelfEvents(status, context, concurrency); for (const auto & child : children) { + // If fine-grained is enabled in the current pipeline, then the child must also be enabled. + RUNTIME_CHECK(!isFineGrainedPipeline() || child->isFineGrainedPipeline()); auto inputs = child->doToEvents(status, context, concurrency, all_events); mapEvents(inputs, self_events); } diff --git a/dbms/src/Flash/Pipeline/Pipeline.h b/dbms/src/Flash/Pipeline/Pipeline.h index bd75f719ba5..f94ec2ec3a7 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.h +++ b/dbms/src/Flash/Pipeline/Pipeline.h @@ -68,6 +68,8 @@ class Pipeline : public std::enable_shared_from_this<Pipeline> Block getSampleBlock() const; + bool isFineGrainedPipeline() const; + private: void toSelfString(FmtBuffer & buffer, size_t level) const; From 63ca1c0fd23f010a5ecc3f6a6a7422f3544e46f0 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Fri, 3 Mar 2023 13:09:44 +0800 Subject: [PATCH 10/24] update --- dbms/src/Flash/Pipeline/Pipeline.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index 137413c87e5..b1ec4e103cd 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -155,6 +155,7 @@ PipelineExecGroup Pipeline::buildExecGroup(PipelineExecutorStatus & exec_status, bool Pipeline::isFineGrainedPipeline() const { + assert(!plan_nodes.empty()); // The source plan node determines whether the execution mode is fine grained or non-fine grained. return plan_nodes.front()->getFineGrainedShuffle().enable(); } From 4b7f61d4432c6af2d5daf9964af0f2c0b2b95154 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Fri, 3 Mar 2023 16:58:47 +0800 Subject: [PATCH 11/24] merge master --- dbms/src/Flash/Pipeline/Pipeline.cpp | 5 ++++- .../Flash/Planner/Plans/PhysicalAggregationBuild.cpp | 10 +++++++--- .../Flash/Planner/Plans/PhysicalAggregationBuild.h | 8 ++++++-- .../Planner/Plans/PhysicalAggregationConvergent.cpp | 12 ++++++++---- .../Planner/Plans/PhysicalAggregationConvergent.h | 8 ++++++-- .../Flash/Planner/Plans/PhysicalMockTableScan.cpp | 4 +--- dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h | 1 - dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp | 4 +--- dbms/src/Flash/Planner/Plans/PhysicalTableScan.h | 1 - 9 files changed, 33 insertions(+), 20 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index a6e6192a24c..89f02742b2a 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -215,12 +215,15 @@ bool Pipeline::isSupported(const tipb::DAGRequest & dag_request) case tipb::ExecType::TypeSelection: case tipb::ExecType::TypeLimit: case tipb::ExecType::TypeTopN: - case tipb::ExecType::TypeAggregation: case tipb::ExecType::TypeTableScan: case tipb::ExecType::TypeExchangeSender: case tipb::ExecType::TypeExchangeReceiver: case tipb::ExecType::TypeExpand: return true; + case tipb::ExecType::TypeAggregation: + // TODO support fine grained shuffle. + if (!FineGrainedShuffle(&executor).enable()) + return true; default: is_supported = false; return false; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp b/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp index a2a09f761c4..dbf154de242 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp @@ -19,18 +19,22 @@ namespace DB { -void PhysicalAggregationBuild::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & context, size_t /*concurrency*/) +void PhysicalAggregationBuild::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t /*concurrency*/) { if (!before_agg_actions->getActions().empty()) { group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(group_builder.exec_status, log->identifier(), before_agg_actions)); + builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(exec_status, log->identifier(), before_agg_actions)); }); } size_t build_index = 0; group_builder.transform([&](auto & builder) { - builder.setSinkOp(std::make_unique<AggregateSinkOp>(group_builder.exec_status, build_index++, aggregate_context, log->identifier())); + builder.setSinkOp(std::make_unique<AggregateSinkOp>(exec_status, build_index++, aggregate_context, log->identifier())); }); Block before_agg_header = group_builder.getCurrentHeader(); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.h b/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.h index e512669bf8e..2198d23b2ad 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.h @@ -38,7 +38,7 @@ class PhysicalAggregationBuild : public PhysicalUnary bool is_final_agg_, const AggregateDescriptions & aggregate_descriptions_, const AggregateContextPtr & aggregate_context_) - : PhysicalUnary(executor_id_, PlanType::AggregationBuild, schema_, req_id, child_) + : PhysicalUnary(executor_id_, PlanType::AggregationBuild, schema_, FineGrainedShuffle{}, req_id, child_) , before_agg_actions(before_agg_actions_) , aggregation_keys(aggregation_keys_) , aggregation_collators(aggregation_collators_) @@ -47,7 +47,11 @@ class PhysicalAggregationBuild : public PhysicalUnary , aggregate_context(aggregate_context_) {} - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & context, size_t /*concurrency*/) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & context, + size_t /*concurrency*/) override; private: DISABLE_USELESS_FUNCTION_FOR_BREAKER diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.cpp b/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.cpp index 92221e90011..74989b4c9a8 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.cpp @@ -19,7 +19,11 @@ namespace DB { -void PhysicalAggregationConvergent::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) +void PhysicalAggregationConvergent::buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t /*concurrency*/) { aggregate_context->initConvergent(); @@ -28,7 +32,7 @@ void PhysicalAggregationConvergent::buildPipelineExec(PipelineExecGroupBuilder & group_builder.init(1); group_builder.transform([&](auto & builder) { builder.setSourceOp(std::make_unique<NullSourceOp>( - group_builder.exec_status, + exec_status, aggregate_context->getHeader(), log->identifier())); }); @@ -39,7 +43,7 @@ void PhysicalAggregationConvergent::buildPipelineExec(PipelineExecGroupBuilder & size_t index = 0; group_builder.transform([&](auto & builder) { builder.setSourceOp(std::make_unique<AggregateConvergentSourceOp>( - group_builder.exec_status, + exec_status, aggregate_context, index++, log->identifier())); @@ -49,7 +53,7 @@ void PhysicalAggregationConvergent::buildPipelineExec(PipelineExecGroupBuilder & if (!expr_after_agg->getActions().empty()) { group_builder.transform([&](auto & builder) { - builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(group_builder.exec_status, log->identifier(), expr_after_agg)); + builder.appendTransformOp(std::make_unique<ExpressionTransformOp>(exec_status, log->identifier(), expr_after_agg)); }); } } diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.h b/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.h index 7895986b6cb..9de698d0170 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregationConvergent.h @@ -31,12 +31,16 @@ class PhysicalAggregationConvergent : public PhysicalLeaf const String & req_id, const AggregateContextPtr & aggregate_context_, const ExpressionActionsPtr & expr_after_agg_) - : PhysicalLeaf(executor_id_, PlanType::AggregationConvergent, schema_, req_id) + : PhysicalLeaf(executor_id_, PlanType::AggregationConvergent, schema_, FineGrainedShuffle{}, req_id) , expr_after_agg(expr_after_agg_) , aggregate_context(aggregate_context_) {} - void buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t concurrency) override; + void buildPipelineExecGroup( + PipelineExecutorStatus & exec_status, + PipelineExecGroupBuilder & group_builder, + Context & /*context*/, + size_t concurrency) override; private: DISABLE_USELESS_FUNCTION_FOR_BREAKER diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index 682d8944c1b..d8cc65c0c01 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -76,12 +76,11 @@ std::pair<NamesAndTypes, BlockInputStreams> mockSchemaAndStreams( PhysicalMockTableScan::PhysicalMockTableScan( const String & executor_id_, const NamesAndTypes & schema_, - const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const Block & sample_block_, const BlockInputStreams & mock_streams_, Int64 table_id_) - : PhysicalLeaf(executor_id_, PlanType::MockTableScan, schema_, fine_grained_shuffle_, req_id) + : PhysicalLeaf(executor_id_, PlanType::MockTableScan, schema_, FineGrainedShuffle{}, req_id) , sample_block(sample_block_) , mock_streams(mock_streams_) , table_id(table_id_) @@ -99,7 +98,6 @@ PhysicalPlanNodePtr PhysicalMockTableScan::build( auto physical_mock_table_scan = std::make_shared<PhysicalMockTableScan>( executor_id, schema, - FineGrainedShuffle{}, log->identifier(), Block(schema), mock_streams, diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h index 6573e6cb29c..cadff9d2c7a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h @@ -39,7 +39,6 @@ class PhysicalMockTableScan : public PhysicalLeaf PhysicalMockTableScan( const String & executor_id_, const NamesAndTypes & schema_, - const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const Block & sample_block_, const BlockInputStreams & mock_streams_, diff --git a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp index 02de8a19925..de2431a0048 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.cpp @@ -28,11 +28,10 @@ namespace DB PhysicalTableScan::PhysicalTableScan( const String & executor_id_, const NamesAndTypes & schema_, - const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const TiDBTableScan & tidb_table_scan_, const Block & sample_block_) - : PhysicalLeaf(executor_id_, PlanType::TableScan, schema_, fine_grained_shuffle_, req_id) + : PhysicalLeaf(executor_id_, PlanType::TableScan, schema_, FineGrainedShuffle{}, req_id) , tidb_table_scan(tidb_table_scan_) , sample_block(sample_block_) {} @@ -46,7 +45,6 @@ PhysicalPlanNodePtr PhysicalTableScan::build( auto physical_table_scan = std::make_shared<PhysicalTableScan>( executor_id, schema, - FineGrainedShuffle{}, log->identifier(), table_scan, Block(schema)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h index 2525f79c052..00c68e783cf 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalTableScan.h @@ -30,7 +30,6 @@ class PhysicalTableScan : public PhysicalLeaf PhysicalTableScan( const String & executor_id_, const NamesAndTypes & schema_, - const FineGrainedShuffle & fine_grained_shuffle_, const String & req_id, const TiDBTableScan & tidb_table_scan_, const Block & sample_block_); From 06c8004482f787d447a9722649de0547b18ec60c Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Fri, 3 Mar 2023 17:45:21 +0800 Subject: [PATCH 12/24] add fullstack test --- .../config/tiflash_dt_enable_pipeline.toml | 54 +++++++++++++++++++ tests/docker/tiflash-dt-enable-pipeline.yaml | 43 +++++++++++++++ tests/tidb-ci/enable_pipeline | 1 + tests/tidb-ci/run.sh | 7 +++ tests/tidb-ci/tiflash-dt-enable-pipeline.yaml | 1 + 5 files changed, 106 insertions(+) create mode 100644 tests/docker/config/tiflash_dt_enable_pipeline.toml create mode 100644 tests/docker/tiflash-dt-enable-pipeline.yaml create mode 120000 tests/tidb-ci/enable_pipeline create mode 120000 tests/tidb-ci/tiflash-dt-enable-pipeline.yaml diff --git a/tests/docker/config/tiflash_dt_enable_pipeline.toml b/tests/docker/config/tiflash_dt_enable_pipeline.toml new file mode 100644 index 00000000000..69e120fbfc5 --- /dev/null +++ b/tests/docker/config/tiflash_dt_enable_pipeline.toml @@ -0,0 +1,54 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +tmp_path = "/tmp/tiflash/data/tmp" + +path = "/tmp/tiflash/data/db" +capacity = "10737418240" + +mark_cache_size = 5368709120 +minmax_index_cache_size = 5368709120 +tcp_port = 9000 +http_port = 8123 + +[flash] +tidb_status_addr = "tidb0:10080" +service_addr = "0.0.0.0:3930" +[flash.flash_cluster] +update_rule_interval = 5 +[flash.proxy] +addr = "0.0.0.0:20170" +advertise-addr = "tiflash0:20170" +data-dir = "/data" +config = "/proxy.toml" +log-file = "/log/proxy.log" +engine-addr = "tiflash0:3930" +status-addr = "0.0.0.0:20181" +advertise-status-addr = "tiflash0:20181" + +[logger] +count = 10 +errorlog = "/tmp/tiflash/log/error.log" +size = "1000M" +log = "/tmp/tiflash/log/server.log" +level = "trace" + +[raft] +pd_addr = "pd0:2379" +ignore_databases = "system,default" + +[profiles] +[profiles.default] +enable_pipeline = 1 +# max_memory_usage = 0 diff --git a/tests/docker/tiflash-dt-enable-pipeline.yaml b/tests/docker/tiflash-dt-enable-pipeline.yaml new file mode 100644 index 00000000000..463cb3ea71e --- /dev/null +++ b/tests/docker/tiflash-dt-enable-pipeline.yaml @@ -0,0 +1,43 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +version: '2.3' + +services: + # for tests under fullstack-test directory + # (engine DeltaTree) + tiflash0: + image: hub.pingcap.net/tiflash/tiflash-ci-base + volumes: + - ./config/tiflash_dt_enable_pipeline.toml:/config.toml:ro + - ./data/tiflash:/tmp/tiflash/data + - ./log/tiflash:/tmp/tiflash/log + - ..:/tests + - ../docker/_env.sh:/tests/_env.sh + - ./log/tiflash-cluster-manager:/tmp/tiflash/data/tmp + - ./config/proxy.toml:/proxy.toml:ro + - ./config/cipher-file-256:/cipher-file-256:ro + - ./data/proxy:/data + - ./log/proxy:/log + - ../.build/tiflash:/tiflash + entrypoint: + - /tiflash/tiflash + - server + - --config-file + - /config.toml + restart: on-failure + depends_on: + - "pd0" + - "tikv0" + diff --git a/tests/tidb-ci/enable_pipeline b/tests/tidb-ci/enable_pipeline new file mode 120000 index 00000000000..1c6442406ac --- /dev/null +++ b/tests/tidb-ci/enable_pipeline @@ -0,0 +1 @@ +../fullstack-test/mpp \ No newline at end of file diff --git a/tests/tidb-ci/run.sh b/tests/tidb-ci/run.sh index 18a242fe4d3..bb7a5f5283d 100755 --- a/tests/tidb-ci/run.sh +++ b/tests/tidb-ci/run.sh @@ -54,6 +54,13 @@ docker-compose -f cluster.yaml -f tiflash-dt-disable-planner.yaml exec -T tiflas docker-compose -f cluster.yaml -f tiflash-dt-disable-planner.yaml down clean_data_log +docker-compose -f cluster.yaml -f tiflash-dt-enable-pipeline.yaml up -d +wait_env +docker-compose -f cluster.yaml -f tiflash-dt-enable-pipeline.yaml exec -T tiflash0 bash -c 'cd /tests ; ./run-test.sh tidb-ci/enable_pipeline' + +docker-compose -f cluster.yaml -f tiflash-dt-enable-pipeline.yaml down +clean_data_log + # run new_collation_fullstack tests docker-compose -f cluster_new_collation.yaml -f tiflash-dt.yaml down clean_data_log diff --git a/tests/tidb-ci/tiflash-dt-enable-pipeline.yaml b/tests/tidb-ci/tiflash-dt-enable-pipeline.yaml new file mode 120000 index 00000000000..75059107013 --- /dev/null +++ b/tests/tidb-ci/tiflash-dt-enable-pipeline.yaml @@ -0,0 +1 @@ +../docker/tiflash-dt-enable-pipeline.yaml \ No newline at end of file From f6b993601232344c9b28d20e45041689a4115118 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Fri, 3 Mar 2023 17:46:44 +0800 Subject: [PATCH 13/24] update --- tests/docker/config/tiflash_dt_enable_pipeline.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/docker/config/tiflash_dt_enable_pipeline.toml b/tests/docker/config/tiflash_dt_enable_pipeline.toml index 69e120fbfc5..18a24fd0253 100644 --- a/tests/docker/config/tiflash_dt_enable_pipeline.toml +++ b/tests/docker/config/tiflash_dt_enable_pipeline.toml @@ -50,5 +50,6 @@ ignore_databases = "system,default" [profiles] [profiles.default] +enable_planner = 1 enable_pipeline = 1 # max_memory_usage = 0 From 88b64c80b9e4ae6efed7aa212462d9aa6ed72049 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Fri, 3 Mar 2023 17:59:29 +0800 Subject: [PATCH 14/24] fix comment --- dbms/src/Flash/Pipeline/Pipeline.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index 89f02742b2a..c4e7b4d79a7 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -53,12 +53,11 @@ void mapEvents(const Events & inputs, const Events & outputs) * PlainPipelineEvent◄────PlainPipelineEvent * ``` * 3. for non fine grained inputs and fine grained outputs - * ``` - * PlainPipelineEvent◄────FineGrainedPipelineEvent - * ``` + * This is not possible, if fine-grained is enabled in outputs, then inputs must also be enabled. + * Checked in `doToEvents`. * 4. for fine grained inputs and non fine grained outputs * ``` - * FineGrainedPipelineEvent◄────PlainPipelineEvent + * PlainPipelineEvent◄────FineGrainedPipelineEvent * ``` */ size_t partition_num = inputs.size(); From 688cc417c262526030354480e941f7c35dc32b64 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Mon, 6 Mar 2023 17:23:57 +0800 Subject: [PATCH 15/24] add more comments --- dbms/src/Flash/Pipeline/Pipeline.cpp | 27 +++++++++++++++++++++------ dbms/src/Flash/Pipeline/Pipeline.h | 2 +- 2 files changed, 22 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index c4e7b4d79a7..a533fe715bb 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -152,7 +152,22 @@ PipelineExecGroup Pipeline::buildExecGroup(PipelineExecutorStatus & exec_status, return builder.build(); } -bool Pipeline::isFineGrainedPipeline() const +/** + * There are two execution modes in pipeline. + * 1. non fine grained mode + * A pipeline generates an event(PlainPipelineEvent). + * This means that all the operators in the pipeline are finished before the next pipeline is triggered. + * 2. fine grained mode + * A pipeline will generate n Events(FineGrainedPipelineEvent), one for each data partition. + * There is a fine-grained mapping of Events between Pipelines, e.g. only Events from the same data partition will have dependencies on each other. + * This means that once some data partition of the previous pipeline has finished, the operators of the next pipeline's corresponding data partition can be started without having to wait for the entire pipeline to finish. + * + * ┌──non fine grained mode──┐ ┌──fine grained mode──┐ + * ┌──FineGrainedPipelineEvent◄───FineGrainedPipelineEvent + * PlainPipelineEvent◄───PlainPipelineEvent◄──┼──FineGrainedPipelineEvent◄───FineGrainedPipelineEvent + * └──FineGrainedPipelineEvent◄───FineGrainedPipelineEvent + */ +bool Pipeline::isFineGrainedMode() const { assert(!plan_nodes.empty()); // The source plan node determines whether the execution mode is fine grained or non-fine grained. @@ -172,18 +187,18 @@ Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context auto memory_tracker = current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr; Events self_events; assert(!plan_nodes.empty()); - if (isFineGrainedPipeline()) + if (isFineGrainedMode()) { auto fine_grained_exec_group = buildExecGroup(status, context, concurrency); assert(!fine_grained_exec_group.empty()); for (auto & pipeline_exec : fine_grained_exec_group) self_events.push_back(std::make_shared<FineGrainedPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), std::move(pipeline_exec))); - LOG_DEBUG(log, "generate {} fine grained pipeline event", self_events.size()); + LOG_DEBUG(log, "Execute in fine grained model and generate {} fine grained pipeline event", self_events.size()); } else { self_events.push_back(std::make_shared<PlainPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), concurrency)); - LOG_DEBUG(log, "generate one plain pipeline event"); + LOG_DEBUG(log, "Execute in non fine grained model and generate one plain pipeline event"); } return self_events; } @@ -193,8 +208,8 @@ Events Pipeline::doToEvents(PipelineExecutorStatus & status, Context & context, auto self_events = toSelfEvents(status, context, concurrency); for (const auto & child : children) { - // If fine-grained is enabled in the current pipeline, then the child must also be enabled. - RUNTIME_CHECK(!isFineGrainedPipeline() || child->isFineGrainedPipeline()); + // If the current pipeline is fine grained model, the child must also be. + RUNTIME_CHECK(!isFineGrainedMode() || child->isFineGrainedMode()); auto inputs = child->doToEvents(status, context, concurrency, all_events); mapEvents(inputs, self_events); } diff --git a/dbms/src/Flash/Pipeline/Pipeline.h b/dbms/src/Flash/Pipeline/Pipeline.h index f94ec2ec3a7..5e248ad1b28 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.h +++ b/dbms/src/Flash/Pipeline/Pipeline.h @@ -68,7 +68,7 @@ class Pipeline : public std::enable_shared_from_this<Pipeline> Block getSampleBlock() const; - bool isFineGrainedPipeline() const; + bool isFineGrainedMode() const; private: void toSelfString(FmtBuffer & buffer, size_t level) const; From d7effec13e9d1b477644716f3fe06841aa5e845a Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Tue, 7 Mar 2023 16:18:53 +0800 Subject: [PATCH 16/24] merge master and fix --- dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp | 4 ++-- dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index d0445cecf19..a921d7729d5 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -114,8 +114,8 @@ void PhysicalMockTableScan::buildBlockInputStreamImpl(DAGPipeline & pipeline, Co void PhysicalMockTableScan::buildPipelineExecGroup( PipelineExecutorStatus & exec_status, PipelineExecGroupBuilder & group_builder, - Context & /*context*/, - size_t /*concurrency*/) + Context & context, + size_t concurrency) { if (context.mockStorage()->useDeltaMerge()) { diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h index cadff9d2c7a..98dd0409ab4 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.h @@ -51,8 +51,8 @@ class PhysicalMockTableScan : public PhysicalLeaf void buildPipelineExecGroup( PipelineExecutorStatus & exec_status, PipelineExecGroupBuilder & group_builder, - Context & /*context*/, - size_t /*concurrency*/) override; + Context & context, + size_t concurrency) override; void initStreams(Context & context); From d86bfebe0985d1710593a6e76ad968a84a77f892 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Mon, 13 Mar 2023 11:47:56 +0800 Subject: [PATCH 17/24] replace 1024 with maxFineGrainedStreamCount --- dbms/src/Flash/Coprocessor/FineGrainedShuffle.h | 2 ++ dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h index 60debed2fde..f6415442efc 100644 --- a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h +++ b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h @@ -21,6 +21,8 @@ namespace DB { static constexpr std::string_view enableFineGrainedShuffleExtraInfo = "enable fine grained shuffle"; +static constexpr size_t maxFineGrainedStreamCount = 1024; + inline bool enableFineGrainedShuffle(uint64_t stream_count) { return stream_count > 0; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp index ab85b33d47e..75e3268b0d3 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalExchangeSender.cpp @@ -65,7 +65,7 @@ void PhysicalExchangeSender::buildBlockInputStreamImpl(DAGPipeline & pipeline, C { extra_info = String(enableFineGrainedShuffleExtraInfo); RUNTIME_CHECK(exchange_type == tipb::ExchangeType::Hash, ExchangeType_Name(exchange_type)); - RUNTIME_CHECK(fine_grained_shuffle.stream_count <= 1024, fine_grained_shuffle.stream_count); + RUNTIME_CHECK(fine_grained_shuffle.stream_count <= maxFineGrainedStreamCount, fine_grained_shuffle.stream_count); } pipeline.transform([&](auto & stream) { // construct writer @@ -96,7 +96,7 @@ void PhysicalExchangeSender::buildPipelineExecGroup( if (fine_grained_shuffle.enable()) { RUNTIME_CHECK(exchange_type == tipb::ExchangeType::Hash, ExchangeType_Name(exchange_type)); - RUNTIME_CHECK(fine_grained_shuffle.stream_count <= 1024, fine_grained_shuffle.stream_count); + RUNTIME_CHECK(fine_grained_shuffle.stream_count <= maxFineGrainedStreamCount, fine_grained_shuffle.stream_count); } group_builder.transform([&](auto & builder) { From dac6717d1f5a2c81bf59fd559a5613f45db7e3b3 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Tue, 14 Mar 2023 17:01:17 +0800 Subject: [PATCH 18/24] refine --- dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp | 11 ++++++++++- .../Pipeline/Exec/tests/gtest_simple_operator.cpp | 2 +- dbms/src/Flash/Pipeline/Pipeline.cpp | 2 +- .../Pipeline/Schedule/Events/PlainPipelineEvent.cpp | 2 +- dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp | 9 ++++----- dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h | 2 +- 6 files changed, 18 insertions(+), 10 deletions(-) diff --git a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp index 33ea7e004be..ef97c6b3e79 100644 --- a/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp +++ b/dbms/src/Debug/MockExecutor/ExchangeSenderBinder.cpp @@ -96,8 +96,17 @@ ExecutorBinderPtr compileExchangeSender( break; } } + auto schema_string = [&]() { + FmtBuffer buffer; + buffer.joinStr( + input->output_schema.cbegin(), + input->output_schema.cend(), + [](const auto & item, FmtBuffer & buf) { buf.append(item.first); }, + ", "); + return buffer.toString(); + }; if (schema_index == input->output_schema.size()) - throw Exception("Unknown partition key: " + partition_key->getColumnName()); + throw Exception(fmt::format("Unknown partition key: {}, schema is [{}]", partition_key->getColumnName(), schema_string())); } ExecutorBinderPtr exchange_sender = std::make_shared<mock::ExchangeSenderBinder>( executor_index, diff --git a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp index 8ab5854f811..7bcaf23210a 100644 --- a/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp +++ b/dbms/src/Flash/Pipeline/Exec/tests/gtest_simple_operator.cpp @@ -65,7 +65,7 @@ class SimpleOperatorTestRunner : public DB::tests::ExecutorTest PipelineExecGroupBuilder group_builder; PhysicalPlanVisitor::visitPostOrder(plan_tree, [&](const PhysicalPlanNodePtr & plan) { assert(plan); - plan->buildPipelineExecGroup(exec_status, group_builder, context.context, /*concurrency=*/1); + plan->buildPipelineExecGroup(exec_status, group_builder, *context.context, /*concurrency=*/1); }); auto result = group_builder.build(); assert(result.size() == 1); diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index fb4aea8289a..d690deb587c 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -190,7 +190,7 @@ Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context if (isFineGrainedMode()) { auto fine_grained_exec_group = buildExecGroup(status, context, concurrency); - assert(!fine_grained_exec_group.empty()); + RUNTIME_CHECK(!fine_grained_exec_group.empty()); for (auto & pipeline_exec : fine_grained_exec_group) self_events.push_back(std::make_shared<FineGrainedPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), std::move(pipeline_exec))); LOG_DEBUG(log, "Execute in fine grained model and generate {} fine grained pipeline event", self_events.size()); diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp index eca5ad3a8d7..4c25c6674db 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp @@ -22,7 +22,7 @@ std::vector<TaskPtr> PlainPipelineEvent::scheduleImpl() { assert(pipeline); auto pipeline_exec_group = pipeline->buildExecGroup(exec_status, context, concurrency); - assert(!pipeline_exec_group.empty()); + RUNTIME_CHECK(!pipeline_exec_group.empty()); std::vector<TaskPtr> tasks; tasks.reserve(pipeline_exec_group.size()); for (auto & pipeline_exec : pipeline_exec_group) diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp index 82605f95402..018a408e784 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.cpp @@ -54,14 +54,13 @@ void EventTask::finalize() noexcept { try { - bool tmp = false; - if (finalized.compare_exchange_strong(tmp, true)) - finalizeImpl(); + RUNTIME_CHECK(!finalized); + finalized = true; + finalizeImpl(); } catch (...) { - // ignore exception from finalizeImpl. - LOG_WARNING(log, "finalizeImpl throw exception: {}", getCurrentExceptionMessage(true, true)); + exec_status.onErrorOccurred(std::current_exception()); } } diff --git a/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h index 2b884831bd4..80163a80ece 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h +++ b/dbms/src/Flash/Pipeline/Schedule/Tasks/EventTask.h @@ -53,7 +53,7 @@ class EventTask : public Task private: PipelineExecutorStatus & exec_status; EventPtr event; - std::atomic_bool finalized{false}; + bool finalized = false; }; } // namespace DB From c1cba4ee850e4dd452dc78e34192c5916719a6d5 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Tue, 14 Mar 2023 17:08:24 +0800 Subject: [PATCH 19/24] u --- dbms/src/TestUtils/ExecutorTestUtils.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index b75b012eda1..ec81dc671ca 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -16,6 +16,7 @@ #include <Common/FmtUtils.h> #include <Debug/MockComputeServerManager.h> #include <Debug/MockStorage.h> +#include <Flash/Pipeline/Pipeline.h> #include <Flash/Pipeline/Schedule/TaskScheduler.h> #include <Flash/executeQuery.h> #include <Interpreters/Context.h> @@ -166,6 +167,8 @@ void ExecutorTest::executeExecutor( std::function<::testing::AssertionResult(const ColumnsWithTypeAndName &)> assert_func) { WRAP_FOR_TEST_BEGIN + if (enable_pipeline && !Pipeline::isSupported(*request)) + continue; std::vector<size_t> concurrencies{1, 2, 10}; for (auto concurrency : concurrencies) { @@ -186,6 +189,8 @@ void ExecutorTest::checkBlockSorted( std::function<::testing::AssertionResult(const ColumnsWithTypeAndName &, const ColumnsWithTypeAndName &)> assert_func) { WRAP_FOR_TEST_BEGIN + if (enable_pipeline && !Pipeline::isSupported(*request)) + continue; std::vector<size_t> concurrencies{2, 5, 10}; for (auto concurrency : concurrencies) { From cd049c9afea538480e694b037b5495be2faa791d Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Tue, 14 Mar 2023 17:10:31 +0800 Subject: [PATCH 20/24] replace magic number --- dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 99e00e61444..f8f43627d09 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -780,7 +780,7 @@ void DAGQueryBlockInterpreter::handleExchangeSender(DAGPipeline & pipeline) { extra_info = String(enableFineGrainedShuffleExtraInfo); RUNTIME_CHECK(exchange_sender.tp() == tipb::ExchangeType::Hash, ExchangeType_Name(exchange_sender.tp())); - RUNTIME_CHECK(stream_count <= 1024, stream_count); + RUNTIME_CHECK(stream_count <= maxFineGrainedStreamCount, stream_count); } pipeline.transform([&](auto & stream) { // construct writer From 08c3138ae2893242ef79a8ca4d1dfbfcc5b9f221 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Tue, 14 Mar 2023 17:47:14 +0800 Subject: [PATCH 21/24] Update Pipeline.cpp --- dbms/src/Flash/Pipeline/Pipeline.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index d690deb587c..9e48fbfb778 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -225,16 +225,16 @@ bool Pipeline::isSupported(const tipb::DAGRequest & dag_request) [&](const tipb::Executor & executor) { switch (executor.tp()) { - case tipb::ExecType::TypeProjection: - case tipb::ExecType::TypeSelection: - case tipb::ExecType::TypeLimit: - case tipb::ExecType::TypeTopN: case tipb::ExecType::TypeTableScan: if (executor.tbl_scan().keep_order()) { is_supported = false; return false; } + case tipb::ExecType::TypeProjection: + case tipb::ExecType::TypeSelection: + case tipb::ExecType::TypeLimit: + case tipb::ExecType::TypeTopN: case tipb::ExecType::TypeExchangeSender: case tipb::ExecType::TypeExchangeReceiver: case tipb::ExecType::TypeExpand: From 0c891de0b3b4e02cc6856cae4f7c15a3c4501b9a Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Tue, 14 Mar 2023 18:21:01 +0800 Subject: [PATCH 22/24] add more unit tests --- .../tests/gtest_fine_grained_shuffle.cpp | 90 +++++++++++++++++++ .../src/Flash/tests/gtest_window_executor.cpp | 2 - 2 files changed, 90 insertions(+), 2 deletions(-) create mode 100644 dbms/src/Flash/tests/gtest_fine_grained_shuffle.cpp diff --git a/dbms/src/Flash/tests/gtest_fine_grained_shuffle.cpp b/dbms/src/Flash/tests/gtest_fine_grained_shuffle.cpp new file mode 100644 index 00000000000..b405b03be79 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_fine_grained_shuffle.cpp @@ -0,0 +1,90 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include <TestUtils/ColumnGenerator.h> +#include <TestUtils/ExecutorTestUtils.h> + +namespace DB::tests +{ +class FineGrainedShuffleTestRunner : public DB::tests::ExecutorTest +{ + void initializeContext() override + { + ExecutorTest::initializeContext(); + + DB::MockColumnInfoVec column_infos{{"partition", TiDB::TP::TypeLong}, {"value", TiDB::TP::TypeLong}}; + DB::MockColumnInfoVec partition_column_infos{{"partition", TiDB::TP::TypeLong}}; + ColumnsWithTypeAndName column_data; + ColumnsWithTypeAndName common_column_data; + size_t table_rows = 1024; + for (const auto & column_info : mockColumnInfosToTiDBColumnInfos(column_infos)) + { + ColumnGeneratorOpts opts{table_rows, getDataTypeByColumnInfoForComputingLayer(column_info)->getName(), RANDOM, column_info.name}; + column_data.push_back(ColumnGenerator::instance().generate(opts)); + } + ColumnWithTypeAndName shuffle_column = ColumnGenerator::instance().generate({table_rows, "UInt64", RANDOM}); + IColumn::Permutation perm; + shuffle_column.column->getPermutation(false, 0, -1, perm); + for (auto & column : column_data) + { + column.column = column.column->permute(perm, 0); + } + + context.addExchangeReceiver("exchange_receiver_1_concurrency", column_infos, column_data, 1, partition_column_infos); + context.addExchangeReceiver("exchange_receiver_3_concurrency", column_infos, column_data, 3, partition_column_infos); + context.addExchangeReceiver("exchange_receiver_5_concurrency", column_infos, column_data, 5, partition_column_infos); + context.addExchangeReceiver("exchange_receiver_10_concurrency", column_infos, column_data, 10, partition_column_infos); + } +}; + +TEST_F(FineGrainedShuffleTestRunner, simpleReceiver) +try +{ + std::vector<size_t> exchange_receiver_concurrency = {1, 3, 5, 10}; + + auto gen_request = [&](size_t exchange_concurrency) { + return context + .receive(fmt::format("exchange_receiver_{}_concurrency", exchange_concurrency), exchange_concurrency) + .build(context); + }; + + auto baseline = executeStreams(gen_request(1), 1); + for (size_t exchange_concurrency : exchange_receiver_concurrency) + { + executeAndAssertColumnsEqual(gen_request(exchange_concurrency), baseline); + } +} +CATCH + +TEST_F(FineGrainedShuffleTestRunner, FineGrainedShuffleReceiverAndThenNonFineGrainedShuffleAgg) +try +{ + std::vector<size_t> exchange_receiver_concurrency = {1, 3, 5, 10}; + + auto gen_request = [&](size_t exchange_concurrency) { + return context + .receive(fmt::format("exchange_receiver_{}_concurrency", exchange_concurrency), exchange_concurrency) + .aggregation({Max(col("partition"))}, {col("value")}) + .build(context); + }; + + auto baseline = executeStreams(gen_request(1), 1); + for (size_t exchange_concurrency : exchange_receiver_concurrency) + { + executeAndAssertColumnsEqual(gen_request(exchange_concurrency), baseline); + } +} +CATCH + +} // namespace DB::tests diff --git a/dbms/src/Flash/tests/gtest_window_executor.cpp b/dbms/src/Flash/tests/gtest_window_executor.cpp index bfae19c24d2..220c189f7eb 100644 --- a/dbms/src/Flash/tests/gtest_window_executor.cpp +++ b/dbms/src/Flash/tests/gtest_window_executor.cpp @@ -18,8 +18,6 @@ namespace DB::tests { class WindowExecutorTestRunner : public DB::tests::ExecutorTest { - static const size_t max_concurrency_level = 10; - public: void initializeContext() override { From c65200c52cbfd5f34bc572fab03b9c688f8ec5f4 Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Tue, 14 Mar 2023 18:29:08 +0800 Subject: [PATCH 23/24] Update ExecutorTestUtils.cpp --- dbms/src/TestUtils/ExecutorTestUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index ec81dc671ca..dcce7b2c034 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -213,8 +213,8 @@ void ExecutorTest::checkBlockSorted( ASSERT_TRUE(assert_func(expected_res, res)) << testInfoMsg(request, enable_planner, enable_pipeline, concurrency, block_size); } }; - WRAP_FOR_TEST_END } + WRAP_FOR_TEST_END } void ExecutorTest::executeAndAssertColumnsEqual(const std::shared_ptr<tipb::DAGRequest> & request, const ColumnsWithTypeAndName & expect_columns) From 4ae0cc557559459d804ffd40c116d52f55b6defe Mon Sep 17 00:00:00 2001 From: SeaRise <hhssearise@foxmail.com> Date: Wed, 15 Mar 2023 23:35:22 +0800 Subject: [PATCH 24/24] address comment --- dbms/src/Flash/Pipeline/Pipeline.cpp | 117 ++++++++++-------- dbms/src/Flash/Pipeline/Pipeline.h | 15 ++- .../Events/FineGrainedPipelineEvent.cpp | 1 - .../Events/FineGrainedPipelineEvent.h | 8 +- .../Pipeline/Schedule/Events/PipelineEvent.h | 50 -------- .../Schedule/Events/PlainPipelineEvent.cpp | 7 ++ .../Schedule/Events/PlainPipelineEvent.h | 15 ++- 7 files changed, 97 insertions(+), 116 deletions(-) delete mode 100644 dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h diff --git a/dbms/src/Flash/Pipeline/Pipeline.cpp b/dbms/src/Flash/Pipeline/Pipeline.cpp index 9e48fbfb778..2684643c47a 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.cpp +++ b/dbms/src/Flash/Pipeline/Pipeline.cpp @@ -33,69 +33,86 @@ FmtBuffer & addPrefix(FmtBuffer & buffer, size_t level) { return buffer.append(String(level, ' ')); } +} // namespace + +PipelineEvents::PipelineEvents(Events && events_, bool is_fine_grained_) + : events(std::move(events_)) + , is_fine_grained(is_fine_grained_) +{ + RUNTIME_CHECK(!events.empty()); + // For non fine grained mode, the size of events must be 1. + RUNTIME_CHECK(is_fine_grained || events.size() == 1); +} -void mapEvents(const Events & inputs, const Events & outputs) +void PipelineEvents::mapInputs(const PipelineEvents & inputs) { - assert(!inputs.empty()); - assert(!outputs.empty()); - if (inputs.size() == outputs.size()) + /// The self events is output. + if (inputs.is_fine_grained && is_fine_grained) { - /** - * 1. for fine grained inputs and fine grained outputs - * ``` - * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent - * ``` - * 2. for non fine grained inputs and non fine grained outputs - * ``` - * PlainPipelineEvent◄────PlainPipelineEvent - * ``` - * 3. for non fine grained inputs and fine grained outputs - * This is not possible, if fine-grained is enabled in outputs, then inputs must also be enabled. - * Checked in `doToEvents`. - * 4. for fine grained inputs and non fine grained outputs - * ``` - * PlainPipelineEvent◄────FineGrainedPipelineEvent - * ``` - */ - size_t partition_num = inputs.size(); - for (size_t index = 0; index < partition_num; ++index) - outputs[index]->addInput(inputs[index]); + if (inputs.events.size() == events.size()) + { + /** + * 1. If the number of partitions match, use fine grained mapping here. + * ``` + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄────FineGrainedPipelineEvent + * ``` + */ + size_t partition_num = inputs.events.size(); + for (size_t index = 0; index < partition_num; ++index) + events[index]->addInput(inputs.events[index]); + } + else + { + /** + * 2. If the number of partitions does not match, it is safer to use full mapping. + * ``` + * FineGrainedPipelineEvent◄──┐ ┌──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┼─┼──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┤ └──FineGrainedPipelineEvent + * FineGrainedPipelineEvent◄──┘ + * ``` + */ + for (const auto & output : events) + { + for (const auto & input : inputs.events) + output->addInput(input); + } + } } else { /** - * 1. for fine grained inputs and fine grained outputs - * If the number of partitions does not match, it is safer to use full mapping. + * Use full mapping here. + * 1. for non fine grained inputs and non fine grained outputs + * The size of inputs and outputs must be the same and 1. * ``` - * FineGrainedPipelineEvent◄──┐ ┌──FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄──┼─┼──FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄──┤ └──FineGrainedPipelineEvent - * FineGrainedPipelineEvent◄──┘ + * PlainPipelineEvent◄────PlainPipelineEvent * ``` - * 2. for non fine grained inputs and non fine grained outputs - * This is not possible, the size of inputs and outputs must be the same and 1. - * 3. for non fine grained inputs and fine grained outputs + * 2. for non fine grained inputs and fine grained outputs * This is not possible, if fine-grained is enabled in outputs, then inputs must also be enabled. - * Checked in `doToEvents`. - * 4. for fine grained inputs and non fine grained outputs + * 3. for fine grained inputs and non fine grained outputs * ``` * ┌──FineGrainedPipelineEvent * PlainPipelineEvent◄──┼──FineGrainedPipelineEvent * ├──FineGrainedPipelineEvent * └──FineGrainedPipelineEvent + * + * PlainPipelineEvent◄────FineGrainedPipelineEvent * ``` */ - for (const auto & output : outputs) + + // If the outputs is fine grained model, the intputs must also be. + RUNTIME_CHECK(inputs.is_fine_grained || !is_fine_grained); + for (const auto & output : events) { - for (const auto & input : inputs) + for (const auto & input : inputs.events) output->addInput(input); } } } -} // namespace void Pipeline::addPlanNode(const PhysicalPlanNodePtr & plan_node) { @@ -182,7 +199,7 @@ Events Pipeline::toEvents(PipelineExecutorStatus & status, Context & context, si return all_events; } -Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency) +PipelineEvents Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency) { auto memory_tracker = current_memory_tracker ? current_memory_tracker->shared_from_this() : nullptr; Events self_events; @@ -190,9 +207,8 @@ Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context if (isFineGrainedMode()) { auto fine_grained_exec_group = buildExecGroup(status, context, concurrency); - RUNTIME_CHECK(!fine_grained_exec_group.empty()); for (auto & pipeline_exec : fine_grained_exec_group) - self_events.push_back(std::make_shared<FineGrainedPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), std::move(pipeline_exec))); + self_events.push_back(std::make_shared<FineGrainedPipelineEvent>(status, memory_tracker, log->identifier(), std::move(pipeline_exec))); LOG_DEBUG(log, "Execute in fine grained model and generate {} fine grained pipeline event", self_events.size()); } else @@ -200,20 +216,15 @@ Events Pipeline::toSelfEvents(PipelineExecutorStatus & status, Context & context self_events.push_back(std::make_shared<PlainPipelineEvent>(status, memory_tracker, log->identifier(), context, shared_from_this(), concurrency)); LOG_DEBUG(log, "Execute in non fine grained model and generate one plain pipeline event"); } - return self_events; + return {std::move(self_events), isFineGrainedMode()}; } -Events Pipeline::doToEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events) +PipelineEvents Pipeline::doToEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events) { auto self_events = toSelfEvents(status, context, concurrency); for (const auto & child : children) - { - // If the current pipeline is fine grained model, the child must also be. - RUNTIME_CHECK(!isFineGrainedMode() || child->isFineGrainedMode()); - auto inputs = child->doToEvents(status, context, concurrency, all_events); - mapEvents(inputs, self_events); - } - all_events.insert(all_events.end(), self_events.cbegin(), self_events.cend()); + self_events.mapInputs(child->doToEvents(status, context, concurrency, all_events)); + all_events.insert(all_events.end(), self_events.events.cbegin(), self_events.events.cend()); return self_events; } diff --git a/dbms/src/Flash/Pipeline/Pipeline.h b/dbms/src/Flash/Pipeline/Pipeline.h index 5e248ad1b28..2f9a4a394bf 100644 --- a/dbms/src/Flash/Pipeline/Pipeline.h +++ b/dbms/src/Flash/Pipeline/Pipeline.h @@ -40,8 +40,15 @@ using PhysicalPlanNodePtr = std::shared_ptr<PhysicalPlanNode>; class PipelineExecutorStatus; -class SharedQueue; -using SharedQueuePtr = std::shared_ptr<SharedQueue>; +struct PipelineEvents +{ + Events events; + bool is_fine_grained; + + PipelineEvents(Events && events_, bool is_fine_grained_); + + void mapInputs(const PipelineEvents & inputs); +}; class Pipeline : public std::enable_shared_from_this<Pipeline> { @@ -73,8 +80,8 @@ class Pipeline : public std::enable_shared_from_this<Pipeline> private: void toSelfString(FmtBuffer & buffer, size_t level) const; - Events toSelfEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency); - Events doToEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events); + PipelineEvents toSelfEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency); + PipelineEvents doToEvents(PipelineExecutorStatus & status, Context & context, size_t concurrency, Events & all_events); private: const UInt32 id; diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp index 368a73a84d1..3cd63cf6c59 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.cpp @@ -20,7 +20,6 @@ namespace DB std::vector<TaskPtr> FineGrainedPipelineEvent::scheduleImpl() { std::vector<TaskPtr> tasks; - tasks.reserve(1); tasks.push_back(std::make_unique<PipelineTask>(mem_tracker, log->identifier(), exec_status, shared_from_this(), std::move(pipeline_exec))); return tasks; } diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h index 545b21be7e4..0c85ec024a4 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h +++ b/dbms/src/Flash/Pipeline/Schedule/Events/FineGrainedPipelineEvent.h @@ -15,21 +15,19 @@ #pragma once #include <Flash/Pipeline/Exec/PipelineExec.h> -#include <Flash/Pipeline/Schedule/Events/PipelineEvent.h> +#include <Flash/Pipeline/Schedule/Events/Event.h> namespace DB { -class FineGrainedPipelineEvent : public PipelineEvent +class FineGrainedPipelineEvent : public Event { public: FineGrainedPipelineEvent( PipelineExecutorStatus & exec_status_, MemoryTrackerPtr mem_tracker_, const String & req_id, - Context & context_, - const PipelinePtr & pipeline_, PipelineExecPtr && pipeline_exec_) - : PipelineEvent(exec_status_, std::move(mem_tracker_), req_id, context_, pipeline_) + : Event(exec_status_, std::move(mem_tracker_), req_id) , pipeline_exec(std::move(pipeline_exec_)) { assert(pipeline_exec); diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h deleted file mode 100644 index d3631bbd3b5..00000000000 --- a/dbms/src/Flash/Pipeline/Schedule/Events/PipelineEvent.h +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright 2023 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include <Flash/Pipeline/Schedule/Events/Event.h> - -namespace DB -{ -class Pipeline; -using PipelinePtr = std::shared_ptr<Pipeline>; - -// The base class of pipeline related event. -class PipelineEvent : public Event -{ -public: - PipelineEvent( - PipelineExecutorStatus & exec_status_, - MemoryTrackerPtr mem_tracker_, - const String & req_id, - Context & context_, - const PipelinePtr & pipeline_) - : Event(exec_status_, std::move(mem_tracker_), req_id) - , context(context_) - , pipeline(pipeline_) - {} - - void finishImpl() override - { - // Plan nodes in pipeline hold resources like hash table for join, when destruction they will operate memory tracker in MPP task. But MPP task may get destructed once `exec_status.onEventFinish()` is called. - // So pipeline needs to be released before `exec_status.onEventFinish()` is called. - pipeline.reset(); - } - -protected: - Context & context; - PipelinePtr pipeline; -}; -} // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp index 4c25c6674db..5485cf59be6 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp +++ b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.cpp @@ -29,4 +29,11 @@ std::vector<TaskPtr> PlainPipelineEvent::scheduleImpl() tasks.push_back(std::make_unique<PipelineTask>(mem_tracker, log->identifier(), exec_status, shared_from_this(), std::move(pipeline_exec))); return tasks; } + +void PlainPipelineEvent::finishImpl() +{ + // Plan nodes in pipeline hold resources like hash table for join, when destruction they will operate memory tracker in MPP task. But MPP task may get destructed once `exec_status.onEventFinish()` is called. + // So pipeline needs to be released before `exec_status.onEventFinish()` is called. + pipeline.reset(); +} } // namespace DB diff --git a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h index e0b4bd9fe05..6df1c425ae6 100644 --- a/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h +++ b/dbms/src/Flash/Pipeline/Schedule/Events/PlainPipelineEvent.h @@ -14,11 +14,14 @@ #pragma once -#include <Flash/Pipeline/Schedule/Events/PipelineEvent.h> +#include <Flash/Pipeline/Schedule/Events/Event.h> namespace DB { -class PlainPipelineEvent : public PipelineEvent +class Pipeline; +using PipelinePtr = std::shared_ptr<Pipeline>; + +class PlainPipelineEvent : public Event { public: PlainPipelineEvent( @@ -28,14 +31,20 @@ class PlainPipelineEvent : public PipelineEvent Context & context_, const PipelinePtr & pipeline_, size_t concurrency_) - : PipelineEvent(exec_status_, std::move(mem_tracker_), req_id, context_, pipeline_) + : Event(exec_status_, std::move(mem_tracker_), req_id) + , context(context_) + , pipeline(pipeline_) , concurrency(concurrency_) {} protected: std::vector<TaskPtr> scheduleImpl() override; + void finishImpl() override; + private: + Context & context; + PipelinePtr pipeline; size_t concurrency; }; } // namespace DB