From ac988a9ae2a5f36d017dd3733748e0af29ecd61a Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 18 Dec 2023 15:18:23 +0800 Subject: [PATCH] ddl: Fix corner case of `FLASHBACK DATABASE` (#8536) (#8538) close pingcap/tiflash#8506 --- dbms/src/Common/FailPoint.cpp | 1 + dbms/src/Debug/MockSchemaGetter.h | 5 - .../KVStore/MultiRaft/PrehandleSnapshot.cpp | 13 +- dbms/src/TiDB/Schema/SchemaBuilder.cpp | 238 +++++++++--------- dbms/src/TiDB/Schema/SchemaBuilder.h | 19 +- dbms/src/TiDB/Schema/SchemaGetter.cpp | 35 --- dbms/src/TiDB/Schema/SchemaGetter.h | 2 - dbms/src/TiDB/tests/gtest_table_info.cpp | 13 +- release-centos7-llvm/env/.gitignore | 3 + .../ddl/flashback/flashback_database.test | 4 + 10 files changed, 157 insertions(+), 176 deletions(-) create mode 100644 release-centos7-llvm/env/.gitignore diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index af56e849c46..ea3195b25a5 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -128,6 +128,7 @@ namespace DB M(pause_when_altering_dt_store) \ M(pause_after_copr_streams_acquired) \ M(pause_query_init) \ + M(pause_before_prehandle_snapshot) \ M(pause_before_wn_establish_task) \ M(pause_passive_flush_before_persist_region) diff --git a/dbms/src/Debug/MockSchemaGetter.h b/dbms/src/Debug/MockSchemaGetter.h index 7b4fae8a2e7..12aedab723c 100644 --- a/dbms/src/Debug/MockSchemaGetter.h +++ b/dbms/src/Debug/MockSchemaGetter.h @@ -44,11 +44,6 @@ struct MockSchemaGetter return {getTableInfo(db_id, table_id), false}; } - static std::tuple getDatabaseAndTableInfo(DatabaseID db_id, TableID table_id) - { - return std::make_tuple(getDatabase(db_id), getTableInfo(db_id, table_id)); - } - static std::vector listDBs() { std::vector res; diff --git a/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp b/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp index ef0228fa006..39c3464497e 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp @@ -43,6 +43,7 @@ namespace FailPoints extern const char force_set_sst_to_dtfile_block_size[]; extern const char force_set_parallel_prehandle_threshold[]; extern const char force_raise_prehandle_exception[]; +extern const char pause_before_prehandle_snapshot[]; } // namespace FailPoints namespace ErrorCodes @@ -191,20 +192,23 @@ PrehandleResult KVStore::preHandleSnapshotToFiles( { new_region->beforePrehandleSnapshot(new_region->id(), deadline_index); ongoing_prehandle_task_count.fetch_add(1); - PrehandleResult result; + + FAIL_POINT_PAUSE(FailPoints::pause_before_prehandle_snapshot); + try { SCOPE_EXIT({ auto ongoing = ongoing_prehandle_task_count.fetch_sub(1) - 1; new_region->afterPrehandleSnapshot(ongoing); }); - result = preHandleSSTsToDTFiles( // + PrehandleResult result = preHandleSSTsToDTFiles( // new_region, snaps, index, term, DM::FileConvertJobType::ApplySnapshot, tmt); + return result; } catch (DB::Exception & e) { @@ -212,7 +216,8 @@ PrehandleResult KVStore::preHandleSnapshotToFiles( fmt::format("(while preHandleSnapshot region_id={}, index={}, term={})", new_region->id(), index, term)); e.rethrow(); } - return result; + + return PrehandleResult{}; } // If size is 0, do not parallel prehandle for this snapshot, which is legacy. @@ -786,4 +791,4 @@ void Region::afterPrehandleSnapshot(int64_t ongoing) } } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index 7727e36e004..68af98b303a 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -95,19 +95,7 @@ void SchemaBuilder::applyCreateTable(DatabaseID database_id, // here (and store the table info to local). // Because `applyPartitionDiffOnLogicalTable` need the logical table for comparing // the latest partitioning and the local partitioning in table info to apply the changes. - auto db_info = getter.getDatabase(database_id); - if (unlikely(db_info == nullptr)) - { - // the database has been dropped - LOG_INFO( - log, - "database is not exist in TiKV, may have been dropped, applyCreateTable is ignored, database_id={} " - "table_id={}", - database_id, - table_id); - return; - } - applyCreateStorageInstance(db_info, table_info, get_by_mvcc); + applyCreateStorageInstance(database_id, table_info, get_by_mvcc); // Register the partition_id -> logical_table_id mapping for (const auto & part_def : table_info->partition.definitions) @@ -187,15 +175,14 @@ void SchemaBuilder::applyExchangeTablePartition(const Schema { LOG_INFO( log, - "ExchangeTablePartition: non_partition_table instance is not created in TiFlash, rename is" - " ignored, table_id={}", + "ExchangeTablePartition: non_partition_table instance is not created in TiFlash" + ", rename is ignored, table_id={}", non_partition_table_id); break; } - auto [new_db_info, new_table_info] - = getter.getDatabaseAndTableInfo(partition_database_id, partition_logical_table_id); - if (unlikely(new_db_info == nullptr || new_table_info == nullptr)) + auto new_table_info = getter.getTableInfo(partition_database_id, partition_logical_table_id); + if (unlikely(new_table_info == nullptr)) { LOG_INFO( log, @@ -205,8 +192,9 @@ void SchemaBuilder::applyExchangeTablePartition(const Schema break; } + String new_db_display_name = tryGetDatabaseDisplayNameFromLocal(partition_database_id); auto part_table_info = new_table_info->producePartitionTableInfo(non_partition_table_id, name_mapper); - applyRenamePhysicalTable(new_db_info, *part_table_info, storage); + applyRenamePhysicalTable(partition_database_id, new_db_display_name, *part_table_info, storage); } while (false); // Rename the exchanged partition table belonging new database. Now it should belong to @@ -219,15 +207,14 @@ void SchemaBuilder::applyExchangeTablePartition(const Schema { LOG_INFO( log, - "ExchangeTablePartition: partition_physical_table instance is not created in TiFlash, rename is" - " ignored, table_id={}", + "ExchangeTablePartition: partition_physical_table instance is not created in TiFlash" + ", rename is ignored, table_id={}", partition_physical_table_id); break; } - auto [new_db_info, new_table_info] - = getter.getDatabaseAndTableInfo(non_partition_database_id, partition_physical_table_id); - if (unlikely(new_db_info == nullptr || new_table_info == nullptr)) + auto new_table_info = getter.getTableInfo(non_partition_database_id, partition_physical_table_id); + if (unlikely(new_table_info == nullptr)) { LOG_INFO( log, @@ -237,7 +224,8 @@ void SchemaBuilder::applyExchangeTablePartition(const Schema break; } - applyRenamePhysicalTable(new_db_info, *new_table_info, storage); + String new_db_display_name = tryGetDatabaseDisplayNameFromLocal(non_partition_database_id); + applyRenamePhysicalTable(non_partition_database_id, new_db_display_name, *new_table_info, storage); } while (false); } @@ -378,8 +366,8 @@ void SchemaBuilder::applyDiff(const SchemaDiff & diff) template void SchemaBuilder::applySetTiFlashReplica(DatabaseID database_id, TableID table_id) { - auto [db_info, table_info] = getter.getDatabaseAndTableInfo(database_id, table_id); - if (unlikely(db_info == nullptr || table_info == nullptr)) + auto table_info = getter.getTableInfo(database_id, table_id); + if (unlikely(table_info == nullptr)) { LOG_WARNING(log, "table is not exist in TiKV, applySetTiFlashReplica is ignored, table_id={}", table_id); return; @@ -399,7 +387,7 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa return; } - applyDropTable(db_info->id, table_id); + applyDropTable(database_id, table_id); return; } @@ -410,7 +398,7 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa { if (!table_id_map.tableIDInDatabaseIdMap(table_id)) { - applyCreateTable(db_info->id, table_id); + applyCreateTable(database_id, table_id); } return; } @@ -418,7 +406,7 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa // Recover the table if tombstone if (storage->isTombstone()) { - applyRecoverLogicalTable(db_info, table_info); + applyRecoverLogicalTable(database_id, table_info); return; } @@ -455,7 +443,7 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa part_storage->alterSchemaChange( alter_lock, local_table_info, - name_mapper.mapDatabaseName(db_info->id, keyspace_id), + name_mapper.mapDatabaseName(database_id, keyspace_id), name_mapper.mapTableName(local_table_info), context); } @@ -483,7 +471,7 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa storage->alterSchemaChange( alter_lock, local_logical_storage_table_info, - name_mapper.mapDatabaseName(db_info->id, keyspace_id), + name_mapper.mapDatabaseName(database_id, keyspace_id), name_mapper.mapTableName(local_logical_storage_table_info), context); } @@ -502,8 +490,8 @@ void SchemaBuilder::applySetTiFlashReplica(DatabaseID databa template void SchemaBuilder::applyPartitionDiff(DatabaseID database_id, TableID table_id) { - auto [db_info, table_info] = getter.getDatabaseAndTableInfo(database_id, table_id); - if (unlikely(db_info == nullptr || table_info == nullptr)) + auto table_info = getter.getTableInfo(database_id, table_id); + if (unlikely(table_info == nullptr)) { LOG_ERROR(log, "table is not exist in TiKV, applyPartitionDiff is ignored, table_id={}", table_id); return; @@ -513,8 +501,8 @@ void SchemaBuilder::applyPartitionDiff(DatabaseID database_i LOG_ERROR( log, "new table in TiKV is not a partition table {}, database_id={} table_id={}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); return; } @@ -530,12 +518,12 @@ void SchemaBuilder::applyPartitionDiff(DatabaseID database_i return; } - applyPartitionDiffOnLogicalTable(db_info, table_info, storage); + applyPartitionDiffOnLogicalTable(database_id, table_info, storage); } template void SchemaBuilder::applyPartitionDiffOnLogicalTable( - const TiDB::DBInfoPtr & db_info, + const DatabaseID database_id, const TableInfoPtr & table_info, const ManageableStoragePtr & storage) { @@ -548,8 +536,8 @@ void SchemaBuilder::applyPartitionDiffOnLogicalTable( LOG_INFO( log, "Altering non-partition table to be a partition table {} with database_id={}, table_id={}", - name_mapper.debugCanonicalName(*db_info, local_table_info), - db_info->id, + name_mapper.debugCanonicalName(local_table_info, database_id, keyspace_id), + database_id, local_table_info.id); } @@ -567,8 +555,8 @@ void SchemaBuilder::applyPartitionDiffOnLogicalTable( LOG_INFO( log, "Applying partition changes {} with database_id={}, table_id={}, old: {}, new: {}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id, local_part_id_set, new_part_id_set); @@ -579,8 +567,8 @@ void SchemaBuilder::applyPartitionDiffOnLogicalTable( log, "No partition changes, partitions_size={} {} with database_id={}, table_id={}", new_part_id_set.size(), - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); return; } @@ -596,7 +584,7 @@ void SchemaBuilder::applyPartitionDiffOnLogicalTable( { if (!new_part_id_set.contains(local_def.id)) { - applyDropPhysicalTable(name_mapper.mapDatabaseName(*db_info), local_def.id); + applyDropPhysicalTable(name_mapper.mapDatabaseName(database_id, keyspace_id), local_def.id); } } @@ -612,7 +600,7 @@ void SchemaBuilder::applyPartitionDiffOnLogicalTable( storage->alterSchemaChange( alter_lock, updated_table_info, - name_mapper.mapDatabaseName(db_info->id, keyspace_id), + name_mapper.mapDatabaseName(database_id, keyspace_id), name_mapper.mapTableName(updated_table_info), context); @@ -620,21 +608,14 @@ void SchemaBuilder::applyPartitionDiffOnLogicalTable( LOG_INFO( log, "Applied partition changes {} with database_id={}, table_id={}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); } template void SchemaBuilder::applyRenameTable(DatabaseID database_id, TableID table_id) { - auto [new_db_info, new_table_info] = getter.getDatabaseAndTableInfo(database_id, table_id); - if (unlikely(new_db_info == nullptr || new_table_info == nullptr)) - { - 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); @@ -649,16 +630,25 @@ void SchemaBuilder::applyRenameTable(DatabaseID database_id, return; } - applyRenameLogicalTable(new_db_info, new_table_info, storage); + auto new_table_info = getter.getTableInfo(database_id, table_id); + if (unlikely(new_table_info == nullptr)) + { + LOG_ERROR(log, "table is not exist in TiKV, applyRenameTable is ignored, table_id={}", table_id); + return; + } + + String new_db_display_name = tryGetDatabaseDisplayNameFromLocal(database_id); + applyRenameLogicalTable(database_id, new_db_display_name, new_table_info, storage); } template void SchemaBuilder::applyRenameLogicalTable( - const DBInfoPtr & new_db_info, + const DatabaseID new_database_id, + const String & new_database_display_name, const TableInfoPtr & new_table_info, const ManageableStoragePtr & storage) { - applyRenamePhysicalTable(new_db_info, *new_table_info, storage); + applyRenamePhysicalTable(new_database_id, new_database_display_name, *new_table_info, storage); if (new_table_info->isLogicalPartitionTable()) { @@ -671,24 +661,26 @@ void SchemaBuilder::applyRenameLogicalTable( LOG_ERROR( log, "Storage instance is not exist in TiFlash, applyRenamePhysicalTable is ignored, " - "physical_table_id={}", - part_def.id); + "physical_table_id={} logical_table_id={}", + part_def.id, + new_table_info->id); return; } auto part_table_info = new_table_info->producePartitionTableInfo(part_def.id, name_mapper); - applyRenamePhysicalTable(new_db_info, *part_table_info, part_storage); + applyRenamePhysicalTable(new_database_id, new_database_display_name, *part_table_info, part_storage); } } } template void SchemaBuilder::applyRenamePhysicalTable( - const DBInfoPtr & new_db_info, + const DatabaseID new_database_id, + const String & new_database_display_name, const TableInfo & new_table_info, const ManageableStoragePtr & storage) { const auto old_mapped_db_name = storage->getDatabaseName(); - const auto new_mapped_db_name = name_mapper.mapDatabaseName(*new_db_info); + const auto new_mapped_db_name = name_mapper.mapDatabaseName(new_database_id, keyspace_id); const auto old_display_table_name = name_mapper.displayTableName(storage->getTableInfo()); const auto new_display_table_name = name_mapper.displayTableName(new_table_info); if (old_mapped_db_name == new_mapped_db_name && old_display_table_name == new_display_table_name) @@ -696,8 +688,8 @@ void SchemaBuilder::applyRenamePhysicalTable( LOG_DEBUG( log, "Table {} name identical, not renaming. database_id={} table_id={}", - name_mapper.debugCanonicalName(*new_db_info, new_table_info), - new_db_info->id, + name_mapper.debugCanonicalName(new_table_info, new_database_id, keyspace_id), + new_database_id, new_table_info.id); return; } @@ -710,18 +702,19 @@ void SchemaBuilder::applyRenamePhysicalTable( old_mapped_db_name, old_mapped_tbl_name, old_display_table_name, - name_mapper.debugCanonicalName(*new_db_info, new_table_info), - new_db_info->id, + name_mapper.debugCanonicalName(new_table_info, new_database_id, keyspace_id), + new_database_id, new_table_info.id); // Note that rename will update table info in table create statement by modifying original table info // with "tidb_display.table" instead of using new_table_info directly, so that other changes // (ALTER commands) won't be saved. Besides, no need to update schema_version as table name is not structural. auto rename = std::make_shared(); - ASTRenameQuery::Table from{old_mapped_db_name, old_mapped_tbl_name}; - ASTRenameQuery::Table to{new_mapped_db_name, name_mapper.mapTableName(new_table_info)}; - ASTRenameQuery::Table display{name_mapper.displayDatabaseName(*new_db_info), new_display_table_name}; - ASTRenameQuery::Element elem{.from = std::move(from), .to = std::move(to), .tidb_display = std::move(display)}; + ASTRenameQuery::Element elem{ + .from = ASTRenameQuery::Table{old_mapped_db_name, old_mapped_tbl_name}, + .to = ASTRenameQuery::Table{new_mapped_db_name, name_mapper.mapTableName(new_table_info)}, + .tidb_display = ASTRenameQuery::Table{new_database_display_name, new_display_table_name}, + }; rename->elements.emplace_back(std::move(elem)); InterpreterRenameQuery(rename, context, getThreadNameAndID()).execute(); @@ -732,16 +725,16 @@ void SchemaBuilder::applyRenamePhysicalTable( old_mapped_db_name, old_mapped_tbl_name, old_display_table_name, - name_mapper.debugCanonicalName(*new_db_info, new_table_info), - new_db_info->id, + name_mapper.debugCanonicalName(new_table_info, new_database_id, keyspace_id), + new_database_id, new_table_info.id); } template void SchemaBuilder::applyRecoverTable(DatabaseID database_id, TiDB::TableID table_id) { - auto [db_info, table_info] = getter.getDatabaseAndTableInfo(database_id, table_id); - if (unlikely(db_info == nullptr || table_info == nullptr)) + auto table_info = getter.getTableInfo(database_id, table_id); + if (unlikely(table_info == nullptr)) { // this table is dropped. LOG_INFO( @@ -751,36 +744,34 @@ void SchemaBuilder::applyRecoverTable(DatabaseID database_id return; } - applyRecoverLogicalTable(db_info, table_info); + applyRecoverLogicalTable(database_id, table_info); } template void SchemaBuilder::applyRecoverLogicalTable( - const TiDB::DBInfoPtr & db_info, + const DatabaseID database_id, const TiDB::TableInfoPtr & table_info) { - assert(db_info != nullptr); assert(table_info != nullptr); if (table_info->isLogicalPartitionTable()) { for (const auto & part_def : table_info->partition.definitions) { auto part_table_info = table_info->producePartitionTableInfo(part_def.id, name_mapper); - tryRecoverPhysicalTable(db_info, part_table_info); + tryRecoverPhysicalTable(database_id, part_table_info); } } - tryRecoverPhysicalTable(db_info, table_info); + tryRecoverPhysicalTable(database_id, table_info); } // Return true - the Storage instance exists and is recovered (or not tombstone) // false - the Storage instance does not exist template bool SchemaBuilder::tryRecoverPhysicalTable( - const TiDB::DBInfoPtr & db_info, + const DatabaseID database_id, const TiDB::TableInfoPtr & table_info) { - assert(db_info != nullptr); assert(table_info != nullptr); auto & tmt_context = context.getTMTContext(); auto storage = tmt_context.getStorages().get(keyspace_id, table_info->id); @@ -798,8 +789,8 @@ bool SchemaBuilder::tryRecoverPhysicalTable( LOG_INFO( log, "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, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); return true; } @@ -807,8 +798,8 @@ bool SchemaBuilder::tryRecoverPhysicalTable( LOG_INFO( log, "Create table {} by recover begin, database_id={} table_id={}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); AlterCommands commands; { @@ -820,15 +811,15 @@ bool SchemaBuilder::tryRecoverPhysicalTable( storage->updateTombstone( alter_lock, commands, - name_mapper.mapDatabaseName(*db_info), + name_mapper.mapDatabaseName(database_id, keyspace_id), *table_info, name_mapper, context); LOG_INFO( log, "Create table {} by recover end, database_id={} table_id={}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); return true; } @@ -883,6 +874,11 @@ bool SchemaBuilder::applyCreateDatabase(DatabaseID database_ auto db_info = getter.getDatabase(database_id); if (unlikely(db_info == nullptr)) { + LOG_INFO( + log, + "Create database is ignored because database is not exist in TiKV," + " database_id={}", + database_id); return false; } applyCreateDatabaseByInfo(db_info); @@ -942,7 +938,7 @@ void SchemaBuilder::applyRecoverDatabase(DatabaseID database for (auto table_id : table_ids) { auto table_info = getter.getTableInfo(database_id, table_id); - applyRecoverLogicalTable(db_info, table_info); + applyRecoverLogicalTable(database_id, table_info); } } @@ -955,13 +951,9 @@ void SchemaBuilder::applyRecoverDatabase(DatabaseID database template void SchemaBuilder::applyDropDatabase(DatabaseID database_id) { - TiDB::DBInfoPtr db_info = databases.getDBInfo(database_id); - if (unlikely(db_info == nullptr)) - { - LOG_INFO(log, "Try to drop database but not found, may has been dropped, database_id={}", database_id); - return; - } - + // The DDL in TiFlash comes after user executed `DROP DATABASE ...`. So the meta key could + // have already been deleted. In order to handle this situation, we should not fetch the + // `DatabaseInfo` from TiKV. { //TODO: it seems may need a lot time, maybe we can do it in a background thread auto table_ids = table_id_map.findTablesByDatabaseID(database_id); @@ -1031,7 +1023,8 @@ std::tuple parseColumnsFromTableInfo(const TiDB::TableIn } String createTableStmt( - const DBInfo & db_info, + const KeyspaceID keyspace_id, + const DatabaseID database_id, const TableInfo & table_info, const SchemaNameMapper & name_mapper, const UInt64 tombstone, @@ -1043,7 +1036,7 @@ String createTableStmt( String stmt; WriteBufferFromString stmt_buf(stmt); writeString("CREATE TABLE ", stmt_buf); - writeBackQuotedString(name_mapper.mapDatabaseName(db_info), stmt_buf); + writeBackQuotedString(name_mapper.mapDatabaseName(database_id, keyspace_id), stmt_buf); writeString(".", stmt_buf); writeBackQuotedString(name_mapper.mapTableName(table_info), stmt_buf); writeString("(", stmt_buf); @@ -1083,23 +1076,22 @@ String createTableStmt( template void SchemaBuilder::applyCreateStorageInstance( - const TiDB::DBInfoPtr & db_info, + const DatabaseID database_id, const TableInfoPtr & table_info, bool is_tombstone) { - assert(db_info != nullptr); assert(table_info != nullptr); GET_METRIC(tiflash_schema_internal_ddl_count, type_create_table).Increment(); LOG_INFO( log, "Create table {} begin, database_id={}, table_id={}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); /// Try to recover the existing storage instance - if (tryRecoverPhysicalTable(db_info, table_info)) + if (tryRecoverPhysicalTable(database_id, table_info)) { return; } @@ -1117,13 +1109,13 @@ void SchemaBuilder::applyCreateStorageInstance( tombstone_ts = context.getTMTContext().getPDClient()->getTS(); } - String stmt = createTableStmt(*db_info, *table_info, name_mapper, tombstone_ts, log); + String stmt = createTableStmt(keyspace_id, database_id, *table_info, name_mapper, tombstone_ts, log); LOG_INFO( log, "Create table {} (database_id={} table_id={}) with statement: {}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id, stmt); @@ -1133,7 +1125,7 @@ void SchemaBuilder::applyCreateStorageInstance( auto * ast_create_query = typeid_cast(ast.get()); ast_create_query->attach = true; ast_create_query->if_not_exists = true; - ast_create_query->database = name_mapper.mapDatabaseName(*db_info); + ast_create_query->database = name_mapper.mapDatabaseName(database_id, keyspace_id); InterpreterCreateQuery interpreter(ast, context); interpreter.setInternal(true); @@ -1142,8 +1134,8 @@ void SchemaBuilder::applyCreateStorageInstance( LOG_INFO( log, "Creat table {} end, database_id={} table_id={}", - name_mapper.debugCanonicalName(*db_info, *table_info), - db_info->id, + name_mapper.debugCanonicalName(*table_info, database_id, keyspace_id), + database_id, table_info->id); } @@ -1304,7 +1296,7 @@ void SchemaBuilder::syncAllSchema() // So `syncAllSchema` will not create tombstone tables. But if there are new rows/new snapshot // sent to TiFlash, TiFlash can create the instance by `applyTable` with force==true in the // related process. - applyCreateStorageInstance(db_info, table_info, false); + applyCreateStorageInstance(db_info->id, table_info, false); if (table_info->isLogicalPartitionTable()) { for (const auto & part_def : table_info->partition.definitions) @@ -1456,16 +1448,12 @@ bool SchemaBuilder::applyTable( auto storage = tmt_context.getStorages().get(keyspace_id, physical_table_id); if (storage == nullptr) { - auto db_info = getter.getDatabase(database_id); - if (db_info == nullptr) - { - LOG_ERROR(log, "database is not exist in TiKV, applyTable need retry, database_id={}", database_id); - return false; - } + // The Raft log or snapshot could comes after user executed `DROP DATABASE ...`. In order to + // handle this situation, we should not fetch the `DatabaseInfo` from TiKV. // Create the instance with the latest table info // If the table info is get by mvcc, it means the table is actually in "dropped" status - applyCreateStorageInstance(db_info, table_info, get_by_mvcc); + applyCreateStorageInstance(database_id, table_info, get_by_mvcc); return true; } @@ -1535,6 +1523,20 @@ void SchemaBuilder::dropAllSchema() LOG_INFO(log, "Drop all schemas end"); } +template +String SchemaBuilder::tryGetDatabaseDisplayNameFromLocal(DatabaseID database_id) +{ + // This method is called in the `applyDiff` loop. The `applyDiff` loop should apply the all the DDL operations before + // the database get dropped, so the database info should be cached in `databases`. + // But for corner cases that the database is dropped on some unkonwn cases, we just return a display database name + // according to the keyspace_id and database_id because display name ususally is not critical. + if (auto new_db_info = databases.getDBInfo(database_id); likely(new_db_info != nullptr)) + { + return name_mapper.displayDatabaseName(*new_db_info); + } + return name_mapper.mapDatabaseName(database_id, keyspace_id); +} + // product env template struct SchemaBuilder; // mock test diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h index 8e84fbeb2df..f0edd6cc12a 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -74,40 +74,41 @@ struct SchemaBuilder void applyRecoverDatabase(DatabaseID database_id); void applyCreateTable(DatabaseID database_id, TableID table_id); - void applyCreateStorageInstance( - const TiDB::DBInfoPtr & db_info, - const TiDB::TableInfoPtr & table_info, - bool is_tombstone); + void applyCreateStorageInstance(DatabaseID database_id, const TiDB::TableInfoPtr & table_info, bool is_tombstone); void applyDropTable(DatabaseID database_id, TableID table_id); /// Parameter schema_name should be mapped. void applyDropPhysicalTable(const String & db_name, TableID table_id); void applyRecoverTable(DatabaseID database_id, TiDB::TableID table_id); - void applyRecoverLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); - bool tryRecoverPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); + void applyRecoverLogicalTable(DatabaseID database_id, const TiDB::TableInfoPtr & table_info); + bool tryRecoverPhysicalTable(DatabaseID database_id, const TiDB::TableInfoPtr & table_info); void applyPartitionDiff(DatabaseID database_id, TableID table_id); void applyPartitionDiffOnLogicalTable( - const TiDB::DBInfoPtr & db_info, + DatabaseID database_id, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); void applyRenameTable(DatabaseID database_id, TiDB::TableID table_id); void applyRenameLogicalTable( - const TiDB::DBInfoPtr & new_db_info, + DatabaseID new_database_id, + const String & new_database_display_name, const TiDB::TableInfoPtr & new_table_info, const ManageableStoragePtr & storage); void applyRenamePhysicalTable( - const TiDB::DBInfoPtr & new_db_info, + DatabaseID new_database_id, + const String & new_database_display_name, const TiDB::TableInfo & new_table_info, const ManageableStoragePtr & storage); void applySetTiFlashReplica(DatabaseID database_id, TableID table_id); void applyExchangeTablePartition(const SchemaDiff & diff); + + String tryGetDatabaseDisplayNameFromLocal(DatabaseID database_id); }; } // namespace DB diff --git a/dbms/src/TiDB/Schema/SchemaGetter.cpp b/dbms/src/TiDB/Schema/SchemaGetter.cpp index 26dae26bb82..4a6ec8d1809 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.cpp +++ b/dbms/src/TiDB/Schema/SchemaGetter.cpp @@ -315,41 +315,6 @@ std::pair SchemaGetter::getTableInfoImpl(DatabaseID db template std::pair SchemaGetter::getTableInfoImpl(DatabaseID db_id, TableID table_id); template std::pair SchemaGetter::getTableInfoImpl(DatabaseID db_id, TableID table_id); -std::tuple SchemaGetter::getDatabaseAndTableInfo( - DatabaseID db_id, - TableID table_id) -{ - String db_key = getDBKey(db_id); - String db_json = TxnStructure::hGet(snap, DBs, db_key); - - if (db_json.empty()) - return std::make_tuple(nullptr, nullptr); - - LOG_DEBUG(log, "Get DB Info from TiKV: {}", db_json); - auto db_info = std::make_shared(db_json, keyspace_id); - - String table_key = getTableKey(table_id); - String table_info_json = TxnStructure::hGet(snap, db_key, table_key); - if (table_info_json.empty()) - { - 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()) - { - LOG_ERROR( - log, - "The table is dropped in TiKV, and the latest table_info is still empty, it should be GCed, " - "table_id={}", - table_id); - return std::make_tuple(db_info, nullptr); - } - } - 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); -} - std::vector SchemaGetter::listDBs() { std::vector res; diff --git a/dbms/src/TiDB/Schema/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h index fe5c1efe803..82ae2e87ba2 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -177,8 +177,6 @@ struct SchemaGetter return getTableInfoImpl(db_id, table_id); } - std::tuple getDatabaseAndTableInfo(DatabaseID db_id, TableID table_id); - std::vector listDBs(); std::vector listTables(DatabaseID db_id); diff --git a/dbms/src/TiDB/tests/gtest_table_info.cpp b/dbms/src/TiDB/tests/gtest_table_info.cpp index 7d75cfc9d34..e96b757f47d 100644 --- a/dbms/src/TiDB/tests/gtest_table_info.cpp +++ b/dbms/src/TiDB/tests/gtest_table_info.cpp @@ -33,10 +33,11 @@ namespace DB { String createTableStmt( - const DBInfo & db_info, + KeyspaceID keyspace_id, + DatabaseID database_id, const TableInfo & table_info, const SchemaNameMapper & name_mapper, - const UInt64 tombstone, + UInt64 tombstone, const LoggerPtr & log); namespace tests @@ -151,7 +152,13 @@ struct StmtCase // generate create statement with db_info and table_info auto verify_stmt = [&](TiDB::StorageEngine engine_type) { table_info.engine_type = engine_type; - String stmt = createTableStmt(db_info, table_info, MockSchemaNameMapper(), tombstone, Logger::get()); + String stmt = createTableStmt( + db_info.keyspace_id, + db_info.id, + table_info, + MockSchemaNameMapper(), + tombstone, + Logger::get()); EXPECT_EQ(stmt, create_stmt_dm) << "Table info create statement mismatch:\n" + stmt + "\n" + create_stmt_dm; json1 = extractTableInfoFromCreateStatement(stmt, table_info.name); diff --git a/release-centos7-llvm/env/.gitignore b/release-centos7-llvm/env/.gitignore new file mode 100644 index 00000000000..a53ab367f81 --- /dev/null +++ b/release-centos7-llvm/env/.gitignore @@ -0,0 +1,3 @@ +tiflash-env +tiflash-env-*.tar.xz +*.log diff --git a/tests/fullstack-test2/ddl/flashback/flashback_database.test b/tests/fullstack-test2/ddl/flashback/flashback_database.test index dd541b5d2a4..e2d2bb8532f 100644 --- a/tests/fullstack-test2/ddl/flashback/flashback_database.test +++ b/tests/fullstack-test2/ddl/flashback/flashback_database.test @@ -97,6 +97,7 @@ func> wait_table d1 t1 mysql> alter table d1.t1 add column b int; >> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) +>> DBGInvoke __enable_fail_point(pause_before_prehandle_snapshot) # exactly write until fail point "pause_before_apply_raft_cmd" to be disable mysql> insert into d1.t1 values(2,2); @@ -107,6 +108,7 @@ mysql> drop database d1; # make write cmd take effect >> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) +>> DBGInvoke __disable_fail_point(pause_before_prehandle_snapshot) # the `t1` is still mark as tombstone >> select tidb_database,tidb_name,is_tombstone,tidb_table_id from system.tables where is_tombstone = 0 and tidb_database = 'd1' and tidb_name='t1'; @@ -145,6 +147,7 @@ mysql> create table d1.t2(id INT NOT NULL,name VARCHAR(30)) PARTITION BY RANGE ( mysql> insert into d1.t2 values(1, 'abc'),(2, 'cde'),(53, 'efg'); >> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) +>> DBGInvoke __enable_fail_point(pause_before_prehandle_snapshot) mysql> alter table d1.t2 set tiflash replica 1; mysql> alter table d1.t2 add column b int; @@ -154,6 +157,7 @@ mysql> drop database d1; # make write cmd take effect >> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) +>> DBGInvoke __disable_fail_point(pause_before_prehandle_snapshot) # check the row is written to the storage or not mysql> flashback database d1 to d1_new