diff --git a/dbms/src/Common/TiFlashException.h b/dbms/src/Common/TiFlashException.h index 3b4e3d75813..b00fdd3c7c9 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/Core/Block.cpp b/dbms/src/Core/Block.cpp index bd67e59df77..69fc45ec3c1 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -49,6 +49,15 @@ Block::Block(const ColumnsWithTypeAndName & data_) } +Block::Block(const NamesAndTypes & names_and_types) +{ + data.reserve(names_and_types.size()); + for (const auto & name_and_type : names_and_types) + data.emplace_back(name_and_type.type, name_and_type.name); + initializeIndexByName(); +} + + void Block::initializeIndexByName() { for (size_t i = 0, size = data.size(); i < size; ++i) diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 04378c7553f..206d6d959cc 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -51,6 +51,7 @@ class Block Block() = default; Block(std::initializer_list il); explicit Block(const ColumnsWithTypeAndName & data_); + explicit Block(const NamesAndTypes & names_and_types); /// insert the column at the specified position void insert(size_t position, const ColumnWithTypeAndName & elem); diff --git a/dbms/src/Core/NamesAndTypes.cpp b/dbms/src/Core/NamesAndTypes.cpp index c290471da40..cb6b5714253 100644 --- a/dbms/src/Core/NamesAndTypes.cpp +++ b/dbms/src/Core/NamesAndTypes.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -31,6 +32,32 @@ namespace ErrorCodes extern const int THERE_IS_NO_COLUMN; } +String dumpJsonStructure(const NamesAndTypes & names_and_types) +{ + FmtBuffer bf; + bf.append("["); + bf.joinStr( + names_and_types.cbegin(), + names_and_types.cend(), + [](const auto & name_and_type, FmtBuffer & fb) { + fb.fmtAppend( + R"json({{"name":"{}","type":{}}})json", + name_and_type.name, + (name_and_type.type ? "\"" + name_and_type.type->getName() + "\"" : "null")); + }, + ", "); + bf.append("]"); + return bf.toString(); +} + +Names toNames(const NamesAndTypes & names_and_types) +{ + Names names; + names.reserve(names_and_types.size()); + for (const auto & name_and_type : names_and_types) + names.push_back(name_and_type.name); + return names; +} void NamesAndTypesList::readText(ReadBuffer & buf) { diff --git a/dbms/src/Core/NamesAndTypes.h b/dbms/src/Core/NamesAndTypes.h index af1a64abc32..3bc71ac4760 100644 --- a/dbms/src/Core/NamesAndTypes.h +++ b/dbms/src/Core/NamesAndTypes.h @@ -50,6 +50,10 @@ struct NameAndTypePair using NamesAndTypes = std::vector; +String dumpJsonStructure(const NamesAndTypes & names_and_types); + +Names toNames(const NamesAndTypes & names_and_types); + class NamesAndTypesList : public std::list { public: diff --git a/dbms/src/Flash/CMakeLists.txt b/dbms/src/Flash/CMakeLists.txt index b32202fa6c5..5162f04fa1b 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_headers_and_sources(flash_service ./Management) @@ -25,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/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index c2136933a14..8ea5a76df55 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -117,13 +118,6 @@ constexpr UInt64 NO_ENGINE_SUBSTITUTION = 1ul << 30ul; constexpr UInt64 ALLOW_INVALID_DATES = 1ul << 32ul; } // namespace TiDBSQLMode -inline bool enableFineGrainedShuffle(uint64_t stream_count) -{ - return stream_count > 0; -} - -static constexpr std::string_view enableFineGrainedShuffleExtraInfo = "enable fine grained shuffle"; - /// A context used to track the information that needs to be passed around during DAG planning. class DAGContext { diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 600c860f252..5df5c95c6e0 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -19,13 +19,12 @@ #include #include #include -#include #include #include #include +#include #include #include -#include #include #include #include @@ -89,10 +88,9 @@ void DAGDriver::execute() try { auto start_time = Clock::now(); - DAGQuerySource dag(context); DAGContext & dag_context = *context.getDAGContext(); - BlockIO streams = executeQuery(dag, context, internal, QueryProcessingStage::Complete); + BlockIO streams = executeQuery(context, internal, QueryProcessingStage::Complete); if (!streams.in || streams.out) // Only query is allowed, so streams.in must not be null and streams.out must be null throw TiFlashException("DAG is not query.", Errors::Coprocessor::Internal); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 63d35abe26d..c42312b95c3 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; } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index df647520094..62600ba04a0 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include #include @@ -161,7 +162,7 @@ void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_s { if (context.columnsForTestEmpty() || context.columnsForTest(table_scan.getTableScanExecutorID()).empty()) { - auto names_and_types = genNamesAndTypes(table_scan); + auto names_and_types = genNamesAndTypes(table_scan, "mock_table_scan"); auto columns_with_type_and_name = getColumnWithTypeAndName(names_and_types); analyzer = std::make_unique(std::move(names_and_types), context); for (size_t i = 0; i < max_streams; ++i) @@ -180,7 +181,7 @@ void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_s void DAGQueryBlockInterpreter::handleTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline) { - const auto push_down_filter = PushDownFilter::toPushDownFilter(query_block.selection_name, query_block.selection); + const auto push_down_filter = PushDownFilter::pushDownFilterFrom(query_block.selection_name, query_block.selection); DAGStorageInterpreter storage_interpreter(context, table_scan, push_down_filter, max_streams); storage_interpreter.execute(pipeline); diff --git a/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.h new file mode 100644 index 00000000000..6f794375a30 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/FineGrainedShuffle.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 +{ +static constexpr std::string_view enableFineGrainedShuffleExtraInfo = "enable fine grained shuffle"; + +inline bool enableFineGrainedShuffle(uint64_t stream_count) +{ + return stream_count > 0; +} + +struct FineGrainedShuffle +{ + explicit FineGrainedShuffle(const tipb::Executor * executor) + : stream_count(executor ? executor->fine_grained_shuffle_stream_count() : 0) + , batch_size(executor ? executor->fine_grained_shuffle_batch_size() : 0) + {} + + bool enable() const + { + return enableFineGrainedShuffle(stream_count); + } + + const UInt64 stream_count; + const UInt64 batch_size; +}; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index efb8a08f1d8..66a2363e6e2 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -11,34 +11,50 @@ // 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 namespace DB { -NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan) +namespace +{ +DataTypePtr getPkType(const ColumnInfo & column_info) +{ + const auto & pk_data_type = getDataTypeByColumnInfoForComputingLayer(column_info); + /// primary key type must be tidb_pk_column_int_type or tidb_pk_column_string_type. + RUNTIME_CHECK( + pk_data_type->equals(*MutableSupport::tidb_pk_column_int_type) || pk_data_type->equals(*MutableSupport::tidb_pk_column_string_type), + Exception( + fmt::format( + "Actual pk_data_type {} is not {} or {}", + pk_data_type->getName(), + MutableSupport::tidb_pk_column_int_type->getName(), + MutableSupport::tidb_pk_column_string_type->getName()), + ErrorCodes::LOGICAL_ERROR)); + return pk_data_type; +} +} // namespace + +NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan, const StringRef & column_prefix) { NamesAndTypes names_and_types; names_and_types.reserve(table_scan.getColumnSize()); for (Int32 i = 0; i < table_scan.getColumnSize(); ++i) { - TiDB::ColumnInfo column_info; - const auto & ci = table_scan.getColumns()[i]; - column_info.tp = static_cast(ci.tp()); - column_info.id = ci.column_id(); - + auto column_info = TiDB::toTiDBColumnInfo(table_scan.getColumns()[i]); switch (column_info.id) { case TiDBPkColumnID: - // TODO: need to check if the type of pk_handle_columns matches the type that used in delta merge tree. - names_and_types.emplace_back(MutableSupport::tidb_pk_column_name, getDataTypeByColumnInfoForComputingLayer(column_info)); + names_and_types.emplace_back(MutableSupport::tidb_pk_column_name, getPkType(column_info)); break; case ExtraTableIDColumnID: names_and_types.emplace_back(MutableSupport::extra_table_id_column_name, MutableSupport::extra_table_id_column_type); break; default: - names_and_types.emplace_back(fmt::format("mock_table_scan_{}", i), getDataTypeByColumnInfoForComputingLayer(column_info)); + names_and_types.emplace_back(fmt::format("{}_{}", column_prefix, i), getDataTypeByColumnInfoForComputingLayer(column_info)); } } return names_and_types; diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h index 96f202d800e..cdd032b3c8c 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h @@ -19,10 +19,11 @@ #include #include #include +#include namespace DB { -NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan); +NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan, const StringRef & column_prefix); ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_types); NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/PushDownFilter.cpp b/dbms/src/Flash/Coprocessor/PushDownFilter.cpp index bc08d9ac8e3..09691e3f015 100644 --- a/dbms/src/Flash/Coprocessor/PushDownFilter.cpp +++ b/dbms/src/Flash/Coprocessor/PushDownFilter.cpp @@ -49,17 +49,22 @@ tipb::Executor * PushDownFilter::constructSelectionForRemoteRead(tipb::Executor } } -PushDownFilter PushDownFilter::toPushDownFilter(const String & executor_id, const tipb::Executor * executor) +PushDownFilter PushDownFilter::pushDownFilterFrom(const String & executor_id, const tipb::Executor * executor) { if (!executor || !executor->has_selection()) { return {"", {}}; } + return pushDownFilterFrom(executor_id, executor->selection()); +} + +PushDownFilter PushDownFilter::pushDownFilterFrom(const String & executor_id, const tipb::Selection & selection) +{ std::vector conditions; - for (const auto & condition : executor->selection().conditions()) + for (const auto & condition : selection.conditions()) conditions.push_back(&condition); return {executor_id, conditions}; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/PushDownFilter.h b/dbms/src/Flash/Coprocessor/PushDownFilter.h index 9645264a71c..47183c5115b 100644 --- a/dbms/src/Flash/Coprocessor/PushDownFilter.h +++ b/dbms/src/Flash/Coprocessor/PushDownFilter.h @@ -23,7 +23,11 @@ namespace DB { struct PushDownFilter { - static PushDownFilter toPushDownFilter(const String & executor_id, const tipb::Executor * executor); + static PushDownFilter pushDownFilterFrom(const String & executor_id, const tipb::Executor * executor); + + static PushDownFilter pushDownFilterFrom(const String & executor_id, const tipb::Selection & selection); + + PushDownFilter() = default; PushDownFilter( const String & executor_id_, @@ -36,4 +40,4 @@ struct PushDownFilter String executor_id; std::vector conditions; }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 3538d0238e3..aec52513929 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 2d4bbaeaf68..72a7c9d1435 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -30,8 +30,8 @@ #include #include #include +#include #include -#include #include #include #include @@ -319,8 +319,7 @@ void MPPTask::preprocess() { auto start_time = Clock::now(); initExchangeReceivers(); - DAGQuerySource dag(*context); - executeQuery(dag, *context, false, QueryProcessingStage::Complete); + executeQuery(*context); auto end_time = Clock::now(); dag_context->compile_time_ns = std::chrono::duration_cast(end_time - start_time).count(); mpp_task_statistics.setCompileTimestamp(start_time, end_time); diff --git a/dbms/src/Flash/Planner/ExecutorIdGenerator.h b/dbms/src/Flash/Planner/ExecutorIdGenerator.h new file mode 100644 index 00000000000..ed70d2ce93f --- /dev/null +++ b/dbms/src/Flash/Planner/ExecutorIdGenerator.h @@ -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. + +#pragma once + +#include +#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()); + RUNTIME_CHECK( + ids.find(executor_id) == ids.end(), + 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); + case tipb::ExecType::TypeTableScan: + case tipb::ExecType::TypePartitionTableScan: + return fmt::format("{}_table_scan", ++current_id); + case tipb::ExecType::TypeSort: + return fmt::format("{}_sort", ++current_id); + case tipb::ExecType::TypeWindow: + return fmt::format("{}_window", ++current_id); + case tipb::ExecType::TypeJoin: + return fmt::format("{}_join", ++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/FinalizeHelper.cpp b/dbms/src/Flash/Planner/FinalizeHelper.cpp new file mode 100644 index 00000000000..6b2c41a81cd --- /dev/null +++ b/dbms/src/Flash/Planner/FinalizeHelper.cpp @@ -0,0 +1,88 @@ +// 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) +{ + assert(columns_from_previous >= actions->getRequiredColumnsWithTypes().size()); + 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) + { + RUNTIME_CHECK( + schema_set.find(parent_require_column) != schema_set.end(), + TiFlashException( + fmt::format("schema {} don't contain parent require column: {}", DB::dumpJsonStructure(schema), parent_require_column), + Errors::Planner::Internal)); + } +} + +void checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema) +{ + for (const auto & schema_column : schema) + { + RUNTIME_CHECK( + sample_block.has(schema_column.name), + TiFlashException( + fmt::format("sample block {} don't contain schema column: {}", sample_block.dumpJsonStructure(), schema_column.name), + Errors::Planner::Internal)); + + const auto & type_in_sample_block = sample_block.getByName(schema_column.name).type; + const auto & type_in_schema = schema_column.type; + RUNTIME_CHECK( + type_in_sample_block->equals(*type_in_schema), + TiFlashException( + fmt::format( + "the type of column `{}` in sample block `{}` is different from the one in schema `{}`", + schema_column.name, + type_in_sample_block->getName(), + type_in_schema->getName()), + Errors::Planner::Internal)); + } +} + +void checkSampleBlockContainsParentRequire(const Block & sample_block, const Names & parent_require) +{ + for (const auto & parent_require_column : parent_require) + { + RUNTIME_CHECK( + sample_block.has(parent_require_column), + TiFlashException( + fmt::format("sample block {} don't contain parent_require column: {}", sample_block.dumpJsonStructure(), parent_require_column), + 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..f3a30fb2271 --- /dev/null +++ b/dbms/src/Flash/Planner/FinalizeHelper.h @@ -0,0 +1,32 @@ +// 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 checkSampleBlockContainsSchema(const Block & sample_block, const NamesAndTypes & schema); + +void checkSampleBlockContainsParentRequire(const Block & sample_block, const Names & parent_require); +} // 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..1384457729e --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -0,0 +1,249 @@ +// 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 +#include +#include +#include +#include +#include + +namespace DB +{ +namespace +{ +bool pushDownSelection(const PhysicalPlanNodePtr & plan, const String & executor_id, const tipb::Selection & selection) +{ + if (plan->tp() == PlanType::TableScan) + { + auto physical_table_scan = std::static_pointer_cast(plan); + return physical_table_scan->pushDownFilter(executor_id, selection); + } + return false; +} + +void fillOrderForListBasedExecutors(DAGContext & dag_context, const PhysicalPlanNodePtr & root_node) +{ + auto & list_based_executors_order = dag_context.list_based_executors_order; + PhysicalPlanVisitor::visitPostOrder(root_node, [&](const PhysicalPlanNodePtr & plan) { + assert(plan); + if (plan->isRecordProfileStreams()) + list_based_executors_order.push_back(plan->execId()); + }); +} +} // namespace + +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: + { + auto child = popBack(); + if (pushDownSelection(child, executor_id, executor->selection())) + pushBack(child); + else + pushBack(PhysicalFilter::build(context, executor_id, log, executor->selection(), child)); + break; + } + case tipb::ExecType::TypeAggregation: + case tipb::ExecType::TypeStreamAgg: + pushBack(PhysicalAggregation::build(context, executor_id, log, executor->aggregation(), popBack())); + break; + case tipb::ExecType::TypeExchangeSender: + { + buildFinalProjection(fmt::format("{}_", executor_id), true); + if (unlikely(context.isExecutorTest())) + pushBack(PhysicalMockExchangeSender::build(executor_id, log, popBack())); + else + { + // for MPP test, we can use real exchangeSender to run an query across different compute nodes + // or use one compute node to simulate MPP process. + pushBack(PhysicalExchangeSender::build(executor_id, log, executor->exchange_sender(), FineGrainedShuffle(executor), popBack())); + } + break; + } + case tipb::ExecType::TypeExchangeReceiver: + { + if (unlikely(context.isExecutorTest())) + pushBack(PhysicalMockExchangeReceiver::build(context, executor_id, log, executor->exchange_receiver())); + else + { + // for MPP test, we can use real exchangeReceiver to run an query across different compute nodes + // or use one compute node to simulate MPP process. + pushBack(PhysicalExchangeReceiver::build(context, executor_id, log)); + } + break; + } + case tipb::ExecType::TypeProjection: + pushBack(PhysicalProjection::build(context, executor_id, log, executor->projection(), popBack())); + break; + case tipb::ExecType::TypeWindow: + pushBack(PhysicalWindow::build(context, executor_id, log, executor->window(), FineGrainedShuffle(executor), popBack())); + break; + case tipb::ExecType::TypeSort: + pushBack(PhysicalWindowSort::build(context, executor_id, log, executor->sort(), FineGrainedShuffle(executor), popBack())); + break; + case tipb::ExecType::TypeTableScan: + case tipb::ExecType::TypePartitionTableScan: + { + TiDBTableScan table_scan(executor, executor_id, dagContext()); + if (unlikely(context.isTest())) + pushBack(PhysicalMockTableScan::build(context, executor_id, log, table_scan)); + else + pushBack(PhysicalTableScan::build(executor_id, log, table_scan)); + dagContext().table_scan_executor_id = executor_id; + break; + } + case tipb::ExecType::TypeJoin: + { + /// Both sides of the join need to have non-root-final-projection to ensure that + /// there are no duplicate columns in the blocks on the build and probe sides. + buildFinalProjection(fmt::format("{}_r_", executor_id), false); + auto right = popBack(); + + buildFinalProjection(fmt::format("{}_l_", executor_id), false); + auto left = popBack(); + + pushBack(PhysicalJoin::build(context, executor_id, log, executor->join(), left, right)); + break; + } + default: + throw TiFlashException(fmt::format("{} executor is not supported", executor->tp()), Errors::Planner::Unimplemented); + } +} + +void PhysicalPlan::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 & PhysicalPlan::dagContext() const +{ + return *context.getDAGContext(); +} + +void PhysicalPlan::pushBack(const PhysicalPlanNodePtr & plan_node) +{ + assert(plan_node); + cur_plan_nodes.push_back(plan_node); +} + +PhysicalPlanNodePtr PhysicalPlan::popBack() +{ + RUNTIME_CHECK(!cur_plan_nodes.empty(), 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; +} + +/// For MPP, root final projection has been added under PhysicalExchangeSender or PhysicalMockExchangeSender. +/// For batchcop/cop that without PhysicalExchangeSender or PhysicalMockExchangeSender, We need to add root final projection. +void PhysicalPlan::addRootFinalProjectionIfNeed() +{ + assert(root_node); + if (root_node->tp() != PlanType::ExchangeSender && root_node->tp() != PlanType::MockExchangeSender) + { + pushBack(root_node); + buildFinalProjection(fmt::format("{}_", root_node->execId()), true); + root_node = popBack(); + } +} + +void PhysicalPlan::outputAndOptimize() +{ + RUNTIME_ASSERT(!root_node, log, "root_node should 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(); + addRootFinalProjectionIfNeed(); + + LOG_FMT_DEBUG( + log, + "build unoptimized physical plan: \n{}", + toString()); + + root_node = optimize(context, root_node, log); + LOG_FMT_DEBUG( + log, + "build optimized physical plan: \n{}", + toString()); + + RUNTIME_ASSERT(root_node, log, "root_node shouldn't be nullptr after `outputAndOptimize`"); + + if (!dagContext().return_executor_id) + fillOrderForListBasedExecutors(dagContext(), root_node); +} + +String PhysicalPlan::toString() const +{ + assert(root_node); + return PhysicalPlanVisitor::visitToString(root_node); +} + +void PhysicalPlan::transform(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + 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 new file mode 100644 index 00000000000..09871e94a04 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlan.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 +#include +#include +#include + +namespace DB +{ +class PhysicalPlan +{ +public: + explicit PhysicalPlan(Context & context_, const String & req_id) + : context(context_) + , log(Logger::get("PhysicalPlan", req_id)) + {} + + void build(const tipb::DAGRequest * dag_request); + + // after outputAndOptimize, the physical plan node tree is done. + void outputAndOptimize(); + + String toString() const; + + void transform(DAGPipeline & pipeline, Context & context, size_t max_streams); + +private: + void addRootFinalProjectionIfNeed(); + + void build(const String & executor_id, const tipb::Executor * executor); + + void buildFinalProjection(const String & column_prefix, bool is_root); + + PhysicalPlanNodePtr popBack(); + + void pushBack(const PhysicalPlanNodePtr & plan); + + DAGContext & dagContext() const; + +private: + std::vector cur_plan_nodes{}; + + // hold the root node of physical plan node tree after `outputAndOptimize`. + PhysicalPlanNodePtr root_node; + + Context & context; + + LoggerPtr 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..923743cdae8 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.cpp @@ -0,0 +1,39 @@ +// 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 +{ +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()); +} + +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()); +} +} // 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..89c1963feaa --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanHelper.h @@ -0,0 +1,25 @@ +// 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::PhysicalPlanHelper +{ +ExpressionActionsPtr newActions(const Block & input_block, const Context & context); + +ExpressionActionsPtr newActions(const NamesAndTypes & input_columns, const Context & context); +} // namespace DB::PhysicalPlanHelper diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp new file mode 100644 index 00000000000..82482548737 --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.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 +#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(DB::toNames(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); + if (is_restore_concurrency) + { + 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..e4437fe27ef --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -0,0 +1,87 @@ +// 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; + + bool isRecordProfileStreams() const { return is_record_profile_streams; } + + void disableRecordProfileStreams() { is_record_profile_streams = false; } + + void disableRestoreConcurrency() { is_restore_concurrency = 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; + bool is_restore_concurrency = true; + + LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp b/dbms/src/Flash/Planner/PhysicalPlanVisitor.cpp new file mode 100644 index 00000000000..1b9e6bb89f4 --- /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 PhysicalPlanNodePtr & physical_plan, size_t level) +{ + visit(physical_plan, [&buffer, &level](const PhysicalPlanNodePtr & 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 PhysicalPlanNodePtr & 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..be523542caf --- /dev/null +++ b/dbms/src/Flash/Planner/PhysicalPlanVisitor.h @@ -0,0 +1,46 @@ +// 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 node tree and apply function. +/// f: (const PhysicalPlanNodePtr &) -> bool, return true to continue visit. +template +void visit(const PhysicalPlanNodePtr & plan, FF && f) +{ + if (f(plan)) + { + for (size_t i = 0; i < plan->childrenSize(); ++i) + { + visit(plan->children(i), std::forward(f)); + } + } +} + +/// visit physical plan node tree in reverse order and apply function. +/// f: (const PhysicalPlanNodePtr &). +template +void visitPostOrder(const PhysicalPlanNodePtr & plan, FF && f) +{ + for (size_t i = 0; i < plan->childrenSize(); ++i) + { + visitPostOrder(plan->children(i), std::forward(f)); + } + f(plan); +} + +String visitToString(const PhysicalPlanNodePtr & plan); +} // namespace DB::PhysicalPlanVisitor diff --git a/dbms/src/Flash/Planner/PlanQuerySource.cpp b/dbms/src/Flash/Planner/PlanQuerySource.cpp new file mode 100644 index 00000000000..2085dc6aff9 --- /dev/null +++ b/dbms/src/Flash/Planner/PlanQuerySource.cpp @@ -0,0 +1,43 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +namespace DB +{ +PlanQuerySource::PlanQuerySource(Context & context_) + : context(context_) +{} + +std::tuple PlanQuerySource::parse(size_t) +{ + // this is a way to avoid NPE when the MergeTreeDataSelectExecutor trying + // to extract key range of the query. + // todo find a way to enable key range extraction for dag query + return {getDAGContext().dag_request->DebugString(), makeDummyQuery()}; +} + +String PlanQuerySource::str(size_t) +{ + return getDAGContext().dag_request->DebugString(); +} + +std::unique_ptr PlanQuerySource::interpreter(Context &, QueryProcessingStage::Enum) +{ + return std::make_unique(context, *this); +} + +} // namespace DB diff --git a/dbms/src/Flash/Planner/PlanQuerySource.h b/dbms/src/Flash/Planner/PlanQuerySource.h new file mode 100644 index 00000000000..a54c5791e11 --- /dev/null +++ b/dbms/src/Flash/Planner/PlanQuerySource.h @@ -0,0 +1,39 @@ +// 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 PlanQuerySource : public IQuerySource +{ +public: + explicit PlanQuerySource(Context & context_); + + std::tuple parse(size_t) override; + String str(size_t max_query_size) override; + std::unique_ptr interpreter(Context & context, QueryProcessingStage::Enum stage) override; + + DAGContext & getDAGContext() const { return *context.getDAGContext(); } + const tipb::DAGRequest & getDAGRequest() const { return *getDAGContext().dag_request; } + +private: + Context & context; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Planner/PlanType.cpp b/dbms/src/Flash/Planner/PlanType.cpp new file mode 100644 index 00000000000..867b4845e9b --- /dev/null +++ b/dbms/src/Flash/Planner/PlanType.cpp @@ -0,0 +1,46 @@ +// 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 PlanType::toString() const +{ + switch (enum_value) + { +#define M(t) \ + case t: \ + return #t; + M(Limit) + M(TopN) + M(Filter) + M(Aggregation) + M(ExchangeSender) + M(MockExchangeSender) + M(ExchangeReceiver) + M(MockExchangeReceiver) + M(Projection) + M(Window) + M(WindowSort) + M(TableScan) + M(MockTableScan) + M(Join) +#undef M + default: + throw TiFlashException("Unknown PlanType", Errors::Planner::Internal); + } +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/PlanType.h b/dbms/src/Flash/Planner/PlanType.h new file mode 100644 index 00000000000..be71f5c3f48 --- /dev/null +++ b/dbms/src/Flash/Planner/PlanType.h @@ -0,0 +1,59 @@ +// 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 +{ +struct PlanType +{ + enum PlanTypeEnum + { + Limit = 0, + TopN = 1, + Filter = 2, + Aggregation = 3, + ExchangeSender = 4, + MockExchangeSender = 5, + ExchangeReceiver = 6, + MockExchangeReceiver = 7, + Projection = 8, + Window = 9, + WindowSort = 10, + TableScan = 11, + MockTableScan = 12, + Join = 13, + }; + 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; + } + + operator int() const // NOLINT(google-explicit-constructor) + { + return this->enum_value; + } + + String toString() const; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp new file mode 100644 index 00000000000..43611db195b --- /dev/null +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -0,0 +1,57 @@ +// 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 PlanQuerySource & plan_source_) + : context(context_) + , plan_source(plan_source_) + , max_streams(context.getMaxStreams()) + , log(Logger::get("Planner", dagContext().log ? dagContext().log->identifier() : "")) +{} + +BlockIO Planner::execute() +{ + DAGPipeline pipeline; + executeImpl(pipeline); + executeCreatingSets(pipeline, context, max_streams, log); + BlockIO res; + res.in = pipeline.firstStream(); + return res; +} + +DAGContext & Planner::dagContext() const +{ + return *context.getDAGContext(); +} + +void Planner::executeImpl(DAGPipeline & pipeline) +{ + PhysicalPlan physical_plan{context, log->identifier()}; + + physical_plan.build(&plan_source.getDAGRequest()); + physical_plan.outputAndOptimize(); + + physical_plan.transform(pipeline, context, max_streams); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/Planner.h b/dbms/src/Flash/Planner/Planner.h new file mode 100644 index 00000000000..bebb18dab2c --- /dev/null +++ b/dbms/src/Flash/Planner/Planner.h @@ -0,0 +1,53 @@ +// 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 Context; +class DAGContext; + +class Planner : public IInterpreter +{ +public: + Planner( + Context & context_, + const PlanQuerySource & plan_source_); + + ~Planner() = default; + + BlockIO execute() override; + +private: + DAGContext & dagContext() const; + + void executeImpl(DAGPipeline & pipeline); + +private: + Context & context; + + const PlanQuerySource & plan_source; + + /// Max streams we will do processing. + size_t max_streams = 1; + + LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/optimize.cpp b/dbms/src/Flash/Planner/optimize.cpp new file mode 100644 index 00000000000..28e026eca84 --- /dev/null +++ b/dbms/src/Flash/Planner/optimize.cpp @@ -0,0 +1,52 @@ +// 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 +{ +class Rule +{ +public: + virtual PhysicalPlanNodePtr apply(const Context & context, PhysicalPlanNodePtr plan, const LoggerPtr & log) = 0; + + virtual ~Rule() = default; +}; +using RulePtr = std::shared_ptr; + +class FinalizeRule : public Rule +{ +public: + PhysicalPlanNodePtr apply(const Context &, PhysicalPlanNodePtr plan, const LoggerPtr &) override + { + plan->finalize(); + return plan; + } + + static RulePtr create() { return std::make_shared(); } +}; + +PhysicalPlanNodePtr optimize(const Context & context, PhysicalPlanNodePtr plan, const LoggerPtr & log) +{ + assert(plan); + static std::vector rules{FinalizeRule::create()}; + for (const auto & rule : rules) + { + plan = rule->apply(context, plan, log); + assert(plan); + } + return plan; +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/optimize.h b/dbms/src/Flash/Planner/optimize.h new file mode 100644 index 00000000000..7821331369c --- /dev/null +++ b/dbms/src/Flash/Planner/optimize.h @@ -0,0 +1,24 @@ +// 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 Context; +PhysicalPlanNodePtr optimize(const Context & context, PhysicalPlanNodePtr plan, const LoggerPtr & log); +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp new file mode 100644 index 00000000000..58c79f43951 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -0,0 +1,182 @@ +// 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 +{ +PhysicalPlanNodePtr PhysicalAggregation::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Aggregation & aggregation, + const PhysicalPlanNodePtr & 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::Planner::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 expr_after_agg_actions = PhysicalPlanHelper::newActions(aggregated_columns, context); + analyzer.reset(aggregated_columns); + analyzer.appendCastAfterAgg(expr_after_agg_actions, aggregation); + /// project action after aggregation to remove useless columns. + const NamesAndTypes & schema = analyzer.getCurrentInputColumns(); + expr_after_agg_actions->add(ExpressionAction::project(DB::toNames(schema))); + + auto physical_agg = std::make_shared( + executor_id, + schema, + log->identifier(), + child, + before_agg_actions, + aggregation_keys, + collators, + AggregationInterpreterHelper::isFinalAgg(aggregation), + aggregate_descriptions, + expr_after_agg_actions); + 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 || pipeline.streams_with_non_joined_data.size() > 1) + { + const Settings & settings = context.getSettingsRef(); + BlockInputStreamPtr stream = std::make_shared( + pipeline.streams, + pipeline.streams_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); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + + restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); + } + else + { + BlockInputStreams inputs; + if (!pipeline.streams.empty()) + inputs.push_back(pipeline.firstStream()); + + if (!pipeline.streams_with_non_joined_data.empty()) + inputs.push_back(pipeline.streams_with_non_joined_data.at(0)); + + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + + pipeline.firstStream() = std::make_shared( + std::make_shared(inputs, log->identifier()), + params, + context.getFileProvider(), + true, + log->identifier()); + } + + // we can record for agg after restore concurrency. + // Because the streams of expr_after_agg will provide the correct ProfileInfo. + // See #3804. + assert(expr_after_agg && !expr_after_agg->getActions().empty()); + executeExpression(pipeline, expr_after_agg, log, "expr after aggregation"); +} + +void PhysicalAggregation::finalize(const Names & parent_require) +{ + // schema.size() >= parent_require.size() + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); + expr_after_agg->finalize(DB::toNames(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 expr_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..5508e629504 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.h @@ -0,0 +1,68 @@ +// 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 PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Aggregation & aggregation, + const PhysicalPlanNodePtr & child); + + PhysicalAggregation( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & 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 & expr_after_agg_) + : PhysicalUnary(executor_id_, PlanType::Aggregation, schema_, req_id, child_) + , before_agg_actions(before_agg_actions_) + , aggregation_keys(aggregation_keys_) + , aggregation_collators(aggregation_collators_) + , is_final_agg(is_final_agg_) + , aggregate_descriptions(aggregate_descriptions_) + , expr_after_agg(expr_after_agg_) + {} + + 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 expr_after_agg; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalBinary.h b/dbms/src/Flash/Planner/plans/PhysicalBinary.h new file mode 100644 index 00000000000..1ae4810e9c2 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalBinary.h @@ -0,0 +1,65 @@ +// 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 +{ +/** + * A physical plan node with two children: left and right. + */ +class PhysicalBinary : public PhysicalPlanNode +{ +public: + PhysicalBinary( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & left_, + const PhysicalPlanNodePtr & right_) + : PhysicalPlanNode(executor_id_, type_, schema_, req_id) + , left(left_) + , right(right_) + { + RUNTIME_ASSERT(left, log, "children(0) shouldn't be nullptr"); + RUNTIME_ASSERT(right, log, "children(1) shouldn't be nullptr"); + } + + PhysicalPlanNodePtr children(size_t i) const override + { + RUNTIME_ASSERT(i <= 1, log, "child_index({}) shouldn't >= childrenSize({})", i, childrenSize()); + return i == 0 ? left : right; + } + + void setChild(size_t i, const PhysicalPlanNodePtr & new_child) override + { + RUNTIME_ASSERT(i <= 1, log, "child_index({}) shouldn't >= childrenSize({})", i, childrenSize()); + 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); + auto & child = i == 0 ? left : right; + child = new_child; + } + + size_t childrenSize() const override { return 2; }; + +protected: + PhysicalPlanNodePtr left; + PhysicalPlanNodePtr right; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp new file mode 100644 index 00000000000..c1a7fc2727f --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -0,0 +1,100 @@ +// 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 + +namespace DB +{ +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_) +{} + +PhysicalPlanNodePtr PhysicalExchangeReceiver::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log) +{ + auto mpp_exchange_receiver = context.getDAGContext()->getMPPExchangeReceiver(executor_id); + if (unlikely(mpp_exchange_receiver == nullptr)) + throw TiFlashException( + fmt::format("Can not find exchange receiver for {}", executor_id), + Errors::Planner::Internal); + + NamesAndTypes schema = toNamesAndTypes(mpp_exchange_receiver->getOutputSchema()); + auto physical_exchange_receiver = std::make_shared( + executor_id, + schema, + log->identifier(), + Block(schema), + 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()[executor_id]; + + const bool enable_fine_grained_shuffle = enableFineGrainedShuffle(mpp_exchange_receiver->getFineGrainedShuffleStreamCount()); + String extra_info = "squashing after exchange receiver"; + size_t stream_count = max_streams; + if (enable_fine_grained_shuffle) + { + extra_info += ", " + String(enableFineGrainedShuffleExtraInfo); + stream_count = std::min(max_streams, mpp_exchange_receiver->getFineGrainedShuffleStreamCount()); + } + + for (size_t i = 0; i < stream_count; ++i) + { + BlockInputStreamPtr stream = std::make_shared(mpp_exchange_receiver, + log->identifier(), + execId(), + /*stream_id=*/enable_fine_grained_shuffle ? i : 0); + exchange_receiver_io_input_streams.push_back(stream); + stream = std::make_shared(stream, 8192, 0, log->identifier()); + stream->setExtraInfo(extra_info); + 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..6dd2412d821 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.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 + +namespace DB +{ +class ExchangeReceiver; + +class PhysicalExchangeReceiver : public PhysicalLeaf +{ +public: + static PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log); + + PhysicalExchangeReceiver( + 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; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + Block sample_block; + + std::shared_ptr mpp_exchange_receiver; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp new file mode 100644 index 00000000000..9cd7b4111e5 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -0,0 +1,113 @@ +// 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 +{ +PhysicalPlanNodePtr PhysicalExchangeSender::build( + const String & executor_id, + const LoggerPtr & log, + const tipb::ExchangeSender & exchange_sender, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & 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(), + log->identifier(), + child, + partition_col_ids, + partition_col_collators, + exchange_sender.tp(), + fine_grained_shuffle); + // executeUnion will be call after sender.transform, so don't need to restore concurrency. + physical_exchange_sender->disableRestoreConcurrency(); + 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; + + if (fine_grained_shuffle.enable()) + { + 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, + fine_grained_shuffle.stream_count, + fine_grained_shuffle.batch_size); + stream = std::make_shared(stream, std::move(response_writer), log->identifier()); + stream->setExtraInfo(String(enableFineGrainedShuffleExtraInfo)); + }); + RUNTIME_CHECK(exchange_type == tipb::ExchangeType::Hash, Exception("exchange_sender has to be hash partition when fine grained shuffle is enabled")); + RUNTIME_CHECK(fine_grained_shuffle.stream_count <= 1024, Exception("fine_grained_shuffle_stream_count should not be greater than 1024")); + } + else + { + pipeline.transform([&](auto & stream) { + 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, + fine_grained_shuffle.stream_count, + fine_grained_shuffle.batch_size); + 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..536cd1e3164 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.h @@ -0,0 +1,63 @@ +// 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 PhysicalExchangeSender : public PhysicalUnary +{ +public: + static PhysicalPlanNodePtr build( + const String & executor_id, + const LoggerPtr & log, + const tipb::ExchangeSender & exchange_sender, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & child); + + PhysicalExchangeSender( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + const std::vector & partition_col_ids_, + const TiDB::TiDBCollators & collators_, + const tipb::ExchangeType & exchange_type_, + const FineGrainedShuffle & fine_grained_shuffle_) + : PhysicalUnary(executor_id_, PlanType::ExchangeSender, schema_, req_id, child_) + , partition_col_ids(partition_col_ids_) + , partition_col_collators(collators_) + , exchange_type(exchange_type_) + , fine_grained_shuffle(fine_grained_shuffle_) + {} + + 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; + + FineGrainedShuffle fine_grained_shuffle; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp b/dbms/src/Flash/Planner/plans/PhysicalFilter.cpp new file mode 100644 index 00000000000..7761e7d4a2a --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.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 +#include +#include + +namespace DB +{ +PhysicalPlanNodePtr PhysicalFilter::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Selection & selection, + const PhysicalPlanNodePtr & 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(), + log->identifier(), + child, + filter_column_name, + before_filter_actions); + + 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.emplace_back(filter_column); + before_filter_actions->finalize(required_output); + + child->finalize(before_filter_actions->getRequiredColumns()); + FinalizeHelper::prependProjectInputIfNeed(before_filter_actions, child->getSampleBlock().columns()); + + FinalizeHelper::checkSampleBlockContainsParentRequire(getSampleBlock(), parent_require); +} + +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..27d050b4a61 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalFilter.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 +#include + +namespace DB +{ +class PhysicalFilter : public PhysicalUnary +{ +public: + static PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Selection & selection, + const PhysicalPlanNodePtr & child); + + PhysicalFilter( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + const String & filter_column_, + const ExpressionActionsPtr & before_filter_actions_) + : PhysicalUnary(executor_id_, PlanType::Filter, schema_, req_id, child_) + , 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 diff --git a/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp b/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp new file mode 100644 index 00000000000..c0cb2904d1c --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalJoin.cpp @@ -0,0 +1,286 @@ +// 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. + +// 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 +{ +namespace FailPoints +{ +extern const char minimum_block_size_for_cross_join[]; +} // namespace FailPoints + +namespace +{ +void recordJoinExecuteInfo( + DAGContext & dag_context, + const String & executor_id, + const String & build_side_executor_id, + const JoinPtr & join_ptr) +{ + JoinExecuteInfo join_execute_info; + join_execute_info.build_side_root_executor_id = build_side_executor_id; + join_execute_info.join_ptr = join_ptr; + assert(join_execute_info.join_ptr); + dag_context.getJoinExecuteInfoMap()[executor_id] = std::move(join_execute_info); +} + +void executeUnionForPreviousNonJoinedData(DAGPipeline & probe_pipeline, Context & context, size_t max_streams, const LoggerPtr & log) +{ + // If there is non-joined-streams here, we need call `executeUnion` + // to ensure that non-joined-streams is executed after joined-streams. + if (!probe_pipeline.streams_with_non_joined_data.empty()) + { + executeUnion(probe_pipeline, max_streams, log, false, "final union for non_joined_data"); + restoreConcurrency(probe_pipeline, context.getDAGContext()->final_concurrency, log); + } +} +} // namespace + +PhysicalPlanNodePtr PhysicalJoin::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Join & join, + const PhysicalPlanNodePtr & left, + const PhysicalPlanNodePtr & right) +{ + assert(left); + assert(right); + + left->finalize(); + right->finalize(); + + const Block & left_input_header = left->getSampleBlock(); + const Block & right_input_header = right->getSampleBlock(); + + JoinInterpreterHelper::TiFlashJoin tiflash_join{join}; + + const auto & probe_plan = tiflash_join.build_side_index == 0 ? right : left; + const auto & build_plan = tiflash_join.build_side_index == 0 ? left : right; + + const Block & probe_side_header = probe_plan->getSampleBlock(); + const Block & build_side_header = build_plan->getSampleBlock(); + + String match_helper_name = tiflash_join.genMatchHelperName(left_input_header, right_input_header); + NamesAndTypesList columns_added_by_join = tiflash_join.genColumnsAddedByJoin(build_side_header, match_helper_name); + NamesAndTypes join_output_schema = tiflash_join.genJoinOutputColumns(left_input_header, right_input_header, match_helper_name); + + auto & dag_context = *context.getDAGContext(); + + /// add necessary transformation if the join key is an expression + + bool is_tiflash_right_join = tiflash_join.isTiFlashRightJoin(); + + // prepare probe side + auto [probe_side_prepare_actions, probe_key_names, probe_filter_column_name] = JoinInterpreterHelper::prepareJoin( + context, + probe_side_header, + tiflash_join.getProbeJoinKeys(), + tiflash_join.join_key_types, + /*left=*/true, + is_tiflash_right_join, + tiflash_join.getProbeConditions()); + RUNTIME_ASSERT(probe_side_prepare_actions, log, "probe_side_prepare_actions cannot be nullptr"); + + // prepare build side + auto [build_side_prepare_actions, build_key_names, build_filter_column_name] = JoinInterpreterHelper::prepareJoin( + context, + build_side_header, + tiflash_join.getBuildJoinKeys(), + tiflash_join.join_key_types, + /*left=*/false, + is_tiflash_right_join, + tiflash_join.getBuildConditions()); + RUNTIME_ASSERT(build_side_prepare_actions, log, "build_side_prepare_actions cannot be nullptr"); + + auto [other_condition_expr, other_filter_column_name, other_eq_filter_from_in_column_name] + = tiflash_join.genJoinOtherConditionAction(context, left_input_header, right_input_header, probe_side_prepare_actions); + + const Settings & settings = context.getSettingsRef(); + size_t max_block_size_for_cross_join = settings.max_block_size; + fiu_do_on(FailPoints::minimum_block_size_for_cross_join, { max_block_size_for_cross_join = 1; }); + + JoinPtr join_ptr = std::make_shared( + probe_key_names, + build_key_names, + true, + SizeLimits(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), + tiflash_join.kind, + tiflash_join.strictness, + log->identifier(), + tiflash_join.join_key_collators, + probe_filter_column_name, + build_filter_column_name, + other_filter_column_name, + other_eq_filter_from_in_column_name, + other_condition_expr, + max_block_size_for_cross_join, + match_helper_name); + + recordJoinExecuteInfo(dag_context, executor_id, build_plan->execId(), join_ptr); + + auto physical_join = std::make_shared( + executor_id, + join_output_schema, + log->identifier(), + probe_plan, + build_plan, + join_ptr, + columns_added_by_join, + probe_side_prepare_actions, + build_side_prepare_actions, + is_tiflash_right_join, + Block(join_output_schema)); + return physical_join; +} + +void PhysicalJoin::probeSideTransform(DAGPipeline & probe_pipeline, Context & context, size_t max_streams) +{ + const auto & settings = context.getSettingsRef(); + auto & dag_context = *context.getDAGContext(); + + // TODO we can call `executeUnionForPreviousNonJoinedData` only when has_non_joined == true. + executeUnionForPreviousNonJoinedData(probe_pipeline, context, max_streams, log); + + /// probe side streams + assert(probe_pipeline.streams_with_non_joined_data.empty()); + executeExpression(probe_pipeline, probe_side_prepare_actions, log, "append join key and join filters for probe side"); + auto join_probe_actions = PhysicalPlanHelper::newActions(probe_pipeline.firstStream()->getHeader(), context); + join_probe_actions->add(ExpressionAction::ordinaryJoin(join_ptr, columns_added_by_join)); + /// add join input stream + if (has_non_joined) + { + auto & join_execute_info = dag_context.getJoinExecuteInfoMap()[execId()]; + size_t not_joined_concurrency = join_ptr->getNotJoinedStreamConcurrency(); + const auto & input_header = probe_pipeline.firstStream()->getHeader(); + for (size_t i = 0; i < not_joined_concurrency; ++i) + { + auto non_joined_stream = join_ptr->createStreamWithNonJoinedRows(input_header, i, not_joined_concurrency, settings.max_block_size); + non_joined_stream->setExtraInfo("add stream with non_joined_data if full_or_right_join"); + probe_pipeline.streams_with_non_joined_data.push_back(non_joined_stream); + join_execute_info.non_joined_streams.push_back(non_joined_stream); + } + } + String join_probe_extra_info = fmt::format("join probe, join_executor_id = {}", execId()); + for (auto & stream : probe_pipeline.streams) + { + stream = std::make_shared(stream, join_probe_actions, log->identifier()); + stream->setExtraInfo(join_probe_extra_info); + } +} + +void PhysicalJoin::buildSideTransform(DAGPipeline & build_pipeline, Context & context, size_t max_streams) +{ + auto & dag_context = *context.getDAGContext(); + const auto & settings = context.getSettingsRef(); + + 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, 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); + String join_build_extra_info = fmt::format("join build, build_side_root_executor_id = {}", build()->execId()); + auto & join_execute_info = dag_context.getJoinExecuteInfoMap()[execId()]; + build_pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, join_ptr, get_concurrency_build_index(), log->identifier()); + stream->setExtraInfo(join_build_extra_info); + join_execute_info.join_build_streams.push_back(stream); + }); + // for test, join executor need the return blocks to output. + executeUnion(build_pipeline, max_streams, log, /*ignore_block=*/!context.isTest(), "for join"); + + SubqueryForSet build_query; + build_query.source = build_pipeline.firstStream(); + build_query.join = join_ptr; + join_ptr->init(build_query.source->getHeader(), join_build_concurrency); + dag_context.addSubquery(execId(), std::move(build_query)); +} + +void PhysicalJoin::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + /// The build side needs to be transformed first. + { + DAGPipeline build_pipeline; + build()->transform(build_pipeline, context, max_streams); + buildSideTransform(build_pipeline, context, max_streams); + } + + { + DAGPipeline & probe_pipeline = pipeline; + probe()->transform(probe_pipeline, context, max_streams); + probeSideTransform(probe_pipeline, context, max_streams); + } + + doSchemaProject(pipeline, context); +} + +void PhysicalJoin::doSchemaProject(DAGPipeline & pipeline, Context & context) +{ + /// add a project to remove all the useless column + NamesWithAliases schema_project_cols; + for (auto & c : schema) + { + /// do not need to care about duplicated column names because + /// it is guaranteed by its children physical plan nodes + schema_project_cols.emplace_back(c.name, c.name); + } + assert(!schema_project_cols.empty()); + ExpressionActionsPtr schema_project = generateProjectExpressionActions(pipeline.firstStream(), context, schema_project_cols); + assert(schema_project && !schema_project->getActions().empty()); + executeExpression(pipeline, schema_project, log, "remove useless column after join"); +} + +void PhysicalJoin::finalize(const Names & parent_require) +{ + // schema.size() >= parent_require.size() + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); +} + +const Block & PhysicalJoin::getSampleBlock() const +{ + return sample_block; +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalJoin.h b/dbms/src/Flash/Planner/plans/PhysicalJoin.h new file mode 100644 index 00000000000..ce4128e31e8 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalJoin.h @@ -0,0 +1,85 @@ +// 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 PhysicalJoin : public PhysicalBinary +{ +public: + static PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Join & join, + const PhysicalPlanNodePtr & left, + const PhysicalPlanNodePtr & right); + + PhysicalJoin( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & probe_, + const PhysicalPlanNodePtr & build_, + const JoinPtr & join_ptr_, + const NamesAndTypesList & columns_added_by_join_, + const ExpressionActionsPtr & probe_side_prepare_actions_, + const ExpressionActionsPtr & build_side_prepare_actions_, + bool has_non_joined_, + const Block & sample_block_) + : PhysicalBinary(executor_id_, PlanType::Join, schema_, req_id, probe_, build_) + , join_ptr(join_ptr_) + , columns_added_by_join(columns_added_by_join_) + , probe_side_prepare_actions(probe_side_prepare_actions_) + , build_side_prepare_actions(build_side_prepare_actions_) + , has_non_joined(has_non_joined_) + , sample_block(sample_block_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void probeSideTransform(DAGPipeline & probe_pipeline, Context & context, size_t max_streams); + + void buildSideTransform(DAGPipeline & build_pipeline, Context & context, size_t max_streams); + + void doSchemaProject(DAGPipeline & pipeline, Context & context); + + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + /// the right side is the build side. + const PhysicalPlanNodePtr & probe() const { return left; } + const PhysicalPlanNodePtr & build() const { return right; } + +private: + JoinPtr join_ptr; + + NamesAndTypesList columns_added_by_join; + + ExpressionActionsPtr probe_side_prepare_actions; + ExpressionActionsPtr build_side_prepare_actions; + + bool has_non_joined; + + Block sample_block; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalLeaf.h b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h new file mode 100644 index 00000000000..343ab66625c --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalLeaf.h @@ -0,0 +1,48 @@ +// 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 PhysicalPlanNode +{ +public: + PhysicalLeaf( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id) + : PhysicalPlanNode(executor_id_, type_, schema_, req_id) + {} + + PhysicalPlanNodePtr children(size_t) const override + { + throw TiFlashException("the children size of PhysicalLeaf is zero", Errors::Planner::Internal); + } + + void setChild(size_t, const PhysicalPlanNodePtr &) 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 new file mode 100644 index 00000000000..2722e7e1869 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.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 +#include + +namespace DB +{ +PhysicalPlanNodePtr PhysicalLimit::build( + const String & executor_id, + const LoggerPtr & log, + const tipb::Limit & limit, + const PhysicalPlanNodePtr & child) +{ + assert(child); + auto physical_limit = std::make_shared( + executor_id, + child->getSchema(), + log->identifier(), + child, + limit.limit()); + 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..24f6fe83044 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalLimit.h @@ -0,0 +1,50 @@ +// 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 PhysicalPlanNodePtr build( + const String & executor_id, + const LoggerPtr & log, + const tipb::Limit & limit, + const PhysicalPlanNodePtr & child); + + PhysicalLimit( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + size_t limit_) + : PhysicalUnary(executor_id_, PlanType::Limit, schema_, req_id, child_) + , 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; +}; +} // 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..1a990a6a2d4 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.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 + +namespace DB +{ +namespace +{ +std::pair mockSchemaAndStreams( + Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::ExchangeReceiver & exchange_receiver) +{ + NamesAndTypes schema; + BlockInputStreams mock_streams; + + auto & dag_context = *context.getDAGContext(); + size_t max_streams = dag_context.initialize_concurrency; + assert(max_streams > 0); + + if (context.columnsForTestEmpty() || context.columnsForTest(executor_id).empty()) + { + /// build with default blocks. + 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 + { + /// build from user input blocks. + 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()); + } + + assert(!schema.empty()); + assert(!mock_streams.empty()); + + return {std::move(schema), std::move(mock_streams)}; +} +} // namespace + +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::MockExchangeReceiver, schema_, req_id) + , sample_block(sample_block_) + , mock_streams(mock_streams_) +{} + +PhysicalPlanNodePtr PhysicalMockExchangeReceiver::build( + Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::ExchangeReceiver & exchange_receiver) +{ + assert(context.isExecutorTest()); + + auto [schema, mock_streams] = mockSchemaAndStreams(context, executor_id, log, exchange_receiver); + + auto physical_mock_exchange_receiver = std::make_shared( + executor_id, + schema, + log->identifier(), + Block(schema), + mock_streams); + return physical_mock_exchange_receiver; +} + +void PhysicalMockExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +{ + pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); +} + +void PhysicalMockExchangeReceiver::finalize(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); +} + +const Block & PhysicalMockExchangeReceiver::getSampleBlock() const +{ + return sample_block; +} +} // 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..df8f808ff81 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeReceiver.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 +{ +/** + * A physical plan node that generates MockExchangeReceiverInputStream. + * Used in gtest to test execution logic. + * Only available with `context.isExecutorTest() == true`. + */ +class PhysicalMockExchangeReceiver : public PhysicalLeaf +{ +public: + static PhysicalPlanNodePtr build( + Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::ExchangeReceiver & exchange_receiver); + + PhysicalMockExchangeReceiver( + 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; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) override; + +private: + Block sample_block; + + BlockInputStreams mock_streams; +}; +} // 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..5e6e83eeb0b --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.cpp @@ -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. + +#include +#include +#include +#include +#include + +namespace DB +{ +PhysicalPlanNodePtr PhysicalMockExchangeSender::build( + const String & executor_id, + const LoggerPtr & log, + const PhysicalPlanNodePtr & child) +{ + assert(child); + + auto physical_mock_exchange_sender = std::make_shared( + executor_id, + child->getSchema(), + log->identifier(), + child); + // executeUnion will be call after sender.transform, so don't need to restore concurrency. + physical_mock_exchange_sender->disableRestoreConcurrency(); + 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..bfebf34c1ea --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockExchangeSender.h @@ -0,0 +1,46 @@ +// 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 PhysicalPlanNodePtr build( + const String & executor_id, + const LoggerPtr & log, + const PhysicalPlanNodePtr & child); + + PhysicalMockExchangeSender( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_) + : PhysicalUnary(executor_id_, PlanType::MockExchangeSender, schema_, req_id, child_) + {} + + 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 diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.cpp new file mode 100644 index 00000000000..a47c5895a6a --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.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 + +namespace DB +{ +namespace +{ +std::pair mockSchemaAndStreams( + Context & context, + const String & executor_id, + const LoggerPtr & log, + const TiDBTableScan & table_scan) +{ + NamesAndTypes schema; + BlockInputStreams mock_streams; + + auto & dag_context = *context.getDAGContext(); + size_t max_streams = dag_context.initialize_concurrency; + assert(max_streams > 0); + + if (context.columnsForTestEmpty() || context.columnsForTest(executor_id).empty()) + { + /// build with default blocks. + schema = genNamesAndTypes(table_scan, "mock_table_scan"); + auto columns_with_type_and_name = getColumnWithTypeAndName(schema); + for (size_t i = 0; i < max_streams; ++i) + mock_streams.emplace_back(std::make_shared(columns_with_type_and_name, context.getSettingsRef().max_block_size)); + } + else + { + /// build from user input blocks. + auto [names_and_types, mock_table_scan_streams] = mockSourceStream(context, max_streams, log, executor_id); + schema = std::move(names_and_types); + mock_streams.insert(mock_streams.end(), mock_table_scan_streams.begin(), mock_table_scan_streams.end()); + } + + assert(!schema.empty()); + assert(!mock_streams.empty()); + + return {std::move(schema), std::move(mock_streams)}; +} +} // namespace + +PhysicalMockTableScan::PhysicalMockTableScan( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const Block & sample_block_, + const BlockInputStreams & mock_streams_) + : PhysicalLeaf(executor_id_, PlanType::MockTableScan, schema_, req_id) + , sample_block(sample_block_) + , mock_streams(mock_streams_) +{} + +PhysicalPlanNodePtr PhysicalMockTableScan::build( + Context & context, + const String & executor_id, + const LoggerPtr & log, + const TiDBTableScan & table_scan) +{ + assert(context.isTest()); + + auto [schema, mock_streams] = mockSchemaAndStreams(context, executor_id, log, table_scan); + + auto physical_mock_table_scan = std::make_shared( + executor_id, + schema, + log->identifier(), + Block(schema), + mock_streams); + return physical_mock_table_scan; +} + +void PhysicalMockTableScan::transformImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) +{ + pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); +} + +void PhysicalMockTableScan::finalize(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); +} + +const Block & PhysicalMockTableScan::getSampleBlock() const +{ + return sample_block; +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.h b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.h new file mode 100644 index 00000000000..5f935044694 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalMockTableScan.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 +#include + +namespace DB +{ +/** + * A physical plan node that generates MockTableScanBlockInputStream. + * Used in gtest to test execution logic. + * Only available with `context.isTest() == true`. + */ +class PhysicalMockTableScan : public PhysicalLeaf +{ +public: + static PhysicalPlanNodePtr build( + Context & context, + const String & executor_id, + const LoggerPtr & log, + const TiDBTableScan & table_scan); + + PhysicalMockTableScan( + 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; + + const Block & getSampleBlock() const override; + +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/PhysicalProjection.cpp b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp new file mode 100644 index 00000000000..34b9a7688e4 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.cpp @@ -0,0 +1,167 @@ +// 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 +{ +PhysicalPlanNodePtr PhysicalProjection::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Projection & projection, + const PhysicalPlanNodePtr & 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; + 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); + schema.emplace_back(alias, col.type); + } + /// 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, + log->identifier(), + child, + "projection", + project_actions); + return physical_projection; +} + +PhysicalPlanNodePtr PhysicalProjection::buildNonRootFinal( + const Context & context, + const LoggerPtr & log, + const String & column_prefix, + const PhysicalPlanNodePtr & 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 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( + child->execId(), + schema, + log->identifier(), + child, + "final projection", + project_actions); + // For final projection, no need to record profile streams. + physical_projection->disableRecordProfileStreams(); + return physical_projection; +} + +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 PhysicalPlanNodePtr & 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( + child->execId(), + schema, + log->identifier(), + child, + "final projection", + project_actions); + // For final projection, no need to record profile streams. + physical_projection->disableRecordProfileStreams(); + 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, extra_info); +} + +void PhysicalProjection::finalize(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); + project_actions->finalize(parent_require); + + child->finalize(project_actions->getRequiredColumns()); + FinalizeHelper::prependProjectInputIfNeed(project_actions, child->getSampleBlock().columns()); + + FinalizeHelper::checkSampleBlockContainsParentRequire(getSampleBlock(), parent_require); +} + +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..bd5e8140f28 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalProjection.h @@ -0,0 +1,76 @@ +// 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 PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Projection & projection, + 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 PhysicalPlanNodePtr buildNonRootFinal( + const Context & context, + const LoggerPtr & log, + const String & column_prefix, + 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 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 PhysicalPlanNodePtr & child); + + PhysicalProjection( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + const String & extra_info_, + const ExpressionActionsPtr & project_actions_) + : PhysicalUnary(executor_id_, PlanType::Projection, schema_, req_id, child_) + , extra_info(extra_info_) + , 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; + + const String extra_info; + + ExpressionActionsPtr project_actions; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp b/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp new file mode 100644 index 00000000000..cb702f60bd1 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalTableScan.cpp @@ -0,0 +1,119 @@ +// 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 +{ +PhysicalTableScan::PhysicalTableScan( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const TiDBTableScan & tidb_table_scan_, + const Block & sample_block_) + : PhysicalLeaf(executor_id_, PlanType::TableScan, schema_, req_id) + , tidb_table_scan(tidb_table_scan_) + , sample_block(sample_block_) +{} + +PhysicalPlanNodePtr PhysicalTableScan::build( + const String & executor_id, + const LoggerPtr & log, + const TiDBTableScan & table_scan) +{ + auto schema = genNamesAndTypes(table_scan, "table_scan"); + auto physical_table_scan = std::make_shared( + executor_id, + schema, + log->identifier(), + table_scan, + Block(schema)); + return physical_table_scan; +} + +void PhysicalTableScan::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + assert(pipeline.streams.empty() && pipeline.streams_with_non_joined_data.empty()); + + DAGStorageInterpreter storage_interpreter(context, tidb_table_scan, push_down_filter, max_streams); + storage_interpreter.execute(pipeline); + + const auto & storage_schema = storage_interpreter.analyzer->getCurrentInputColumns(); + RUNTIME_CHECK( + storage_schema.size() == schema.size(), + TiFlashException( + fmt::format( + "Expected col num does not match actual col num {}", + schema.size(), + storage_schema.size()), + Errors::Planner::Internal)); + NamesWithAliases schema_project_cols; + for (size_t i = 0; i < schema.size(); ++i) + { + RUNTIME_CHECK( + schema[i].type->equals(*storage_schema[i].type), + TiFlashException( + fmt::format( + "The type of schema col <{}, {}> does not match the type of actual col <{}, {}>", + schema[i].name, + schema[i].type->getName(), + storage_schema[i].name, + storage_schema[i].type->getName()), + Errors::Planner::Internal)); + assert(!storage_schema[i].name.empty() && !schema[i].name.empty()); + schema_project_cols.emplace_back(storage_schema[i].name, schema[i].name); + } + /// In order to keep BlockInputStream's schema consistent with PhysicalPlan's schema. + /// It is worth noting that the column uses the name as the unique identifier in the Block, so the column name must also be consistent. + ExpressionActionsPtr schema_project = generateProjectExpressionActions(pipeline.firstStream(), context, schema_project_cols); + executeExpression(pipeline, schema_project, log, "table scan schema projection"); +} + +void PhysicalTableScan::finalize(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); +} + +const Block & PhysicalTableScan::getSampleBlock() const +{ + return sample_block; +} + +bool PhysicalTableScan::pushDownFilter(const String & filter_executor_id, const tipb::Selection & selection) +{ + /// Since there is at most one selection on the table scan, pushDownFilter will only be called at most once. + /// So in this case hasPushDownFilter() is always false. + if (unlikely(hasPushDownFilter())) + { + return false; + } + + push_down_filter = PushDownFilter::pushDownFilterFrom(filter_executor_id, selection); + return true; +} + +bool PhysicalTableScan::hasPushDownFilter() const +{ + return push_down_filter.hasValue(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalTableScan.h b/dbms/src/Flash/Planner/plans/PhysicalTableScan.h new file mode 100644 index 00000000000..f064e69e811 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalTableScan.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. + +#include +#include +#include +#include + +namespace DB +{ +class PhysicalTableScan : public PhysicalLeaf +{ +public: + static PhysicalPlanNodePtr build( + const String & executor_id, + const LoggerPtr & log, + const TiDBTableScan & table_scan); + + PhysicalTableScan( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const TiDBTableScan & tidb_table_scan_, + const Block & sample_block_); + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + + bool pushDownFilter(const String & filter_executor_id, const tipb::Selection & selection); + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + + bool hasPushDownFilter() const; + +private: + PushDownFilter push_down_filter; + + TiDBTableScan tidb_table_scan; + + Block sample_block; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp new file mode 100644 index 00000000000..aaa2ef16cae --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -0,0 +1,86 @@ +// 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 +{ +PhysicalPlanNodePtr PhysicalTopN::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::TopN & top_n, + const PhysicalPlanNodePtr & child) +{ + assert(child); + + if (unlikely(top_n.order_by_size() == 0)) + { + //should not reach here + throw TiFlashException("TopN executor without order by exprs", Errors::Planner::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(), + log->identifier(), + child, + order_descr, + before_sort_actions, + top_n.limit()); + return physical_top_n; +} + +void PhysicalTopN::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + executeExpression(pipeline, before_sort_actions, log, "before TopN"); + + orderStreams(pipeline, max_streams, order_descr, limit, false, context, log); +} + +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.emplace_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::checkSampleBlockContainsParentRequire(getSampleBlock(), parent_require); +} + +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..bfabb5f4261 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.h @@ -0,0 +1,59 @@ +// 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 PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::TopN & top_n, + const PhysicalPlanNodePtr & child); + + PhysicalTopN( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + const SortDescription & order_descr_, + const ExpressionActionsPtr & before_sort_actions_, + size_t limit_) + : PhysicalUnary(executor_id_, PlanType::TopN, schema_, req_id, child_) + , 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 diff --git a/dbms/src/Flash/Planner/plans/PhysicalUnary.h b/dbms/src/Flash/Planner/plans/PhysicalUnary.h new file mode 100644 index 00000000000..e418653135f --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalUnary.h @@ -0,0 +1,61 @@ +// 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 +{ +/** + * A physical plan node with single child. + */ +class PhysicalUnary : public PhysicalPlanNode +{ +public: + PhysicalUnary( + const String & executor_id_, + const PlanType & type_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_) + : PhysicalPlanNode(executor_id_, type_, schema_, req_id) + , child(child_) + { + RUNTIME_ASSERT(child, log, "children(0) shouldn't be nullptr"); + } + + 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 PhysicalPlanNodePtr & new_child) override + { + RUNTIME_ASSERT(i == 0, log, "child_index({}) shouldn't >= childrenSize({})", i, childrenSize()); + 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; + } + + size_t childrenSize() const override { return 1; }; + +protected: + PhysicalPlanNodePtr child; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindow.cpp b/dbms/src/Flash/Planner/plans/PhysicalWindow.cpp new file mode 100644 index 00000000000..c3dfc48faaf --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalWindow.cpp @@ -0,0 +1,96 @@ +// 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 +{ +PhysicalPlanNodePtr PhysicalWindow::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Window & window, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & child) +{ + assert(child); + /// The plan tree will be `PhysicalWindow <-- ... <-- PhysicalWindow <-- ... <-- PhysicalSort`. + /// PhysicalWindow relies on the ordered data stream provided by PhysicalSort, + /// so the child plan cannot call `restoreConcurrency` that would destroy the ordering of the input data. + child->disableRestoreConcurrency(); + + DAGExpressionAnalyzer analyzer(child->getSchema(), context); + WindowDescription window_description = analyzer.buildWindowDescription(window); + + /// project action after window to remove useless columns. + const auto & schema = window_description.after_window_columns; + window_description.after_window->add(ExpressionAction::project(DB::toNames(schema))); + + auto physical_window = std::make_shared( + executor_id, + schema, + log->identifier(), + child, + window_description, + fine_grained_shuffle); + return physical_window; +} + +void PhysicalWindow::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + executeExpression(pipeline, window_description.before_window, log, "before window"); + + if (fine_grained_shuffle.enable()) + { + /// Window function can be multiple threaded when fine grained shuffle is enabled. + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, window_description, log->identifier()); + stream->setExtraInfo(String(enableFineGrainedShuffleExtraInfo)); + }); + } + else + { + /// If there are several streams, we merge them into one. + executeUnion(pipeline, max_streams, log, false, "merge into one for window input"); + assert(pipeline.streams.size() == 1); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), window_description, log->identifier()); + } + + executeExpression(pipeline, window_description.after_window, log, "cast after window"); +} + +void PhysicalWindow::finalize(const Names & parent_require) +{ + FinalizeHelper::checkSchemaContainsParentRequire(schema, parent_require); + + child->finalize(window_description.before_window->getRequiredColumns()); + FinalizeHelper::prependProjectInputIfNeed(window_description.before_window, child->getSampleBlock().columns()); +} + +const Block & PhysicalWindow::getSampleBlock() const +{ + return window_description.after_window->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindow.h b/dbms/src/Flash/Planner/plans/PhysicalWindow.h new file mode 100644 index 00000000000..6d37e11a597 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalWindow.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 PhysicalWindow : public PhysicalUnary +{ +public: + static PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Window & window, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & child); + + PhysicalWindow( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + const WindowDescription & window_description_, + const FineGrainedShuffle & fine_grained_shuffle_) + : PhysicalUnary(executor_id_, PlanType::Window, schema_, req_id, child_) + , window_description(window_description_) + , fine_grained_shuffle(fine_grained_shuffle_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + +private: + WindowDescription window_description; + FineGrainedShuffle fine_grained_shuffle; +}; +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindowSort.cpp b/dbms/src/Flash/Planner/plans/PhysicalWindowSort.cpp new file mode 100644 index 00000000000..2f58fb95e58 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalWindowSort.cpp @@ -0,0 +1,74 @@ +// 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 +{ +PhysicalPlanNodePtr PhysicalWindowSort::build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Sort & window_sort, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & child) +{ + assert(child); + + RUNTIME_ASSERT(window_sort.ispartialsort(), log, "for window sort, ispartialsort must be true"); + + DAGExpressionAnalyzer analyzer(child->getSchema(), context); + const auto & order_columns = analyzer.buildWindowOrderColumns(window_sort); + const SortDescription & order_descr = getSortDescription(order_columns, window_sort.byitems()); + + auto physical_window_sort = std::make_shared( + executor_id, + child->getSchema(), + log->identifier(), + child, + order_descr, + fine_grained_shuffle); + return physical_window_sort; +} + +void PhysicalWindowSort::transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) +{ + child->transform(pipeline, context, max_streams); + + orderStreams(pipeline, max_streams, order_descr, 0, fine_grained_shuffle.enable(), context, log); +} + +void PhysicalWindowSort::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.emplace_back(desc.column_name); + + child->finalize(required_output); +} + +const Block & PhysicalWindowSort::getSampleBlock() const +{ + return child->getSampleBlock(); +} +} // namespace DB diff --git a/dbms/src/Flash/Planner/plans/PhysicalWindowSort.h b/dbms/src/Flash/Planner/plans/PhysicalWindowSort.h new file mode 100644 index 00000000000..39ea96abbd2 --- /dev/null +++ b/dbms/src/Flash/Planner/plans/PhysicalWindowSort.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 PhysicalWindowSort : public PhysicalUnary +{ +public: + static PhysicalPlanNodePtr build( + const Context & context, + const String & executor_id, + const LoggerPtr & log, + const tipb::Sort & window_sort, + const FineGrainedShuffle & fine_grained_shuffle, + const PhysicalPlanNodePtr & child); + + PhysicalWindowSort( + const String & executor_id_, + const NamesAndTypes & schema_, + const String & req_id, + const PhysicalPlanNodePtr & child_, + const SortDescription & order_descr_, + const FineGrainedShuffle & fine_grained_shuffle_) + : PhysicalUnary(executor_id_, PlanType::WindowSort, schema_, req_id, child_) + , order_descr(order_descr_) + , fine_grained_shuffle(fine_grained_shuffle_) + {} + + void finalize(const Names & parent_require) override; + + const Block & getSampleBlock() const override; + +private: + void transformImpl(DAGPipeline & pipeline, Context & context, size_t max_streams) override; + +private: + SortDescription order_descr; + FineGrainedShuffle fine_grained_shuffle; +}; +} // namespace DB 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..9dc3452de32 --- /dev/null +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -0,0 +1,594 @@ +// 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 +{ +namespace tests +{ +class PhysicalPlanTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.context.setExecutorTest(); + + 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("exchange2", + {{"partition", TiDB::TP::TypeLongLong}, {"order", TiDB::TP::TypeLongLong}}, + {toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2})}); + + 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( + {"multi_test", "t1"}, + {{"a", TiDB::TP::TypeLong}, {"b", TiDB::TP::TypeLong}, {"c", TiDB::TP::TypeLong}}, + {toVec("a", {1, 3, 0}), + toVec("b", {2, 2, 0}), + toVec("c", {3, 2, 0})}); + context.addMockTable( + {"multi_test", "t2"}, + {{"a", TiDB::TP::TypeLong}, {"b", TiDB::TP::TypeLong}, {"c", TiDB::TP::TypeLong}}, + {toVec("a", {3, 3, 0}), + toVec("b", {4, 2, 0}), + toVec("c", {5, 3, 0})}); + context.addMockTable( + {"multi_test", "t3"}, + {{"a", TiDB::TP::TypeLong}, {"b", TiDB::TP::TypeLong}}, + {toVec("a", {1, 2, 0}), + toVec("b", {2, 2, 0})}); + context.addMockTable( + {"multi_test", "t4"}, + {{"a", TiDB::TP::TypeLong}, {"b", TiDB::TP::TypeLong}}, + {toVec("a", {3, 2, 0}), + toVec("b", {4, 2, 0})}); + } + + void execute( + 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; + + context.context.setColumnsForTest(context.executorIdColumnsMap()); + + DAGContext dag_context(*request, "executor_test", max_streams); + context.context.setDAGContext(&dag_context); + + PhysicalPlan physical_plan{context.context, log->identifier()}; + assert(request); + physical_plan.build(request.get()); + physical_plan.outputAndOptimize(); + + 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); + executeCreatingSets(pipeline, context.context, max_streams, log); + final_stream = pipeline.firstStream(); + FmtBuffer fb; + final_stream->dumpTree(fb); + ASSERT_EQ(Poco::trim(expected_streams), Poco::trim(fb.toString())); + } + + ASSERT_COLUMNS_EQ_R(expect_columns, readBlock(final_stream)); + } + + std::tuple multiTestScan() + { + return {context.scan("multi_test", "t1"), context.scan("multi_test", "t2"), context.scan("multi_test", "t3"), context.scan("multi_test", "t4")}; + } + + LoggerPtr log = Logger::get("PhysicalPlanTestRunner", "test_physical_plan"); +}; + +TEST_F(PhysicalPlanTestRunner, Filter) +try +{ + auto request = context.receive("exchange1") + .filter(eq(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: , )", + /*expected_streams=*/R"( +Expression: + Filter + MockExchangeReceiver)", + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); +} +CATCH + +TEST_F(PhysicalPlanTestRunner, Limit) +try +{ + auto request = context.receive("exchange1") + .limit(1) + .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: , )", + /*expected_streams=*/R"( +Expression: + Limit, limit = 1 + MockExchangeReceiver)", + {toNullableVec({"banana"}), + toNullableVec({"apple"})}); +} +CATCH + +TEST_F(PhysicalPlanTestRunner, TopN) +try +{ + auto request = context.receive("exchange1") + .topN("s2", false, 1) + .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: , )", + /*expected_streams=*/R"( +Expression: + MergeSorting, limit = 1 + PartialSorting: limit = 1 + MockExchangeReceiver)", + {toNullableVec({{}}), + toNullableVec({{}})}); +} +CATCH + +// 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); + + 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: + Aggregating + Concat + MockExchangeReceiver)", + {toNullableVec({{}, "banana"}), + toNullableVec({{}, "banana"})}); +} +CATCH + +TEST_F(PhysicalPlanTestRunner, Projection) +try +{ + auto request = context.receive("exchange1") + .project({concat(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: , )", + /*expected_streams=*/R"( +Expression: + Expression: + MockExchangeReceiver)", + {toNullableVec({"bananaapple", {}, "bananabanana"})}); +} +CATCH + +TEST_F(PhysicalPlanTestRunner, MockExchangeSender) +try +{ + auto request = context.receive("exchange1") + .exchangeSender(tipb::Hash) + .build(context); + + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +MockExchangeSender + Expression: + MockExchangeReceiver)", + {toNullableVec({"banana", {}, "banana"}), + toNullableVec({"apple", {}, "banana"})}); +} +CATCH + +TEST_F(PhysicalPlanTestRunner, MockExchangeReceiver) +try +{ + auto request = context.receive("exchange1") + .build(context); + + 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 + +TEST_F(PhysicalPlanTestRunner, WindowFunction) +try +{ + auto get_request = [&](bool enable_fine_grained_shuffle) { + static const uint64_t enable = 8; + static const uint64_t disable = 0; + bool fine_grained_shuffle_stream_count = enable_fine_grained_shuffle ? enable : disable; + return context + .receive("exchange2", fine_grained_shuffle_stream_count) + .sort({{"partition", false}, {"order", false}, {"partition", false}, {"order", false}}, true, fine_grained_shuffle_stream_count) + .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame(), fine_grained_shuffle_stream_count) + .build(context); + }; + + auto request = get_request(false); + 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: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting, limit = 0 + PartialSorting: limit = 0 + MockExchangeReceiver)", + {toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); + + request = get_request(true); + 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: + Window: , function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting: , limit = 0 + PartialSorting: : limit = 0 + MockExchangeReceiver)", + {toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); +} +CATCH + +TEST_F(PhysicalPlanTestRunner, MockTableScan) +try +{ + auto request = context.scan("test_db", "test_table") + .build(context); + + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +Expression: + MockTableScan)", + {toNullableVec({"banana", {}, "banana"}), + toNullableVec({"apple", {}, "banana"})}); +} +CATCH + +TEST_F(PhysicalPlanTestRunner, Join) +try +{ + // Simple Join + { + auto get_request = [&](const tipb::JoinType & join_type) { + return context + .receive("exchange_l_table") + .join(context.receive("exchange_r_table"), join_type, {col("join_c"), col("join_c")}) + .build(context); + }; + + auto request = get_request(tipb::JoinType::TypeInnerJoin); + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , , , + | is_record_profile_streams: true, schema: , , , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +CreatingSets + HashJoinBuildBlockInputStream: , join_kind = Inner + Expression: + Expression: + MockExchangeReceiver + Expression: + Expression: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver)", + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); + + request = get_request(tipb::JoinType::TypeLeftOuterJoin); + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , , , + | is_record_profile_streams: true, schema: , , , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +CreatingSets + HashJoinBuildBlockInputStream: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Expression: + Expression: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver)", + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); + + request = get_request(tipb::JoinType::TypeRightOuterJoin); + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , , , + | is_record_profile_streams: true, schema: , , , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +CreatingSets + HashJoinBuildBlockInputStream: , join_kind = Right + Expression: + Expression: + MockExchangeReceiver + Union: + Expression: + Expression: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver + Expression: + Expression: + NonJoined: )", + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); + } + + // MultiRightInnerJoin + { + auto [t1, t2, t3, t4] = multiTestScan(); + auto request = t1.join(t2, tipb::JoinType::TypeRightOuterJoin, {col("a")}) + .join(t3.join(t4, tipb::JoinType::TypeRightOuterJoin, {col("a")}), + tipb::JoinType::TypeInnerJoin, + {col("b")}) + .build(context); + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , , , , , , , , , + | is_record_profile_streams: true, schema: , , , , , , , , , + | is_record_profile_streams: false, schema: , , , , , + | is_record_profile_streams: true, schema: , , , , , + | is_record_profile_streams: false, schema: , , + | is_record_profile_streams: true, schema: , , + | is_record_profile_streams: false, schema: , , + | is_record_profile_streams: true, schema: , , + | is_record_profile_streams: false, schema: , , , + | is_record_profile_streams: true, schema: , , , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +CreatingSets + HashJoinBuildBlockInputStream x 2: , join_kind = Right + Expression: + Expression: + MockTableScan + Union: + HashJoinBuildBlockInputStream: , join_kind = Inner + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + Expression: + MockTableScan + HashJoinBuildBlockInputStream: , join_kind = Inner + Expression: + Expression: + Expression: + NonJoined: + Expression: + Expression: + HashJoinProbe: + Union: + Expression: + Expression: + HashJoinProbe: + Expression: + Expression: + MockTableScan + Expression: + Expression: + NonJoined: )", + {toNullableVec({3, 3, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({3, 3, 0}), + toNullableVec({4, 2, 0}), + toNullableVec({5, 3, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0})}); + } + + // MultiRightLeftJoin + { + auto [t1, t2, t3, t4] = multiTestScan(); + auto request = t1.join(t2, tipb::JoinType::TypeRightOuterJoin, {col("a")}) + .join(t3.join(t4, tipb::JoinType::TypeRightOuterJoin, {col("a")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("b")}) + .build(context); + execute( + request, + /*expected_physical_plan=*/R"( + | is_record_profile_streams: false, schema: , , , , , , , , , + | is_record_profile_streams: true, schema: , , , , , , , , , + | is_record_profile_streams: false, schema: , , , , , + | is_record_profile_streams: true, schema: , , , , , + | is_record_profile_streams: false, schema: , , + | is_record_profile_streams: true, schema: , , + | is_record_profile_streams: false, schema: , , + | is_record_profile_streams: true, schema: , , + | is_record_profile_streams: false, schema: , , , + | is_record_profile_streams: true, schema: , , , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , + | is_record_profile_streams: false, schema: , + | is_record_profile_streams: true, schema: , )", + /*expected_streams=*/R"( +CreatingSets + HashJoinBuildBlockInputStream x 2: , join_kind = Right + Expression: + Expression: + MockTableScan + Union: + HashJoinBuildBlockInputStream: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + Expression: + MockTableScan + HashJoinBuildBlockInputStream: , join_kind = Left + Expression: + Expression: + Expression: + NonJoined: + Expression: + Expression: + HashJoinProbe: + Union: + Expression: + Expression: + HashJoinProbe: + Expression: + Expression: + MockTableScan + Expression: + Expression: + NonJoined: )", + {toNullableVec({3, 3, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({3, 3, 0}), + toNullableVec({4, 2, 0}), + toNullableVec({5, 3, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0}), + toNullableVec({2, 2, 0})}); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/executeQuery.cpp b/dbms/src/Flash/executeQuery.cpp new file mode 100644 index 00000000000..903409c4118 --- /dev/null +++ b/dbms/src/Flash/executeQuery.cpp @@ -0,0 +1,38 @@ +// 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 + +namespace DB +{ +BlockIO executeQuery( + Context & context, + bool internal, + QueryProcessingStage::Enum stage) +{ + if (context.getSettingsRef().enable_planner) + { + PlanQuerySource plan(context); + return executeQuery(plan, context, internal, stage); + } + else + { + DAGQuerySource dag(context); + return executeQuery(dag, context, internal, stage); + } +} +} // namespace DB diff --git a/dbms/src/Flash/executeQuery.h b/dbms/src/Flash/executeQuery.h new file mode 100644 index 00000000000..a8b56f765d0 --- /dev/null +++ b/dbms/src/Flash/executeQuery.h @@ -0,0 +1,27 @@ +// 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 +{ +BlockIO executeQuery( + Context & context, + bool internal = false, + QueryProcessingStage::Enum stage = QueryProcessingStage::Complete); +} diff --git a/dbms/src/Flash/tests/gtest_aggregation_executor.cpp b/dbms/src/Flash/tests/gtest_aggregation_executor.cpp index 30923dd5475..e3deeab3129 100644 --- a/dbms/src/Flash/tests/gtest_aggregation_executor.cpp +++ b/dbms/src/Flash/tests/gtest_aggregation_executor.cpp @@ -116,8 +116,10 @@ class ExecutorAggTestRunner : public DB::tests::ExecutorTest void executeWithConcurrency(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) { + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN for (size_t i = 1; i <= max_concurrency; i += step) ASSERT_COLUMNS_EQ_UR(expect_columns, executeStreams(request, i)); + WRAP_FOR_DIS_ENABLE_PLANNER_END } static const size_t max_concurrency = 10; diff --git a/dbms/src/Flash/tests/gtest_executor.cpp b/dbms/src/Flash/tests/gtest_executor.cpp index 9d6982a9e48..b8fac961832 100644 --- a/dbms/src/Flash/tests/gtest_executor.cpp +++ b/dbms/src/Flash/tests/gtest_executor.cpp @@ -39,6 +39,7 @@ class ExecutorTestRunner : public DB::tests::ExecutorTest TEST_F(ExecutorTestRunner, Filter) try { + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN auto request = context .scan("test_db", "test_table") .filter(eq(col("s1"), col("s2"))) @@ -57,6 +58,7 @@ try createColumns({toNullableVec({"banana"}), toNullableVec({"banana"})})); } + WRAP_FOR_DIS_ENABLE_PLANNER_END } CATCH diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index dfeb050f993..a053c2743fb 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::ExecutorTest { ExecutorTest::initializeContext(); + enablePlanner(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_join_executor.cpp b/dbms/src/Flash/tests/gtest_join_executor.cpp index bb5be88f179..124e04e4123 100644 --- a/dbms/src/Flash/tests/gtest_join_executor.cpp +++ b/dbms/src/Flash/tests/gtest_join_executor.cpp @@ -62,10 +62,12 @@ class JoinExecutorTestRunner : public DB::tests::ExecutorTest void executeWithConcurrency(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) { + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN for (size_t i = 1; i <= max_concurrency_level; ++i) { ASSERT_COLUMNS_EQ_UR(expect_columns, executeStreams(request, i)); } + WRAP_FOR_DIS_ENABLE_PLANNER_END } static constexpr size_t join_type_num = 7; diff --git a/dbms/src/Flash/tests/gtest_limit_executor.cpp b/dbms/src/Flash/tests/gtest_limit_executor.cpp index 47482540b39..8f77b036bf0 100644 --- a/dbms/src/Flash/tests/gtest_limit_executor.cpp +++ b/dbms/src/Flash/tests/gtest_limit_executor.cpp @@ -68,7 +68,9 @@ try else expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.begin() + limit_num))}; + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols); + WRAP_FOR_DIS_ENABLE_PLANNER_END } } CATCH diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp new file mode 100644 index 00000000000..4085ef7a2e2 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -0,0 +1,988 @@ +// 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 PlannerInterpreterExecuteTest : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + + enablePlanner(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::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + } +}; + +TEST_F(PlannerInterpreterExecuteTest, StrangeQuery) +try +{ + auto request = context.scan("test_db", "test_table_1") + .filter(eq(col("s2"), col("s3"))) + .filter(eq(col("s1"), col("s3"))) + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Filter + Filter + Filter + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .limit(10) + .limit(9) + .limit(8) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Limit, limit = 8 + Union: + Limit x 10, limit = 8 + SharedQuery: + Limit, limit = 9 + Union: + Limit x 10, limit = 9 + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .topN("s3", false, 10) + .topN("s2", false, 9) + .topN("s1", false, 8) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + MergeSorting, limit = 8 + Union: + PartialSorting x 10: limit = 8 + SharedQuery: + MergeSorting, limit = 9 + Union: + PartialSorting x 10: limit = 9 + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, 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: + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Filter + Expression: + 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: + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Filter + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Filter + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, ParallelQuery) +try +{ + /// executor with table scan + auto request = context.scan("test_db", "test_table_1") + .limit(10) + .build(context); + { + String expected = R"( +Expression: + Limit, limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 5, limit = 10 + 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: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + 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: + Expression: + Aggregating + Concat + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 5, final: true + MockTableScan x 5)"; + 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: + Expression x 5: + SharedQuery: + 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: + Filter + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 1); + + expected = R"( +Union: + Expression x 5: + 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: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +Expression: + Expression: + Aggregating + Concat + Expression: + Limit, limit = 10 + 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: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +Expression: + Expression: + Aggregating + Concat + 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: + 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: + Expression: + Aggregating + Concat + Expression: + Expression: + Aggregating + Concat + 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: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + MockTableScan x 10)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +MockExchangeSender + Expression: + Expression: + Aggregating + Concat + 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 + Expression: + SharedQuery: + 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 + Expression: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + + expected = R"( +MockExchangeSender + Expression: + Limit, limit = 10 + 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), tipb::JoinType::TypeLeftOuterJoin, {col("join_c")}).build(context); + { + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + SharedQuery: + Limit, limit = 1 + Union: + Limit x 10, limit = 1 + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, 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: + 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: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + 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: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + 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: + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + Filter + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + 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: + 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: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, Window) +try +{ + auto request = context + .scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .sort({{"s1", true}, {"s2", false}}, true) + .project({"s1", "s2", "s3"}) + .window(RowNumber(), {"s1", true}, {"s1", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "s3", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Union: + Expression x 10: + SharedQuery: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, FineGrainedShuffle) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + auto request = context + .receive("sender_1", enable) + .sort({{"s1", true}, {"s2", false}}, true, enable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), enable) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Window: , function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting: , limit = 0 + PartialSorting: : limit = 0 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + auto topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + String topn_expected = R"( +Union: + Expression x 10: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); + + // fine-grained shuffle is disabled. + request = context + .receive("sender_1", disable) + .sort({{"s1", true}, {"s2", false}}, true, disable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), disable) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, Join) +try +{ + // TODO: Find a way to write the request easier. + { + // 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"); + + auto request = table1.join( + table2.join( + table3.join(table4, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .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); + } + + { + // 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"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .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 + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}), + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .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 + +TEST_F(PlannerInterpreterExecuteTest, JoinThenAgg) +try +{ + { + // Left Join. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + tipb::JoinType::TypeLeftOuterJoin, + {col("join_c")}) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right Join + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + tipb::JoinType::TypeRightOuterJoin, + {col("join_c")}) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Right + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + HashJoinProbe: + Expression: + Expression: + MockTableScan + Expression x 10: + Expression: + NonJoined: )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2, + tipb::JoinType::TypeRightOuterJoin, + {col("join_c")}) + .aggregation({Sum(col("r_a"))}, {col("join_c")}) + .limit(10) + .exchangeSender(tipb::PassThrough) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 20: , join_kind = Right + Expression: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 20 + Expression: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 20, limit = 10 + Expression: + SharedQuery: + ParallelAggregating, max_threads: 20, final: true + Expression x 20: + Expression: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver + Expression x 20: + Expression: + NonJoined: )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 20); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp index 8443dedeb49..3cb458d1045 100644 --- a/dbms/src/Flash/tests/gtest_projection_executor.cpp +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -51,10 +51,12 @@ class ExecutorProjectionTestRunner : public DB::tests::ExecutorTest void executeWithConcurrency(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) { + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN for (size_t i = 1; i < 10; i += 2) { ASSERT_COLUMNS_EQ_UR(executeStreams(request, i), expect_columns); } + WRAP_FOR_DIS_ENABLE_PLANNER_END } /// Prepare column data diff --git a/dbms/src/Flash/tests/gtest_split_tasks.cpp b/dbms/src/Flash/tests/gtest_split_tasks.cpp index a2f8da097bb..151e22baa5b 100644 --- a/dbms/src/Flash/tests/gtest_split_tasks.cpp +++ b/dbms/src/Flash/tests/gtest_split_tasks.cpp @@ -25,6 +25,9 @@ class SplitTaskTest : public DB::tests::ExecutorTest void initializeContext() override { ExecutorTest::initializeContext(); + + enablePlanner(false); + context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "l_table"}, {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "r_table"}, {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); diff --git a/dbms/src/Flash/tests/gtest_topn_executor.cpp b/dbms/src/Flash/tests/gtest_topn_executor.cpp index d79573524ad..e18f7b8869d 100644 --- a/dbms/src/Flash/tests/gtest_topn_executor.cpp +++ b/dbms/src/Flash/tests/gtest_topn_executor.cpp @@ -78,6 +78,7 @@ class ExecutorTopNTestRunner : public DB::tests::ExecutorTest TEST_F(ExecutorTopNTestRunner, TopN) try { + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN std::shared_ptr request; std::vector expect_cols; @@ -142,6 +143,7 @@ try ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[i]); } } + WRAP_FOR_DIS_ENABLE_PLANNER_END } CATCH @@ -159,6 +161,7 @@ try ASTPtr col3_ast = col(col_name[3]); ASTPtr func_ast; + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN { /// "and" function expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{{}, {}, 32, 27, 36, 34}), @@ -214,6 +217,7 @@ try } /// TODO more functions... + WRAP_FOR_DIS_ENABLE_PLANNER_END } CATCH diff --git a/dbms/src/Flash/tests/gtest_window_executor.cpp b/dbms/src/Flash/tests/gtest_window_executor.cpp index d09122e5815..d7f796ffee9 100644 --- a/dbms/src/Flash/tests/gtest_window_executor.cpp +++ b/dbms/src/Flash/tests/gtest_window_executor.cpp @@ -62,20 +62,22 @@ class WindowExecutorTestRunner : public DB::tests::ExecutorTest void executeWithConcurrency(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) { - ASSERT_COLUMNS_EQ_R(expect_columns, executeStreams(request)); + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN for (size_t i = 1; i <= max_concurrency_level; ++i) { ASSERT_COLUMNS_EQ_UR(expect_columns, executeStreams(request, i)); } + WRAP_FOR_DIS_ENABLE_PLANNER_END } void executeWithTableScanAndConcurrency(const std::shared_ptr & request, const ColumnsWithTypeAndName & source_columns, const ColumnsWithTypeAndName & expect_columns) { - ASSERT_COLUMNS_EQ_R(expect_columns, executeStreamsWithSingleSource(request, source_columns, SourceType::TableScan)); + WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN for (size_t i = 1; i <= max_concurrency_level; ++i) { ASSERT_COLUMNS_EQ_UR(expect_columns, executeStreamsWithSingleSource(request, source_columns, SourceType::TableScan)); } + WRAP_FOR_DIS_ENABLE_PLANNER_END } }; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index dc2cb70e63b..22b48833dd6 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -367,8 +367,9 @@ 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(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") // clang-format on #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) TYPE NAME{DEFAULT}; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 78ad4b41ce6..77a3a76d842 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -427,7 +428,7 @@ BlockIO executeQuery( } -BlockIO executeQuery(DAGQuerySource & dag, Context & context, bool internal, QueryProcessingStage::Enum stage) +BlockIO executeQuery(IQuerySource & dag, Context & context, bool internal, QueryProcessingStage::Enum stage) { BlockIO streams; std::tie(std::ignore, streams) = executeQueryImpl(dag, context, internal, stage); diff --git a/dbms/src/Interpreters/executeQuery.h b/dbms/src/Interpreters/executeQuery.h index ea5d70a7a91..22950368db8 100644 --- a/dbms/src/Interpreters/executeQuery.h +++ b/dbms/src/Interpreters/executeQuery.h @@ -16,7 +16,7 @@ #include #include -#include +#include namespace DB @@ -53,6 +53,6 @@ BlockIO executeQuery( ); -BlockIO executeQuery(DAGQuerySource & dag, Context & context, bool internal, QueryProcessingStage::Enum stage); +BlockIO executeQuery(IQuerySource & dag, Context & context, bool internal, QueryProcessingStage::Enum stage); } // namespace DB diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index 6d07c47f235..cd1dc583a7c 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -1170,4 +1170,17 @@ ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type) return ret; } +ColumnInfo toTiDBColumnInfo(const tipb::ColumnInfo & tipb_column_info) +{ + ColumnInfo tidb_column_info; + tidb_column_info.tp = static_cast(tipb_column_info.tp()); + tidb_column_info.id = tipb_column_info.column_id(); + tidb_column_info.flag = tipb_column_info.flag(); + tidb_column_info.flen = tipb_column_info.columnlen(); + tidb_column_info.decimal = tipb_column_info.decimal(); + for (int i = 0; i < tipb_column_info.elems_size(); ++i) + tidb_column_info.elems.emplace_back(tipb_column_info.elems(i), i + 1); + return tidb_column_info; +} + } // namespace TiDB diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index a9d46b60c13..0f136786fd5 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -20,6 +20,7 @@ #include #include #include +#include #include @@ -420,6 +421,7 @@ String genJsonNull(); tipb::FieldType columnInfoToFieldType(const ColumnInfo & ci); ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type); +ColumnInfo toTiDBColumnInfo(const tipb::ColumnInfo & tipb_column_info); String TiFlashModeToString(TiFlashMode tiflash_mode); TiFlashMode parseTiFlashMode(String mode_str); diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 0f1b8f0128b..682b783e2af 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -14,8 +14,7 @@ #include #include -#include -#include +#include #include #include @@ -69,8 +68,7 @@ void ExecutorTest::executeInterpreter(const String & expected_string, const std: context.context.setDAGContext(&dag_context); context.context.setExecutorTest(); // Currently, don't care about regions information in interpreter tests. - DAGQuerySource dag(context.context); - auto res = executeQuery(dag, context.context, false, QueryProcessingStage::Complete); + auto res = executeQuery(context.context); FmtBuffer fb; res.in->dumpTree(fb); ASSERT_EQ(Poco::trim(expected_string), Poco::trim(fb.toString())); @@ -105,6 +103,7 @@ 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 DB::ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream) { @@ -117,7 +116,11 @@ DB::ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream) stream->readSuffix(); return mergeBlocks(actual_blocks).getColumnsWithTypeAndName(); } -} // namespace + +void ExecutorTest::enablePlanner(bool is_enable) +{ + context.context.setSetting("enable_planner", is_enable ? "true" : "false"); +} DB::ColumnsWithTypeAndName ExecutorTest::executeStreams(const std::shared_ptr & request, std::unordered_map & source_columns_map, size_t concurrency) { @@ -126,8 +129,7 @@ DB::ColumnsWithTypeAndName ExecutorTest::executeStreams(const std::shared_ptr & request, size_t concurrency) diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index ee1743fcd78..48f2cc68513 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -29,6 +29,16 @@ void executeInterpreter(const std::shared_ptr & request, Conte ::testing::AssertionResult check_columns_equality(const ColumnsWithTypeAndName & expected, const ColumnsWithTypeAndName & actual, bool _restrict); +DB::ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream); + +#define WRAP_FOR_DIS_ENABLE_PLANNER_BEGIN \ + std::vector bools{false, true}; \ + for (auto flag : bools) \ + { \ + enablePlanner(flag); + +#define WRAP_FOR_DIS_ENABLE_PLANNER_END } + class ExecutorTest : public ::testing::Test { protected: @@ -50,6 +60,8 @@ class ExecutorTest : public ::testing::Test DAGContext & getDAGContext(); + void enablePlanner(bool is_enable); + static void dagRequestEqual(const String & expected_string, const std::shared_ptr & actual); void executeInterpreter(const String & expected_string, const std::shared_ptr & request, size_t concurrency); diff --git a/tests/docker/config/tiflash_dt_disable_planner.toml b/tests/docker/config/tiflash_dt_disable_planner.toml new file mode 100644 index 00000000000..7f56b2c105f --- /dev/null +++ b/tests/docker/config/tiflash_dt_disable_planner.toml @@ -0,0 +1,54 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +tmp_path = "/tmp/tiflash/data/tmp" + +path = "/tmp/tiflash/data/db" +capacity = "10737418240" + +mark_cache_size = 5368709120 +minmax_index_cache_size = 5368709120 +tcp_port = 9000 +http_port = 8123 + +[flash] +tidb_status_addr = "tidb0:10080" +service_addr = "0.0.0.0:3930" +[flash.flash_cluster] +update_rule_interval = 5 +[flash.proxy] +addr = "0.0.0.0:20170" +advertise-addr = "tiflash0:20170" +data-dir = "/data" +config = "/proxy.toml" +log-file = "/log/proxy.log" +engine-addr = "tiflash0:3930" +status-addr = "0.0.0.0:20181" +advertise-status-addr = "tiflash0:20181" + +[logger] +count = 10 +errorlog = "/tmp/tiflash/log/error.log" +size = "1000M" +log = "/tmp/tiflash/log/server.log" +level = "trace" + +[raft] +pd_addr = "pd0:2379" +ignore_databases = "system,default" + +[profiles] +[profiles.default] +enable_planner = 0 +# max_memory_usage = 0 diff --git a/tests/docker/tiflash-dt-disable-planner.yaml b/tests/docker/tiflash-dt-disable-planner.yaml new file mode 100644 index 00000000000..7586bb3edc1 --- /dev/null +++ b/tests/docker/tiflash-dt-disable-planner.yaml @@ -0,0 +1,43 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +version: '2.3' + +services: + # for tests under fullstack-test directory + # (engine DeltaTree) + tiflash0: + image: hub.pingcap.net/tiflash/tiflash-ci-base + volumes: + - ./config/tiflash_dt_disable_planner.toml:/config.toml:ro + - ./data/tiflash:/tmp/tiflash/data + - ./log/tiflash:/tmp/tiflash/log + - ..:/tests + - ../docker/_env.sh:/tests/_env.sh + - ./log/tiflash-cluster-manager:/tmp/tiflash/data/tmp + - ./config/proxy.toml:/proxy.toml:ro + - ./config/cipher-file-256:/cipher-file-256:ro + - ./data/proxy:/data + - ./log/proxy:/log + - ../.build/tiflash:/tiflash + entrypoint: + - /tiflash/tiflash + - server + - --config-file + - /config.toml + restart: on-failure + depends_on: + - "pd0" + - "tikv0" + diff --git a/tests/fullstack-test2/dml/test_table_scan.test b/tests/fullstack-test2/dml/test_table_scan.test new file mode 100644 index 00000000000..ba234cebd6f --- /dev/null +++ b/tests/fullstack-test2/dml/test_table_scan.test @@ -0,0 +1,728 @@ +# 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. + +# TINYINT +## clustered_index +mysql> drop table if exists test.t_TINYINT_CI +mysql> create table test.t_TINYINT_CI(pk TINYINT NOT NULL, col1 TINYINT, col2 TINYINT NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_TINYINT_CI values(0, NULL, 0); +mysql> alter table test.t_TINYINT_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_TINYINT_NCI +mysql> create table test.t_TINYINT_NCI(pk TINYINT NOT NULL, col1 TINYINT, col2 TINYINT NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_TINYINT_NCI values(0, NULL, 0); +mysql> alter table test.t_TINYINT_NCI set tiflash replica 1 + +# SMALLINT +## clustered_index +mysql> drop table if exists test.t_SMALLINT_CI +mysql> create table test.t_SMALLINT_CI(pk SMALLINT NOT NULL, col1 SMALLINT, col2 SMALLINT NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_SMALLINT_CI values(0, NULL, 0); +mysql> alter table test.t_SMALLINT_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_SMALLINT_NCI +mysql> create table test.t_SMALLINT_NCI(pk SMALLINT NOT NULL, col1 SMALLINT, col2 SMALLINT NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_SMALLINT_NCI values(0, NULL, 0); +mysql> alter table test.t_SMALLINT_NCI set tiflash replica 1 + +# MEDIUMINT +## clustered_index +mysql> drop table if exists test.t_MEDIUMINT_CI +mysql> create table test.t_MEDIUMINT_CI(pk MEDIUMINT NOT NULL, col1 MEDIUMINT, col2 MEDIUMINT NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_MEDIUMINT_CI values(0, NULL, 0); +mysql> alter table test.t_MEDIUMINT_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_MEDIUMINT_NCI +mysql> create table test.t_MEDIUMINT_NCI(pk MEDIUMINT NOT NULL, col1 MEDIUMINT, col2 MEDIUMINT NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_MEDIUMINT_NCI values(0, NULL, 0); +mysql> alter table test.t_MEDIUMINT_NCI set tiflash replica 1 + +# INT +## clustered_index +mysql> drop table if exists test.t_INT_CI +mysql> create table test.t_INT_CI(pk INT NOT NULL, col1 INT, col2 INT NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_INT_CI values(0, NULL, 0); +mysql> alter table test.t_INT_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_INT_NCI +mysql> create table test.t_INT_NCI(pk INT NOT NULL, col1 INT, col2 INT NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_INT_NCI values(0, NULL, 0); +mysql> alter table test.t_INT_NCI set tiflash replica 1 + +# BIGINT +## clustered_index +mysql> drop table if exists test.t_BIGINT_CI +mysql> create table test.t_BIGINT_CI(pk BIGINT NOT NULL, col1 BIGINT, col2 BIGINT NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_BIGINT_CI values(0, NULL, 0); +mysql> alter table test.t_BIGINT_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_BIGINT_NCI +mysql> create table test.t_BIGINT_NCI(pk BIGINT NOT NULL, col1 BIGINT, col2 BIGINT NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_BIGINT_NCI values(0, NULL, 0); +mysql> alter table test.t_BIGINT_NCI set tiflash replica 1 + +# FLOAT +## clustered_index +mysql> drop table if exists test.t_FLOAT_CI +mysql> create table test.t_FLOAT_CI(pk FLOAT NOT NULL, col1 FLOAT, col2 FLOAT NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_FLOAT_CI values(0.0, NULL, 0.0); +mysql> alter table test.t_FLOAT_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_FLOAT_NCI +mysql> create table test.t_FLOAT_NCI(pk FLOAT NOT NULL, col1 FLOAT, col2 FLOAT NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_FLOAT_NCI values(0.0, NULL, 0.0); +mysql> alter table test.t_FLOAT_NCI set tiflash replica 1 + +# DOUBLE +## clustered_index +mysql> drop table if exists test.t_DOUBLE_CI +mysql> create table test.t_DOUBLE_CI(pk DOUBLE NOT NULL, col1 DOUBLE, col2 DOUBLE NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_DOUBLE_CI values(0.0, NULL, 0.0); +mysql> alter table test.t_DOUBLE_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_DOUBLE_NCI +mysql> create table test.t_DOUBLE_NCI(pk DOUBLE NOT NULL, col1 DOUBLE, col2 DOUBLE NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_DOUBLE_NCI values(0.0, NULL, 0.0); +mysql> alter table test.t_DOUBLE_NCI set tiflash replica 1 + +# DECIMAL32 +## clustered_index +mysql> drop table if exists test.t_DECIMAL32_CI +mysql> create table test.t_DECIMAL32_CI(pk decimal(9, 1) NOT NULL, col1 decimal(9, 1), col2 decimal(9, 1) NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_DECIMAL32_CI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL32_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_DECIMAL32_NCI +mysql> create table test.t_DECIMAL32_NCI(pk decimal(9, 1) NOT NULL, col1 decimal(9, 1), col2 decimal(9, 1) NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_DECIMAL32_NCI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL32_NCI set tiflash replica 1 + +# DECIMAL64 +## clustered_index +mysql> drop table if exists test.t_DECIMAL64_CI +mysql> create table test.t_DECIMAL64_CI(pk decimal(18, 1) NOT NULL, col1 decimal(18, 1), col2 decimal(18, 1) NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_DECIMAL64_CI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL64_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_DECIMAL64_NCI +mysql> create table test.t_DECIMAL64_NCI(pk decimal(18, 1) NOT NULL, col1 decimal(18, 1), col2 decimal(18, 1) NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_DECIMAL64_NCI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL64_NCI set tiflash replica 1 + +# DECIMAL128 +## clustered_index +mysql> drop table if exists test.t_DECIMAL128_CI +mysql> create table test.t_DECIMAL128_CI(pk decimal(38, 1) NOT NULL, col1 decimal(38, 1), col2 decimal(38, 1) NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_DECIMAL128_CI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL128_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_DECIMAL128_NCI +mysql> create table test.t_DECIMAL128_NCI(pk decimal(38, 1) NOT NULL, col1 decimal(38, 1), col2 decimal(38, 1) NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_DECIMAL128_NCI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL128_NCI set tiflash replica 1 + +# DECIMAL256 +## clustered_index +mysql> drop table if exists test.t_DECIMAL256_CI +mysql> create table test.t_DECIMAL256_CI(pk decimal(65, 1) NOT NULL, col1 decimal(65, 1), col2 decimal(65, 1) NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_DECIMAL256_CI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL256_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_DECIMAL256_NCI +mysql> create table test.t_DECIMAL256_NCI(pk decimal(65, 1) NOT NULL, col1 decimal(65, 1), col2 decimal(65, 1) NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_DECIMAL256_NCI values(0.1, NULL, 0.1); +mysql> alter table test.t_DECIMAL256_NCI set tiflash replica 1 + +# CHAR +## clustered_index +mysql> drop table if exists test.t_CHAR_CI +mysql> create table test.t_CHAR_CI(pk CHAR NOT NULL, col1 CHAR, col2 CHAR NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_CHAR_CI values('a', NULL, 'b'); +mysql> alter table test.t_CHAR_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_CHAR_NCI +mysql> create table test.t_CHAR_NCI(pk CHAR NOT NULL, col1 CHAR, col2 CHAR NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_CHAR_NCI values('a', NULL, 'b'); +mysql> alter table test.t_CHAR_NCI set tiflash replica 1 + +# VARCHAR +## clustered_index +mysql> drop table if exists test.t_VARCHAR_CI +mysql> create table test.t_VARCHAR_CI(pk VARCHAR(30) NOT NULL, col1 VARCHAR(30), col2 VARCHAR(30) NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_VARCHAR_CI values('a', NULL, 'b'); +mysql> alter table test.t_VARCHAR_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_VARCHAR_NCI +mysql> create table test.t_VARCHAR_NCI(pk VARCHAR(30) NOT NULL, col1 VARCHAR(30), col2 VARCHAR(30) NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_VARCHAR_NCI values('a', NULL, 'b'); +mysql> alter table test.t_VARCHAR_NCI set tiflash replica 1 + +# BINARY +## clustered_index +mysql> drop table if exists test.t_BINARY_CI +mysql> create table test.t_BINARY_CI(pk BINARY NOT NULL, col1 BINARY, col2 BINARY NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_BINARY_CI values('1', NULL, '2'); +mysql> alter table test.t_BINARY_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_BINARY_NCI +mysql> create table test.t_BINARY_NCI(pk BINARY NOT NULL, col1 BINARY, col2 BINARY NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_BINARY_NCI values('1', NULL, '2'); +mysql> alter table test.t_BINARY_NCI set tiflash replica 1 + +# VARBINARY +## clustered_index +mysql> drop table if exists test.t_VARBINARY_CI +mysql> create table test.t_VARBINARY_CI(pk VARBINARY(30) NOT NULL, col1 VARBINARY(30), col2 VARBINARY(30) NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_VARBINARY_CI values('1234', NULL, '5678'); +mysql> alter table test.t_VARBINARY_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_VARBINARY_NCI +mysql> create table test.t_VARBINARY_NCI(pk VARBINARY(30) NOT NULL, col1 VARBINARY(30), col2 VARBINARY(30) NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_VARBINARY_NCI values('1234', NULL, '5678'); +mysql> alter table test.t_VARBINARY_NCI set tiflash replica 1 + +# BLOB +mysql> drop table if exists test.t_BLOB +mysql> create table test.t_BLOB(col1 BLOB, col2 BLOB NOT NULL); +mysql> insert into test.t_BLOB values(NULL, '1234'); +mysql> alter table test.t_BLOB set tiflash replica 1 + +# TEXT +mysql> drop table if exists test.t_TEXT +mysql> create table test.t_TEXT(col1 TEXT, col2 TEXT NOT NULL); +mysql> insert into test.t_TEXT values(NULL, 'b'); +mysql> alter table test.t_TEXT set tiflash replica 1 + +# ENUM +## clustered_index +mysql> drop table if exists test.t_ENUM_CI +mysql> create table test.t_ENUM_CI(pk enum('a','b') NOT NULL, col1 enum('a','b'), col2 enum('a','b') NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_ENUM_CI values('a', NULL, 'b'); +mysql> alter table test.t_ENUM_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_ENUM_NCI +mysql> create table test.t_ENUM_NCI(pk enum('a','b') NOT NULL, col1 enum('a','b'), col2 enum('a','b') NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_ENUM_NCI values('a', NULL, 'b'); +mysql> alter table test.t_ENUM_NCI set tiflash replica 1 + +# SET +## clustered_index +mysql> drop table if exists test.t_SET_CI +mysql> create table test.t_SET_CI(pk set('a','b') NOT NULL, col1 set('a','b'), col2 set('a','b') NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_SET_CI values('a', NULL, 'b'); +mysql> alter table test.t_SET_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_SET_NCI +mysql> create table test.t_SET_NCI(pk set('a','b') NOT NULL, col1 set('a','b'), col2 set('a','b') NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_SET_NCI values('a', NULL, 'b'); +mysql> alter table test.t_SET_NCI set tiflash replica 1 + +# DATE +## clustered_index +mysql> drop table if exists test.t_DATE_CI +mysql> create table test.t_DATE_CI(pk DATE NOT NULL, col1 DATE, col2 DATE NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_DATE_CI values('2010-01-01', NULL, '2010-01-02'); +mysql> alter table test.t_DATE_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_DATE_NCI +mysql> create table test.t_DATE_NCI(pk DATE NOT NULL, col1 DATE, col2 DATE NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_DATE_NCI values('2010-01-01', NULL, '2010-01-02'); +mysql> alter table test.t_DATE_NCI set tiflash replica 1 + +# TIME +## clustered_index +mysql> drop table if exists test.t_TIME_CI +mysql> create table test.t_TIME_CI(pk TIME NOT NULL, col1 TIME, col2 TIME NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_TIME_CI values('10:00:00', NULL, '11:00:00'); +mysql> alter table test.t_TIME_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_TIME_NCI +mysql> create table test.t_TIME_NCI(pk TIME NOT NULL, col1 TIME, col2 TIME NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_TIME_NCI values('10:00:00', NULL, '11:00:00'); +mysql> alter table test.t_TIME_NCI set tiflash replica 1 + +# DATETIME +## clustered_index +mysql> drop table if exists test.t_DATETIME_CI +mysql> create table test.t_DATETIME_CI(pk DATETIME NOT NULL, col1 DATETIME, col2 DATETIME NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_DATETIME_CI values('2010-01-01 11:00:00', NULL, '2011-01-01 12:00:00'); +mysql> alter table test.t_DATETIME_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_DATETIME_NCI +mysql> create table test.t_DATETIME_NCI(pk DATETIME NOT NULL, col1 DATETIME, col2 DATETIME NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_DATETIME_NCI values('2010-01-01 11:00:00', NULL, '2011-01-01 12:00:00'); +mysql> alter table test.t_DATETIME_NCI set tiflash replica 1 + +# TIMESTAMP +## clustered_index +mysql> drop table if exists test.t_TIMESTAMP_CI +mysql> create table test.t_TIMESTAMP_CI(pk TIMESTAMP NOT NULL, col1 TIMESTAMP, col2 TIMESTAMP NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_TIMESTAMP_CI values('2010-01-01 11:00:00', NULL, '2011-01-01 12:00:00'); +mysql> alter table test.t_TIMESTAMP_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_TIMESTAMP_NCI +mysql> create table test.t_TIMESTAMP_NCI(pk TIMESTAMP NOT NULL, col1 TIMESTAMP, col2 TIMESTAMP NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_TIMESTAMP_NCI values('2010-01-01 11:00:00', NULL, '2011-01-01 12:00:00'); +mysql> alter table test.t_TIMESTAMP_NCI set tiflash replica 1 + +# YEAR +## clustered_index +mysql> drop table if exists test.t_YEAR_CI +mysql> create table test.t_YEAR_CI(pk YEAR NOT NULL, col1 YEAR, col2 YEAR NOT NULL, PRIMARY KEY CLUSTERED (pk)); +mysql> insert into test.t_YEAR_CI values('2021', NULL, '2022'); +mysql> alter table test.t_YEAR_CI set tiflash replica 1 +## nonclustered_index +mysql> drop table if exists test.t_YEAR_NCI +mysql> create table test.t_YEAR_NCI(pk YEAR NOT NULL, col1 YEAR, col2 YEAR NOT NULL, PRIMARY KEY NONCLUSTERED (pk)); +mysql> insert into test.t_YEAR_NCI values('2021', NULL, '2022'); +mysql> alter table test.t_YEAR_NCI set tiflash replica 1 + +# JSON +mysql> drop table if exists test.t_JSON; +mysql> create table test.t_JSON(col1 JSON, col2 JSON NOT NULL); +mysql> insert into test.t_JSON values(NULL, '[1, 2, 3]'); +mysql> alter table test.t_JSON set tiflash replica 1 + + +# wait table +func> wait_table test t_TINYINT_CI +func> wait_table test t_TINYINT_NCI +func> wait_table test t_SMALLINT_CI +func> wait_table test t_SMALLINT_NCI +func> wait_table test t_MEDIUMINT_CI +func> wait_table test t_MEDIUMINT_NCI +func> wait_table test t_INT_CI +func> wait_table test t_INT_NCI +func> wait_table test t_BIGINT_CI +func> wait_table test t_BIGINT_NCI +func> wait_table test t_FLOAT_CI +func> wait_table test t_FLOAT_NCI +func> wait_table test t_DOUBLE_CI +func> wait_table test t_DOUBLE_NCI +func> wait_table test t_DECIMAL32_CI +func> wait_table test t_DECIMAL32_NCI +func> wait_table test t_DECIMAL64_CI +func> wait_table test t_DECIMAL64_NCI +func> wait_table test t_DECIMAL128_CI +func> wait_table test t_DECIMAL128_NCI +func> wait_table test t_DECIMAL256_CI +func> wait_table test t_DECIMAL256_NCI +func> wait_table test t_CHAR_CI +func> wait_table test t_CHAR_NCI +func> wait_table test t_VARCHAR_CI +func> wait_table test t_VARCHAR_NCI +func> wait_table test t_BINARY_CI +func> wait_table test t_BINARY_NCI +func> wait_table test t_VARBINARY_CI +func> wait_table test t_VARBINARY_NCI +func> wait_table test t_BLOB +func> wait_table test t_TEXT +func> wait_table test t_ENUM_CI +func> wait_table test t_ENUM_NCI +func> wait_table test t_SET_CI +func> wait_table test t_SET_NCI +func> wait_table test t_DATE_CI +func> wait_table test t_DATE_NCI +func> wait_table test t_TIME_CI +func> wait_table test t_TIME_NCI +func> wait_table test t_DATETIME_CI +func> wait_table test t_DATETIME_NCI +func> wait_table test t_TIMESTAMP_CI +func> wait_table test t_TIMESTAMP_NCI +func> wait_table test t_YEAR_CI +func> wait_table test t_YEAR_NCI +func> wait_table test t_JSON + + +# TINYINT +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_TINYINT_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_TINYINT_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ + +# SMALLINT +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_SMALLINT_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_SMALLINT_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ + +# MEDIUMINT +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_MEDIUMINT_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_MEDIUMINT_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ + +# INT +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_INT_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_INT_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ + +# BIGINT +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_BIGINT_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_BIGINT_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ + +# FLOAT +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_FLOAT_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_FLOAT_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ + +# DOUBLE +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DOUBLE_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DOUBLE_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 0 | NULL | 0 | ++----+------+------+ + +# DECIMAL32 +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL32_CI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL32_NCI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ + +# DECIMAL64 +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL64_CI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL64_NCI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ + +# DECIMAL128 +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL128_CI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL128_NCI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ + +# DECIMAL256 +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL256_CI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DECIMAL256_NCI; ++-----+------+------+ +| pk | col1 | col2 | ++-----+------+------+ +| 0.1 | NULL | 0.1 | ++-----+------+------+ + +# CHAR +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_CHAR_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_CHAR_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ + +# VARCHAR +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_VARCHAR_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_VARCHAR_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ + +# BINARY +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_BINARY_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 1 | NULL | 2 | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_BINARY_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| 1 | NULL | 2 | ++----+------+------+ + +# VARBINARY +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_VARBINARY_CI; ++------+------+------+ +| pk | col1 | col2 | ++------+------+------+ +| 1234 | NULL | 5678 | ++------+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_VARBINARY_NCI; ++------+------+------+ +| pk | col1 | col2 | ++------+------+------+ +| 1234 | NULL | 5678 | ++------+------+------+ + +# BLOB +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_BLOB; ++------+------+ +| col1 | col2 | ++------+------+ +| NULL | 1234 | ++------+------+ + +# TEXT +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_TEXT; ++------+------+ +| col1 | col2 | ++------+------+ +| NULL | b | ++------+------+ + +# ENUM +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_ENUM_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_ENUM_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ + +# SET +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_SET_CI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_SET_NCI; ++----+------+------+ +| pk | col1 | col2 | ++----+------+------+ +| a | NULL | b | ++----+------+------+ + +# DATE +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DATE_CI; ++------------+------+------------+ +| pk | col1 | col2 | ++------------+------+------------+ +| 2010-01-01 | NULL | 2010-01-02 | ++------------+------+------------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DATE_NCI; ++------------+------+------------+ +| pk | col1 | col2 | ++------------+------+------------+ +| 2010-01-01 | NULL | 2010-01-02 | ++------------+------+------------+ + +# TIME +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_TIME_CI; ++----------+------+----------+ +| pk | col1 | col2 | ++----------+------+----------+ +| 10:00:00 | NULL | 11:00:00 | ++----------+------+----------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_TIME_NCI; ++----------+------+----------+ +| pk | col1 | col2 | ++----------+------+----------+ +| 10:00:00 | NULL | 11:00:00 | ++----------+------+----------+ + +# DATETIME +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DATETIME_CI; ++---------------------+------+---------------------+ +| pk | col1 | col2 | ++---------------------+------+---------------------+ +| 2010-01-01 11:00:00 | NULL | 2011-01-01 12:00:00 | ++---------------------+------+---------------------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_DATETIME_NCI; ++---------------------+------+---------------------+ +| pk | col1 | col2 | ++---------------------+------+---------------------+ +| 2010-01-01 11:00:00 | NULL | 2011-01-01 12:00:00 | ++---------------------+------+---------------------+ + +# TIMESTAMP +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_TIMESTAMP_CI; ++---------------------+------+---------------------+ +| pk | col1 | col2 | ++---------------------+------+---------------------+ +| 2010-01-01 11:00:00 | NULL | 2011-01-01 12:00:00 | ++---------------------+------+---------------------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_TIMESTAMP_NCI; ++---------------------+------+---------------------+ +| pk | col1 | col2 | ++---------------------+------+---------------------+ +| 2010-01-01 11:00:00 | NULL | 2011-01-01 12:00:00 | ++---------------------+------+---------------------+ + +# YEAR +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_YEAR_CI; ++------+------+------+ +| pk | col1 | col2 | ++------+------+------+ +| 2021 | NULL | 2022 | ++------+------+------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_YEAR_NCI; ++------+------+------+ +| pk | col1 | col2 | ++------+------+------+ +| 2021 | NULL | 2022 | ++------+------+------+ + +# JSON +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_JSON; ++------+-----------+ +| col1 | col2 | ++------+-----------+ +| NULL | [1, 2, 3] | ++------+-----------+ + + +# drop table +mysql> drop table if exists test.t_TINYINT_CI +mysql> drop table if exists test.t_TINYINT_NCI +mysql> drop table if exists test.t_SMALLINT_CI +mysql> drop table if exists test.t_SMALLINT_NCI +mysql> drop table if exists test.t_MEDIUMINT_CI +mysql> drop table if exists test.t_MEDIUMINT_NCI +mysql> drop table if exists test.t_INT_CI +mysql> drop table if exists test.t_INT_NCI +mysql> drop table if exists test.t_BIGINT_CI +mysql> drop table if exists test.t_BIGINT_NCI +mysql> drop table if exists test.t_FLOAT_CI +mysql> drop table if exists test.t_FLOAT_NCI +mysql> drop table if exists test.t_DOUBLE_CI +mysql> drop table if exists test.t_DOUBLE_NCI +mysql> drop table if exists test.t_DECIMAL32_CI +mysql> drop table if exists test.t_DECIMAL32_NCI +mysql> drop table if exists test.t_DECIMAL64_CI +mysql> drop table if exists test.t_DECIMAL64_NCI +mysql> drop table if exists test.t_DECIMAL128_CI +mysql> drop table if exists test.t_DECIMAL128_NCI +mysql> drop table if exists test.t_DECIMAL256_CI +mysql> drop table if exists test.t_DECIMAL256_NCI +mysql> drop table if exists test.t_CHAR_CI +mysql> drop table if exists test.t_CHAR_NCI +mysql> drop table if exists test.t_VARCHAR_CI +mysql> drop table if exists test.t_VARCHAR_NCI +mysql> drop table if exists test.t_BINARY_CI +mysql> drop table if exists test.t_BINARY_NCI +mysql> drop table if exists test.t_VARBINARY_CI +mysql> drop table if exists test.t_VARBINARY_NCI +mysql> drop table if exists test.t_BLOB +mysql> drop table if exists test.t_TEXT +mysql> drop table if exists test.t_ENUM_CI +mysql> drop table if exists test.t_ENUM_NCI +mysql> drop table if exists test.t_SET_CI +mysql> drop table if exists test.t_SET_NCI +mysql> drop table if exists test.t_DATE_CI +mysql> drop table if exists test.t_DATE_NCI +mysql> drop table if exists test.t_TIME_CI +mysql> drop table if exists test.t_TIME_NCI +mysql> drop table if exists test.t_DATETIME_CI +mysql> drop table if exists test.t_DATETIME_NCI +mysql> drop table if exists test.t_TIMESTAMP_CI +mysql> drop table if exists test.t_TIMESTAMP_NCI +mysql> drop table if exists test.t_YEAR_CI +mysql> drop table if exists test.t_YEAR_NCI +mysql> drop table if exists test.t_JSON diff --git a/tests/tidb-ci/disable_planner b/tests/tidb-ci/disable_planner new file mode 120000 index 00000000000..1c6442406ac --- /dev/null +++ b/tests/tidb-ci/disable_planner @@ -0,0 +1 @@ +../fullstack-test/mpp \ No newline at end of file diff --git a/tests/tidb-ci/run.sh b/tests/tidb-ci/run.sh index 1dde9ee11e8..18a242fe4d3 100755 --- a/tests/tidb-ci/run.sh +++ b/tests/tidb-ci/run.sh @@ -47,6 +47,13 @@ docker-compose -f cluster.yaml -f tiflash-dt-disable-local-tunnel.yaml exec -T t docker-compose -f cluster.yaml -f tiflash-dt-disable-local-tunnel.yaml down clean_data_log +docker-compose -f cluster.yaml -f tiflash-dt-disable-planner.yaml up -d +wait_env +docker-compose -f cluster.yaml -f tiflash-dt-disable-planner.yaml exec -T tiflash0 bash -c 'cd /tests ; ./run-test.sh tidb-ci/disable_planner' + +docker-compose -f cluster.yaml -f tiflash-dt-disable-planner.yaml down +clean_data_log + # run new_collation_fullstack tests docker-compose -f cluster_new_collation.yaml -f tiflash-dt.yaml down clean_data_log diff --git a/tests/tidb-ci/tiflash-dt-disable-planner.yaml b/tests/tidb-ci/tiflash-dt-disable-planner.yaml new file mode 120000 index 00000000000..51abe9246db --- /dev/null +++ b/tests/tidb-ci/tiflash-dt-disable-planner.yaml @@ -0,0 +1 @@ +../docker/tiflash-dt-disable-planner.yaml \ No newline at end of file