From d68c8a901068a42d6a41d1e9b7907cc264ff8edc Mon Sep 17 00:00:00 2001 From: jinhelin Date: Tue, 21 Nov 2023 12:16:09 +0800 Subject: [PATCH 1/6] Fix unstable ut. (#8393) ref pingcap/tiflash#6233 --- dbms/src/Encryption/tests/gtest_rate_limiter.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/dbms/src/Encryption/tests/gtest_rate_limiter.cpp b/dbms/src/Encryption/tests/gtest_rate_limiter.cpp index 9156d0df7a2..86fd4becccf 100644 --- a/dbms/src/Encryption/tests/gtest_rate_limiter.cpp +++ b/dbms/src/Encryption/tests/gtest_rate_limiter.cpp @@ -67,11 +67,10 @@ TEST(WriteLimiterTest, Rate) thread.join(); auto elapsed = watch.elapsedSeconds(); auto actual_rate = write_limiter->getTotalBytesThrough() / elapsed; - // make sure that 0.8 * target <= actual_rate <= 1.25 * target - // hint: the range [0.8, 1.25] is copied from rocksdb, - // if tests fail, try to enlarge this range. - // enlarge the range to [0.75, 1.30] - EXPECT_GE(actual_rate / target, 0.75); + // For environments with high loads, latency can be very large. + // In theory, the upper bound of `elapsed` cannot be guaranteed. + // So that we cannot guarantee the lower bound of `actual_rate`. + // EXPECT_GE(actual_rate / target, 0.75); EXPECT_LE(actual_rate / target, 1.30); } } From 3d3a5f14d51a55c636223e1ac9c9038e082ebb62 Mon Sep 17 00:00:00 2001 From: jinhelin Date: Tue, 21 Nov 2023 21:32:40 +0800 Subject: [PATCH 2/6] Rename some names of DB::DM::Remote::Serializer. (#8401) ref pingcap/tiflash#6834 --- .../WNEstablishDisaggTaskHandler.cpp | 3 +- .../Storages/DeltaMerge/Remote/Serializer.cpp | 30 ++++++++-------- .../Storages/DeltaMerge/Remote/Serializer.h | 35 ++++++++----------- .../Storages/DeltaMerge/SegmentReadTask.cpp | 8 ++--- .../tests/gtest_segment_read_task.cpp | 2 +- 5 files changed, 35 insertions(+), 43 deletions(-) diff --git a/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp b/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp index a68c1bc3685..3fa5214d03a 100644 --- a/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp +++ b/dbms/src/Flash/Disaggregated/WNEstablishDisaggTaskHandler.cpp @@ -101,7 +101,8 @@ void WNEstablishDisaggTaskHandler::execute(disaggregated::EstablishDisaggTaskRes using DM::Remote::Serializer; snap->iterateTableSnapshots([&](const DM::Remote::DisaggPhysicalTableReadSnapshotPtr & snap) { - response->add_tables(Serializer::serializeTo(snap, task_id, mem_tracker_wrapper).SerializeAsString()); + response->add_tables( + Serializer::serializePhysicalTable(snap, task_id, mem_tracker_wrapper).SerializeAsString()); }); } diff --git a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp index ff2f0b346c2..4321f826952 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.cpp @@ -43,7 +43,7 @@ extern const Metric DT_SnapshotOfDisaggReadNodeRead; namespace DB::DM::Remote { -RemotePb::RemotePhysicalTable Serializer::serializeTo( +RemotePb::RemotePhysicalTable Serializer::serializePhysicalTable( const DisaggPhysicalTableReadSnapshotPtr & snap, const DisaggTaskId & task_id, MemTrackerWrapper & mem_tracker_wrapper) @@ -55,7 +55,7 @@ RemotePb::RemotePhysicalTable Serializer::serializeTo( remote_table.set_table_id(snap->ks_physical_table_id.second); for (const auto & [seg_id, seg_task] : snap->tasks) { - auto remote_seg = Serializer::serializeTo( + auto remote_seg = Serializer::serializeSegment( seg_task->read_snapshot, seg_id, seg_task->segment->segmentEpoch(), @@ -67,7 +67,7 @@ RemotePb::RemotePhysicalTable Serializer::serializeTo( return remote_table; } -RemotePb::RemoteSegment Serializer::serializeTo( +RemotePb::RemoteSegment Serializer::serializeSegment( const SegmentSnapshotPtr & snap, PageIdU64 segment_id, UInt64 segment_epoch, @@ -99,9 +99,9 @@ RemotePb::RemoteSegment Serializer::serializeTo( checkpoint_info->set_data_file_id(dt_file->path()); // It should be a key to remote path } remote.mutable_column_files_memtable()->CopyFrom( - serializeTo(snap->delta->getMemTableSetSnapshot(), mem_tracker_wrapper)); + serializeColumnFileSet(snap->delta->getMemTableSetSnapshot(), mem_tracker_wrapper)); remote.mutable_column_files_persisted()->CopyFrom( - serializeTo(snap->delta->getPersistedFileSetSnapshot(), mem_tracker_wrapper)); + serializeColumnFileSet(snap->delta->getPersistedFileSetSnapshot(), mem_tracker_wrapper)); // serialize the read ranges to read node for (const auto & read_range : read_ranges) @@ -114,7 +114,7 @@ RemotePb::RemoteSegment Serializer::serializeTo( return remote; } -SegmentSnapshotPtr Serializer::deserializeSegmentSnapshotFrom( +SegmentSnapshotPtr Serializer::deserializeSegment( const DMContext & dm_context, StoreID remote_store_id, KeyspaceID keyspace_id, @@ -177,7 +177,7 @@ SegmentSnapshotPtr Serializer::deserializeSegmentSnapshotFrom( Logger::get(dm_context.tracing_id)); } -RepeatedPtrField Serializer::serializeTo( +RepeatedPtrField Serializer::serializeColumnFileSet( const ColumnFileSetSnapshotPtr & snap, MemTrackerWrapper & mem_tracker_wrapper) { @@ -187,19 +187,19 @@ RepeatedPtrField Serializer::serializeTo( { if (auto * cf_in_mem = file->tryToInMemoryFile(); cf_in_mem) { - ret.Add(serializeTo(*cf_in_mem)); + ret.Add(serializeCFInMemory(*cf_in_mem)); } else if (auto * cf_tiny = file->tryToTinyFile(); cf_tiny) { - ret.Add(serializeTo(*cf_tiny, snap->getDataProvider())); + ret.Add(serializeCFTiny(*cf_tiny, snap->getDataProvider())); } else if (auto * cf_delete_range = file->tryToDeleteRange(); cf_delete_range) { - ret.Add(serializeTo(*cf_delete_range)); + ret.Add(serializeCFDeleteRange(*cf_delete_range)); } else if (auto * cf_big = file->tryToBigFile(); cf_big) { - ret.Add(serializeTo(*cf_big)); + ret.Add(serializeCFBig(*cf_big)); } else { @@ -253,7 +253,7 @@ ColumnFileSetSnapshotPtr Serializer::deserializeColumnFileSet( return ret; } -RemotePb::ColumnFileRemote Serializer::serializeTo(const ColumnFileInMemory & cf_in_mem) +RemotePb::ColumnFileRemote Serializer::serializeCFInMemory(const ColumnFileInMemory & cf_in_mem) { RemotePb::ColumnFileRemote ret; auto * remote_in_memory = ret.mutable_in_memory(); @@ -315,7 +315,7 @@ ColumnFileInMemoryPtr Serializer::deserializeCFInMemory(const RemotePb::ColumnFi return std::make_shared(schema, cache); } -RemotePb::ColumnFileRemote Serializer::serializeTo( +RemotePb::ColumnFileRemote Serializer::serializeCFTiny( const ColumnFileTiny & cf_tiny, IColumnFileDataProviderPtr data_provider) { @@ -352,7 +352,7 @@ ColumnFileTinyPtr Serializer::deserializeCFTiny(const RemotePb::ColumnFileTiny & return cf; } -RemotePb::ColumnFileRemote Serializer::serializeTo(const ColumnFileDeleteRange & cf_delete_range) +RemotePb::ColumnFileRemote Serializer::serializeCFDeleteRange(const ColumnFileDeleteRange & cf_delete_range) { RemotePb::ColumnFileRemote ret; auto * remote_del = ret.mutable_delete_range(); @@ -373,7 +373,7 @@ ColumnFileDeleteRangePtr Serializer::deserializeCFDeleteRange(const RemotePb::Co return std::make_shared(range); } -RemotePb::ColumnFileRemote Serializer::serializeTo(const ColumnFileBig & cf_big) +RemotePb::ColumnFileRemote Serializer::serializeCFBig(const ColumnFileBig & cf_big) { RemotePb::ColumnFileRemote ret; auto * remote_big = ret.mutable_big(); diff --git a/dbms/src/Storages/DeltaMerge/Remote/Serializer.h b/dbms/src/Storages/DeltaMerge/Remote/Serializer.h index c72c21550a5..de24ccc226c 100644 --- a/dbms/src/Storages/DeltaMerge/Remote/Serializer.h +++ b/dbms/src/Storages/DeltaMerge/Remote/Serializer.h @@ -56,14 +56,21 @@ namespace DB::DM::Remote { struct Serializer { - static RemotePb::RemotePhysicalTable serializeTo( +public: + static RemotePb::RemotePhysicalTable serializePhysicalTable( const DisaggPhysicalTableReadSnapshotPtr & snap, const DisaggTaskId & task_id, MemTrackerWrapper & mem_tracker_wrapper); - /// segment snapshot /// + static SegmentSnapshotPtr deserializeSegment( + const DMContext & dm_context, + StoreID remote_store_id, + KeyspaceID keyspace_id, + TableID table_id, + const RemotePb::RemoteSegment & proto); - static RemotePb::RemoteSegment serializeTo( +private: + static RemotePb::RemoteSegment serializeSegment( const SegmentSnapshotPtr & snap, PageIdU64 segment_id, UInt64 segment_epoch, @@ -71,19 +78,9 @@ struct Serializer const RowKeyRanges & read_ranges, MemTrackerWrapper & mem_tracker_wrapper); - static SegmentSnapshotPtr deserializeSegmentSnapshotFrom( - const DMContext & dm_context, - StoreID remote_store_id, - KeyspaceID keyspace_id, - TableID table_id, - const RemotePb::RemoteSegment & proto); - - /// column file set /// - - static google::protobuf::RepeatedPtrField serializeTo( + static google::protobuf::RepeatedPtrField serializeColumnFileSet( const ColumnFileSetSnapshotPtr & snap, MemTrackerWrapper & mem_tracker_wrapper); - /// Note: This function always build a snapshot over nop data provider. In order to read from this snapshot, /// you must explicitly assign a proper data provider. static ColumnFileSetSnapshotPtr deserializeColumnFileSet( @@ -91,20 +88,18 @@ struct Serializer const Remote::IDataStorePtr & data_store, const RowKeyRange & segment_range); - /// column file /// - - static RemotePb::ColumnFileRemote serializeTo(const ColumnFileInMemory & cf_in_mem); + static RemotePb::ColumnFileRemote serializeCFInMemory(const ColumnFileInMemory & cf_in_mem); static ColumnFileInMemoryPtr deserializeCFInMemory(const RemotePb::ColumnFileInMemory & proto); - static RemotePb::ColumnFileRemote serializeTo( + static RemotePb::ColumnFileRemote serializeCFTiny( const ColumnFileTiny & cf_tiny, IColumnFileDataProviderPtr data_provider); static ColumnFileTinyPtr deserializeCFTiny(const RemotePb::ColumnFileTiny & proto); - static RemotePb::ColumnFileRemote serializeTo(const ColumnFileDeleteRange & cf_delete_range); + static RemotePb::ColumnFileRemote serializeCFDeleteRange(const ColumnFileDeleteRange & cf_delete_range); static ColumnFileDeleteRangePtr deserializeCFDeleteRange(const RemotePb::ColumnFileDeleteRange & proto); - static RemotePb::ColumnFileRemote serializeTo(const ColumnFileBig & cf_big); + static RemotePb::ColumnFileRemote serializeCFBig(const ColumnFileBig & cf_big); static ColumnFileBigPtr deserializeCFBig( const RemotePb::ColumnFileBig & proto, const Remote::IDataStorePtr & data_store, diff --git a/dbms/src/Storages/DeltaMerge/SegmentReadTask.cpp b/dbms/src/Storages/DeltaMerge/SegmentReadTask.cpp index ed3d711b713..f950b426d71 100644 --- a/dbms/src/Storages/DeltaMerge/SegmentReadTask.cpp +++ b/dbms/src/Storages/DeltaMerge/SegmentReadTask.cpp @@ -100,12 +100,8 @@ SegmentReadTask::SegmentReadTask( nullptr, nullptr); - read_snapshot = Remote::Serializer::deserializeSegmentSnapshotFrom( - *dm_context, - store_id, - keyspace_id, - physical_table_id, - proto); + read_snapshot + = Remote::Serializer::deserializeSegment(*dm_context, store_id, keyspace_id, physical_table_id, proto); ranges.reserve(proto.read_key_ranges_size()); for (const auto & read_key_range : proto.read_key_ranges()) diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp index 548b58a169a..2106fc7d02e 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_read_task.cpp @@ -186,7 +186,7 @@ try snap->column_defines = std::make_shared(store->getTableColumns()); MemTrackerWrapper mem_tracker_wrapper(nullptr); - auto remote_table_pb = Remote::Serializer::serializeTo(snap, /*task_id*/ {}, mem_tracker_wrapper); + auto remote_table_pb = Remote::Serializer::serializePhysicalTable(snap, /*task_id*/ {}, mem_tracker_wrapper); ASSERT_GT(remote_table_pb.segments_size(), 0); From 9e0934a4f847f8b6f5db281627e61e5b8f257cde Mon Sep 17 00:00:00 2001 From: JaySon Date: Wed, 22 Nov 2023 19:17:41 +0800 Subject: [PATCH 3/6] ddl: Fix broken TableIDMapping lead to data loss or crashes (#8402) close pingcap/tiflash#8398 --- dbms/src/Debug/MockSchemaGetter.h | 2 +- .../Encryption/tests/gtest_rate_limiter.cpp | 6 +- .../Coprocessor/DAGStorageInterpreter.cpp | 6 +- dbms/src/TiDB/Schema/SchemaBuilder.cpp | 420 +++++++++++++----- dbms/src/TiDB/Schema/SchemaBuilder.h | 102 ++--- dbms/src/TiDB/Schema/SchemaGetter.cpp | 20 +- dbms/src/TiDB/Schema/SchemaGetter.h | 11 +- dbms/src/TiDB/Schema/TiDB.cpp | 24 +- dbms/src/TiDB/Schema/TiDBSchemaSyncer.cpp | 101 +++-- dbms/src/TiDB/Schema/TiDBSchemaSyncer.h | 7 + .../TiDB/Schema/tests/gtest_table_id_map.cpp | 166 +++++++ .../ddl/alter_exchange_partition.test | 99 +++++ .../ddl/rename_table_across_databases.test | 42 ++ 13 files changed, 796 insertions(+), 210 deletions(-) create mode 100644 dbms/src/TiDB/Schema/tests/gtest_table_id_map.cpp diff --git a/dbms/src/Debug/MockSchemaGetter.h b/dbms/src/Debug/MockSchemaGetter.h index 61701bdd250..0208ce49c5c 100644 --- a/dbms/src/Debug/MockSchemaGetter.h +++ b/dbms/src/Debug/MockSchemaGetter.h @@ -34,7 +34,7 @@ struct MockSchemaGetter static bool checkSchemaDiffExists(Int64 version) { return MockTiDB::instance().checkSchemaDiffExists(version); } - static TiDB::TableInfoPtr getTableInfo(DatabaseID, TableID table_id) + static TiDB::TableInfoPtr getTableInfo(DatabaseID, TableID table_id, [[maybe_unused]] bool try_mvcc = true) { return MockTiDB::instance().getTableInfoByID(table_id); } diff --git a/dbms/src/Encryption/tests/gtest_rate_limiter.cpp b/dbms/src/Encryption/tests/gtest_rate_limiter.cpp index 86fd4becccf..9feab4c35b1 100644 --- a/dbms/src/Encryption/tests/gtest_rate_limiter.cpp +++ b/dbms/src/Encryption/tests/gtest_rate_limiter.cpp @@ -70,8 +70,10 @@ TEST(WriteLimiterTest, Rate) // For environments with high loads, latency can be very large. // In theory, the upper bound of `elapsed` cannot be guaranteed. // So that we cannot guarantee the lower bound of `actual_rate`. - // EXPECT_GE(actual_rate / target, 0.75); - EXPECT_LE(actual_rate / target, 1.30); + // EXPECT_GE(actual_rate / target, 0.75) + // << fmt::format("actual_rate={} target={} elapsed={:.3f}s", actual_rate, target, elapsed); + EXPECT_LE(actual_rate / target, 1.30) + << fmt::format("actual_rate={} target={} elapsed={:.3f}s", actual_rate, target, elapsed); } } diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index c4bc532eaba..7fe083c0934 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -1344,7 +1344,11 @@ std::unordered_map DAG if (!table_store) { if (schema_synced) - throw TiFlashException(fmt::format("Table {} doesn't exist.", table_id), Errors::Table::NotExists); + throw TiFlashException( + Errors::Table::NotExists, + "Table doesn't exist, keyspace={} table_id={}", + keyspace_id, + table_id); else return {{}, {}}; } diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index bcfd77b100d..d54960ee776 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -58,6 +59,128 @@ extern const int DDL_ERROR; extern const int SYNTAX_ERROR; } // namespace ErrorCodes +void TableIDMap::doEmplaceTableID( + TableID table_id, + DatabaseID database_id, + std::string_view log_prefix, + const std::unique_lock &) +{ + if (auto iter = table_id_to_database_id.find(table_id); // + iter != table_id_to_database_id.end()) + { + if (iter->second != database_id) + { + LOG_WARNING( + log, + "{}table_id to database_id is being overwrite, table_id={}" + " old_database_id={} new_database_id={}", + log_prefix, + table_id, + iter->second, + database_id); + iter->second = database_id; + } + } + else + table_id_to_database_id.emplace(table_id, database_id); +} + +void TableIDMap::doEmplacePartitionTableID( + TableID partition_id, + TableID table_id, + std::string_view log_prefix, + const std::unique_lock &) +{ + if (auto iter = partition_id_to_logical_id.find(partition_id); // + iter != partition_id_to_logical_id.end()) + { + if (iter->second != table_id) + { + LOG_WARNING( + log, + "{}partition_id to table_id is being overwrite, physical_table_id={}" + " old_logical_table_id={} new_logical_table_id={}", + log_prefix, + partition_id, + iter->second, + table_id); + iter->second = table_id; + } + } + else + partition_id_to_logical_id.emplace(partition_id, table_id); +} + +void TableIDMap::exchangeTablePartition( + DatabaseID non_partition_database_id, + TableID non_partition_table_id, + DatabaseID /*partition_database_id*/, + TableID partition_logical_table_id, + TableID partition_physical_table_id) +{ + // Change all under the same lock + std::unique_lock lock(mtx_id_mapping); + // erase the non partition table + if (auto iter = table_id_to_database_id.find(non_partition_table_id); iter != table_id_to_database_id.end()) + table_id_to_database_id.erase(iter); + else + LOG_WARNING( + log, + "ExchangeTablePartition: non partition table not in table_id_to_database_id, table_id={}", + non_partition_table_id); + + // make the partition table to be a non-partition table + doEmplaceTableID(partition_physical_table_id, non_partition_database_id, "ExchangeTablePartition: ", lock); + + // remove the partition table to logical table mapping + if (auto iter = partition_id_to_logical_id.find(partition_physical_table_id); + iter != partition_id_to_logical_id.end()) + { + partition_id_to_logical_id.erase(iter); + } + else + { + LOG_WARNING( + log, + "ExchangeTablePartition: partition table not in partition_id_to_logical_id, physical_table_id={}", + partition_physical_table_id); + } + + // make the non partition table as a partition to logical table + doEmplacePartitionTableID(non_partition_table_id, partition_logical_table_id, "ExchangeTablePartition: ", lock); +} + +std::tuple TableIDMap::findDatabaseIDAndLogicalTableID(TableID physical_table_id) const +{ + std::shared_lock lock(mtx_id_mapping); + DatabaseID database_id = -1; + if (auto database_iter = table_id_to_database_id.find(physical_table_id); + database_iter != table_id_to_database_id.end()) + { + database_id = database_iter->second; + // This is a non-partition table or the logical_table of partition table. + return {true, database_id, physical_table_id}; + } + + /// if we can't find physical_table_id in table_id_to_database_id, + /// we should first try to find it in partition_id_to_logical_id because it could be the pysical_table_id of partition tables + TableID logical_table_id = -1; + if (auto logical_table_iter = partition_id_to_logical_id.find(physical_table_id); + logical_table_iter != partition_id_to_logical_id.end()) + { + logical_table_id = logical_table_iter->second; + // try to get the database_id of logical_table_id + if (auto database_iter = table_id_to_database_id.find(logical_table_id); + database_iter != table_id_to_database_id.end()) + { + database_id = database_iter->second; + // This is a non-partition table or the logical_table of partition table. + return {true, database_id, logical_table_id}; + } + } + return {false, 0, 0}; +} + bool isReservedDatabase(Context & context, const String & database_name) { return context.getTMTContext().getIgnoreDatabases().count(database_name) > 0; @@ -69,7 +192,10 @@ void SchemaBuilder::applyCreateTable(DatabaseID database_id, auto table_info = getter.getTableInfo(database_id, table_id); if (table_info == nullptr) // the database maybe dropped { - LOG_DEBUG(log, "table is not exist in TiKV, may have been dropped, table_id={}", table_id); + LOG_INFO( + log, + "table is not exist in TiKV, may have been dropped, applyCreateTable is ignored, table_id={}", + table_id); return; } @@ -112,70 +238,120 @@ void SchemaBuilder::applyExchangeTablePartition(const Schema diff.table_id); return; } - LOG_DEBUG( - log, - "Table and partition is exchanged. database_id={} table_id={}, part_db_id={}, part_table_id={} partition_id={}", - diff.old_schema_id, - diff.old_table_id, - diff.affected_opts[0].schema_id, - diff.affected_opts[0].table_id, - diff.table_id); - /// Table_id in diff is the partition id of which will be exchanged, + + if (diff.affected_opts.empty()) + { + throw TiFlashException( + Errors::DDL::Internal, + "Invalid exchange partition schema diff without affected_opts, affected_opts_size={}", + diff.affected_opts.size()); + } + + /// `ALTER TABLE partition_table EXCHANGE PARTITION partition_name WITH TABLE non_partition_table` + /// Table_id in diff is the partition id of which will be exchanged /// Schema_id in diff is the non-partition table's schema id /// Old_table_id in diff is the non-partition table's table id /// Table_id in diff.affected_opts[0] is the table id of the partition table /// Schema_id in diff.affected_opts[0] is the schema id of the partition table - table_id_map.eraseTableIDOrLogError(diff.old_table_id); - table_id_map.emplaceTableID(diff.table_id, diff.schema_id); - table_id_map.erasePartitionTableIDOrLogError(diff.table_id); - table_id_map.emplacePartitionTableID(diff.old_table_id, diff.affected_opts[0].table_id); + const auto non_partition_database_id = diff.old_schema_id; + const auto non_partition_table_id = diff.old_table_id; + const auto partition_database_id = diff.affected_opts[0].schema_id; + const auto partition_logical_table_id = diff.affected_opts[0].table_id; + const auto partition_physical_table_id = diff.table_id; + LOG_INFO( + log, + "Execute exchange partition begin. database_id={} table_id={} part_database_id={} part_logical_table_id={}" + " physical_table_id={}", + non_partition_database_id, + non_partition_table_id, + partition_database_id, + partition_logical_table_id, + partition_physical_table_id); + GET_METRIC(tiflash_schema_internal_ddl_count, type_exchange_partition).Increment(); - if (diff.schema_id != diff.affected_opts[0].schema_id) + table_id_map.exchangeTablePartition( + non_partition_database_id, + non_partition_table_id, + partition_database_id, + partition_logical_table_id, + partition_physical_table_id); + + if (non_partition_database_id != partition_database_id) { - // rename old_table_id(non-partition table) + // Rename old non-partition table belonging new database. Now it should be belong to + // the database of partition table. + auto & tmt_context = context.getTMTContext(); + do { - auto [new_db_info, new_table_info] - = getter.getDatabaseAndTableInfo(diff.affected_opts[0].schema_id, diff.affected_opts[0].table_id); - if (new_table_info == nullptr) + // skip if the instance is not created + auto storage = tmt_context.getStorages().get(keyspace_id, non_partition_table_id); + if (storage == nullptr) { - LOG_ERROR(log, "table is not exist in TiKV, table_id={}", diff.affected_opts[0].table_id); - return; + LOG_INFO( + log, + "ExchangeTablePartition: non_partition_table instance is not created in TiFlash, rename is" + " ignored, table_id={}", + non_partition_table_id); + break; } - auto & tmt_context = context.getTMTContext(); - auto storage = tmt_context.getStorages().get(keyspace_id, diff.old_table_id); - if (storage == nullptr) + auto [new_db_info, new_table_info] + = getter.getDatabaseAndTableInfo(partition_database_id, partition_logical_table_id); + if (new_table_info == nullptr) { - LOG_ERROR(log, "table is not exist in TiFlash, table_id={}", diff.old_table_id); - return; + LOG_INFO( + log, + "ExchangeTablePartition: part_logical_table table is not exist in TiKV, rename is ignored," + " table_id={}", + partition_logical_table_id); + break; } - auto part_table_info = new_table_info->producePartitionTableInfo(diff.old_table_id, name_mapper); + auto part_table_info = new_table_info->producePartitionTableInfo(non_partition_table_id, name_mapper); applyRenamePhysicalTable(new_db_info, *part_table_info, storage); - } + } while (false); - // rename table_id(the exchanged partition table) + // Rename the exchanged partition table belonging new database. Now it should belong to + // the database of non-partition table + do { - auto [new_db_info, new_table_info] = getter.getDatabaseAndTableInfo(diff.schema_id, diff.table_id); - if (new_table_info == nullptr) + // skip if the instance is not created + auto storage = tmt_context.getStorages().get(keyspace_id, partition_physical_table_id); + if (storage == nullptr) { - LOG_ERROR(log, "table is not exist in TiKV, table_id={}", diff.table_id); - return; + LOG_INFO( + log, + "ExchangeTablePartition: partition_physical_table instance is not created in TiFlash, rename is" + " ignored, table_id={}", + partition_physical_table_id); + break; } - auto & tmt_context = context.getTMTContext(); - auto storage = tmt_context.getStorages().get(keyspace_id, diff.table_id); - if (storage == nullptr) + auto [new_db_info, new_table_info] + = getter.getDatabaseAndTableInfo(non_partition_database_id, partition_physical_table_id); + if (new_table_info == nullptr) { - LOG_ERROR(log, "table is not exist in TiFlash, table_id={}", diff.old_table_id); - return; + LOG_INFO( + log, + "ExchangeTablePartition: partition_physical_table is not exist in TiKV, rename is ignored," + " table_id={}", + partition_physical_table_id); + break; } applyRenamePhysicalTable(new_db_info, *new_table_info, storage); - } + } while (false); } - GET_METRIC(tiflash_schema_internal_ddl_count, type_exchange_partition).Increment(); + LOG_INFO( + log, + "Execute exchange partition end. database_id={} table_id={} part_database_id={} part_logical_table_id={}" + " physical_table_id={}", + non_partition_database_id, + non_partition_table_id, + partition_database_id, + partition_logical_table_id, + partition_physical_table_id); } template @@ -281,13 +457,13 @@ void SchemaBuilder::applyDiff(const SchemaDiff & diff) { if (diff.type < SchemaActionType::MaxRecognizedType) { - LOG_INFO(log, "Ignore change type: {}", magic_enum::enum_name(diff.type)); + LOG_INFO(log, "Ignore change type: {}, diff_version={}", magic_enum::enum_name(diff.type), diff.version); } else { // >= SchemaActionType::MaxRecognizedType // log down the Int8 value directly - LOG_ERROR(log, "Unsupported change type: {}", static_cast(diff.type)); + LOG_ERROR(log, "Unsupported change type: {}, diff_version={}", static_cast(diff.type), diff.version); } break; @@ -301,7 +477,7 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa auto [db_info, table_info] = getter.getDatabaseAndTableInfo(database_id, table_id); if (unlikely(table_info == nullptr)) { - LOG_ERROR(log, "table is not exist in TiKV, table_id={}", table_id); + LOG_WARNING(log, "table is not exist in TiKV, applySetTiFlashReplica is ignored, table_id={}", table_id); return; } @@ -312,7 +488,7 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa auto storage = tmt_context.getStorages().get(keyspace_id, table_info->id); if (unlikely(storage == nullptr)) { - LOG_ERROR(log, "table is not exist in TiFlash, table_id={}", table_id); + LOG_ERROR(log, "table is not exist in TiFlash, applySetTiFlashReplica is ignored, table_id={}", table_id); return; } @@ -387,14 +563,14 @@ void SchemaBuilder::applyPartitionDiff(DatabaseID database_i auto [db_info, table_info] = getter.getDatabaseAndTableInfo(database_id, table_id); if (table_info == nullptr) { - LOG_ERROR(log, "table is not exist in TiKV, table_id={}", table_id); + LOG_ERROR(log, "table is not exist in TiKV, applyPartitionDiff is ignored, table_id={}", table_id); return; } if (!table_info->isLogicalPartitionTable()) { LOG_ERROR( log, - "new table in TiKV not partition table {} with database_id={}, table_id={}", + "new table in TiKV is not a partition table {}, database_id={} table_id={}", name_mapper.debugCanonicalName(*db_info, *table_info), db_info->id, table_info->id); @@ -457,8 +633,8 @@ void SchemaBuilder::applyPartitionDiff( LOG_INFO( log, "No partition changes, paritions_size={} {} with database_id={}, table_id={}", - name_mapper.debugCanonicalName(*db_info, *table_info), new_part_id_set.size(), + name_mapper.debugCanonicalName(*db_info, *table_info), db_info->id, table_info->id); return; @@ -510,21 +686,22 @@ void SchemaBuilder::applyRenameTable(DatabaseID database_id, auto [new_db_info, new_table_info] = getter.getDatabaseAndTableInfo(database_id, table_id); if (new_table_info == nullptr) { - LOG_ERROR(log, "table is not exist in TiKV, table_id={}", table_id); + LOG_ERROR(log, "table is not exist in TiKV, applyRenameTable is ignored, table_id={}", table_id); return; } + // update the table_id_map no matter storage instance is created or not + table_id_map.emplaceTableID(table_id, database_id); + auto & tmt_context = context.getTMTContext(); auto storage = tmt_context.getStorages().get(keyspace_id, table_id); if (storage == nullptr) { - LOG_ERROR(log, "table is not exist in TiFlash, table_id={}", table_id); + LOG_WARNING(log, "table is not exist in TiFlash, applyRenameTable is ignored, table_id={}", table_id); return; } applyRenameLogicalTable(new_db_info, new_table_info, storage); - - table_id_map.emplaceTableID(table_id, database_id); } template @@ -543,7 +720,10 @@ void SchemaBuilder::applyRenameLogicalTable( auto part_storage = tmt_context.getStorages().get(keyspace_id, part_def.id); if (part_storage == nullptr) { - LOG_ERROR(log, "table is not exist in TiFlash, physical_table_id={}", part_def.id); + LOG_ERROR( + log, + "table is not exist in TiFlash, applyRenamePhysicalTable is ignored, physical_table_id={}", + part_def.id); return; } auto part_table_info = new_table_info->producePartitionTableInfo(part_def.id, name_mapper); @@ -577,7 +757,7 @@ void SchemaBuilder::applyRenamePhysicalTable( GET_METRIC(tiflash_schema_internal_ddl_count, type_rename_table).Increment(); LOG_INFO( log, - "Renaming table {}.{} (display name: {}) to {} with database_id={}, table_id={}", + "Rename table {}.{} (display name: {}) to {} begin, database_id={} table_id={}", old_mapped_db_name, old_mapped_tbl_name, old_display_table_name, @@ -599,7 +779,7 @@ void SchemaBuilder::applyRenamePhysicalTable( LOG_INFO( log, - "Renamed table {}.{} (display name: {}) to {} with database_id={}, table_id={}", + "Rename table {}.{} (display name: {}) to {} end, database_id={} table_id={}", old_mapped_db_name, old_mapped_tbl_name, old_display_table_name, @@ -615,7 +795,10 @@ void SchemaBuilder::applyRecoverTable(DatabaseID database_id if (table_info == nullptr) { // this table is dropped. - LOG_DEBUG(log, "table is not exist in TiKV, may have been dropped, table_id={}", table_id); + LOG_INFO( + log, + "table is not exist in TiKV, may have been dropped, recover table is ignored, table_id={}", + table_id); return; } @@ -637,23 +820,30 @@ void SchemaBuilder::applyRecoverPhysicalTable( const TiDB::TableInfoPtr & table_info) { auto & tmt_context = context.getTMTContext(); - if (auto * storage = tmt_context.getStorages().get(keyspace_id, table_info->id).get(); storage) + auto storage = tmt_context.getStorages().get(keyspace_id, table_info->id); + if (storage == nullptr) + { + LOG_INFO( + log, + "Storage instance does not exist, tryRecoverPhysicalTable is ignored, table_id={}", + table_info->id); + } + else { if (!storage->isTombstone()) { - LOG_DEBUG( + LOG_INFO( log, - "Trying to recover table {} but it already exists and is not marked as tombstone, database_id={} " - "table_id={}", + "Trying to recover table {} but it is not marked as tombstone, skip, database_id={} table_id={}", name_mapper.debugCanonicalName(*db_info, *table_info), db_info->id, table_info->id); return; } - LOG_DEBUG( + LOG_INFO( log, - "Recovering table {} with database_id={}, table_id={}", + "Create table {} by recover begin, database_id={} table_id={}", name_mapper.debugCanonicalName(*db_info, *table_info), db_info->id, table_info->id); @@ -673,7 +863,7 @@ void SchemaBuilder::applyRecoverPhysicalTable( context); LOG_INFO( log, - "Created table {} with database_id={}, table_id={}", + "Create table {} by recover end, database_id={} table_id={}", name_mapper.debugCanonicalName(*db_info, *table_info), db_info->id, table_info->id); @@ -966,7 +1156,7 @@ void SchemaBuilder::applyCreateStorageInstance( LOG_INFO( log, - "Creating table {} (database_id={} table_id={}) with statement: {}", + "Create table {} (database_id={} table_id={}) with statement: {}", name_mapper.debugCanonicalName(*db_info, *table_info), db_info->id, table_info->id, @@ -1000,9 +1190,10 @@ void SchemaBuilder::applyDropPhysicalTable(const String & db auto storage = tmt_context.getStorages().get(keyspace_id, table_id); if (storage == nullptr) { - LOG_DEBUG(log, "table does not exist, table_id={}", table_id); + LOG_INFO(log, "Storage instance does not exist, applyDropPhysicalTable is ignored, table_id={}", table_id); return; } + GET_METRIC(tiflash_schema_internal_ddl_count, type_drop_table).Increment(); LOG_INFO( log, @@ -1038,10 +1229,10 @@ template void SchemaBuilder::applyDropTable(DatabaseID database_id, TableID table_id) { auto & tmt_context = context.getTMTContext(); - auto * storage = tmt_context.getStorages().get(keyspace_id, table_id).get(); + auto storage = tmt_context.getStorages().get(keyspace_id, table_id); if (storage == nullptr) { - LOG_DEBUG(log, "table does not exist, table_id={}", table_id); + LOG_INFO(log, "Storage instance does not exist, applyDropTable is ignored, table_id={}", table_id); return; } const auto & table_info = storage->getTableInfo(); @@ -1106,7 +1297,7 @@ void SchemaBuilder::syncAllSchema() created_db_set.emplace(name_mapper.mapDatabaseName(*db)); } - LOG_DEBUG( + LOG_INFO( log, "Database {} created during sync all schemas, database_id={}", name_mapper.debugDatabaseName(*db), @@ -1171,7 +1362,7 @@ void SchemaBuilder::syncAllSchema() if (!table_id_map.tableIDInTwoMaps(table_info.id)) { applyDropPhysicalTable(it->second->getDatabaseName(), table_info.id); - LOG_DEBUG( + LOG_INFO( log, "Table {}.{} dropped during sync all schemas, table_id={}", it->second->getDatabaseName(), @@ -1192,7 +1383,7 @@ void SchemaBuilder::syncAllSchema() if (created_db_set.count(it->first) == 0 && !isReservedDatabase(context, it->first)) { applyDropSchema(it->first); - LOG_DEBUG(log, "DB {} dropped during sync all schemas", it->first); + LOG_INFO(log, "Database {} dropped during sync all schemas", it->first); } } @@ -1200,45 +1391,58 @@ void SchemaBuilder::syncAllSchema() } template -void SchemaBuilder::applyTable( +bool SchemaBuilder::applyTable( DatabaseID database_id, TableID logical_table_id, TableID physical_table_id) { - auto table_info = getter.getTableInfo(database_id, logical_table_id); + // Here we get table info without mvcc. If the table has been renamed to another + // database, it will return false and the caller should update the table_id_map + // then retry. + auto table_info = getter.getTableInfo(database_id, logical_table_id, /*try_mvcc*/ false); if (table_info == nullptr) { - LOG_ERROR(log, "table is not exist in TiKV, database_id={} logical_table_id={}", database_id, logical_table_id); - return; + LOG_WARNING( + log, + "table is not exist in TiKV, applyTable need retry, database_id={} logical_table_id={}", + database_id, + logical_table_id); + return false; } + // For physical table of partition table if (logical_table_id != physical_table_id) { if (!table_info->isLogicalPartitionTable()) { - LOG_ERROR( + LOG_WARNING( log, - "new table in TiKV is not partition table {}, database_id={} table_id={}", + "new table info in TiKV is not partition table {}, applyTable need retry, database_id={} table_id={}", name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), database_id, table_info->id); - return; + return false; } + try { + // Try to produce table info by the logical table table_info = table_info->producePartitionTableInfo(physical_table_id, name_mapper); } catch (const Exception & e) { - /// when we do a ddl and insert, then we do reorganize partition. - /// Besides, reorganize reach tiflash before insert, so when insert, - /// the old partition_id is not exist, so we just ignore it. + // The following DDLs could change to mapping: + // - ALTER TABLE ... EXCHANGE PARTITION + // - ALTER TABLE ... PARTITION BY + // - ALTER TABLE ... REMOVE PARTITIONING + // If the physical_table does not belong to the logical table in the + // latest table info. It could now become a normal table or belong to another + // logical table now. The caller should update the table_id_map then retry. LOG_WARNING( log, - "producePartitionTableInfo meet exception : {} \n stack is {}", - e.displayText(), - e.getStackTrace().toString()); - return; + "producePartitionTableInfo meet exception, applyTable need retry, message={}", + e.message()); + return false; } } @@ -1249,30 +1453,38 @@ void SchemaBuilder::applyTable( auto db_info = getter.getDatabase(database_id); if (db_info == nullptr) { - LOG_ERROR(log, "database is not exist in TiKV, database_id={}", database_id); - return; + LOG_ERROR(log, "database is not exist in TiKV, applyTable need retry, database_id={}", database_id); + return false; } + // Create the instance with the latest table info applyCreateStorageInstance(db_info, table_info); + return true; } - else - { - LOG_INFO( - log, - "Altering table {}, database_id={} table_id={}", - name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), - database_id, - table_info->id); - GET_METRIC(tiflash_schema_internal_ddl_count, type_modify_column).Increment(); - auto alter_lock = storage->lockForAlter(getThreadNameAndID()); - storage->alterSchemaChange( - alter_lock, - *table_info, - name_mapper.mapDatabaseName(database_id, keyspace_id), - name_mapper.mapTableName(*table_info), - context); - } + // Alter the existing instance with the latest table info + LOG_INFO( + log, + "Alter table {} begin, database_id={} table_id={}", + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, + table_info->id); + GET_METRIC(tiflash_schema_internal_ddl_count, type_modify_column).Increment(); + auto alter_lock = storage->lockForAlter(getThreadNameAndID()); + storage->alterSchemaChange( + alter_lock, + *table_info, + name_mapper.mapDatabaseName(database_id, keyspace_id), + name_mapper.mapTableName(*table_info), + context); + + LOG_INFO( + log, + "Alter table {} end, database_id={} table_id={}", + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, + table_info->id); + return true; } template @@ -1292,7 +1504,7 @@ void SchemaBuilder::dropAllSchema() continue; } applyDropPhysicalTable(storage.second->getDatabaseName(), table_info.id); - LOG_DEBUG( + LOG_INFO( log, "Table {}.{} dropped during drop all schemas, table_id={}", storage.second->getDatabaseName(), @@ -1310,7 +1522,7 @@ void SchemaBuilder::dropAllSchema() continue; } applyDropSchema(db.first); - LOG_DEBUG(log, "DB {} dropped during drop all schemas", db.first); + LOG_INFO(log, "Database {} dropped during drop all schemas", db.first); } LOG_INFO(log, "Drop all schemas end"); diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h index dd9ea341b78..99fdeebfeb7 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -23,80 +23,48 @@ namespace DB /// TableIDMap use to store the mapping between table_id -> database_id and partition_id -> logical_table_id struct TableIDMap { - LoggerPtr & log; - std::shared_mutex shared_mutex_for_table_id_map; - std::unordered_map table_id_to_database_id; - std::unordered_map partition_id_to_logical_id; - - explicit TableIDMap(LoggerPtr & log_) + explicit TableIDMap(const LoggerPtr & log_) : log(log_) {} void erase(DB::TableID table_id) { - std::unique_lock lock(shared_mutex_for_table_id_map); + std::unique_lock lock(mtx_id_mapping); table_id_to_database_id.erase(table_id); partition_id_to_logical_id.erase(table_id); } void clear() { - std::unique_lock lock(shared_mutex_for_table_id_map); + std::unique_lock lock(mtx_id_mapping); table_id_to_database_id.clear(); partition_id_to_logical_id.clear(); } void emplaceTableID(TableID table_id, DatabaseID database_id) { - std::unique_lock lock(shared_mutex_for_table_id_map); - if (table_id_to_database_id.find(table_id) != table_id_to_database_id.end()) - { - LOG_WARNING(log, "table {} is already exists in table_id_to_database_id ", table_id); - table_id_to_database_id[table_id] = database_id; - } - else - table_id_to_database_id.emplace(table_id, database_id); + std::unique_lock lock(mtx_id_mapping); + doEmplaceTableID(table_id, database_id, "", lock); } void emplacePartitionTableID(TableID partition_id, TableID table_id) { - std::unique_lock lock(shared_mutex_for_table_id_map); - if (partition_id_to_logical_id.find(partition_id) != partition_id_to_logical_id.end()) - { - LOG_WARNING(log, "partition id {} is already exists in partition_id_to_logical_id ", partition_id); - partition_id_to_logical_id[partition_id] = table_id; - } - else - partition_id_to_logical_id.emplace(partition_id, table_id); - } - - void eraseTableIDOrLogError(TableID table_id) - { - std::unique_lock lock(shared_mutex_for_table_id_map); - if (table_id_to_database_id.find(table_id) != table_id_to_database_id.end()) - table_id_to_database_id.erase(table_id); - else - LOG_ERROR(log, "table_id {} not in table_id_to_database_id", table_id); + std::unique_lock lock(mtx_id_mapping); + doEmplacePartitionTableID(partition_id, table_id, "", lock); } - void erasePartitionTableIDOrLogError(TableID table_id) - { - std::unique_lock lock(shared_mutex_for_table_id_map); - if (partition_id_to_logical_id.find(table_id) != partition_id_to_logical_id.end()) - { - partition_id_to_logical_id.erase(table_id); - } - else - { - LOG_ERROR(log, "table_id {} not in partition_id_to_logical_id", table_id); - } - } + void exchangeTablePartition( + DatabaseID non_partition_database_id, + TableID non_partition_table_id, + DatabaseID partition_database_id, + TableID partition_logical_table_id, + TableID partition_physical_table_id); - std::vector findTablesByDatabaseID(DatabaseID database_id) + std::vector findTablesByDatabaseID(DatabaseID database_id) const { - std::shared_lock lock(shared_mutex_for_table_id_map); + std::shared_lock lock(mtx_id_mapping); std::vector tables; - for (auto & table_id : table_id_to_database_id) + for (const auto & table_id : table_id_to_database_id) { if (table_id.second == database_id) { @@ -106,24 +74,24 @@ struct TableIDMap return tables; } - bool tableIDInTwoMaps(TableID table_id) + bool tableIDInTwoMaps(TableID table_id) const { - std::shared_lock lock(shared_mutex_for_table_id_map); + std::shared_lock lock(mtx_id_mapping); return !( table_id_to_database_id.find(table_id) == table_id_to_database_id.end() && partition_id_to_logical_id.find(table_id) == partition_id_to_logical_id.end()); } - bool tableIDInDatabaseIdMap(TableID table_id) + bool tableIDInDatabaseIdMap(TableID table_id) const { - std::shared_lock lock(shared_mutex_for_table_id_map); + std::shared_lock lock(mtx_id_mapping); return !(table_id_to_database_id.find(table_id) == table_id_to_database_id.end()); } // if not find,than return -1 - DatabaseID findTableIDInDatabaseMap(TableID table_id) + DatabaseID findTableIDInDatabaseMap(TableID table_id) const { - std::shared_lock lock(shared_mutex_for_table_id_map); + std::shared_lock lock(mtx_id_mapping); auto database_iter = table_id_to_database_id.find(table_id); if (database_iter == table_id_to_database_id.end()) return -1; @@ -132,15 +100,35 @@ struct TableIDMap } // if not find,than return -1 - TableID findTableIDInPartitionMap(TableID partition_id) + TableID findTableIDInPartitionMap(TableID partition_id) const { - std::shared_lock lock(shared_mutex_for_table_id_map); + std::shared_lock lock(mtx_id_mapping); auto logical_table_iter = partition_id_to_logical_id.find(partition_id); if (logical_table_iter == partition_id_to_logical_id.end()) return -1; return logical_table_iter->second; } + + std::tuple findDatabaseIDAndLogicalTableID(TableID physical_table_id) const; + +private: + void doEmplaceTableID( + TableID table_id, + DatabaseID database_id, + std::string_view log_prefix, + const std::unique_lock &); + void doEmplacePartitionTableID( + TableID partition_id, + TableID table_id, + std::string_view log_prefix, + const std::unique_lock &); + +private: + LoggerPtr log; + mutable std::shared_mutex mtx_id_mapping; + std::unordered_map table_id_to_database_id; + std::unordered_map partition_id_to_logical_id; }; template @@ -183,7 +171,7 @@ struct SchemaBuilder void dropAllSchema(); - void applyTable(DatabaseID database_id, TableID logical_table_id, TableID physical_table_id); + bool applyTable(DatabaseID database_id, TableID logical_table_id, TableID physical_table_id); private: void applyDropSchema(DatabaseID schema_id); diff --git a/dbms/src/TiDB/Schema/SchemaGetter.cpp b/dbms/src/TiDB/Schema/SchemaGetter.cpp index 5e170dd7fd6..56b751db6c6 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.cpp +++ b/dbms/src/TiDB/Schema/SchemaGetter.cpp @@ -275,12 +275,13 @@ TiDB::DBInfoPtr SchemaGetter::getDatabase(DatabaseID db_id) if (json.empty()) return nullptr; - LOG_DEBUG(log, "Get DB Info from TiKV: {}", json); + LOG_DEBUG(log, "Get DB Info from TiKV, database_id={} {}", db_id, json); auto db_info = std::make_shared(json, keyspace_id); return db_info; } -TiDB::TableInfoPtr SchemaGetter::getTableInfo(DatabaseID db_id, TableID table_id) +template +TiDB::TableInfoPtr SchemaGetter::getTableInfoImpl(DatabaseID db_id, TableID table_id) { String db_key = getDBKey(db_id); if (!checkDBExists(db_key)) @@ -292,6 +293,11 @@ TiDB::TableInfoPtr SchemaGetter::getTableInfo(DatabaseID db_id, TableID table_id String table_info_json = TxnStructure::hGet(snap, db_key, table_key); if (table_info_json.empty()) { + if constexpr (!mvcc_get) + { + return nullptr; + } + LOG_WARNING(log, "The table is dropped in TiKV, try to get the latest table_info, table_id={}", table_id); table_info_json = TxnStructure::mvccGet(snap, db_key, table_key); if (table_info_json.empty()) @@ -304,11 +310,11 @@ TiDB::TableInfoPtr SchemaGetter::getTableInfo(DatabaseID db_id, TableID table_id return nullptr; } } - LOG_DEBUG(log, "Get Table Info from TiKV: {}", table_info_json); - TiDB::TableInfoPtr table_info = std::make_shared(table_info_json, keyspace_id); - - return table_info; + LOG_DEBUG(log, "Get Table Info from TiKV, table_id={} {}", table_id, table_info_json); + return std::make_shared(table_info_json, keyspace_id); } +template TiDB::TableInfoPtr SchemaGetter::getTableInfoImpl(DatabaseID db_id, TableID table_id); +template TiDB::TableInfoPtr SchemaGetter::getTableInfoImpl(DatabaseID db_id, TableID table_id); std::tuple SchemaGetter::getDatabaseAndTableInfo( DatabaseID db_id, @@ -339,7 +345,7 @@ std::tuple SchemaGetter::getDatabaseAndTabl return std::make_tuple(db_info, nullptr); } } - LOG_DEBUG(log, "Get Table Info from TiKV: {}", table_info_json); + LOG_DEBUG(log, "Get Table Info from TiKV, table_id={} {}", table_id, table_info_json); TiDB::TableInfoPtr table_info = std::make_shared(table_info_json, keyspace_id); return std::make_tuple(db_info, table_info); diff --git a/dbms/src/TiDB/Schema/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h index dd9fa754217..751b25a4984 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -164,7 +164,12 @@ struct SchemaGetter TiDB::DBInfoPtr getDatabase(DatabaseID db_id); - TiDB::TableInfoPtr getTableInfo(DatabaseID db_id, TableID table_id); + TiDB::TableInfoPtr getTableInfo(DatabaseID db_id, TableID table_id, bool try_mvcc = true) + { + if (try_mvcc) + return getTableInfoImpl(db_id, table_id); + return getTableInfoImpl(db_id, table_id); + } std::tuple getDatabaseAndTableInfo(DatabaseID db_id, TableID table_id); @@ -173,6 +178,10 @@ struct SchemaGetter std::vector listTables(DatabaseID db_id); KeyspaceID getKeyspaceID() const { return keyspace_id; } + +private: + template + TiDB::TableInfoPtr getTableInfoImpl(DatabaseID db_id, TableID table_id); }; } // namespace DB diff --git a/dbms/src/TiDB/Schema/TiDB.cpp b/dbms/src/TiDB/Schema/TiDB.cpp index 8b867037627..d88be9c6263 100644 --- a/dbms/src/TiDB/Schema/TiDB.cpp +++ b/dbms/src/TiDB/Schema/TiDB.cpp @@ -29,6 +29,7 @@ #include #include +#include #include namespace DB @@ -1157,9 +1158,10 @@ TableInfoPtr TableInfo::producePartitionTableInfo(TableID table_or_partition_id, // Some sanity checks for partition table. if (unlikely(!(is_partition_table && partition.enable))) throw Exception( - "Table ID " + std::to_string(id) + " seeing partition ID " + std::to_string(table_or_partition_id) - + " but it's not a partition table", - DB::ErrorCodes::LOGICAL_ERROR); + DB::ErrorCodes::LOGICAL_ERROR, + "Try to produce partition table on a non-partition table, table_id={} partition_table_id={}", + id, + table_or_partition_id); if (unlikely( std::find_if( @@ -1167,10 +1169,20 @@ TableInfoPtr TableInfo::producePartitionTableInfo(TableID table_or_partition_id, partition.definitions.end(), [table_or_partition_id](const auto & d) { return d.id == table_or_partition_id; }) == partition.definitions.end())) + { + std::vector part_ids; + std::for_each( + partition.definitions.begin(), + partition.definitions.end(), + [&part_ids](const PartitionDefinition & part) { part_ids.emplace_back(part.id); }); throw Exception( - "Couldn't find partition with ID " + std::to_string(table_or_partition_id) + " in table ID " - + std::to_string(id), - DB::ErrorCodes::LOGICAL_ERROR); + DB::ErrorCodes::LOGICAL_ERROR, + "Can not find partition id in partition table, partition_table_id={} logical_table_id={} " + "available_ids={}", + table_or_partition_id, + id, + part_ids); + } // This is a TiDB partition table, adjust the table ID by making it to physical table ID (partition ID). auto new_table = std::make_shared(); diff --git a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.cpp b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.cpp index fa5d2ec9590..f514cfd6fda 100644 --- a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.cpp +++ b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.cpp @@ -23,20 +23,25 @@ namespace DB { template bool TiDBSchemaSyncer::syncSchemas(Context & context) +{ + auto getter = createSchemaGetter(keyspace_id); + return syncSchemasByGetter(context, getter); +} + +template +bool TiDBSchemaSyncer::syncSchemasByGetter(Context & context, Getter & getter) { std::lock_guard lock(mutex_for_sync_schema); GET_METRIC(tiflash_sync_schema_applying).Increment(); - auto getter = createSchemaGetter(keyspace_id); - const Int64 version = getter.getVersion(); - Stopwatch watch; SCOPE_EXIT({ GET_METRIC(tiflash_schema_apply_duration_seconds).Observe(watch.elapsedSeconds()); GET_METRIC(tiflash_sync_schema_applying).Decrement(); }); + const Int64 version = getter.getVersion(); if (version == SchemaGetter::SchemaVersionNotExist) { // Tables and databases are already tombstoned and waiting for GC. @@ -162,7 +167,7 @@ std::tuple TiDBSchemaSyncer database_id = table_id_map.findTableIDInDatabaseMap(logical_table_id); } - if (database_id != -1 and logical_table_id != -1) + if (database_id != -1 && logical_table_id != -1) { return std::make_tuple(true, database_id, logical_table_id); } @@ -170,9 +175,45 @@ std::tuple TiDBSchemaSyncer return std::make_tuple(false, 0, 0); } +template +std::tuple TiDBSchemaSyncer::trySyncTableSchema( + Context & context, + TableID physical_table_id, + Getter & getter, + const char * next_action) +{ + // Get logical_table_id and database_id by physical_table_id. + // If the table is a partition table, logical_table_id != physical_table_id, otherwise, logical_table_id == physical_table_id; + auto [found, database_id, logical_table_id] = findDatabaseIDAndTableID(physical_table_id); + if (!found) + { + String message = fmt::format( + "Can not find related database_id and logical_table_id from table_id_map, {}." + " physical_table_id={}", + next_action, + physical_table_id); + return {true, message}; + } + + // Try to fetch the latest table info from TiKV. + // If the table schema apply is failed, then we need to update the table-id-mapping + // and retry. + SchemaBuilder builder(getter, context, databases, table_id_map, shared_mutex_for_databases); + if (!builder.applyTable(database_id, logical_table_id, physical_table_id)) + { + String message = fmt::format( + "Can not apply table schema because the table_id_map is not up-to-date, {}." + " physical_table_id={} database_id={} logical_table_id={}", + next_action, + physical_table_id, + database_id, + logical_table_id); + return {true, message}; + } + // apply is done successfully + return {false, ""}; +} -/// we don't need a lock at the beginning of syncTableSchema, -/// we will catch the AlterLock for storage later. template bool TiDBSchemaSyncer::syncTableSchema(Context & context, TableID physical_table_id) { @@ -182,36 +223,34 @@ bool TiDBSchemaSyncer::syncTableSchema(Context & conte .Observe(watch.elapsedSeconds()); }); - LOG_INFO(log, "Start sync table schema, table_id={}", physical_table_id); - auto getter = createSchemaGetter(keyspace_id); + LOG_INFO(log, "Sync table schema begin, table_id={}", physical_table_id); + auto getter = createSchemaGetter(keyspace_id); // use the same tso for getting schema - // get logical_table_id and database_id based on physical_table_id, - // if the table is a partition table, logical_table_id != physical_table_id, otherwise, logical_table_id = physical_table_id; - auto [find, database_id, logical_table_id] = findDatabaseIDAndTableID(physical_table_id); - if (!find) + /// Note that we don't need a lock at the beginning of syncTableSchema. + /// The AlterLock for storage will be acquired in `SchemaBuilder::applyTable`. + auto [need_update_id_mapping, message] + = trySyncTableSchema(context, physical_table_id, getter, "try to syncSchemas"); + if (!need_update_id_mapping) { - LOG_WARNING( - log, - "Can't find related database_id and logical_table_id from table_id_map, try to syncSchemas. " - "physical_table_id={}", - physical_table_id); - GET_METRIC(tiflash_schema_trigger_count, type_sync_table_schema).Increment(); - syncSchemas(context); - std::tie(find, database_id, logical_table_id) = findDatabaseIDAndTableID(physical_table_id); - if (!find) - { - LOG_ERROR( - log, - "Still can't find related database_id and logical_table_id from table_id_map, physical_table_id={}", - physical_table_id); - return false; - } + LOG_INFO(log, "Sync table schema end, table_id={}", physical_table_id); + return true; } - SchemaBuilder builder(getter, context, databases, table_id_map, shared_mutex_for_databases); - builder.applyTable(database_id, logical_table_id, physical_table_id); + LOG_WARNING(log, message); + GET_METRIC(tiflash_schema_trigger_count, type_sync_table_schema).Increment(); + // Notice: must use the same getter + syncSchemasByGetter(context, getter); + std::tie(need_update_id_mapping, message) + = trySyncTableSchema(context, physical_table_id, getter, "sync table schema fail"); + if (likely(!need_update_id_mapping)) + { + LOG_INFO(log, "Sync table schema end after syncSchemas, table_id={}", physical_table_id); + return true; + } - return true; + // Still fail, maybe some unknown bugs? + LOG_ERROR(log, message); + return false; } template class TiDBSchemaSyncer; diff --git a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h index 43ced6164c8..f5ccadda369 100644 --- a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h +++ b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h @@ -105,6 +105,13 @@ class TiDBSchemaSyncer : public SchemaSyncer Int64 syncSchemaDiffs(Context & context, Getter & getter, Int64 latest_version); Int64 syncAllSchemas(Context & context, Getter & getter, Int64 version); + bool syncSchemasByGetter(Context & context, Getter & getter); + std::tuple trySyncTableSchema( + Context & context, + TableID physical_table_id, + Getter & getter, + const char * next_action); + TiDB::DBInfoPtr getDBInfoByName(const String & database_name) override { std::shared_lock lock(shared_mutex_for_databases); diff --git a/dbms/src/TiDB/Schema/tests/gtest_table_id_map.cpp b/dbms/src/TiDB/Schema/tests/gtest_table_id_map.cpp new file mode 100644 index 00000000000..eec6dedc1e2 --- /dev/null +++ b/dbms/src/TiDB/Schema/tests/gtest_table_id_map.cpp @@ -0,0 +1,166 @@ +// Copyright 2023 PingCAP, Inc. +// +// 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::tests +{ +::testing::AssertionResult isSameMapping( + const char * lhs_expr, + const char * rhs_expr, + std::tuple lhs, + std::tuple rhs) +{ + if (std::get<0>(lhs) == std::get<0>(rhs)) + return ::testing::AssertionSuccess(); + return ::testing::internal::EqFailure( + lhs_expr, + rhs_expr, + fmt::format("", std::get<0>(lhs), std::get<1>(lhs), std::get<2>(lhs)), + fmt::format("", std::get<0>(rhs), std::get<1>(rhs), std::get<2>(rhs)), + false); +} + +#define ASSERT_MAPPING_EQ(val1, val2) ASSERT_PRED_FORMAT2(isSameMapping, val1, val2) + +class TableIDMapTest : public ::testing::Test +{ +public: + TableIDMapTest() + : log(Logger::get()) + {} + + +protected: + LoggerPtr log; +}; + +TEST_F(TableIDMapTest, Basic) +{ + TableIDMap mapping(log); + + // partition table + TableID partition_logical_table_id = 100; + mapping.emplaceTableID(partition_logical_table_id, 2); + mapping.emplacePartitionTableID(101, 100); + mapping.emplacePartitionTableID(102, 100); + mapping.emplacePartitionTableID(103, 100); + // non-partition table + TableID non_partition_table_id = 200; + mapping.emplaceTableID(non_partition_table_id, 2); + + ASSERT_EQ(mapping.findTableIDInDatabaseMap(partition_logical_table_id), 2); + ASSERT_EQ(mapping.findTableIDInPartitionMap(101), partition_logical_table_id); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(101), -1); + ASSERT_EQ(mapping.findTableIDInPartitionMap(non_partition_table_id), -1); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(non_partition_table_id), 2); + + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(100)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(101)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(102)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(103)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 200), mapping.findDatabaseIDAndLogicalTableID(200)); + + // non exist + ASSERT_MAPPING_EQ(std::make_tuple(false, 0, 0), mapping.findDatabaseIDAndLogicalTableID(900)); + // broken state, physical_table_id -> logical_table_id, but no logical_table_id -> database_id + mapping.emplacePartitionTableID(901, 900); + ASSERT_MAPPING_EQ(std::make_tuple(false, 0, 0), mapping.findDatabaseIDAndLogicalTableID(901)); +} + +TEST_F(TableIDMapTest, ExchangePartition) +{ + TableIDMap mapping(log); + + /// Prepare + // partition table + TableID partition_logical_table_id = 100; + mapping.emplaceTableID(partition_logical_table_id, 2); + mapping.emplacePartitionTableID(101, 100); + mapping.emplacePartitionTableID(102, 100); + mapping.emplacePartitionTableID(103, 100); + // non-partition table + TableID non_partition_table_id = 200; + mapping.emplaceTableID(non_partition_table_id, 2); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(partition_logical_table_id), 2); + ASSERT_EQ(mapping.findTableIDInPartitionMap(101), partition_logical_table_id); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(101), -1); + ASSERT_EQ(mapping.findTableIDInPartitionMap(non_partition_table_id), -1); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(non_partition_table_id), 2); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(100)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(101)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(102)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(103)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 200), mapping.findDatabaseIDAndLogicalTableID(200)); + + // exchange + mapping.exchangeTablePartition(2, non_partition_table_id, 2, partition_logical_table_id, 101); + + ASSERT_EQ(mapping.findTableIDInDatabaseMap(partition_logical_table_id), 2); + ASSERT_EQ(mapping.findTableIDInPartitionMap(101), -1); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(101), 2); + ASSERT_EQ(mapping.findTableIDInPartitionMap(non_partition_table_id), partition_logical_table_id); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(non_partition_table_id), -1); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(100)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 101), mapping.findDatabaseIDAndLogicalTableID(101)); // changed + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(102)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(103)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(200)); // changed +} + +TEST_F(TableIDMapTest, ExchangePartitionCrossDatabase) +{ + TableIDMap mapping(log); + + /// Prepare + // partition table + TableID partition_logical_table_id = 100; + mapping.emplaceTableID(partition_logical_table_id, 2); + mapping.emplacePartitionTableID(101, 100); + mapping.emplacePartitionTableID(102, 100); + mapping.emplacePartitionTableID(103, 100); + // non-partition table + TableID non_partition_table_id = 200; + mapping.emplaceTableID(non_partition_table_id, 7); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(partition_logical_table_id), 2); + ASSERT_EQ(mapping.findTableIDInPartitionMap(101), partition_logical_table_id); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(101), -1); + ASSERT_EQ(mapping.findTableIDInPartitionMap(non_partition_table_id), -1); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(non_partition_table_id), 7); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(100)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(101)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(102)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(103)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 7, 200), mapping.findDatabaseIDAndLogicalTableID(200)); + + // exchange + mapping.exchangeTablePartition(7, non_partition_table_id, 2, partition_logical_table_id, 101); + + ASSERT_EQ(mapping.findTableIDInDatabaseMap(partition_logical_table_id), 2); + ASSERT_EQ(mapping.findTableIDInPartitionMap(101), -1); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(101), 7); + ASSERT_EQ(mapping.findTableIDInPartitionMap(non_partition_table_id), partition_logical_table_id); + ASSERT_EQ(mapping.findTableIDInDatabaseMap(non_partition_table_id), -1); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(100)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 7, 101), mapping.findDatabaseIDAndLogicalTableID(101)); // changed + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(102)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(103)); + ASSERT_MAPPING_EQ(std::make_tuple(true, 2, 100), mapping.findDatabaseIDAndLogicalTableID(200)); // changed +} + +} // namespace DB::tests diff --git a/tests/fullstack-test2/ddl/alter_exchange_partition.test b/tests/fullstack-test2/ddl/alter_exchange_partition.test index ccbef2af7b8..8854c401e7e 100644 --- a/tests/fullstack-test2/ddl/alter_exchange_partition.test +++ b/tests/fullstack-test2/ddl/alter_exchange_partition.test @@ -83,6 +83,105 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test_new mysql> alter table test.e drop column c1; >> DBGInvoke __refresh_table_schema(test, e) +# cleanup +mysql> drop table if exists test.e; +mysql> drop table if exists test.e2; +mysql> drop table if exists test_new.e2; +mysql> drop database if exists test_new; + +# case 11, create non-partition table and execute exchagne partition immediately +mysql> create table test.e(id INT NOT NULL,fname VARCHAR(30),lname VARCHAR(30)) PARTITION BY RANGE (id) ( PARTITION p0 VALUES LESS THAN (50),PARTITION p1 VALUES LESS THAN (100),PARTITION p2 VALUES LESS THAN (150), PARTITION p3 VALUES LESS THAN (MAXVALUE)); +mysql> insert into test.e values (1, 'a', 'b'),(108, 'a', 'b'); +# sync the partition table to tiflash +>> DBGInvoke __refresh_schemas() + +mysql> create table test.e2(id int not null,fname varchar(30),lname varchar(30)); +>> DBGInvoke __refresh_schemas() +mysql> insert into test.e2 values (2, 'a', 'b'); +mysql> set @@tidb_enable_exchange_partition=1; alter table test.e exchange partition p0 with table test.e2 + +mysql> alter table test.e set tiflash replica 1; +mysql> alter table test.e2 set tiflash replica 1; +func> wait_table test e e2 +>> DBGInvoke __refresh_schemas() +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 2 | a | b | +| 108 | a | b | ++-----+-------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 1 | a | b | ++-----+-------+-------+ + +# ensure the swap out table is not mark as tombstone +>> DBGInvoke __enable_schema_sync_service('true') +>> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __enable_schema_sync_service('false') +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 2 | a | b | +| 108 | a | b | ++-----+-------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 1 | a | b | ++-----+-------+-------+ + +# case 12, create partition table, non-partition table and execute exchagne partition immediately +mysql> drop table if exists test.e +mysql> drop table if exists test.e2 +mysql> create table test.e(id INT NOT NULL,fname VARCHAR(30),lname VARCHAR(30)) PARTITION BY RANGE (id) ( PARTITION p0 VALUES LESS THAN (50),PARTITION p1 VALUES LESS THAN (100),PARTITION p2 VALUES LESS THAN (150), PARTITION p3 VALUES LESS THAN (MAXVALUE)); +mysql> insert into test.e values (1, 'a', 'b'),(108, 'a', 'b'); +mysql> create table test.e2(id int not null,fname varchar(30),lname varchar(30)); +mysql> insert into test.e2 values (2, 'a', 'b'); +mysql> set @@tidb_enable_exchange_partition=1; alter table test.e exchange partition p0 with table test.e2 + +mysql> alter table test.e set tiflash replica 1; +mysql> alter table test.e2 set tiflash replica 1; +func> wait_table test e e2 +# tiflash the final result +>> DBGInvoke __refresh_schemas() +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 2 | a | b | +| 108 | a | b | ++-----+-------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 1 | a | b | ++-----+-------+-------+ +# ensure the swap out table is not mark as tombstone +>> DBGInvoke __enable_schema_sync_service('true') +>> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __enable_schema_sync_service('false') +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 2 | a | b | +| 108 | a | b | ++-----+-------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 order by id; ++-----+-------+-------+ +| id | fname | lname | ++-----+-------+-------+ +| 1 | a | b | ++-----+-------+-------+ + +# cleanup mysql> drop table if exists test.e; mysql> drop table if exists test.e2; mysql> drop table if exists test_new.e2; diff --git a/tests/fullstack-test2/ddl/rename_table_across_databases.test b/tests/fullstack-test2/ddl/rename_table_across_databases.test index b3ce9630ae2..62d44a3351e 100644 --- a/tests/fullstack-test2/ddl/rename_table_across_databases.test +++ b/tests/fullstack-test2/ddl/rename_table_across_databases.test @@ -63,3 +63,45 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test_new mysql> drop table if exists test.t; mysql> drop table if exists test_new.t2; mysql> drop database if exists test_new; + +# rename table across database case 2 +mysql> create database if not exists test +mysql> create database if not exists test_new +# (required) stop regular schema sync +=> DBGInvoke __enable_schema_sync_service('false') + +mysql> create table test.t(a int, b int); +mysql> insert into test.t values (1, 1); insert into test.t values (1, 2); +# (required) sync table id mapping to tiflash +=> DBGInvoke __refresh_schemas() +mysql> rename table test.t to test_new.t2; +mysql> alter table test_new.t2 set tiflash replica 1; +# new snapshot sync to tiflash, but the table id mapping is not updated +func> wait_table test_new t2 +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test_new.t2; ++------+------+ +| a | b | ++------+------+ +| 1 | 1 | +| 1 | 2 | ++------+------+ + +mysql> drop table if exists test.t; +mysql> drop table if exists test_new.t2; +mysql> drop database if exists test_new; + +# (required) create a new table and sync to tiflash, check whether it can apply +mysql> drop table if exists test.t3; +mysql> create table test.t3(c int, d int); +mysql> insert into test.t3 values (3,3),(3,4); +mysql> alter table test.t3 set tiflash replica 1; +func> wait_table test t3 +mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t3; ++------+------+ +| c | d | ++------+------+ +| 3 | 3 | +| 3 | 4 | ++------+------+ + +mysql> drop table if exists test.t3; From 2f4d38c23bdc83bc07ffda000d5a339dbcc3c78c Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 22 Nov 2023 21:50:42 +0800 Subject: [PATCH 4/6] Cache probe column when key has many entries (#8320) close pingcap/tiflash#8295 --- dbms/src/Columns/IColumn.h | 6 + .../HashJoinProbeBlockInputStream.cpp | 2 +- dbms/src/DataStreams/HashJoinProbeExec.cpp | 2 +- .../ScanHashMapAfterProbeBlockInputStream.cpp | 59 ++- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 6 +- dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp | 6 +- dbms/src/Flash/tests/gtest_join.h | 17 +- dbms/src/Flash/tests/gtest_join_executor.cpp | 124 +++++- dbms/src/Interpreters/ExpressionActions.cpp | 2 +- dbms/src/Interpreters/Join.cpp | 78 ++-- dbms/src/Interpreters/Join.h | 33 +- dbms/src/Interpreters/JoinHashMap.h | 43 +- dbms/src/Interpreters/JoinPartition.cpp | 409 +++++++++++++----- dbms/src/Interpreters/JoinPartition.h | 31 +- dbms/src/Interpreters/ProbeProcessInfo.h | 6 +- dbms/src/Interpreters/Settings.h | 5 +- .../Operators/HashJoinProbeTransformOp.cpp | 4 +- 17 files changed, 621 insertions(+), 212 deletions(-) diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 4b21af115b1..42af37ac33f 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -390,6 +390,12 @@ class IColumn : public COWPtr return res; } + MutablePtr cloneFullColumn() const + { + MutablePtr res = clone(); + res->forEachSubcolumn([](Ptr & subcolumn) { subcolumn = subcolumn->clone(); }); + return res; + } /** Some columns can contain another columns inside. * So, we have a tree of columns. But not all combinations are possible. diff --git a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp index 47c59e09774..46b3f3e0dc4 100644 --- a/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp +++ b/dbms/src/DataStreams/HashJoinProbeBlockInputStream.cpp @@ -37,7 +37,7 @@ HashJoinProbeBlockInputStream::HashJoinProbeBlockInputStream( probe_exec.set(HashJoinProbeExec::build(req_id, original_join, stream_index, input, max_block_size_)); probe_exec->setCancellationHook([&]() { return isCancelledOrThrowIfKilled(); }); - ProbeProcessInfo header_probe_process_info(0); + ProbeProcessInfo header_probe_process_info(0, 0); header_probe_process_info.resetBlock(input->getHeader()); header = original_join->joinBlock(header_probe_process_info, true); } diff --git a/dbms/src/DataStreams/HashJoinProbeExec.cpp b/dbms/src/DataStreams/HashJoinProbeExec.cpp index 3b54b2b1ffe..ffd7be1a396 100644 --- a/dbms/src/DataStreams/HashJoinProbeExec.cpp +++ b/dbms/src/DataStreams/HashJoinProbeExec.cpp @@ -63,7 +63,7 @@ HashJoinProbeExec::HashJoinProbeExec( , need_scan_hash_map_after_probe(need_scan_hash_map_after_probe_) , scan_hash_map_after_probe_stream(scan_hash_map_after_probe_stream_) , max_block_size(max_block_size_) - , probe_process_info(max_block_size_) + , probe_process_info(max_block_size_, join->getProbeCacheColumnThreshold()) {} void HashJoinProbeExec::waitUntilAllBuildFinished() diff --git a/dbms/src/DataStreams/ScanHashMapAfterProbeBlockInputStream.cpp b/dbms/src/DataStreams/ScanHashMapAfterProbeBlockInputStream.cpp index 4bef83420ab..f818fd8b1f4 100644 --- a/dbms/src/DataStreams/ScanHashMapAfterProbeBlockInputStream.cpp +++ b/dbms/src/DataStreams/ScanHashMapAfterProbeBlockInputStream.cpp @@ -42,6 +42,7 @@ struct AdderMapEntry size_t num_columns_right, MutableColumns & columns_right, const void *&, + size_t, const size_t) { for (size_t j = 0; j < num_columns_left; ++j) @@ -68,20 +69,40 @@ struct AdderMapEntry size_t num_columns_right, MutableColumns & columns_right, const void *& next_element_in_row_list, + size_t probe_cached_rows_threshold, const size_t max_row_added) { size_t rows_added = 0; - auto current = &static_cast(mapped); - if unlikely (next_element_in_row_list != nullptr) - current = reinterpret_cast(next_element_in_row_list); - for (; rows_added < max_row_added && current != nullptr; current = current->next) - { + assert(rows_added < max_row_added); + const auto * current = &static_cast(mapped); + + auto add_one_row = [&]() { /// handle left columns later to utilize insertManyDefaults for (size_t j = 0; j < num_columns_right; ++j) columns_right[j]->insertFrom( *current->block->getByPosition(key_num + j).column.get(), current->row_num); ++rows_added; + }; + if unlikely (next_element_in_row_list != nullptr) + { + current = reinterpret_cast(next_element_in_row_list); + } + else + { + add_one_row(); + if unlikely (probe_cached_rows_threshold > 0 && current->list_length >= probe_cached_rows_threshold) + { + current = reinterpret_cast(current->cached_column_info->next); + } + else + { + current = current->next; + } + } + for (; rows_added < max_row_added && current != nullptr; current = current->next) + { + add_one_row(); } for (size_t j = 0; j < num_columns_left; ++j) /// should fill the key column with key columns from right block @@ -106,14 +127,14 @@ struct AdderRowFlaggedMapEntry size_t num_columns_right, MutableColumns & columns_right, const void *& next_element_in_row_list, + size_t probe_cached_rows_threshold, const size_t max_row_added) { size_t rows_added = 0; + assert(rows_added < max_row_added); const auto * current = &static_cast(mapped); - if unlikely (next_element_in_row_list != nullptr) - current = reinterpret_cast(next_element_in_row_list); - for (; rows_added < max_row_added && current != nullptr; current = current->next) - { + + auto check_and_add_one_row = [&]() { bool flag = current->getUsed(); if constexpr (!add_joined) flag = !flag; @@ -126,6 +147,24 @@ struct AdderRowFlaggedMapEntry current->row_num); ++rows_added; } + }; + if unlikely (next_element_in_row_list != nullptr) + current = reinterpret_cast(next_element_in_row_list); + else + { + check_and_add_one_row(); + if unlikely (probe_cached_rows_threshold > 0 && current->list_length >= probe_cached_rows_threshold) + { + current = reinterpret_cast(current->cached_column_info->next); + } + else + { + current = current->next; + } + } + for (; rows_added < max_row_added && current != nullptr; current = current->next) + { + check_and_add_one_row(); } for (size_t j = 0; j < num_columns_left; ++j) /// should fill the key column with key columns from right block @@ -488,6 +527,7 @@ void ScanHashMapAfterProbeBlockInputStream::fillColumns( num_columns_right, mutable_columns_right, next_element_in_row_list, + parent.probe_cache_column_threshold, row_count_info.availableRowCount())); else { @@ -508,6 +548,7 @@ void ScanHashMapAfterProbeBlockInputStream::fillColumns( num_columns_right, mutable_columns_right, next_element_in_row_list, + parent.probe_cache_column_threshold, row_count_info.availableRowCount())); } assert(row_count_info.getCurrentRows() <= max_block_size); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 596ebdade33..7da090989f3 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -332,12 +332,11 @@ void DAGQueryBlockInterpreter::handleJoin( build_key_names, tiflash_join.kind, log->identifier(), - enableFineGrainedShuffle(fine_grained_shuffle_count), fine_grained_shuffle_count, settings.max_bytes_before_external_join, build_spill_config, probe_spill_config, - settings.join_restore_concurrency, + RestoreConfig{settings.join_restore_concurrency, 0, 0}, join_output_column_names, [&](const OperatorSpillContextPtr & operator_spill_context) { if (context.getDAGContext() != nullptr) @@ -352,8 +351,7 @@ void DAGQueryBlockInterpreter::handleJoin( settings.shallow_copy_cross_probe_threshold, match_helper_name, flag_mapped_entry_helper_name, - 0, - 0, + settings.join_probe_cache_columns_threshold, context.isTest()); recordJoinExecuteInfo(tiflash_join.build_side_index, join_ptr); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp index d1b9b1c5f72..f07ddc499ff 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp @@ -170,12 +170,11 @@ PhysicalPlanNodePtr PhysicalJoin::build( build_key_names, tiflash_join.kind, join_req_id, - fine_grained_shuffle.enable(), fine_grained_shuffle.stream_count, max_bytes_before_external_join, build_spill_config, probe_spill_config, - settings.join_restore_concurrency, + RestoreConfig{settings.join_restore_concurrency, 0, 0}, join_output_column_names, [&](const OperatorSpillContextPtr & operator_spill_context) { if (context.getDAGContext() != nullptr) @@ -190,8 +189,7 @@ PhysicalPlanNodePtr PhysicalJoin::build( settings.shallow_copy_cross_probe_threshold, match_helper_name, flag_mapped_entry_helper_name, - 0, - 0, + settings.join_probe_cache_columns_threshold, context.isTest(), runtime_filter_list); diff --git a/dbms/src/Flash/tests/gtest_join.h b/dbms/src/Flash/tests/gtest_join.h index ee39f5361e6..6b3e621acc2 100644 --- a/dbms/src/Flash/tests/gtest_join.h +++ b/dbms/src/Flash/tests/gtest_join.h @@ -74,7 +74,7 @@ class JoinTestRunner : public DB::tests::ExecutorTest for (const auto & column_info : mockColumnInfosToTiDBColumnInfos(left_column_infos)) { ColumnGeneratorOpts opts{ - common_rows, + common_rows / 2, getDataTypeByColumnInfoForComputingLayer(column_info)->getName(), RANDOM, column_info.name}; @@ -103,11 +103,22 @@ class JoinTestRunner : public DB::tests::ExecutorTest for (size_t i = 0; i < common_column_data.size(); ++i) { - left_column_data[i].column->assumeMutable()->insertRangeFrom(*common_column_data[i].column, 0, common_rows); + left_column_data[i].column->assumeMutable()->insertRangeFrom( + *common_column_data[i].column, + 0, + common_rows / 2); + left_column_data[i].column->assumeMutable()->insertRangeFrom( + *common_column_data[i].column, + 0, + common_rows / 2); + right_column_data[i].column->assumeMutable()->insertRangeFrom( + *common_column_data[i].column, + 0, + common_rows / 2); right_column_data[i].column->assumeMutable()->insertRangeFrom( *common_column_data[i].column, 0, - common_rows); + common_rows / 2); } ColumnWithTypeAndName shuffle_column = ColumnGenerator::instance().generate({table_rows, "UInt64", RANDOM}); diff --git a/dbms/src/Flash/tests/gtest_join_executor.cpp b/dbms/src/Flash/tests/gtest_join_executor.cpp index 2ed45832bc2..55350f363c3 100644 --- a/dbms/src/Flash/tests/gtest_join_executor.cpp +++ b/dbms/src/Flash/tests/gtest_join_executor.cpp @@ -154,6 +154,7 @@ try toNullableVec({0, 0, 0, 1, 1})}, }; + std::vector probe_cache_column_threshold{2, 1000}; for (size_t i = 0; i < join_type_num; ++i) { for (size_t j = 0; j < simple_test_num; ++j) @@ -163,7 +164,13 @@ try .join(context.scan("simple_test", r), join_types[i], {col(k)}) .build(context); - executeAndAssertColumnsEqual(request, expected_cols[i * simple_test_num + j]); + for (auto threshold : probe_cache_column_threshold) + { + context.context->setSetting( + "join_probe_cache_columns_threshold", + Field(static_cast(threshold))); + executeAndAssertColumnsEqual(request, expected_cols[i * simple_test_num + j]); + } } } } @@ -2120,6 +2127,7 @@ try context.context->setSetting("max_block_size", Field(static_cast(max_block_size))); /// use right_table left join left_table as the reference auto ref_columns = executeStreams(request, original_max_streams); + std::vector probe_cache_column_threshold{2, 1000}; /// case 1.1 table scan join table scan for (auto & left_table_name : left_table_names) @@ -2133,8 +2141,14 @@ try {col("a")}) .build(context); WRAP_FOR_JOIN_TEST_BEGIN - ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) - << "left_table_name = " << left_table_name << ", right_table_name = " << right_table_name; + for (auto threshold : probe_cache_column_threshold) + { + context.context->setSetting( + "join_probe_cache_columns_threshold", + Field(static_cast(threshold))); + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) + << "left_table_name = " << left_table_name << ", right_table_name = " << right_table_name; + } WRAP_FOR_JOIN_TEST_END } } @@ -2157,13 +2171,19 @@ try exchange_concurrency) .build(context); WRAP_FOR_JOIN_TEST_BEGIN - ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) - << "left_table_name = " << left_table_name - << ", right_exchange_receiver_concurrency = " << exchange_concurrency; - if (original_max_streams_small < exchange_concurrency) - ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams_small)) + for (auto threshold : probe_cache_column_threshold) + { + context.context->setSetting( + "join_probe_cache_columns_threshold", + Field(static_cast(threshold))); + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) << "left_table_name = " << left_table_name << ", right_exchange_receiver_concurrency = " << exchange_concurrency; + if (original_max_streams_small < exchange_concurrency) + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams_small)) + << "left_table_name = " << left_table_name + << ", right_exchange_receiver_concurrency = " << exchange_concurrency; + } WRAP_FOR_JOIN_TEST_END } } @@ -2204,8 +2224,14 @@ try 0) .build(context); WRAP_FOR_JOIN_TEST_BEGIN - ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) - << "left_table_name = " << left_table_name << ", right_table_name = " << right_table_name; + for (auto threshold : probe_cache_column_threshold) + { + context.context->setSetting( + "join_probe_cache_columns_threshold", + Field(static_cast(threshold))); + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) + << "left_table_name = " << left_table_name << ", right_table_name = " << right_table_name; + } WRAP_FOR_JOIN_TEST_END } } @@ -2229,13 +2255,21 @@ try exchange_concurrency) .build(context); WRAP_FOR_JOIN_TEST_BEGIN - ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) - << "left_table_name = " << left_table_name - << ", right_exchange_receiver_concurrency = " << exchange_concurrency; - if (original_max_streams_small < exchange_concurrency) - ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams_small)) + for (auto threshold : probe_cache_column_threshold) + { + context.context->setSetting( + "join_probe_cache_columns_threshold", + Field(static_cast(threshold))); + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams)) << "left_table_name = " << left_table_name - << ", right_exchange_receiver_concurrency = " << exchange_concurrency; + << ", right_exchange_receiver_concurrency = " << exchange_concurrency + << ", join_probe_cache_columns_threshold = " << threshold; + if (original_max_streams_small < exchange_concurrency) + ASSERT_COLUMNS_EQ_UR(ref_columns, executeStreams(request, original_max_streams_small)) + << "left_table_name = " << left_table_name + << ", right_exchange_receiver_concurrency = " << exchange_concurrency + << ", join_probe_cache_columns_threshold = " << threshold; + } WRAP_FOR_JOIN_TEST_END } } @@ -2799,6 +2833,64 @@ try } CATCH +TEST_F(JoinExecutorTestRunner, ProbeCacheColumnsForRightSemiJoin) +try +{ + UInt64 max_block_size = 800; + size_t max_streams = 2; + std::vector left_table_names = {"left_table_1_concurrency", "left_table_3_concurrency"}; + std::vector right_table_names = {"right_table_1_concurrency", "right_table_3_concurrency"}; + context.context->setSetting("max_block_size", Field(static_cast(max_block_size))); + + for (const auto type : {tipb::JoinType::TypeSemiJoin, tipb::JoinType::TypeAntiSemiJoin}) + { + auto request = context.scan("outer_join_test", left_table_names[0]) + .join( + context.scan("outer_join_test", right_table_names[0]), + type, + {col("a")}, + {}, + {}, + {lt(col(left_table_names[0] + ".a"), col(right_table_names[0] + ".b"))}, + {}, + 0, + false, + 0) + .build(context); + auto reference = executeStreams(request, max_streams); + std::vector probe_cache_column_threshold{2, 1000}; + for (auto & left_table_name : left_table_names) + { + for (auto & right_table_name : right_table_names) + { + request = context.scan("outer_join_test", left_table_name) + .join( + context.scan("outer_join_test", right_table_name), + type, + {col("a")}, + {}, + {}, + {lt(col(left_table_name + ".a"), col(right_table_name + ".b"))}, + {}, + 0, + false, + 0) + .build(context); + for (auto threshold : probe_cache_column_threshold) + { + context.context->setSetting( + "join_probe_cache_columns_threshold", + Field(static_cast(threshold))); + ASSERT_COLUMNS_EQ_UR(reference, executeStreams(request, max_streams)) + << "left_table_name = " << left_table_name << ", right_table_name = " << right_table_name + << ", join_probe_cache_columns_threshold = " << threshold; + } + } + } + } +} +CATCH + TEST_F(JoinExecutorTestRunner, RightSemiFamilyJoin) try { diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 37b4364b85d..840900afe11 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -358,7 +358,7 @@ void ExpressionAction::execute(Block & block) const //TODO: Clean up all Join logic in ExpressionAction case JOIN: { - ProbeProcessInfo probe_process_info(0); + ProbeProcessInfo probe_process_info(0, 0); probe_process_info.block = block; join->joinBlock(probe_process_info); break; diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 3bc3d5a038c..4765d0677e8 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -127,12 +127,11 @@ Join::Join( const Names & key_names_right_, ASTTableJoin::Kind kind_, const String & req_id, - bool enable_fine_grained_shuffle_, size_t fine_grained_shuffle_count_, size_t max_bytes_before_external_join, const SpillConfig & build_spill_config, const SpillConfig & probe_spill_config, - Int64 join_restore_concurrency_, + const RestoreConfig & restore_config_, const Names & tidb_output_column_names_, const RegisterOperatorSpillContext & register_operator_spill_context_, AutoSpillTrigger * auto_spill_trigger_, @@ -142,11 +141,10 @@ Join::Join( size_t shallow_copy_cross_probe_threshold_, const String & match_helper_name_, const String & flag_mapped_entry_helper_name_, - size_t restore_round_, - size_t restore_part, + size_t probe_cache_column_threshold_, bool is_test_, const std::vector & runtime_filter_list_) - : restore_round(restore_round_) + : restore_config(restore_config_) , match_helper_name(match_helper_name_) , flag_mapped_entry_helper_name(flag_mapped_entry_helper_name_) , kind(kind_) @@ -162,18 +160,22 @@ Join::Join( , non_equal_conditions(non_equal_conditions_) , max_block_size(max_block_size_) , runtime_filter_list(runtime_filter_list_) - , join_restore_concurrency(join_restore_concurrency_) , register_operator_spill_context(register_operator_spill_context_) , auto_spill_trigger(auto_spill_trigger_) , shallow_copy_cross_probe_threshold( shallow_copy_cross_probe_threshold_ > 0 ? shallow_copy_cross_probe_threshold_ : std::max(1, max_block_size / 10)) + , probe_cache_column_threshold(probe_cache_column_threshold_) , tidb_output_column_names(tidb_output_column_names_) , is_test(is_test_) , log(Logger::get( - restore_round == 0 ? join_req_id - : fmt::format("{}_round_{}_part_{}", join_req_id, restore_round, restore_part))) - , enable_fine_grained_shuffle(enable_fine_grained_shuffle_) + restore_config.restore_round == 0 ? join_req_id + : fmt::format( + "{}_round_{}_part_{}", + join_req_id, + restore_config.restore_round, + restore_config.restore_partition_id))) + , enable_fine_grained_shuffle(fine_grained_shuffle_count_ > 0) , fine_grained_shuffle_count(fine_grained_shuffle_count_) { has_other_condition = non_equal_conditions.other_cond_expr != nullptr; @@ -182,6 +184,9 @@ Join::Join( else strictness = ASTTableJoin::Strictness::All; + if (isNullAwareSemiFamily(kind)) + probe_cache_column_threshold = 0; + if unlikely (kind == ASTTableJoin::Kind::Cross_RightOuter) throw Exception("Cross right outer join should be converted to cross Left outer join during compile"); RUNTIME_CHECK(!(isNecessaryKindToUseRowFlaggedHashMap(kind) && strictness == ASTTableJoin::Strictness::Any)); @@ -199,7 +204,7 @@ Join::Join( max_restore_round = MAX_RESTORE_ROUND_IN_GTEST; #endif - if (hash_join_spill_context->supportSpill() && restore_round >= max_restore_round) + if (hash_join_spill_context->supportSpill() && restore_config.restore_round >= max_restore_round) { LOG_WARNING(log, fmt::format("restore round reach to {}, spilling will be disabled.", max_restore_round)); hash_join_spill_context->disableSpill(); @@ -368,12 +373,14 @@ std::shared_ptr Join::createRestoreJoin(size_t max_bytes_before_external_j key_names_right, kind, join_req_id, - false, + /// restore join never enable fine grained shuffle 0, max_bytes_before_external_join_, - hash_join_spill_context->createBuildSpillConfig(fmt::format("{}_{}_build", join_req_id, restore_round + 1)), - hash_join_spill_context->createProbeSpillConfig(fmt::format("{}_{}_probe", join_req_id, restore_round + 1)), - join_restore_concurrency, + hash_join_spill_context->createBuildSpillConfig( + fmt::format("{}_{}_build", join_req_id, restore_config.restore_round + 1)), + hash_join_spill_context->createProbeSpillConfig( + fmt::format("{}_{}_probe", join_req_id, restore_config.restore_round + 1)), + RestoreConfig{restore_config.join_restore_concurrency, restore_config.restore_round + 1, restore_partition_id}, tidb_output_column_names, register_operator_spill_context, auto_spill_trigger, @@ -383,8 +390,7 @@ std::shared_ptr Join::createRestoreJoin(size_t max_bytes_before_external_j shallow_copy_cross_probe_threshold, match_helper_name, flag_mapped_entry_helper_name, - restore_round + 1, - restore_partition_id, + probe_cache_column_threshold, is_test); } @@ -622,7 +628,7 @@ bool Join::isEnableSpill() const bool Join::isRestoreJoin() const { - return restore_round > 0; + return restore_config.restore_round > 0; } void Join::insertFromBlockInternal(Block * stored_block, size_t stream_index) @@ -735,7 +741,8 @@ void Join::insertFromBlockInternal(Block * stored_block, size_t stream_index) stream_index, getBuildConcurrency(), enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); } // generator in runtime filter @@ -1235,14 +1242,14 @@ Block Join::doJoinBlockHash(ProbeProcessInfo & probe_process_info, const JoinBui right_indexes.push_back(num_columns_to_skip + i); } - /// For RightSemi/RightAnti join with other conditions, using this column to record hash entries that matches keys - /// Note: this column will record map entry addresses, so should use it carefully and better limit its usage in this function only. - MutableColumnPtr flag_mapped_entry_helper_column = nullptr; - if (useRowFlaggedHashMap(kind, has_other_condition)) + bool use_row_flagged_hash_map = useRowFlaggedHashMap(kind, has_other_condition); + if (use_row_flagged_hash_map) { - flag_mapped_entry_helper_column = flag_mapped_entry_helper_type->createColumn(); + /// For RightSemi/RightAnti join with other conditions, using this column to record hash entries that matches keys + /// Note: this column will record map entry addresses, so should use it carefully and better limit its usage in this function only. // todo figure out more accurate `rows` - flag_mapped_entry_helper_column->reserve(rows); + added_columns.push_back(flag_mapped_entry_helper_type->createColumn()); + added_columns.back()->reserve(rows); } IColumn::Offset current_offset = 0; @@ -1262,11 +1269,10 @@ Block Join::doJoinBlockHash(ProbeProcessInfo & probe_process_info, const JoinBui right_indexes, collators, join_build_info, - probe_process_info, - flag_mapped_entry_helper_column); + probe_process_info); FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_join_prob_failpoint); /// For RIGHT_SEMI/RIGHT_ANTI join without other conditions, hash table has been marked already, just return empty build table header - if (isRightSemiFamily(kind) && !flag_mapped_entry_helper_column) + if (isRightSemiFamily(kind) && !use_row_flagged_hash_map) { return sample_block_with_columns_to_add; } @@ -1276,9 +1282,9 @@ Block Join::doJoinBlockHash(ProbeProcessInfo & probe_process_info, const JoinBui const ColumnWithTypeAndName & sample_col = sample_block_with_columns_to_add.getByPosition(i); block.insert(ColumnWithTypeAndName(std::move(added_columns[i]), sample_col.type, sample_col.name)); } - if (flag_mapped_entry_helper_column) + if (use_row_flagged_hash_map) block.insert(ColumnWithTypeAndName( - std::move(flag_mapped_entry_helper_column), + std::move(added_columns[num_columns_to_add]), flag_mapped_entry_helper_type, flag_mapped_entry_helper_name)); @@ -1384,7 +1390,7 @@ Block Join::joinBlockHash(ProbeProcessInfo & probe_process_info) const isEnableSpill(), hash_join_spill_context->isSpilled(), build_concurrency, - restore_round}; + restore_config.restore_round}; probe_process_info.prepareForHashProbe( key_names_left, non_equal_conditions.left_filter_column, @@ -1392,7 +1398,7 @@ Block Join::joinBlockHash(ProbeProcessInfo & probe_process_info) const strictness, join_build_info.needVirtualDispatchForProbeBlock(), collators, - restore_round); + restore_config.restore_round); while (true) { auto block = doJoinBlockHash(probe_process_info, join_build_info); @@ -2141,7 +2147,7 @@ IColumn::Selector Join::selectDispatchBlock(const Strings & key_columns_names, c sort_key_containers.resize(key_columns.size()); WeakHash32 hash(0); - computeDispatchHash(num_rows, key_columns, collators, sort_key_containers, restore_round, hash); + computeDispatchHash(num_rows, key_columns, collators, sort_key_containers, restore_config.restore_round, hash); return hashToSelector(hash); } @@ -2186,7 +2192,7 @@ void Join::spillMostMemoryUsedPartitionIfNeed(size_t stream_index) #ifdef DBMS_PUBLIC_GTEST // for join spill to disk gtest - if (restore_round == std::max(2, MAX_RESTORE_ROUND_IN_GTEST) - 1 + if (restore_config.restore_round == std::max(2, MAX_RESTORE_ROUND_IN_GTEST) - 1 && hash_join_spill_context->spilledPartitionCount() >= partitions.size() / 2) return; #endif @@ -2201,7 +2207,7 @@ void Join::spillMostMemoryUsedPartitionIfNeed(size_t stream_index) log, fmt::format( "Join with restore round: {}, used {} bytes, will spill partition: {}.", - restore_round, + restore_config.restore_round, getTotalByteCount(), partition_to_be_spilled)); @@ -2261,7 +2267,7 @@ std::optional Join::getOneRestoreStream(size_t max_block_size_) restore_join_build_concurrency = getRestoreJoinBuildConcurrency( partitions.size(), remaining_partition_indexes_to_restore.size(), - join_restore_concurrency, + restore_config.join_restore_concurrency, probe_concurrency); /// for restore join we make sure that the restore_join_build_concurrency is at least 2, so it can be spill again. /// And restore_join_build_concurrency should not be greater than probe_concurrency, Otherwise some restore_stream will never be executed. @@ -2275,7 +2281,7 @@ std::optional Join::getOneRestoreStream(size_t max_block_size_) log, "Begin restore data from disk for hash join, partition {}, restore round {}, build concurrency {}.", spilled_partition_index, - restore_round, + restore_config.restore_round, restore_join_build_concurrency); auto restore_build_streams = hash_join_spill_context->getBuildSpiller()->restoreBlocks( diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 6b6723f82de..ebe74c42eca 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -97,6 +97,13 @@ struct PartitionBlock }; using PartitionBlocks = std::list; +struct RestoreConfig +{ + Int64 join_restore_concurrency; + size_t restore_round; + size_t restore_partition_id; +}; + /** Data structure for implementation of JOIN. * It is just a hash table: keys -> rows of joined ("right") table. * Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys. @@ -159,27 +166,25 @@ class Join const Names & key_names_right_, ASTTableJoin::Kind kind_, const String & req_id, - bool enable_fine_grained_shuffle_, size_t fine_grained_shuffle_count_, size_t max_bytes_before_external_join_, const SpillConfig & build_spill_config_, const SpillConfig & probe_spill_config_, - Int64 join_restore_concurrency_, + const RestoreConfig & restore_config_, const Names & tidb_output_column_names_, const RegisterOperatorSpillContext & register_operator_spill_context_, AutoSpillTrigger * auto_spill_trigger_, - const TiDB::TiDBCollators & collators_ = TiDB::dummy_collators, - const JoinNonEqualConditions & non_equal_conditions_ = {}, - size_t max_block_size = 0, - size_t shallow_copy_cross_probe_threshold_ = 0, - const String & match_helper_name_ = "", - const String & flag_mapped_entry_helper_name_ = "", - size_t restore_round = 0, - size_t restore_partition_id = 0, - bool is_test = true, + const TiDB::TiDBCollators & collators_, + const JoinNonEqualConditions & non_equal_conditions_, + size_t max_block_size, + size_t shallow_copy_cross_probe_threshold_, + const String & match_helper_name_, + const String & flag_mapped_entry_helper_name_, + size_t probe_cache_column_threshold_, + bool is_test, const std::vector & runtime_filter_list_ = dummy_runtime_filter_list); - size_t restore_round; + RestoreConfig restore_config; /** Call `setBuildConcurrencyAndInitJoinPartition` and `setSampleBlock`. * You must call this method before subsequent calls to insertFromBlock. @@ -304,6 +309,7 @@ class Join const MarkedSpillData & getProbeSideMarkedSpillData(size_t stream_index) const; bool hasProbeSideMarkedSpillData(size_t stream_index) const; void flushProbeSideMarkedSpillData(size_t stream_index); + size_t getProbeCacheColumnThreshold() const { return probe_cache_column_threshold; } static const String match_helper_prefix; static const DataTypePtr match_helper_type; @@ -369,8 +375,6 @@ class Join std::list remaining_partition_indexes_to_restore; - Int64 join_restore_concurrency; - std::atomic peak_build_bytes_usage{0}; std::vector restore_infos; @@ -396,6 +400,7 @@ class Join CrossProbeMode cross_probe_mode = CrossProbeMode::DEEP_COPY_RIGHT_BLOCK; size_t right_rows_to_be_added_when_matched_for_cross_join = 0; size_t shallow_copy_cross_probe_threshold; + size_t probe_cache_column_threshold; JoinMapMethod join_map_method = JoinMapMethod::EMPTY; diff --git a/dbms/src/Interpreters/JoinHashMap.h b/dbms/src/Interpreters/JoinHashMap.h index 02e58cb92cf..fe38be922f4 100644 --- a/dbms/src/Interpreters/JoinHashMap.h +++ b/dbms/src/Interpreters/JoinHashMap.h @@ -24,7 +24,7 @@ using Sizes = std::vector; struct RowRef { const Block * block; - size_t row_num; + UInt32 row_num; RowRef() = default; RowRef(const Block * block_, size_t row_num_) @@ -33,23 +33,55 @@ struct RowRef {} }; +enum class CachedColumnState +{ + NOT_CACHED, + CONSTRUCT_CACHE, + CACHED, +}; +struct CachedColumnInfo +{ + std::mutex mu; + Columns columns; + CachedColumnState state = CachedColumnState::NOT_CACHED; + void * next; + explicit CachedColumnInfo(void * next_) + : next(next_) + {} +}; + /// Single linked list of references to rows. Used for ALL JOINs (non-unique JOINs) struct RowRefList : RowRef { - RowRefList * next = nullptr; + UInt32 list_length = 0; + union + { + RowRefList * next = nullptr; + CachedColumnInfo * cached_column_info; + }; RowRefList() = default; RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {} + /// for head node + RowRefList(const Block * block_, size_t row_num_, bool sentinel_head) + : RowRef(block_, row_num_) + , list_length(sentinel_head ? 0 : 1) + {} }; /// Single linked list of references to rows with used flag for each row struct RowRefListWithUsedFlag : RowRef { + UInt32 list_length = 0; using Base_t = RowRefListWithUsedFlag; mutable std::atomic used{}; - RowRefListWithUsedFlag * next = nullptr; + union + { + RowRefListWithUsedFlag * next = nullptr; + CachedColumnInfo * cached_column_info; + }; void setUsed() const { @@ -61,6 +93,11 @@ struct RowRefListWithUsedFlag : RowRef RowRefListWithUsedFlag(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {} + /// for head node + RowRefListWithUsedFlag(const Block * block_, size_t row_num_, bool sentinel_head) + : RowRef(block_, row_num_) + , list_length(sentinel_head ? 0 : 1) + {} }; /** Depending on template parameter, adds or doesn't add a flag, that element was used (row was joined). diff --git a/dbms/src/Interpreters/JoinPartition.cpp b/dbms/src/Interpreters/JoinPartition.cpp index 6b42e7d1fb7..5110e33a0b8 100644 --- a/dbms/src/Interpreters/JoinPartition.cpp +++ b/dbms/src/Interpreters/JoinPartition.cpp @@ -37,10 +37,27 @@ extern const char random_fail_in_resize_callback[]; namespace { template -void insertRowToList(List * list, Elem * elem) +void insertRowToList(JoinArenaPool & pool, List * list, Elem * elem, size_t cache_columns_threshold) { - elem->next = list->next; // NOLINT(clang-analyzer-core.NullDereference) - list->next = elem; + ++list->list_length; + if unlikely (cache_columns_threshold > 0 && list->list_length >= cache_columns_threshold) + { + if unlikely (cache_columns_threshold == list->list_length) + { + auto * cached_column_info + = reinterpret_cast(pool.arena.alloc(sizeof(CachedColumnInfo))); + new (cached_column_info) CachedColumnInfo(list->next); + pool.cached_column_infos.push_back(cached_column_info); + list->cached_column_info = cached_column_info; + } + elem->next = reinterpret_cast(list->cached_column_info->next); + list->cached_column_info->next = elem; + } + else + { + elem->next = list->next; // NOLINT(clang-analyzer-core.NullDereference) + list->next = elem; + } } } // namespace @@ -51,7 +68,7 @@ extern const char random_join_build_failpoint[]; using PointerHelper = PointerTypeColumnHelper; -void RowsNotInsertToMap::insertRow(Block * stored_block, size_t index, bool need_materialize, Arena & pool) +void RowsNotInsertToMap::insertRow(Block * stored_block, size_t index, bool need_materialize, JoinArenaPool & pool) { if (need_materialize) { @@ -66,9 +83,10 @@ void RowsNotInsertToMap::insertRow(Block * stored_block, size_t index, bool need } else { - auto * elem = reinterpret_cast(pool.alloc(sizeof(RowRefList))); + auto * elem = reinterpret_cast(pool.arena.alloc(sizeof(RowRefList))); new (elem) RowRefList(stored_block, index); - insertRowToList(&head, elem); + /// don't need cache column since it will explicitly materialize of need_materialize is true + insertRowToList(pool, &head, elem, 0); } ++total_size; } @@ -254,7 +272,7 @@ void JoinPartition::setResizeCallbackIfNeeded() return ret; }; assert(pool != nullptr); - pool->setResizeCallback(resize_callback); + pool->arena.setResizeCallback(resize_callback); setResizeCallbackImpl(maps_any, join_map_method, resize_callback); setResizeCallbackImpl(maps_all, join_map_method, resize_callback); setResizeCallbackImpl(maps_any_full, join_map_method, resize_callback); @@ -337,12 +355,12 @@ void JoinPartition::releaseProbePartitionBlocks(std::unique_lock &) void JoinPartition::releasePartitionPoolAndHashMap(std::unique_lock &) { - pool.reset(); clearMaps(maps_any, join_map_method); clearMaps(maps_all, join_map_method); clearMaps(maps_any_full, join_map_method); clearMaps(maps_all_full, join_map_method); clearMaps(maps_all_full_with_row_flag, join_map_method); + pool.reset(); LOG_DEBUG(log, "release {} memories from partition {}", hash_table_pool_memory_usage, partition_index); hash_table_pool_memory_usage = 0; } @@ -452,8 +470,9 @@ struct Inserter const typename Map::key_type & key, Block * stored_block, size_t i, - Arena & pool, - std::vector & sort_key_containers); + JoinArenaPool & pool, + std::vector & sort_key_containers, + size_t cache_column_threshold); }; template @@ -464,10 +483,11 @@ struct Inserter KeyGetter & key_getter, Block * stored_block, size_t i, - Arena & pool, - std::vector & sort_key_container) + JoinArenaPool & pool, + std::vector & sort_key_container, + size_t /* cache_column_threshold */) { - auto emplace_result = key_getter.emplaceKey(map, i, pool, sort_key_container); + auto emplace_result = key_getter.emplaceKey(map, i, pool.arena, sort_key_container); if (emplace_result.isInserted()) new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); @@ -483,22 +503,23 @@ struct Inserter KeyGetter & key_getter, Block * stored_block, size_t i, - Arena & pool, - std::vector & sort_key_container) + JoinArenaPool & pool, + std::vector & sort_key_container, + size_t cache_column_threshold) { - auto emplace_result = key_getter.emplaceKey(map, i, pool, sort_key_container); + auto emplace_result = key_getter.emplaceKey(map, i, pool.arena, sort_key_container); if (emplace_result.isInserted()) - new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); + new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i, 0); else { /** The first element of the list is stored in the value of the hash table, the rest in the pool. * We will insert each time the element into the second place. * That is, the former second element, if it was, will be the third, and so on. */ - auto elem = reinterpret_cast(pool.alloc(sizeof(MappedType))); + auto elem = reinterpret_cast(pool.arena.alloc(sizeof(MappedType))); new (elem) typename Map::mapped_type(stored_block, i); - insertRowToList(&emplace_result.getMapped(), elem); + insertRowToList(pool, &emplace_result.getMapped(), elem, cache_column_threshold); } } }; @@ -518,7 +539,8 @@ void NO_INLINE insertBlockIntoMapTypeCase( const TiDB::TiDBCollators & collators, Block * stored_block, ConstNullMapPtr null_map, - RowsNotInsertToMap * rows_not_inserted_to_map) + RowsNotInsertToMap * rows_not_inserted_to_map, + size_t probe_cache_column_threshold) { auto & pool = *join_partition.getPartitionPool(); @@ -543,7 +565,14 @@ void NO_INLINE insertBlockIntoMapTypeCase( } } - Inserter::insert(map, key_getter, stored_block, i, pool, sort_key_containers); + Inserter::insert( + map, + key_getter, + stored_block, + i, + pool, + sort_key_containers, + probe_cache_column_threshold); } } @@ -563,7 +592,8 @@ void NO_INLINE insertBlockIntoMapsTypeCase( Block * stored_block, ConstNullMapPtr null_map, size_t stream_index, - RowsNotInsertToMap * rows_not_inserted_to_map) + RowsNotInsertToMap * rows_not_inserted_to_map, + size_t probe_cache_column_threshold) { auto & current_join_partition = join_partitions[stream_index]; auto & pool = *current_join_partition->getPartitionPool(); @@ -605,7 +635,7 @@ void NO_INLINE insertBlockIntoMapsTypeCase( continue; } } - auto key_holder = key_getter.getKeyHolder(i, &pool, sort_key_containers); + auto key_holder = key_getter.getKeyHolder(i, &pool.arena, sort_key_containers); SCOPE_EXIT(keyHolderDiscardKey(key_holder)); auto key = keyHolderGetKey(key_holder); @@ -635,7 +665,8 @@ void NO_INLINE insertBlockIntoMapsTypeCase( stored_block, \ s_i, \ pool, \ - sort_key_containers); \ + sort_key_containers, \ + probe_cache_column_threshold); \ } #define INSERT_TO_NOT_INSERTED_MAP \ @@ -710,7 +741,8 @@ void insertBlockIntoMapsImplType( size_t stream_index, size_t insert_concurrency, bool enable_fine_grained_shuffle, - bool enable_join_spill) + bool enable_join_spill, + size_t probe_cache_column_threshold) { auto & current_join_partition = join_partitions[stream_index]; auto * rows_not_inserted_to_map = current_join_partition->getRowsNotInsertedToMap(); @@ -728,7 +760,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - rows_not_inserted_to_map); + rows_not_inserted_to_map, + probe_cache_column_threshold); else insertBlockIntoMapTypeCase( *current_join_partition, @@ -738,7 +771,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - nullptr); + nullptr, + probe_cache_column_threshold); } else { @@ -750,7 +784,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - nullptr); + nullptr, + probe_cache_column_threshold); } return; } @@ -768,7 +803,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - rows_not_inserted_to_map); + rows_not_inserted_to_map, + probe_cache_column_threshold); else insertBlockIntoMapTypeCase( *current_join_partition, @@ -778,7 +814,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - nullptr); + nullptr, + probe_cache_column_threshold); } else { @@ -790,7 +827,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - nullptr); + nullptr, + probe_cache_column_threshold); } } else if (insert_concurrency > 1) @@ -808,7 +846,8 @@ void insertBlockIntoMapsImplType( stored_block, null_map, stream_index, - rows_not_inserted_to_map); + rows_not_inserted_to_map, + probe_cache_column_threshold); else insertBlockIntoMapsTypeCase( join_partitions, @@ -819,7 +858,8 @@ void insertBlockIntoMapsImplType( stored_block, null_map, stream_index, - nullptr); + nullptr, + probe_cache_column_threshold); } else { @@ -832,7 +872,8 @@ void insertBlockIntoMapsImplType( stored_block, null_map, stream_index, - nullptr); + nullptr, + probe_cache_column_threshold); } } else @@ -850,7 +891,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - rows_not_inserted_to_map); + rows_not_inserted_to_map, + probe_cache_column_threshold); else insertBlockIntoMapTypeCase( *current_join_partition, @@ -860,7 +902,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - nullptr); + nullptr, + probe_cache_column_threshold); } else { @@ -872,7 +915,8 @@ void insertBlockIntoMapsImplType( collators, stored_block, null_map, - nullptr); + nullptr, + probe_cache_column_threshold); } } } @@ -889,7 +933,8 @@ void insertBlockIntoMapsImpl( size_t stream_index, size_t insert_concurrency, bool enable_fine_grained_shuffle, - bool enable_join_spill) + bool enable_join_spill, + size_t probe_cache_column_threshold) { switch (join_partitions[stream_index]->getJoinMapMethod()) { @@ -914,7 +959,8 @@ void insertBlockIntoMapsImpl( stream_index, \ insert_concurrency, \ enable_fine_grained_shuffle, \ - enable_join_spill); \ + enable_join_spill, \ + probe_cache_column_threshold); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -963,7 +1009,8 @@ void JoinPartition::insertBlockIntoMaps( size_t stream_index, size_t insert_concurrency, bool enable_fine_grained_shuffle, - bool enable_join_spill) + bool enable_join_spill, + size_t probe_cache_column_threshold) { auto & current_join_partition = join_partitions[stream_index]; assert(!current_join_partition->isSpill()); @@ -982,7 +1029,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); else insertBlockIntoMapsImpl( join_partitions, @@ -995,7 +1043,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); } else if (isNecessaryKindToUseRowFlaggedHashMap(current_kind)) { @@ -1011,7 +1060,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); else insertBlockIntoMapsImpl( join_partitions, @@ -1024,7 +1074,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); } else if (getFullness(current_kind)) { @@ -1040,7 +1091,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); else insertBlockIntoMapsImpl( join_partitions, @@ -1053,7 +1105,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); } else { @@ -1069,7 +1122,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); else insertBlockIntoMapsImpl( join_partitions, @@ -1082,7 +1136,8 @@ void JoinPartition::insertBlockIntoMaps( stream_index, insert_concurrency, enable_fine_grained_shuffle, - enable_join_spill); + enable_join_spill, + probe_cache_column_threshold); } } namespace @@ -1239,6 +1294,77 @@ struct Adder checkCachedColumnInfo(CachedColumnInfo * cached_column_info) +{ + std::unique_lock lock(cached_column_info->mu); + bool has_cached_columns = cached_column_info->state == CachedColumnState::CACHED; + if (!has_cached_columns) + { + if (cached_column_info->state == CachedColumnState::NOT_CACHED) + { + cached_column_info->state = CachedColumnState::CONSTRUCT_CACHE; + return {false, true}; + } + else + { + return {false, false}; + } + } + return {true, false}; +} + +void insertCachedColumns( + CachedColumnInfo * cached_column_info, + MutableColumns & added_columns, + size_t rows_added, + size_t columns_to_added) +{ + if (columns_to_added == 0) + return; + if (added_columns[0]->empty()) + { + for (size_t j = 0; j < columns_to_added; ++j) + { + added_columns[j] = cached_column_info->columns[j]->cloneFullColumn(); + } + } + else + { + for (size_t j = 0; j < columns_to_added; ++j) + { + added_columns[j]->insertRangeFrom(*cached_column_info->columns[j], 0, rows_added); + } + } +} + +void cacheColumns(CachedColumnInfo * cached_column_info, MutableColumns & added_columns, size_t rows, size_t columns) +{ + Columns cached_columns; + if (columns > 0) + { + assert(added_columns[0]->size() >= rows); + size_t start_offset = added_columns[0]->size() - rows; + if (start_offset == 0) + { + for (size_t j = 0; j < columns; ++j) + cached_columns.push_back(added_columns[j]->cloneFullColumn()); + } + else + { + for (size_t j = 0; j < columns; ++j) + cached_columns.push_back(added_columns[j]->cut(start_offset, rows)); + } + } + std::unique_lock lock(cached_column_info->mu); + if (!cached_columns.empty()) + { + assert(cached_column_info->columns.empty()); + cached_column_info->columns.swap(cached_columns); + } + cached_column_info->state = CachedColumnState::CACHED; +} + template struct Adder { @@ -1251,22 +1377,55 @@ struct Adder & right_indexes, - ProbeProcessInfo & /*probe_process_info*/) + ProbeProcessInfo & probe_process_info) { - for (auto current = &static_cast(it->getMapped()); - current != nullptr; - current = current->next) - { + auto & mapped_value = static_cast(it->getMapped()); + size_t rows_joined = mapped_value.list_length; + bool need_generate_cached_columns = false; + auto * current = &mapped_value; + auto add_one_row = [&]() { for (size_t j = 0; j < num_columns_to_add - 1; ++j) added_columns[j]->insertFrom( *current->block->getByPosition(right_indexes[j]).column.get(), current->row_num); - ++current_offset; + }; + if unlikely ( + probe_process_info.cache_columns_threshold > 0 && rows_joined >= probe_process_info.cache_columns_threshold) + { + assert(mapped_value.cached_column_info != nullptr); + auto check_result = checkCachedColumnInfo(mapped_value.cached_column_info); + need_generate_cached_columns = check_result.second; + if (check_result.first) + { + insertCachedColumns( + mapped_value.cached_column_info, + added_columns, + rows_joined, + num_columns_to_add - 1); + current_offset += rows_joined; + (*offsets)[i] = current_offset; + /// we insert only one row to `match-helper` for each row of left block + /// so before the execution of `HandleOtherConditions`, column sizes of temporary block may be different. + added_columns[num_columns_to_add - 1]->insert(FIELD_INT8_1); + return false; + } + add_one_row(); + current = static_cast(current->cached_column_info->next); } + for (; current != nullptr; current = current->next) + { + add_one_row(); + } + current_offset += rows_joined; (*offsets)[i] = current_offset; /// we insert only one row to `match-helper` for each row of left block /// so before the execution of `HandleOtherConditions`, column sizes of temporary block may be different. added_columns[num_columns_to_add - 1]->insert(FIELD_INT8_1); + + if unlikely (need_generate_cached_columns) + { + cacheColumns(mapped_value.cached_column_info, added_columns, rows_joined, num_columns_to_add - 1); + } return false; } @@ -1304,28 +1463,48 @@ struct Adder const std::vector & right_indexes, ProbeProcessInfo & probe_process_info) { - size_t rows_joined = 0; + auto & mapped_value = static_cast(it->getMapped()); + size_t rows_joined = mapped_value.list_length; // If there are too many rows in the column to split, record the number of rows that have been expanded for next read. // and it means the rows in this block are not joined finish. - - for (auto current = &static_cast(it->getMapped()); - current != nullptr; - current = current->next) - ++rows_joined; - if (current_offset && current_offset + rows_joined > probe_process_info.max_block_size) { return true; } - for (auto current = &static_cast(it->getMapped()); - current != nullptr; - current = current->next) - { + bool need_generate_cached_columns = false; + auto * current = &mapped_value; + auto add_one_row = [&]() { for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertFrom( *current->block->getByPosition(right_indexes[j]).column.get(), current->row_num); + }; + if unlikely ( + probe_process_info.cache_columns_threshold > 0 && rows_joined >= probe_process_info.cache_columns_threshold) + { + assert(mapped_value.cached_column_info != nullptr); + auto check_result = checkCachedColumnInfo(mapped_value.cached_column_info); + need_generate_cached_columns = check_result.second; + if (check_result.first) + { + insertCachedColumns(mapped_value.cached_column_info, added_columns, rows_joined, num_columns_to_add); + current_offset += rows_joined; + (*offsets)[i] = current_offset; + if constexpr (KIND == ASTTableJoin::Kind::Anti) + /// anti join with other condition is very special: if the row is matched during probe stage, we can not throw it + /// away because it might failed in other condition, so we add the matched rows to the result, but set (*filter)[i] = 0 + /// to indicate that the row is matched during probe stage, this will be used in handleOtherConditions + (*filter)[i] = 0; + return false; + } + add_one_row(); + current = static_cast(current->cached_column_info->next); + } + + for (; current != nullptr; current = current->next) + { + add_one_row(); } current_offset += rows_joined; @@ -1336,6 +1515,10 @@ struct Adder /// to indicate that the row is matched during probe stage, this will be used in handleOtherConditions (*filter)[i] = 0; + if unlikely (need_generate_cached_columns) + { + cacheColumns(mapped_value.cached_column_info, added_columns, rows_joined, num_columns_to_add); + } return false; } @@ -1382,38 +1565,62 @@ struct RowFlaggedHashMapAdder IColumn::Offset & current_offset, IColumn::Offsets * offsets, const std::vector & right_indexes, - ProbeProcessInfo & probe_process_info, - MutableColumnPtr & ptr_col) + ProbeProcessInfo & probe_process_info) { - size_t rows_joined = 0; + /// the last column in added_columns is the ptr_col + assert(num_columns_to_add + 1 == added_columns.size()); + auto & mapped_value = static_cast(it->getMapped()); + size_t rows_joined = mapped_value.list_length; // If there are too many rows in the column to split, record the number of rows that have been expanded for next read. // and it means the rows in this block are not joined finish. - - for (auto current = &static_cast(it->getMapped()); - current != nullptr; - current = current->next) - ++rows_joined; - if (current_offset && current_offset + rows_joined > probe_process_info.max_block_size) { return true; } - auto & actual_ptr_col = static_cast(*ptr_col); + bool need_generate_cached_columns = false; + auto * current = &mapped_value; + auto & actual_ptr_col = static_cast(*added_columns[num_columns_to_add]); auto & container = static_cast(actual_ptr_col.getData()); - for (auto current = &static_cast(it->getMapped()); - current != nullptr; - current = current->next) - { + auto add_one_row = [&]() { for (size_t j = 0; j < num_columns_to_add; ++j) added_columns[j]->insertFrom( *current->block->getByPosition(right_indexes[j]).column.get(), current->row_num); - container.template push_back(reinterpret_cast(current)); - ++current_offset; + }; + if unlikely ( + probe_process_info.cache_columns_threshold > 0 && rows_joined >= probe_process_info.cache_columns_threshold) + { + assert(mapped_value.cached_column_info != nullptr); + auto check_result = checkCachedColumnInfo(mapped_value.cached_column_info); + need_generate_cached_columns = check_result.second; + if (check_result.first) + { + insertCachedColumns( + mapped_value.cached_column_info, + added_columns, + rows_joined, + num_columns_to_add + 1); + current_offset += rows_joined; + (*offsets)[i] = current_offset; + return false; + } + add_one_row(); + current = static_cast(current->cached_column_info->next); + } + + for (; current != nullptr; current = current->next) + { + add_one_row(); } + + current_offset += rows_joined; (*offsets)[i] = current_offset; + if unlikely (need_generate_cached_columns) + { + cacheColumns(mapped_value.cached_column_info, added_columns, rows_joined, num_columns_to_add + 1); + } return false; } @@ -1444,8 +1651,7 @@ void NO_INLINE probeBlockImplTypeCase( const std::vector & right_indexes, const TiDB::TiDBCollators & collators, const JoinBuildInfo & join_build_info, - ProbeProcessInfo & probe_process_info, - MutableColumnPtr & record_mapped_entry_column) + ProbeProcessInfo & probe_process_info) { if (rows == 0) { @@ -1567,8 +1773,7 @@ void NO_INLINE probeBlockImplTypeCase( current_offset, offsets_to_replicate.get(), right_indexes, - probe_process_info, - record_mapped_entry_column); + probe_process_info); } else if constexpr (KIND == ASTTableJoin::Kind::RightSemi || KIND == ASTTableJoin::Kind::RightAnti) { @@ -1641,8 +1846,7 @@ void probeBlockImplType( const std::vector & right_indexes, const TiDB::TiDBCollators & collators, const JoinBuildInfo & join_build_info, - ProbeProcessInfo & probe_process_info, - MutableColumnPtr & record_mapped_entry_column) + ProbeProcessInfo & probe_process_info) { if (null_map) { @@ -1659,8 +1863,7 @@ void probeBlockImplType( right_indexes, collators, join_build_info, - probe_process_info, - record_mapped_entry_column); + probe_process_info); } else { @@ -1677,8 +1880,7 @@ void probeBlockImplType( right_indexes, collators, join_build_info, - probe_process_info, - record_mapped_entry_column); + probe_process_info); } } template < @@ -1893,14 +2095,14 @@ void JoinPartition::probeBlock( const std::vector & right_indexes, const TiDB::TiDBCollators & collators, const JoinBuildInfo & join_build_info, - ProbeProcessInfo & probe_process_info, - MutableColumnPtr & record_mapped_entry_column) + ProbeProcessInfo & probe_process_info) { using enum ASTTableJoin::Strictness; using enum ASTTableJoin::Kind; const auto & current_partition = join_partitions[probe_process_info.partition_index]; auto kind = current_partition->kind; auto strictness = current_partition->strictness; + bool use_row_flagged_map = added_columns.size() > right_indexes.size(); assert(rows == 0 || !current_partition->isSpill()); #define CALL(KIND, STRICTNESS, MAP, row_flagged_map) \ @@ -1917,8 +2119,7 @@ void JoinPartition::probeBlock( right_indexes, \ collators, \ join_build_info, \ - probe_process_info, \ - record_mapped_entry_column); + probe_process_info); if (kind == Inner && strictness == All) CALL(Inner, All, MapsAll, false) @@ -1930,9 +2131,9 @@ void JoinPartition::probeBlock( CALL(LeftOuter, Any, MapsAnyFull, false) else if (kind == Full && strictness == All) CALL(LeftOuter, All, MapsAllFull, false) - else if (kind == RightOuter && strictness == All && !record_mapped_entry_column) + else if (kind == RightOuter && strictness == All && !use_row_flagged_map) CALL(Inner, All, MapsAllFull, false) - else if (kind == RightOuter && strictness == All && record_mapped_entry_column) + else if (kind == RightOuter && strictness == All && use_row_flagged_map) CALL(RightOuter, All, MapsAllFullWithRowFlag, true) else if (kind == Semi && strictness == Any) CALL(Semi, Any, MapsAny, false) @@ -1950,13 +2151,13 @@ void JoinPartition::probeBlock( CALL(LeftOuterSemi, Any, MapsAny, false) else if (kind == LeftOuterAnti && strictness == All) CALL(LeftOuterSemi, All, MapsAll, false) - else if (kind == RightSemi && record_mapped_entry_column) + else if (kind == RightSemi && use_row_flagged_map) CALL(RightSemi, All, MapsAllFullWithRowFlag, true) - else if (kind == RightSemi && !record_mapped_entry_column) + else if (kind == RightSemi && !use_row_flagged_map) CALL(RightSemi, All, MapsAllFull, false) - else if (kind == RightAnti && record_mapped_entry_column) + else if (kind == RightAnti && use_row_flagged_map) CALL(RightAnti, All, MapsAllFullWithRowFlag, true) - else if (kind == RightAnti && !record_mapped_entry_column) + else if (kind == RightAnti && !use_row_flagged_map) CALL(RightAnti, All, MapsAllFull, false) else throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR); @@ -1977,8 +2178,7 @@ void JoinPartition::probeBlockImpl( const std::vector & right_indexes, const TiDB::TiDBCollators & collators, const JoinBuildInfo & join_build_info, - ProbeProcessInfo & probe_process_info, - MutableColumnPtr & record_mapped_entry_column) + ProbeProcessInfo & probe_process_info) { const auto & current_join_partition = join_partitions[probe_process_info.partition_index]; auto method = current_join_partition->join_map_method; @@ -2004,8 +2204,7 @@ void JoinPartition::probeBlockImpl( right_indexes, \ collators, \ join_build_info, \ - probe_process_info, \ - record_mapped_entry_column); \ + probe_process_info); \ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M diff --git a/dbms/src/Interpreters/JoinPartition.h b/dbms/src/Interpreters/JoinPartition.h index b1040e0b6ed..5ea674be9e9 100644 --- a/dbms/src/Interpreters/JoinPartition.h +++ b/dbms/src/Interpreters/JoinPartition.h @@ -45,6 +45,20 @@ struct ProbePartition size_t bytes{0}; }; +struct JoinArenaPool +{ + Arena arena; + std::vector cached_column_infos; + size_t size() const { return cached_column_infos.capacity() * sizeof(CachedColumnInfo *) + arena.size(); } + ~JoinArenaPool() + { + for (auto * cci : cached_column_infos) + cci->~CachedColumnInfo(); + cached_column_infos.clear(); + } +}; +using JoinArenaPoolPtr = std::shared_ptr; + class Join; struct alignas(ABSL_CACHELINE_SIZE) RowsNotInsertToMap { @@ -65,7 +79,7 @@ struct alignas(ABSL_CACHELINE_SIZE) RowsNotInsertToMap /// Insert row to this structure. /// If need_materialize is true, row will be inserted into materialized_columns_vec. /// Else it will be inserted into row list. - void insertRow(Block * stored_block, size_t index, bool need_materialize, Arena & pool); + void insertRow(Block * stored_block, size_t index, bool need_materialize, JoinArenaPool & pool); }; class JoinPartition; @@ -83,10 +97,10 @@ class JoinPartition const LoggerPtr & log_, bool has_other_condition_) : partition_index(partition_index_) + , pool(std::make_shared()) , kind(kind_) , strictness(strictness_) , join_map_method(join_map_type_) - , pool(std::make_shared()) , hash_join_spill_context(hash_join_spill_context_) , has_other_condition(has_other_condition_) , log(log_) @@ -138,7 +152,7 @@ class JoinPartition JoinMapMethod getJoinMapMethod() const { return join_map_method; } ASTTableJoin::Kind getJoinKind() const { return kind; } Block * getLastBuildBlock() { return &build_partition.blocks.back(); } - ArenaPtr & getPartitionPool() + JoinArenaPoolPtr & getPartitionPool() { assert(pool != nullptr); return pool; @@ -166,7 +180,8 @@ class JoinPartition size_t stream_index, size_t insert_concurrency, bool enable_fine_grained_shuffle, - bool enable_join_spill); + bool enable_join_spill, + size_t probe_cache_column_threshold); /// probe the block using hash maps in `JoinPartitions` static void probeBlock( @@ -182,8 +197,7 @@ class JoinPartition const std::vector & right_indexes, const TiDB::TiDBCollators & collators, const JoinBuildInfo & join_build_info, - ProbeProcessInfo & probe_process_info, - MutableColumnPtr & record_mapped_entry_column); + ProbeProcessInfo & probe_process_info); template static void probeBlockImpl( const JoinPartitions & join_partitions, @@ -198,8 +212,7 @@ class JoinPartition const std::vector & right_indexes, const TiDB::TiDBCollators & collators, const JoinBuildInfo & join_build_info, - ProbeProcessInfo & probe_process_info, - MutableColumnPtr & record_mapped_entry_column); + ProbeProcessInfo & probe_process_info); template static std::pair>, std::list *>> probeBlockNullAware( @@ -224,6 +237,7 @@ class JoinPartition BuildPartition build_partition; ProbePartition probe_partition; + JoinArenaPoolPtr pool; ASTTableJoin::Kind kind; ASTTableJoin::Strictness strictness; JoinMapMethod join_map_method; @@ -238,7 +252,6 @@ class JoinPartition /// 2. Rows that are filtered by right join conditions /// For null-aware semi join family, including rows with NULL join keys. std::unique_ptr rows_not_inserted_to_map; - ArenaPtr pool; HashJoinSpillContextPtr hash_join_spill_context; bool has_other_condition; /// only update this field when spill is enabled. todo support this field in non-spill mode diff --git a/dbms/src/Interpreters/ProbeProcessInfo.h b/dbms/src/Interpreters/ProbeProcessInfo.h index 79583fee1e9..9f145fb577e 100644 --- a/dbms/src/Interpreters/ProbeProcessInfo.h +++ b/dbms/src/Interpreters/ProbeProcessInfo.h @@ -38,6 +38,7 @@ struct ProbeProcessInfo size_t start_row; size_t end_row; bool all_rows_joined_finish; + UInt64 cache_columns_threshold; /// these should be inited before probe each block bool prepare_for_probe_done = false; @@ -72,13 +73,14 @@ struct ProbeProcessInfo /// used for left outer semi/left outer anti join bool has_row_null = false; - explicit ProbeProcessInfo(UInt64 max_block_size_) + ProbeProcessInfo(UInt64 max_block_size_, UInt64 cache_columns_threshold_) : partition_index(0) , max_block_size(max_block_size_) , min_result_block_size((max_block_size + 1) / 2) , start_row(0) , end_row(0) - , all_rows_joined_finish(true){}; + , all_rows_joined_finish(true) + , cache_columns_threshold(cache_columns_threshold_){}; void resetBlock(Block && block_, size_t partition_index_ = 0); template diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index d56729ced7c..f46af7d4719 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -39,7 +39,7 @@ struct Settings * but we are not going to do it, because settings is used everywhere as static struct fields. */ -// clang-format off + // clang-format off #define APPLY_FOR_SETTINGS(M) \ M(SettingString, regions, "", "Deprecated. the region need to be read.") \ M(SettingBool, resolve_locks, false, "resolve locks for TiDB transaction") \ @@ -318,7 +318,8 @@ struct Settings M(SettingUInt64, remote_read_queue_size, 0, "size of remote read queue, 0 means it is determined automatically") \ M(SettingBool, enable_cop_stream_for_remote_read, false, "Enable cop stream for remote read") \ M(SettingUInt64, cop_timeout_for_remote_read, 60, "cop timeout seconds for remote read") \ - M(SettingUInt64, auto_spill_check_min_interval_ms, 10, "The minimum interval in millisecond between two successive auto spill check, default value is 100, 0 means no limit") + M(SettingUInt64, auto_spill_check_min_interval_ms, 10, "The minimum interval in millisecond between two successive auto spill check, default value is 100, 0 means no limit") \ + M(SettingUInt64, join_probe_cache_columns_threshold, 1000, "The threshold that a join key will cache its output columns during probe stage, 0 means never cache") // clang-format on diff --git a/dbms/src/Operators/HashJoinProbeTransformOp.cpp b/dbms/src/Operators/HashJoinProbeTransformOp.cpp index d5b341fc518..ee5de46641e 100644 --- a/dbms/src/Operators/HashJoinProbeTransformOp.cpp +++ b/dbms/src/Operators/HashJoinProbeTransformOp.cpp @@ -33,7 +33,7 @@ HashJoinProbeTransformOp::HashJoinProbeTransformOp( const Block & input_header) : TransformOp(exec_context_, req_id) , origin_join(join_) - , probe_process_info(max_block_size) + , probe_process_info(max_block_size, join_->getProbeCacheColumnThreshold()) { RUNTIME_CHECK_MSG(origin_join != nullptr, "join ptr should not be null."); RUNTIME_CHECK_MSG(origin_join->getProbeConcurrency() > 0, "Join probe concurrency must be greater than 0"); @@ -56,7 +56,7 @@ HashJoinProbeTransformOp::HashJoinProbeTransformOp( void HashJoinProbeTransformOp::transformHeaderImpl(Block & header_) { - ProbeProcessInfo header_probe_process_info(0); + ProbeProcessInfo header_probe_process_info(0, 0); header_probe_process_info.resetBlock(std::move(header_)); header_ = origin_join->joinBlock(header_probe_process_info, true); } From f246f350923eb27e1dcfd271b7792481d5551bce Mon Sep 17 00:00:00 2001 From: JaySon Date: Wed, 22 Nov 2023 23:13:12 +0800 Subject: [PATCH 5/6] *: Fix invalid memory access when shutting down (#8409) close pingcap/tiflash#8408 --- dbms/src/Common/MemoryTracker.cpp | 16 ++++++++----- dbms/src/Common/MemoryTracker.h | 24 +++++++++---------- dbms/src/Common/tests/gtest_memtracker.cpp | 9 +++---- dbms/src/Interpreters/ProcessList.cpp | 4 ++-- .../src/Storages/BackgroundProcessingPool.cpp | 3 +-- 5 files changed, 27 insertions(+), 29 deletions(-) diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 29972e4d012..4d9cf4b283c 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -39,7 +39,7 @@ MemoryTracker::~MemoryTracker() if (is_global_root) return; - if (peak) + if (peak && log_peak_memory_usage_in_destructor) { try { @@ -305,19 +305,23 @@ void initStorageMemoryTracker(Int64 limit, Int64 larger_than_limit) "Storage task memory limit={}, larger_than_limit={}", formatReadableSizeWithBinarySuffix(limit), formatReadableSizeWithBinarySuffix(larger_than_limit)); + + // When these (static) mem tracker are reset, it means the process is shutdown and the logging system is stopped. + // Do not log down the usage info in dctor of them. + bool log_in_destructor = false; RUNTIME_CHECK(sub_root_of_query_storage_task_mem_trackers == nullptr); - sub_root_of_query_storage_task_mem_trackers = MemoryTracker::create(limit); + sub_root_of_query_storage_task_mem_trackers = MemoryTracker::create(limit, nullptr, log_in_destructor); sub_root_of_query_storage_task_mem_trackers->setBytesThatRssLargerThanLimit(larger_than_limit); sub_root_of_query_storage_task_mem_trackers->setAmountMetric(CurrentMetrics::MemoryTrackingQueryStorageTask); RUNTIME_CHECK(fetch_pages_mem_tracker == nullptr); - fetch_pages_mem_tracker = MemoryTracker::create(); - fetch_pages_mem_tracker->setNext(sub_root_of_query_storage_task_mem_trackers.get()); + fetch_pages_mem_tracker + = MemoryTracker::create(0, sub_root_of_query_storage_task_mem_trackers.get(), log_in_destructor); fetch_pages_mem_tracker->setAmountMetric(CurrentMetrics::MemoryTrackingFetchPages); RUNTIME_CHECK(shared_column_data_mem_tracker == nullptr); - shared_column_data_mem_tracker = MemoryTracker::create(); - shared_column_data_mem_tracker->setNext(sub_root_of_query_storage_task_mem_trackers.get()); + shared_column_data_mem_tracker + = MemoryTracker::create(0, sub_root_of_query_storage_task_mem_trackers.get(), log_in_destructor); shared_column_data_mem_tracker->setAmountMetric(CurrentMetrics::MemoryTrackingSharedColumnData); } diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index b2a6d782752..62563abf0e5 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -18,6 +18,7 @@ #include #include +#include extern std::atomic real_rss, proc_num_threads, baseline_of_query_mem_tracker; extern std::atomic proc_virt_size; @@ -46,6 +47,7 @@ class MemoryTracker : public std::enable_shared_from_this double fault_probability = 0; bool is_global_root = false; + bool log_peak_memory_usage_in_destructor = true; /// To test the accuracy of memory track, it throws an exception when the part exceeding the tracked amount is greater than accuracy_diff_for_test. std::atomic accuracy_diff_for_test{0}; @@ -65,7 +67,6 @@ class MemoryTracker : public std::enable_shared_from_this std::atomic description = nullptr; /// Make constructors private to ensure all objects of this class is created by `MemoryTracker::create`. - MemoryTracker() = default; explicit MemoryTracker(Int64 limit_) : limit(limit_) {} @@ -79,16 +80,15 @@ class MemoryTracker : public std::enable_shared_from_this public: /// Using `std::shared_ptr` and `new` instread of `std::make_shared` is because `std::make_shared` cannot call private constructors. - static MemoryTrackerPtr create(Int64 limit = 0) + static MemoryTrackerPtr create( + Int64 limit = 0, + MemoryTracker * parent = nullptr, + bool log_peak_memory_usage_in_destructor = true) { - if (limit == 0) - { - return std::shared_ptr(new MemoryTracker); - } - else - { - return std::shared_ptr(new MemoryTracker(limit)); - } + auto p = std::shared_ptr(new MemoryTracker(limit)); + p->setParent(parent); + p->log_peak_memory_usage_in_destructor = log_peak_memory_usage_in_destructor; + return p; } static MemoryTrackerPtr createGlobalRoot() { return std::shared_ptr(new MemoryTracker(0, true)); } @@ -128,7 +128,7 @@ class MemoryTracker : public std::enable_shared_from_this void setAccuracyDiffForTest(Int64 value) { accuracy_diff_for_test.store(value, std::memory_order_relaxed); } /// next should be changed only once: from nullptr to some value. - void setNext(MemoryTracker * elem) + void setParent(MemoryTracker * elem) { MemoryTracker * old_val = nullptr; if (!next.compare_exchange_strong(old_val, elem, std::memory_order_seq_cst, std::memory_order_relaxed)) @@ -187,8 +187,6 @@ void free(Int64 size); } // namespace CurrentMemoryTracker -#include - struct TemporarilyDisableMemoryTracker : private boost::noncopyable { MemoryTracker * memory_tracker; diff --git a/dbms/src/Common/tests/gtest_memtracker.cpp b/dbms/src/Common/tests/gtest_memtracker.cpp index a92c0ff0d45..51fe37a2831 100644 --- a/dbms/src/Common/tests/gtest_memtracker.cpp +++ b/dbms/src/Common/tests/gtest_memtracker.cpp @@ -39,8 +39,7 @@ TEST_F(MemTrackerTest, testRootAndChild) try { auto root_mem_tracker = MemoryTracker::create(); - auto child_mem_tracker = MemoryTracker::create(512); - child_mem_tracker->setNext(root_mem_tracker.get()); + auto child_mem_tracker = MemoryTracker::create(512, root_mem_tracker.get()); // alloc 500 child_mem_tracker->alloc(500); ASSERT_EQ(500, child_mem_tracker->get()); @@ -71,10 +70,8 @@ TEST_F(MemTrackerTest, testRootAndMultipleChild) try { auto root = MemoryTracker::create(512); // limit 512 - auto child1 = MemoryTracker::create(512); // limit 512 - auto child2 = MemoryTracker::create(512); // limit 512 - child1->setNext(root.get()); - child2->setNext(root.get()); + auto child1 = MemoryTracker::create(512, root.get()); // limit 512 + auto child2 = MemoryTracker::create(512, root.get()); // limit 512 // alloc 500 on child1 child1->alloc(500); ASSERT_EQ(500, child1->get()); diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 414ddbff06a..17384b2d747 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -172,7 +172,7 @@ ProcessList::EntryPtr ProcessList::insert( user_process_list.user_memory_tracker->setOrRaiseLimit( settings.max_memory_usage_for_user.getActualBytes(total_memory)); user_process_list.user_memory_tracker->setDescription("(for user)"); - current_memory_tracker->setNext(user_process_list.user_memory_tracker.get()); + current_memory_tracker->setParent(user_process_list.user_memory_tracker.get()); /// Track memory usage for all simultaneously running queries. /// You should specify this value in configuration for default profile, @@ -183,7 +183,7 @@ ProcessList::EntryPtr ProcessList::insert( total_memory_tracker->setBytesThatRssLargerThanLimit(settings.bytes_that_rss_larger_than_limit); total_memory_tracker->setDescription("(total)"); total_memory_tracker->setAccuracyDiffForTest(settings.memory_tracker_accuracy_diff_for_test); - user_process_list.user_memory_tracker->setNext(total_memory_tracker.get()); + user_process_list.user_memory_tracker->setParent(total_memory_tracker.get()); } if (!total_network_throttler && settings.max_network_bandwidth_for_all_users) diff --git a/dbms/src/Storages/BackgroundProcessingPool.cpp b/dbms/src/Storages/BackgroundProcessingPool.cpp index 6b8dfebf87e..4ea1181a4d0 100644 --- a/dbms/src/Storages/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/BackgroundProcessingPool.cpp @@ -213,8 +213,7 @@ void BackgroundProcessingPool::threadFunction(size_t thread_idx) noexcept } // set up the thread local memory tracker - auto memory_tracker = MemoryTracker::create(); - memory_tracker->setNext(root_of_non_query_mem_trackers.get()); + auto memory_tracker = MemoryTracker::create(0, root_of_non_query_mem_trackers.get()); memory_tracker->setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); current_memory_tracker = memory_tracker.get(); From d1949c6a206dfed958c38eceed9f7436aac24494 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 23 Nov 2023 14:38:41 +0800 Subject: [PATCH 6/6] Don't log too many useless information in log (#8400) close pingcap/tiflash#8399 --- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 6 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 8 +- dbms/src/Flash/FlashService.cpp | 78 ++++++++++++------- dbms/src/Flash/Mpp/MPPTaskId.cpp | 5 +- dbms/src/Flash/Mpp/MPPTaskId.h | 16 +--- dbms/src/Flash/Mpp/MPPTaskManager.cpp | 8 +- dbms/src/Flash/Planner/PhysicalPlanNode.cpp | 3 +- dbms/src/Flash/Planner/PhysicalPlanNode.h | 1 + .../Planner/Plans/PhysicalAggregation.cpp | 8 +- .../Plans/PhysicalAggregationBuild.cpp | 2 +- dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp | 4 +- 11 files changed, 79 insertions(+), 60 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 7da090989f3..67947e74ca4 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -301,7 +301,7 @@ void DAGQueryBlockInterpreter::handleJoin( const Settings & settings = context.getSettingsRef(); SpillConfig build_spill_config( context.getTemporaryPath(), - fmt::format("{}_hash_join_0_build", log->identifier()), + fmt::format("{}_0_build", log->identifier()), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, @@ -310,7 +310,7 @@ void DAGQueryBlockInterpreter::handleJoin( settings.max_block_size); SpillConfig probe_spill_config( context.getTemporaryPath(), - fmt::format("{}_hash_join_0_probe", log->identifier()), + fmt::format("{}_0_probe", log->identifier()), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, @@ -497,7 +497,7 @@ void DAGQueryBlockInterpreter::executeAggregation( AggregationInterpreterHelper::fillArgColumnNumbers(aggregate_descriptions, before_agg_header); SpillConfig spill_config( context.getTemporaryPath(), - fmt::format("{}_aggregation", log->identifier()), + log->identifier(), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 4d1e273b9fa..646e388a4e8 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -207,7 +207,7 @@ void orderStreams( getAverageThreshold(settings.max_bytes_before_external_sort, pipeline.streams.size()), SpillConfig( context.getTemporaryPath(), - fmt::format("{}_sort", log->identifier()), + log->identifier(), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, @@ -239,7 +239,7 @@ void orderStreams( // todo use identifier_executor_id as the spill id SpillConfig( context.getTemporaryPath(), - fmt::format("{}_sort", log->identifier()), + log->identifier(), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, @@ -295,7 +295,7 @@ void executeLocalSort( fine_grained_spill_context = std::make_shared("sort", log); SpillConfig spill_config{ context.getTemporaryPath(), - fmt::format("{}_sort", log->identifier()), + log->identifier(), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, @@ -353,7 +353,7 @@ void executeFinalSort( SpillConfig spill_config{ context.getTemporaryPath(), - fmt::format("{}_sort", log->identifier()), + log->identifier(), settings.max_cached_data_bytes_in_spiller, settings.max_spilled_rows_per_file, settings.max_spilled_bytes_per_file, diff --git a/dbms/src/Flash/FlashService.cpp b/dbms/src/Flash/FlashService.cpp index 2950a3f1432..23ddd334c33 100644 --- a/dbms/src/Flash/FlashService.cpp +++ b/dbms/src/Flash/FlashService.cpp @@ -206,14 +206,16 @@ grpc::Status FlashService::Coprocessor( request->context().region_id(), request->context().region_epoch().conf_ver(), request->context().region_epoch().version()); - auto log_level = is_remote_read ? Poco::Message::PRIO_INFORMATION : Poco::Message::PRIO_DEBUG; - LOG_IMPL( + LOG_INFO( log, - log_level, - "Handling coprocessor request, is_remote_read: {}, start ts: {}, region info: {}", + "Handling coprocessor request, is_remote_read: {}, start ts: {}, region info: {}, resource_group: {}, conn_id: " + "{}, conn_alias: {}", is_remote_read, request->start_ts(), - region_info); + region_info, + request->context().resource_control_context().resource_group_name(), + request->connection_id(), + request->connection_alias()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) @@ -255,6 +257,7 @@ grpc::Status FlashService::Coprocessor( } } + auto log_level = is_remote_read ? Poco::Message::PRIO_INFORMATION : Poco::Message::PRIO_DEBUG; auto exec_func = [&]() -> grpc::Status { auto wait_ms = watch.elapsedMilliseconds(); if (wait_ms > 1000) @@ -279,14 +282,10 @@ grpc::Status FlashService::Coprocessor( }); CoprocessorContext cop_context(*db_context, request->context(), *grpc_context); auto request_identifier = fmt::format( - "Coprocessor, is_remote_read: {}, start_ts: {}, region_info: {}, resource_group: {}, conn_id: {}, " - "conn_alias: {}", + "Coprocessor, is_remote_read: {}, start_ts: {}, region_info: {}", is_remote_read, request->start_ts(), - region_info, - request->context().resource_control_context().resource_group_name(), - request->connection_id(), - request->connection_alias()); + region_info); CoprocessorHandler cop_handler(cop_context, request, response, request_identifier); return cop_handler.execute(); }; @@ -318,7 +317,13 @@ grpc::Status FlashService::BatchCoprocessor( grpc::ServerWriter * writer) { CPUAffinityManager::getInstance().bindSelfGrpcThread(); - LOG_INFO(log, "Handling batch coprocessor request, start ts: {}", request->start_ts()); + LOG_INFO( + log, + "Handling batch coprocessor request, start ts: {}, resource_group: {}, conn_id: {}, conn_alias: {}", + request->start_ts(), + request->context().resource_control_context().resource_group_name(), + request->connection_id(), + request->connection_alias()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) @@ -342,12 +347,7 @@ grpc::Status FlashService::BatchCoprocessor( return status; } CoprocessorContext cop_context(*db_context, request->context(), *grpc_context); - auto request_identifier = fmt::format( - "BatchCoprocessor, start_ts: {}, resource_group: {}, conn_id: {}, conn_alias: {}", - request->start_ts(), - request->context().resource_control_context().resource_group_name(), - request->connection_id(), - request->connection_alias()); + auto request_identifier = fmt::format("BatchCoprocessor, start_ts: {}", request->start_ts()); BatchCoprocessorHandler cop_handler(cop_context, request, writer, request_identifier); return cop_handler.execute(); }); @@ -372,14 +372,16 @@ grpc::Status FlashService::CoprocessorStream( request->context().region_id(), request->context().region_epoch().conf_ver(), request->context().region_epoch().version()); - auto log_level = is_remote_read ? Poco::Message::PRIO_INFORMATION : Poco::Message::PRIO_DEBUG; - LOG_IMPL( + LOG_INFO( log, - log_level, - "Handling coprocessor stream request, is_remote_read: {}, start ts: {}, region info: {}", + "Handling coprocessor stream request, is_remote_read: {}, start ts: {}, region info: {}, resource_group: {}, " + "conn_id: {}, conn_alias: {}", is_remote_read, request->start_ts(), - region_info); + region_info, + request->context().resource_control_context().resource_group_name(), + request->connection_id(), + request->connection_alias()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) @@ -425,6 +427,7 @@ grpc::Status FlashService::CoprocessorStream( } } + auto log_level = is_remote_read ? Poco::Message::PRIO_INFORMATION : Poco::Message::PRIO_DEBUG; auto exec_func = [&]() -> grpc::Status { auto wait_ms = watch.elapsedMilliseconds(); if (wait_ms > 1000) @@ -449,11 +452,10 @@ grpc::Status FlashService::CoprocessorStream( }); CoprocessorContext cop_context(*db_context, request->context(), *grpc_context); auto request_identifier = fmt::format( - "Coprocessor(stream), is_remote_read: {}, start_ts: {}, region_info: {}, resource_group: {}", + "Coprocessor(stream), is_remote_read: {}, start_ts: {}, region_info: {}", is_remote_read, request->start_ts(), - region_info, - request->context().resource_control_context().resource_group_name()); + region_info); CoprocessorHandler cop_handler(cop_context, request, writer, request_identifier); return cop_handler.execute(); }; @@ -488,7 +490,14 @@ grpc::Status FlashService::DispatchMPPTask( mpp::DispatchTaskResponse * response) { CPUAffinityManager::getInstance().bindSelfGrpcThread(); - LOG_INFO(log, "Handling mpp dispatch request, task meta: {}", request->meta().DebugString()); + const auto & task_meta = request->meta(); + LOG_INFO( + log, + "Handling mpp dispatch request, task: {}, resource_group: {}, conn_id: {}, conn_alias: {}", + MPPTaskId(task_meta).toString(), + task_meta.resource_group_name(), + task_meta.connection_id(), + task_meta.connection_alias()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) return check_result; @@ -615,7 +624,20 @@ grpc::Status FlashService::EstablishMPPConnection( CPUAffinityManager::getInstance().bindSelfGrpcThread(); // Establish a pipe for data transferring. The pipes have registered by the task in advance. // We need to find it out and bind the grpc stream with it. - LOG_INFO(log, "Handling establish mpp connection request: {}", request->DebugString()); + const auto & receiver_meta = request->receiver_meta(); + const auto & sender_meta = request->sender_meta(); + assert(receiver_meta.resource_group_name() == sender_meta.resource_group_name()); + assert(receiver_meta.connection_id() == sender_meta.connection_id()); + assert(receiver_meta.connection_alias() == receiver_meta.connection_alias()); + LOG_INFO( + log, + "Handling establish mpp connection request, receiver: {}, sender: {}, resource_group: {}, conn_id: {}, " + "conn_alias: {}", + MPPTaskId(receiver_meta).toString(), + MPPTaskId(sender_meta).toString(), + receiver_meta.resource_group_name(), + receiver_meta.connection_id(), + receiver_meta.connection_alias()); auto check_result = checkGrpcContext(grpc_context); if (!check_result.ok()) diff --git a/dbms/src/Flash/Mpp/MPPTaskId.cpp b/dbms/src/Flash/Mpp/MPPTaskId.cpp index 3940675ed5b..168ab7ac871 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskId.cpp @@ -101,14 +101,13 @@ size_t MPPGatherIdHash::operator()(MPPGatherId const & mpp_gather_id) const noex String MPPTaskId::toString() const { return isUnknown() ? "MPP" - : fmt::format("MPP", gather_id.toString(), task_id); + : fmt::format("MPP<{},task_id:{}>", gather_id.toString(), task_id); } const MPPTaskId MPPTaskId::unknown_mpp_task_id = MPPTaskId{}; constexpr UInt64 MAX_UINT64 = std::numeric_limits::max(); -const MPPQueryId MPPTaskId::Max_Query_Id - = MPPQueryId(MAX_UINT64, MAX_UINT64, MAX_UINT64, MAX_UINT64, "", MAX_UINT64, ""); +const MPPQueryId MPPTaskId::Max_Query_Id = MPPQueryId(MAX_UINT64, MAX_UINT64, MAX_UINT64, MAX_UINT64, "", 0, ""); bool operator==(const MPPTaskId & lid, const MPPTaskId & rid) { diff --git a/dbms/src/Flash/Mpp/MPPTaskId.h b/dbms/src/Flash/Mpp/MPPTaskId.h index 783f5fd7b69..d9e0946424f 100644 --- a/dbms/src/Flash/Mpp/MPPTaskId.h +++ b/dbms/src/Flash/Mpp/MPPTaskId.h @@ -64,15 +64,11 @@ struct MPPQueryId String toString() const { return fmt::format( - "", + "", query_ts, local_query_id, server_id, - start_ts, - resource_group_name, - connection_id, - connection_alias); + start_ts); } }; @@ -113,16 +109,12 @@ struct MPPGatherId String toString() const { return fmt::format( - "", + "gather_id:{}, query_ts:{}, local_query_id:{}, server_id:{}, start_ts:{}", gather_id, query_id.query_ts, query_id.local_query_id, query_id.server_id, - query_id.start_ts, - query_id.resource_group_name, - query_id.connection_id, - query_id.connection_alias); + query_id.start_ts); } bool hasMeaningfulGatherId() const { return gather_id > 0; } bool operator==(const MPPGatherId & rid) const; diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.cpp b/dbms/src/Flash/Mpp/MPPTaskManager.cpp index bb42cd3871d..cf5a3ba0ee8 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskManager.cpp @@ -147,7 +147,7 @@ std::pair MPPTaskManager::findAsyncTunnel( if (!error_msg.empty()) { /// if the gather is aborted, return the error message - LOG_WARNING(log, "{}: Gather {} is aborted, all its tasks are invalid.", req_info, id.gather_id.toString()); + LOG_WARNING(log, "{}: Gather <{}> is aborted, all its tasks are invalid.", req_info, id.gather_id.toString()); /// meet error return {nullptr, error_msg}; } @@ -229,7 +229,11 @@ std::pair MPPTaskManager::findTunnelWithTimeout( if (!error_msg.empty()) { /// if the gather is aborted, return true to stop waiting timeout. - LOG_WARNING(log, "{}: Gather {} is aborted, all its tasks are invalid.", req_info, id.gather_id.toString()); + LOG_WARNING( + log, + "{}: Gather <{}> is aborted, all its tasks are invalid.", + req_info, + id.gather_id.toString()); cancelled = true; error_message = error_msg; return true; diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp index 4ba718337a3..8b8849530bc 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.cpp @@ -30,8 +30,9 @@ PhysicalPlanNode::PhysicalPlanNode( const PlanType & type_, const NamesAndTypes & schema_, const FineGrainedShuffle & fine_grained_shuffle_, - const String & req_id) + const String & req_id_) : executor_id(executor_id_) + , req_id(req_id_) , type(type_) , schema(schema_) , fine_grained_shuffle(fine_grained_shuffle_) diff --git a/dbms/src/Flash/Planner/PhysicalPlanNode.h b/dbms/src/Flash/Planner/PhysicalPlanNode.h index 7bc5134923c..cfc96724e85 100644 --- a/dbms/src/Flash/Planner/PhysicalPlanNode.h +++ b/dbms/src/Flash/Planner/PhysicalPlanNode.h @@ -116,6 +116,7 @@ class PhysicalPlanNode : public std::enable_shared_from_this void recordProfileStreams(DAGPipeline & pipeline, const Context & context); String executor_id; + String req_id; PlanType type; NamesAndTypes schema; diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp index 25f3e3892c3..3cf5b5fc3d7 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp @@ -100,7 +100,7 @@ void PhysicalAggregation::buildBlockInputStreamImpl(DAGPipeline & pipeline, Cont AggregationInterpreterHelper::fillArgColumnNumbers(aggregate_descriptions, before_agg_header); SpillConfig spill_config( context.getTemporaryPath(), - fmt::format("{}_aggregation", log->identifier()), + log->identifier(), context.getSettingsRef().max_cached_data_bytes_in_spiller, context.getSettingsRef().max_spilled_rows_per_file, context.getSettingsRef().max_spilled_bytes_per_file, @@ -216,7 +216,7 @@ void PhysicalAggregation::buildPipelineExecGroupImpl( AggregationInterpreterHelper::fillArgColumnNumbers(aggregate_descriptions, before_agg_header); SpillConfig spill_config( context.getTemporaryPath(), - fmt::format("{}_aggregation", log->identifier()), + log->identifier(), context.getSettingsRef().max_cached_data_bytes_in_spiller, context.getSettingsRef().max_spilled_rows_per_file, context.getSettingsRef().max_spilled_bytes_per_file, @@ -264,7 +264,7 @@ void PhysicalAggregation::buildPipeline( auto agg_build = std::make_shared( executor_id, schema, - log->identifier(), + req_id, child, before_agg_actions, aggregation_keys, @@ -281,7 +281,7 @@ void PhysicalAggregation::buildPipeline( auto agg_convergent = std::make_shared( executor_id, schema, - log->identifier(), + req_id, aggregate_context, expr_after_agg); builder.addPlanNode(agg_convergent); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp b/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp index 43b6890858e..2c2d521250e 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregationBuild.cpp @@ -40,7 +40,7 @@ void PhysicalAggregationBuild::buildPipelineExecGroupImpl( AggregationInterpreterHelper::fillArgColumnNumbers(aggregate_descriptions, before_agg_header); SpillConfig spill_config( context.getTemporaryPath(), - fmt::format("{}_aggregation", log->identifier()), + log->identifier(), context.getSettingsRef().max_cached_data_bytes_in_spiller, context.getSettingsRef().max_spilled_rows_per_file, context.getSettingsRef().max_spilled_bytes_per_file, diff --git a/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp index f07ddc499ff..be2c1bc3e2c 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalJoin.cpp @@ -303,7 +303,7 @@ void PhysicalJoin::buildPipeline(PipelineBuilder & builder, Context & context, P executor_id, build()->getSchema(), fine_grained_shuffle, - log->identifier(), + req_id, build(), join_ptr, build_side_prepare_actions); @@ -317,7 +317,7 @@ void PhysicalJoin::buildPipeline(PipelineBuilder & builder, Context & context, P auto join_probe = std::make_shared( executor_id, schema, - log->identifier(), + req_id, probe(), join_ptr, probe_side_prepare_actions);