diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index f7d881f0d06..993c455d854 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -77,6 +77,7 @@ namespace DB M(skip_check_segment_update) \ M(force_set_page_file_write_errno) \ M(force_split_io_size_4k) \ + M(force_set_num_regions_for_table) \ M(minimum_block_size_for_cross_join) \ M(random_exception_after_dt_write_done) \ M(random_slow_page_storage_write) \ diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index be65513ab68..fd7e91e9037 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -64,17 +64,47 @@ MockTiDB::MockTiDB() databases["default"] = 0; } -TablePtr MockTiDB::dropTableInternal(Context & context, const String & database_name, const String & table_name, bool drop_regions) +TablePtr MockTiDB::dropTableByNameImpl(Context & context, const String & database_name, const String & table_name, bool drop_regions) { String qualified_name = database_name + "." + table_name; auto it_by_name = tables_by_name.find(qualified_name); if (it_by_name == tables_by_name.end()) return nullptr; + auto table = it_by_name->second; + dropTableInternal(context, table, drop_regions); + + tables_by_name.erase(it_by_name); + return table; +} + +TablePtr MockTiDB::dropTableByIdImpl(Context & context, const TableID table_id, bool drop_regions) +{ + auto iter = tables_by_id.find(table_id); + if (iter == tables_by_id.end()) + return nullptr; + + auto table = iter->second; + dropTableInternal(context, table, drop_regions); + + // erase from `tables_by_name` + for (auto iter_by_name = tables_by_name.begin(); iter_by_name != tables_by_name.end(); /* empty */) + { + if (table != iter_by_name->second) + { + ++iter_by_name; + continue; + } + LOG_INFO(Logger::get(), "removing table from MockTiDB, name={} table_id={}", iter_by_name->first, table_id); + iter_by_name = tables_by_name.erase(iter_by_name); + } + return table; +} +TablePtr MockTiDB::dropTableInternal(Context & context, const TablePtr & table, bool drop_regions) +{ auto & kvstore = context.getTMTContext().getKVStore(); auto & region_table = context.getTMTContext().getRegionTable(); - auto table = it_by_name->second; if (table->isPartitionTable()) { for (const auto & partition : table->table_info.partition.definitions) @@ -90,15 +120,12 @@ TablePtr MockTiDB::dropTableInternal(Context & context, const String & database_ } tables_by_id.erase(table->id()); - tables_by_name.erase(it_by_name); - if (drop_regions) { for (auto & e : region_table.getRegionsByTable(NullspaceID, table->id())) kvstore->mockRemoveRegion(e.first, region_table); region_table.removeTable(NullspaceID, table->id()); } - return table; } @@ -113,7 +140,7 @@ void MockTiDB::dropDB(Context & context, const String & database_name, bool drop }); for (const auto & table_name : table_names) - dropTableInternal(context, database_name, table_name, drop_regions); + dropTableByNameImpl(context, database_name, table_name, drop_regions); version++; @@ -132,8 +159,25 @@ void MockTiDB::dropDB(Context & context, const String & database_name, bool drop void MockTiDB::dropTable(Context & context, const String & database_name, const String & table_name, bool drop_regions) { std::lock_guard lock(tables_mutex); + auto table = dropTableByNameImpl(context, database_name, table_name, drop_regions); + if (!table) + return; + + version++; - auto table = dropTableInternal(context, database_name, table_name, drop_regions); + SchemaDiff diff; + diff.type = SchemaActionType::DropTable; + diff.schema_id = table->database_id; + diff.table_id = table->id(); + diff.version = version; + version_diff[version] = diff; +} + +void MockTiDB::dropTableById(Context & context, const TableID & table_id, bool drop_regions) +{ + std::lock_guard lock(tables_mutex); + + auto table = dropTableByIdImpl(context, table_id, drop_regions); if (!table) return; @@ -273,13 +317,15 @@ TableID MockTiDB::newTable( return addTable(database_name, std::move(*table_info)); } -int MockTiDB::newTables( +std::vector MockTiDB::newTables( const String & database_name, const std::vector> & tables, Timestamp tso, const String & engine_type) { - std::lock_guard lock(tables_mutex); + std::scoped_lock lock(tables_mutex); + std::vector table_ids; + table_ids.reserve(tables.size()); if (databases.find(database_name) == databases.end()) { throw Exception("MockTiDB not found db: " + database_name, ErrorCodes::LOGICAL_ERROR); @@ -300,7 +346,8 @@ int MockTiDB::newTables( table_info.id = table_id_allocator++; table_info.update_timestamp = tso; - auto table = std::make_shared(database_name, databases[database_name], table_info.name, std::move(table_info)); + auto table + = std::make_shared
(database_name, databases[database_name], table_info.name, std::move(table_info)); tables_by_id.emplace(table->table_info.id, table); tables_by_name.emplace(qualified_name, table); @@ -310,6 +357,8 @@ int MockTiDB::newTables( opt.old_schema_id = table->database_id; opt.old_table_id = table->id(); diff.affected_opts.push_back(std::move(opt)); + + table_ids.push_back(table->id()); } if (diff.affected_opts.empty()) @@ -318,7 +367,8 @@ int MockTiDB::newTables( diff.schema_id = diff.affected_opts[0].schema_id; diff.version = version; version_diff[version] = diff; - return 0; + + return table_ids; } TableID MockTiDB::addTable(const String & database_name, TiDB::TableInfo && table_info) diff --git a/dbms/src/Debug/MockTiDB.h b/dbms/src/Debug/MockTiDB.h index ce9fc18b764..82f3f6e8291 100644 --- a/dbms/src/Debug/MockTiDB.h +++ b/dbms/src/Debug/MockTiDB.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include #include @@ -82,7 +83,7 @@ class MockTiDB : public ext::Singleton const String & handle_pk_name, const String & engine_type); - int newTables( + std::vector newTables( const String & database_name, const std::vector> & tables, Timestamp tso, @@ -104,6 +105,7 @@ class MockTiDB : public ext::Singleton void dropPartition(const String & database_name, const String & table_name, TableID partition_id); void dropTable(Context & context, const String & database_name, const String & table_name, bool drop_regions); + void dropTableById(Context & context, const TableID & table_id, bool drop_regions); void dropDB(Context & context, const String & database_name, bool drop_regions); @@ -151,7 +153,9 @@ class MockTiDB : public ext::Singleton private: TableID newPartitionImpl(const TablePtr & logical_table, TableID partition_id, const String & partition_name, Timestamp tso, bool is_add_part); - TablePtr dropTableInternal(Context & context, const String & database_name, const String & table_name, bool drop_regions); + TablePtr dropTableByNameImpl(Context & context, const String & database_name, const String & table_name, bool drop_regions); + TablePtr dropTableByIdImpl(Context & context, TableID table_id, bool drop_regions); + TablePtr dropTableInternal(Context & context, const TablePtr & table, bool drop_regions); TablePtr getTableByNameInternal(const String & database_name, const String & table_name); TablePtr getTableByID(TableID table_id); diff --git a/dbms/src/Debug/dbgFuncSchema.cpp b/dbms/src/Debug/dbgFuncSchema.cpp index 090cfc4c53b..a3bf9e17a52 100644 --- a/dbms/src/Debug/dbgFuncSchema.cpp +++ b/dbms/src/Debug/dbgFuncSchema.cpp @@ -86,16 +86,20 @@ void dbgFuncRefreshSchemas(Context & context, const ASTs &, DBGInvoker::Printer // Trigger gc on all databases / tables. // Usage: -// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point])" +// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point, ignore_remain_regions])" void dbgFuncGcSchemas(Context & context, const ASTs & args, DBGInvoker::Printer output) { auto & service = context.getSchemaSyncService(); Timestamp gc_safe_point = 0; + bool ignore_remain_regions = false; if (args.empty()) gc_safe_point = PDClientHelper::getGCSafePointWithRetry(context.getTMTContext().getPDClient()); - else + if (!args.empty()) gc_safe_point = safeGet(typeid_cast(*args[0]).value); - service->gc(gc_safe_point, NullspaceID); + if (args.size() >= 2) + ignore_remain_regions = safeGet(typeid_cast(*args[1]).value) == "true"; + // Note that only call it in tests, we need to ignore remain regions + service->gcImpl(gc_safe_point, NullspaceID, ignore_remain_regions); output("schemas gc done"); } @@ -139,4 +143,4 @@ void dbgFuncIsTombstone(Context & context, const ASTs & args, DBGInvoker::Printe } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Debug/dbgFuncSchema.h b/dbms/src/Debug/dbgFuncSchema.h index a92dc2418ab..46c3e5e8450 100644 --- a/dbms/src/Debug/dbgFuncSchema.h +++ b/dbms/src/Debug/dbgFuncSchema.h @@ -34,7 +34,7 @@ void dbgFuncRefreshSchemas(Context & context, const ASTs & args, DBGInvoker::Pri // Trigger gc on all databases / tables. // Usage: -// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point])" +// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point, ignore_remain_regions])" void dbgFuncGcSchemas(Context & context, const ASTs & args, DBGInvoker::Printer output); // Reset schemas. diff --git a/dbms/src/Storages/Transaction/RegionTable.cpp b/dbms/src/Storages/Transaction/RegionTable.cpp index f3dbcb38ac5..52bcbf884c4 100644 --- a/dbms/src/Storages/Transaction/RegionTable.cpp +++ b/dbms/src/Storages/Transaction/RegionTable.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -26,7 +27,11 @@ #include #include #include +#include #include +#include + +#include namespace DB { @@ -37,6 +42,10 @@ extern const int UNKNOWN_TABLE; extern const int ILLFORMAT_RAFT_ROW; extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes +namespace FailPoints +{ +extern const char force_set_num_regions_for_table[]; +} // namespace FailPoints RegionTable::Table & RegionTable::getOrCreateTable(const KeyspaceID keyspace_id, const TableID table_id) { @@ -285,8 +294,8 @@ void RegionTable::removeRegion(const RegionID region_id, bool remove_data, const { tables.erase(ks_tb_id); } - LOG_INFO(log, "remove [region {}] in RegionTable done", region_id); } + LOG_INFO(log, "remove [region {}] in RegionTable done", region_id); // Sometime we don't need to remove data. e.g. remove region after region merge. if (remove_data) @@ -431,6 +440,31 @@ void RegionTable::handleInternalRegionsByTable(const KeyspaceID keyspace_id, con } } +std::vector RegionTable::getRegionIdsByTable(KeyspaceID keyspace_id, TableID table_id) const +{ + fiu_do_on(FailPoints::force_set_num_regions_for_table, { + if (auto v = FailPointHelper::getFailPointVal(FailPoints::force_set_num_regions_for_table); v) + { + auto num_regions = std::any_cast>(v.value()); + return num_regions; + } + }); + + std::lock_guard lock(mutex); + if (auto iter = tables.find(KeyspaceTableID{keyspace_id, table_id}); // + unlikely(iter != tables.end())) + { + std::vector ret_regions; + ret_regions.reserve(iter->second.regions.size()); + for (const auto & r : iter->second.regions) + { + ret_regions.emplace_back(r.first); + } + return ret_regions; + } + return {}; +} + std::vector> RegionTable::getRegionsByTable(const KeyspaceID keyspace_id, const TableID table_id) const { auto & kvstore = context->getTMTContext().getKVStore(); diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index 95bcbd76aca..69ff456c8b0 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -170,6 +170,8 @@ class RegionTable : private boost::noncopyable RegionDataReadInfoList tryWriteBlockByRegionAndFlush(const RegionPtrWithBlock & region, bool try_persist); void handleInternalRegionsByTable(KeyspaceID keyspace_id, TableID table_id, std::function && callback) const; + + std::vector getRegionIdsByTable(KeyspaceID keyspace_id, TableID table_id) const; std::vector> getRegionsByTable(KeyspaceID keyspace_id, TableID table_id) const; /// Write the data of the given region into the table with the given table ID, fill the data list for outer to remove. diff --git a/dbms/src/TiDB/Schema/SchemaSyncService.cpp b/dbms/src/TiDB/Schema/SchemaSyncService.cpp index 54cd4231f13..2fe1109188a 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncService.cpp +++ b/dbms/src/TiDB/Schema/SchemaSyncService.cpp @@ -41,15 +41,26 @@ SchemaSyncService::SchemaSyncService(DB::Context & context_) , log(Logger::get()) { // Add task for adding and removing keyspace sync schema tasks. - handle = background_pool.addTask( - [&, this] { - addKeyspaceGCTasks(); - removeKeyspaceGCTasks(); - - return false; - }, - false, - interval_seconds * 1000); + auto interval_ms = interval_seconds * 1000; + if (interval_ms == 0) + { + LOG_WARNING( + log, + "The background task of SchemaSyncService is disabled, please check the ddl_sync_interval_seconds " + "settings"); + } + else + { + handle = background_pool.addTask( + [&, this] { + addKeyspaceGCTasks(); + removeKeyspaceGCTasks(); + + return false; + }, + false, + interval_ms); + } } void SchemaSyncService::addKeyspaceGCTasks() @@ -142,7 +153,7 @@ void SchemaSyncService::removeKeyspaceGCTasks() LOG_IMPL(log, log_level, "remove sync schema task for keyspaces done, num_remove_tasks={}", num_remove_tasks); } -SchemaSyncService::~SchemaSyncService() +void SchemaSyncService::shutdown() { background_pool.removeTask(handle); for (auto const & iter : ks_handle_map) @@ -153,6 +164,11 @@ SchemaSyncService::~SchemaSyncService() LOG_INFO(log, "SchemaSyncService stopped"); } +SchemaSyncService::~SchemaSyncService() +{ + shutdown(); +} + bool SchemaSyncService::syncSchemas(KeyspaceID keyspace_id) { return context.getTMTContext().getSchemaSyncer()->syncSchemas(context, keyspace_id); @@ -187,6 +203,11 @@ void SchemaSyncService::updateLastGcSafepoint(KeyspaceID keyspace_id, Timestamp } bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) +{ + return gcImpl(gc_safepoint, keyspace_id, /*ignore_remain_regions*/ false); +} + +bool SchemaSyncService::gcImpl(Timestamp gc_safepoint, KeyspaceID keyspace_id, bool ignore_remain_regions) { const std::optional last_gc_safepoint = lastGcSafePoint(keyspace_id); // for new deploy cluster, there is an interval that gc_safepoint return 0, skip it @@ -242,6 +263,7 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } } + auto & tmt_context = context.getTMTContext(); // Physically drop tables bool succeeded = true; for (auto & storage_ptr : storages_to_gc) @@ -267,6 +289,38 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) *database_id, table_info.id); }(); + + auto & region_table = tmt_context.getRegionTable(); + if (auto remain_regions = region_table.getRegionIdsByTable(keyspace_id, table_info.id); // + !remain_regions.empty()) + { + if (likely(!ignore_remain_regions)) + { + LOG_WARNING( + keyspace_log, + "Physically drop table is skip, regions are not totally removed from TiFlash, remain_region_ids={}" + " table_tombstone={} safepoint={} {}", + remain_regions, + storage->getTombstone(), + gc_safepoint, + canonical_name); + succeeded = false; // dropping this table is skipped, do not succee the `last_gc_safepoint` + continue; + } + else + { + LOG_WARNING( + keyspace_log, + "Physically drop table is executed while regions are not totally removed from TiFlash," + " remain_region_ids={} ignore_remain_regions={} table_tombstone={} safepoint={} {} ", + remain_regions, + ignore_remain_regions, + storage->getTombstone(), + gc_safepoint, + canonical_name); + } + } + LOG_INFO( keyspace_log, "Physically drop table begin, table_tombstone={} safepoint={} {}", @@ -288,7 +342,7 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } catch (DB::Exception & e) { - succeeded = false; + succeeded = false; // dropping this table is skipped, do not succee the `last_gc_safepoint` String err_msg; // Maybe a read lock of a table is held for a long time, just ignore it this round. if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) @@ -338,7 +392,7 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } catch (DB::Exception & e) { - succeeded = false; + succeeded = false; // dropping this database is skipped, do not succee the `last_gc_safepoint` String err_msg; if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) err_msg = "locking attempt has timed out!"; // ignore verbose stack for this error @@ -348,6 +402,8 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) } } + // TODO: Optimize it after `BackgroundProcessingPool` can the task return how many seconds to sleep + // before next round. if (succeeded) { updateLastGcSafepoint(keyspace_id, gc_safepoint); @@ -357,6 +413,11 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) num_tables_removed, num_databases_removed, gc_safepoint); + // This round of GC could run for a long time. Run immediately to check whether + // the latest gc_safepoint has been updated in PD. + // - gc_safepoint is not updated, it will be skipped because gc_safepoint == last_gc_safepoint + // - gc_safepoint is updated, run again immediately to cleanup other dropped data + return true; } else { @@ -366,9 +427,10 @@ bool SchemaSyncService::gc(Timestamp gc_safepoint, KeyspaceID keyspace_id) "Schema GC meet error, will try again later, last_safepoint={} safepoint={}", last_gc_safepoint_str, gc_safepoint); + // Return false to let it run again after `ddl_sync_interval_seconds` even if the gc_safepoint + // on PD is not updated. + return false; } - - return true; } } // namespace DB diff --git a/dbms/src/TiDB/Schema/SchemaSyncService.h b/dbms/src/TiDB/Schema/SchemaSyncService.h index 77257ace6e0..20e2e67f28f 100644 --- a/dbms/src/TiDB/Schema/SchemaSyncService.h +++ b/dbms/src/TiDB/Schema/SchemaSyncService.h @@ -35,6 +35,11 @@ using ASTs = std::vector; using DBGInvokerPrinter = std::function; extern void dbgFuncGcSchemas(Context &, const ASTs &, DBGInvokerPrinter); +namespace tests +{ +class SchemaSyncTest; +} + class SchemaSyncService : public std::enable_shared_from_this , private boost::noncopyable @@ -43,17 +48,22 @@ class SchemaSyncService explicit SchemaSyncService(Context & context_); ~SchemaSyncService(); + friend class tests::SchemaSyncTest; + bool gc(Timestamp gc_safepoint, KeyspaceID keyspace_id); + + void shutdown(); + private: bool syncSchemas(KeyspaceID keyspace_id); void removeCurrentVersion(KeyspaceID keyspace_id); - bool gc(Timestamp gc_safepoint, KeyspaceID keyspace_id); void addKeyspaceGCTasks(); void removeKeyspaceGCTasks(); std::optional lastGcSafePoint(KeyspaceID keyspace_id) const; void updateLastGcSafepoint(KeyspaceID keyspace_id, Timestamp gc_safepoint); + bool gcImpl(Timestamp gc_safepoint, KeyspaceID keyspace_id, bool ignore_remain_regions); private: Context & context; diff --git a/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp b/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp index c0613e82696..8c5cf561b55 100644 --- a/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp +++ b/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp @@ -39,6 +39,7 @@ namespace FailPoints extern const char exception_before_rename_table_old_meta_removed[]; extern const char force_schema_sync_too_old_schema[]; extern const char force_context_path[]; +extern const char force_set_num_regions_for_table[]; } // namespace FailPoints namespace tests { @@ -167,6 +168,11 @@ class SchemaSyncTest : public ::testing::Test drop_interpreter.execute(); } + static std::optional lastGcSafePoint(const SchemaSyncServicePtr & sync_service, KeyspaceID keyspace_id) + { + return sync_service->lastGcSafePoint(keyspace_id); + } + private: static void recreateMetadataPath() { @@ -187,7 +193,7 @@ try // Note that if we want to add new fields here, please firstly check if it is present. // Otherwise it will break when doing upgrading test. SchemaDiff diff; - std::string data = "{\"version\":40,\"type\":31,\"schema_id\":69,\"table_id\":71,\"old_table_id\":0,\"old_schema_id\":0,\"affected_options\":null}"; + std::string data = R"({"version":40,"type":31,"schema_id":69,"table_id":71,"old_table_id":0,"old_schema_id":0,"affected_options":null})"; ASSERT_NO_THROW(diff.deserialize(data)); } CATCH @@ -230,6 +236,113 @@ try } CATCH +TEST_F(SchemaSyncTest, PhysicalDropTable) +try +{ + auto pd_client = global_ctx.getTMTContext().getPDClient(); + + const String db_name = "mock_db"; + MockTiDB::instance().newDataBase(db_name); + + auto cols = ColumnsDescription({ + {"col1", typeFromString("String")}, + {"col2", typeFromString("Int64")}, + }); + // table_name, cols, pk_name + std::vector> tables{ + {"t1", cols, ""}, + {"t2", cols, ""}, + }; + auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS(), "dt"); + + refreshSchema(); + + mustGetSyncedTableByName(db_name, "t1"); + mustGetSyncedTableByName(db_name, "t2"); + + MockTiDB::instance().dropTable(global_ctx, db_name, "t1", true); + + refreshSchema(); + + // Create a temporary context with ddl sync task disabled + auto sync_service = std::make_shared(global_ctx); + sync_service->shutdown(); // shutdown the background tasks + + // run gc with safepoint == 0, will be skip + ASSERT_FALSE(sync_service->gc(0, NullspaceID)); + ASSERT_TRUE(sync_service->gc(10000000, NullspaceID)); + // run gc with the same safepoint, will be skip + ASSERT_FALSE(sync_service->gc(10000000, NullspaceID)); + // run gc for another keyspace with same safepoint, will be executed + ASSERT_TRUE(sync_service->gc(10000000, 1024)); + // run gc with changed safepoint + ASSERT_TRUE(sync_service->gc(20000000, 1024)); + // run gc with the same safepoint + ASSERT_FALSE(sync_service->gc(20000000, 1024)); +} +CATCH + +TEST_F(SchemaSyncTest, PhysicalDropTableMeetsUnRemovedRegions) +try +{ + auto pd_client = global_ctx.getTMTContext().getPDClient(); + + const String db_name = "mock_db"; + MockTiDB::instance().newDataBase(db_name); + + auto cols = ColumnsDescription({ + {"col1", typeFromString("String")}, + {"col2", typeFromString("Int64")}, + }); + // table_name, cols, pk_name + std::vector> tables{ + {"t1", cols, ""}, + }; + auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS(), "dt"); + + refreshSchema(); + + mustGetSyncedTableByName(db_name, "t1"); + + MockTiDB::instance().dropTable(global_ctx, db_name, "t1", true); + + refreshSchema(); + + // prevent the storage instance from being physically removed + FailPointHelper::enableFailPoint( + FailPoints::force_set_num_regions_for_table, + std::vector{1001, 1002, 1003}); + SCOPE_EXIT({ FailPointHelper::disableFailPoint(FailPoints::force_set_num_regions_for_table); }); + + auto sync_service = std::make_shared(global_ctx); + sync_service->shutdown(); // shutdown the background tasks + + { + // ensure gc_safe_point cache is empty + auto last_gc_safe_point = lastGcSafePoint(sync_service, NullspaceID); + ASSERT_FALSE(last_gc_safe_point.has_value()); + } + + // Run GC, but the table is not physically dropped because `force_set_num_regions_for_table` + ASSERT_FALSE(sync_service->gc(std::numeric_limits::max(), NullspaceID)); + { + // gc_safe_point cache is not updated + auto last_gc_safe_point = lastGcSafePoint(sync_service, NullspaceID); + ASSERT_FALSE(last_gc_safe_point.has_value()); + } + + // ensure the table is not physically dropped + size_t num_remain_tables = 0; + for (auto table_id : table_ids) + { + auto storage = global_ctx.getTMTContext().getStorages().get(NullspaceID, table_id); + ASSERT_TRUE(storage->isTombstone()); + ++num_remain_tables; + } + ASSERT_EQ(num_remain_tables, 1); +} +CATCH + TEST_F(SchemaSyncTest, RenamePartitionTable) try { diff --git a/tests/fullstack-test2/ddl/alter_exchange_partition.test b/tests/fullstack-test2/ddl/alter_exchange_partition.test index d393b1fc69f..7982a6ffbf2 100644 --- a/tests/fullstack-test2/ddl/alter_exchange_partition.test +++ b/tests/fullstack-test2/ddl/alter_exchange_partition.test @@ -303,7 +303,7 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 # ensure the swap out table is not mark as tombstone >> DBGInvoke __enable_schema_sync_service('true') ->> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __gc_schemas(18446744073709551615, 'true') >> DBGInvoke __enable_schema_sync_service('false') mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; +-----+-------+-------+ @@ -348,7 +348,7 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e2 +-----+-------+-------+ # ensure the swap out table is not mark as tombstone >> DBGInvoke __enable_schema_sync_service('true') ->> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __gc_schemas(18446744073709551615, 'true') >> DBGInvoke __enable_schema_sync_service('false') mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.e order by id; +-----+-------+-------+ diff --git a/tests/fullstack-test2/ddl/flashback_database.test b/tests/fullstack-test2/ddl/flashback_database.test index 733e40b0d64..80f81f02299 100644 --- a/tests/fullstack-test2/ddl/flashback_database.test +++ b/tests/fullstack-test2/ddl/flashback_database.test @@ -55,7 +55,7 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from d1_new.t # ensure the flashbacked table and database is not mark as tombstone >> DBGInvoke __enable_schema_sync_service('true') ->> DBGInvoke __gc_schemas(18446744073709551615) +>> DBGInvoke __gc_schemas(18446744073709551615, 'true') mysql> set session tidb_isolation_read_engines='tiflash'; select * from d1_new.t3 order by a; +------+------+