From d987c89865369d0575fae9bc7eb3507cf95229f6 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 25 Apr 2022 21:51:01 +0800 Subject: [PATCH 01/57] introd physical plan --- dbms/src/Flash/CMakeLists.txt | 2 + dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 25 +++-- dbms/src/Flash/Planner/FinalizeHelper.cpp | 95 +++++++++++++++++++ dbms/src/Flash/Planner/FinalizeHelper.h | 20 ++++ dbms/src/Flash/Planner/PhysicalPlan.cpp | 62 ++++++++++++ dbms/src/Flash/Planner/PhysicalPlan.h | 84 ++++++++++++++++ dbms/src/Flash/Planner/PhysicalPlanHelper.cpp | 13 +++ dbms/src/Flash/Planner/PhysicalPlanHelper.h | 8 ++ dbms/src/Flash/Planner/PlanType.cpp | 42 ++++++++ dbms/src/Flash/Planner/PlanType.h | 33 +++++++ dbms/src/Flash/Planner/Planner.cpp | 67 +++++++++++++ dbms/src/Flash/Planner/Planner.h | 60 ++++++++++++ dbms/src/Flash/Planner/plans/PhysicalUnary.h | 48 ++++++++++ dbms/src/Interpreters/Settings.h | 14 +-- 14 files changed, 560 insertions(+), 13 deletions(-) create mode 100644 dbms/src/Flash/Planner/FinalizeHelper.cpp create mode 100644 dbms/src/Flash/Planner/FinalizeHelper.h create mode 100644 dbms/src/Flash/Planner/PhysicalPlan.cpp create mode 100644 dbms/src/Flash/Planner/PhysicalPlan.h create mode 100644 dbms/src/Flash/Planner/PhysicalPlanHelper.cpp create mode 100644 dbms/src/Flash/Planner/PhysicalPlanHelper.h create mode 100644 dbms/src/Flash/Planner/PlanType.cpp create mode 100644 dbms/src/Flash/Planner/PlanType.h create mode 100644 dbms/src/Flash/Planner/Planner.cpp create mode 100644 dbms/src/Flash/Planner/Planner.h create mode 100644 dbms/src/Flash/Planner/plans/PhysicalUnary.h diff --git a/dbms/src/Flash/CMakeLists.txt b/dbms/src/Flash/CMakeLists.txt index 1b9e3e0aaf8..1d3aa288f29 100644 --- a/dbms/src/Flash/CMakeLists.txt +++ b/dbms/src/Flash/CMakeLists.txt @@ -17,6 +17,8 @@ include(${TiFlash_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(flash_service .) add_headers_and_sources(flash_service ./Coprocessor) add_headers_and_sources(flash_service ./Mpp) +add_headers_and_sources(flash_service ./Planner) +add_headers_and_sources(flash_service ./Planner/plans) add_headers_and_sources(flash_service ./Statistics) add_library(flash_service ${flash_service_headers} ${flash_service_sources}) diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 6b118f1dd40..fb247e3ad2a 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include namespace DB @@ -62,12 +63,24 @@ BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block) BlockInputStreams child_streams = executeQueryBlock(*child); input_streams_vec.push_back(child_streams); } - DAGQueryBlockInterpreter query_block_interpreter( - context, - input_streams_vec, - query_block, - max_streams); - return query_block_interpreter.execute(); + if (context.getSettingsRef().enable_planner && Planner::isSupported(query_block)) + { + Planner planner( + context, + input_streams_vec, + query_block, + max_streams); + return planner.execute(); + } + else + { + DAGQueryBlockInterpreter query_block_interpreter( + context, + input_streams_vec, + query_block, + max_streams); + return query_block_interpreter.execute(); + } } BlockIO InterpreterDAG::execute() diff --git a/dbms/src/Flash/Planner/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp new file mode 100644 index 00000000000..ad80b29e800 --- /dev/null +++ b/dbms/src/Flash/Planner/FinalizeHelper.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include +#include + +#include + +namespace DB::FinalizeHelper +{ +void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous) +{ + if (!actions->getRequiredColumnsWithTypes().empty() + && columns_from_previous > actions->getRequiredColumnsWithTypes().size()) + { + actions->prependProjectInput(); + } +} + +void checkSchemaContainsParentRequire(const NamesAndTypes & schema, const Names & parent_require) +{ + NameSet schema_set; + for (const auto & column : schema) + schema_set.insert(column.name); + for (const auto & parent_require_column : parent_require) + { + if (unlikely(schema_set.find(parent_require_column) == schema_set.end())) + throw TiFlashException( + fmt::format("schema don't contain parent require column: {}", parent_require_column), + Errors::Coprocessor::Internal); + } +} + +void checkParentRequireContainsSchema(const Names & parent_require, const NamesAndTypes & schema) +{ + NameSet parent_require_set; + for (const auto & parent_require_column : parent_require) + parent_require_set.insert(parent_require_column); + for (const auto & schema_column : schema) + { + if (unlikely(parent_require_set.find(schema_column.name) == parent_require_set.end())) + throw TiFlashException( + fmt::format("parent require don't contain schema column: {}", schema_column.name), + Errors::Coprocessor::Internal); + } +} + +void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema) +{ + for (const auto & schema_column : schema) + { + if (unlikely(!sample_block.has(schema_column.name))) + throw TiFlashException( + fmt::format("sample block don't contain schema column: {}", schema_column.name), + Errors::Coprocessor::Internal); + + const auto & type_in_sample_block = sample_block.getByName(schema_column.name).type->getName(); + const auto & type_in_schema = schema_column.type->getName(); + if (unlikely(type_in_sample_block != type_in_schema)) + throw TiFlashException( + fmt::format( + "the type of column `{}` in sample block `{}` is difference from the one in schema `{}`", + schema_column.name, + type_in_sample_block, + type_in_schema), + Errors::Coprocessor::Internal); + } +} + +void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & sample_block) +{ + std::unordered_map schema_map; + for (const auto & column : schema) + schema_map[column.name] = column.type; + for (const auto & sample_block_column : sample_block) + { + auto it = schema_map.find(sample_block_column.name); + if (unlikely(it == schema_map.end())) + throw TiFlashException( + fmt::format("schema don't contain sample block column: {}", sample_block_column.name), + Errors::Coprocessor::Internal); + + const auto & type_in_schema = it->second->getName(); + const auto & type_in_sample_block = sample_block_column.type->getName(); + if (unlikely(type_in_sample_block != type_in_schema)) + throw TiFlashException( + fmt::format( + "the type of column `{}` in schema `{}` is difference from the one in sample block `{}`", + sample_block_column.name, + type_in_schema, + type_in_sample_block), + Errors::Coprocessor::Internal); + } +} +} // namespace DB::FinalizeHelper diff --git a/dbms/src/Flash/Planner/FinalizeHelper.h b/dbms/src/Flash/Planner/FinalizeHelper.h new file mode 100644 index 00000000000..26fc0ffdb11 --- /dev/null +++ b/dbms/src/Flash/Planner/FinalizeHelper.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB::FinalizeHelper +{ +void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous); + +void checkSchemaContainsParentRequire(const NamesAndTypes & schema, const Names & parent_require); + +void checkParentRequireContainsSchema(const Names & parent_require, const NamesAndTypes & schema); + +void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema); + +void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & sample_block); +} // namespace DB::FinalizeHelper diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp new file mode 100644 index 00000000000..1d4f95f3716 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -0,0 +1,62 @@ +// 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 +#include +#include +#include +#include +#include + +namespace DB +{ +String PhysicalPlan::toString() +{ + auto schema_to_string = [&]() { + FmtBuffer buffer; + buffer.joinStr( + schema.cbegin(), + schema.cend(), + [](const auto & item, FmtBuffer & buf) { buf.fmtAppend("<{}, {}>", item.name, item.type->getName()); }, + ", "); + return buffer.toString(); + }; + return fmt::format( + "type: {}, executor_id: {}, is_record_profile_streams: {}, schema: {}", + DB::toString(type), + executor_id, + is_record_profile_streams, + schema_to_string()); +} + +void PhysicalPlan::finalize() +{ + finalize(PhysicalPlanHelper::schemaToNames(schema)); +} + +void PhysicalPlan::recordProfileStreams(DAGPipeline & pipeline, const Context & context) +{ + if (is_record_profile_streams) + { + auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[executor_id]; + pipeline.transform([&profile_streams](auto & stream) { profile_streams.push_back(stream); }); + } +} + +void PhysicalPlan::transform(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + transformImpl(pipeline, context, max_streams); + recordProfileStreams(pipeline, context); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h new file mode 100644 index 00000000000..49959cc10ad --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -0,0 +1,84 @@ +// 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. + +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +struct DAGPipeline; +class Context; +class DAGContext; + +class PhysicalPlan; +using PhysicalPlanPtr = std::shared_ptr; + +class PhysicalPlan +{ +public: + PhysicalPlan(const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, const String & req_id) + : executor_id(executor_id_) + , type(type_) + , schema(schema_) + , log(DB::toString(type_), req_id) + {} + + virtual ~PhysicalPlan() = default; + + virtual PhysicalPlanPtr children(size_t /*i*/) const = 0; + + virtual void setChild(size_t /*i*/, const PhysicalPlanPtr & /*new_child*/) = 0; + + const PlanType & tp() const { return type; } + + const String & execId() const { return executor_id; } + + const NamesAndTypes & getSchema() const { return schema; } + + virtual void appendChild(const PhysicalPlanPtr & /*new_child*/) = 0; + + virtual size_t childrenSize() const = 0; + + virtual void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); + + virtual void finalize(const Names & parent_require) = 0; + void finalize(); + + /// Obtain a sample block that contains the names and types of result columns. + virtual const Block & getSampleBlock() const = 0; + + void disableRecordProfileStreams() { is_record_profile_streams = false; } + + String toString(); + +protected: + virtual void transformImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/){}; + + void recordProfileStreams(DAGPipeline & pipeline, const Context & context); + + String executor_id; + PlanType type; + NamesAndTypes schema; + bool is_record_profile_streams = true; + + Logger log; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp new file mode 100644 index 00000000000..9d1fc4fee2c --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp @@ -0,0 +1,13 @@ +#include + +namespace DB::PhysicalPlanHelper +{ +Names schemaToNames(const NamesAndTypes & schema) +{ + Names names; + names.reserve(schema.size()); + for (const auto & column : schema) + names.push_back(column.name); + return names; +} +} // namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.h b/dbms/src/Flash/Planner/PhysicalPlanHelper.h new file mode 100644 index 00000000000..22ccf876eac --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.h @@ -0,0 +1,8 @@ +#pragma once + +#include + +namespace DB::PhysicalPlanHelper +{ +Names schemaToNames(const NamesAndTypes & schema); +} // namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp new file mode 100644 index 00000000000..0b733fd74e2 --- /dev/null +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -0,0 +1,42 @@ +// 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 +#include + +namespace DB +{ +String toString(const PlanType & plan_type) +{ + switch (plan_type) + { + case Aggregation: + return "Aggregation"; + case ExchangeSender: + return "ExchangeSender"; + case Limit: + return "Limit"; + case Projection: + return "Projection"; + case Selection: + return "Selection"; + case Source: + return "Source"; + case TopN: + return "TopN"; + default: + throw Exception("Unknown PlanType"); + } +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h new file mode 100644 index 00000000000..e9ab9cb6f67 --- /dev/null +++ b/dbms/src/Flash/Planner/PlanType.h @@ -0,0 +1,33 @@ +// 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. + +#pragma once + +#include + +namespace DB +{ +enum PlanType +{ + Aggregation, + ExchangeSender, + Limit, + Projection, + Selection, + Source, + TopN, +}; + +String toString(const PlanType & plan_type); +} // namespace DB diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp new file mode 100644 index 00000000000..cd597f33998 --- /dev/null +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -0,0 +1,67 @@ +// 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 +#include +#include +#include +#include +#include + +namespace DB +{ +Planner::Planner( + Context & context_, + const std::vector & input_streams_vec_, + const DAGQueryBlock & query_block_, + size_t max_streams_) + : context(context_) + , input_streams_vec(input_streams_vec_) + , query_block(query_block_) + , max_streams(max_streams_) + , log(Logger::get("Planner", dagContext().log ? dagContext().log->identifier() : "")) +{} + +BlockInputStreams Planner::execute() +{ + DAGPipeline pipeline; + executeImpl(pipeline); + if (!pipeline.streams_with_non_joined_data.empty()) + { + executeUnion(pipeline, max_streams, log); + restorePipelineConcurrency(pipeline); + } + return pipeline.streams; +} + +bool Planner::isSupported(const DAGQueryBlock & query_block) +{ + return false; +} + +DAGContext & Planner::dagContext() const +{ + return *context.getDAGContext(); +} + +void Planner::restorePipelineConcurrency(DAGPipeline & pipeline) +{ + if (query_block.can_restore_pipeline_concurrency) + restoreConcurrency(pipeline, dagContext().final_concurrency, log); +} + +void Planner::executeImpl(DAGPipeline & pipeline) +{ +} +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/Planner.h b/dbms/src/Flash/Planner/Planner.h new file mode 100644 index 00000000000..7abe258a1aa --- /dev/null +++ b/dbms/src/Flash/Planner/Planner.h @@ -0,0 +1,60 @@ +// 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. + +#pragma once + +#include +#include +#include + +namespace DB +{ +class Context; +class DAGContext; + +class Planner +{ +public: + Planner( + Context & context_, + const std::vector & input_streams_vec_, + const DAGQueryBlock & query_block_, + size_t max_streams_); + + ~Planner() = default; + + BlockInputStreams execute(); + + static bool isSupported(const DAGQueryBlock & query_block); + +private: + DAGContext & dagContext() const; + + void restorePipelineConcurrency(DAGPipeline & pipeline); + + void executeImpl(DAGPipeline & pipeline); + +private: + Context & context; + + std::vector input_streams_vec; + + const DAGQueryBlock & query_block; + + /// How many streams we ask for storage to produce, and in how many threads we will do further processing. + size_t max_streams = 1; + + LoggerPtr log; +}; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/plans/PhysicalUnary.h new file mode 100644 index 00000000000..2b978b0f743 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalUnary.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +/** + * A physical plan node with single child. + */ +class PhysicalUnary : public PhysicalPlan +{ +public: + PhysicalUnary(const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, const String & req_id) + : PhysicalPlan(executor_id_, type_, schema_, req_id) + {} + + PhysicalPlanPtr children(size_t i) const override + { + RUNTIME_ASSERT(i == 0, log, fmt::format("child_index({}) should not >= childrenSize({})", i, childrenSize())); + assert(child); + return child; + } + + void setChild(size_t i, const PhysicalPlanPtr & new_child) override + { + RUNTIME_ASSERT(i == 0, log, fmt::format("child_index({}) should not >= childrenSize({})", i, childrenSize())); + assert(new_child); + assert(new_child.get() != this); + child = new_child; + } + + void appendChild(const PhysicalPlanPtr & new_child) override + { + RUNTIME_ASSERT(!child, log, fmt::format("the actual children size had be the max size({}), don't append child again", childrenSize())); + assert(new_child); + assert(new_child.get() != this); + child = new_child; + } + + size_t childrenSize() const override { return 1; }; + +protected: + PhysicalPlanPtr child; +}; +} // namespace DB diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 6adecea4d60..70503b8d857 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -209,7 +209,7 @@ struct Settings * Basically, limits are checked for each block (not every row). That is, the limits can be slightly violated. \ * Almost all limits apply only to SELECTs. \ * Almost all limits apply to each stream individually. \ - */ \ + */ \ \ M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it " \ "is only checked on a remote server.") \ @@ -272,8 +272,8 @@ struct Settings M(SettingUInt64, dt_segment_delta_small_column_file_size, 8388608, "Determine whether a column file in delta is small or not. 8MB by default.") \ M(SettingUInt64, dt_segment_stable_pack_rows, DEFAULT_MERGE_BLOCK_SIZE, "Expected stable pack rows in DeltaTree Engine.") \ M(SettingFloat, dt_segment_wait_duration_factor, 1, "The factor of wait duration in a write stall.") \ - M(SettingUInt64, dt_bg_gc_check_interval, 5, "Background gc thread check interval, the unit is second.") \ - M(SettingInt64, dt_bg_gc_max_segments_to_check_every_round, 100, "Max segments to check in every gc round, value less than or equal to 0 means gc no segments.") \ + M(SettingUInt64, dt_bg_gc_check_interval, 5, "Background gc thread check interval, the unit is second.") \ + M(SettingInt64, dt_bg_gc_max_segments_to_check_every_round, 100, "Max segments to check in every gc round, value less than or equal to 0 means gc no segments.") \ M(SettingFloat, dt_bg_gc_ratio_threhold_to_trigger_gc, 1.2, "Trigger segment's gc when the ratio of invalid version exceed this threhold. Values smaller than or equal to 1.0 means gc all " \ "segments") \ M(SettingFloat, dt_bg_gc_delta_delete_ratio_to_trigger_gc, 0.3, "Trigger segment's gc when the ratio of delta delete range to stable exceeds this ratio.") \ @@ -350,13 +350,13 @@ struct Settings M(SettingUInt64, elastic_threadpool_init_cap, 400, "The size of elastic thread pool.") \ M(SettingUInt64, elastic_threadpool_shrink_period_ms, 300000, "The shrink period(ms) of elastic thread pool.") \ M(SettingBool, enable_local_tunnel, true, "Enable local data transfer between local MPP tasks.") \ - M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ - M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.")\ + M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ + M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.") \ M(SettingBool, enable_async_server, true, "Enable async rpc server.") \ M(SettingUInt64, async_pollers_per_cq, 200, "grpc async pollers per cqs") \ M(SettingUInt64, async_cqs, 1, "grpc async cqs") \ - M(SettingUInt64, preallocated_request_count_per_poller, 20, "grpc preallocated_request_count_per_poller") - + M(SettingUInt64, preallocated_request_count_per_poller, 20, "grpc preallocated_request_count_per_poller") \ + M(SettingBool, enable_planner, true, "Enable planner") // clang-format on #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) TYPE NAME{DEFAULT}; From e24489acad2425f02a402fd9b5880e4c5749f1be Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 25 Apr 2022 22:51:17 +0800 Subject: [PATCH 02/57] add source --- dbms/src/Flash/Planner/FinalizeHelper.cpp | 95 ------------------- dbms/src/Flash/Planner/FinalizeHelper.h | 20 ---- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 32 +++++++ dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 47 +++++++++ dbms/src/Flash/Planner/PlanType.cpp | 12 --- dbms/src/Flash/Planner/PlanType.h | 6 -- dbms/src/Flash/Planner/Planner.cpp | 14 ++- dbms/src/Flash/Planner/plans/PhysicalLeaf.h | 49 ++++++++++ dbms/src/Flash/Planner/plans/PhysicalSource.h | 55 +++++++++++ dbms/src/Flash/Planner/plans/PhysicalUnary.h | 14 +++ 10 files changed, 209 insertions(+), 135 deletions(-) delete mode 100644 dbms/src/Flash/Planner/FinalizeHelper.cpp delete mode 100644 dbms/src/Flash/Planner/FinalizeHelper.h create mode 100644 dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp create mode 100644 dbms/src/Flash/Planner/PhysicalPlanBuilder.h create mode 100644 dbms/src/Flash/Planner/plans/PhysicalLeaf.h create mode 100644 dbms/src/Flash/Planner/plans/PhysicalSource.h diff --git a/dbms/src/Flash/Planner/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp deleted file mode 100644 index ad80b29e800..00000000000 --- a/dbms/src/Flash/Planner/FinalizeHelper.cpp +++ /dev/null @@ -1,95 +0,0 @@ -#include -#include -#include -#include -#include - -#include - -namespace DB::FinalizeHelper -{ -void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous) -{ - if (!actions->getRequiredColumnsWithTypes().empty() - && columns_from_previous > actions->getRequiredColumnsWithTypes().size()) - { - actions->prependProjectInput(); - } -} - -void checkSchemaContainsParentRequire(const NamesAndTypes & schema, const Names & parent_require) -{ - NameSet schema_set; - for (const auto & column : schema) - schema_set.insert(column.name); - for (const auto & parent_require_column : parent_require) - { - if (unlikely(schema_set.find(parent_require_column) == schema_set.end())) - throw TiFlashException( - fmt::format("schema don't contain parent require column: {}", parent_require_column), - Errors::Coprocessor::Internal); - } -} - -void checkParentRequireContainsSchema(const Names & parent_require, const NamesAndTypes & schema) -{ - NameSet parent_require_set; - for (const auto & parent_require_column : parent_require) - parent_require_set.insert(parent_require_column); - for (const auto & schema_column : schema) - { - if (unlikely(parent_require_set.find(schema_column.name) == parent_require_set.end())) - throw TiFlashException( - fmt::format("parent require don't contain schema column: {}", schema_column.name), - Errors::Coprocessor::Internal); - } -} - -void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema) -{ - for (const auto & schema_column : schema) - { - if (unlikely(!sample_block.has(schema_column.name))) - throw TiFlashException( - fmt::format("sample block don't contain schema column: {}", schema_column.name), - Errors::Coprocessor::Internal); - - const auto & type_in_sample_block = sample_block.getByName(schema_column.name).type->getName(); - const auto & type_in_schema = schema_column.type->getName(); - if (unlikely(type_in_sample_block != type_in_schema)) - throw TiFlashException( - fmt::format( - "the type of column `{}` in sample block `{}` is difference from the one in schema `{}`", - schema_column.name, - type_in_sample_block, - type_in_schema), - Errors::Coprocessor::Internal); - } -} - -void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & sample_block) -{ - std::unordered_map schema_map; - for (const auto & column : schema) - schema_map[column.name] = column.type; - for (const auto & sample_block_column : sample_block) - { - auto it = schema_map.find(sample_block_column.name); - if (unlikely(it == schema_map.end())) - throw TiFlashException( - fmt::format("schema don't contain sample block column: {}", sample_block_column.name), - Errors::Coprocessor::Internal); - - const auto & type_in_schema = it->second->getName(); - const auto & type_in_sample_block = sample_block_column.type->getName(); - if (unlikely(type_in_sample_block != type_in_schema)) - throw TiFlashException( - fmt::format( - "the type of column `{}` in schema `{}` is difference from the one in sample block `{}`", - sample_block_column.name, - type_in_schema, - type_in_sample_block), - Errors::Coprocessor::Internal); - } -} -} // namespace DB::FinalizeHelper diff --git a/dbms/src/Flash/Planner/FinalizeHelper.h b/dbms/src/Flash/Planner/FinalizeHelper.h deleted file mode 100644 index 26fc0ffdb11..00000000000 --- a/dbms/src/Flash/Planner/FinalizeHelper.h +++ /dev/null @@ -1,20 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB::FinalizeHelper -{ -void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous); - -void checkSchemaContainsParentRequire(const NamesAndTypes & schema, const Names & parent_require); - -void checkParentRequireContainsSchema(const Names & parent_require, const NamesAndTypes & schema); - -void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema); - -void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & sample_block); -} // namespace DB::FinalizeHelper diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp new file mode 100644 index 00000000000..d8ef2e7f0c5 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -0,0 +1,32 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace +{ +PhysicalPlanPtr popBack(std::vector vec) +{ + assert(!vec.empty()); + PhysicalPlanPtr back = vec.back(); + vec.pop_back(); + return back; +} +} // namespace + +void PhysicalPlanBuilder::buildSource(const Block & sample_block) +{ + cur_plans.push_back(PhysicalSource::build(source_sample_block, log.identifier())); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h new file mode 100644 index 00000000000..0731f3de887 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -0,0 +1,47 @@ +// 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. + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class PhysicalPlanBuilder +{ +public: + explicit PhysicalPlanBuilder(Context & context_, const String & req_id) + : context(context_) + , log("PhysicalPlanBuilder", req_id) + {} + + void buildSource(const Block & sample_block); + + PhysicalPlanPtr getResult() const + { + RUNTIME_ASSERT(cur_plans.size() == 1, log, "There can only be one plan output, but here are {}", cur_plans.size()); + return cur_plans.back(); + } + +private: + std::vector cur_plans; + + Context & context; + + Logger log; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index 0b733fd74e2..b766a890455 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -21,20 +21,8 @@ String toString(const PlanType & plan_type) { switch (plan_type) { - case Aggregation: - return "Aggregation"; - case ExchangeSender: - return "ExchangeSender"; - case Limit: - return "Limit"; - case Projection: - return "Projection"; - case Selection: - return "Selection"; case Source: return "Source"; - case TopN: - return "TopN"; default: throw Exception("Unknown PlanType"); } diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index e9ab9cb6f67..47bd1d8316d 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -20,13 +20,7 @@ namespace DB { enum PlanType { - Aggregation, - ExchangeSender, - Limit, - Projection, - Selection, Source, - TopN, }; String toString(const PlanType & plan_type); diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index cd597f33998..ef0e3f4aeeb 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -14,8 +14,8 @@ #include #include -#include #include +#include #include #include @@ -45,7 +45,7 @@ BlockInputStreams Planner::execute() return pipeline.streams; } -bool Planner::isSupported(const DAGQueryBlock & query_block) +bool Planner::isSupported(const DAGQueryBlock &) { return false; } @@ -63,5 +63,15 @@ void Planner::restorePipelineConcurrency(DAGPipeline & pipeline) void Planner::executeImpl(DAGPipeline & pipeline) { + PhysicalPlanBuilder builder{context, log->identifier()}; + for (const auto & input_streams : input_streams_vec) + { + assert(!input_streams.empty()); + builder.buildSource(input_streams.back()->getHeader()); + } + + auto physical_plan = builder.getResult(); + physical_plan->finalize(); + physical_plan->transform(pipeline, context, max_streams); } } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h new file mode 100644 index 00000000000..038c0316f20 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h @@ -0,0 +1,49 @@ +// 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. + +#pragma once + +#include +#include + +namespace DB +{ +/** + * A physical plan node with no children. + */ +class PhysicalLeaf : public PhysicalPlan +{ +public: + PhysicalLeaf(const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, const String & req_id) + : PhysicalPlan(executor_id_, type_, schema_, req_id) + {} + + PhysicalPlanPtr children(size_t) const override + { + throw Exception("the children size of PhysicalLeaf is zero"); + } + + void setChild(size_t, const PhysicalPlanPtr &) override + { + throw Exception("the children size of PhysicalLeaf is zero"); + } + + void appendChild(const PhysicalPlanPtr &) override + { + throw Exception("the children size of PhysicalLeaf is zero"); + } + + size_t childrenSize() const override { return 0; }; +}; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.h b/dbms/src/Flash/Planner/plans/PhysicalSource.h new file mode 100644 index 00000000000..6b6837de107 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.h @@ -0,0 +1,55 @@ +// 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. + +#pragma once + +#include +#include + +namespace DB +{ +class PhysicalSource : public PhysicalLeaf +{ +public: + static PhysicalPlanPtr build( + const Block & sample_block, + const String & req_id) + { + NamesAndTypes schema; + for (const auto & col : sample_block) + schema.emplace_back(col.name, col.type); + return std::make_shared("source", schema, sample_block, req_id); + } + + PhysicalSource( + const String & executor_id_, + const NamesAndTypes & schema_, + const Block & sample_block_, + const String & req_id) + : PhysicalLeaf(executor_id_, PlanType::Source, schema_, req_id) + , sample_block(sample_block_) + { + is_record_profile_streams = false; + } + + void transformImpl(DAGPipeline &, Context &, size_t) override {} + + void finalize(const Names &) override {} + + const Block & getSampleBlock() const override { return sample_block; } + +private: + Block sample_block; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/plans/PhysicalUnary.h index 2b978b0f743..a5fab90ddbc 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalUnary.h +++ b/dbms/src/Flash/Planner/plans/PhysicalUnary.h @@ -1,3 +1,17 @@ +// 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. + #pragma once #include From 99c6a74b8a8688fa4ea6be593739cc850a690ca0 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 25 Apr 2022 22:59:56 +0800 Subject: [PATCH 03/57] fix --- dbms/src/Flash/Planner/PhysicalPlan.h | 4 ++-- dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp | 11 ----------- dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 4 ++-- 3 files changed, 4 insertions(+), 15 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 49959cc10ad..6e07f7aeefa 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -38,7 +38,7 @@ class PhysicalPlan : executor_id(executor_id_) , type(type_) , schema(schema_) - , log(DB::toString(type_), req_id) + , log(Logger::get(DB::toString(type_), req_id)) {} virtual ~PhysicalPlan() = default; @@ -79,6 +79,6 @@ class PhysicalPlan NamesAndTypes schema; bool is_record_profile_streams = true; - Logger log; + LoggerPtr log; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index d8ef2e7f0c5..a2cd2ee9839 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -14,17 +14,6 @@ namespace DB { -namespace -{ -PhysicalPlanPtr popBack(std::vector vec) -{ - assert(!vec.empty()); - PhysicalPlanPtr back = vec.back(); - vec.pop_back(); - return back; -} -} // namespace - void PhysicalPlanBuilder::buildSource(const Block & sample_block) { cur_plans.push_back(PhysicalSource::build(source_sample_block, log.identifier())); diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index 0731f3de887..79ba07a5ab6 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -26,7 +26,7 @@ class PhysicalPlanBuilder public: explicit PhysicalPlanBuilder(Context & context_, const String & req_id) : context(context_) - , log("PhysicalPlanBuilder", req_id) + , log(Logger::get("PhysicalPlanBuilder", req_id)) {} void buildSource(const Block & sample_block); @@ -42,6 +42,6 @@ class PhysicalPlanBuilder Context & context; - Logger log; + LoggerPtr log; }; } // namespace DB From c6363c8b81581644b960b3116b33754fcb0cfdf2 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 25 Apr 2022 23:34:46 +0800 Subject: [PATCH 04/57] fix --- dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index a2cd2ee9839..3cbeec61e16 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -1,16 +1,5 @@ -#include -#include #include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include namespace DB { From bb44d1f5af7b865a0f1ae073f6ff27ec1b999676 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 25 Apr 2022 23:38:02 +0800 Subject: [PATCH 05/57] fix --- dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp | 2 +- dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 3cbeec61e16..520381216ce 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -5,6 +5,6 @@ namespace DB { void PhysicalPlanBuilder::buildSource(const Block & sample_block) { - cur_plans.push_back(PhysicalSource::build(source_sample_block, log.identifier())); + cur_plans.push_back(PhysicalSource::build(sample_block, log->identifier())); } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index 79ba07a5ab6..bc97d84f5b3 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -40,7 +40,7 @@ class PhysicalPlanBuilder private: std::vector cur_plans; - Context & context; + [[maybe_unused]] Context & context; LoggerPtr log; }; From c51c1525263d5d23de4906addc24db4a69390ffe Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 10 May 2022 15:32:38 +0800 Subject: [PATCH 06/57] add optimize --- dbms/src/Flash/Planner/Planner.cpp | 3 ++- dbms/src/Flash/Planner/optimize.cpp | 38 +++++++++++++++++++++++++++++ dbms/src/Flash/Planner/optimize.h | 23 +++++++++++++++++ 3 files changed, 63 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Flash/Planner/optimize.cpp create mode 100644 dbms/src/Flash/Planner/optimize.h diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index ef0e3f4aeeb..8b695a3aa92 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include namespace DB @@ -71,7 +72,7 @@ void Planner::executeImpl(DAGPipeline & pipeline) } auto physical_plan = builder.getResult(); - physical_plan->finalize(); + optimize(context, physical_plan); physical_plan->transform(pipeline, context, max_streams); } } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/optimize.cpp b/dbms/src/Flash/Planner/optimize.cpp new file mode 100644 index 00000000000..46c527f92f3 --- /dev/null +++ b/dbms/src/Flash/Planner/optimize.cpp @@ -0,0 +1,38 @@ +// +// Created by root on 5/10/22. +// + +#include +#include + +namespace DB +{ +class Rule +{ +public: + virtual PhysicalPlanPtr apply(const Context & context, PhysicalPlanPtr plan) = 0; + + virtual ~Rule() = default; +}; +using RulePtr = std::shared_ptr; + +class FinalizeRule : public Rule +{ +public: + PhysicalPlanPtr apply(const Context &, PhysicalPlanPtr plan) override + { + plan->finalize(); + return plan; + } + + static RulePtr create() { return std::make_shared(); } +}; + +void optimize(const Context & context, PhysicalPlanPtr plan) +{ + assert(plan); + static std::vector rules{FinalizeRule::create()}; + for (const auto & rule : rules) + plan = rule->apply(context, plan); +} +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/optimize.h b/dbms/src/Flash/Planner/optimize.h new file mode 100644 index 00000000000..03da563d1c0 --- /dev/null +++ b/dbms/src/Flash/Planner/optimize.h @@ -0,0 +1,23 @@ +// 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. + +#pragma once + +#include + +namespace DB +{ +class Context; +void optimize(const Context & context, PhysicalPlanPtr plan); +} // namespace DB From a5647fddcda8868ed0e61f51da11c035fa8a9879 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 10 May 2022 15:55:10 +0800 Subject: [PATCH 07/57] format --- dbms/src/Flash/Planner/optimize.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Planner/optimize.cpp b/dbms/src/Flash/Planner/optimize.cpp index 46c527f92f3..e1f2c127c0d 100644 --- a/dbms/src/Flash/Planner/optimize.cpp +++ b/dbms/src/Flash/Planner/optimize.cpp @@ -35,4 +35,4 @@ void optimize(const Context & context, PhysicalPlanPtr plan) for (const auto & rule : rules) plan = rule->apply(context, plan); } -} // namespace DB \ No newline at end of file +} // namespace DB From 39cc3ecd4086875f4acd685d47b188cf728e6eb2 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 10 May 2022 15:58:16 +0800 Subject: [PATCH 08/57] license --- dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp | 14 ++++++++++++++ dbms/src/Flash/Planner/PhysicalPlanHelper.cpp | 14 ++++++++++++++ dbms/src/Flash/Planner/PhysicalPlanHelper.h | 14 ++++++++++++++ dbms/src/Flash/Planner/optimize.cpp | 12 +++++++++++- 4 files changed, 53 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 520381216ce..b4037746ae5 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -1,3 +1,17 @@ +// 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 #include diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp index 9d1fc4fee2c..456ea70101e 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp @@ -1,3 +1,17 @@ +// 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 namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.h b/dbms/src/Flash/Planner/PhysicalPlanHelper.h index 22ccf876eac..8a39921ec51 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.h +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.h @@ -1,3 +1,17 @@ +// 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. + #pragma once #include diff --git a/dbms/src/Flash/Planner/optimize.cpp b/dbms/src/Flash/Planner/optimize.cpp index e1f2c127c0d..025c25fd9f1 100644 --- a/dbms/src/Flash/Planner/optimize.cpp +++ b/dbms/src/Flash/Planner/optimize.cpp @@ -1,6 +1,16 @@ +// Copyright 2022 PingCAP, Ltd. // -// Created by root on 5/10/22. +// 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 #include From 3960244e87d7bad422637f18761c74c28ae37706 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 10 May 2022 16:18:08 +0800 Subject: [PATCH 09/57] plan type --- dbms/src/Flash/Planner/PhysicalPlan.cpp | 13 +++++++++++- dbms/src/Flash/Planner/PhysicalPlan.h | 11 +++++----- dbms/src/Flash/Planner/PlanType.cpp | 4 ++-- dbms/src/Flash/Planner/PlanType.h | 27 +++++++++++++++++++++---- 4 files changed, 42 insertions(+), 13 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 1d4f95f3716..73fcb839ae1 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -21,6 +21,17 @@ namespace DB { +PhysicalPlan::PhysicalPlan( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id) + : executor_id(executor_id_) + , type(type_) + , schema(schema_) + , log(Logger::get(type_.toString(), req_id)) +{} + String PhysicalPlan::toString() { auto schema_to_string = [&]() { @@ -34,7 +45,7 @@ String PhysicalPlan::toString() }; return fmt::format( "type: {}, executor_id: {}, is_record_profile_streams: {}, schema: {}", - DB::toString(type), + type.toString(), executor_id, is_record_profile_streams, schema_to_string()); diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 6e07f7aeefa..8a69545f10b 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -34,12 +34,11 @@ using PhysicalPlanPtr = std::shared_ptr; class PhysicalPlan { public: - PhysicalPlan(const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, const String & req_id) - : executor_id(executor_id_) - , type(type_) - , schema(schema_) - , log(Logger::get(DB::toString(type_), req_id)) - {} + PhysicalPlan( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id); virtual ~PhysicalPlan() = default; diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index b766a890455..f4b685fbd93 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -17,9 +17,9 @@ namespace DB { -String toString(const PlanType & plan_type) +String PlanType::toString() const { - switch (plan_type) + switch (enum_value) { case Source: return "Source"; diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index 47bd1d8316d..9e80b152652 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -18,10 +18,29 @@ namespace DB { -enum PlanType +struct PlanType { - Source, -}; + enum PlanTypeEnum + { + Source = 0x1, + }; + PlanTypeEnum enum_value; + + PlanType(int value = 0) // NOLINT(google-explicit-constructor) + : enum_value(static_cast(value)) + {} + + PlanType & operator=(int value) + { + this->enum_value = static_cast(value); + return *this; + } -String toString(const PlanType & plan_type); + operator int() const // NOLINT(google-explicit-constructor) + { + return this->enum_value; + } + + String toString() const; +}; } // namespace DB From 9b86c4194067f685ae58eae8a6ed8c5d20ba5b28 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 10 May 2022 17:39:47 +0800 Subject: [PATCH 10/57] u --- dbms/src/Flash/Planner/Planner.cpp | 2 +- dbms/src/Flash/Planner/Planner.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 8b695a3aa92..5e234dd3365 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -75,4 +75,4 @@ void Planner::executeImpl(DAGPipeline & pipeline) optimize(context, physical_plan); physical_plan->transform(pipeline, context, max_streams); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Planner/Planner.h b/dbms/src/Flash/Planner/Planner.h index 7abe258a1aa..36ba918d780 100644 --- a/dbms/src/Flash/Planner/Planner.h +++ b/dbms/src/Flash/Planner/Planner.h @@ -57,4 +57,4 @@ class Planner LoggerPtr log; }; -} // namespace DB \ No newline at end of file +} // namespace DB From 4bde83b015f9e209c39260d4c803afb249a8a2fc Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 25 May 2022 14:40:08 +0800 Subject: [PATCH 11/57] license check --- .github/workflows/license-checker.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/license-checker.yml b/.github/workflows/license-checker.yml index e156c1b2b4c..c4c510677b1 100644 --- a/.github/workflows/license-checker.yml +++ b/.github/workflows/license-checker.yml @@ -3,10 +3,10 @@ name: License checker on: push: branches: - - master + - planner_refactory pull_request: branches: - - master + - planner_refactory jobs: check-license: From 2fc02b7ba3b9f8806d52661f9fdef478eb0ef880 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 26 May 2022 15:29:32 +0800 Subject: [PATCH 12/57] add tests --- dbms/src/Flash/tests/gtest_interpreter.cpp | 2 + dbms/src/Flash/tests/gtest_planner.cpp | 388 +++++++++++++++++++++ dbms/src/Interpreters/Settings.h | 2 +- 3 files changed, 391 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Flash/tests/gtest_planner.cpp diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index aed9d9e90f9..f21f0701e90 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -26,6 +26,8 @@ class InterpreterExecuteTest : public DB::tests::InterpreterTest { InterpreterTest::initializeContext(); + context.context->setSetting("enable_planner", "false"); + context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "r_table"}, {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); diff --git a/dbms/src/Flash/tests/gtest_planner.cpp b/dbms/src/Flash/tests/gtest_planner.cpp new file mode 100644 index 00000000000..c2742080a59 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_planner.cpp @@ -0,0 +1,388 @@ +// 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 +#include + +namespace DB::tests +{ +class PlannerTest : public DB::tests::InterpreterTest +{ +public: + void initializeContext() override + { + InterpreterTest::initializeContext(); + + context.context->setSetting("enable_planner", "true"); + + context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); + context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); + context.addMockTable({"test_db", "r_table"}, {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addMockTable({"test_db", "l_table"}, {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_1", {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_l", {{"l_a", TiDB::TP::TypeString}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeString}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + } +}; + +TEST_F(PlannerTest, SingleQueryBlock) +try +{ + auto request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .filter(eq(col("s2"), col("s3"))) + .topN("s2", false, 10) + .build(context); + { + String expected = R"( +Union: + SharedQuery x 10: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Filter: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Filter: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .filter(eq(col("s2"), col("s3"))) + .limit(10) + .build(context); + + { + String expected = R"( +Union: + SharedQuery x 10: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + Expression: + Filter: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Filter: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerTest, MultipleQueryBlockWithSource) +try +{ + auto request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project("s1") + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) + .project({"max(s1)", "s1", "s2"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) + .project({"max(s1)", "s1", "s2"}) + .filter(eq(col("s1"), col("s2"))) + .project({"max(s1)", "s1"}) + .limit(10) + .build(context); + { + String expected = R"( +Union: + SharedQuery x 10: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + Expression: + Expression: + Expression: + Expression: + Filter: + Expression: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + // Join Source. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + DAGRequestBuilder table3 = context.scan("test_db", "r_table"); + DAGRequestBuilder table4 = context.scan("test_db", "l_table"); + + request = table1.join( + table2.join( + table3.join(table4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + { + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.receive("sender_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project("s1") + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.receive("sender_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project("s1") + .exchangeSender(tipb::Broadcast) + .build(context); + { + String expected = R"( +Union: + MockExchangeSender x 10 + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + // only join + ExchangeReceiver + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + { + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + // join + receiver + sender + // TODO: Find a way to write the request easier. + DAGRequestBuilder receiver5 = context.receive("sender_l"); + DAGRequestBuilder receiver6 = context.receive("sender_r"); + DAGRequestBuilder receiver7 = context.receive("sender_l"); + DAGRequestBuilder receiver8 = context.receive("sender_r"); + request = receiver5.join( + receiver6.join( + receiver7.join(receiver8, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 10 + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +} \ No newline at end of file diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 44786ded44e..b87b8ccc3d8 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -364,7 +364,7 @@ struct Settings M(SettingUInt64, manual_compact_pool_size, 1, "The number of worker threads to handle manual compact requests.") \ M(SettingUInt64, manual_compact_max_concurrency, 10, "Max concurrent tasks. It should be larger than pool size.") \ M(SettingUInt64, manual_compact_more_until_ms, 60000, "Continuously compact more segments until reaching specified elapsed time. If 0 is specified, only one segment will be compacted each round.") \ - M(SettingBool, enable_planner, true, "Enable planner") + M(SettingBool, enable_planner, true, "Enable planner") // clang-format on #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) TYPE NAME{DEFAULT}; From 6a91f869c5b75e80fd8f66603fe7f5a0132a52d6 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 26 May 2022 15:46:08 +0800 Subject: [PATCH 13/57] f --- dbms/src/Flash/tests/gtest_interpreter.cpp | 2 +- dbms/src/Flash/tests/gtest_planner.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index f21f0701e90..7ddbc975515 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -26,7 +26,7 @@ class InterpreterExecuteTest : public DB::tests::InterpreterTest { InterpreterTest::initializeContext(); - context.context->setSetting("enable_planner", "false"); + context.context.setSetting("enable_planner", "false"); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); diff --git a/dbms/src/Flash/tests/gtest_planner.cpp b/dbms/src/Flash/tests/gtest_planner.cpp index c2742080a59..754df6993d0 100644 --- a/dbms/src/Flash/tests/gtest_planner.cpp +++ b/dbms/src/Flash/tests/gtest_planner.cpp @@ -24,7 +24,7 @@ class PlannerTest : public DB::tests::InterpreterTest { InterpreterTest::initializeContext(); - context.context->setSetting("enable_planner", "true"); + context.context.setSetting("enable_planner", "true"); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); @@ -385,4 +385,4 @@ CreatingSets } CATCH -} \ No newline at end of file +} // namespace DB::tests \ No newline at end of file From dcaa84abf53749f2a614a34c985b9ef2f6be4fc2 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 31 May 2022 11:38:02 +0800 Subject: [PATCH 14/57] Errors::Planner --- dbms/src/Common/TiFlashException.h | 17 +++++++++++++++++ dbms/src/Flash/Planner/PlanType.cpp | 4 ++-- dbms/src/Flash/Planner/plans/PhysicalLeaf.h | 8 ++++---- 3 files changed, 23 insertions(+), 6 deletions(-) diff --git a/dbms/src/Common/TiFlashException.h b/dbms/src/Common/TiFlashException.h index 2026571859e..d84615de932 100644 --- a/dbms/src/Common/TiFlashException.h +++ b/dbms/src/Common/TiFlashException.h @@ -96,6 +96,23 @@ namespace DB "This error usually occurs when the TiFlash server is busy or the TiFlash node is down.\n", \ ""); \ ) \ + C(Planner, \ + E(BadRequest, "Bad TiDB DAGRequest.", \ + "This error is usually caused by incorrect TiDB DAGRequest. \n" \ + "Please contact with developer, \n" \ + "better providing information about your cluster(log, topology information etc.).", \ + ""); \ + E(Unimplemented, "Some features are unimplemented.", \ + "This error may caused by unmatched TiDB and TiFlash versions, \n" \ + "and should not occur in common case. \n" \ + "Please contact with developer, \n" \ + "better providing information about your cluster(log, topology information etc.).", \ + ""); \ + E(Internal, "TiFlash Planner internal error.", \ + "Please contact with developer, \n" \ + "better providing information about your cluster(log, topology information etc.).", \ + ""); \ + ) \ C(Table, \ E(SchemaVersionError, "Schema version of target table in TiFlash is different from that in query.", \ "TiFlash will sync the newest schema from TiDB before processing every query. \n" \ diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index f4b685fbd93..131a9c13b3a 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include namespace DB @@ -24,7 +24,7 @@ String PlanType::toString() const case Source: return "Source"; default: - throw Exception("Unknown PlanType"); + throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); } } } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h index 038c0316f20..f281ce7f1e0 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h @@ -14,7 +14,7 @@ #pragma once -#include +#include #include namespace DB @@ -31,17 +31,17 @@ class PhysicalLeaf : public PhysicalPlan PhysicalPlanPtr children(size_t) const override { - throw Exception("the children size of PhysicalLeaf is zero"); + throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); } void setChild(size_t, const PhysicalPlanPtr &) override { - throw Exception("the children size of PhysicalLeaf is zero"); + throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); } void appendChild(const PhysicalPlanPtr &) override { - throw Exception("the children size of PhysicalLeaf is zero"); + throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); } size_t childrenSize() const override { return 0; }; From 64b6d2c0264440b65823fdce860156a0c1555a99 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 8 Jun 2022 17:23:54 +0800 Subject: [PATCH 15/57] add tests and test --- ...nner.cpp => gtest_planner_interpreter.cpp} | 184 ++++++++++++++++-- dbms/src/TestUtils/ExecutorTestUtils.cpp | 4 +- 2 files changed, 169 insertions(+), 19 deletions(-) rename dbms/src/Flash/tests/{gtest_planner.cpp => gtest_planner_interpreter.cpp} (75%) diff --git a/dbms/src/Flash/tests/gtest_planner.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp similarity index 75% rename from dbms/src/Flash/tests/gtest_planner.cpp rename to dbms/src/Flash/tests/gtest_planner_interpreter.cpp index 754df6993d0..a1a39d444ca 100644 --- a/dbms/src/Flash/tests/gtest_planner.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -12,17 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include -namespace DB::tests +namespace DB { -class PlannerTest : public DB::tests::InterpreterTest +namespace tests +{ +class PlannerInterpreterExecuteTest : public DB::tests::ExecutorTest { public: void initializeContext() override { - InterpreterTest::initializeContext(); + ExecutorTest::initializeContext(); context.context.setSetting("enable_planner", "true"); @@ -36,7 +38,7 @@ class PlannerTest : public DB::tests::InterpreterTest } }; -TEST_F(PlannerTest, SingleQueryBlock) +TEST_F(PlannerInterpreterExecuteTest, SimpleQuery) try { auto request = context.scan("test_db", "test_table_1") @@ -47,7 +49,7 @@ try .build(context); { String expected = R"( -Union: +Union: SharedQuery x 10: Expression: MergeSorting, limit = 10 @@ -72,7 +74,7 @@ Union: { String expected = R"( -Union: +Union: SharedQuery x 10: Limit, limit = 10 Union: @@ -90,7 +92,7 @@ Union: } CATCH -TEST_F(PlannerTest, MultipleQueryBlockWithSource) +TEST_F(PlannerInterpreterExecuteTest, ComplexQuery) try { auto request = context.scan("test_db", "test_table_1") @@ -100,7 +102,7 @@ try .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -122,7 +124,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -147,7 +149,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -181,7 +183,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: SharedQuery x 10: Limit, limit = 10 Union: @@ -244,7 +246,7 @@ CreatingSets HashJoinProbe: Expression: MockTableScan - Union: + Union: Expression x 10: Expression: HashJoinProbe: @@ -260,7 +262,7 @@ CreatingSets .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -283,7 +285,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: MockExchangeSender x 10 Expression: Expression: @@ -331,7 +333,7 @@ CreatingSets HashJoinProbe: Expression: MockExchangeReceiver - Union: + Union: Expression x 10: Expression: HashJoinProbe: @@ -373,7 +375,7 @@ CreatingSets HashJoinProbe: Expression: MockExchangeReceiver - Union: + Union: MockExchangeSender x 10 Expression: Expression: @@ -385,4 +387,150 @@ CreatingSets } CATCH -} // namespace DB::tests \ No newline at end of file +TEST_F(PlannerInterpreterExecuteTest, ParallelQuery) +try +{ + /// executor with table scan + auto request = context.scan("test_db", "test_table_1") + .limit(10) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + /// other cases + request = context.scan("test_db", "test_table_1") + .limit(10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .project({"s2", "s3"}) + .aggregation({Max(col("s2"))}, {col("s3")}) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .limit(10) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"()"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + request = table1.join(table2.limit(1), {col("join_c")}, ASTTableJoin::Kind::Left).build(context); + { + String expected; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +} // namespace tests +} // namespace DB \ No newline at end of file diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 67a21d12286..3347246553b 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -62,7 +62,9 @@ void ExecutorTest::executeInterpreter(const String & expected_string, const std: auto res = executeQuery(dag, context.context, false, QueryProcessingStage::Complete); FmtBuffer fb; res.in->dumpTree(fb); - ASSERT_EQ(Poco::trim(expected_string), Poco::trim(fb.toString())); +// ASSERT_EQ(Poco::trim(expected_string), Poco::trim(fb.toString())); + std::cout << "\n\n" << expected_string << std::endl; + std::cout << Poco::trim(fb.toString()) << std::endl; } namespace From ca45abfe044887c73eea2d5192edf53a655998d6 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 8 Jun 2022 18:17:15 +0800 Subject: [PATCH 16/57] add parallel tests --- .../Flash/tests/gtest_planner_interpreter.cpp | 281 ++++++++++++--- ...terpreter.cpp => gtest_qb_interpreter.cpp} | 324 +++++++++++++++++- dbms/src/TestUtils/ExecutorTestUtils.cpp | 4 +- 3 files changed, 549 insertions(+), 60 deletions(-) rename dbms/src/Flash/tests/{gtest_interpreter.cpp => gtest_qb_interpreter.cpp} (60%) diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp index a1a39d444ca..acb5ae0d2c9 100644 --- a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -392,133 +392,290 @@ try { /// executor with table scan auto request = context.scan("test_db", "test_table_1") - .limit(10) - .build(context); + .limit(10) + .build(context); { - String expected; + String expected = R"( +Limit, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); - expected = R"()"; + expected = R"( +Union: + SharedQuery x 5: + Limit, limit = 10 + Union: + Limit x 5, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } request = context.scan("test_db", "test_table_1") - .project({"s1", "s2", "s3"}) - .build(context); + .project({"s1", "s2", "s3"}) + .build(context); { - String expected; + String expected = R"( +Expression: + Expression: + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); - expected = R"()"; + expected = R"( +Union: + Expression x 5: + Expression: + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } request = context.scan("test_db", "test_table_1") - .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) - .build(context); + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); { - String expected; + String expected = R"( +Expression: + Aggregating + Concat + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); - expected = R"()"; + expected = R"( +Union: + Expression x 5: + SharedQuery: + ParallelAggregating, max_threads: 5, final: true + Expression x 5: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } request = context.scan("test_db", "test_table_1") - .topN("s2", false, 10) - .build(context); + .topN("s2", false, 10) + .build(context); { - String expected; + String expected = R"( +Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); - expected = R"()"; + expected = R"( +Union: + SharedQuery x 5: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 5: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } request = context.scan("test_db", "test_table_1") - .filter(eq(col("s2"), col("s3"))) - .build(context); + .filter(eq(col("s2"), col("s3"))) + .build(context); { - String expected; + String expected = R"( +Expression: + Expression: + Filter: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); - expected = R"()"; + expected = R"( +Union: + Expression x 5: + Expression: + Filter: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } /// other cases request = context.scan("test_db", "test_table_1") - .limit(10) - .project({"s1", "s2", "s3"}) - .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) - .build(context); + .limit(10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); { - String expected; + String expected = R"( +Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - expected = R"()"; + expected = R"(Expression: + Aggregating + Concat + Expression: + Expression: + Expression: + Limit, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } request = context.scan("test_db", "test_table_1") - .topN("s2", false, 10) - .project({"s1", "s2", "s3"}) - .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) - .build(context); + .topN("s2", false, 10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); { - String expected; + String expected = R"( +Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - expected = R"()"; + expected = R"( +Expression: + Aggregating + Concat + Expression: + Expression: + Expression: + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } request = context.scan("test_db", "test_table_1") - .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) - .project({"s2", "s3"}) - .aggregation({Max(col("s2"))}, {col("s3")}) - .build(context); + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .project({"s2", "s3"}) + .aggregation({Max(col("s2"))}, {col("s3")}) + .build(context); { - String expected; + String expected = R"( +Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - expected = R"()"; + expected = R"( +Expression: + Aggregating + Concat + Expression: + Expression: + Expression: + Expression: + Aggregating + Concat + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } request = context.scan("test_db", "test_table_1") - .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) - .exchangeSender(tipb::PassThrough) - .build(context); + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .exchangeSender(tipb::PassThrough) + .build(context); { - String expected; + String expected = R"( +Union: + MockExchangeSender x 10 + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - expected = R"()"; + expected = R"( +MockExchangeSender + Expression: + Aggregating + Concat + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } request = context.scan("test_db", "test_table_1") - .topN("s2", false, 10) - .exchangeSender(tipb::PassThrough) - .build(context); + .topN("s2", false, 10) + .exchangeSender(tipb::PassThrough) + .build(context); { - String expected; + String expected = R"( +Union: + MockExchangeSender x 10 + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - expected = R"()"; + expected = R"( +MockExchangeSender + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } request = context.scan("test_db", "test_table_1") - .limit(10) - .exchangeSender(tipb::PassThrough) - .build(context); + .limit(10) + .exchangeSender(tipb::PassThrough) + .build(context); { - String expected; + String expected = R"( +Union: + MockExchangeSender x 10 + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - expected = R"()"; + expected = R"( +MockExchangeSender + Limit, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -526,7 +683,23 @@ try DAGRequestBuilder table2 = context.scan("test_db", "l_table"); request = table1.join(table2.limit(1), {col("join_c")}, ASTTableJoin::Kind::Left).build(context); { - String expected; + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + SharedQuery: + Limit, limit = 1 + Union: + Limit x 10, limit = 1 + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } } diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_qb_interpreter.cpp similarity index 60% rename from dbms/src/Flash/tests/gtest_interpreter.cpp rename to dbms/src/Flash/tests/gtest_qb_interpreter.cpp index c59a6c7a085..c8ac422fdb3 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_qb_interpreter.cpp @@ -19,7 +19,7 @@ namespace DB { namespace tests { -class InterpreterExecuteTest : public DB::tests::ExecutorTest +class QBInterpreterExecuteTest : public DB::tests::ExecutorTest { public: void initializeContext() override @@ -38,7 +38,7 @@ class InterpreterExecuteTest : public DB::tests::ExecutorTest } }; -TEST_F(InterpreterExecuteTest, SingleQueryBlock) +TEST_F(QBInterpreterExecuteTest, SingleQueryBlock) try { auto request = context.scan("test_db", "test_table_1") @@ -92,7 +92,7 @@ Union: } CATCH -TEST_F(InterpreterExecuteTest, MultipleQueryBlockWithSource) +TEST_F(QBInterpreterExecuteTest, MultipleQueryBlockWithSource) try { auto request = context.scan("test_db", "test_table_1") @@ -387,5 +387,323 @@ CreatingSets } CATCH +TEST_F(QBInterpreterExecuteTest, ParallelQuery) +try +{ + /// executor with table scan + auto request = context.scan("test_db", "test_table_1") + .limit(10) + .build(context); + { + String expected = R"( +Limit, limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + SharedQuery x 5: + Limit, limit = 10 + Union: + Limit x 5, limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .build(context); + { + String expected = R"( +Expression: + Expression: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + Expression: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected = R"( +Expression: + Aggregating + Concat + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + SharedQuery: + ParallelAggregating, max_threads: 5, final: true + Expression x 5: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .build(context); + { + String expected = R"( +Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + SharedQuery x 5: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 5: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .build(context); + { + String expected = R"( +Expression: + Expression: + Filter: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + Expression: + Filter: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); + } + + /// other cases + request = context.scan("test_db", "test_table_1") + .limit(10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"(Expression: + Aggregating + Concat + Expression: + Expression: + Expression: + Limit, limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .project({"s1", "s2", "s3"}) + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +Expression: + Aggregating + Concat + Expression: + Expression: + Expression: + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .project({"s2", "s3"}) + .aggregation({Max(col("s2"))}, {col("s3")}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +Expression: + Aggregating + Concat + Expression: + Expression: + Expression: + Expression: + Aggregating + Concat + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .aggregation({Max(col("s1"))}, {col("s2"), col("s3")}) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected = R"( +Union: + MockExchangeSender x 10 + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +MockExchangeSender + Expression: + Aggregating + Concat + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .topN("s2", false, 10) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected = R"( +Union: + MockExchangeSender x 10 + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +MockExchangeSender + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + request = context.scan("test_db", "test_table_1") + .limit(10) + .exchangeSender(tipb::PassThrough) + .build(context); + { + String expected = R"( +Union: + MockExchangeSender x 10 + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +MockExchangeSender + Limit, limit = 10 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + } + + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + request = table1.join(table2.limit(1), {col("join_c")}, ASTTableJoin::Kind::Left).build(context); + { + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + SharedQuery: + Limit, limit = 1 + Union: + Limit x 10, limit = 1 + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + } // namespace tests } // namespace DB \ No newline at end of file diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 3347246553b..67a21d12286 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -62,9 +62,7 @@ void ExecutorTest::executeInterpreter(const String & expected_string, const std: auto res = executeQuery(dag, context.context, false, QueryProcessingStage::Complete); FmtBuffer fb; res.in->dumpTree(fb); -// ASSERT_EQ(Poco::trim(expected_string), Poco::trim(fb.toString())); - std::cout << "\n\n" << expected_string << std::endl; - std::cout << Poco::trim(fb.toString()) << std::endl; + ASSERT_EQ(Poco::trim(expected_string), Poco::trim(fb.toString())); } namespace From 5a95ca0550994fcf31fe6b8be38772387ad3e506 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 8 Jun 2022 18:33:04 +0800 Subject: [PATCH 17/57] fix --- dbms/src/Flash/Planner/Planner.cpp | 4 ++-- dbms/src/Flash/Planner/optimize.cpp | 3 ++- dbms/src/Flash/Planner/optimize.h | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 5e234dd3365..3ccfc1234d3 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -67,12 +67,12 @@ void Planner::executeImpl(DAGPipeline & pipeline) PhysicalPlanBuilder builder{context, log->identifier()}; for (const auto & input_streams : input_streams_vec) { - assert(!input_streams.empty()); + RUNTIME_ASSERT(!input_streams.empty(), log, "input streams cannot be empty"); builder.buildSource(input_streams.back()->getHeader()); } auto physical_plan = builder.getResult(); - optimize(context, physical_plan); + physical_plan = optimize(context, physical_plan); physical_plan->transform(pipeline, context, max_streams); } } // namespace DB diff --git a/dbms/src/Flash/Planner/optimize.cpp b/dbms/src/Flash/Planner/optimize.cpp index 025c25fd9f1..244ddd534b6 100644 --- a/dbms/src/Flash/Planner/optimize.cpp +++ b/dbms/src/Flash/Planner/optimize.cpp @@ -38,11 +38,12 @@ class FinalizeRule : public Rule static RulePtr create() { return std::make_shared(); } }; -void optimize(const Context & context, PhysicalPlanPtr plan) +PhysicalPlanPtr optimize(const Context & context, PhysicalPlanPtr plan) { assert(plan); static std::vector rules{FinalizeRule::create()}; for (const auto & rule : rules) plan = rule->apply(context, plan); + return plan; } } // namespace DB diff --git a/dbms/src/Flash/Planner/optimize.h b/dbms/src/Flash/Planner/optimize.h index 03da563d1c0..8ba738c9f77 100644 --- a/dbms/src/Flash/Planner/optimize.h +++ b/dbms/src/Flash/Planner/optimize.h @@ -19,5 +19,5 @@ namespace DB { class Context; -void optimize(const Context & context, PhysicalPlanPtr plan); +PhysicalPlanPtr optimize(const Context & context, PhysicalPlanPtr plan); } // namespace DB From 6f9dc619748f93756b6f346c4924b614835d9861 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 9 Jun 2022 11:13:35 +0800 Subject: [PATCH 18/57] format --- dbms/src/Flash/Planner/plans/PhysicalLeaf.h | 6 +++++- dbms/src/Flash/Planner/plans/PhysicalUnary.h | 12 ++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h index f281ce7f1e0..50ced412c13 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h @@ -25,7 +25,11 @@ namespace DB class PhysicalLeaf : public PhysicalPlan { public: - PhysicalLeaf(const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, const String & req_id) + PhysicalLeaf( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id) : PhysicalPlan(executor_id_, type_, schema_, req_id) {} diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/plans/PhysicalUnary.h index a5fab90ddbc..4d0091bb8e3 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalUnary.h +++ b/dbms/src/Flash/Planner/plans/PhysicalUnary.h @@ -27,20 +27,24 @@ namespace DB class PhysicalUnary : public PhysicalPlan { public: - PhysicalUnary(const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, const String & req_id) + PhysicalUnary( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id) : PhysicalPlan(executor_id_, type_, schema_, req_id) {} PhysicalPlanPtr children(size_t i) const override { - RUNTIME_ASSERT(i == 0, log, fmt::format("child_index({}) should not >= childrenSize({})", i, childrenSize())); + RUNTIME_ASSERT(i == 0, log, "child_index({}) should not >= childrenSize({})", i, childrenSize()); assert(child); return child; } void setChild(size_t i, const PhysicalPlanPtr & new_child) override { - RUNTIME_ASSERT(i == 0, log, fmt::format("child_index({}) should not >= childrenSize({})", i, childrenSize())); + RUNTIME_ASSERT(i == 0, log, "child_index({}) should not >= childrenSize({})", i, childrenSize()); assert(new_child); assert(new_child.get() != this); child = new_child; @@ -48,7 +52,7 @@ class PhysicalUnary : public PhysicalPlan void appendChild(const PhysicalPlanPtr & new_child) override { - RUNTIME_ASSERT(!child, log, fmt::format("the actual children size had be the max size({}), don't append child again", childrenSize())); + RUNTIME_ASSERT(!child, log, "the actual children size had be the max size({}), don't append child again", childrenSize()); assert(new_child); assert(new_child.get() != this); child = new_child; From edf26bce64914a550e3beb1e698ee2e69cbdde45 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 9 Jun 2022 11:55:21 +0800 Subject: [PATCH 19/57] address comment --- dbms/src/Flash/Planner/Planner.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Planner/Planner.h b/dbms/src/Flash/Planner/Planner.h index 36ba918d780..8941420caf9 100644 --- a/dbms/src/Flash/Planner/Planner.h +++ b/dbms/src/Flash/Planner/Planner.h @@ -52,7 +52,7 @@ class Planner const DAGQueryBlock & query_block; - /// How many streams we ask for storage to produce, and in how many threads we will do further processing. + /// How many streams we will do processing. size_t max_streams = 1; LoggerPtr log; From d31b310c52172d3f82ca4e5bb347ad0d42c3f9d1 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 10 Jun 2022 15:23:21 +0800 Subject: [PATCH 20/57] address comments --- dbms/src/Flash/Planner/PlanType.h | 2 +- dbms/src/Flash/Planner/Planner.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index 9e80b152652..9a5f26a497b 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -22,7 +22,7 @@ struct PlanType { enum PlanTypeEnum { - Source = 0x1, + Source = 0, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/Planner.h b/dbms/src/Flash/Planner/Planner.h index 8941420caf9..482d9cc5d76 100644 --- a/dbms/src/Flash/Planner/Planner.h +++ b/dbms/src/Flash/Planner/Planner.h @@ -52,7 +52,7 @@ class Planner const DAGQueryBlock & query_block; - /// How many streams we will do processing. + /// Max streams we will do processing. size_t max_streams = 1; LoggerPtr log; From 472dd1ca58fee9a25831902dbd4a57ab424ae522 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 14 Jun 2022 03:25:02 +0800 Subject: [PATCH 21/57] tmp save --- dbms/src/Flash/Coprocessor/DAGContext.cpp | 5 + dbms/src/Flash/Coprocessor/DAGContext.h | 2 + .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 12 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 7 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 16 +++ dbms/src/Flash/Coprocessor/InterpreterUtils.h | 6 + dbms/src/Flash/Planner/PhysicalPlan.cpp | 3 + .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 33 ++++++ dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 10 +- dbms/src/Flash/Planner/PhysicalPlanHelper.cpp | 6 + dbms/src/Flash/Planner/PhysicalPlanHelper.h | 5 + dbms/src/Flash/Planner/PlanType.cpp | 4 + dbms/src/Flash/Planner/PlanType.h | 2 + .../src/Flash/Planner/plans/PhysicalLimit.cpp | 58 ++++++++++ dbms/src/Flash/Planner/plans/PhysicalLimit.h | 49 ++++++++ dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 108 ++++++++++++++++++ dbms/src/Flash/Planner/plans/PhysicalTopN.h | 58 ++++++++++ 17 files changed, 375 insertions(+), 9 deletions(-) create mode 100644 dbms/src/Flash/Planner/plans/PhysicalLimit.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalLimit.h create mode 100644 dbms/src/Flash/Planner/plans/PhysicalTopN.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalTopN.h diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index 1736e0b6cec..e7f1a54dc2a 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -75,6 +75,11 @@ std::unordered_map & DAGContext::getProfileStreamsMap return profile_streams_map; } +void DAGContext::updateFinalConcurrency(size_t cur_streams_size, size_t max_streams) +{ + final_concurrency = std::min(std::max(final_concurrency, cur_streams_size), max_streams); +} + void DAGContext::initExecutorIdToJoinIdMap() { // only mpp task has join executor diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index c20eb3a367e..10b805a2af2 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -297,6 +297,8 @@ class DAGContext return sql_mode & f; } + void updateFinalConcurrency(size_t cur_streams_size, size_t max_streams); + bool isTest() const { return is_test; } void setColumnsForTest(std::unordered_map & columns_for_test_map_) { columns_for_test_map = columns_for_test_map_; } ColumnsWithTypeAndName columnsForTest(String executor_id); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 3b7112af02d..4b632de4fc6 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -56,6 +56,12 @@ class DAGExpressionAnalyzer : private boost::noncopyable const Context & getContext() const { return context; } + void reset(const std::vector & source_columns_) + { + source_columns = source_columns_; + prepared_sets.clear(); + } + const std::vector & getCurrentInputColumns() const; DAGPreparedSets & getPreparedSets() { return prepared_sets; } @@ -153,13 +159,13 @@ class DAGExpressionAnalyzer : private boost::noncopyable const tipb::Window & window, const size_t window_columns_start_index); -#ifndef DBMS_PUBLIC_GTEST -private: -#endif NamesAndTypes buildOrderColumns( const ExpressionActionsPtr & actions, const ::google::protobuf::RepeatedPtrField & order_by); +#ifndef DBMS_PUBLIC_GTEST +private: +#endif void appendCastAfterAgg( const ExpressionActionsPtr & actions, const tipb::Aggregation & agg); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 86d6428c92a..0eb8bf6e19c 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -548,10 +548,7 @@ void DAGQueryBlockInterpreter::handleProjection(DAGPipeline & pipeline, const ti output_columns.emplace_back(alias, col.type); project_cols.emplace_back(col.name, alias); } - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, chain.getLastActions(), log->identifier()); - stream->setExtraInfo("before projection"); - }); + executeExpression(pipeline, chain.getLastActions(), log, "before projection"); executeProject(pipeline, project_cols, "projection"); analyzer = std::make_unique(std::move(output_columns), context); } @@ -663,7 +660,7 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) "execution stream size for query block(before aggregation) {} is {}", query_block.qb_column_prefix, pipeline.streams.size()); - dagContext().final_concurrency = std::min(std::max(dagContext().final_concurrency, pipeline.streams.size()), max_streams); + dagContext().updateFinalConcurrency(pipeline.streams.size(), max_streams); if (res.before_aggregation) { diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index c9810454218..ec445658863 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -102,4 +103,19 @@ ExpressionActionsPtr generateProjectExpressionActions( project->add(ExpressionAction::project(project_cols)); return project; } + +void executeExpression( + DAGPipeline & pipeline, + const ExpressionActionsPtr & expr_actions, + const LoggerPtr & log, + const String & extra_info = "") +{ + if (expr_actions && !expr_actions->getActions().empty()) + { + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, expr_actions, log->identifier()); + stream->setExtraInfo(extra_info); + }); + } +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 5c4d4721d5e..c5c6a30799b 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -44,4 +44,10 @@ ExpressionActionsPtr generateProjectExpressionActions( const BlockInputStreamPtr & stream, const Context & context, const NamesWithAliases & project_cols); + +void executeExpression( + DAGPipeline & pipeline, + const ExpressionActionsPtr & expr_actions, + const LoggerPtr & log, + const String & extra_info = ""); } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 73fcb839ae1..1d2f8844041 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -69,5 +70,7 @@ void PhysicalPlan::transform(DAGPipeline & pipeline, Context & context, size_t m { transformImpl(pipeline, context, max_streams); recordProfileStreams(pipeline, context); + context.getDAGContext()->updateFinalConcurrency(pipeline.streams.size(), max_streams); + restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index b4037746ae5..b72764a7372 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -12,11 +12,44 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include +#include #include +#include namespace DB { +void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor * executor) +{ + assert(executor); + switch (executor->tp()) + { + case tipb::ExecType::TypeLimit: + cur_plans.push_back(PhysicalLimit::build(executor_id, log->identifier(), executor->limit(), popBack())); + break; + case tipb::ExecType::TypeTopN: + cur_plans.push_back(PhysicalTopN::build(context, executor_id, log->identifier(), executor->topn(), popBack())); + break; + default: + throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); + } +} + +DAGContext & PhysicalPlanBuilder::dagContext() const +{ + return *context.getDAGContext(); +} + +PhysicalPlanPtr PhysicalPlanBuilder::popBack() +{ + RUNTIME_ASSERT(!cur_plans.empty(), log, "cur_plans is empty, cannot popBack"); + PhysicalPlanPtr back = cur_plans.back(); + cur_plans.pop_back(); + return back; +} + void PhysicalPlanBuilder::buildSource(const Block & sample_block) { cur_plans.push_back(PhysicalSource::build(sample_block, log->identifier())); diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index bc97d84f5b3..7c3f681ee89 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { @@ -29,6 +30,8 @@ class PhysicalPlanBuilder , log(Logger::get("PhysicalPlanBuilder", req_id)) {} + void build(const String & executor_id, const tipb::Executor * executor); + void buildSource(const Block & sample_block); PhysicalPlanPtr getResult() const @@ -37,10 +40,15 @@ class PhysicalPlanBuilder return cur_plans.back(); } +private: + PhysicalPlanPtr popBack(); + + DAGContext & dagContext() const; + private: std::vector cur_plans; - [[maybe_unused]] Context & context; + Context & context; LoggerPtr log; }; diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp index 456ea70101e..2e1589cd45c 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp @@ -24,4 +24,10 @@ Names schemaToNames(const NamesAndTypes & schema) names.push_back(column.name); return names; } + +ExpressionActionsPtr newActions(const Block & input_block, const Context & context) +{ + const ColumnsWithTypeAndName & actions_input_columns = input_block.getColumnsWithTypeAndName(); + return std::make_shared(actions_input_columns, context.getSettingsRef()); +} } // namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.h b/dbms/src/Flash/Planner/PhysicalPlanHelper.h index 8a39921ec51..d912778a38b 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.h +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.h @@ -14,9 +14,14 @@ #pragma once +#include #include +#include +#include namespace DB::PhysicalPlanHelper { Names schemaToNames(const NamesAndTypes & schema); + +ExpressionActionsPtr newActions(const Block & input_block, const Context & context); } // namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index 131a9c13b3a..82756e76db0 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -23,6 +23,10 @@ String PlanType::toString() const { case Source: return "Source"; + case Limit: + return "Limit"; + case TopN: + return "TopN"; default: throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); } diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index 9a5f26a497b..0256b7eba52 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -23,6 +23,8 @@ struct PlanType enum PlanTypeEnum { Source = 0, + Limit = 1, + TopN = 2, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp new file mode 100644 index 00000000000..ff5f33c5a12 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp @@ -0,0 +1,58 @@ +// 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 +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalLimit::build( + const String & executor_id, + const String & req_id, + const tipb::Limit & limit, + PhysicalPlanPtr child) +{ + assert(child); + auto physical_limit = std::make_shared(executor_id, child->getSchema(), req_id, limit.limit()); + physical_limit->appendChild(child); + return physical_limit; +} + +void PhysicalLimit::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, log->identifier(), false); }); + if (pipeline.hasMoreThanOneStream()) + { + executeUnion(pipeline, max_streams, log, false, "for partial limit"); + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit, 0, log->identifier(), false); }); + } +} + +void PhysicalLimit::finalize(const Names & parent_require) +{ + child->finalize(parent_require); +} + +const Block & PhysicalLimit::getSampleBlock() const +{ + return child->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.h b/dbms/src/Flash/Planner/plans/PhysicalLimit.h new file mode 100644 index 00000000000..5501a9127f7 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.h @@ -0,0 +1,49 @@ +// 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. + +#pragma once + +#include +#include + +namespace DB +{ +class PhysicalLimit : public PhysicalUnary +{ +public: + static PhysicalPlanPtr build( + const String & executor_id, + const String & req_id, + const tipb::Limit & limit, + PhysicalPlanPtr child); + + PhysicalLimit( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + size_t limit_) + : PhysicalUnary(executor_id_, PlanType::Limit, schema_, req_id) + , limit(limit_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + size_t limit; +} +} diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp new file mode 100644 index 00000000000..a3bceee9bca --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -0,0 +1,108 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalTopN::build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::TopN & top_n, + PhysicalPlanPtr child) +{ + assert(child); + + if (unlikely(top_n.order_by_size() == 0)) + { + //should not reach here + throw TiFlashException("TopN executor without order by exprs", Errors::Coprocessor::BadRequest); + } + + DAGExpressionAnalyzer analyzer{child->getSchema(), context}; + ExpressionActionsPtr before_sort_actions = PhysicalPlanHelper::newActions(child->getSampleBlock(), context); + + auto order_columns = analyzer.buildOrderColumns(before_sort_actions, top_n.order_by()); + SortDescription order_descr = getSortDescription(order_columns, top_n.order_by()); + + auto physical_top_n = std::make_shared(executor_id, child->getSchema(), req_id, order_descr, before_sort_actions, top_n.limit()); + physical_top_n->appendChild(child); + return physical_top_n; +} + +void PhysicalTopN::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + const Settings & settings = context.getSettingsRef(); + + executeExpression(pipeline, before_sort_actions, log); + + pipeline.transform([&](auto & stream) { + auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); + + /// Limits on sorting + IProfilingBlockInputStream::LocalLimits limits; + limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sorting_stream->setLimits(limits); + + stream = sorting_stream; + }); + + /// If there are several streams, we merge them into one + executeUnion(pipeline, max_streams, log); + + /// Merge the sorted blocks. + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); +} + +void PhysicalTopN::finalize(const Names & parent_require) +{ + Names required_output = parent_require; + required_output.reserve(required_output.size() + order_descr.size()); + for (const auto & desc : order_descr) + required_output.push_back(desc.column_name); + before_sort_actions->finalize(required_output); + + child->finalize(before_sort_actions->getRequiredColumns()); + FinalizeHelper::prependProjectInputIfNeed(before_sort_actions, child->getSampleBlock().columns()); + + FinalizeHelper::checkSampleBlockContainsSchema(getSampleBlock(), schema); +} + +const Block & PhysicalTopN::getSampleBlock() const +{ + return before_sort_actions->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.h b/dbms/src/Flash/Planner/plans/PhysicalTopN.h new file mode 100644 index 00000000000..b92e187671f --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.h @@ -0,0 +1,58 @@ +// 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. + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class PhysicalTopN : public PhysicalUnary + { + public: + static PhysicalPlanPtr build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::TopN & top_n, + PhysicalPlanPtr child); + + PhysicalTopN( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const SortDescription & order_descr_, + const ExpressionActionsPtr & before_sort_actions_, + size_t limit_) + : PhysicalUnary(executor_id_, PlanType::TopN, schema_, req_id) + , order_descr(order_descr_) + , before_sort_actions(before_sort_actions_) + , limit(limit_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + + private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + SortDescription order_descr; + ExpressionActionsPtr before_sort_actions; + size_t limit; + }; +} // namespace DB From 6cc1a9c0ff61f3ec9cf7f548a8b69e67b046bc90 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 14 Jun 2022 11:24:11 +0800 Subject: [PATCH 22/57] limit and topn --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 55 ++------- .../Coprocessor/DAGQueryBlockInterpreter.h | 2 - .../Flash/Coprocessor/InterpreterUtils.cpp | 37 ++++++ dbms/src/Flash/Coprocessor/InterpreterUtils.h | 8 ++ dbms/src/Flash/Planner/FinalizeHelper.cpp | 112 ++++++++++++++++++ dbms/src/Flash/Planner/FinalizeHelper.h | 34 ++++++ dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 30 +---- dbms/src/Flash/Planner/plans/PhysicalTopN.h | 66 +++++------ 8 files changed, 233 insertions(+), 111 deletions(-) create mode 100644 dbms/src/Flash/Planner/FinalizeHelper.cpp create mode 100644 dbms/src/Flash/Planner/FinalizeHelper.h diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 0eb8bf6e19c..7b9ee2f158f 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -268,7 +268,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & size_t join_build_concurrency = settings.join_concurrent_build ? std::min(max_streams, build_pipeline.streams.size()) : 1; /// build side streams - executeExpression(build_pipeline, build_side_prepare_actions, "append join key and join filters for build side"); + executeExpression(build_pipeline, build_side_prepare_actions, log, "append join key and join filters for build side"); // add a HashJoinBuildBlockInputStream to build a shared hash table auto get_concurrency_build_index = JoinInterpreterHelper::concurrencyBuildIndexGenerator(join_build_concurrency); build_pipeline.transform([&](auto & stream) { @@ -284,7 +284,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & join_ptr->init(right_query.source->getHeader(), join_build_concurrency); /// probe side streams - executeExpression(probe_pipeline, probe_side_prepare_actions, "append join key and join filters for probe side"); + executeExpression(probe_pipeline, probe_side_prepare_actions, log, "append join key and join filters for probe side"); NamesAndTypes source_columns; for (const auto & p : probe_pipeline.firstStream()->getHeader()) source_columns.emplace_back(p.name, p.type); @@ -349,7 +349,7 @@ void DAGQueryBlockInterpreter::executeWindow( DAGPipeline & pipeline, WindowDescription & window_description) { - executeExpression(pipeline, window_description.before_window, "before window"); + executeExpression(pipeline, window_description.before_window, log, "before window"); /// If there are several streams, we merge them into one executeUnion(pipeline, max_streams, log, false, "merge into one for window input"); @@ -421,56 +421,15 @@ void DAGQueryBlockInterpreter::executeAggregation( } } -void DAGQueryBlockInterpreter::executeExpression(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, const String & extra_info) -{ - if (!expressionActionsPtr->getActions().empty()) - { - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, expressionActionsPtr, log->identifier()); - stream->setExtraInfo(extra_info); - }); - } -} - void DAGQueryBlockInterpreter::executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc) { - orderStreams(pipeline, sort_desc, 0); + orderStreams(pipeline, sort_desc, 0, context, log); } void DAGQueryBlockInterpreter::executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns) { Int64 limit = query_block.limit_or_topn->topn().limit(); - orderStreams(pipeline, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit); -} - -void DAGQueryBlockInterpreter::orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit) -{ - const Settings & settings = context.getSettingsRef(); - - pipeline.transform([&](auto & stream) { - auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); - - /// Limits on sorting - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; - limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sorting_stream->setLimits(limits); - - stream = sorting_stream; - }); - - /// If there are several streams, we merge them into one - executeUnion(pipeline, max_streams, log, false, "for partial order"); - - /// Merge the sorted blocks. - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), - order_descr, - settings.max_block_size, - limit, - settings.max_bytes_before_external_sort, - context.getTemporaryPath(), - log->identifier()); + orderStreams(pipeline, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit, context, log); } void DAGQueryBlockInterpreter::recordProfileStreams(DAGPipeline & pipeline, const String & key) @@ -563,7 +522,7 @@ void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb:: DAGExpressionAnalyzer dag_analyzer(input_columns, context); WindowDescription window_description = dag_analyzer.buildWindowDescription(window); executeWindow(pipeline, window_description); - executeExpression(pipeline, window_description.after_window, "cast after window"); + executeExpression(pipeline, window_description.after_window, log, "cast after window"); analyzer = std::make_unique(window_description.after_window_columns, context); } @@ -675,7 +634,7 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) } if (res.before_order_and_select) { - executeExpression(pipeline, res.before_order_and_select, "before order and select"); + executeExpression(pipeline, res.before_order_and_select, log, "before order and select"); } if (!res.order_columns.empty()) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index e68c4f91cee..cabdd4dc9be 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -67,9 +67,7 @@ class DAGQueryBlockInterpreter void handleWindow(DAGPipeline & pipeline, const tipb::Window & window); void handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort); void executeWhere(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column, const String & extra_info = ""); - void executeExpression(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, const String & extra_info = ""); void executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc); - void orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit); void executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns); void executeLimit(DAGPipeline & pipeline); void executeWindow( diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index ec445658863..9757ebf1aa6 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -14,6 +14,8 @@ #include #include +#include +#include #include #include #include @@ -118,4 +120,39 @@ void executeExpression( }); } } + +void orderStreams( + DAGPipeline & pipeline, + SortDescription order_descr, + Int64 limit, + const Context & context, + const LoggerPtr & log) +{ + const Settings & settings = context.getSettingsRef(); + + pipeline.transform([&](auto & stream) { + auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); + + /// Limits on sorting + IProfilingBlockInputStream::LocalLimits limits; + limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sorting_stream->setLimits(limits); + + stream = sorting_stream; + }); + + /// If there are several streams, we merge them into one + executeUnion(pipeline, max_streams, log, false, "for partial order"); + + /// Merge the sorted blocks. + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index c5c6a30799b..95d5ab7b53a 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include @@ -50,4 +51,11 @@ void executeExpression( const ExpressionActionsPtr & expr_actions, const LoggerPtr & log, const String & extra_info = ""); + +void orderStreams( + DAGPipeline & pipeline, + SortDescription order_descr, + Int64 limit, + const Context & context, + const LoggerPtr & log); } // namespace DB diff --git a/dbms/src/Flash/Planner/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp new file mode 100644 index 00000000000..31f21b3dc8e --- /dev/null +++ b/dbms/src/Flash/Planner/FinalizeHelper.cpp @@ -0,0 +1,112 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB::FinalizeHelper +{ +void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous) +{ + if (!actions->getRequiredColumnsWithTypes().empty() + && columns_from_previous > actions->getRequiredColumnsWithTypes().size()) + { + actions->prependProjectInput(); + } +} + +void checkSchemaContainsParentRequire(const NamesAndTypes & schema, const Names & parent_require) +{ + NameSet schema_set; + for (const auto & column : schema) + schema_set.insert(column.name); + for (const auto & parent_require_column : parent_require) + { + if (unlikely(schema_set.find(parent_require_column) == schema_set.end())) + throw TiFlashException( + fmt::format("schema don't contain parent require column: {}", parent_require_column), + Errors::Planner::Internal); + } +} + +void checkParentRequireContainsSchema(const Names & parent_require, const NamesAndTypes & schema) +{ + NameSet parent_require_set; + for (const auto & parent_require_column : parent_require) + parent_require_set.insert(parent_require_column); + for (const auto & schema_column : schema) + { + if (unlikely(parent_require_set.find(schema_column.name) == parent_require_set.end())) + throw TiFlashException( + fmt::format("parent require don't contain schema column: {}", schema_column.name), + Errors::Planner::Internal); + } +} + +void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema) +{ + for (const auto & schema_column : schema) + { + if (unlikely(!sample_block.has(schema_column.name))) + throw TiFlashException( + fmt::format("sample block don't contain schema column: {}", schema_column.name), + Errors::Planner::Internal); + + const auto & type_in_sample_block = sample_block.getByName(schema_column.name).type->getName(); + const auto & type_in_schema = schema_column.type->getName(); + if (unlikely(type_in_sample_block != type_in_schema)) + throw TiFlashException( + fmt::format( + "the type of column `{}` in sample block `{}` is difference from the one in schema `{}`", + schema_column.name, + type_in_sample_block, + type_in_schema), + Errors::Planner::Internal); + } +} + +void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & sample_block) +{ + std::unordered_map schema_map; + for (const auto & column : schema) + schema_map[column.name] = column.type; + for (const auto & sample_block_column : sample_block) + { + auto it = schema_map.find(sample_block_column.name); + if (unlikely(it == schema_map.end())) + throw TiFlashException( + fmt::format("schema don't contain sample block column: {}", sample_block_column.name), + Errors::Planner::Internal); + + const auto & type_in_schema = it->second->getName(); + const auto & type_in_sample_block = sample_block_column.type->getName(); + if (unlikely(type_in_sample_block != type_in_schema)) + throw TiFlashException( + fmt::format( + "the type of column `{}` in schema `{}` is difference from the one in sample block `{}`", + sample_block_column.name, + type_in_schema, + type_in_sample_block), + Errors::Planner::Internal); + } +} +} // namespace DB::FinalizeHelper diff --git a/dbms/src/Flash/Planner/FinalizeHelper.h b/dbms/src/Flash/Planner/FinalizeHelper.h new file mode 100644 index 00000000000..7e9bd3681ce --- /dev/null +++ b/dbms/src/Flash/Planner/FinalizeHelper.h @@ -0,0 +1,34 @@ +// 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. + +#pragma once + +#include +#include +#include +#include +#include + +namespace DB::FinalizeHelper +{ +void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous); + +void checkSchemaContainsParentRequire(const NamesAndTypes & schema, const Names & parent_require); + +void checkParentRequireContainsSchema(const Names & parent_require, const NamesAndTypes & schema); + +void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema); + +void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & sample_block); +} // namespace DB::FinalizeHelper diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp index a3bceee9bca..4a81740560b 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -13,9 +13,6 @@ // limitations under the License. #include -#include -#include -#include #include #include #include @@ -59,32 +56,9 @@ void PhysicalTopN::transformImpl(DAGPipeline & pipeline, Context & context, size const Settings & settings = context.getSettingsRef(); - executeExpression(pipeline, before_sort_actions, log); + executeExpression(pipeline, before_sort_actions, log, "before TopN"); - pipeline.transform([&](auto & stream) { - auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); - - /// Limits on sorting - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; - limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sorting_stream->setLimits(limits); - - stream = sorting_stream; - }); - - /// If there are several streams, we merge them into one - executeUnion(pipeline, max_streams, log); - - /// Merge the sorted blocks. - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), - order_descr, - settings.max_block_size, - limit, - settings.max_bytes_before_external_sort, - context.getTemporaryPath(), - log->identifier()); + orderStreams(pipeline, order_descr, limit, context, log); } void PhysicalTopN::finalize(const Names & parent_require) diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.h b/dbms/src/Flash/Planner/plans/PhysicalTopN.h index b92e187671f..2dfc9111712 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.h +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.h @@ -22,37 +22,37 @@ namespace DB { class PhysicalTopN : public PhysicalUnary - { - public: - static PhysicalPlanPtr build( - const Context & context, - const String & executor_id, - const String & req_id, - const tipb::TopN & top_n, - PhysicalPlanPtr child); - - PhysicalTopN( - const String & executor_id_, - const NamesAndTypes & schema_, - const String & req_id, - const SortDescription & order_descr_, - const ExpressionActionsPtr & before_sort_actions_, - size_t limit_) - : PhysicalUnary(executor_id_, PlanType::TopN, schema_, req_id) - , order_descr(order_descr_) - , before_sort_actions(before_sort_actions_) - , limit(limit_) - {} - - void finalize(const Names & parent_require) override; - - const Block & getSampleBlock() const override; - - private: - void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; - - SortDescription order_descr; - ExpressionActionsPtr before_sort_actions; - size_t limit; - }; +{ +public: + static PhysicalPlanPtr build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::TopN & top_n, + PhysicalPlanPtr child); + + PhysicalTopN( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const SortDescription & order_descr_, + const ExpressionActionsPtr & before_sort_actions_, + size_t limit_) + : PhysicalUnary(executor_id_, PlanType::TopN, schema_, req_id) + , order_descr(order_descr_) + , before_sort_actions(before_sort_actions_) + , limit(limit_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + SortDescription order_descr; + ExpressionActionsPtr before_sort_actions; + size_t limit; +}; } // namespace DB From c0e5e5bc9450c3d260bcf1c67b3d0a85eab73fcb Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 14 Jun 2022 11:31:58 +0800 Subject: [PATCH 23/57] filter --- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 8 +-- dbms/src/Flash/Planner/PlanType.cpp | 2 + dbms/src/Flash/Planner/PlanType.h | 1 + .../Flash/Planner/plans/PhysicalFilter.cpp | 71 +++++++++++++++++++ dbms/src/Flash/Planner/plans/PhysicalFilter.h | 54 ++++++++++++++ 5 files changed, 132 insertions(+), 4 deletions(-) create mode 100644 dbms/src/Flash/Planner/plans/PhysicalFilter.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalFilter.h diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 4b632de4fc6..fd495791bb9 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -163,6 +163,10 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const ::google::protobuf::RepeatedPtrField & order_by); + String buildFilterColumn( + const ExpressionActionsPtr & actions, + const std::vector & conditions); + #ifndef DBMS_PUBLIC_GTEST private: #endif @@ -281,10 +285,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const String & column_name); - String buildFilterColumn( - const ExpressionActionsPtr & actions, - const std::vector & conditions); - NamesWithAliases genNonRootFinalProjectAliases(const String & column_prefix) const; NamesWithAliases genRootFinalProjectAliases( diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index 82756e76db0..930bc495b51 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -27,6 +27,8 @@ String PlanType::toString() const return "Limit"; case TopN: return "TopN"; + case Selection: + return "Selection"; default: throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); } diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index 0256b7eba52..92ff6afeeee 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -25,6 +25,7 @@ struct PlanType Source = 0, Limit = 1, TopN = 2, + Selection = 3, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp new file mode 100644 index 00000000000..874e79246dc --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp @@ -0,0 +1,71 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalFilter::build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::Selection & selection, + PhysicalPlanPtr child) +{ + assert(child); + + DAGExpressionAnalyzer analyzer{child->getSchema(), context}; + ExpressionActionsPtr before_filter_actions = PhysicalPlanHelper::newActions(child->getSampleBlock(), context); + + std::vector conditions; + for (const auto & c : selection.conditions()) + conditions.push_back(&c); + String filter_column_name = analyzer.buildFilterColumn(before_filter_actions, conditions); + + auto physical_filter = std::make_shared(executor_id, child->getSchema(), req_id, filter_column_name, before_filter_actions); + physical_filter->appendChild(child); + return physical_filter; +} + +void PhysicalFilter::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, before_filter_actions, filter_column, log->identifier()); }); +} + +void PhysicalFilter::finalize(const Names & parent_require) +{ + Names required_output = parent_require; + required_output.push_back(filter_column); + before_filter_actions->finalize(required_output); + + child->finalize(before_filter_actions->getRequiredColumns()); + FinalizeHelper::prependProjectInputIfNeed(before_filter_actions, child->getSampleBlock().columns()); + + FinalizeHelper::checkSampleBlockContainsSchema(getSampleBlock(), schema); +} + +const Block & PhysicalFilter::getSampleBlock() const +{ + return before_filter_actions->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.h b/dbms/src/Flash/Planner/plans/PhysicalFilter.h new file mode 100644 index 00000000000..749e714d7d7 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.h @@ -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. + +#pragma once + +#include +#include +#include + +namespace DB +{ +class PhysicalFilter : public PhysicalUnary +{ +public: + static PhysicalPlanPtr build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::Selection & selection, + PhysicalPlanPtr child); + + PhysicalFilter( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const String & filter_column_, + const ExpressionActionsPtr & before_filter_actions_) + : PhysicalUnary(executor_id_, PlanType::Selection, schema_, req_id) + , filter_column(filter_column_) + , before_filter_actions(before_filter_actions_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + String filter_column; + ExpressionActionsPtr before_filter_actions; +}; +} // namespace DB From 237dd2eefef5c884071710687571832692c1b292 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 14 Jun 2022 11:59:19 +0800 Subject: [PATCH 24/57] agg --- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 38 ++-- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 11 +- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 7 + dbms/src/Flash/Planner/PlanType.cpp | 2 + dbms/src/Flash/Planner/PlanType.h | 1 + .../Planner/plans/PhysicalAggregation.cpp | 171 ++++++++++++++++++ .../Flash/Planner/plans/PhysicalAggregation.h | 67 +++++++ 7 files changed, 269 insertions(+), 28 deletions(-) create mode 100644 dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalAggregation.h diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index fd495791bb9..2d9a02a83c2 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -167,13 +167,29 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const std::vector & conditions); -#ifndef DBMS_PUBLIC_GTEST -private: -#endif + void buildAggFuncs( + const tipb::Aggregation & aggregation, + const ExpressionActionsPtr & actions, + AggregateDescriptions & aggregate_descriptions, + NamesAndTypes & aggregated_columns); + + void buildAggGroupBy( + const google::protobuf::RepeatedPtrField & group_by, + const ExpressionActionsPtr & actions, + AggregateDescriptions & aggregate_descriptions, + NamesAndTypes & aggregated_columns, + Names & aggregation_keys, + std::unordered_set & agg_key_set, + bool group_by_collation_sensitive, + TiDB::TiDBCollators & collators); + void appendCastAfterAgg( const ExpressionActionsPtr & actions, const tipb::Aggregation & agg); +#ifndef DBMS_PUBLIC_GTEST +private: +#endif String buildTupleFunctionForGroupConcat( const tipb::Expr & expr, SortDescription & sort_desc, @@ -197,22 +213,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable NamesAndTypes & aggregated_columns, bool empty_input_as_null); - void buildAggFuncs( - const tipb::Aggregation & aggregation, - const ExpressionActionsPtr & actions, - AggregateDescriptions & aggregate_descriptions, - NamesAndTypes & aggregated_columns); - - void buildAggGroupBy( - const google::protobuf::RepeatedPtrField & group_by, - const ExpressionActionsPtr & actions, - AggregateDescriptions & aggregate_descriptions, - NamesAndTypes & aggregated_columns, - Names & aggregation_keys, - std::unordered_set & agg_key_set, - bool group_by_collation_sensitive, - TiDB::TiDBCollators & collators); - void fillArgumentDetail( const ExpressionActionsPtr & actions, const tipb::Expr & arg, diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 7b9ee2f158f..9742cee9e01 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -365,10 +364,7 @@ void DAGQueryBlockInterpreter::executeAggregation( AggregateDescriptions & aggregate_descriptions, bool is_final_agg) { - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, expression_actions_ptr, log->identifier()); - stream->setExtraInfo("before aggregation"); - }); + executeExpression(pipeline, expression_actions_ptr, log, "before aggregation"); Block before_agg_header = pipeline.firstStream()->getHeader(); @@ -670,10 +666,7 @@ void DAGQueryBlockInterpreter::executeProject(DAGPipeline & pipeline, NamesWithA if (project_cols.empty()) return; ExpressionActionsPtr project = generateProjectExpressionActions(pipeline.firstStream(), context, project_cols); - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, project, log->identifier()); - stream->setExtraInfo(extra_info); - }); + executeExpression(pipeline, project, log, extra_info); } void DAGQueryBlockInterpreter::executeLimit(DAGPipeline & pipeline) diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index b72764a7372..c6be82485b9 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -32,6 +32,13 @@ void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor case tipb::ExecType::TypeTopN: cur_plans.push_back(PhysicalTopN::build(context, executor_id, log->identifier(), executor->topn(), popBack())); break; + case tipb::ExecType::TypeSelection: + cur_plans.push_back(PhysicalFilter::build(context, executor_id, log->identifier(), executor->selection(), popBack())); + break; + case tipb::ExecType::TypeAggregation: + case tipb::ExecType::TypeStreamAgg: + cur_plans.push_back(PhysicalAggregation::build(context, executor_id, log->identifier(), executor->aggregation(), popBack())); + break; default: throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); } diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index 930bc495b51..3d17c894ade 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -29,6 +29,8 @@ String PlanType::toString() const return "TopN"; case Selection: return "Selection"; + case Aggregation: + return "Aggregation"; default: throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); } diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index 92ff6afeeee..c18c72d6a50 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -26,6 +26,7 @@ struct PlanType Limit = 1, TopN = 2, Selection = 3, + Aggregation = 4, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp new file mode 100644 index 00000000000..ed1a657ee91 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -0,0 +1,171 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalAggregation::build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::Aggregation & aggregation, + PhysicalPlanPtr child) +{ + assert(child); + + if (unlikely(aggregation.group_by_size() == 0 && aggregation.agg_func_size() == 0)) + { + //should not reach here + throw TiFlashException("Aggregation executor without group by/agg exprs", Errors::Coprocessor::BadRequest); + } + + DAGExpressionAnalyzer analyzer{child->getSchema(), context}; + ExpressionActionsPtr before_agg_actions = PhysicalPlanHelper::newActions(child->getSampleBlock(), context); + NamesAndTypes aggregated_columns; + AggregateDescriptions aggregate_descriptions; + Names aggregation_keys; + TiDB::TiDBCollators collators; + { + std::unordered_set agg_key_set; + analyzer.buildAggFuncs(aggregation, before_agg_actions, aggregate_descriptions, aggregated_columns); + analyzer.buildAggGroupBy( + aggregation.group_by(), + before_agg_actions, + aggregate_descriptions, + aggregated_columns, + aggregation_keys, + agg_key_set, + AggregationInterpreterHelper::isGroupByCollationSensitive(context), + collators); + } + + auto cast_after_agg_actions = PhysicalPlanHelper::newActions(aggregated_columns, context); + analyzer.reset(aggregated_columns); + analyzer.appendCastAfterAgg(cast_after_agg_actions, aggregation); + + const NamesAndTypes & schema = analyzer.getCurrentInputColumns(); + auto physical_agg = std::make_shared( + executor_id, + schema, + req_id, + before_agg_actions, + aggregation_keys, + collators, + AggregationInterpreterHelper::isFinalAgg(aggregation), + aggregate_descriptions, + cast_after_agg_actions); + physical_agg->appendChild(child); + return physical_agg; +} + +void PhysicalAggregation::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + executeExpression(pipeline, before_agg_actions, log, "before aggregation"); + + Block before_agg_header = pipeline.firstStream()->getHeader(); + AggregationInterpreterHelper::fillArgColumnNumbers(aggregate_descriptions, before_agg_header); + auto params = AggregationInterpreterHelper::buildParams( + context, + before_agg_header, + pipeline.streams.size(), + aggregation_keys, + aggregation_collators, + aggregate_descriptions, + is_final_agg); + + /// If there are several sources, then we perform parallel aggregation + if (pipeline.streams.size() > 1) + { + const Settings & settings = context.getSettingsRef(); + BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); + pipeline.firstStream() = std::make_shared( + pipeline.streams, + stream_with_non_joined_data, + params, + context.getFileProvider(), + true, + max_streams, + settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads), + log->identifier()); + pipeline.streams.resize(1); + restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); + } + else + { + BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); + BlockInputStreams inputs; + if (!pipeline.streams.empty()) + inputs.push_back(pipeline.firstStream()); + else + pipeline.streams.resize(1); + if (stream_with_non_joined_data) + inputs.push_back(stream_with_non_joined_data); + pipeline.firstStream() = std::make_shared( + std::make_shared(inputs, log->identifier()), + params, + context.getFileProvider(), + true, + log->identifier()); + } + + executeExpression(pipeline, cast_after_agg, log, "cast after aggregation"); +} + +void PhysicalAggregation::finalize(const Names & parent_require) +{ + // schema.size() >= parent_require.size() + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); + cast_after_agg->finalize(PhysicalPlanHelper::schemaToNames(schema)); + + Names before_agg_output; + // set required output for agg funcs's arguments and group by keys. + for (const auto & aggregate_description : aggregate_descriptions) + { + for (const auto & argument_name : aggregate_description.argument_names) + before_agg_output.push_back(argument_name); + } + for (const auto & aggregation_key : aggregation_keys) + { + before_agg_output.push_back(aggregation_key); + } + + before_agg_actions->finalize(before_agg_output); + child->finalize(before_agg_actions->getRequiredColumns()); + FinalizeHelper::prependProjectInputIfNeed(before_agg_actions, child->getSampleBlock().columns()); + + FinalizeHelper::checkSampleBlockContainsSchema(getSampleBlock(), schema); +} + +const Block & PhysicalAggregation::getSampleBlock() const +{ + return cast_after_agg->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h new file mode 100644 index 00000000000..a49fc2e41ee --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h @@ -0,0 +1,67 @@ +// 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. + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ +class PhysicalAggregation : public PhysicalUnary +{ +public: + static PhysicalPlanPtr build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::Aggregation & aggregation, + PhysicalPlanPtr child); + + PhysicalAggregation( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const ExpressionActionsPtr & before_agg_actions_, + const Names & aggregation_keys_, + const TiDB::TiDBCollators & aggregation_collators_, + bool is_final_agg_, + const AggregateDescriptions & aggregate_descriptions_, + const ExpressionActionsPtr & castAfterAgg_) + : PhysicalUnary(executor_id_, PlanType::Aggregation, schema_, req_id) + , before_agg_actions(before_agg_actions_) + , aggregation_keys(aggregation_keys_) + , aggregation_collators(aggregation_collators_) + , is_final_agg(is_final_agg_) + , aggregate_descriptions(aggregate_descriptions_) + , cast_after_agg(castAfterAgg_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + ExpressionActionsPtr before_agg_actions; + Names aggregation_keys; + TiDB::TiDBCollators aggregation_collators; + bool is_final_agg; + AggregateDescriptions aggregate_descriptions; + ExpressionActionsPtr cast_after_agg; +}; +} // namespace DB From 2b858734ceb262f40b2c061c3d3d8908d980f29b Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 14 Jun 2022 13:15:16 +0800 Subject: [PATCH 25/57] ExchangeSender --- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 13 ++- dbms/src/Flash/Planner/PlanType.cpp | 4 + dbms/src/Flash/Planner/PlanType.h | 2 + .../Planner/plans/PhysicalExchangeSender.cpp | 83 +++++++++++++++++++ .../Planner/plans/PhysicalExchangeSender.h | 56 +++++++++++++ .../plans/PhysicalMockExchangeSender.cpp | 54 ++++++++++++ .../plans/PhysicalMockExchangeSender.h | 45 ++++++++++ 7 files changed, 256 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h create mode 100644 dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index c6be82485b9..c1b451540d0 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -14,9 +14,12 @@ #include #include +#include +#include #include -#include +#include #include +#include #include namespace DB @@ -39,6 +42,14 @@ void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor case tipb::ExecType::TypeStreamAgg: cur_plans.push_back(PhysicalAggregation::build(context, executor_id, log->identifier(), executor->aggregation(), popBack())); break; + case tipb::ExecType::TypeExchangeSender: + { + if (unlikely(dagContext().isTest())) + cur_plans.push_back(PhysicalMockExchangeSender::build(executor_id, log->identifier(), popBack())); + else + cur_plans.push_back(PhysicalExchangeSender::build(executor_id, log->identifier(), executor->exchange_sender(), popBack())); + break; + } default: throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); } diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index 3d17c894ade..95fa1a44e12 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -31,6 +31,10 @@ String PlanType::toString() const return "Selection"; case Aggregation: return "Aggregation"; + case ExchangeSender: + return "ExchangeSender"; + case MockExchangeSender: + return "MockExchangeSender"; default: throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); } diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index c18c72d6a50..196737d85d1 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -27,6 +27,8 @@ struct PlanType TopN = 2, Selection = 3, Aggregation = 4, + ExchangeSender = 5, + MockExchangeSender = 6, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp new file mode 100644 index 00000000000..a2347ac447a --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -0,0 +1,83 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalExchangeSender::build( + const String & executor_id, + const String & req_id, + const tipb::ExchangeSender & exchange_sender, + PhysicalPlanPtr child) +{ + assert(child); + + std::vector partition_col_ids = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender); + TiDB::TiDBCollators partition_col_collators = ExchangeSenderInterpreterHelper::genPartitionColCollators(exchange_sender); + + auto physical_exchange_sender = std::make_shared( + executor_id, + child->getSchema(), + req_id, + partition_col_ids, + partition_col_collators, + exchange_sender.tp()); + physical_exchange_sender->appendChild(child); + return physical_exchange_sender; +} + +void PhysicalExchangeSender::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + auto & dag_context = *context.getDAGContext(); + restoreConcurrency(pipeline, dag_context.final_concurrency, log); + + RUNTIME_ASSERT(dag_context.isMPPTask() && dag_context.tunnel_set != nullptr, log, "exchange_sender only run in MPP"); + + int stream_id = 0; + pipeline.transform([&](auto & stream) { + // construct writer + std::unique_ptr response_writer = std::make_unique>( + dag_context.tunnel_set, + partition_col_ids, + partition_col_collators, + exchange_type, + context.getSettingsRef().dag_records_per_chunk, + context.getSettingsRef().batch_send_min_limit, + stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response + dag_context); + stream = std::make_shared(stream, std::move(response_writer), log->identifier()); + }); +} + +void PhysicalExchangeSender::finalize(const Names & parent_require) +{ + child->finalize(parent_require); +} + +const Block & PhysicalExchangeSender::getSampleBlock() const +{ + return child->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h new file mode 100644 index 00000000000..25bcce451cf --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h @@ -0,0 +1,56 @@ +// 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. + +#pragma once + +#include +#include +#include + +namespace DB +{ +class PhysicalExchangeSender : public PhysicalUnary +{ +public: + static PhysicalPlanPtr build( + const String & executor_id, + const String & req_id, + const tipb::ExchangeSender & exchange_sender, + PhysicalPlanPtr child); + + PhysicalExchangeSender( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const std::vector & partition_col_ids_, + const TiDB::TiDBCollators & collators_, + const tipb::ExchangeType & exchange_type_) + : PhysicalUnary(executor_id_, PlanType::ExchangeSender, schema_, req_id) + , partition_col_ids(partition_col_ids_) + , partition_col_collators(collators_) + , exchange_type(exchange_type_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + std::vector partition_col_ids; + TiDB::TiDBCollators partition_col_collators; + tipb::ExchangeType exchange_type; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp new file mode 100644 index 00000000000..c7fd744c5bb --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp @@ -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. + +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalMockExchangeSender::build( + const String & executor_id, + const String & req_id, + PhysicalPlanPtr child) +{ + assert(child); + + auto physical_mock_exchange_sender = std::make_shared( + executor_id, + child->getSchema(), + req_id); + physical_mock_exchange_sender->appendChild(child); + return physical_mock_exchange_sender; +} + +void PhysicalMockExchangeSender::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, log->identifier()); }); +} + +void PhysicalMockExchangeSender::finalize(const Names & parent_require) +{ + child->finalize(parent_require); +} + +const Block & PhysicalMockExchangeSender::getSampleBlock() const +{ + return child->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h new file mode 100644 index 00000000000..cb1816941b3 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h @@ -0,0 +1,45 @@ +// 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. + +#pragma once + +#include +#include +#include + +namespace DB +{ +class PhysicalMockExchangeSender : public PhysicalUnary +{ +public: + static PhysicalPlanPtr build( + const String & executor_id, + const String & req_id, + PhysicalPlanPtr child); + + PhysicalMockExchangeSender( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id) + : PhysicalUnary(executor_id_, PlanType::MockExchangeSender, schema_, req_id) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; +}; +} // namespace DB From 09d4287e366791ea2dc88b73cbdf5b8a917da35a Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 14 Jun 2022 13:51:04 +0800 Subject: [PATCH 26/57] receiver --- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 8 ++ dbms/src/Flash/Planner/PhysicalPlanHelper.cpp | 23 +++++ dbms/src/Flash/Planner/PhysicalPlanHelper.h | 3 + dbms/src/Flash/Planner/PlanType.cpp | 4 + dbms/src/Flash/Planner/PlanType.h | 2 + .../plans/PhysicalExchangeReceiver.cpp | 91 +++++++++++++++++++ .../Planner/plans/PhysicalExchangeReceiver.h | 47 ++++++++++ .../plans/PhysicalMockExchangeReceiver.cpp | 77 ++++++++++++++++ .../plans/PhysicalMockExchangeReceiver.h | 47 ++++++++++ 9 files changed, 302 insertions(+) create mode 100644 dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h create mode 100644 dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index c1b451540d0..1db2678e5de 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -50,6 +50,14 @@ void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor cur_plans.push_back(PhysicalExchangeSender::build(executor_id, log->identifier(), executor->exchange_sender(), popBack())); break; } + case tipb::ExecType::TypeExchangeReceiver: + { + if (unlikely(dagContext().isTest())) + cur_plans.push_back(PhysicalMockExchangeReceiver::build(context, executor_id, log->identifier(), popBack(), executor->exchange_receiver())); + else + cur_plans.push_back(PhysicalExchangeReceiver::build(context, executor_id, log->identifier())); + break; + } default: throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); } diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp index 2e1589cd45c..cf289fe695a 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp @@ -30,4 +30,27 @@ ExpressionActionsPtr newActions(const Block & input_block, const Context & conte const ColumnsWithTypeAndName & actions_input_columns = input_block.getColumnsWithTypeAndName(); return std::make_shared(actions_input_columns, context.getSettingsRef()); } + +ExpressionActionsPtr newActions(const NamesAndTypes & input_columns, const Context & context) +{ + NamesAndTypesList actions_input_column; + std::unordered_set column_name_set; + for (const auto & col : input_columns) + { + if (column_name_set.find(col.name) == column_name_set.end()) + { + actions_input_column.emplace_back(col.name, col.type); + column_name_set.emplace(col.name); + } + } + return std::make_shared(actions_input_column, context.getSettingsRef()); +} + +Block constructBlockFromSchema(const NamesAndTypes & schema) +{ + ColumnsWithTypeAndName columns; + for (const auto & column : schema) + columns.emplace_back(column.type, column.name); + return Block(columns); +} } // namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.h b/dbms/src/Flash/Planner/PhysicalPlanHelper.h index 13e3a697768..2dfa3b47563 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.h +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.h @@ -25,4 +25,7 @@ Names schemaToNames(const NamesAndTypes & schema); ExpressionActionsPtr newActions(const Block & input_block, const Context & context); +ExpressionActionsPtr newActions(const NamesAndTypes & input_columns, const Context & context); + +Block constructBlockFromSchema(const NamesAndTypes & schema); } // namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index 95fa1a44e12..e420c9d11f0 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -35,6 +35,10 @@ String PlanType::toString() const return "ExchangeSender"; case MockExchangeSender: return "MockExchangeSender"; + case ExchangeReceiver: + return "ExchangeReceiver"; + case MockExchangeReceiver: + return "MockExchangeReceiver"; default: throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); } diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index 196737d85d1..fd9a9a11edf 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -29,6 +29,8 @@ struct PlanType Aggregation = 4, ExchangeSender = 5, MockExchangeSender = 6, + ExchangeReceiver = 7, + MockExchangeReceiver = 8, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp new file mode 100644 index 00000000000..7ef29f6c875 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -0,0 +1,91 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalExchangeReceiver::PhysicalExchangeReceiver( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const std::shared_ptr & mpp_exchange_receiver_) + : PhysicalLeaf(executor_id_, PlanType::ExchangeReceiver, schema_, req_id) + , mpp_exchange_receiver(mpp_exchange_receiver_) + , sample_block(PhysicalPlanHelper::constructBlockFromSchema(schema_)) +{} + +PhysicalPlanPtr PhysicalExchangeReceiver::build( + const Context & context, + const String & executor_id, + const String & req_id) +{ + const auto & mpp_exchange_receiver_map = context.getDAGContext()->getMPPExchangeReceiverMap(); + + auto it = mpp_exchange_receiver_map.find(executor_id); + if (unlikely(it == mpp_exchange_receiver_map.end())) + throw TiFlashException( + fmt::format("Can not find exchange receiver for {}", executor_id), + Errors::Planner::Internal); + + const auto & mpp_exchange_receiver = it->second; + NamesAndTypes schema; + for (const auto & col : mpp_exchange_receiver->getOutputSchema()) + { + auto tp = getDataTypeByColumnInfoForComputingLayer(col.second); + schema.emplace_back(col.first, tp); + } + auto physical_exchange_receiver = std::make_shared( + executor_id, + schema, + req_id, + mpp_exchange_receiver); + return physical_exchange_receiver; +} + +void PhysicalExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + auto & dag_context = *context.getDAGContext(); + // todo choose a more reasonable stream number + auto & exchange_receiver_io_input_streams = dag_context.getInBoundIOInputStreamsMap()[query_block.source_name]; + for (size_t i = 0; i < max_streams; ++i) + { + BlockInputStreamPtr stream = std::make_shared(it->second, log->identifier(), executor_id); + exchange_receiver_io_input_streams.push_back(stream); + stream = std::make_shared(stream, 8192, 0, log->identifier()); + stream->setExtraInfo("squashing after exchange receiver"); + pipeline.streams.push_back(stream); + } +} + +void PhysicalExchangeReceiver::finalize(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); +} + +const Block & PhysicalExchangeReceiver::getSampleBlock() const +{ + return sample_block; +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h new file mode 100644 index 00000000000..11f6063ea8c --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h @@ -0,0 +1,47 @@ +// 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. + +#pragma once + +#include + +namespace DB +{ +class ExchangeReceiver; + +class PhysicalExchangeReceiver : public PhysicalLeaf +{ +public: + static PhysicalPlanPtr build( + const Context & context, + const String & executor_id, + const String & req_id); + + PhysicalExchangeReceiver( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const std::shared_ptr & mpp_exchange_receiver_); + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + std::shared_ptr mpp_exchange_receiver; + Block sample_block; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp new file mode 100644 index 00000000000..445ae17d6c2 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp @@ -0,0 +1,77 @@ +// 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 +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalMockExchangeReceiver::PhysicalMockExchangeReceiver( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const BlockInputStreams & mock_streams_) + : PhysicalLeaf(executor_id_, PlanType::ExchangeReceiver, schema_, req_id) + , mock_streams(mock_streams_) +{} + +PhysicalPlanPtr PhysicalMockExchangeReceiver::build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::ExchangeReceiver & exchange_receiver) +{ + NamesAndTypes schema; + BlockInputStreams mock_streams; + size_t max_streams = context.getDAGContext().initialize_concurrency; + if (context.getDAGContext()->columnsForTestEmpty() || context.getDAGContext()->columnsForTest(executor_id).empty()) + { + for (size_t i = 0; i < max_streams; ++i) + // use max_block_size / 10 to determine the mock block's size + mock_streams.push_back(std::make_shared(exchange_receiver, context.getSettingsRef().max_block_size, context.getSettingsRef().max_block_size / 10)); + for (const auto & col : mock_streams.back()->getHeader()) + schema.emplace_back(col.name, col.type); + } + else + { + std::tie(schema, mock_streams) = mockSourceStream(context, max_streams, log, executor_id); + } + + auto physical_mock_exchange_receiver = std::make_shared( + executor_id, + schema, + req_id, + mock_streams); + return physical_mock_exchange_receiver; +} + +void PhysicalExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +{ + pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); +} + +void PhysicalExchangeReceiver::finalize(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); +} + +const Block & PhysicalExchangeReceiver::getSampleBlock() const +{ + return mock_streams.back()->getHeader(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h new file mode 100644 index 00000000000..e8dfba26db4 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h @@ -0,0 +1,47 @@ +// 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. + +#pragma once + +#include +#include +#include + +namespace DB +{ +class PhysicalMockExchangeReceiver : public PhysicalLeaf +{ +public: + static PhysicalPlanPtr build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::ExchangeReceiver & exchange_receiver); + + PhysicalMockExchangeReceiver( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const BlockInputStreams & mock_streams_); + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; + + BlockInputStreams mock_streams; +}; +} // namespace DB From 0c3f6b3c522f8f1d9943d15b5b6a5bd95302c953 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 14 Jun 2022 16:00:39 +0800 Subject: [PATCH 27/57] projection --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 24 ++- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 11 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 7 +- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 1 + dbms/src/Flash/Planner/PhysicalPlan.cpp | 1 + .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 25 ++- dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 2 + dbms/src/Flash/Planner/PlanType.cpp | 2 + dbms/src/Flash/Planner/PlanType.h | 1 + dbms/src/Flash/Planner/Planner.cpp | 44 ++++- .../Planner/plans/PhysicalProjection.cpp | 161 ++++++++++++++++++ .../Flash/Planner/plans/PhysicalProjection.h | 66 +++++++ 12 files changed, 331 insertions(+), 14 deletions(-) create mode 100644 dbms/src/Flash/Planner/plans/PhysicalProjection.cpp create mode 100644 dbms/src/Flash/Planner/plans/PhysicalProjection.h diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index aa269469cdb..9b765f30cc6 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -1124,30 +1124,40 @@ std::pair DAGExpressionAnalyzer::isCastRequiredForRootFinalProjec return std::make_pair(need_append_type_cast, std::move(need_append_type_cast_vec)); } -NamesWithAliases DAGExpressionAnalyzer::appendFinalProjectForRootQueryBlock( - ExpressionActionsChain & chain, +NamesWithAliases DAGExpressionAnalyzer::buildFinalProjection( + const ExpressionActionsPtr & actions, const std::vector & schema, const std::vector & output_offsets, const String & column_prefix, bool keep_session_timezone_info) { if (unlikely(output_offsets.empty())) - throw Exception("Root Query block without output_offsets", ErrorCodes::LOGICAL_ERROR); + throw Exception("DAGRequest without output_offsets", ErrorCodes::LOGICAL_ERROR); bool need_append_timezone_cast = !keep_session_timezone_info && !context.getTimezoneInfo().is_utc_timezone; auto [need_append_type_cast, need_append_type_cast_vec] = isCastRequiredForRootFinalProjection(schema, output_offsets); assert(need_append_type_cast_vec.size() == output_offsets.size()); - auto & step = initAndGetLastStep(chain); - if (need_append_timezone_cast || need_append_type_cast) { // after appendCastForRootFinalProjection, source_columns has been modified. - appendCastForRootFinalProjection(step.actions, schema, output_offsets, need_append_timezone_cast, need_append_type_cast_vec); + appendCastForRootFinalProjection(actions, schema, output_offsets, need_append_timezone_cast, need_append_type_cast_vec); } // generate project aliases from source_columns. - NamesWithAliases final_project = genRootFinalProjectAliases(column_prefix, output_offsets); + return genRootFinalProjectAliases(column_prefix, output_offsets); +} + +NamesWithAliases DAGExpressionAnalyzer::appendFinalProjectForRootQueryBlock( + ExpressionActionsChain & chain, + const std::vector & schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info) +{ + auto & step = initAndGetLastStep(chain); + + NamesWithAliases final_project = buildFinalProjection(step.actions, schema, output_offsets, column_prefix, keep_session_timezone_info); for (const auto & name : final_project) { diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 2d9a02a83c2..7506047b34f 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -108,6 +108,8 @@ class DAGExpressionAnalyzer : private boost::noncopyable ExpressionActionsChain & chain, const String & column_prefix) const; + NamesWithAliases genNonRootFinalProjectAliases(const String & column_prefix) const; + // Generate a project action for root DAGQueryBlock, // to keep the schema of Block and tidb-schema the same. NamesWithAliases appendFinalProjectForRootQueryBlock( @@ -117,6 +119,13 @@ class DAGExpressionAnalyzer : private boost::noncopyable const String & column_prefix, bool keep_session_timezone_info); + NamesWithAliases buildFinalProjection( + const ExpressionActionsPtr & actions, + const std::vector & schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info); + String getActions( const tipb::Expr & expr, const ExpressionActionsPtr & actions, @@ -285,8 +294,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const String & column_name); - NamesWithAliases genNonRootFinalProjectAliases(const String & column_prefix) const; - NamesWithAliases genRootFinalProjectAliases( const String & column_prefix, const std::vector & output_offsets) const; diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 9757ebf1aa6..9de5b83626f 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -13,9 +13,9 @@ // limitations under the License. #include -#include -#include #include +#include +#include #include #include #include @@ -110,7 +110,7 @@ void executeExpression( DAGPipeline & pipeline, const ExpressionActionsPtr & expr_actions, const LoggerPtr & log, - const String & extra_info = "") + const String & extra_info) { if (expr_actions && !expr_actions->getActions().empty()) { @@ -123,6 +123,7 @@ void executeExpression( void orderStreams( DAGPipeline & pipeline, + size_t max_streams, SortDescription order_descr, Int64 limit, const Context & context, diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 95d5ab7b53a..36280f3b903 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -54,6 +54,7 @@ void executeExpression( void orderStreams( DAGPipeline & pipeline, + size_t max_streams, SortDescription order_descr, Int64 limit, const Context & context, diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 1d2f8844041..255171dbd53 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -70,6 +70,7 @@ void PhysicalPlan::transform(DAGPipeline & pipeline, Context & context, size_t m { transformImpl(pipeline, context, max_streams); recordProfileStreams(pipeline, context); + // todo modify logic after supporting window function. context.getDAGContext()->updateFinalConcurrency(pipeline.streams.size(), max_streams); restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); } diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 1db2678e5de..082e534c8bb 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -53,16 +54,38 @@ void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor case tipb::ExecType::TypeExchangeReceiver: { if (unlikely(dagContext().isTest())) - cur_plans.push_back(PhysicalMockExchangeReceiver::build(context, executor_id, log->identifier(), popBack(), executor->exchange_receiver())); + cur_plans.push_back(PhysicalMockExchangeReceiver::build(context, executor_id, log->identifier(), executor->exchange_receiver())); else cur_plans.push_back(PhysicalExchangeReceiver::build(context, executor_id, log->identifier())); break; } + case tipb::ExecType::TypeProjection: + cur_plans.push_back(PhysicalProjection::build(context, executor_id, log->identifier(), executor->projection(), popBack())); + break; default: throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); } } +void PhysicalPlanBuilder::buildFinalProjection(const String & column_prefix, bool is_root) +{ + const auto & final_projection = is_root + ? PhysicalProjection::buildRootFinal( + context, + log->identifier(), + dagContext().output_field_types, + dagContext().output_offsets, + column_prefix, + dagContext().keep_session_timezone_info, + popBack()) + : PhysicalProjection::buildNonRootFinal( + context, + log->identifier(), + column_prefix, + popBack()); + cur_plans.push_back(final_projection); +} + DAGContext & PhysicalPlanBuilder::dagContext() const { return *context.getDAGContext(); diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index 7c3f681ee89..7d0ada09569 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -34,6 +34,8 @@ class PhysicalPlanBuilder void buildSource(const Block & sample_block); + void buildFinalProjection(const String & column_prefix, bool is_root); + PhysicalPlanPtr getResult() const { RUNTIME_ASSERT(cur_plans.size() == 1, log, "There can only be one plan output, but here are {}", cur_plans.size()); diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index e420c9d11f0..93083784091 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -39,6 +39,8 @@ String PlanType::toString() const return "ExchangeReceiver"; case MockExchangeReceiver: return "MockExchangeReceiver"; + case Projection: + return "Projection"; default: throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); } diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index fd9a9a11edf..c8f00fb4235 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -31,6 +31,7 @@ struct PlanType MockExchangeSender = 6, ExchangeReceiver = 7, MockExchangeReceiver = 8, + Projection = 9, }; PlanTypeEnum enum_value; diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 3ccfc1234d3..91c62d3529d 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -22,6 +22,44 @@ namespace DB { +namespace +{ +void analyzePhysicalPlan(PhysicalPlanBuilder & builder, const DAGQueryBlock & query_block) +{ + assert(query_block.source); + builder.build(query_block.source_name, query_block.source); + + // selection on table scan had been executed in table scan. + if (query_block.selection && !query_block.isTableScanSource()) + { + builder.build(query_block.selection_name, query_block.selection); + } + + if (query_block.aggregation) + { + builder.build(query_block.aggregation_name, query_block.aggregation); + + if (query_block.having) + { + builder.build(query_block.having_name, query_block.having); + } + } + + // TopN/Limit + if (query_block.limit_or_topn) + { + builder.build(query_block.limit_or_topn_name, query_block.limit_or_topn); + } + + builder.buildFinalProjection(query_block.qb_column_prefix, query_block.isRootQueryBlock()); + + if (query_block.exchange_sender) + { + builder.build(query_block.exchange_sender_name, query_block.exchange_sender); + } +} +} // namespace + Planner::Planner( Context & context_, const std::vector & input_streams_vec_, @@ -48,7 +86,9 @@ BlockInputStreams Planner::execute() bool Planner::isSupported(const DAGQueryBlock &) { - return false; + return query_block.source + && (query_block.source->tp() == tipb::ExecType::TypeProjection + || query_block.source->tp() == tipb::ExecType::TypeExchangeReceiver); } DAGContext & Planner::dagContext() const @@ -71,6 +111,8 @@ void Planner::executeImpl(DAGPipeline & pipeline) builder.buildSource(input_streams.back()->getHeader()); } + analyzePhysicalPlan(builder, query_block); + auto physical_plan = builder.getResult(); physical_plan = optimize(context, physical_plan); physical_plan->transform(pipeline, context, max_streams); diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp new file mode 100644 index 00000000000..87625da064d --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp @@ -0,0 +1,161 @@ +// 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalProjection::build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::Projection & projection, + PhysicalPlanPtr child) +{ + assert(child); + + DAGExpressionAnalyzer analyzer{child->getSchema(), context}; + ExpressionActionsPtr project_actions = PhysicalPlanHelper::newActions(child->getSampleBlock(), context); + + NamesAndTypes schema; + NamesWithAliases project_aliases; + UniqueNameGenerator unique_name_generator; + bool should_add_project_alias = false; + for (const auto & expr : projection.exprs()) + { + auto expr_name = analyzer.getActions(expr, project_actions); + const auto & col = project_actions->getSampleBlock().getByName(expr_name); + + String alias = unique_name_generator.toUniqueName(col.name); + project_aliases.emplace_back(col.name, alias); + should_add_project_alias |= (alias != col.name); + + schema.emplace_back(alias, col.type); + } + if (should_add_project_alias) + project_actions->add(ExpressionAction::project(project_aliases)); + + auto physical_projection = std::make_shared(executor_id, schema, req_id, project_actions); + physical_projection->appendChild(child); + return physical_projection; +} + +PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( + const Context & context, + const String & req_id, + const String & column_prefix, + PhysicalPlanPtr child) +{ + assert(child); + + DAGExpressionAnalyzer analyzer{child->getSchema(), context}; + ExpressionActionsPtr project_actions = PhysicalPlanHelper::newActions(child->getSampleBlock(), context); + auto final_project_aliases = analyzer.genNonRootFinalProjectAliases(column_prefix); + project_actions->add(ExpressionAction::project(final_project_aliases)); + + NamesAndTypes schema = child->getSchema(); + assert(final_project_aliases.size() == schema.size()); + // replace name by alias. + for (size_t i = 0; i < final_project_aliases.size(); ++i) + { + assert(schema[i].name == final_project_aliases[i].first); + schema[i].name = final_project_aliases[i].second; + } + + auto physical_projection = std::make_shared("NonRootFinalProjection", schema, req_id, project_actions); + // For final projection, no need to record profile streams. + physical_projection->disableRecordProfileStreams(); + physical_projection->appendChild(child); + return physical_projection; +} + +PhysicalPlanPtr PhysicalProjection::buildRootFinal( + const Context & context, + const String & req_id, + const std::vector & require_schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info, + const PhysicalPlanPtr & child) +{ + assert(child); + + DAGExpressionAnalyzer analyzer{child->getSchema(), context}; + ExpressionActionsPtr project_actions = PhysicalPlanHelper::newActions(child->getSampleBlock(), context); + + NamesWithAliases final_project_aliases = analyzer.buildFinalProjection( + project_actions, + require_schema, + output_offsets, + column_prefix, + keep_session_timezone_info); + + project_actions->add(ExpressionAction::project(final_project_aliases)); + + assert(final_project_aliases.size() == output_offsets.size()); + NamesAndTypes schema; + for (size_t i = 0; i < final_project_aliases.size(); ++i) + { + const auto & alias = final_project_aliases[i].second; + assert(!alias.empty()); + const auto & type = analyzer.getCurrentInputColumns()[output_offsets[i]].type; + schema.emplace_back(alias, type); + } + + auto physical_projection = std::make_shared("RootFinalProjection", schema, req_id, project_actions); + // For final projection, no need to record profile streams. + physical_projection->disableRecordProfileStreams(); + physical_projection->appendChild(child); + return physical_projection; +} + +void PhysicalProjection::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + executeExpression(pipeline, project_actions, log, "projection"); +} + +void PhysicalProjection::finalize(const Names & parent_require) +{ + // Maybe parent_require.size() > schema.size() for non-final projection. + if (parent_require.size() > schema.size()) + FinalizeHelper::checkParentRequireContainsSchema(parent_require, schema); + else + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); + project_actions->finalize(parent_require); + + child->finalize(project_actions->getRequiredColumns()); + FinalizeHelper::prependProjectInputIfNeed(project_actions, child->getSampleBlock().columns()); + + if (parent_require.size() > schema.size()) + FinalizeHelper::checkSampleBlockContainsSchema(getSampleBlock(), schema); + else + FinalizeHelper::checkSchemaContainsSampleBlock(schema, getSampleBlock()); +} + +const Block & PhysicalProjection::getSampleBlock() const +{ + return project_actions->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.h b/dbms/src/Flash/Planner/plans/PhysicalProjection.h new file mode 100644 index 00000000000..209a0a67f17 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.h @@ -0,0 +1,66 @@ +// 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. + +#pragma once + +#include +#include +#include + +namespace DB +{ +class PhysicalProjection : public PhysicalUnary +{ +public: + static PhysicalPlanPtr build( + const Context & context, + const String & executor_id, + const String & req_id, + const tipb::Projection & projection, + PhysicalPlanPtr child); + + static PhysicalPlanPtr buildNonRootFinal( + const Context & context, + const String & req_id, + const String & column_prefix, + PhysicalPlanPtr child); + + static PhysicalPlanPtr buildRootFinal( + const Context & context, + const String & req_id, + const std::vector & require_schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info, + const PhysicalPlanPtr & child); + + PhysicalProjection( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const ExpressionActionsPtr & project_actions_) + : PhysicalUnary(executor_id_, PlanType::Projection, schema_, req_id) + , project_actions(project_actions_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + ExpressionActionsPtr project_actions; +}; +} // namespace DB From 27346c7dab5586d45b8ef128214b66b800f03ccd Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 15 Jun 2022 11:10:27 +0800 Subject: [PATCH 28/57] fix --- dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 4 ++-- dbms/src/Flash/Planner/plans/PhysicalLimit.h | 4 ++-- dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 9742cee9e01..81fe2c8f713 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -419,13 +419,13 @@ void DAGQueryBlockInterpreter::executeAggregation( void DAGQueryBlockInterpreter::executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc) { - orderStreams(pipeline, sort_desc, 0, context, log); + orderStreams(pipeline, max_streams, sort_desc, 0, context, log); } void DAGQueryBlockInterpreter::executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns) { Int64 limit = query_block.limit_or_topn->topn().limit(); - orderStreams(pipeline, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit, context, log); + orderStreams(pipeline, max_streams, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit, context, log); } void DAGQueryBlockInterpreter::recordProfileStreams(DAGPipeline & pipeline, const String & key) diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.h b/dbms/src/Flash/Planner/plans/PhysicalLimit.h index 5501a9127f7..c4f62c7d9ab 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.h @@ -45,5 +45,5 @@ class PhysicalLimit : public PhysicalUnary void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; size_t limit; -} -} +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp index 4a81740560b..d05638df128 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -58,7 +58,7 @@ void PhysicalTopN::transformImpl(DAGPipeline & pipeline, Context & context, size executeExpression(pipeline, before_sort_actions, log, "before TopN"); - orderStreams(pipeline, order_descr, limit, context, log); + orderStreams(pipeline, max_streams, order_descr, limit, context, log); } void PhysicalTopN::finalize(const Names & parent_require) From 9d16d81bb332228b31d73cd41c96abdc72558b93 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 15 Jun 2022 14:11:01 +0800 Subject: [PATCH 29/57] fix --- dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp | 3 +++ dbms/src/Flash/Planner/Planner.cpp | 2 +- dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp | 4 ++-- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 082e534c8bb..9dd1b8dad82 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -15,8 +15,11 @@ #include #include #include +#include #include +#include #include +#include #include #include #include diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 91c62d3529d..3f3ad844d3b 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -84,7 +84,7 @@ BlockInputStreams Planner::execute() return pipeline.streams; } -bool Planner::isSupported(const DAGQueryBlock &) +bool Planner::isSupported(const DAGQueryBlock & query_block) { return query_block.source && (query_block.source->tp() == tipb::ExecType::TypeProjection diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp index 7ef29f6c875..e30e035bc50 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -68,10 +68,10 @@ void PhysicalExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & c { auto & dag_context = *context.getDAGContext(); // todo choose a more reasonable stream number - auto & exchange_receiver_io_input_streams = dag_context.getInBoundIOInputStreamsMap()[query_block.source_name]; + auto & exchange_receiver_io_input_streams = dag_context.getInBoundIOInputStreamsMap()[executor_id]; for (size_t i = 0; i < max_streams; ++i) { - BlockInputStreamPtr stream = std::make_shared(it->second, log->identifier(), executor_id); + BlockInputStreamPtr stream = std::make_shared(mpp_exchange_receiver, log->identifier(), executor_id); exchange_receiver_io_input_streams.push_back(stream); stream = std::make_shared(stream, 8192, 0, log->identifier()); stream->setExtraInfo("squashing after exchange receiver"); From 508e838a3a7486859370f84fa32eeeb6ce5a3525 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 10:47:32 +0800 Subject: [PATCH 30/57] fix --- .../plans/PhysicalMockExchangeReceiver.cpp | 25 ++++++++++++------- .../plans/PhysicalMockExchangeReceiver.h | 5 +++- dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 2 -- 3 files changed, 20 insertions(+), 12 deletions(-) diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp index 445ae17d6c2..3a5097c9efc 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -25,21 +26,24 @@ PhysicalMockExchangeReceiver::PhysicalMockExchangeReceiver( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const Block & sample_block_, const BlockInputStreams & mock_streams_) - : PhysicalLeaf(executor_id_, PlanType::ExchangeReceiver, schema_, req_id) + : PhysicalLeaf(executor_id_, PlanType::MockExchangeReceiver, schema_, req_id) + , sample_block(sample_block_) , mock_streams(mock_streams_) {} PhysicalPlanPtr PhysicalMockExchangeReceiver::build( - const Context & context, + Context & context, const String & executor_id, const String & req_id, const tipb::ExchangeReceiver & exchange_receiver) { NamesAndTypes schema; BlockInputStreams mock_streams; - size_t max_streams = context.getDAGContext().initialize_concurrency; - if (context.getDAGContext()->columnsForTestEmpty() || context.getDAGContext()->columnsForTest(executor_id).empty()) + auto & dag_context = *context.getDAGContext(); + size_t max_streams = dag_context.initialize_concurrency; + if (dag_context.columnsForTestEmpty() || dag_context.columnsForTest(executor_id).empty()) { for (size_t i = 0; i < max_streams; ++i) // use max_block_size / 10 to determine the mock block's size @@ -49,29 +53,32 @@ PhysicalPlanPtr PhysicalMockExchangeReceiver::build( } else { - std::tie(schema, mock_streams) = mockSourceStream(context, max_streams, log, executor_id); + auto [names_and_types, mock_exchange_streams] = mockSourceStream(context, max_streams, dag_context.log, executor_id); + schema = std::move(names_and_types); + mock_streams.insert(mock_streams.end(), mock_exchange_streams.begin(), mock_exchange_streams.end()); } auto physical_mock_exchange_receiver = std::make_shared( executor_id, schema, req_id, + PhysicalPlanHelper::constructBlockFromSchema(schema), mock_streams); return physical_mock_exchange_receiver; } -void PhysicalExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +void PhysicalMockExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) { pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); } -void PhysicalExchangeReceiver::finalize(const Names & parent_require) +void PhysicalMockExchangeReceiver::finalize(const Names & parent_require) { FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); } -const Block & PhysicalExchangeReceiver::getSampleBlock() const +const Block & PhysicalMockExchangeReceiver::getSampleBlock() const { - return mock_streams.back()->getHeader(); + return sample_block; } } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h index e8dfba26db4..afb5b2f649e 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h @@ -24,7 +24,7 @@ class PhysicalMockExchangeReceiver : public PhysicalLeaf { public: static PhysicalPlanPtr build( - const Context & context, + Context & context, const String & executor_id, const String & req_id, const tipb::ExchangeReceiver & exchange_receiver); @@ -33,6 +33,7 @@ class PhysicalMockExchangeReceiver : public PhysicalLeaf const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const Block & sample_block_, const BlockInputStreams & mock_streams_); void finalize(const Names & parent_require) override; @@ -42,6 +43,8 @@ class PhysicalMockExchangeReceiver : public PhysicalLeaf private: void transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; + Block sample_block; + BlockInputStreams mock_streams; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp index d05638df128..ac809ed4065 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -54,8 +54,6 @@ void PhysicalTopN::transformImpl(DAGPipeline & pipeline, Context & context, size { child->transform(pipeline, context, max_streams); - const Settings & settings = context.getSettingsRef(); - executeExpression(pipeline, before_sort_actions, log, "before TopN"); orderStreams(pipeline, max_streams, order_descr, limit, context, log); From b169bcacb97f43a8c5f485e66ace7b1bf8832246 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 14:29:20 +0800 Subject: [PATCH 31/57] add physical plan test --- dbms/src/Flash/CMakeLists.txt | 1 + .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 12 + dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 3 + dbms/src/Flash/Planner/PhysicalPlanHelper.h | 1 + .../src/Flash/Planner/PhysicalPlanVisitor.cpp | 54 ++++ dbms/src/Flash/Planner/PhysicalPlanVisitor.h | 34 +++ dbms/src/Flash/Planner/Planner.cpp | 8 + dbms/src/Flash/Planner/tests/CMakeLists.txt | 15 ++ .../Planner/tests/gtest_physical_plan.cpp | 190 ++++++++++++++ ...xecutor.cpp => gtest_planner_executor.cpp} | 3 + dbms/src/Flash/tests/gtest_qb_executor.cpp | 233 ++++++++++++++++++ dbms/src/TestUtils/ColumnsToTiPBExpr.cpp | 2 + dbms/src/TestUtils/ExecutorTestUtils.cpp | 6 +- dbms/src/TestUtils/ExecutorTestUtils.h | 2 + 14 files changed, 561 insertions(+), 3 deletions(-) create mode 100644 dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp create mode 100644 dbms/src/Flash/Planner/PhysicalPlanVisitor.h create mode 100644 dbms/src/Flash/Planner/tests/CMakeLists.txt create mode 100644 dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp rename dbms/src/Flash/tests/{gtest_executor.cpp => gtest_planner_executor.cpp} (99%) create mode 100644 dbms/src/Flash/tests/gtest_qb_executor.cpp diff --git a/dbms/src/Flash/CMakeLists.txt b/dbms/src/Flash/CMakeLists.txt index f250029b333..5162f04fa1b 100644 --- a/dbms/src/Flash/CMakeLists.txt +++ b/dbms/src/Flash/CMakeLists.txt @@ -27,5 +27,6 @@ target_link_libraries(flash_service dbms) if (ENABLE_TESTS) add_subdirectory(Coprocessor/tests) + add_subdirectory(Planner/tests) add_subdirectory(tests) endif () diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 9dd1b8dad82..06795214303 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -24,10 +24,22 @@ #include #include #include +#include #include namespace DB { +void PhysicalPlanBuilder::build(const tipb::DAGRequest * dag_request) +{ + traverseExecutorsReverse( + dag_request, + [&](const tipb::Executor & executor) { + assert(executor.has_executor_id()); + build(executor.executor_id(), &executor); + return true; + }); +} + void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor * executor) { assert(executor); diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index 7d0ada09569..2eddd39b7f1 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB { @@ -30,6 +31,8 @@ class PhysicalPlanBuilder , log(Logger::get("PhysicalPlanBuilder", req_id)) {} + void build(const tipb::DAGRequest * dag_request); + void build(const String & executor_id, const tipb::Executor * executor); void buildSource(const Block & sample_block); diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.h b/dbms/src/Flash/Planner/PhysicalPlanHelper.h index 2dfa3b47563..84bfacde2d8 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.h +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.h @@ -1,4 +1,5 @@ // Copyright 2022 PingCAP, Ltd. +// 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. diff --git a/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp b/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp new file mode 100644 index 00000000000..c2c37a33651 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp @@ -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. + +#include +#include + +namespace DB::PhysicalPlanVisitor +{ +namespace +{ +void addPrefix(FmtBuffer & buffer, size_t level) +{ + buffer.append(String(level, ' ')); +} + +void doVisitToString(FmtBuffer & buffer, const PhysicalPlanPtr & physical_plan, size_t level) +{ + visit(physical_plan, [&buffer, &level](const PhysicalPlanPtr & plan) { + assert(plan); + addPrefix(buffer, level); + buffer.fmtAppend("{}\n", plan->toString()); + ++level; + if (plan->childrenSize() <= 1) + { + return true; + } + else + { + for (size_t i = 0; i < plan->childrenSize(); ++i) + doVisitToString(buffer, plan->children(i), level); + return false; + } + }); +} +} // namespace + +String visitToString(const PhysicalPlanPtr & plan) +{ + FmtBuffer buffer; + doVisitToString(buffer, plan, 0); + return buffer.toString(); +} +} // namespace DB::PhysicalPlanVisitor diff --git a/dbms/src/Flash/Planner/PhysicalPlanVisitor.h b/dbms/src/Flash/Planner/PhysicalPlanVisitor.h new file mode 100644 index 00000000000..cec8d08869a --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanVisitor.h @@ -0,0 +1,34 @@ +// 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 + +namespace DB::PhysicalPlanVisitor +{ +/// visit physical plan tree and apply function. +/// f: (const PhysicalPlanPtr &) -> bool, return true to continue visit. +template +void visit(const PhysicalPlanPtr & plan, FF && f) +{ + if (f(plan)) + { + for (size_t i = 0; i < plan->childrenSize(); ++i) + { + visit(plan->children(i), std::forward(f)); + } + } +} + +String visitToString(const PhysicalPlanPtr & plan); +} // namespace DB::PhysicalPlanVisitor diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 3f3ad844d3b..8cf69c1431a 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -16,9 +16,11 @@ #include #include #include +#include #include #include #include +#include namespace DB { @@ -114,6 +116,12 @@ void Planner::executeImpl(DAGPipeline & pipeline) analyzePhysicalPlan(builder, query_block); auto physical_plan = builder.getResult(); + + LOG_FMT_DEBUG( + log, + "build physical plan tree: \n{}", + PhysicalPlanVisitor::visitToString(physical_plan)); + physical_plan = optimize(context, physical_plan); physical_plan->transform(pipeline, context, max_streams); } diff --git a/dbms/src/Flash/Planner/tests/CMakeLists.txt b/dbms/src/Flash/Planner/tests/CMakeLists.txt new file mode 100644 index 00000000000..944908dcb25 --- /dev/null +++ b/dbms/src/Flash/Planner/tests/CMakeLists.txt @@ -0,0 +1,15 @@ +# 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_directories (${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp new file mode 100644 index 00000000000..8384c702e4c --- /dev/null +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -0,0 +1,190 @@ +// 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 +#include +#include +#include +#include + +namespace DB +{ +namespace tests +{ +class PhysicalPlanTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + context.addMockTable({"test_db", "test_table"}, + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + context.addExchangeReceiver("exchange1", + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + } + + void testPhysicalPlan( + const std::shared_ptr & request, + const String & expected_physical_plan, + const String & expected_streams, + const ColumnsWithTypeAndName & expect_columns) + { + // TODO support multi-streams. + size_t max_streams = 1; + PhysicalPlanBuilder builder{context.context, log->identifier()}; + builder.build(request.get()); + auto physical_plan = builder.getResult(); + + ASSERT_EQ(Poco::trim(expected_physical_plan), Poco::trim(PhysicalPlanVisitor::visitToString(physical_plan))); + + BlockInputStreamPtr final_stream; + { + DAGPipeline pipeline; + physical_plan->transform(pipeline, context.context, max_streams); + assert(pipeline.streams.size() == 1 && pipeline.streams_with_non_joined_data.size() == 1); + final_stream = pipeline.firstStream(); + FmtBuffer fb; + final_stream->dumpTree(fb); + ASSERT_EQ(Poco::trim(expected_streams), Poco::trim(fb.toString())); + } + + readAndAssertBlock(final_stream, expect_columns); + } + +protected: + LoggerPtr log = Logger::get("PhysicalPlanTestRunner", "test_physical_plan"); +}; + +TEST_F(ExecutorTestRunner, Filter) +try +{ + auto request = context.receive("exchange1") + .filter(eq(col("s1"), col("s2"))) + .build(context); + + testPhysicalPlan( + request.get(), + "", + "", + {toNullableVec({"banana"}), + toNullableVec({"banana"})} + ); +} +CATCH + +TEST_F(ExecutorTestRunner, Limit) +try +{ + auto request = context.receive("exchange1") + .limit(1) + .build(context); + + testPhysicalPlan( + request.get(), + "", + "", + {toNullableVec({"banana"}), + toNullableVec({"apple"})} + ); +} +CATCH + +TEST_F(ExecutorTestRunner, TopN) +try +{ + auto request = context.receive("exchange1") + .topN("s2", false, 1) + .build(context); + + testPhysicalPlan( + request.get(), + "", + "", + {toNullableVec({{}}), + toNullableVec({{}})} + ); +} +CATCH + +TEST_F(ExecutorTestRunner, Aggregation) +try +{ + auto request = context.receive("exchange1") + .aggregation(Max(col("s2")), col("s1")) + .build(context); + + testPhysicalPlan( + request.get(), + "", + "", + {toNullableVec({"banana"})} + ); +} +CATCH + +TEST_F(ExecutorTestRunner, Projection) +try +{ + auto request = context.receive("exchange1") + .project({col("s1")}) + .build(context); + + testPhysicalPlan( + request.get(), + "", + "", + {toNullableVec({"banana", {}, "banana"})} + ); +} +CATCH + +TEST_F(ExecutorTestRunner, MockExchangeSender) +try +{ + auto request = context.receive("exchange1") + .exchangeSender(tipb::Hash) + .build(context); + + testPhysicalPlan( + request.get(), + "", + "", + {toNullableVec({"banana", {}, "banana"}), + toNullableVec({"apple", {}, "banana"})} + ); +} +CATCH + +TEST_F(ExecutorTestRunner, MockExchangeReceiver) +try +{ + auto request = context.receive("exchange1") + .build(context); + + testPhysicalPlan( + request.get(), + "", + "", + {toNullableVec({"banana", {}, "banana"}), + toNullableVec({"apple", {}, "banana"})} + ); +} +CATCH + +} // namespace tests +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/tests/gtest_executor.cpp b/dbms/src/Flash/tests/gtest_planner_executor.cpp similarity index 99% rename from dbms/src/Flash/tests/gtest_executor.cpp rename to dbms/src/Flash/tests/gtest_planner_executor.cpp index 64c60f14bb6..569fbfa6acd 100644 --- a/dbms/src/Flash/tests/gtest_executor.cpp +++ b/dbms/src/Flash/tests/gtest_planner_executor.cpp @@ -25,6 +25,9 @@ class ExecutorTestRunner : public DB::tests::ExecutorTest void initializeContext() override { ExecutorTest::initializeContext(); + + context.context.setSetting("enable_planner", "true"); + context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, {toNullableVec("s1", {"banana", {}, "banana"}), diff --git a/dbms/src/Flash/tests/gtest_qb_executor.cpp b/dbms/src/Flash/tests/gtest_qb_executor.cpp new file mode 100644 index 00000000000..dba7ee5f42c --- /dev/null +++ b/dbms/src/Flash/tests/gtest_qb_executor.cpp @@ -0,0 +1,233 @@ +// 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 +#include + +namespace DB +{ +namespace tests +{ +class ExecutorTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.context.setSetting("enable_planner", "false"); + + context.addMockTable({"test_db", "test_table"}, + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + context.addExchangeReceiver("exchange1", + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + + context.addExchangeReceiver("exchange_r_table", + {{"s1", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toNullableVec("s", {"banana", "banana"}), + toNullableVec("join_c", {"apple", "banana"})}); + + context.addExchangeReceiver("exchange_l_table", + {{"s1", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toNullableVec("s", {"banana", "banana"}), + toNullableVec("join_c", {"apple", "banana"})}); + + context.addMockTable({"test_db", "r_table"}, + {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toVec("s", {"banana", "banana"}), + toVec("join_c", {"apple", "banana"})}); + + context.addMockTable({"test_db", "r_table_2"}, + {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toVec("s", {"banana", "banana", "banana"}), + toVec("join_c", {"apple", "apple", "apple"})}); + + context.addMockTable({"test_db", "l_table"}, + {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toVec("s", {"banana", "banana"}), + toVec("join_c", {"apple", "banana"})}); + } +}; + +TEST_F(ExecutorTestRunner, Filter) +try +{ + auto request = context + .scan("test_db", "test_table") + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + executeStreams(request, + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); + } + + request = context.receive("exchange1") + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + executeStreams(request, + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); + } +} +CATCH + +TEST_F(ExecutorTestRunner, JoinWithTableScan) +try +{ + auto request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 5); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); + } + request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .project({"s", "join_c"}) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_4 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " project_3 | {<0, String>, <1, String>}\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + } + + request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table_2"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 4) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 4\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})}, + 2); + executeStreams(request, + {toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})}, + 3); + } +} +CATCH + +TEST_F(ExecutorTestRunner, JoinWithExchangeReceiver) +try +{ + auto request = context + .receive("exchange_l_table") + .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " exchange_receiver_0 | type:PassThrough, {<0, String>, <1, String>}\n" + " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 5); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); + } +} +CATCH + +TEST_F(ExecutorTestRunner, JoinWithTableScanAndReceiver) +try +{ + auto request = context + .scan("test_db", "l_table") + .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + } +} +CATCH + +} // namespace tests +} // namespace DB \ No newline at end of file diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp index dcf727614b1..ea19ff08dd3 100644 --- a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp @@ -36,6 +36,7 @@ void columnToTiPBExpr(tipb::Expr * expr, const ColumnWithTypeAndName column, siz if (column.column->isColumnNullable()) { auto [col, null_map] = removeNullable(column.column.get()); + (void)null_map; is_const = col->isColumnConst(); } } @@ -97,6 +98,7 @@ void columnsToTiPBExprForTiDBCast( if (type_column.column->isColumnNullable()) { auto [col, null_map] = removeNullable(type_column.column.get()); + (void)null_map; is_const = col->isColumnConst(); } } diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 67a21d12286..91c1430f7a0 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -94,8 +94,9 @@ Block mergeBlocks(Blocks blocks) actual_columns.push_back({std::move(actual_cols[i]), sample_block.getColumnsWithTypeAndName()[i].type, sample_block.getColumnsWithTypeAndName()[i].name, sample_block.getColumnsWithTypeAndName()[i].column_id}); return Block(actual_columns); } +} // namespace -void readBlock(BlockInputStreamPtr stream, const ColumnsWithTypeAndName & expect_columns) +void ExecutorTest::readAndAssertBlock(BlockInputStreamPtr stream, const ColumnsWithTypeAndName & expect_columns) { Blocks actual_blocks; Block except_block(expect_columns); @@ -108,7 +109,6 @@ void readBlock(BlockInputStreamPtr stream, const ColumnsWithTypeAndName & expect Block actual_block = mergeBlocks(actual_blocks); ASSERT_BLOCK_EQ(except_block, actual_block); } -} // namespace void ExecutorTest::executeStreams(const std::shared_ptr & request, std::unordered_map & source_columns_map, const ColumnsWithTypeAndName & expect_columns, size_t concurrency) { @@ -117,7 +117,7 @@ void ExecutorTest::executeStreams(const std::shared_ptr & requ context.context.setDAGContext(&dag_context); // Currently, don't care about regions information in tests. DAGQuerySource dag(context.context); - readBlock(executeQuery(dag, context.context, false, QueryProcessingStage::Complete).in, expect_columns); + readAndAssertBlock(executeQuery(dag, context.context, false, QueryProcessingStage::Complete).in, expect_columns); } void ExecutorTest::executeStreams(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns, size_t concurrency) diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index 977b46abbd2..56a07085e50 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -86,6 +86,8 @@ class ExecutorTest : public ::testing::Test return createColumn(v, name); } + static void readAndAssertBlock(BlockInputStreamPtr stream, const ColumnsWithTypeAndName & expect_columns); + protected: MockDAGRequestContext context; std::unique_ptr dag_context_ptr; From dcc22332e34762e97b9bfae9cf1e8e2b61554139 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 14:51:37 +0800 Subject: [PATCH 32/57] fmt --- .../Planner/tests/gtest_physical_plan.cpp | 63 +++++++++---------- 1 file changed, 28 insertions(+), 35 deletions(-) diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index 8384c702e4c..5af8d1cd618 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -62,7 +62,7 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest final_stream->dumpTree(fb); ASSERT_EQ(Poco::trim(expected_streams), Poco::trim(fb.toString())); } - + readAndAssertBlock(final_stream, expect_columns); } @@ -74,16 +74,15 @@ TEST_F(ExecutorTestRunner, Filter) try { auto request = context.receive("exchange1") - .filter(eq(col("s1"), col("s2"))) - .build(context); - + .filter(eq(col("s1"), col("s2"))) + .build(context); + testPhysicalPlan( request.get(), "", "", {toNullableVec({"banana"}), - toNullableVec({"banana"})} - ); + toNullableVec({"banana"})}); } CATCH @@ -91,16 +90,15 @@ TEST_F(ExecutorTestRunner, Limit) try { auto request = context.receive("exchange1") - .limit(1) - .build(context); - + .limit(1) + .build(context); + testPhysicalPlan( request.get(), "", "", {toNullableVec({"banana"}), - toNullableVec({"apple"})} - ); + toNullableVec({"apple"})}); } CATCH @@ -108,16 +106,15 @@ TEST_F(ExecutorTestRunner, TopN) try { auto request = context.receive("exchange1") - .topN("s2", false, 1) - .build(context); - + .topN("s2", false, 1) + .build(context); + testPhysicalPlan( request.get(), "", "", {toNullableVec({{}}), - toNullableVec({{}})} - ); + toNullableVec({{}})}); } CATCH @@ -125,15 +122,14 @@ TEST_F(ExecutorTestRunner, Aggregation) try { auto request = context.receive("exchange1") - .aggregation(Max(col("s2")), col("s1")) - .build(context); - + .aggregation(Max(col("s2")), col("s1")) + .build(context); + testPhysicalPlan( request.get(), "", "", - {toNullableVec({"banana"})} - ); + {toNullableVec({"banana"})}); } CATCH @@ -141,15 +137,14 @@ TEST_F(ExecutorTestRunner, Projection) try { auto request = context.receive("exchange1") - .project({col("s1")}) - .build(context); - + .project({col("s1")}) + .build(context); + testPhysicalPlan( request.get(), "", "", - {toNullableVec({"banana", {}, "banana"})} - ); + {toNullableVec({"banana", {}, "banana"})}); } CATCH @@ -157,16 +152,15 @@ TEST_F(ExecutorTestRunner, MockExchangeSender) try { auto request = context.receive("exchange1") - .exchangeSender(tipb::Hash) - .build(context); - + .exchangeSender(tipb::Hash) + .build(context); + testPhysicalPlan( request.get(), "", "", {toNullableVec({"banana", {}, "banana"}), - toNullableVec({"apple", {}, "banana"})} - ); + toNullableVec({"apple", {}, "banana"})}); } CATCH @@ -174,15 +168,14 @@ TEST_F(ExecutorTestRunner, MockExchangeReceiver) try { auto request = context.receive("exchange1") - .build(context); - + .build(context); + testPhysicalPlan( request.get(), "", "", {toNullableVec({"banana", {}, "banana"}), - toNullableVec({"apple", {}, "banana"})} - ); + toNullableVec({"apple", {}, "banana"})}); } CATCH From 0a1dd006231b9003e2dfe7e545dc169272a8ea51 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 14:56:38 +0800 Subject: [PATCH 33/57] fix lience --- dbms/src/Flash/Planner/PhysicalPlanHelper.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlanHelper.h b/dbms/src/Flash/Planner/PhysicalPlanHelper.h index 84bfacde2d8..2dfa3b47563 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanHelper.h +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.h @@ -1,5 +1,4 @@ // Copyright 2022 PingCAP, Ltd. -// 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. From 1bfb173bd7d24fa884adc256860e7fe18c492ecf Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 19:19:17 +0800 Subject: [PATCH 34/57] add gtest_physical_plan --- dbms/src/Debug/astToExecutor.cpp | 9 + dbms/src/Flash/Planner/ExecutorIdGenerator.h | 60 +++++ dbms/src/Flash/Planner/PhysicalPlan.cpp | 2 +- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 37 +-- dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 4 +- dbms/src/Flash/Planner/PlanType.cpp | 4 +- dbms/src/Flash/Planner/PlanType.h | 2 +- .../Flash/Planner/plans/PhysicalFilter.cpp | 1 + dbms/src/Flash/Planner/plans/PhysicalFilter.h | 2 +- .../plans/PhysicalMockExchangeReceiver.cpp | 2 + .../Planner/tests/gtest_physical_plan.cpp | 134 ++++++---- ...lanner_executor.cpp => gtest_executor.cpp} | 40 +-- dbms/src/Flash/tests/gtest_qb_executor.cpp | 233 ------------------ dbms/src/TestUtils/mockExecutor.h | 1 + 14 files changed, 220 insertions(+), 311 deletions(-) create mode 100644 dbms/src/Flash/Planner/ExecutorIdGenerator.h rename dbms/src/Flash/tests/{gtest_planner_executor.cpp => gtest_executor.cpp} (92%) delete mode 100644 dbms/src/Flash/tests/gtest_qb_executor.cpp diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index 82f894905e6..a1e9295b3f5 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -170,6 +170,7 @@ std::unordered_map func_name_to_sig({ {"cast_decimal_datetime", tipb::ScalarFuncSig::CastDecimalAsTime}, {"cast_time_datetime", tipb::ScalarFuncSig::CastTimeAsTime}, {"cast_string_datetime", tipb::ScalarFuncSig::CastStringAsTime}, + {"concat", tipb::ScalarFuncSig::Concat}, {"round_int", tipb::ScalarFuncSig::RoundInt}, {"round_uint", tipb::ScalarFuncSig::RoundInt}, {"round_dec", tipb::ScalarFuncSig::RoundDec}, @@ -455,6 +456,14 @@ void functionToPB(const DAGSchema & input, ASTFunction * func, tipb::Expr * expr ft->set_collate(collator_id); break; } + case tipb::ScalarFuncSig::Concat: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeString); + ft->set_collate(collator_id); + break; + } case tipb::ScalarFuncSig::RoundInt: case tipb::ScalarFuncSig::RoundWithFracInt: { diff --git a/dbms/src/Flash/Planner/ExecutorIdGenerator.h b/dbms/src/Flash/Planner/ExecutorIdGenerator.h new file mode 100644 index 00000000000..874543f0b36 --- /dev/null +++ b/dbms/src/Flash/Planner/ExecutorIdGenerator.h @@ -0,0 +1,60 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +class ExecutorIdGenerator +{ +public: + String generate(const tipb::Executor & executor) + { + String executor_id = executor.has_executor_id() ? executor.executor_id() : doGenerate(executor); + assert(!executor_id.empty()); + if (unlikely(ids.find(executor_id) != ids.end())) + throw TiFlashException( + fmt::format("executor id ({}) duplicate", executor_id), + Errors::Planner::Internal); + ids.insert(executor_id); + return executor_id; + } + +private: + String doGenerate(const tipb::Executor & executor) + { + assert(!executor.has_executor_id()); + switch (executor.tp()) + { + case tipb::ExecType::TypeSelection: + return fmt::format("{}_selection", ++current_id); + case tipb::ExecType::TypeProjection: + return fmt::format("{}_projection", ++current_id); + case tipb::ExecType::TypeStreamAgg: + case tipb::ExecType::TypeAggregation: + return fmt::format("{}_aggregation", ++current_id); + case tipb::ExecType::TypeTopN: + return fmt::format("{}_top_n", ++current_id); + case tipb::ExecType::TypeLimit: + return fmt::format("{}_limit", ++current_id); + case tipb::ExecType::TypeExchangeSender: + return fmt::format("{}_exchange_sender", ++current_id); + case tipb::ExecType::TypeExchangeReceiver: + return fmt::format("{}_exchange_receiver", ++current_id); + default: + throw TiFlashException( + fmt::format("Unsupported executor in DAG request: {}", executor.DebugString()), + Errors::Planner::Unimplemented); + } + } + + UInt32 current_id = 0; + + std::unordered_set ids; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 255171dbd53..76f894d609b 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -45,7 +45,7 @@ String PhysicalPlan::toString() return buffer.toString(); }; return fmt::format( - "type: {}, executor_id: {}, is_record_profile_streams: {}, schema: {}", + "<{}, {}> | is_record_profile_streams: {}, schema: {}", type.toString(), executor_id, is_record_profile_streams, diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 06795214303..a89ba07fcd7 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -31,11 +32,12 @@ namespace DB { void PhysicalPlanBuilder::build(const tipb::DAGRequest * dag_request) { + assert(dag_request); + ExecutorIdGenerator id_generator; traverseExecutorsReverse( dag_request, [&](const tipb::Executor & executor) { - assert(executor.has_executor_id()); - build(executor.executor_id(), &executor); + build(id_generator.generate(executor), &executor); return true; }); } @@ -46,36 +48,36 @@ void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor switch (executor->tp()) { case tipb::ExecType::TypeLimit: - cur_plans.push_back(PhysicalLimit::build(executor_id, log->identifier(), executor->limit(), popBack())); + pushBack(PhysicalLimit::build(executor_id, log->identifier(), executor->limit(), popBack())); break; case tipb::ExecType::TypeTopN: - cur_plans.push_back(PhysicalTopN::build(context, executor_id, log->identifier(), executor->topn(), popBack())); + pushBack(PhysicalTopN::build(context, executor_id, log->identifier(), executor->topn(), popBack())); break; case tipb::ExecType::TypeSelection: - cur_plans.push_back(PhysicalFilter::build(context, executor_id, log->identifier(), executor->selection(), popBack())); + pushBack(PhysicalFilter::build(context, executor_id, log->identifier(), executor->selection(), popBack())); break; case tipb::ExecType::TypeAggregation: case tipb::ExecType::TypeStreamAgg: - cur_plans.push_back(PhysicalAggregation::build(context, executor_id, log->identifier(), executor->aggregation(), popBack())); + pushBack(PhysicalAggregation::build(context, executor_id, log->identifier(), executor->aggregation(), popBack())); break; case tipb::ExecType::TypeExchangeSender: { if (unlikely(dagContext().isTest())) - cur_plans.push_back(PhysicalMockExchangeSender::build(executor_id, log->identifier(), popBack())); + pushBack(PhysicalMockExchangeSender::build(executor_id, log->identifier(), popBack())); else - cur_plans.push_back(PhysicalExchangeSender::build(executor_id, log->identifier(), executor->exchange_sender(), popBack())); + pushBack(PhysicalExchangeSender::build(executor_id, log->identifier(), executor->exchange_sender(), popBack())); break; } case tipb::ExecType::TypeExchangeReceiver: { if (unlikely(dagContext().isTest())) - cur_plans.push_back(PhysicalMockExchangeReceiver::build(context, executor_id, log->identifier(), executor->exchange_receiver())); + pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log->identifier(), executor->exchange_receiver())); else - cur_plans.push_back(PhysicalExchangeReceiver::build(context, executor_id, log->identifier())); + pushBack(PhysicalExchangeReceiver::build(context, executor_id, log->identifier())); break; } case tipb::ExecType::TypeProjection: - cur_plans.push_back(PhysicalProjection::build(context, executor_id, log->identifier(), executor->projection(), popBack())); + pushBack(PhysicalProjection::build(context, executor_id, log->identifier(), executor->projection(), popBack())); break; default: throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); @@ -98,7 +100,7 @@ void PhysicalPlanBuilder::buildFinalProjection(const String & column_prefix, boo log->identifier(), column_prefix, popBack()); - cur_plans.push_back(final_projection); + pushBack(final_projection); } DAGContext & PhysicalPlanBuilder::dagContext() const @@ -106,9 +108,16 @@ DAGContext & PhysicalPlanBuilder::dagContext() const return *context.getDAGContext(); } +void PhysicalPlanBuilder::pushBack(const PhysicalPlanPtr & plan) +{ + assert(plan); + cur_plans.push_back(plan); +} + PhysicalPlanPtr PhysicalPlanBuilder::popBack() { - RUNTIME_ASSERT(!cur_plans.empty(), log, "cur_plans is empty, cannot popBack"); + if (unlikely(cur_plans.empty())) + throw TiFlashException("cur_plans is empty, cannot popBack", Errors::Planner::Internal); PhysicalPlanPtr back = cur_plans.back(); cur_plans.pop_back(); return back; @@ -116,6 +125,6 @@ PhysicalPlanPtr PhysicalPlanBuilder::popBack() void PhysicalPlanBuilder::buildSource(const Block & sample_block) { - cur_plans.push_back(PhysicalSource::build(sample_block, log->identifier())); + pushBack(PhysicalSource::build(sample_block, log->identifier())); } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index 2eddd39b7f1..c7c1ca25b24 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -48,10 +48,12 @@ class PhysicalPlanBuilder private: PhysicalPlanPtr popBack(); + void pushBack(const PhysicalPlanPtr & plan); + DAGContext & dagContext() const; private: - std::vector cur_plans; + std::vector cur_plans{}; Context & context; diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp index 93083784091..e04f08d005f 100644 --- a/dbms/src/Flash/Planner/PlanType.cpp +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -27,8 +27,8 @@ String PlanType::toString() const return "Limit"; case TopN: return "TopN"; - case Selection: - return "Selection"; + case Filter: + return "Filter"; case Aggregation: return "Aggregation"; case ExchangeSender: diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h index c8f00fb4235..7cd0c28d2e8 100644 --- a/dbms/src/Flash/Planner/PlanType.h +++ b/dbms/src/Flash/Planner/PlanType.h @@ -25,7 +25,7 @@ struct PlanType Source = 0, Limit = 1, TopN = 2, - Selection = 3, + Filter = 3, Aggregation = 4, ExchangeSender = 5, MockExchangeSender = 6, diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp index 874e79246dc..d84fe9893c6 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp @@ -42,6 +42,7 @@ PhysicalPlanPtr PhysicalFilter::build( auto physical_filter = std::make_shared(executor_id, child->getSchema(), req_id, filter_column_name, before_filter_actions); physical_filter->appendChild(child); + return physical_filter; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.h b/dbms/src/Flash/Planner/plans/PhysicalFilter.h index 749e714d7d7..fc81376f6c8 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.h +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.h @@ -36,7 +36,7 @@ class PhysicalFilter : public PhysicalUnary const String & req_id, const String & filter_column_, const ExpressionActionsPtr & before_filter_actions_) - : PhysicalUnary(executor_id_, PlanType::Selection, schema_, req_id) + : PhysicalUnary(executor_id_, PlanType::Filter, schema_, req_id) , filter_column(filter_column_) , before_filter_actions(before_filter_actions_) {} diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp index 3a5097c9efc..5af89d33a18 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp @@ -57,6 +57,8 @@ PhysicalPlanPtr PhysicalMockExchangeReceiver::build( schema = std::move(names_and_types); mock_streams.insert(mock_streams.end(), mock_exchange_streams.begin(), mock_exchange_streams.end()); } + assert(!schema.empty()); + assert(!mock_streams.empty()); auto physical_mock_exchange_receiver = std::make_shared( executor_id, diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index 5af8d1cd618..5ffc83364c9 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -38,7 +38,7 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest toNullableVec("s2", {"apple", {}, "banana"})}); } - void testPhysicalPlan( + void execute( const std::shared_ptr & request, const String & expected_physical_plan, const String & expected_streams, @@ -46,8 +46,14 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest { // TODO support multi-streams. size_t max_streams = 1; + DAGContext dag_context(*request, "executor_test", max_streams); + dag_context.setColumnsForTest(context.executorIdColumnsMap()); + context.context.setDAGContext(&dag_context); + PhysicalPlanBuilder builder{context.context, log->identifier()}; + assert(request); builder.build(request.get()); + builder.buildFinalProjection("physical_plan_", true); auto physical_plan = builder.getResult(); ASSERT_EQ(Poco::trim(expected_physical_plan), Poco::trim(PhysicalPlanVisitor::visitToString(physical_plan))); @@ -56,7 +62,8 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest { DAGPipeline pipeline; physical_plan->transform(pipeline, context.context, max_streams); - assert(pipeline.streams.size() == 1 && pipeline.streams_with_non_joined_data.size() == 1); + // TODO support non-joined streams. + assert(pipeline.streams.size() == 1 && pipeline.streams_with_non_joined_data.empty()); final_stream = pipeline.firstStream(); FmtBuffer fb; final_stream->dumpTree(fb); @@ -66,118 +73,161 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest readAndAssertBlock(final_stream, expect_columns); } -protected: LoggerPtr log = Logger::get("PhysicalPlanTestRunner", "test_physical_plan"); }; -TEST_F(ExecutorTestRunner, Filter) +TEST_F(PhysicalPlanTestRunner, Filter) try { auto request = context.receive("exchange1") .filter(eq(col("s1"), col("s2"))) .build(context); - testPhysicalPlan( - request.get(), - "", - "", + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + Filter + MockExchangeReceiver)", {toNullableVec({"banana"}), toNullableVec({"banana"})}); } CATCH -TEST_F(ExecutorTestRunner, Limit) +TEST_F(PhysicalPlanTestRunner, Limit) try { auto request = context.receive("exchange1") .limit(1) .build(context); - testPhysicalPlan( - request.get(), - "", - "", + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + Limit, limit = 1 + MockExchangeReceiver)", {toNullableVec({"banana"}), toNullableVec({"apple"})}); } CATCH -TEST_F(ExecutorTestRunner, TopN) +TEST_F(PhysicalPlanTestRunner, TopN) try { auto request = context.receive("exchange1") .topN("s2", false, 1) .build(context); - testPhysicalPlan( - request.get(), - "", - "", + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + MergeSorting, limit = 1 + PartialSorting: limit = 1 + MockExchangeReceiver)", {toNullableVec({{}}), toNullableVec({{}})}); } CATCH -TEST_F(ExecutorTestRunner, Aggregation) +// agg's schema = agg funcs + agg group bys +TEST_F(PhysicalPlanTestRunner, Aggregation) try { auto request = context.receive("exchange1") .aggregation(Max(col("s2")), col("s1")) .build(context); - testPhysicalPlan( - request.get(), - "", - "", - {toNullableVec({"banana"})}); + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + Aggregating + Concat + MockExchangeReceiver)", + {toNullableVec({{}, "banana"}), + toNullableVec({{}, "banana"})}); } CATCH -TEST_F(ExecutorTestRunner, Projection) +TEST_F(PhysicalPlanTestRunner, Projection) try { auto request = context.receive("exchange1") - .project({col("s1")}) + .project({concat(col("s1"), col("s2"))}) .build(context); - testPhysicalPlan( - request.get(), - "", - "", - {toNullableVec({"banana", {}, "banana"})}); + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: + | is_record_profile_streams: true, schema: + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + Expression: + MockExchangeReceiver)", + {toNullableVec({"bananaapple", {}, "bananabanana"})}); } CATCH -TEST_F(ExecutorTestRunner, MockExchangeSender) +TEST_F(PhysicalPlanTestRunner, MockExchangeSender) try { auto request = context.receive("exchange1") .exchangeSender(tipb::Hash) .build(context); - testPhysicalPlan( - request.get(), - "", - "", + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + MockExchangeSender + MockExchangeReceiver)", {toNullableVec({"banana", {}, "banana"}), toNullableVec({"apple", {}, "banana"})}); } CATCH -TEST_F(ExecutorTestRunner, MockExchangeReceiver) +TEST_F(PhysicalPlanTestRunner, MockExchangeReceiver) try { auto request = context.receive("exchange1") .build(context); - testPhysicalPlan( - request.get(), - "", - "", + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + MockExchangeReceiver)", {toNullableVec({"banana", {}, "banana"}), toNullableVec({"apple", {}, "banana"})}); } CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_planner_executor.cpp b/dbms/src/Flash/tests/gtest_executor.cpp similarity index 92% rename from dbms/src/Flash/tests/gtest_planner_executor.cpp rename to dbms/src/Flash/tests/gtest_executor.cpp index 569fbfa6acd..1698ba20955 100644 --- a/dbms/src/Flash/tests/gtest_planner_executor.cpp +++ b/dbms/src/Flash/tests/gtest_executor.cpp @@ -25,9 +25,6 @@ class ExecutorTestRunner : public DB::tests::ExecutorTest void initializeContext() override { ExecutorTest::initializeContext(); - - context.context.setSetting("enable_planner", "true"); - context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, {toNullableVec("s1", {"banana", {}, "banana"}), @@ -62,6 +59,17 @@ class ExecutorTestRunner : public DB::tests::ExecutorTest {toVec("s", {"banana", "banana"}), toVec("join_c", {"apple", "banana"})}); } + + void executeExecutor(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns, size_t concurrency = 1) + { + // std::vector enable_planners{"true", "false"}; + std::vector enable_planners{"false"}; + for (auto enable : enable_planners) + { + context.context.setSetting("enable_planner", enable); + executeStreams(request, expect_columns, concurrency); + } + } }; TEST_F(ExecutorTestRunner, Filter) @@ -72,7 +80,7 @@ try .filter(eq(col("s1"), col("s2"))) .build(context); { - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana"}), toNullableVec({"banana"})}); } @@ -81,7 +89,7 @@ try .filter(eq(col("s1"), col("s2"))) .build(context); { - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana"}), toNullableVec({"banana"})}); } @@ -102,21 +110,21 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"}), toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"})}, 2); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"}), toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"})}, 5); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"}), toNullableVec({"banana", "banana"}), @@ -135,7 +143,7 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"})}, 2); @@ -152,13 +160,13 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana", "banana", "banana"}), toNullableVec({"apple", "apple", "apple", "banana"}), toNullableVec({"banana", "banana", "banana", {}}), toNullableVec({"apple", "apple", "apple", {}})}, 2); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana", "banana", "banana"}), toNullableVec({"apple", "apple", "apple", "banana"}), toNullableVec({"banana", "banana", "banana", {}}), @@ -182,21 +190,21 @@ try " exchange_receiver_0 | type:PassThrough, {<0, String>, <1, String>}\n" " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"}), toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"})}, 2); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"}), toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"})}, 5); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"}), toNullableVec({"banana", "banana"}), @@ -219,7 +227,7 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, + executeExecutor(request, {toNullableVec({"banana", "banana"}), toNullableVec({"apple", "banana"}), toNullableVec({"banana", "banana"}), @@ -230,4 +238,4 @@ try CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_qb_executor.cpp b/dbms/src/Flash/tests/gtest_qb_executor.cpp deleted file mode 100644 index dba7ee5f42c..00000000000 --- a/dbms/src/Flash/tests/gtest_qb_executor.cpp +++ /dev/null @@ -1,233 +0,0 @@ -// 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 -#include - -namespace DB -{ -namespace tests -{ -class ExecutorTestRunner : public DB::tests::ExecutorTest -{ -public: - void initializeContext() override - { - ExecutorTest::initializeContext(); - - context.context.setSetting("enable_planner", "false"); - - context.addMockTable({"test_db", "test_table"}, - {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, - {toNullableVec("s1", {"banana", {}, "banana"}), - toNullableVec("s2", {"apple", {}, "banana"})}); - context.addExchangeReceiver("exchange1", - {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, - {toNullableVec("s1", {"banana", {}, "banana"}), - toNullableVec("s2", {"apple", {}, "banana"})}); - - context.addExchangeReceiver("exchange_r_table", - {{"s1", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, - {toNullableVec("s", {"banana", "banana"}), - toNullableVec("join_c", {"apple", "banana"})}); - - context.addExchangeReceiver("exchange_l_table", - {{"s1", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, - {toNullableVec("s", {"banana", "banana"}), - toNullableVec("join_c", {"apple", "banana"})}); - - context.addMockTable({"test_db", "r_table"}, - {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, - {toVec("s", {"banana", "banana"}), - toVec("join_c", {"apple", "banana"})}); - - context.addMockTable({"test_db", "r_table_2"}, - {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, - {toVec("s", {"banana", "banana", "banana"}), - toVec("join_c", {"apple", "apple", "apple"})}); - - context.addMockTable({"test_db", "l_table"}, - {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, - {toVec("s", {"banana", "banana"}), - toVec("join_c", {"apple", "banana"})}); - } -}; - -TEST_F(ExecutorTestRunner, Filter) -try -{ - auto request = context - .scan("test_db", "test_table") - .filter(eq(col("s1"), col("s2"))) - .build(context); - { - executeStreams(request, - {toNullableVec({"banana"}), - toNullableVec({"banana"})}); - } - - request = context.receive("exchange1") - .filter(eq(col("s1"), col("s2"))) - .build(context); - { - executeStreams(request, - {toNullableVec({"banana"}), - toNullableVec({"banana"})}); - } -} -CATCH - -TEST_F(ExecutorTestRunner, JoinWithTableScan) -try -{ - auto request = context - .scan("test_db", "l_table") - .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " table_scan_1 | {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 5); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}); - } - request = context - .scan("test_db", "l_table") - .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .project({"s", "join_c"}) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_4 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " project_3 | {<0, String>, <1, String>}\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " table_scan_1 | {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); - } - - request = context - .scan("test_db", "l_table") - .join(context.scan("test_db", "r_table_2"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 4) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 4\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " table_scan_1 | {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})}, - 2); - executeStreams(request, - {toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})}, - 3); - } -} -CATCH - -TEST_F(ExecutorTestRunner, JoinWithExchangeReceiver) -try -{ - auto request = context - .receive("exchange_l_table") - .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " exchange_receiver_0 | type:PassThrough, {<0, String>, <1, String>}\n" - " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 5); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}); - } -} -CATCH - -TEST_F(ExecutorTestRunner, JoinWithTableScanAndReceiver) -try -{ - auto request = context - .scan("test_db", "l_table") - .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); - } -} -CATCH - -} // namespace tests -} // namespace DB \ No newline at end of file diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 88d98158b74..95551cdfc9e 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -166,6 +166,7 @@ ASTPtr buildOrderByItemList(MockOrderByItems order_by_items); #define col(name) buildColumn((name)) #define lit(field) buildLiteral((field)) +#define concat(expr1, expr2) makeASTFunction("concat", (expr1), (expr2)) #define eq(expr1, expr2) makeASTFunction("equals", (expr1), (expr2)) #define Not_eq(expr1, expr2) makeASTFunction("notEquals", (expr1), (expr2)) #define lt(expr1, expr2) makeASTFunction("less", (expr1), (expr2)) From 95e832eae6e4194f0be512935ce61010aebe5585 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 19:28:58 +0800 Subject: [PATCH 35/57] fmt --- dbms/src/Flash/Planner/ExecutorIdGenerator.h | 14 +++ dbms/src/Flash/tests/gtest_executor.cpp | 103 +++++++++---------- 2 files changed, 65 insertions(+), 52 deletions(-) diff --git a/dbms/src/Flash/Planner/ExecutorIdGenerator.h b/dbms/src/Flash/Planner/ExecutorIdGenerator.h index 874543f0b36..2a600f447f1 100644 --- a/dbms/src/Flash/Planner/ExecutorIdGenerator.h +++ b/dbms/src/Flash/Planner/ExecutorIdGenerator.h @@ -1,3 +1,17 @@ +// 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. + #pragma once #include diff --git a/dbms/src/Flash/tests/gtest_executor.cpp b/dbms/src/Flash/tests/gtest_executor.cpp index 1698ba20955..49512b9271f 100644 --- a/dbms/src/Flash/tests/gtest_executor.cpp +++ b/dbms/src/Flash/tests/gtest_executor.cpp @@ -62,8 +62,7 @@ class ExecutorTestRunner : public DB::tests::ExecutorTest void executeExecutor(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns, size_t concurrency = 1) { - // std::vector enable_planners{"true", "false"}; - std::vector enable_planners{"false"}; + std::vector enable_planners{"true", "false"}; for (auto enable : enable_planners) { context.context.setSetting("enable_planner", enable); @@ -81,8 +80,8 @@ try .build(context); { executeExecutor(request, - {toNullableVec({"banana"}), - toNullableVec({"banana"})}); + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); } request = context.receive("exchange1") @@ -90,8 +89,8 @@ try .build(context); { executeExecutor(request, - {toNullableVec({"banana"}), - toNullableVec({"banana"})}); + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); } } CATCH @@ -111,24 +110,24 @@ try " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 5); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 5); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); } request = context .scan("test_db", "l_table") @@ -144,9 +143,9 @@ try " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); } request = context @@ -161,17 +160,17 @@ try " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); executeExecutor(request, - {toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})}, - 2); + {toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})}, + 2); executeExecutor(request, - {toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})}, - 3); + {toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})}, + 3); } } CATCH @@ -191,24 +190,24 @@ try " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 5); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 5); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); } } CATCH @@ -228,11 +227,11 @@ try " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); executeExecutor(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); } } CATCH From 9f908d19abb27a5bf763a92fa0569ff29d323d21 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 19:30:47 +0800 Subject: [PATCH 36/57] revert --- dbms/src/TestUtils/ColumnsToTiPBExpr.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp index ea19ff08dd3..dcf727614b1 100644 --- a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp @@ -36,7 +36,6 @@ void columnToTiPBExpr(tipb::Expr * expr, const ColumnWithTypeAndName column, siz if (column.column->isColumnNullable()) { auto [col, null_map] = removeNullable(column.column.get()); - (void)null_map; is_const = col->isColumnConst(); } } @@ -98,7 +97,6 @@ void columnsToTiPBExprForTiDBCast( if (type_column.column->isColumnNullable()) { auto [col, null_map] = removeNullable(type_column.column.get()); - (void)null_map; is_const = col->isColumnConst(); } } From 840c9f760b079122b78152ca82ebad85cca715ee Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 20 Jun 2022 20:28:22 +0800 Subject: [PATCH 37/57] fix executor test --- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 1 + .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 13 +++++-- dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 8 ++--- dbms/src/Flash/Planner/Planner.cpp | 6 ++-- .../Flash/Planner/plans/PhysicalSource.cpp | 36 +++++++++++++++++++ dbms/src/Flash/Planner/plans/PhysicalSource.h | 18 +++++----- 6 files changed, 60 insertions(+), 22 deletions(-) create mode 100644 dbms/src/Flash/Planner/plans/PhysicalSource.cpp diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 570271ec93b..5ca8d26758f 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -68,6 +68,7 @@ BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block) } if (context.getSettingsRef().enable_planner && Planner::isSupported(query_block)) { + LOG_FMT_DEBUG(dagContext().log, "use planer for query block with source {}", query_block.source_name); Planner planner( context, input_streams_vec, diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index a89ba07fcd7..46c18d441b5 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -123,8 +124,16 @@ PhysicalPlanPtr PhysicalPlanBuilder::popBack() return back; } -void PhysicalPlanBuilder::buildSource(const Block & sample_block) +void PhysicalPlanBuilder::buildSource(const BlockInputStreams & source_streams) { - pushBack(PhysicalSource::build(sample_block, log->identifier())); + pushBack(PhysicalSource::build(source_streams, log->identifier())); +} + +PhysicalPlanPtr PhysicalPlanBuilder::getResult() const +{ + RUNTIME_ASSERT(cur_plans.size() == 1, log, "There can only be one plan output, but here are {}", cur_plans.size()); + PhysicalPlanPtr physical_plan = cur_plans.back(); + physical_plan = optimize(context, physical_plan); + return physical_plan; } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index c7c1ca25b24..cb3a580a37f 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -35,15 +35,11 @@ class PhysicalPlanBuilder void build(const String & executor_id, const tipb::Executor * executor); - void buildSource(const Block & sample_block); + void buildSource(const BlockInputStreams & source_streams); void buildFinalProjection(const String & column_prefix, bool is_root); - PhysicalPlanPtr getResult() const - { - RUNTIME_ASSERT(cur_plans.size() == 1, log, "There can only be one plan output, but here are {}", cur_plans.size()); - return cur_plans.back(); - } + PhysicalPlanPtr getResult() const; private: PhysicalPlanPtr popBack(); diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 8cf69c1431a..2863ff3a5af 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include @@ -110,7 +109,7 @@ void Planner::executeImpl(DAGPipeline & pipeline) for (const auto & input_streams : input_streams_vec) { RUNTIME_ASSERT(!input_streams.empty(), log, "input streams cannot be empty"); - builder.buildSource(input_streams.back()->getHeader()); + builder.buildSource(input_streams); } analyzePhysicalPlan(builder, query_block); @@ -119,10 +118,9 @@ void Planner::executeImpl(DAGPipeline & pipeline) LOG_FMT_DEBUG( log, - "build physical plan tree: \n{}", + "build physical plan: \n{}", PhysicalPlanVisitor::visitToString(physical_plan)); - physical_plan = optimize(context, physical_plan); physical_plan->transform(pipeline, context, max_streams); } } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.cpp b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp new file mode 100644 index 00000000000..0d28be47342 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp @@ -0,0 +1,36 @@ +// 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 +#include + +namespace DB +{ +PhysicalPlanPtr PhysicalSource::build( + const BlockInputStreams & source_streams, + const String & req_id) +{ + assert(!source_streams.empty()); + Block sample_block = source_streams.back()->getHeader(); + NamesAndTypes schema; + for (const auto & col : sample_block) + schema.emplace_back(col.name, col.type); + return std::make_shared("source", schema, req_id, sample_block, source_streams); +} + +void PhysicalSource::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +{ + pipeline.streams.insert(pipeline.streams.end(), source_streams.begin(), source_streams.end()); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.h b/dbms/src/Flash/Planner/plans/PhysicalSource.h index 6b6837de107..ff6f6c3135d 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalSource.h +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.h @@ -23,27 +23,23 @@ class PhysicalSource : public PhysicalLeaf { public: static PhysicalPlanPtr build( - const Block & sample_block, - const String & req_id) - { - NamesAndTypes schema; - for (const auto & col : sample_block) - schema.emplace_back(col.name, col.type); - return std::make_shared("source", schema, sample_block, req_id); - } + const BlockInputStreams & source_streams, + const String & req_id); PhysicalSource( const String & executor_id_, const NamesAndTypes & schema_, + const String & req_id, const Block & sample_block_, - const String & req_id) + const BlockInputStreams & source_streams_) : PhysicalLeaf(executor_id_, PlanType::Source, schema_, req_id) , sample_block(sample_block_) + , source_streams(source_streams_) { is_record_profile_streams = false; } - void transformImpl(DAGPipeline &, Context &, size_t) override {} + void transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; void finalize(const Names &) override {} @@ -51,5 +47,7 @@ class PhysicalSource : public PhysicalLeaf private: Block sample_block; + + BlockInputStreams source_streams; }; } // namespace DB From 40009eb845acab0354b47432b2a99217bc12f6f6 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 06:04:13 +0800 Subject: [PATCH 38/57] format --- dbms/src/Flash/Planner/plans/PhysicalSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.cpp b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp index 0d28be47342..f3a3cb848c6 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalSource.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp @@ -29,7 +29,7 @@ PhysicalPlanPtr PhysicalSource::build( return std::make_shared("source", schema, req_id, sample_block, source_streams); } -void PhysicalSource::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +void PhysicalSource::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) { pipeline.streams.insert(pipeline.streams.end(), source_streams.begin(), source_streams.end()); } From 0408cd2d243b2d767ae692e1386e59054de1cc29 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 06:51:06 +0800 Subject: [PATCH 39/57] fix tests --- .../Planner/plans/PhysicalProjection.cpp | 8 +- .../Flash/Planner/plans/PhysicalProjection.h | 4 + .../Flash/tests/gtest_planner_interpreter.cpp | 225 +++++++---------- dbms/src/Flash/tests/gtest_qb_interpreter.cpp | 226 +++++++----------- 4 files changed, 186 insertions(+), 277 deletions(-) diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp index 87625da064d..656ba1d182a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp @@ -55,7 +55,7 @@ PhysicalPlanPtr PhysicalProjection::build( if (should_add_project_alias) project_actions->add(ExpressionAction::project(project_aliases)); - auto physical_projection = std::make_shared(executor_id, schema, req_id, project_actions); + auto physical_projection = std::make_shared(executor_id, schema, req_id, "projection", project_actions); physical_projection->appendChild(child); return physical_projection; } @@ -82,7 +82,7 @@ PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( schema[i].name = final_project_aliases[i].second; } - auto physical_projection = std::make_shared("NonRootFinalProjection", schema, req_id, project_actions); + auto physical_projection = std::make_shared("NonRootFinalProjection", schema, req_id, "final projection", project_actions); // For final projection, no need to record profile streams. physical_projection->disableRecordProfileStreams(); physical_projection->appendChild(child); @@ -122,7 +122,7 @@ PhysicalPlanPtr PhysicalProjection::buildRootFinal( schema.emplace_back(alias, type); } - auto physical_projection = std::make_shared("RootFinalProjection", schema, req_id, project_actions); + auto physical_projection = std::make_shared("RootFinalProjection", schema, req_id, "final projection", project_actions); // For final projection, no need to record profile streams. physical_projection->disableRecordProfileStreams(); physical_projection->appendChild(child); @@ -133,7 +133,7 @@ void PhysicalProjection::transformImpl(DAGPipeline & pipeline, Context & context { child->transform(pipeline, context, max_streams); - executeExpression(pipeline, project_actions, log, "projection"); + executeExpression(pipeline, project_actions, log, extra_info); } void PhysicalProjection::finalize(const Names & parent_require) diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.h b/dbms/src/Flash/Planner/plans/PhysicalProjection.h index 209a0a67f17..19d5bd1ea17 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.h @@ -49,8 +49,10 @@ class PhysicalProjection : public PhysicalUnary const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const String & extra_info_, const ExpressionActionsPtr & project_actions_) : PhysicalUnary(executor_id_, PlanType::Projection, schema_, req_id) + , extra_info(extra_info_) , project_actions(project_actions_) {} @@ -61,6 +63,8 @@ class PhysicalProjection : public PhysicalUnary private: void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + const String extra_info; + ExpressionActionsPtr project_actions; }; } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp index acb5ae0d2c9..c0611f3d31e 100644 --- a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -105,15 +105,11 @@ try Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + Expression: + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -127,16 +123,13 @@ Union: Union: Expression x 10: Expression: - Expression: + Expression: SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -151,23 +144,17 @@ Union: String expected = R"( Union: Expression x 10: - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -184,33 +171,25 @@ Union: { String expected = R"( Union: - SharedQuery x 10: - Limit, limit = 10 - Union: - Limit x 10, limit = 10 - Expression: + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 Expression: - Expression: - Expression: - Expression: - Filter: - Expression: - Expression: + Expression: + Filter + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: Expression: SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -265,15 +244,11 @@ CreatingSets Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; + Expression: + Expression: + Expression: + Expression: + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -289,15 +264,11 @@ Union: MockExchangeSender x 10 Expression: Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; + Expression: + Expression: + Expression: + Expression: + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -418,19 +389,15 @@ Union: { String expected = R"( Expression: - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); expected = R"( Union: Expression x 5: - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } @@ -442,8 +409,7 @@ Union: Expression: Aggregating Concat - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); expected = R"( @@ -451,8 +417,7 @@ Union: Expression x 5: SharedQuery: ParallelAggregating, max_threads: 5, final: true - Expression x 5: - MockTableScan)"; + MockTableScan x 5)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } @@ -510,26 +475,20 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Limit, limit = 10 - Union: - Limit x 10, limit = 10 - Expression: - MockTableScan)"; + SharedQuery x 10: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"(Expression: Aggregating Concat - Expression: - Expression: - Expression: - Limit, limit = 10 - Expression: - MockTableScan)"; + Limit, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -544,28 +503,22 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - MockTableScan)"; + SharedQuery x 10: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( Expression: Aggregating Concat - Expression: - Expression: - Expression: - Expression: - MergeSorting, limit = 10 - PartialSorting: limit = 10 - MockTableScan)"; + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -580,28 +533,22 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - MockTableScan)"; + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + MockTableScan x 10)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( Expression: Aggregating Concat - Expression: - Expression: - Expression: - Expression: - Aggregating - Concat - Expression: - MockTableScan)"; + Expression: + Expression: + Aggregating + Concat + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -616,8 +563,7 @@ Union: Expression: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - MockTableScan)"; + MockTableScan x 10)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( @@ -625,8 +571,7 @@ MockExchangeSender Expression: Aggregating Concat - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -706,4 +651,4 @@ CreatingSets CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_qb_interpreter.cpp b/dbms/src/Flash/tests/gtest_qb_interpreter.cpp index c8ac422fdb3..9c4c15857d3 100644 --- a/dbms/src/Flash/tests/gtest_qb_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_qb_interpreter.cpp @@ -105,15 +105,12 @@ try Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -127,16 +124,14 @@ Union: Union: Expression x 10: Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -152,22 +147,18 @@ Union: Union: Expression x 10: Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -190,27 +181,22 @@ Union: Limit x 10, limit = 10 Expression: Expression: - Expression: - Expression: - Expression: - Filter: - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + Expression: + Filter: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -265,15 +251,12 @@ CreatingSets Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -289,15 +272,12 @@ Union: MockExchangeSender x 10 Expression: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -419,18 +399,16 @@ Union: String expected = R"( Expression: Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); expected = R"( Union: Expression x 5: Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } @@ -442,8 +420,7 @@ Union: Expression: Aggregating Concat - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); expected = R"( @@ -451,8 +428,7 @@ Union: Expression x 5: SharedQuery: ParallelAggregating, max_threads: 5, final: true - Expression x 5: - MockTableScan)"; + MockTableScan x 5)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } @@ -510,26 +486,22 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Limit, limit = 10 - Union: - Limit x 10, limit = 10 - Expression: - MockTableScan)"; + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"(Expression: Aggregating Concat - Expression: - Expression: - Expression: - Limit, limit = 10 - Expression: - MockTableScan)"; + Expression: + Limit, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -544,28 +516,24 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - MockTableScan)"; + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( Expression: Aggregating Concat - Expression: - Expression: - Expression: - Expression: - MergeSorting, limit = 10 - PartialSorting: limit = 10 - MockTableScan)"; + Expression: + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -580,28 +548,22 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - MockTableScan)"; + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + MockTableScan x 10)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( Expression: Aggregating Concat - Expression: - Expression: - Expression: - Expression: - Aggregating - Concat - Expression: - MockTableScan)"; + Expression: + Expression: + Aggregating + Concat + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -616,8 +578,7 @@ Union: Expression: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - MockTableScan)"; + MockTableScan x 10)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( @@ -625,8 +586,7 @@ MockExchangeSender Expression: Aggregating Concat - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -706,4 +666,4 @@ CreatingSets CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB From 2975694a7573b77eaa276c6b65c0b199ab856c22 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 10:56:40 +0800 Subject: [PATCH 40/57] fix ut --- .../Flash/Planner/tests/gtest_physical_plan.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index 5ffc83364c9..c8eceb87bbc 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -90,7 +90,7 @@ try | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: Filter MockExchangeReceiver)", {toNullableVec({"banana"}), @@ -112,7 +112,7 @@ try | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: Limit, limit = 1 MockExchangeReceiver)", {toNullableVec({"banana"}), @@ -134,7 +134,7 @@ try | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: MergeSorting, limit = 1 PartialSorting: limit = 1 MockExchangeReceiver)", @@ -158,7 +158,7 @@ try | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: Aggregating Concat MockExchangeReceiver)", @@ -181,7 +181,7 @@ try | is_record_profile_streams: true, schema: | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: Expression: MockExchangeReceiver)", {toNullableVec({"bananaapple", {}, "bananabanana"})}); @@ -202,7 +202,7 @@ try | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: MockExchangeSender MockExchangeReceiver)", {toNullableVec({"banana", {}, "banana"}), @@ -222,7 +222,7 @@ try | is_record_profile_streams: false, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: MockExchangeReceiver)", {toNullableVec({"banana", {}, "banana"}), toNullableVec({"apple", {}, "banana"})}); From b00a8749324be1fc420f24ce092806bb176b3674 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 14:21:21 +0800 Subject: [PATCH 41/57] fix projection bug --- .../DataStreams/TiRemoteBlockInputStream.h | 6 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 10 ++ dbms/src/Flash/Coprocessor/DAGUtils.h | 4 +- dbms/src/Flash/Planner/FinalizeHelper.cpp | 42 +++++- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 7 + .../plans/PhysicalExchangeReceiver.cpp | 11 +- .../Planner/plans/PhysicalExchangeReceiver.h | 4 +- .../Planner/plans/PhysicalProjection.cpp | 19 +-- .../Flash/tests/gtest_planner_interpreter.cpp | 120 ++++++++++-------- 9 files changed, 139 insertions(+), 84 deletions(-) diff --git a/dbms/src/DataStreams/TiRemoteBlockInputStream.h b/dbms/src/DataStreams/TiRemoteBlockInputStream.h index f249bf1a0dc..de06bbf41d6 100644 --- a/dbms/src/DataStreams/TiRemoteBlockInputStream.h +++ b/dbms/src/DataStreams/TiRemoteBlockInputStream.h @@ -178,11 +178,9 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream { // generate sample block ColumnsWithTypeAndName columns; - for (auto & dag_col : remote_reader->getOutputSchema()) + for (const NameAndTypePair & col : toNamesAndTypes(remote_reader->getOutputSchema())) { - auto tp = getDataTypeByColumnInfoForComputingLayer(dag_col.second); - ColumnWithTypeAndName col(tp, dag_col.first); - columns.emplace_back(col); + columns.emplace_back(col.type, col.name); } for (size_t i = 0; i < source_num; i++) { diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 87f58131c8c..1b95c708f82 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1437,4 +1437,14 @@ tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) return func_name_sig_map[name]; } +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema) +{ + NamesAndTypes names_and_types; + for (const auto & col : dag_schema) + { + auto tp = getDataTypeByColumnInfoForComputingLayer(col.second); + names_and_types.emplace_back(col.first, tp); + } + return names_and_types; +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 5776edf0098..136a1b26b06 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -94,7 +95,7 @@ class UniqueNameGenerator while (it != existing_name_map.end()) { ret_name.append("_").append(std::to_string(it->second)); - it->second++; + ++it->second; it = existing_name_map.find(ret_name); } existing_name_map.try_emplace(ret_name, 1); @@ -106,4 +107,5 @@ tipb::DAGRequest getDAGRequestFromStringWithRetry(const String & s); tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context); tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name); +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); } // namespace DB diff --git a/dbms/src/Flash/Planner/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp index 31f21b3dc8e..b9c00819f89 100644 --- a/dbms/src/Flash/Planner/FinalizeHelper.cpp +++ b/dbms/src/Flash/Planner/FinalizeHelper.cpp @@ -25,6 +25,40 @@ namespace DB::FinalizeHelper { +namespace +{ +String namesToString(const Names & names) +{ + return fmt::format("{{{}}}", fmt::join(names, ",")); +} + +String schemaToString(const NamesAndTypes & schema) +{ + FmtBuffer bf; + bf.append("{"); + bf.joinStr( + schema.cbegin(), + schema.cend(), + [](const auto & col, FmtBuffer & fb) { fb.fmtAppend("<{}, {}>", col.name, col.type->getName()); }, + ", "); + bf.append("}"); + return bf.toString(); +} + +String blockToString(const Block & block) +{ + FmtBuffer bf; + bf.append("{"); + bf.joinStr( + block.cbegin(), + block.cend(), + [](const ColumnWithTypeAndName & col, FmtBuffer & fb) { fb.fmtAppend("<{}, {}>", col.name, col.type->getName()); }, + ", "); + bf.append("}"); + return bf.toString(); +} +} + void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous) { if (!actions->getRequiredColumnsWithTypes().empty() @@ -43,7 +77,7 @@ void checkSchemaContainsParentRequire(const NamesAndTypes & schema, const Names { if (unlikely(schema_set.find(parent_require_column) == schema_set.end())) throw TiFlashException( - fmt::format("schema don't contain parent require column: {}", parent_require_column), + fmt::format("schema {} don't contain parent require column: {}", schemaToString(schema), parent_require_column), Errors::Planner::Internal); } } @@ -57,7 +91,7 @@ void checkParentRequireContainsSchema(const Names & parent_require, const NamesA { if (unlikely(parent_require_set.find(schema_column.name) == parent_require_set.end())) throw TiFlashException( - fmt::format("parent require don't contain schema column: {}", schema_column.name), + fmt::format("parent require {} don't contain schema column: {}", namesToString(parent_require), schema_column.name), Errors::Planner::Internal); } } @@ -68,7 +102,7 @@ void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTy { if (unlikely(!sample_block.has(schema_column.name))) throw TiFlashException( - fmt::format("sample block don't contain schema column: {}", schema_column.name), + fmt::format("sample block {} don't contain schema column: {}", blockToString(sample_block), schema_column.name), Errors::Planner::Internal); const auto & type_in_sample_block = sample_block.getByName(schema_column.name).type->getName(); @@ -94,7 +128,7 @@ void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & auto it = schema_map.find(sample_block_column.name); if (unlikely(it == schema_map.end())) throw TiFlashException( - fmt::format("schema don't contain sample block column: {}", sample_block_column.name), + fmt::format("schema {} don't contain sample block column: {}", schemaToString(schema), sample_block_column.name), Errors::Planner::Internal); const auto & type_in_schema = it->second->getName(); diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 46c18d441b5..2f56ac3c72f 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -133,6 +134,12 @@ PhysicalPlanPtr PhysicalPlanBuilder::getResult() const { RUNTIME_ASSERT(cur_plans.size() == 1, log, "There can only be one plan output, but here are {}", cur_plans.size()); PhysicalPlanPtr physical_plan = cur_plans.back(); + + LOG_FMT_DEBUG( + log, + "build unoptimized physical plan: \n{}", + PhysicalPlanVisitor::visitToString(physical_plan)); + physical_plan = optimize(context, physical_plan); return physical_plan; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp index e30e035bc50..31c17cc7288 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -30,10 +30,11 @@ PhysicalExchangeReceiver::PhysicalExchangeReceiver( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const Block & sample_block_, const std::shared_ptr & mpp_exchange_receiver_) : PhysicalLeaf(executor_id_, PlanType::ExchangeReceiver, schema_, req_id) + , sample_block(sample_block_) , mpp_exchange_receiver(mpp_exchange_receiver_) - , sample_block(PhysicalPlanHelper::constructBlockFromSchema(schema_)) {} PhysicalPlanPtr PhysicalExchangeReceiver::build( @@ -50,16 +51,12 @@ PhysicalPlanPtr PhysicalExchangeReceiver::build( Errors::Planner::Internal); const auto & mpp_exchange_receiver = it->second; - NamesAndTypes schema; - for (const auto & col : mpp_exchange_receiver->getOutputSchema()) - { - auto tp = getDataTypeByColumnInfoForComputingLayer(col.second); - schema.emplace_back(col.first, tp); - } + NamesAndTypes schema = toNamesAndTypes(mpp_exchange_receiver->getOutputSchema()); auto physical_exchange_receiver = std::make_shared( executor_id, schema, req_id, + PhysicalPlanHelper::constructBlockFromSchema(schema), mpp_exchange_receiver); return physical_exchange_receiver; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h index 11f6063ea8c..7d726275d87 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h @@ -32,6 +32,7 @@ class PhysicalExchangeReceiver : public PhysicalLeaf const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const Block & sample_block_, const std::shared_ptr & mpp_exchange_receiver_); void finalize(const Names & parent_require) override; @@ -41,7 +42,8 @@ class PhysicalExchangeReceiver : public PhysicalLeaf private: void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; - std::shared_ptr mpp_exchange_receiver; Block sample_block; + + std::shared_ptr mpp_exchange_receiver; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp index 656ba1d182a..205220a94db 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp @@ -40,7 +40,6 @@ PhysicalPlanPtr PhysicalProjection::build( NamesAndTypes schema; NamesWithAliases project_aliases; UniqueNameGenerator unique_name_generator; - bool should_add_project_alias = false; for (const auto & expr : projection.exprs()) { auto expr_name = analyzer.getActions(expr, project_actions); @@ -48,12 +47,11 @@ PhysicalPlanPtr PhysicalProjection::build( String alias = unique_name_generator.toUniqueName(col.name); project_aliases.emplace_back(col.name, alias); - should_add_project_alias |= (alias != col.name); - schema.emplace_back(alias, col.type); } - if (should_add_project_alias) - project_actions->add(ExpressionAction::project(project_aliases)); + /// TODO When there is no alias, there is no need to add the project action. + /// https://github.com/pingcap/tiflash/issues/3921 + project_actions->add(ExpressionAction::project(project_aliases)); auto physical_projection = std::make_shared(executor_id, schema, req_id, "projection", project_actions); physical_projection->appendChild(child); @@ -138,20 +136,13 @@ void PhysicalProjection::transformImpl(DAGPipeline & pipeline, Context & context void PhysicalProjection::finalize(const Names & parent_require) { - // Maybe parent_require.size() > schema.size() for non-final projection. - if (parent_require.size() > schema.size()) - FinalizeHelper::checkParentRequireContainsSchema(parent_require, schema); - else - FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); project_actions->finalize(parent_require); child->finalize(project_actions->getRequiredColumns()); FinalizeHelper::prependProjectInputIfNeed(project_actions, child->getSampleBlock().columns()); - if (parent_require.size() > schema.size()) - FinalizeHelper::checkSampleBlockContainsSchema(getSampleBlock(), schema); - else - FinalizeHelper::checkSchemaContainsSampleBlock(schema, getSampleBlock()); + FinalizeHelper::checkSampleBlockContainsSchema(getSampleBlock(), schema); } const Block & PhysicalProjection::getSampleBlock() const diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp index c0611f3d31e..ec5987f5756 100644 --- a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -108,8 +108,9 @@ Union: Expression: Expression: Expression: - Expression: - MockTableScan)"; + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -128,8 +129,9 @@ Union: MergeSorting, limit = 10 Union: PartialSorting x 10: limit = 10 - Expression: - MockTableScan)"; + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -144,17 +146,19 @@ Union: String expected = R"( Union: Expression x 10: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - SharedQuery: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - MockTableScan)"; + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -179,17 +183,19 @@ Union: Expression: Expression: Filter - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - SharedQuery: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - MockTableScan)"; + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -247,8 +253,9 @@ Union: Expression: Expression: Expression: - Expression: - MockExchangeReceiver)"; + Expression: + Expression: + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -267,8 +274,9 @@ Union: Expression: Expression: Expression: - Expression: - MockExchangeReceiver)"; + Expression: + Expression: + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -389,15 +397,17 @@ Union: { String expected = R"( Expression: - Expression: - MockTableScan)"; + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); expected = R"( Union: Expression x 5: - Expression: - MockTableScan)"; + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 5); } @@ -475,20 +485,22 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - SharedQuery x 10: - Limit, limit = 10 - Union: - Limit x 10, limit = 10 - Expression: - MockTableScan)"; + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"(Expression: Aggregating Concat - Limit, limit = 10 - Expression: - MockTableScan)"; + Expression: + Limit, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -503,22 +515,24 @@ Union: Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - SharedQuery x 10: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - MockTableScan)"; + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( Expression: Aggregating Concat - Expression: - MergeSorting, limit = 10 - PartialSorting: limit = 10 - MockTableScan)"; + Expression: + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } From 362acb52b77ab7f6e43bd8ecb8216ecdaac01c61 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 14:22:06 +0800 Subject: [PATCH 42/57] format --- dbms/src/Flash/Planner/FinalizeHelper.cpp | 10 +++++----- .../src/Flash/Planner/plans/PhysicalExchangeReceiver.h | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/Planner/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp index b9c00819f89..84baad226bd 100644 --- a/dbms/src/Flash/Planner/FinalizeHelper.cpp +++ b/dbms/src/Flash/Planner/FinalizeHelper.cpp @@ -37,8 +37,8 @@ String schemaToString(const NamesAndTypes & schema) FmtBuffer bf; bf.append("{"); bf.joinStr( - schema.cbegin(), - schema.cend(), + schema.cbegin(), + schema.cend(), [](const auto & col, FmtBuffer & fb) { fb.fmtAppend("<{}, {}>", col.name, col.type->getName()); }, ", "); bf.append("}"); @@ -50,14 +50,14 @@ String blockToString(const Block & block) FmtBuffer bf; bf.append("{"); bf.joinStr( - block.cbegin(), - block.cend(), + block.cbegin(), + block.cend(), [](const ColumnWithTypeAndName & col, FmtBuffer & fb) { fb.fmtAppend("<{}, {}>", col.name, col.type->getName()); }, ", "); bf.append("}"); return bf.toString(); } -} +} // namespace void prependProjectInputIfNeed(ExpressionActionsPtr & actions, size_t columns_from_previous) { diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h index 7d726275d87..e587d6b81b8 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h @@ -43,7 +43,7 @@ class PhysicalExchangeReceiver : public PhysicalLeaf void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; Block sample_block; - + std::shared_ptr mpp_exchange_receiver; }; } // namespace DB From 90f91855035160598eea22ec3ab15c835bc2f40e Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 14:32:13 +0800 Subject: [PATCH 43/57] update --- dbms/src/DataStreams/TiRemoteBlockInputStream.h | 11 +++-------- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 11 ----------- dbms/src/Flash/Coprocessor/DAGUtils.h | 3 --- dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp | 13 ++++++++++++- dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h | 4 +++- .../Planner/plans/PhysicalExchangeReceiver.cpp | 1 + 6 files changed, 19 insertions(+), 24 deletions(-) diff --git a/dbms/src/DataStreams/TiRemoteBlockInputStream.h b/dbms/src/DataStreams/TiRemoteBlockInputStream.h index de06bbf41d6..76fda0b57d0 100644 --- a/dbms/src/DataStreams/TiRemoteBlockInputStream.h +++ b/dbms/src/DataStreams/TiRemoteBlockInputStream.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -176,19 +177,13 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream , log(Logger::get(name, req_id, executor_id)) , total_rows(0) { - // generate sample block - ColumnsWithTypeAndName columns; - for (const NameAndTypePair & col : toNamesAndTypes(remote_reader->getOutputSchema())) - { - columns.emplace_back(col.type, col.name); - } - for (size_t i = 0; i < source_num; i++) + for (size_t i = 0; i < source_num; ++i) { execution_summaries_inited[i].store(false); } execution_summaries.resize(source_num); connection_profile_infos.resize(source_num); - sample_block = Block(columns); + sample_block = Block(getColumnWithTypeAndName(toNamesAndTypes(remote_reader->getOutputSchema()))); } Block getHeader() const override { return sample_block; } diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 1b95c708f82..8c5e017f7dd 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1436,15 +1436,4 @@ tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) throw Exception(fmt::format("Unsupported function {}", name)); return func_name_sig_map[name]; } - -NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema) -{ - NamesAndTypes names_and_types; - for (const auto & col : dag_schema) - { - auto tp = getDataTypeByColumnInfoForComputingLayer(col.second); - names_and_types.emplace_back(col.first, tp); - } - return names_and_types; -} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 136a1b26b06..916713e3f2b 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -106,6 +105,4 @@ class UniqueNameGenerator tipb::DAGRequest getDAGRequestFromStringWithRetry(const String & s); tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context); tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name); - -NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index be3475f714f..efb8a08f1d8 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -54,4 +54,15 @@ ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_ } return column_with_type_and_names; } -} // namespace DB \ No newline at end of file + +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema) +{ + NamesAndTypes names_and_types; + for (const auto & col : dag_schema) + { + auto tp = getDataTypeByColumnInfoForComputingLayer(col.second); + names_and_types.emplace_back(col.first, tp); + } + return names_and_types; +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h index 617f69de925..96f202d800e 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -23,4 +24,5 @@ namespace DB { NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan); ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_types); -} // namespace DB \ No newline at end of file +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp index 31c17cc7288..2fe3a63151c 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include From 47b552a26c541b0d102ef28c8f622eb094cffbc0 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 14:38:49 +0800 Subject: [PATCH 44/57] revert useless change --- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 1 + dbms/src/Flash/Coprocessor/DAGUtils.h | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 8c5e017f7dd..87f58131c8c 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1436,4 +1436,5 @@ tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) throw Exception(fmt::format("Unsupported function {}", name)); return func_name_sig_map[name]; } + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 916713e3f2b..5776edf0098 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -94,7 +94,7 @@ class UniqueNameGenerator while (it != existing_name_map.end()) { ret_name.append("_").append(std::to_string(it->second)); - ++it->second; + it->second++; it = existing_name_map.find(ret_name); } existing_name_map.try_emplace(ret_name, 1); @@ -105,4 +105,5 @@ class UniqueNameGenerator tipb::DAGRequest getDAGRequestFromStringWithRetry(const String & s); tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context); tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name); + } // namespace DB From 15aac9ca04e1d50a18c75c627ebf9a71e535ab88 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 21 Jun 2022 17:53:33 +0800 Subject: [PATCH 45/57] fix test from https://github.com/pingcap/tiflash/pull/5158#discussion_r900751990 --- tests/fullstack-test/mpp/mpp_fail.test | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/fullstack-test/mpp/mpp_fail.test b/tests/fullstack-test/mpp/mpp_fail.test index 02259a90681..7af5fef3f89 100644 --- a/tests/fullstack-test/mpp/mpp_fail.test +++ b/tests/fullstack-test/mpp/mpp_fail.test @@ -21,6 +21,7 @@ mysql> insert into test.t values(1,'a'),(2,'b'),(3,'c') mysql> alter table test.t set tiflash replica 1 func> wait_table test t +mysql> analyze table test.t # Data. From bd74ec975c072ea992b01831d11438c48776e2dc Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 22 Jun 2022 15:11:38 +0800 Subject: [PATCH 46/57] mini refactor update --- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 33 +++++++++---------- dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 2 +- dbms/src/Flash/Planner/Planner.cpp | 2 +- .../Planner/plans/PhysicalAggregation.cpp | 6 ++-- .../Flash/Planner/plans/PhysicalAggregation.h | 2 +- .../plans/PhysicalExchangeReceiver.cpp | 4 +-- .../Planner/plans/PhysicalExchangeReceiver.h | 2 +- .../Planner/plans/PhysicalExchangeSender.cpp | 4 +-- .../Planner/plans/PhysicalExchangeSender.h | 2 +- .../Flash/Planner/plans/PhysicalFilter.cpp | 4 +-- dbms/src/Flash/Planner/plans/PhysicalFilter.h | 2 +- .../src/Flash/Planner/plans/PhysicalLimit.cpp | 4 +-- dbms/src/Flash/Planner/plans/PhysicalLimit.h | 2 +- .../plans/PhysicalMockExchangeReceiver.cpp | 6 ++-- .../plans/PhysicalMockExchangeReceiver.h | 2 +- .../plans/PhysicalMockExchangeSender.cpp | 4 +-- .../plans/PhysicalMockExchangeSender.h | 2 +- .../Planner/plans/PhysicalProjection.cpp | 14 ++++---- .../Flash/Planner/plans/PhysicalProjection.h | 6 ++-- .../Flash/Planner/plans/PhysicalSource.cpp | 7 ++-- dbms/src/Flash/Planner/plans/PhysicalSource.h | 2 +- dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 6 ++-- dbms/src/Flash/Planner/plans/PhysicalTopN.h | 2 +- .../Planner/tests/gtest_physical_plan.cpp | 2 +- 24 files changed, 61 insertions(+), 61 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp index 2f56ac3c72f..d42f87ab5ce 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp @@ -50,36 +50,36 @@ void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor switch (executor->tp()) { case tipb::ExecType::TypeLimit: - pushBack(PhysicalLimit::build(executor_id, log->identifier(), executor->limit(), popBack())); + pushBack(PhysicalLimit::build(executor_id, log, executor->limit(), popBack())); break; case tipb::ExecType::TypeTopN: - pushBack(PhysicalTopN::build(context, executor_id, log->identifier(), executor->topn(), popBack())); + pushBack(PhysicalTopN::build(context, executor_id, log, executor->topn(), popBack())); break; case tipb::ExecType::TypeSelection: - pushBack(PhysicalFilter::build(context, executor_id, log->identifier(), executor->selection(), popBack())); + pushBack(PhysicalFilter::build(context, executor_id, log, executor->selection(), popBack())); break; case tipb::ExecType::TypeAggregation: case tipb::ExecType::TypeStreamAgg: - pushBack(PhysicalAggregation::build(context, executor_id, log->identifier(), executor->aggregation(), popBack())); + pushBack(PhysicalAggregation::build(context, executor_id, log, executor->aggregation(), popBack())); break; case tipb::ExecType::TypeExchangeSender: { if (unlikely(dagContext().isTest())) - pushBack(PhysicalMockExchangeSender::build(executor_id, log->identifier(), popBack())); + pushBack(PhysicalMockExchangeSender::build(executor_id, log, popBack())); else - pushBack(PhysicalExchangeSender::build(executor_id, log->identifier(), executor->exchange_sender(), popBack())); + pushBack(PhysicalExchangeSender::build(executor_id, log, executor->exchange_sender(), popBack())); break; } case tipb::ExecType::TypeExchangeReceiver: { if (unlikely(dagContext().isTest())) - pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log->identifier(), executor->exchange_receiver())); + pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log, executor->exchange_receiver())); else - pushBack(PhysicalExchangeReceiver::build(context, executor_id, log->identifier())); + pushBack(PhysicalExchangeReceiver::build(context, executor_id, log)); break; } case tipb::ExecType::TypeProjection: - pushBack(PhysicalProjection::build(context, executor_id, log->identifier(), executor->projection(), popBack())); + pushBack(PhysicalProjection::build(context, executor_id, log, executor->projection(), popBack())); break; default: throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); @@ -91,7 +91,7 @@ void PhysicalPlanBuilder::buildFinalProjection(const String & column_prefix, boo const auto & final_projection = is_root ? PhysicalProjection::buildRootFinal( context, - log->identifier(), + log, dagContext().output_field_types, dagContext().output_offsets, column_prefix, @@ -99,7 +99,7 @@ void PhysicalPlanBuilder::buildFinalProjection(const String & column_prefix, boo popBack()) : PhysicalProjection::buildNonRootFinal( context, - log->identifier(), + log, column_prefix, popBack()); pushBack(final_projection); @@ -127,20 +127,19 @@ PhysicalPlanPtr PhysicalPlanBuilder::popBack() void PhysicalPlanBuilder::buildSource(const BlockInputStreams & source_streams) { - pushBack(PhysicalSource::build(source_streams, log->identifier())); + pushBack(PhysicalSource::build(source_streams, log)); } -PhysicalPlanPtr PhysicalPlanBuilder::getResult() const +PhysicalPlanPtr PhysicalPlanBuilder::outputAndOptimize() { RUNTIME_ASSERT(cur_plans.size() == 1, log, "There can only be one plan output, but here are {}", cur_plans.size()); - PhysicalPlanPtr physical_plan = cur_plans.back(); + auto origin_physical_plan = popBack(); LOG_FMT_DEBUG( log, "build unoptimized physical plan: \n{}", - PhysicalPlanVisitor::visitToString(physical_plan)); + PhysicalPlanVisitor::visitToString(origin_physical_plan)); - physical_plan = optimize(context, physical_plan); - return physical_plan; + return optimize(context, origin_physical_plan); } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h index cb3a580a37f..2581d3014d0 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h @@ -39,7 +39,7 @@ class PhysicalPlanBuilder void buildFinalProjection(const String & column_prefix, bool is_root); - PhysicalPlanPtr getResult() const; + PhysicalPlanPtr outputAndOptimize(); private: PhysicalPlanPtr popBack(); diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 2863ff3a5af..a736f043a61 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -114,7 +114,7 @@ void Planner::executeImpl(DAGPipeline & pipeline) analyzePhysicalPlan(builder, query_block); - auto physical_plan = builder.getResult(); + auto physical_plan = builder.outputAndOptimize(); LOG_FMT_DEBUG( log, diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp index ed1a657ee91..56166210593 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -33,7 +33,7 @@ namespace DB PhysicalPlanPtr PhysicalAggregation::build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Aggregation & aggregation, PhysicalPlanPtr child) { @@ -42,7 +42,7 @@ PhysicalPlanPtr PhysicalAggregation::build( if (unlikely(aggregation.group_by_size() == 0 && aggregation.agg_func_size() == 0)) { //should not reach here - throw TiFlashException("Aggregation executor without group by/agg exprs", Errors::Coprocessor::BadRequest); + throw TiFlashException("Aggregation executor without group by/agg exprs", Errors::Planner::BadRequest); } DAGExpressionAnalyzer analyzer{child->getSchema(), context}; @@ -73,7 +73,7 @@ PhysicalPlanPtr PhysicalAggregation::build( auto physical_agg = std::make_shared( executor_id, schema, - req_id, + log->identifier(), before_agg_actions, aggregation_keys, collators, diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h index a49fc2e41ee..c0955d69ed3 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h @@ -27,7 +27,7 @@ class PhysicalAggregation : public PhysicalUnary static PhysicalPlanPtr build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Aggregation & aggregation, PhysicalPlanPtr child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp index 2fe3a63151c..ae38c09206a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -41,7 +41,7 @@ PhysicalExchangeReceiver::PhysicalExchangeReceiver( PhysicalPlanPtr PhysicalExchangeReceiver::build( const Context & context, const String & executor_id, - const String & req_id) + const LoggerPtr & log) { const auto & mpp_exchange_receiver_map = context.getDAGContext()->getMPPExchangeReceiverMap(); @@ -56,7 +56,7 @@ PhysicalPlanPtr PhysicalExchangeReceiver::build( auto physical_exchange_receiver = std::make_shared( executor_id, schema, - req_id, + log->identifier(), PhysicalPlanHelper::constructBlockFromSchema(schema), mpp_exchange_receiver); return physical_exchange_receiver; diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h index e587d6b81b8..beeec2b7513 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h @@ -26,7 +26,7 @@ class PhysicalExchangeReceiver : public PhysicalLeaf static PhysicalPlanPtr build( const Context & context, const String & executor_id, - const String & req_id); + const LoggerPtr & log); PhysicalExchangeReceiver( const String & executor_id_, diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index a2347ac447a..dcab3b79155 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -26,7 +26,7 @@ namespace DB { PhysicalPlanPtr PhysicalExchangeSender::build( const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::ExchangeSender & exchange_sender, PhysicalPlanPtr child) { @@ -38,7 +38,7 @@ PhysicalPlanPtr PhysicalExchangeSender::build( auto physical_exchange_sender = std::make_shared( executor_id, child->getSchema(), - req_id, + log->identifier(), partition_col_ids, partition_col_collators, exchange_sender.tp()); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h index 25bcce451cf..5cc48d9e138 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h @@ -25,7 +25,7 @@ class PhysicalExchangeSender : public PhysicalUnary public: static PhysicalPlanPtr build( const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::ExchangeSender & exchange_sender, PhysicalPlanPtr child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp index d84fe9893c6..8a9770f516e 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp @@ -26,7 +26,7 @@ namespace DB PhysicalPlanPtr PhysicalFilter::build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Selection & selection, PhysicalPlanPtr child) { @@ -40,7 +40,7 @@ PhysicalPlanPtr PhysicalFilter::build( conditions.push_back(&c); String filter_column_name = analyzer.buildFilterColumn(before_filter_actions, conditions); - auto physical_filter = std::make_shared(executor_id, child->getSchema(), req_id, filter_column_name, before_filter_actions); + auto physical_filter = std::make_shared(executor_id, child->getSchema(), log->identifier(), filter_column_name, before_filter_actions); physical_filter->appendChild(child); return physical_filter; diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.h b/dbms/src/Flash/Planner/plans/PhysicalFilter.h index fc81376f6c8..06129da39ed 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.h +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.h @@ -26,7 +26,7 @@ class PhysicalFilter : public PhysicalUnary static PhysicalPlanPtr build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Selection & selection, PhysicalPlanPtr child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp index ff5f33c5a12..eef00c1c5b7 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp @@ -24,12 +24,12 @@ namespace DB { PhysicalPlanPtr PhysicalLimit::build( const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Limit & limit, PhysicalPlanPtr child) { assert(child); - auto physical_limit = std::make_shared(executor_id, child->getSchema(), req_id, limit.limit()); + auto physical_limit = std::make_shared(executor_id, child->getSchema(), log->identifier(), limit.limit()); physical_limit->appendChild(child); return physical_limit; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.h b/dbms/src/Flash/Planner/plans/PhysicalLimit.h index c4f62c7d9ab..982bbbeac67 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.h @@ -24,7 +24,7 @@ class PhysicalLimit : public PhysicalUnary public: static PhysicalPlanPtr build( const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Limit & limit, PhysicalPlanPtr child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp index 5af89d33a18..5a20a9550dd 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp @@ -36,7 +36,7 @@ PhysicalMockExchangeReceiver::PhysicalMockExchangeReceiver( PhysicalPlanPtr PhysicalMockExchangeReceiver::build( Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::ExchangeReceiver & exchange_receiver) { NamesAndTypes schema; @@ -53,7 +53,7 @@ PhysicalPlanPtr PhysicalMockExchangeReceiver::build( } else { - auto [names_and_types, mock_exchange_streams] = mockSourceStream(context, max_streams, dag_context.log, executor_id); + auto [names_and_types, mock_exchange_streams] = mockSourceStream(context, max_streams, log, executor_id); schema = std::move(names_and_types); mock_streams.insert(mock_streams.end(), mock_exchange_streams.begin(), mock_exchange_streams.end()); } @@ -63,7 +63,7 @@ PhysicalPlanPtr PhysicalMockExchangeReceiver::build( auto physical_mock_exchange_receiver = std::make_shared( executor_id, schema, - req_id, + log->identifier(), PhysicalPlanHelper::constructBlockFromSchema(schema), mock_streams); return physical_mock_exchange_receiver; diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h index afb5b2f649e..93e394f6c0f 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h @@ -26,7 +26,7 @@ class PhysicalMockExchangeReceiver : public PhysicalLeaf static PhysicalPlanPtr build( Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::ExchangeReceiver & exchange_receiver); PhysicalMockExchangeReceiver( diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp index c7fd744c5bb..e7c736adf1f 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp @@ -22,7 +22,7 @@ namespace DB { PhysicalPlanPtr PhysicalMockExchangeSender::build( const String & executor_id, - const String & req_id, + const LoggerPtr & log, PhysicalPlanPtr child) { assert(child); @@ -30,7 +30,7 @@ PhysicalPlanPtr PhysicalMockExchangeSender::build( auto physical_mock_exchange_sender = std::make_shared( executor_id, child->getSchema(), - req_id); + log->identifier()); physical_mock_exchange_sender->appendChild(child); return physical_mock_exchange_sender; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h index cb1816941b3..0e2cbedf8bc 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h @@ -25,7 +25,7 @@ class PhysicalMockExchangeSender : public PhysicalUnary public: static PhysicalPlanPtr build( const String & executor_id, - const String & req_id, + const LoggerPtr & log, PhysicalPlanPtr child); PhysicalMockExchangeSender( diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp index 205220a94db..ba828f9b8ee 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp @@ -28,7 +28,7 @@ namespace DB PhysicalPlanPtr PhysicalProjection::build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Projection & projection, PhysicalPlanPtr child) { @@ -53,14 +53,14 @@ PhysicalPlanPtr PhysicalProjection::build( /// https://github.com/pingcap/tiflash/issues/3921 project_actions->add(ExpressionAction::project(project_aliases)); - auto physical_projection = std::make_shared(executor_id, schema, req_id, "projection", project_actions); + auto physical_projection = std::make_shared(executor_id, schema, log->identifier(), "projection", project_actions); physical_projection->appendChild(child); return physical_projection; } PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( const Context & context, - const String & req_id, + const LoggerPtr & log, const String & column_prefix, PhysicalPlanPtr child) { @@ -73,14 +73,14 @@ PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( NamesAndTypes schema = child->getSchema(); assert(final_project_aliases.size() == schema.size()); - // replace name by alias. + // replace column name of schema by alias. for (size_t i = 0; i < final_project_aliases.size(); ++i) { assert(schema[i].name == final_project_aliases[i].first); schema[i].name = final_project_aliases[i].second; } - auto physical_projection = std::make_shared("NonRootFinalProjection", schema, req_id, "final projection", project_actions); + auto physical_projection = std::make_shared("NonRootFinalProjection", schema, log->identifier(), "final projection", project_actions); // For final projection, no need to record profile streams. physical_projection->disableRecordProfileStreams(); physical_projection->appendChild(child); @@ -89,7 +89,7 @@ PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( PhysicalPlanPtr PhysicalProjection::buildRootFinal( const Context & context, - const String & req_id, + const LoggerPtr & log, const std::vector & require_schema, const std::vector & output_offsets, const String & column_prefix, @@ -120,7 +120,7 @@ PhysicalPlanPtr PhysicalProjection::buildRootFinal( schema.emplace_back(alias, type); } - auto physical_projection = std::make_shared("RootFinalProjection", schema, req_id, "final projection", project_actions); + auto physical_projection = std::make_shared("RootFinalProjection", schema, log->identifier(), "final projection", project_actions); // For final projection, no need to record profile streams. physical_projection->disableRecordProfileStreams(); physical_projection->appendChild(child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.h b/dbms/src/Flash/Planner/plans/PhysicalProjection.h index 19d5bd1ea17..6cc52ec4f2d 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.h @@ -26,19 +26,19 @@ class PhysicalProjection : public PhysicalUnary static PhysicalPlanPtr build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::Projection & projection, PhysicalPlanPtr child); static PhysicalPlanPtr buildNonRootFinal( const Context & context, - const String & req_id, + const LoggerPtr & log, const String & column_prefix, PhysicalPlanPtr child); static PhysicalPlanPtr buildRootFinal( const Context & context, - const String & req_id, + const LoggerPtr & log, const std::vector & require_schema, const std::vector & output_offsets, const String & column_prefix, diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.cpp b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp index f3a3cb848c6..55eb39a82b6 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalSource.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp @@ -14,19 +14,20 @@ #include #include +#include namespace DB { PhysicalPlanPtr PhysicalSource::build( const BlockInputStreams & source_streams, - const String & req_id) + const LoggerPtr & log) { - assert(!source_streams.empty()); + RUNTIME_ASSERT(!source_streams.empty(), log, "source streams cannot be empty"); Block sample_block = source_streams.back()->getHeader(); NamesAndTypes schema; for (const auto & col : sample_block) schema.emplace_back(col.name, col.type); - return std::make_shared("source", schema, req_id, sample_block, source_streams); + return std::make_shared("source", schema, log->identifier(), sample_block, source_streams); } void PhysicalSource::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.h b/dbms/src/Flash/Planner/plans/PhysicalSource.h index ff6f6c3135d..e3e877bf4b4 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalSource.h +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.h @@ -24,7 +24,7 @@ class PhysicalSource : public PhysicalLeaf public: static PhysicalPlanPtr build( const BlockInputStreams & source_streams, - const String & req_id); + const LoggerPtr & log); PhysicalSource( const String & executor_id_, diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp index ac809ed4065..647d9ca975b 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -27,7 +27,7 @@ namespace DB PhysicalPlanPtr PhysicalTopN::build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::TopN & top_n, PhysicalPlanPtr child) { @@ -36,7 +36,7 @@ PhysicalPlanPtr PhysicalTopN::build( if (unlikely(top_n.order_by_size() == 0)) { //should not reach here - throw TiFlashException("TopN executor without order by exprs", Errors::Coprocessor::BadRequest); + throw TiFlashException("TopN executor without order by exprs", Errors::Planner::BadRequest); } DAGExpressionAnalyzer analyzer{child->getSchema(), context}; @@ -45,7 +45,7 @@ PhysicalPlanPtr PhysicalTopN::build( auto order_columns = analyzer.buildOrderColumns(before_sort_actions, top_n.order_by()); SortDescription order_descr = getSortDescription(order_columns, top_n.order_by()); - auto physical_top_n = std::make_shared(executor_id, child->getSchema(), req_id, order_descr, before_sort_actions, top_n.limit()); + auto physical_top_n = std::make_shared(executor_id, child->getSchema(), log->identifier(), order_descr, before_sort_actions, top_n.limit()); physical_top_n->appendChild(child); return physical_top_n; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.h b/dbms/src/Flash/Planner/plans/PhysicalTopN.h index 2dfc9111712..f19c3acb1ec 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.h +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.h @@ -27,7 +27,7 @@ class PhysicalTopN : public PhysicalUnary static PhysicalPlanPtr build( const Context & context, const String & executor_id, - const String & req_id, + const LoggerPtr & log, const tipb::TopN & top_n, PhysicalPlanPtr child); diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index c8eceb87bbc..efa34b2796d 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -54,7 +54,7 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest assert(request); builder.build(request.get()); builder.buildFinalProjection("physical_plan_", true); - auto physical_plan = builder.getResult(); + auto physical_plan = builder.outputAndOptimize(); ASSERT_EQ(Poco::trim(expected_physical_plan), Poco::trim(PhysicalPlanVisitor::visitToString(physical_plan))); From 53264e0f831a552e30543507c1600a888618cd2b Mon Sep 17 00:00:00 2001 From: SeaRise Date: Wed, 22 Jun 2022 19:13:08 +0800 Subject: [PATCH 47/57] address comments --- dbms/src/Flash/Planner/FinalizeHelper.cpp | 4 +-- dbms/src/Flash/Planner/PhysicalPlan.h | 2 -- .../Planner/plans/PhysicalAggregation.cpp | 4 +-- .../Flash/Planner/plans/PhysicalAggregation.h | 5 +-- .../Planner/plans/PhysicalExchangeSender.cpp | 4 +-- .../Planner/plans/PhysicalExchangeSender.h | 5 +-- .../Flash/Planner/plans/PhysicalFilter.cpp | 11 +++++-- dbms/src/Flash/Planner/plans/PhysicalFilter.h | 5 +-- dbms/src/Flash/Planner/plans/PhysicalLeaf.h | 5 --- .../src/Flash/Planner/plans/PhysicalLimit.cpp | 10 ++++-- dbms/src/Flash/Planner/plans/PhysicalLimit.h | 5 +-- .../plans/PhysicalMockExchangeSender.cpp | 6 ++-- .../plans/PhysicalMockExchangeSender.h | 7 +++-- .../Planner/plans/PhysicalProjection.cpp | 31 ++++++++++++++----- .../Flash/Planner/plans/PhysicalProjection.h | 12 +++++-- dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 12 +++++-- dbms/src/Flash/Planner/plans/PhysicalTopN.h | 5 +-- dbms/src/Flash/Planner/plans/PhysicalUnary.h | 12 ++----- 18 files changed, 87 insertions(+), 58 deletions(-) diff --git a/dbms/src/Flash/Planner/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp index 84baad226bd..1ee6ef00b64 100644 --- a/dbms/src/Flash/Planner/FinalizeHelper.cpp +++ b/dbms/src/Flash/Planner/FinalizeHelper.cpp @@ -110,7 +110,7 @@ void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTy if (unlikely(type_in_sample_block != type_in_schema)) throw TiFlashException( fmt::format( - "the type of column `{}` in sample block `{}` is difference from the one in schema `{}`", + "the type of column `{}` in sample block `{}` is different from the one in schema `{}`", schema_column.name, type_in_sample_block, type_in_schema), @@ -136,7 +136,7 @@ void checkSchemaContainsSampleBlock(const NamesAndTypes & schema, const Block & if (unlikely(type_in_sample_block != type_in_schema)) throw TiFlashException( fmt::format( - "the type of column `{}` in schema `{}` is difference from the one in sample block `{}`", + "the type of column `{}` in schema `{}` is different from the one in sample block `{}`", sample_block_column.name, type_in_schema, type_in_sample_block), diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 8a69545f10b..126cf329226 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -52,8 +52,6 @@ class PhysicalPlan const NamesAndTypes & getSchema() const { return schema; } - virtual void appendChild(const PhysicalPlanPtr & /*new_child*/) = 0; - virtual size_t childrenSize() const = 0; virtual void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp index 56166210593..ac48322ef2f 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -35,7 +35,7 @@ PhysicalPlanPtr PhysicalAggregation::build( const String & executor_id, const LoggerPtr & log, const tipb::Aggregation & aggregation, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); @@ -74,13 +74,13 @@ PhysicalPlanPtr PhysicalAggregation::build( executor_id, schema, log->identifier(), + child, before_agg_actions, aggregation_keys, collators, AggregationInterpreterHelper::isFinalAgg(aggregation), aggregate_descriptions, cast_after_agg_actions); - physical_agg->appendChild(child); return physical_agg; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h index c0955d69ed3..bc9ef542b04 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h @@ -29,19 +29,20 @@ class PhysicalAggregation : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const tipb::Aggregation & aggregation, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); PhysicalAggregation( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const PhysicalPlanPtr & child_, const ExpressionActionsPtr & before_agg_actions_, const Names & aggregation_keys_, const TiDB::TiDBCollators & aggregation_collators_, bool is_final_agg_, const AggregateDescriptions & aggregate_descriptions_, const ExpressionActionsPtr & castAfterAgg_) - : PhysicalUnary(executor_id_, PlanType::Aggregation, schema_, req_id) + : PhysicalUnary(executor_id_, PlanType::Aggregation, schema_, req_id, child_) , before_agg_actions(before_agg_actions_) , aggregation_keys(aggregation_keys_) , aggregation_collators(aggregation_collators_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index dcab3b79155..4a4e4141f32 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -28,7 +28,7 @@ PhysicalPlanPtr PhysicalExchangeSender::build( const String & executor_id, const LoggerPtr & log, const tipb::ExchangeSender & exchange_sender, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); @@ -39,10 +39,10 @@ PhysicalPlanPtr PhysicalExchangeSender::build( executor_id, child->getSchema(), log->identifier(), + child, partition_col_ids, partition_col_collators, exchange_sender.tp()); - physical_exchange_sender->appendChild(child); return physical_exchange_sender; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h index 5cc48d9e138..6664b1a2c9a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h @@ -27,16 +27,17 @@ class PhysicalExchangeSender : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const tipb::ExchangeSender & exchange_sender, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); PhysicalExchangeSender( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const PhysicalPlanPtr & child_, const std::vector & partition_col_ids_, const TiDB::TiDBCollators & collators_, const tipb::ExchangeType & exchange_type_) - : PhysicalUnary(executor_id_, PlanType::ExchangeSender, schema_, req_id) + : PhysicalUnary(executor_id_, PlanType::ExchangeSender, schema_, req_id, child_) , partition_col_ids(partition_col_ids_) , partition_col_collators(collators_) , exchange_type(exchange_type_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp index 8a9770f516e..0741514321c 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp @@ -28,7 +28,7 @@ PhysicalPlanPtr PhysicalFilter::build( const String & executor_id, const LoggerPtr & log, const tipb::Selection & selection, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); @@ -40,8 +40,13 @@ PhysicalPlanPtr PhysicalFilter::build( conditions.push_back(&c); String filter_column_name = analyzer.buildFilterColumn(before_filter_actions, conditions); - auto physical_filter = std::make_shared(executor_id, child->getSchema(), log->identifier(), filter_column_name, before_filter_actions); - physical_filter->appendChild(child); + auto physical_filter = std::make_shared( + executor_id, + child->getSchema(), + log->identifier(), + child, + filter_column_name, + before_filter_actions); return physical_filter; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.h b/dbms/src/Flash/Planner/plans/PhysicalFilter.h index 06129da39ed..07cc2163a33 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.h +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.h @@ -28,15 +28,16 @@ class PhysicalFilter : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const tipb::Selection & selection, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); PhysicalFilter( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const PhysicalPlanPtr & child_, const String & filter_column_, const ExpressionActionsPtr & before_filter_actions_) - : PhysicalUnary(executor_id_, PlanType::Filter, schema_, req_id) + : PhysicalUnary(executor_id_, PlanType::Filter, schema_, req_id, child_) , filter_column(filter_column_) , before_filter_actions(before_filter_actions_) {} diff --git a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h index 50ced412c13..a2f6aaaf8ca 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h @@ -43,11 +43,6 @@ class PhysicalLeaf : public PhysicalPlan throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); } - void appendChild(const PhysicalPlanPtr &) override - { - throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); - } - size_t childrenSize() const override { return 0; }; }; } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp index eef00c1c5b7..ed2989ab85e 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp @@ -26,11 +26,15 @@ PhysicalPlanPtr PhysicalLimit::build( const String & executor_id, const LoggerPtr & log, const tipb::Limit & limit, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); - auto physical_limit = std::make_shared(executor_id, child->getSchema(), log->identifier(), limit.limit()); - physical_limit->appendChild(child); + auto physical_limit = std::make_shared( + executor_id, + child->getSchema(), + log->identifier(), + child, + limit.limit()); return physical_limit; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.h b/dbms/src/Flash/Planner/plans/PhysicalLimit.h index 982bbbeac67..a17b82ebd1e 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.h @@ -26,14 +26,15 @@ class PhysicalLimit : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const tipb::Limit & limit, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); PhysicalLimit( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const PhysicalPlanPtr & child_, size_t limit_) - : PhysicalUnary(executor_id_, PlanType::Limit, schema_, req_id) + : PhysicalUnary(executor_id_, PlanType::Limit, schema_, req_id, child_) , limit(limit_) {} diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp index e7c736adf1f..0aef9fb6ca7 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp @@ -23,15 +23,15 @@ namespace DB PhysicalPlanPtr PhysicalMockExchangeSender::build( const String & executor_id, const LoggerPtr & log, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); auto physical_mock_exchange_sender = std::make_shared( executor_id, child->getSchema(), - log->identifier()); - physical_mock_exchange_sender->appendChild(child); + log->identifier(), + child); return physical_mock_exchange_sender; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h index 0e2cbedf8bc..4f325f07ad1 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h @@ -26,13 +26,14 @@ class PhysicalMockExchangeSender : public PhysicalUnary static PhysicalPlanPtr build( const String & executor_id, const LoggerPtr & log, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); PhysicalMockExchangeSender( const String & executor_id_, const NamesAndTypes & schema_, - const String & req_id) - : PhysicalUnary(executor_id_, PlanType::MockExchangeSender, schema_, req_id) + const String & req_id, + const PhysicalPlanPtr & child_) + : PhysicalUnary(executor_id_, PlanType::MockExchangeSender, schema_, req_id, child_) {} void finalize(const Names & parent_require) override; diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp index ba828f9b8ee..c331c0500f0 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp @@ -30,7 +30,7 @@ PhysicalPlanPtr PhysicalProjection::build( const String & executor_id, const LoggerPtr & log, const tipb::Projection & projection, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); @@ -53,8 +53,13 @@ PhysicalPlanPtr PhysicalProjection::build( /// https://github.com/pingcap/tiflash/issues/3921 project_actions->add(ExpressionAction::project(project_aliases)); - auto physical_projection = std::make_shared(executor_id, schema, log->identifier(), "projection", project_actions); - physical_projection->appendChild(child); + auto physical_projection = std::make_shared( + executor_id, + schema, + log->identifier(), + child, + "projection", + project_actions); return physical_projection; } @@ -62,7 +67,7 @@ PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( const Context & context, const LoggerPtr & log, const String & column_prefix, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); @@ -80,10 +85,15 @@ PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( schema[i].name = final_project_aliases[i].second; } - auto physical_projection = std::make_shared("NonRootFinalProjection", schema, log->identifier(), "final projection", project_actions); + auto physical_projection = std::make_shared( + "NonRootFinalProjection", + schema, + log->identifier(), + child, + "final projection", + project_actions); // For final projection, no need to record profile streams. physical_projection->disableRecordProfileStreams(); - physical_projection->appendChild(child); return physical_projection; } @@ -120,10 +130,15 @@ PhysicalPlanPtr PhysicalProjection::buildRootFinal( schema.emplace_back(alias, type); } - auto physical_projection = std::make_shared("RootFinalProjection", schema, log->identifier(), "final projection", project_actions); + auto physical_projection = std::make_shared( + "RootFinalProjection", + schema, + log->identifier(), + child, + "final projection", + project_actions); // For final projection, no need to record profile streams. physical_projection->disableRecordProfileStreams(); - physical_projection->appendChild(child); return physical_projection; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.h b/dbms/src/Flash/Planner/plans/PhysicalProjection.h index 6cc52ec4f2d..f9af596293d 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.h @@ -28,14 +28,19 @@ class PhysicalProjection : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const tipb::Projection & projection, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); + // Generate a project action to keep the schema of Block and tidb-schema the same, + // and guarantee that left/right block of join don't have duplicated column names. static PhysicalPlanPtr buildNonRootFinal( const Context & context, const LoggerPtr & log, const String & column_prefix, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); + // Generate a project action for root executor, + // to keep the schema of Block and tidb-schema the same. + // Because the output of the root executor is sent to other TiFlash or TiDB. static PhysicalPlanPtr buildRootFinal( const Context & context, const LoggerPtr & log, @@ -49,9 +54,10 @@ class PhysicalProjection : public PhysicalUnary const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const PhysicalPlanPtr & child_, const String & extra_info_, const ExpressionActionsPtr & project_actions_) - : PhysicalUnary(executor_id_, PlanType::Projection, schema_, req_id) + : PhysicalUnary(executor_id_, PlanType::Projection, schema_, req_id, child_) , extra_info(extra_info_) , project_actions(project_actions_) {} diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp index 647d9ca975b..53286247fe5 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -29,7 +29,7 @@ PhysicalPlanPtr PhysicalTopN::build( const String & executor_id, const LoggerPtr & log, const tipb::TopN & top_n, - PhysicalPlanPtr child) + const PhysicalPlanPtr & child) { assert(child); @@ -45,8 +45,14 @@ PhysicalPlanPtr PhysicalTopN::build( auto order_columns = analyzer.buildOrderColumns(before_sort_actions, top_n.order_by()); SortDescription order_descr = getSortDescription(order_columns, top_n.order_by()); - auto physical_top_n = std::make_shared(executor_id, child->getSchema(), log->identifier(), order_descr, before_sort_actions, top_n.limit()); - physical_top_n->appendChild(child); + auto physical_top_n = std::make_shared( + executor_id, + child->getSchema(), + log->identifier(), + child, + order_descr, + before_sort_actions, + top_n.limit()); return physical_top_n; } diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.h b/dbms/src/Flash/Planner/plans/PhysicalTopN.h index f19c3acb1ec..b1ce4bfe87a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.h +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.h @@ -29,16 +29,17 @@ class PhysicalTopN : public PhysicalUnary const String & executor_id, const LoggerPtr & log, const tipb::TopN & top_n, - PhysicalPlanPtr child); + const PhysicalPlanPtr & child); PhysicalTopN( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, + const PhysicalPlanPtr & child_, const SortDescription & order_descr_, const ExpressionActionsPtr & before_sort_actions_, size_t limit_) - : PhysicalUnary(executor_id_, PlanType::TopN, schema_, req_id) + : PhysicalUnary(executor_id_, PlanType::TopN, schema_, req_id, child_) , order_descr(order_descr_) , before_sort_actions(before_sort_actions_) , limit(limit_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/plans/PhysicalUnary.h index 4d0091bb8e3..1634f7826b7 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalUnary.h +++ b/dbms/src/Flash/Planner/plans/PhysicalUnary.h @@ -31,8 +31,10 @@ class PhysicalUnary : public PhysicalPlan const String & executor_id_, const PlanType & type_, const NamesAndTypes & schema_, - const String & req_id) + const String & req_id, + const PhysicalPlanPtr & child_) : PhysicalPlan(executor_id_, type_, schema_, req_id) + , child(child_) {} PhysicalPlanPtr children(size_t i) const override @@ -50,14 +52,6 @@ class PhysicalUnary : public PhysicalPlan child = new_child; } - void appendChild(const PhysicalPlanPtr & new_child) override - { - RUNTIME_ASSERT(!child, log, "the actual children size had be the max size({}), don't append child again", childrenSize()); - assert(new_child); - assert(new_child.get() != this); - child = new_child; - } - size_t childrenSize() const override { return 1; }; protected: From 7037098b2118bbdbc32ab621d76be3d7096f0b2b Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 23 Jun 2022 13:50:56 +0800 Subject: [PATCH 48/57] fix bug of physical agg --- dbms/src/Flash/Planner/PhysicalPlan.cpp | 10 ++++------ dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp | 5 +++++ 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 76f894d609b..39bbc0fa47b 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -59,17 +59,15 @@ void PhysicalPlan::finalize() void PhysicalPlan::recordProfileStreams(DAGPipeline & pipeline, const Context & context) { - if (is_record_profile_streams) - { - auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[executor_id]; - pipeline.transform([&profile_streams](auto & stream) { profile_streams.push_back(stream); }); - } + auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[executor_id]; + pipeline.transform([&profile_streams](auto & stream) { profile_streams.push_back(stream); }); } void PhysicalPlan::transform(DAGPipeline & pipeline, Context & context, size_t max_streams) { transformImpl(pipeline, context, max_streams); - recordProfileStreams(pipeline, context); + if (is_record_profile_streams) + recordProfileStreams(pipeline, context); // todo modify logic after supporting window function. context.getDAGContext()->updateFinalConcurrency(pipeline.streams.size(), max_streams); restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp index ac48322ef2f..ca53f515a56 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -81,6 +81,8 @@ PhysicalPlanPtr PhysicalAggregation::build( AggregationInterpreterHelper::isFinalAgg(aggregation), aggregate_descriptions, cast_after_agg_actions); + // For agg, `recordProfileStreams` has been called in `transformImpl`. + physical_agg->disableRecordProfileStreams(); return physical_agg; } @@ -116,6 +118,8 @@ void PhysicalAggregation::transformImpl(DAGPipeline & pipeline, Context & contex settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads), log->identifier()); pipeline.streams.resize(1); + // should record for agg before restore concurrency. See #3804. + recordProfileStreams(pipeline, context); restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); } else @@ -134,6 +138,7 @@ void PhysicalAggregation::transformImpl(DAGPipeline & pipeline, Context & contex context.getFileProvider(), true, log->identifier()); + recordProfileStreams(pipeline, context); } executeExpression(pipeline, cast_after_agg, log, "cast after aggregation"); From 130da7420663c5f181b850b4c6be8a419714c913 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 23 Jun 2022 14:41:57 +0800 Subject: [PATCH 49/57] refactor tests --- .../Planner/tests/gtest_physical_plan.cpp | 59 ++++++++----------- 1 file changed, 25 insertions(+), 34 deletions(-) diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index efa34b2796d..639be8959ac 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -53,7 +53,6 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest PhysicalPlanBuilder builder{context.context, log->identifier()}; assert(request); builder.build(request.get()); - builder.buildFinalProjection("physical_plan_", true); auto physical_plan = builder.outputAndOptimize(); ASSERT_EQ(Poco::trim(expected_physical_plan), Poco::trim(PhysicalPlanVisitor::visitToString(physical_plan))); @@ -81,16 +80,17 @@ try { auto request = context.receive("exchange1") .filter(eq(col("s1"), col("s2"))) + .project({col("s1"), col("s2")}) .build(context); execute( request, /*expected_physical_plan=*/R"( - | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: Filter MockExchangeReceiver)", {toNullableVec({"banana"}), @@ -108,13 +108,11 @@ try execute( request, /*expected_physical_plan=*/R"( - | is_record_profile_streams: false, schema: , - | is_record_profile_streams: true, schema: , - | is_record_profile_streams: true, schema: , )", + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: - Limit, limit = 1 - MockExchangeReceiver)", +Limit, limit = 1 + MockExchangeReceiver)", {toNullableVec({"banana"}), toNullableVec({"apple"})}); } @@ -125,16 +123,17 @@ try { auto request = context.receive("exchange1") .topN("s2", false, 1) + .project({col("s1"), col("s2")}) .build(context); execute( request, /*expected_physical_plan=*/R"( - | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: +Expression: MergeSorting, limit = 1 PartialSorting: limit = 1 MockExchangeReceiver)", @@ -154,14 +153,12 @@ try execute( request, /*expected_physical_plan=*/R"( - | is_record_profile_streams: false, schema: , - | is_record_profile_streams: true, schema: , - | is_record_profile_streams: true, schema: , )", + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: - Aggregating - Concat - MockExchangeReceiver)", +Aggregating + Concat + MockExchangeReceiver)", {toNullableVec({{}, "banana"}), toNullableVec({{}, "banana"})}); } @@ -177,13 +174,11 @@ try execute( request, /*expected_physical_plan=*/R"( - | is_record_profile_streams: false, schema: - | is_record_profile_streams: true, schema: - | is_record_profile_streams: true, schema: , )", + | is_record_profile_streams: true, schema: + | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: - Expression: - MockExchangeReceiver)", +Expression: + MockExchangeReceiver)", {toNullableVec({"bananaapple", {}, "bananabanana"})}); } CATCH @@ -198,13 +193,11 @@ try execute( request, /*expected_physical_plan=*/R"( - | is_record_profile_streams: false, schema: , - | is_record_profile_streams: true, schema: , - | is_record_profile_streams: true, schema: , )", + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: - MockExchangeSender - MockExchangeReceiver)", +MockExchangeSender + MockExchangeReceiver)", {toNullableVec({"banana", {}, "banana"}), toNullableVec({"apple", {}, "banana"})}); } @@ -219,11 +212,9 @@ try execute( request, /*expected_physical_plan=*/R"( - | is_record_profile_streams: false, schema: , - | is_record_profile_streams: true, schema: , )", + | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Expression: - MockExchangeReceiver)", +MockExchangeReceiver)", {toNullableVec({"banana", {}, "banana"}), toNullableVec({"apple", {}, "banana"})}); } From c265c58e5da0a477927e16c14b4b67274384fbc8 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 23 Jun 2022 19:47:37 +0800 Subject: [PATCH 50/57] update --- dbms/src/Flash/Planner/Planner.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index a736f043a61..470326701b9 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -25,13 +25,14 @@ namespace DB { namespace { -void analyzePhysicalPlan(PhysicalPlanBuilder & builder, const DAGQueryBlock & query_block) +void analyzePhysicalPlan(Context & context, PhysicalPlanBuilder & builder, const DAGQueryBlock & query_block) { assert(query_block.source); builder.build(query_block.source_name, query_block.source); // selection on table scan had been executed in table scan. - if (query_block.selection && !query_block.isTableScanSource()) + // In test mode, filter is not pushed down to table scan. + if (query_block.selection && (!query_block.isTableScanSource() || context.getDAGContext()->isTest())) { builder.build(query_block.selection_name, query_block.selection); } @@ -112,7 +113,7 @@ void Planner::executeImpl(DAGPipeline & pipeline) builder.buildSource(input_streams); } - analyzePhysicalPlan(builder, query_block); + analyzePhysicalPlan(context, builder, query_block); auto physical_plan = builder.outputAndOptimize(); From 7ace19dec7a36267acff581c0e8c7e012b15bfac Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 24 Jun 2022 11:03:57 +0800 Subject: [PATCH 51/57] add project action in cast after agg fix test --- .../Planner/plans/PhysicalAggregation.cpp | 4 +- .../Planner/tests/gtest_physical_plan.cpp | 7 +- .../Flash/tests/gtest_planner_interpreter.cpp | 145 ++++++++++-------- 3 files changed, 84 insertions(+), 72 deletions(-) diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp index ca53f515a56..2a15031bd87 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -68,8 +68,10 @@ PhysicalPlanPtr PhysicalAggregation::build( auto cast_after_agg_actions = PhysicalPlanHelper::newActions(aggregated_columns, context); analyzer.reset(aggregated_columns); analyzer.appendCastAfterAgg(cast_after_agg_actions, aggregation); - + /// project action after aggregation to remove useless columns. const NamesAndTypes & schema = analyzer.getCurrentInputColumns(); + cast_after_agg_actions->add(ExpressionAction::project(PhysicalPlanHelper::schemaToNames(schema))); + auto physical_agg = std::make_shared( executor_id, schema, diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index 639be8959ac..77e35c7ed93 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -156,9 +156,10 @@ try | is_record_profile_streams: false, schema: , | is_record_profile_streams: true, schema: , )", /*expected_streams=*/R"( -Aggregating - Concat - MockExchangeReceiver)", +Expression: + Aggregating + Concat + MockExchangeReceiver)", {toNullableVec({{}, "banana"}), toNullableVec({{}, "banana"})}); } diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp index ec5987f5756..3840b858340 100644 --- a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -148,17 +148,18 @@ Union: Expression x 10: Expression: Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - SharedQuery: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - MockTableScan)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -185,17 +186,18 @@ Union: Filter Expression: Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - SharedQuery: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - MockTableScan)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -483,24 +485,27 @@ Union: String expected = R"( Union: Expression x 10: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - SharedQuery: - Limit, limit = 10 - Union: - Limit x 10, limit = 10 - Expression: - MockTableScan)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - expected = R"(Expression: - Aggregating - Concat - Expression: - Limit, limit = 10 - Expression: - MockTableScan)"; + expected = R"( +Expression: + Expression: + Aggregating + Concat + Expression: + Limit, limit = 10 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -513,26 +518,28 @@ Union: String expected = R"( Union: Expression x 10: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - MockTableScan)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( Expression: - Aggregating - Concat - Expression: - Expression: - MergeSorting, limit = 10 - PartialSorting: limit = 10 - MockTableScan)"; + Expression: + Aggregating + Concat + Expression: + Expression: + MergeSorting, limit = 10 + PartialSorting: limit = 10 + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } @@ -545,24 +552,26 @@ Expression: String expected = R"( Union: Expression x 10: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - MockTableScan x 10)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + MockTableScan x 10)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); expected = R"( Expression: - Aggregating - Concat - Expression: - Expression: - Aggregating - Concat - MockTableScan)"; + Expression: + Aggregating + Concat + Expression: + Expression: + Aggregating + Concat + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); } From 8e213b63b6e87bb1c4183f8cfea52227f616f95c Mon Sep 17 00:00:00 2001 From: SeaRise Date: Tue, 5 Jul 2022 14:16:55 +0800 Subject: [PATCH 52/57] address comment --- dbms/src/Flash/Planner/FinalizeHelper.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Flash/Planner/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp index 1ee6ef00b64..fad85908a8d 100644 --- a/dbms/src/Flash/Planner/FinalizeHelper.cpp +++ b/dbms/src/Flash/Planner/FinalizeHelper.cpp @@ -45,7 +45,7 @@ String schemaToString(const NamesAndTypes & schema) return bf.toString(); } -String blockToString(const Block & block) +String blockMetaToString(const Block & block) { FmtBuffer bf; bf.append("{"); @@ -102,7 +102,7 @@ void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTy { if (unlikely(!sample_block.has(schema_column.name))) throw TiFlashException( - fmt::format("sample block {} don't contain schema column: {}", blockToString(sample_block), schema_column.name), + fmt::format("sample block {} don't contain schema column: {}", blockMetaToString(sample_block), schema_column.name), Errors::Planner::Internal); const auto & type_in_sample_block = sample_block.getByName(schema_column.name).type->getName(); From c0eab66e18c0dbc4e7f24286569c8948fc867352 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 8 Jul 2022 10:40:44 +0800 Subject: [PATCH 53/57] address comments --- dbms/src/Flash/Planner/ExecutorIdGenerator.h | 3 ++- dbms/src/Flash/Planner/plans/PhysicalUnary.h | 11 ++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/dbms/src/Flash/Planner/ExecutorIdGenerator.h b/dbms/src/Flash/Planner/ExecutorIdGenerator.h index 2a600f447f1..4c48b5d2aad 100644 --- a/dbms/src/Flash/Planner/ExecutorIdGenerator.h +++ b/dbms/src/Flash/Planner/ExecutorIdGenerator.h @@ -20,6 +20,7 @@ #include #include +#include #include namespace DB @@ -69,6 +70,6 @@ class ExecutorIdGenerator UInt32 current_id = 0; - std::unordered_set ids; + std::unordered_set ids; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/plans/PhysicalUnary.h index 1634f7826b7..43133f79a42 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalUnary.h +++ b/dbms/src/Flash/Planner/plans/PhysicalUnary.h @@ -35,20 +35,21 @@ class PhysicalUnary : public PhysicalPlan const PhysicalPlanPtr & child_) : PhysicalPlan(executor_id_, type_, schema_, req_id) , child(child_) - {} + { + RUNTIME_ASSERT(child, log, "children(0) shouldn't be nullptr"); + } PhysicalPlanPtr children(size_t i) const override { - RUNTIME_ASSERT(i == 0, log, "child_index({}) should not >= childrenSize({})", i, childrenSize()); - assert(child); + RUNTIME_ASSERT(i == 0, log, "child_index({}) shouldn't >= childrenSize({})", i, childrenSize()); return child; } void setChild(size_t i, const PhysicalPlanPtr & new_child) override { RUNTIME_ASSERT(i == 0, log, "child_index({}) should not >= childrenSize({})", i, childrenSize()); - assert(new_child); - assert(new_child.get() != this); + RUNTIME_ASSERT(new_child, log, "new_child for child_index({}) shouldn't be nullptr", i); + RUNTIME_ASSERT(new_child.get() != this, log, "new_child for child_index({}) shouldn't be itself", i); child = new_child; } From d3a0cc4f288df765d65d16195af32f0da0734975 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 8 Jul 2022 10:45:44 +0800 Subject: [PATCH 54/57] address comment --- dbms/src/Flash/Coprocessor/DAGContext.cpp | 4 ++-- dbms/src/Flash/Coprocessor/DAGContext.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index e7f1a54dc2a..a17eaf53b64 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -75,9 +75,9 @@ std::unordered_map & DAGContext::getProfileStreamsMap return profile_streams_map; } -void DAGContext::updateFinalConcurrency(size_t cur_streams_size, size_t max_streams) +void DAGContext::updateFinalConcurrency(size_t cur_streams_size, size_t streams_upper_limit) { - final_concurrency = std::min(std::max(final_concurrency, cur_streams_size), max_streams); + final_concurrency = std::min(std::max(final_concurrency, cur_streams_size), streams_upper_limit); } void DAGContext::initExecutorIdToJoinIdMap() diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 10b805a2af2..8d84a7c6add 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -297,7 +297,7 @@ class DAGContext return sql_mode & f; } - void updateFinalConcurrency(size_t cur_streams_size, size_t max_streams); + void updateFinalConcurrency(size_t cur_streams_size, size_t streams_upper_limit); bool isTest() const { return is_test; } void setColumnsForTest(std::unordered_map & columns_for_test_map_) { columns_for_test_map = columns_for_test_map_; } From 8776c18b45261b49acb83ef68309a481adcf71d7 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 8 Jul 2022 15:17:48 +0800 Subject: [PATCH 55/57] address comment: change PhysicalPlan --> PhysicalPlanNode, PhysicalPlanBuilder --> PhysicalPlan --- dbms/src/Flash/Planner/PhysicalPlan.cpp | 167 +++++++++++++----- dbms/src/Flash/Planner/PhysicalPlan.h | 68 +++---- .../src/Flash/Planner/PhysicalPlanBuilder.cpp | 145 --------------- dbms/src/Flash/Planner/PhysicalPlanBuilder.h | 58 ------ dbms/src/Flash/Planner/PhysicalPlanNode.cpp | 75 ++++++++ dbms/src/Flash/Planner/PhysicalPlanNode.h | 81 +++++++++ .../src/Flash/Planner/PhysicalPlanVisitor.cpp | 6 +- dbms/src/Flash/Planner/PhysicalPlanVisitor.h | 10 +- dbms/src/Flash/Planner/Planner.cpp | 31 ++-- dbms/src/Flash/Planner/optimize.cpp | 7 +- dbms/src/Flash/Planner/optimize.h | 4 +- .../Planner/plans/PhysicalAggregation.cpp | 4 +- .../Flash/Planner/plans/PhysicalAggregation.h | 6 +- .../plans/PhysicalExchangeReceiver.cpp | 2 +- .../Planner/plans/PhysicalExchangeReceiver.h | 2 +- .../Planner/plans/PhysicalExchangeSender.cpp | 4 +- .../Planner/plans/PhysicalExchangeSender.h | 6 +- .../Flash/Planner/plans/PhysicalFilter.cpp | 4 +- dbms/src/Flash/Planner/plans/PhysicalFilter.h | 6 +- dbms/src/Flash/Planner/plans/PhysicalLeaf.h | 10 +- .../src/Flash/Planner/plans/PhysicalLimit.cpp | 4 +- dbms/src/Flash/Planner/plans/PhysicalLimit.h | 6 +- .../plans/PhysicalMockExchangeReceiver.cpp | 2 +- .../plans/PhysicalMockExchangeReceiver.h | 2 +- .../plans/PhysicalMockExchangeSender.cpp | 4 +- .../plans/PhysicalMockExchangeSender.h | 6 +- .../Planner/plans/PhysicalProjection.cpp | 12 +- .../Flash/Planner/plans/PhysicalProjection.h | 14 +- .../Flash/Planner/plans/PhysicalSource.cpp | 2 +- dbms/src/Flash/Planner/plans/PhysicalSource.h | 2 +- dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 4 +- dbms/src/Flash/Planner/plans/PhysicalTopN.h | 6 +- dbms/src/Flash/Planner/plans/PhysicalUnary.h | 14 +- .../Planner/tests/gtest_physical_plan.cpp | 12 +- 34 files changed, 405 insertions(+), 381 deletions(-) delete mode 100644 dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp delete mode 100644 dbms/src/Flash/Planner/PhysicalPlanBuilder.h create mode 100644 dbms/src/Flash/Planner/PhysicalPlanNode.cpp create mode 100644 dbms/src/Flash/Planner/PhysicalPlanNode.h diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 39bbc0fa47b..db97423772b 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -12,64 +12,149 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include -#include -#include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include namespace DB { -PhysicalPlan::PhysicalPlan( - const String & executor_id_, - const PlanType & type_, - const NamesAndTypes & schema_, - const String & req_id) - : executor_id(executor_id_) - , type(type_) - , schema(schema_) - , log(Logger::get(type_.toString(), req_id)) -{} +void PhysicalPlan::build(const tipb::DAGRequest * dag_request) +{ + assert(dag_request); + ExecutorIdGenerator id_generator; + traverseExecutorsReverse( + dag_request, + [&](const tipb::Executor & executor) { + build(id_generator.generate(executor), &executor); + return true; + }); +} + +void PhysicalPlan::build(const String & executor_id, const tipb::Executor * executor) +{ + assert(executor); + switch (executor->tp()) + { + case tipb::ExecType::TypeLimit: + pushBack(PhysicalLimit::build(executor_id, log, executor->limit(), popBack())); + break; + case tipb::ExecType::TypeTopN: + pushBack(PhysicalTopN::build(context, executor_id, log, executor->topn(), popBack())); + break; + case tipb::ExecType::TypeSelection: + pushBack(PhysicalFilter::build(context, executor_id, log, executor->selection(), popBack())); + break; + case tipb::ExecType::TypeAggregation: + case tipb::ExecType::TypeStreamAgg: + pushBack(PhysicalAggregation::build(context, executor_id, log, executor->aggregation(), popBack())); + break; + case tipb::ExecType::TypeExchangeSender: + { + if (unlikely(dagContext().isTest())) + pushBack(PhysicalMockExchangeSender::build(executor_id, log, popBack())); + else + pushBack(PhysicalExchangeSender::build(executor_id, log, executor->exchange_sender(), popBack())); + break; + } + case tipb::ExecType::TypeExchangeReceiver: + { + if (unlikely(dagContext().isTest())) + pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log, executor->exchange_receiver())); + else + pushBack(PhysicalExchangeReceiver::build(context, executor_id, log)); + break; + } + case tipb::ExecType::TypeProjection: + pushBack(PhysicalProjection::build(context, executor_id, log, executor->projection(), popBack())); + break; + default: + throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); + } +} -String PhysicalPlan::toString() +void PhysicalPlan::buildFinalProjection(const String & column_prefix, bool is_root) { - auto schema_to_string = [&]() { - FmtBuffer buffer; - buffer.joinStr( - schema.cbegin(), - schema.cend(), - [](const auto & item, FmtBuffer & buf) { buf.fmtAppend("<{}, {}>", item.name, item.type->getName()); }, - ", "); - return buffer.toString(); - }; - return fmt::format( - "<{}, {}> | is_record_profile_streams: {}, schema: {}", - type.toString(), - executor_id, - is_record_profile_streams, - schema_to_string()); + const auto & final_projection = is_root + ? PhysicalProjection::buildRootFinal( + context, + log, + dagContext().output_field_types, + dagContext().output_offsets, + column_prefix, + dagContext().keep_session_timezone_info, + popBack()) + : PhysicalProjection::buildNonRootFinal( + context, + log, + column_prefix, + popBack()); + pushBack(final_projection); } -void PhysicalPlan::finalize() +DAGContext & PhysicalPlan::dagContext() const { - finalize(PhysicalPlanHelper::schemaToNames(schema)); + return *context.getDAGContext(); +} + +void PhysicalPlan::pushBack(const PhysicalPlanNodePtr & plan_node) +{ + assert(plan_node); + cur_plan_nodes.push_back(plan_node); +} + +PhysicalPlanNodePtr PhysicalPlan::popBack() +{ + if (unlikely(cur_plan_nodes.empty())) + throw TiFlashException("cur_plan_nodes is empty, cannot popBack", Errors::Planner::Internal); + PhysicalPlanNodePtr back = cur_plan_nodes.back(); + assert(back); + cur_plan_nodes.pop_back(); + return back; +} + +void PhysicalPlan::buildSource(const BlockInputStreams & source_streams) +{ + pushBack(PhysicalSource::build(source_streams, log)); +} + +void PhysicalPlan::outputAndOptimize() +{ + RUNTIME_ASSERT(!root_node, log, "root_node shoud be nullptr before `outputAndOptimize`"); + RUNTIME_ASSERT(cur_plan_nodes.size() == 1, log, "There can only be one plan node output, but here are {}", cur_plan_nodes.size()); + root_node = popBack(); + + LOG_FMT_DEBUG( + log, + "build unoptimized physical plan: \n{}", + toString()); + + root_node = optimize(context, root_node); + RUNTIME_ASSERT(root_node, log, "root_node shoudn't be nullptr after `outputAndOptimize`"); } -void PhysicalPlan::recordProfileStreams(DAGPipeline & pipeline, const Context & context) +String PhysicalPlan::toString() const { - auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[executor_id]; - pipeline.transform([&profile_streams](auto & stream) { profile_streams.push_back(stream); }); + assert(root_node); + return PhysicalPlanVisitor::visitToString(root_node); } void PhysicalPlan::transform(DAGPipeline & pipeline, Context & context, size_t max_streams) { - transformImpl(pipeline, context, max_streams); - if (is_record_profile_streams) - recordProfileStreams(pipeline, context); - // todo modify logic after supporting window function. - context.getDAGContext()->updateFinalConcurrency(pipeline.streams.size(), max_streams); - restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); + assert(root_node); + root_node->transform(pipeline, context, max_streams); } } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.h b/dbms/src/Flash/Planner/PhysicalPlan.h index 126cf329226..0dfdda1b941 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.h +++ b/dbms/src/Flash/Planner/PhysicalPlan.h @@ -14,67 +14,53 @@ #pragma once +#include #include -#include -#include -#include -#include - -#include +#include +#include +#include +#include +#include namespace DB { -struct DAGPipeline; -class Context; -class DAGContext; - -class PhysicalPlan; -using PhysicalPlanPtr = std::shared_ptr; - class PhysicalPlan { public: - PhysicalPlan( - const String & executor_id_, - const PlanType & type_, - const NamesAndTypes & schema_, - const String & req_id); - - virtual ~PhysicalPlan() = default; - - virtual PhysicalPlanPtr children(size_t /*i*/) const = 0; + explicit PhysicalPlan(Context & context_, const String & req_id) + : context(context_) + , log(Logger::get("PhysicalPlan", req_id)) + {} - virtual void setChild(size_t /*i*/, const PhysicalPlanPtr & /*new_child*/) = 0; + void build(const tipb::DAGRequest * dag_request); - const PlanType & tp() const { return type; } + void build(const String & executor_id, const tipb::Executor * executor); - const String & execId() const { return executor_id; } + void buildSource(const BlockInputStreams & source_streams); - const NamesAndTypes & getSchema() const { return schema; } + void buildFinalProjection(const String & column_prefix, bool is_root); - virtual size_t childrenSize() const = 0; + // after outputAndOptimize, the physical plan node tree is done. + void outputAndOptimize(); - virtual void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); + String toString() const; - virtual void finalize(const Names & parent_require) = 0; - void finalize(); + void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); - /// Obtain a sample block that contains the names and types of result columns. - virtual const Block & getSampleBlock() const = 0; +private: + PhysicalPlanNodePtr popBack(); - void disableRecordProfileStreams() { is_record_profile_streams = false; } + void pushBack(const PhysicalPlanNodePtr & plan); - String toString(); + DAGContext & dagContext() const; -protected: - virtual void transformImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/){}; +private: + std::vector cur_plan_nodes{}; - void recordProfileStreams(DAGPipeline & pipeline, const Context & context); + // hold the root node of physical plan node tree after `outputAndOptimize`. + PhysicalPlanNodePtr root_node; - String executor_id; - PlanType type; - NamesAndTypes schema; - bool is_record_profile_streams = true; + Context & context; LoggerPtr log; }; diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp b/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp deleted file mode 100644 index d42f87ab5ce..00000000000 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.cpp +++ /dev/null @@ -1,145 +0,0 @@ -// 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 -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -void PhysicalPlanBuilder::build(const tipb::DAGRequest * dag_request) -{ - assert(dag_request); - ExecutorIdGenerator id_generator; - traverseExecutorsReverse( - dag_request, - [&](const tipb::Executor & executor) { - build(id_generator.generate(executor), &executor); - return true; - }); -} - -void PhysicalPlanBuilder::build(const String & executor_id, const tipb::Executor * executor) -{ - assert(executor); - switch (executor->tp()) - { - case tipb::ExecType::TypeLimit: - pushBack(PhysicalLimit::build(executor_id, log, executor->limit(), popBack())); - break; - case tipb::ExecType::TypeTopN: - pushBack(PhysicalTopN::build(context, executor_id, log, executor->topn(), popBack())); - break; - case tipb::ExecType::TypeSelection: - pushBack(PhysicalFilter::build(context, executor_id, log, executor->selection(), popBack())); - break; - case tipb::ExecType::TypeAggregation: - case tipb::ExecType::TypeStreamAgg: - pushBack(PhysicalAggregation::build(context, executor_id, log, executor->aggregation(), popBack())); - break; - case tipb::ExecType::TypeExchangeSender: - { - if (unlikely(dagContext().isTest())) - pushBack(PhysicalMockExchangeSender::build(executor_id, log, popBack())); - else - pushBack(PhysicalExchangeSender::build(executor_id, log, executor->exchange_sender(), popBack())); - break; - } - case tipb::ExecType::TypeExchangeReceiver: - { - if (unlikely(dagContext().isTest())) - pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log, executor->exchange_receiver())); - else - pushBack(PhysicalExchangeReceiver::build(context, executor_id, log)); - break; - } - case tipb::ExecType::TypeProjection: - pushBack(PhysicalProjection::build(context, executor_id, log, executor->projection(), popBack())); - break; - default: - throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); - } -} - -void PhysicalPlanBuilder::buildFinalProjection(const String & column_prefix, bool is_root) -{ - const auto & final_projection = is_root - ? PhysicalProjection::buildRootFinal( - context, - log, - dagContext().output_field_types, - dagContext().output_offsets, - column_prefix, - dagContext().keep_session_timezone_info, - popBack()) - : PhysicalProjection::buildNonRootFinal( - context, - log, - column_prefix, - popBack()); - pushBack(final_projection); -} - -DAGContext & PhysicalPlanBuilder::dagContext() const -{ - return *context.getDAGContext(); -} - -void PhysicalPlanBuilder::pushBack(const PhysicalPlanPtr & plan) -{ - assert(plan); - cur_plans.push_back(plan); -} - -PhysicalPlanPtr PhysicalPlanBuilder::popBack() -{ - if (unlikely(cur_plans.empty())) - throw TiFlashException("cur_plans is empty, cannot popBack", Errors::Planner::Internal); - PhysicalPlanPtr back = cur_plans.back(); - cur_plans.pop_back(); - return back; -} - -void PhysicalPlanBuilder::buildSource(const BlockInputStreams & source_streams) -{ - pushBack(PhysicalSource::build(source_streams, log)); -} - -PhysicalPlanPtr PhysicalPlanBuilder::outputAndOptimize() -{ - RUNTIME_ASSERT(cur_plans.size() == 1, log, "There can only be one plan output, but here are {}", cur_plans.size()); - auto origin_physical_plan = popBack(); - - LOG_FMT_DEBUG( - log, - "build unoptimized physical plan: \n{}", - PhysicalPlanVisitor::visitToString(origin_physical_plan)); - - return optimize(context, origin_physical_plan); -} -} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h b/dbms/src/Flash/Planner/PhysicalPlanBuilder.h deleted file mode 100644 index 2581d3014d0..00000000000 --- a/dbms/src/Flash/Planner/PhysicalPlanBuilder.h +++ /dev/null @@ -1,58 +0,0 @@ -// 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. - -#pragma once - -#include -#include -#include -#include -#include -#include - -namespace DB -{ -class PhysicalPlanBuilder -{ -public: - explicit PhysicalPlanBuilder(Context & context_, const String & req_id) - : context(context_) - , log(Logger::get("PhysicalPlanBuilder", req_id)) - {} - - void build(const tipb::DAGRequest * dag_request); - - void build(const String & executor_id, const tipb::Executor * executor); - - void buildSource(const BlockInputStreams & source_streams); - - void buildFinalProjection(const String & column_prefix, bool is_root); - - PhysicalPlanPtr outputAndOptimize(); - -private: - PhysicalPlanPtr popBack(); - - void pushBack(const PhysicalPlanPtr & plan); - - DAGContext & dagContext() const; - -private: - std::vector cur_plans{}; - - Context & context; - - LoggerPtr log; -}; -} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp new file mode 100644 index 00000000000..975584a8e61 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp @@ -0,0 +1,75 @@ +// 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 +#include +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanNode::PhysicalPlanNode( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id) + : executor_id(executor_id_) + , type(type_) + , schema(schema_) + , log(Logger::get(type_.toString(), req_id)) +{} + +String PhysicalPlanNode::toString() +{ + auto schema_to_string = [&]() { + FmtBuffer buffer; + buffer.joinStr( + schema.cbegin(), + schema.cend(), + [](const auto & item, FmtBuffer & buf) { buf.fmtAppend("<{}, {}>", item.name, item.type->getName()); }, + ", "); + return buffer.toString(); + }; + return fmt::format( + "<{}, {}> | is_record_profile_streams: {}, schema: {}", + type.toString(), + executor_id, + is_record_profile_streams, + schema_to_string()); +} + +void PhysicalPlanNode::finalize() +{ + finalize(PhysicalPlanHelper::schemaToNames(schema)); +} + +void PhysicalPlanNode::recordProfileStreams(DAGPipeline & pipeline, const Context & context) +{ + auto & profile_streams = context.getDAGContext()->getProfileStreamsMap()[executor_id]; + pipeline.transform([&profile_streams](auto & stream) { profile_streams.push_back(stream); }); +} + +void PhysicalPlanNode::transform(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + transformImpl(pipeline, context, max_streams); + if (is_record_profile_streams) + recordProfileStreams(pipeline, context); + // todo modify logic after supporting window function. + context.getDAGContext()->updateFinalConcurrency(pipeline.streams.size(), max_streams); + restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.h b/dbms/src/Flash/Planner/PhysicalPlanNode.h new file mode 100644 index 00000000000..47d41dd68ff --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -0,0 +1,81 @@ +// 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. + +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +struct DAGPipeline; +class Context; +class DAGContext; + +class PhysicalPlanNode; +using PhysicalPlanNodePtr = std::shared_ptr; + +class PhysicalPlanNode +{ +public: + PhysicalPlanNode( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id); + + virtual ~PhysicalPlanNode() = default; + + virtual PhysicalPlanNodePtr children(size_t /*i*/) const = 0; + + virtual void setChild(size_t /*i*/, const PhysicalPlanNodePtr & /*new_child*/) = 0; + + const PlanType & tp() const { return type; } + + const String & execId() const { return executor_id; } + + const NamesAndTypes & getSchema() const { return schema; } + + virtual size_t childrenSize() const = 0; + + virtual void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); + + virtual void finalize(const Names & parent_require) = 0; + void finalize(); + + /// Obtain a sample block that contains the names and types of result columns. + virtual const Block & getSampleBlock() const = 0; + + void disableRecordProfileStreams() { is_record_profile_streams = false; } + + String toString(); + +protected: + virtual void transformImpl(DAGPipeline & /*pipeline*/, Context & /*context*/, size_t /*max_streams*/){}; + + void recordProfileStreams(DAGPipeline & pipeline, const Context & context); + + String executor_id; + PlanType type; + NamesAndTypes schema; + bool is_record_profile_streams = true; + + LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp b/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp index c2c37a33651..1b9e6bb89f4 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp @@ -24,9 +24,9 @@ void addPrefix(FmtBuffer & buffer, size_t level) buffer.append(String(level, ' ')); } -void doVisitToString(FmtBuffer & buffer, const PhysicalPlanPtr & physical_plan, size_t level) +void doVisitToString(FmtBuffer & buffer, const PhysicalPlanNodePtr & physical_plan, size_t level) { - visit(physical_plan, [&buffer, &level](const PhysicalPlanPtr & plan) { + visit(physical_plan, [&buffer, &level](const PhysicalPlanNodePtr & plan) { assert(plan); addPrefix(buffer, level); buffer.fmtAppend("{}\n", plan->toString()); @@ -45,7 +45,7 @@ void doVisitToString(FmtBuffer & buffer, const PhysicalPlanPtr & physical_plan, } } // namespace -String visitToString(const PhysicalPlanPtr & plan) +String visitToString(const PhysicalPlanNodePtr & plan) { FmtBuffer buffer; doVisitToString(buffer, plan, 0); diff --git a/dbms/src/Flash/Planner/PhysicalPlanVisitor.h b/dbms/src/Flash/Planner/PhysicalPlanVisitor.h index cec8d08869a..c2d6c701f6c 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanVisitor.h +++ b/dbms/src/Flash/Planner/PhysicalPlanVisitor.h @@ -12,14 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include namespace DB::PhysicalPlanVisitor { -/// visit physical plan tree and apply function. -/// f: (const PhysicalPlanPtr &) -> bool, return true to continue visit. +/// visit physical plan node tree and apply function. +/// f: (const PhysicalPlanNodePtr &) -> bool, return true to continue visit. template -void visit(const PhysicalPlanPtr & plan, FF && f) +void visit(const PhysicalPlanNodePtr & plan, FF && f) { if (f(plan)) { @@ -30,5 +30,5 @@ void visit(const PhysicalPlanPtr & plan, FF && f) } } -String visitToString(const PhysicalPlanPtr & plan); +String visitToString(const PhysicalPlanNodePtr & plan); } // namespace DB::PhysicalPlanVisitor diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index 470326701b9..b798123de71 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -15,8 +15,7 @@ #include #include #include -#include -#include +#include #include #include #include @@ -25,39 +24,39 @@ namespace DB { namespace { -void analyzePhysicalPlan(Context & context, PhysicalPlanBuilder & builder, const DAGQueryBlock & query_block) +void analyzePhysicalPlan(Context & context, PhysicalPlan & physical_plan, const DAGQueryBlock & query_block) { assert(query_block.source); - builder.build(query_block.source_name, query_block.source); + physical_plan.build(query_block.source_name, query_block.source); // selection on table scan had been executed in table scan. // In test mode, filter is not pushed down to table scan. if (query_block.selection && (!query_block.isTableScanSource() || context.getDAGContext()->isTest())) { - builder.build(query_block.selection_name, query_block.selection); + physical_plan.build(query_block.selection_name, query_block.selection); } if (query_block.aggregation) { - builder.build(query_block.aggregation_name, query_block.aggregation); + physical_plan.build(query_block.aggregation_name, query_block.aggregation); if (query_block.having) { - builder.build(query_block.having_name, query_block.having); + physical_plan.build(query_block.having_name, query_block.having); } } // TopN/Limit if (query_block.limit_or_topn) { - builder.build(query_block.limit_or_topn_name, query_block.limit_or_topn); + physical_plan.build(query_block.limit_or_topn_name, query_block.limit_or_topn); } - builder.buildFinalProjection(query_block.qb_column_prefix, query_block.isRootQueryBlock()); + physical_plan.buildFinalProjection(query_block.qb_column_prefix, query_block.isRootQueryBlock()); if (query_block.exchange_sender) { - builder.build(query_block.exchange_sender_name, query_block.exchange_sender); + physical_plan.build(query_block.exchange_sender_name, query_block.exchange_sender); } } } // namespace @@ -106,22 +105,22 @@ void Planner::restorePipelineConcurrency(DAGPipeline & pipeline) void Planner::executeImpl(DAGPipeline & pipeline) { - PhysicalPlanBuilder builder{context, log->identifier()}; + PhysicalPlan physical_plan{context, log->identifier()}; for (const auto & input_streams : input_streams_vec) { RUNTIME_ASSERT(!input_streams.empty(), log, "input streams cannot be empty"); - builder.buildSource(input_streams); + physical_plan.buildSource(input_streams); } - analyzePhysicalPlan(context, builder, query_block); + analyzePhysicalPlan(context, physical_plan, query_block); - auto physical_plan = builder.outputAndOptimize(); + physical_plan.outputAndOptimize(); LOG_FMT_DEBUG( log, "build physical plan: \n{}", - PhysicalPlanVisitor::visitToString(physical_plan)); + physical_plan.toString()); - physical_plan->transform(pipeline, context, max_streams); + physical_plan.transform(pipeline, context, max_streams); } } // namespace DB diff --git a/dbms/src/Flash/Planner/optimize.cpp b/dbms/src/Flash/Planner/optimize.cpp index 244ddd534b6..1e16bcb5469 100644 --- a/dbms/src/Flash/Planner/optimize.cpp +++ b/dbms/src/Flash/Planner/optimize.cpp @@ -20,7 +20,7 @@ namespace DB class Rule { public: - virtual PhysicalPlanPtr apply(const Context & context, PhysicalPlanPtr plan) = 0; + virtual PhysicalPlanNodePtr apply(const Context & context, PhysicalPlanNodePtr plan) = 0; virtual ~Rule() = default; }; @@ -29,7 +29,7 @@ using RulePtr = std::shared_ptr; class FinalizeRule : public Rule { public: - PhysicalPlanPtr apply(const Context &, PhysicalPlanPtr plan) override + PhysicalPlanNodePtr apply(const Context &, PhysicalPlanNodePtr plan) override { plan->finalize(); return plan; @@ -38,12 +38,13 @@ class FinalizeRule : public Rule static RulePtr create() { return std::make_shared(); } }; -PhysicalPlanPtr optimize(const Context & context, PhysicalPlanPtr plan) +PhysicalPlanNodePtr optimize(const Context & context, PhysicalPlanNodePtr plan) { assert(plan); static std::vector rules{FinalizeRule::create()}; for (const auto & rule : rules) plan = rule->apply(context, plan); + assert(plan); return plan; } } // namespace DB diff --git a/dbms/src/Flash/Planner/optimize.h b/dbms/src/Flash/Planner/optimize.h index 8ba738c9f77..8053ba5532b 100644 --- a/dbms/src/Flash/Planner/optimize.h +++ b/dbms/src/Flash/Planner/optimize.h @@ -14,10 +14,10 @@ #pragma once -#include +#include namespace DB { class Context; -PhysicalPlanPtr optimize(const Context & context, PhysicalPlanPtr plan); +PhysicalPlanNodePtr optimize(const Context & context, PhysicalPlanNodePtr plan); } // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp index 2a15031bd87..45e4586dd18 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -30,12 +30,12 @@ namespace DB { -PhysicalPlanPtr PhysicalAggregation::build( +PhysicalPlanNodePtr PhysicalAggregation::build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::Aggregation & aggregation, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h index bc9ef542b04..3d3d27384cb 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.h +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h @@ -24,18 +24,18 @@ namespace DB class PhysicalAggregation : public PhysicalUnary { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::Aggregation & aggregation, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); PhysicalAggregation( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_, + const PhysicalPlanNodePtr & child_, const ExpressionActionsPtr & before_agg_actions_, const Names & aggregation_keys_, const TiDB::TiDBCollators & aggregation_collators_, diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp index ae38c09206a..ee40e42e1aa 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -38,7 +38,7 @@ PhysicalExchangeReceiver::PhysicalExchangeReceiver( , mpp_exchange_receiver(mpp_exchange_receiver_) {} -PhysicalPlanPtr PhysicalExchangeReceiver::build( +PhysicalPlanNodePtr PhysicalExchangeReceiver::build( const Context & context, const String & executor_id, const LoggerPtr & log) diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h index beeec2b7513..6dd2412d821 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.h @@ -23,7 +23,7 @@ class ExchangeReceiver; class PhysicalExchangeReceiver : public PhysicalLeaf { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const Context & context, const String & executor_id, const LoggerPtr & log); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 4a4e4141f32..373b04a3941 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -24,11 +24,11 @@ namespace DB { -PhysicalPlanPtr PhysicalExchangeSender::build( +PhysicalPlanNodePtr PhysicalExchangeSender::build( const String & executor_id, const LoggerPtr & log, const tipb::ExchangeSender & exchange_sender, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h index 6664b1a2c9a..1323674c3d4 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h @@ -23,17 +23,17 @@ namespace DB class PhysicalExchangeSender : public PhysicalUnary { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const String & executor_id, const LoggerPtr & log, const tipb::ExchangeSender & exchange_sender, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); PhysicalExchangeSender( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_, + const PhysicalPlanNodePtr & child_, const std::vector & partition_col_ids_, const TiDB::TiDBCollators & collators_, const tipb::ExchangeType & exchange_type_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp index 0741514321c..c014f3b54af 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp @@ -23,12 +23,12 @@ namespace DB { -PhysicalPlanPtr PhysicalFilter::build( +PhysicalPlanNodePtr PhysicalFilter::build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::Selection & selection, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.h b/dbms/src/Flash/Planner/plans/PhysicalFilter.h index 07cc2163a33..27d050b4a61 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalFilter.h +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.h @@ -23,18 +23,18 @@ namespace DB class PhysicalFilter : public PhysicalUnary { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::Selection & selection, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); PhysicalFilter( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_, + const PhysicalPlanNodePtr & child_, const String & filter_column_, const ExpressionActionsPtr & before_filter_actions_) : PhysicalUnary(executor_id_, PlanType::Filter, schema_, req_id, child_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h index a2f6aaaf8ca..343ab66625c 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h @@ -15,14 +15,14 @@ #pragma once #include -#include +#include namespace DB { /** * A physical plan node with no children. */ -class PhysicalLeaf : public PhysicalPlan +class PhysicalLeaf : public PhysicalPlanNode { public: PhysicalLeaf( @@ -30,15 +30,15 @@ class PhysicalLeaf : public PhysicalPlan const PlanType & type_, const NamesAndTypes & schema_, const String & req_id) - : PhysicalPlan(executor_id_, type_, schema_, req_id) + : PhysicalPlanNode(executor_id_, type_, schema_, req_id) {} - PhysicalPlanPtr children(size_t) const override + PhysicalPlanNodePtr children(size_t) const override { throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); } - void setChild(size_t, const PhysicalPlanPtr &) override + void setChild(size_t, const PhysicalPlanNodePtr &) override { throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); } diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp index ed2989ab85e..2722e7e1869 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.cpp @@ -22,11 +22,11 @@ namespace DB { -PhysicalPlanPtr PhysicalLimit::build( +PhysicalPlanNodePtr PhysicalLimit::build( const String & executor_id, const LoggerPtr & log, const tipb::Limit & limit, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); auto physical_limit = std::make_shared( diff --git a/dbms/src/Flash/Planner/plans/PhysicalLimit.h b/dbms/src/Flash/Planner/plans/PhysicalLimit.h index a17b82ebd1e..24f6fe83044 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalLimit.h +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.h @@ -22,17 +22,17 @@ namespace DB class PhysicalLimit : public PhysicalUnary { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const String & executor_id, const LoggerPtr & log, const tipb::Limit & limit, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); PhysicalLimit( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_, + const PhysicalPlanNodePtr & child_, size_t limit_) : PhysicalUnary(executor_id_, PlanType::Limit, schema_, req_id, child_) , limit(limit_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp index 5a20a9550dd..1dc5b680937 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.cpp @@ -33,7 +33,7 @@ PhysicalMockExchangeReceiver::PhysicalMockExchangeReceiver( , mock_streams(mock_streams_) {} -PhysicalPlanPtr PhysicalMockExchangeReceiver::build( +PhysicalPlanNodePtr PhysicalMockExchangeReceiver::build( Context & context, const String & executor_id, const LoggerPtr & log, diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h index 93e394f6c0f..b269ee438a7 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.h @@ -23,7 +23,7 @@ namespace DB class PhysicalMockExchangeReceiver : public PhysicalLeaf { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( Context & context, const String & executor_id, const LoggerPtr & log, diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp index 0aef9fb6ca7..8405ee2283f 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp @@ -20,10 +20,10 @@ namespace DB { -PhysicalPlanPtr PhysicalMockExchangeSender::build( +PhysicalPlanNodePtr PhysicalMockExchangeSender::build( const String & executor_id, const LoggerPtr & log, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h index 4f325f07ad1..bfebf34c1ea 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h @@ -23,16 +23,16 @@ namespace DB class PhysicalMockExchangeSender : public PhysicalUnary { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const String & executor_id, const LoggerPtr & log, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); PhysicalMockExchangeSender( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_) + const PhysicalPlanNodePtr & child_) : PhysicalUnary(executor_id_, PlanType::MockExchangeSender, schema_, req_id, child_) {} diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp index c331c0500f0..0835fd557b1 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp @@ -25,12 +25,12 @@ namespace DB { -PhysicalPlanPtr PhysicalProjection::build( +PhysicalPlanNodePtr PhysicalProjection::build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::Projection & projection, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); @@ -63,11 +63,11 @@ PhysicalPlanPtr PhysicalProjection::build( return physical_projection; } -PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( +PhysicalPlanNodePtr PhysicalProjection::buildNonRootFinal( const Context & context, const LoggerPtr & log, const String & column_prefix, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); @@ -97,14 +97,14 @@ PhysicalPlanPtr PhysicalProjection::buildNonRootFinal( return physical_projection; } -PhysicalPlanPtr PhysicalProjection::buildRootFinal( +PhysicalPlanNodePtr PhysicalProjection::buildRootFinal( const Context & context, const LoggerPtr & log, const std::vector & require_schema, const std::vector & output_offsets, const String & column_prefix, bool keep_session_timezone_info, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalProjection.h b/dbms/src/Flash/Planner/plans/PhysicalProjection.h index f9af596293d..bd5e8140f28 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalProjection.h +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.h @@ -23,38 +23,38 @@ namespace DB class PhysicalProjection : public PhysicalUnary { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::Projection & projection, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); // Generate a project action to keep the schema of Block and tidb-schema the same, // and guarantee that left/right block of join don't have duplicated column names. - static PhysicalPlanPtr buildNonRootFinal( + static PhysicalPlanNodePtr buildNonRootFinal( const Context & context, const LoggerPtr & log, const String & column_prefix, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); // Generate a project action for root executor, // to keep the schema of Block and tidb-schema the same. // Because the output of the root executor is sent to other TiFlash or TiDB. - static PhysicalPlanPtr buildRootFinal( + static PhysicalPlanNodePtr buildRootFinal( const Context & context, const LoggerPtr & log, const std::vector & require_schema, const std::vector & output_offsets, const String & column_prefix, bool keep_session_timezone_info, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); PhysicalProjection( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_, + const PhysicalPlanNodePtr & child_, const String & extra_info_, const ExpressionActionsPtr & project_actions_) : PhysicalUnary(executor_id_, PlanType::Projection, schema_, req_id, child_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.cpp b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp index 55eb39a82b6..b694622314a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalSource.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.cpp @@ -18,7 +18,7 @@ namespace DB { -PhysicalPlanPtr PhysicalSource::build( +PhysicalPlanNodePtr PhysicalSource::build( const BlockInputStreams & source_streams, const LoggerPtr & log) { diff --git a/dbms/src/Flash/Planner/plans/PhysicalSource.h b/dbms/src/Flash/Planner/plans/PhysicalSource.h index e3e877bf4b4..eb178583c6a 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalSource.h +++ b/dbms/src/Flash/Planner/plans/PhysicalSource.h @@ -22,7 +22,7 @@ namespace DB class PhysicalSource : public PhysicalLeaf { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const BlockInputStreams & source_streams, const LoggerPtr & log); diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp index 53286247fe5..d572435d645 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -24,12 +24,12 @@ namespace DB { -PhysicalPlanPtr PhysicalTopN::build( +PhysicalPlanNodePtr PhysicalTopN::build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::TopN & top_n, - const PhysicalPlanPtr & child) + const PhysicalPlanNodePtr & child) { assert(child); diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.h b/dbms/src/Flash/Planner/plans/PhysicalTopN.h index b1ce4bfe87a..bfabb5f4261 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.h +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.h @@ -24,18 +24,18 @@ namespace DB class PhysicalTopN : public PhysicalUnary { public: - static PhysicalPlanPtr build( + static PhysicalPlanNodePtr build( const Context & context, const String & executor_id, const LoggerPtr & log, const tipb::TopN & top_n, - const PhysicalPlanPtr & child); + const PhysicalPlanNodePtr & child); PhysicalTopN( const String & executor_id_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_, + const PhysicalPlanNodePtr & child_, const SortDescription & order_descr_, const ExpressionActionsPtr & before_sort_actions_, size_t limit_) diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/plans/PhysicalUnary.h index 43133f79a42..37e0cb707b2 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalUnary.h +++ b/dbms/src/Flash/Planner/plans/PhysicalUnary.h @@ -15,7 +15,7 @@ #pragma once #include -#include +#include #include #include @@ -24,7 +24,7 @@ namespace DB /** * A physical plan node with single child. */ -class PhysicalUnary : public PhysicalPlan +class PhysicalUnary : public PhysicalPlanNode { public: PhysicalUnary( @@ -32,20 +32,20 @@ class PhysicalUnary : public PhysicalPlan const PlanType & type_, const NamesAndTypes & schema_, const String & req_id, - const PhysicalPlanPtr & child_) - : PhysicalPlan(executor_id_, type_, schema_, req_id) + const PhysicalPlanNodePtr & child_) + : PhysicalPlanNode(executor_id_, type_, schema_, req_id) , child(child_) { RUNTIME_ASSERT(child, log, "children(0) shouldn't be nullptr"); } - PhysicalPlanPtr children(size_t i) const override + PhysicalPlanNodePtr children(size_t i) const override { RUNTIME_ASSERT(i == 0, log, "child_index({}) shouldn't >= childrenSize({})", i, childrenSize()); return child; } - void setChild(size_t i, const PhysicalPlanPtr & new_child) override + void setChild(size_t i, const PhysicalPlanNodePtr & new_child) override { RUNTIME_ASSERT(i == 0, log, "child_index({}) should not >= childrenSize({})", i, childrenSize()); RUNTIME_ASSERT(new_child, log, "new_child for child_index({}) shouldn't be nullptr", i); @@ -56,6 +56,6 @@ class PhysicalUnary : public PhysicalPlan size_t childrenSize() const override { return 1; }; protected: - PhysicalPlanPtr child; + PhysicalPlanNodePtr child; }; } // namespace DB diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index 77e35c7ed93..7ad06fbeb04 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -13,7 +13,7 @@ // limitations under the License. #include -#include +#include #include #include #include @@ -50,17 +50,17 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest dag_context.setColumnsForTest(context.executorIdColumnsMap()); context.context.setDAGContext(&dag_context); - PhysicalPlanBuilder builder{context.context, log->identifier()}; + PhysicalPlan physical_plan{context.context, log->identifier()}; assert(request); - builder.build(request.get()); - auto physical_plan = builder.outputAndOptimize(); + physical_plan.build(request.get()); + physical_plan.outputAndOptimize(); - ASSERT_EQ(Poco::trim(expected_physical_plan), Poco::trim(PhysicalPlanVisitor::visitToString(physical_plan))); + ASSERT_EQ(Poco::trim(expected_physical_plan), Poco::trim(physical_plan.toString())); BlockInputStreamPtr final_stream; { DAGPipeline pipeline; - physical_plan->transform(pipeline, context.context, max_streams); + physical_plan.transform(pipeline, context.context, max_streams); // TODO support non-joined streams. assert(pipeline.streams.size() == 1 && pipeline.streams_with_non_joined_data.empty()); final_stream = pipeline.firstStream(); From 564eeea43c96f877e3921f80b618b1195ea822e7 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 8 Jul 2022 15:41:27 +0800 Subject: [PATCH 56/57] format --- dbms/src/Flash/Planner/PhysicalPlanNode.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp index 975584a8e61..293dae3bb28 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp @@ -16,8 +16,8 @@ #include #include #include -#include #include +#include #include namespace DB From 240d6d0617116aced051390dbfb7f538e95b8072 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Fri, 8 Jul 2022 16:52:54 +0800 Subject: [PATCH 57/57] fix license-checker fail like https://github.com/pingcap/tiflash/pull/5318 --- .github/workflows/license-checker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/license-checker.yml b/.github/workflows/license-checker.yml index c4c510677b1..85bce2ca9e1 100644 --- a/.github/workflows/license-checker.yml +++ b/.github/workflows/license-checker.yml @@ -15,7 +15,7 @@ jobs: steps: - uses: actions/checkout@v2 - name: Check License Header - uses: apache/skywalking-eyes@main + uses: apache/skywalking-eyes@v0.3.0 env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} with: