diff --git a/dbms/src/Debug/MockKVStore/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockKVStore/MockRaftStoreProxy.cpp index 4a9e54e295d..7c5b4ebb145 100644 --- a/dbms/src/Debug/MockKVStore/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockKVStore/MockRaftStoreProxy.cpp @@ -694,8 +694,8 @@ TableID MockRaftStoreProxy::bootstrapTable(Context & ctx, KVStore & kvs, TMTCont } MockTiDB::instance().newDataBase("d"); // Make sure there is a table with smaller id. - MockTiDB::instance().newTable("d", "prevt" + toString(random()), columns, tso, "", "dt"); - UInt64 table_id = MockTiDB::instance().newTable("d", "t" + toString(random()), columns, tso, "", "dt"); + MockTiDB::instance().newTable("d", "prevt" + toString(random()), columns, tso, ""); + UInt64 table_id = MockTiDB::instance().newTable("d", "t" + toString(random()), columns, tso, ""); auto schema_syncer = tmt.getSchemaSyncerManager(); schema_syncer->syncSchemas(ctx, NullspaceID); diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index 0a73f4dc892..d053e5669df 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -234,8 +234,7 @@ DatabaseID MockTiDB::newDataBase(const String & database_name) TiDB::TableInfoPtr MockTiDB::parseColumns( const String & tbl_name, const ColumnsDescription & columns, - const String & handle_pk_name, - String engine_type) + const String & handle_pk_name) { TableInfo table_info; table_info.name = tbl_name; @@ -280,7 +279,6 @@ TiDB::TableInfoPtr MockTiDB::parseColumns( index_info.id = 1; index_info.is_primary = true; index_info.idx_name = "PRIMARY"; - index_info.tbl_name = tbl_name; index_info.is_unique = true; index_info.index_type = 1; index_info.idx_cols.resize(string_tokens.count()); @@ -296,20 +294,6 @@ TiDB::TableInfoPtr MockTiDB::parseColumns( table_info.pk_is_handle = true; } - table_info.comment = "Mocked."; - - // set storage engine type - std::transform(engine_type.begin(), engine_type.end(), engine_type.begin(), [](unsigned char c) { - return std::tolower(c); - }); - if (engine_type == "dt") - table_info.engine_type = TiDB::StorageEngine::DT; - - if (table_info.engine_type != TiDB::StorageEngine::DT) - { - throw Exception("Unknown engine type : " + engine_type + ", must be 'dt'", ErrorCodes::BAD_ARGUMENTS); - } - return std::make_shared(std::move(table_info)); } @@ -318,8 +302,7 @@ TableID MockTiDB::newTable( const String & table_name, const ColumnsDescription & columns, Timestamp tso, - const String & handle_pk_name, - const String & engine_type) + const String & handle_pk_name) { std::scoped_lock lock(tables_mutex); @@ -334,7 +317,7 @@ TableID MockTiDB::newTable( throw Exception("MockTiDB not found db: " + database_name, ErrorCodes::LOGICAL_ERROR); } - auto table_info = parseColumns(table_name, columns, handle_pk_name, engine_type); + auto table_info = parseColumns(table_name, columns, handle_pk_name); table_info->id = table_id_allocator++; table_info->update_timestamp = tso; return addTable(database_name, std::move(*table_info)); @@ -346,7 +329,6 @@ std::tuple> MockTiDB::newPartitionTable( const ColumnsDescription & columns, Timestamp tso, const String & handle_pk_name, - const String & engine_type, const Strings & part_names) { std::scoped_lock lock(tables_mutex); @@ -364,7 +346,7 @@ std::tuple> MockTiDB::newPartitionTable( std::vector physical_table_ids; physical_table_ids.reserve(part_names.size()); - auto table_info = parseColumns(table_name, columns, handle_pk_name, engine_type); + auto table_info = parseColumns(table_name, columns, handle_pk_name); table_info->id = table_id_allocator++; table_info->is_partition_table = true; table_info->partition.enable = true; @@ -385,8 +367,7 @@ std::tuple> MockTiDB::newPartitionTable( std::vector MockTiDB::newTables( const String & database_name, const std::vector> & tables, - Timestamp tso, - const String & engine_type) + Timestamp tso) { std::scoped_lock lock(tables_mutex); std::vector table_ids; @@ -407,7 +388,7 @@ std::vector MockTiDB::newTables( throw Exception("Mock TiDB table " + qualified_name + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS); } - auto table_info = *parseColumns(table_name, columns, handle_pk_name, engine_type); + auto table_info = *parseColumns(table_name, columns, handle_pk_name); table_info.id = table_id_allocator++; table_info.update_timestamp = tso; diff --git a/dbms/src/Debug/MockTiDB.h b/dbms/src/Debug/MockTiDB.h index eea2412fad5..55c7386d6bb 100644 --- a/dbms/src/Debug/MockTiDB.h +++ b/dbms/src/Debug/MockTiDB.h @@ -84,8 +84,7 @@ class MockTiDB : public ext::Singleton const String & table_name, const ColumnsDescription & columns, Timestamp tso, - const String & handle_pk_name, - const String & engine_type); + const String & handle_pk_name); // Mock to create a partition table with given partition names // Return @@ -95,22 +94,19 @@ class MockTiDB : public ext::Singleton const ColumnsDescription & columns, Timestamp tso, const String & handle_pk_name, - const String & engine_type, const Strings & part_names); std::vector newTables( const String & database_name, const std::vector> & tables, - Timestamp tso, - const String & engine_type); + Timestamp tso); TableID addTable(const String & database_name, TiDB::TableInfo && table_info); static TiDB::TableInfoPtr parseColumns( const String & tbl_name, const ColumnsDescription & columns, - const String & handle_pk_name, - String engine_type); + const String & handle_pk_name); DatabaseID newDataBase(const String & database_name); diff --git a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp index 2b3c85d6068..e0a97f25f68 100644 --- a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp +++ b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp @@ -43,8 +43,7 @@ void MockTiDBTable::dbgFuncMockTiDBTable(Context & context, const ASTs & args, D { if (args.size() != 3 && args.size() != 4 && args.size() != 5) throw Exception( - "Args not matched, should be: database-name, table-name, schema-string [, handle_pk_name], [, " - "engine-type(tmt|dt)]", + "Args not matched, should be: database-name, table-name, schema-string [, handle_pk_name]", ErrorCodes::BAD_ARGUMENTS); const String & database_name = typeid_cast(*args[0]).name; @@ -65,16 +64,9 @@ void MockTiDBTable::dbgFuncMockTiDBTable(Context & context, const ASTs & args, D ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(typeid_cast(*columns_ast), context); - String engine_type("dt"); - if (context.getTMTContext().getEngineType() == ::TiDB::StorageEngine::TMT) - engine_type = "tmt"; - if (args.size() == 5) - engine_type = safeGet(typeid_cast(*args[4]).value); - auto tso = context.getTMTContext().getPDClient()->getTS(); - TableID table_id - = MockTiDB::instance().newTable(database_name, table_name, columns, tso, handle_pk_name, engine_type); + TableID table_id = MockTiDB::instance().newTable(database_name, table_name, columns, tso, handle_pk_name); output(fmt::format("mock table #{}", table_id)); } @@ -354,10 +346,7 @@ void MockTiDBTable::dbgFuncCreateTiDBTables(Context & context, const ASTs & args tables.emplace_back(table_name, columns, ""); } auto tso = context.getTMTContext().getPDClient()->getTS(); - String engine_type("dt"); - if (context.getTMTContext().getEngineType() == ::TiDB::StorageEngine::TMT) - engine_type = "tmt"; - MockTiDB::instance().newTables(database_name, tables, tso, engine_type); + MockTiDB::instance().newTables(database_name, tables, tso); output(""); } diff --git a/dbms/src/Debug/dbgFuncMockTiDBTable.h b/dbms/src/Debug/dbgFuncMockTiDBTable.h index 133a6317c41..55468338e65 100644 --- a/dbms/src/Debug/dbgFuncMockTiDBTable.h +++ b/dbms/src/Debug/dbgFuncMockTiDBTable.h @@ -27,7 +27,7 @@ struct MockTiDBTable // Inject mocked TiDB table. // Usage: - // ./storages-client.sh "DBGInvoke mock_tidb_table(database_name, table_name, 'col1 type1, col2 type2, ...' [, handle_pk_name, engine-type(tmt|dt)])" + // ./storages-client.sh "DBGInvoke mock_tidb_table(database_name, table_name, 'col1 type1, col2 type2, ...' [, handle_pk_name])" // engine: [tmt, dt], tmt by default static void dbgFuncMockTiDBTable(Context & context, const ASTs & args, DBGInvoker::Printer output); diff --git a/dbms/src/Debug/dbgKVStore/dbgFuncMockRaftSnapshot.cpp b/dbms/src/Debug/dbgKVStore/dbgFuncMockRaftSnapshot.cpp index 5921aad8fde..baefef60223 100644 --- a/dbms/src/Debug/dbgKVStore/dbgFuncMockRaftSnapshot.cpp +++ b/dbms/src/Debug/dbgKVStore/dbgFuncMockRaftSnapshot.cpp @@ -727,7 +727,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFiles( ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription( typeid_cast(*columns_ast), context); - mocked_table_info = MockTiDB::parseColumns(table_name, columns, handle_pk_name, "dt"); + mocked_table_info = MockTiDB::parseColumns(table_name, columns, handle_pk_name); } MockTiDB::TablePtr table = MockTiDB::instance().getTableByName(database_name, table_name); @@ -830,7 +830,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFilesWithHandles( ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription( typeid_cast(*columns_ast), context); - mocked_table_info = MockTiDB::parseColumns(table_name, columns, handle_pk_name, "dt"); + mocked_table_info = MockTiDB::parseColumns(table_name, columns, handle_pk_name); } MockTiDB::TablePtr table = MockTiDB::instance().getTableByName(database_name, table_name); diff --git a/dbms/src/Interpreters/tests/gtest_interpreter_create_query.cpp b/dbms/src/Interpreters/tests/gtest_interpreter_create_query.cpp index 1e1675b7cc4..60b7bb376d5 100644 --- a/dbms/src/Interpreters/tests/gtest_interpreter_create_query.cpp +++ b/dbms/src/Interpreters/tests/gtest_interpreter_create_query.cpp @@ -108,7 +108,6 @@ class InterperCreateQueryTiFlashTest : public ::testing::Test TiDB::DBInfo db_info(db_info_json, NullspaceID); TiDB::TableInfo table_info(table_info_json, NullspaceID); - table_info.engine_type = ::TiDB::StorageEngine::DT; ParserCreateQuery parser; ASTPtr ast @@ -167,4 +166,4 @@ try } CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Server/DTTool/DTTool.h b/dbms/src/Server/DTTool/DTTool.h index 40976fdbb1e..f0399b8de5d 100644 --- a/dbms/src/Server/DTTool/DTTool.h +++ b/dbms/src/Server/DTTool/DTTool.h @@ -116,7 +116,6 @@ class ImitativeEnv global_context->initializeGlobalPageIdAllocator(); global_context->initializeGlobalStoragePoolIfNeed(global_context->getPathPool()); raft_config.ignore_databases = {"default", "system"}; - raft_config.engine = TiDB::StorageEngine::DT; global_context->createTMTContext(raft_config, pingcap::ClusterConfig()); global_context->setDeltaIndexManager(1024 * 1024 * 100 /*100MB*/); diff --git a/dbms/src/Server/RaftConfigParser.cpp b/dbms/src/Server/RaftConfigParser.cpp index a151b07d4c6..0f5a1db5b06 100644 --- a/dbms/src/Server/RaftConfigParser.cpp +++ b/dbms/src/Server/RaftConfigParser.cpp @@ -85,20 +85,6 @@ TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::AbstractConfigura LOG_INFO(log, "Found ignore databases: {}", fmt_buf.toString()); } - if (config.has("raft.storage_engine")) - { - String s_engine = config.getString("raft.storage_engine"); - std::transform(s_engine.begin(), s_engine.end(), s_engine.begin(), [](char ch) { return std::tolower(ch); }); - if (s_engine == "tmt") - res.engine = ::TiDB::StorageEngine::TMT; - else if (s_engine == "dt") - res.engine = ::TiDB::StorageEngine::DT; - else - res.engine = DEFAULT_ENGINE; - } - - LOG_INFO(log, "Default storage engine [type={}]", static_cast(res.engine)); - return res; } diff --git a/dbms/src/Server/RaftConfigParser.h b/dbms/src/Server/RaftConfigParser.h index 2019374df5f..33dbcff85ae 100644 --- a/dbms/src/Server/RaftConfigParser.h +++ b/dbms/src/Server/RaftConfigParser.h @@ -44,9 +44,6 @@ struct TiFlashRaftConfig bool for_unit_test = false; - static constexpr TiDB::StorageEngine DEFAULT_ENGINE = TiDB::StorageEngine::DT; - TiDB::StorageEngine engine = DEFAULT_ENGINE; - public: TiFlashRaftConfig() = default; diff --git a/dbms/src/Storages/KVStore/TMTContext.cpp b/dbms/src/Storages/KVStore/TMTContext.cpp index 825d8d7d8cc..0ff11218734 100644 --- a/dbms/src/Storages/KVStore/TMTContext.cpp +++ b/dbms/src/Storages/KVStore/TMTContext.cpp @@ -158,7 +158,6 @@ TMTContext::TMTContext( context.getSettingsRef().task_scheduler_thread_soft_limit, context.getSettingsRef().task_scheduler_thread_hard_limit, context.getSettingsRef().task_scheduler_active_set_soft_limit))) - , engine(raft_config.engine) , batch_read_index_timeout_ms(DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS) , wait_index_timeout_ms(DEFAULT_WAIT_INDEX_TIMEOUT_MS) , read_index_worker_tick_ms(DEFAULT_READ_INDEX_WORKER_TICK_MS) diff --git a/dbms/src/Storages/KVStore/TMTContext.h b/dbms/src/Storages/KVStore/TMTContext.h index 55694ce3bc8..03bec14ab64 100644 --- a/dbms/src/Storages/KVStore/TMTContext.h +++ b/dbms/src/Storages/KVStore/TMTContext.h @@ -126,8 +126,6 @@ class TMTContext : private boost::noncopyable const std::unordered_set & getIgnoreDatabases() const; - ::TiDB::StorageEngine getEngineType() const { return engine; } - void reloadConfig(const Poco::Util::AbstractConfiguration & config); bool isInitialized() const; @@ -172,8 +170,6 @@ class TMTContext : private boost::noncopyable std::shared_ptr schema_sync_manager; MPPTaskManagerPtr mpp_task_manager; - ::TiDB::StorageEngine engine; - std::atomic_uint64_t batch_read_index_timeout_ms; std::atomic_uint64_t wait_index_timeout_ms; std::atomic_uint64_t read_index_worker_tick_ms; diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index 6a937f47031..2e93b3c67d9 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -164,7 +164,6 @@ void TiFlashTestEnv::addGlobalContext( TiFlashRaftConfig raft_config; raft_config.ignore_databases = {"system"}; - raft_config.engine = TiDB::StorageEngine::DT; raft_config.for_unit_test = true; global_context->createTMTContext(raft_config, pingcap::ClusterConfig()); diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index e5fccd1ebfe..7f46967e349 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -1089,27 +1089,16 @@ String createTableStmt( writeString(columns[i].type->getName(), stmt_buf); } - // storage engine type - if (table_info.engine_type == TiDB::StorageEngine::DT) + writeString(") Engine = DeltaMerge((", stmt_buf); + for (size_t i = 0; i < pks.size(); i++) { - writeString(") Engine = DeltaMerge((", stmt_buf); - for (size_t i = 0; i < pks.size(); i++) - { - if (i > 0) - writeString(", ", stmt_buf); - writeBackQuotedString(pks[i], stmt_buf); - } - writeString("), '", stmt_buf); - writeEscapedString(table_info.serialize(), stmt_buf); - writeString(fmt::format("', {})", tombstone), stmt_buf); - } - else - { - throw TiFlashException( - Errors::DDL::Internal, - "Unknown engine type : {}", - fmt::underlying(table_info.engine_type)); + if (i > 0) + writeString(", ", stmt_buf); + writeBackQuotedString(pks[i], stmt_buf); } + writeString("), '", stmt_buf); + writeEscapedString(table_info.serialize(), stmt_buf); + writeString(fmt::format("', {})", tombstone), stmt_buf); return stmt; } @@ -1138,12 +1127,6 @@ void SchemaBuilder::applyCreateStorageInstance( return; } // Else the storage instance does not exist, create it. - /// Normal CREATE table. - if (table_info->engine_type == StorageEngine::UNSPECIFIED) - { - auto & tmt_context = context.getTMTContext(); - table_info->engine_type = tmt_context.getEngineType(); - } // We need to create a Storage instance to handle its raft log and snapshot when it // is "dropped" but not physically removed in TiDB. To handle it porperly, we get a diff --git a/dbms/src/TiDB/Schema/TiDB.cpp b/dbms/src/TiDB/Schema/TiDB.cpp index e6e728721f2..485f205e45e 100644 --- a/dbms/src/TiDB/Schema/TiDB.cpp +++ b/dbms/src/TiDB/Schema/TiDB.cpp @@ -38,7 +38,8 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; -} +extern const int INCORRECT_DATA; +} // namespace ErrorCodes extern const UInt8 TYPE_CODE_LITERAL; extern const UInt8 LITERAL_NIL; @@ -381,27 +382,29 @@ try json->set("origin_default", origin_default_value); json->set("default", default_value); json->set("default_bit", default_bit_value); - Poco::JSON::Object::Ptr tp_json = new Poco::JSON::Object(); - tp_json->set("Tp", static_cast(tp)); - tp_json->set("Flag", flag); - tp_json->set("Flen", flen); - tp_json->set("Decimal", decimal); - tp_json->set("Charset", charset); - tp_json->set("Collate", collate); - if (!elems.empty()) - { - Poco::JSON::Array::Ptr elem_arr = new Poco::JSON::Array(); - for (const auto & elem : elems) - elem_arr->add(elem.first); - tp_json->set("Elems", elem_arr); - } - else { - tp_json->set("Elems", Poco::Dynamic::Var()); + // "type" field + Poco::JSON::Object::Ptr tp_json = new Poco::JSON::Object(); + tp_json->set("Tp", static_cast(tp)); + tp_json->set("Flag", flag); + tp_json->set("Flen", flen); + tp_json->set("Decimal", decimal); + tp_json->set("Charset", charset); + tp_json->set("Collate", collate); + if (!elems.empty()) + { + Poco::JSON::Array::Ptr elem_arr = new Poco::JSON::Array(); + for (const auto & elem : elems) + elem_arr->add(elem.first); + tp_json->set("Elems", elem_arr); + } + else + { + tp_json->set("Elems", Poco::Dynamic::Var()); + } + json->set("type", tp_json); } - json->set("type", tp_json); json->set("state", static_cast(state)); - json->set("comment", comment); #ifndef NDEBUG // Check stringify in Debug mode @@ -430,28 +433,30 @@ try default_value = json->get("default"); if (!json->isNull("default_bit")) default_bit_value = json->get("default_bit"); - auto type_json = json->getObject("type"); - tp = static_cast(type_json->getValue("Tp")); - flag = type_json->getValue("Flag"); - flen = type_json->getValue("Flen"); - decimal = type_json->getValue("Decimal"); - if (!type_json->isNull("Elems")) { - auto elems_arr = type_json->getArray("Elems"); - size_t elems_size = elems_arr->size(); - for (size_t i = 1; i <= elems_size; i++) + // type + auto type_json = json->getObject("type"); + tp = static_cast(type_json->getValue("Tp")); + flag = type_json->getValue("Flag"); + flen = type_json->getValue("Flen"); + decimal = type_json->getValue("Decimal"); + if (!type_json->isNull("Elems")) { - elems.push_back(std::make_pair(elems_arr->getElement(i - 1), static_cast(i))); + auto elems_arr = type_json->getArray("Elems"); + size_t elems_size = elems_arr->size(); + for (size_t i = 1; i <= elems_size; i++) + { + elems.push_back(std::make_pair(elems_arr->getElement(i - 1), static_cast(i))); + } } + /// need to do this check for forward compatibility + if (!type_json->isNull("Charset")) + charset = type_json->get("Charset"); + /// need to do this check for forward compatibility + if (!type_json->isNull("Collate")) + collate = type_json->get("Collate"); } - /// need to do this check for forward compatibility - if (!type_json->isNull("Charset")) - charset = type_json->get("Charset"); - /// need to do this check for forward compatibility - if (!type_json->isNull("Collate")) - collate = type_json->get("Collate"); state = static_cast(json->getValue("state")); - comment = json->getValue("comment"); } catch (const Poco::Exception & e) { @@ -478,7 +483,6 @@ try name_json->set("O", name); name_json->set("L", name); json->set("name", name_json); - json->set("comment", comment); #ifndef NDEBUG // Check stringify in Debug mode @@ -500,8 +504,6 @@ try { id = json->getValue("id"); name = json->getObject("name")->getValue("L"); - if (json->has("comment")) - comment = json->getValue("comment"); } catch (const Poco::Exception & e) { @@ -766,16 +768,10 @@ catch (const Poco::Exception & e) /////////////////////// IndexInfo::IndexInfo(Poco::JSON::Object::Ptr json) - : id(0) - , state(TiDB::SchemaState::StateNone) - , index_type(0) - , is_unique(true) - , is_primary(true) - , is_invisible(true) - , is_global(true) { deserialize(json); } + Poco::JSON::Object::Ptr IndexInfo::getJSONObject() const try { @@ -788,11 +784,6 @@ try idx_name_json->set("L", idx_name); json->set("idx_name", idx_name_json); - Poco::JSON::Object::Ptr tbl_name_json = new Poco::JSON::Object(); - tbl_name_json->set("O", tbl_name); - tbl_name_json->set("L", tbl_name); - json->set("tbl_name", tbl_name_json); - Poco::JSON::Array::Ptr cols_array = new Poco::JSON::Array(); for (const auto & col : idx_cols) { @@ -826,7 +817,6 @@ try { id = json->getValue("id"); idx_name = json->getObject("idx_name")->getValue("L"); - tbl_name = json->getObject("tbl_name")->getValue("L"); auto cols_array = json->getArray("idx_cols"); idx_cols.clear(); @@ -907,7 +897,6 @@ try json->set("state", static_cast(state)); json->set("pk_is_handle", pk_is_handle); json->set("is_common_handle", is_common_handle); - json->set("comment", comment); json->set("update_timestamp", update_timestamp); if (is_partition_table) { @@ -972,16 +961,21 @@ try } } - auto index_arr = obj->getArray("index_info"); index_infos.clear(); - if (!index_arr.isNull()) + bool has_primary_index = false; + if (auto index_arr = obj->getArray("index_info"); !index_arr.isNull()) { for (size_t i = 0; i < index_arr->size(); i++) { auto index_info_json = index_arr->getObject(i); IndexInfo index_info(index_info_json); + // We only keep the "primary index" in tiflash now if (index_info.is_primary) - index_infos.emplace_back(index_info); + { + has_primary_index = true; + // always put the primary_index at the front of all index_info + index_infos.insert(index_infos.begin(), std::move(index_info)); + } } } @@ -989,9 +983,6 @@ try pk_is_handle = obj->getValue("pk_is_handle"); if (obj->has("is_common_handle")) is_common_handle = obj->getValue("is_common_handle"); - if (!is_common_handle) - index_infos.clear(); - comment = obj->getValue("comment"); if (obj->has("update_timestamp")) update_timestamp = obj->getValue("update_timestamp"); auto partition_obj = obj->getObject("partition"); @@ -1022,12 +1013,13 @@ try replica_info.deserialize(replica_obj); } } - if (is_common_handle && index_infos.size() != 1) + if (is_common_handle && !has_primary_index) { throw DB::Exception( - std::string(__PRETTY_FUNCTION__) - + ": Parse TiDB schema JSON failed (TableInfo): clustered index without primary key info, json: " - + JSONToString(obj)); + DB::ErrorCodes::INCORRECT_DATA, + "{}: Parse TiDB schema JSON failed (TableInfo): clustered index without primary key info, json: ", + __PRETTY_FUNCTION__, + JSONToString(obj)); } } catch (const Poco::Exception & e) diff --git a/dbms/src/TiDB/Schema/TiDB.h b/dbms/src/TiDB/Schema/TiDB.h index c42c1a7e612..88398458139 100644 --- a/dbms/src/TiDB/Schema/TiDB.h +++ b/dbms/src/TiDB/Schema/TiDB.h @@ -125,7 +125,6 @@ struct ColumnInfo // Elems is the element list for enum and set type. std::vector> elems; SchemaState state = StateNone; - String comment; #ifdef M #error "Please undefine macro M first." @@ -167,7 +166,6 @@ struct PartitionDefinition TableID id = DB::InvalidTableID; String name; // LessThan []string `json:"less_than"` - String comment; }; struct PartitionInfo @@ -252,7 +250,6 @@ struct IndexInfo Int64 id = -1; String idx_name; - String tbl_name; std::vector idx_cols; SchemaState state = StatePublic; Int32 index_type = -1; @@ -292,14 +289,12 @@ struct TableInfo std::vector columns; /// index_infos stores the index info from TiDB. But we do not store all /// the index infos because most of the index info is useless in TiFlash. - /// If is_common_handle = true, the primary index info is stored - /// otherwise, all of the index info are ignored + /// Only the primary index info is stored now std::vector index_infos; SchemaState state = StateNone; bool pk_is_handle = false; /// when is_common_handle = true, it means this table is a clustered index table bool is_common_handle = false; - String comment; Timestamp update_timestamp = 0; bool is_partition_table = false; TableID belonging_table_id = DB::InvalidTableID; @@ -314,8 +309,6 @@ struct TableInfo // The TiFlash replica info persisted by TiDB TiFlashReplicaInfo replica_info; - TiDB::StorageEngine engine_type = TiDB::StorageEngine::UNSPECIFIED; // TODO(hyy):seems could be removed - ColumnID getColumnID(const String & name) const; String getColumnName(ColumnID id) const; KeyspaceID getKeyspaceID() const; diff --git a/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp b/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp index 14b1e6ad2f7..35f76da50d2 100644 --- a/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp +++ b/dbms/src/TiDB/Schema/tests/gtest_schema_sync.cpp @@ -252,7 +252,7 @@ try {"t1", cols, ""}, {"t2", cols, ""}, }; - auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS(), "dt"); + auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS()); refreshSchema(); @@ -295,7 +295,7 @@ try {"t1", cols, ""}, {"t2", cols, ""}, }; - auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS(), "dt"); + auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS()); refreshSchema(); for (auto table_id : table_ids) @@ -350,7 +350,7 @@ try std::vector> tables{ {"t1", cols, ""}, }; - auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS(), "dt"); + auto table_ids = MockTiDB::instance().newTables(db_name, tables, pd_client->getTS()); refreshSchema(); for (auto table_id : table_ids) @@ -420,7 +420,7 @@ try }); MockTiDB::instance().newDataBase(db_name); - auto logical_table_id = MockTiDB::instance().newTable(db_name, tbl_name, cols, pd_client->getTS(), "", "dt"); + auto logical_table_id = MockTiDB::instance().newTable(db_name, tbl_name, cols, pd_client->getTS(), ""); auto part1_id = MockTiDB::instance().newPartition(logical_table_id, "red", pd_client->getTS(), /*is_add_part*/ true); auto part2_id @@ -485,7 +485,6 @@ try cols, pd_client->getTS(), "", - "dt", {"red", "blue", "yellow"}); ASSERT_EQ(physical_table_ids.size(), 3); @@ -583,7 +582,6 @@ try cols, pd_client->getTS(), "", - "dt", {"red", "blue", "yellow"}); ASSERT_EQ(physical_table_ids.size(), 3); @@ -702,7 +700,7 @@ try }); auto db_id = MockTiDB::instance().newDataBase(db_name); - auto logical_table_id = MockTiDB::instance().newTable(db_name, tbl_name, cols, pd_client->getTS(), "", "dt"); + auto logical_table_id = MockTiDB::instance().newTable(db_name, tbl_name, cols, pd_client->getTS(), ""); auto part1_id = MockTiDB::instance().newPartition(logical_table_id, "red", pd_client->getTS(), /*is_add_part*/ true); auto part2_id diff --git a/dbms/src/TiDB/tests/gtest_table_info.cpp b/dbms/src/TiDB/Schema/tests/gtest_table_info.cpp similarity index 73% rename from dbms/src/TiDB/tests/gtest_table_info.cpp rename to dbms/src/TiDB/Schema/tests/gtest_table_info.cpp index e96b757f47d..5d622c3e240 100644 --- a/dbms/src/TiDB/tests/gtest_table_info.cpp +++ b/dbms/src/TiDB/Schema/tests/gtest_table_info.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -150,24 +151,19 @@ struct StmtCase EXPECT_EQ(json1, json2) << "Table info unescaped serde mismatch:\n" + json1 + "\n" + json2; // 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.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); - table_info.deserialize(json1); - json2 = table_info.serialize(); - EXPECT_EQ(json1, json2) << "Table info escaped serde mismatch:\n" + json1 + "\n" + json2; - }; - - verify_stmt(TiDB::StorageEngine::DT); + 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); + table_info.deserialize(json1); + json2 = table_info.serialize(); + EXPECT_EQ(json1, json2) << "Table info escaped serde mismatch:\n" + json1 + "\n" + json2; } private: @@ -190,57 +186,58 @@ try 1145, // 0, R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":1145,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","partition":null})json", // - R"stmt(CREATE TABLE `db_1939`.`t_1145`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":1145,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":0}', 0))stmt", // + R"json({"id":1145,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5}],"state":5,"pk_is_handle":true,"schema_version":-1,"partition":null})json", // + R"stmt(CREATE TABLE `db_1939`.`t_1145`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"id":1145,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":0}', 0))stmt", // }, StmtCase{ 2049, // 0, R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","update_timestamp":404545295996944390,"partition":null})json", // - R"stmt(CREATE TABLE `db_1939`.`t_2049`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":2049,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}', 0))stmt", // + R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5}],"state":5,"pk_is_handle":true,"schema_version":-1,"update_timestamp":404545295996944390,"partition":null})json", // + R"stmt(CREATE TABLE `db_1939`.`t_2049`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"id":2049,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}', 0))stmt", // }, StmtCase{ 31, // 0, R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db_1`.`t_31`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":31,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}', 0))stmt", // + R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db_1`.`t_31`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"id":31,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}', 0))stmt", // }, StmtCase{ 33, // 0, R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db_2`.`t_33`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"comment":"","id":33,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}', 0))stmt", // + R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db_2`.`t_33`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"id":33,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}', 0))stmt", // }, StmtCase{ 35, // 0, R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db_1`.`t_35`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":35,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}', 0))stmt", // + R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db_1`.`t_35`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"id":35,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}', 0))stmt", // }, StmtCase{ 37, // 0, R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", - R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5,"comment":""}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"comment":"","index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db_2`.`t_37`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"comment":"","id":37,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}', 0))stmt", // + R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null})json", // + // The primary index is kept + R"stmt(CREATE TABLE `db_2`.`t_37`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"id":37,"index_info":[{"id":1,"idx_cols":[{"length":-1,"name":{"L":"mycol","O":"mycol"},"offset":0}],"idx_name":{"L":"primary","O":"primary"},"index_type":1,"is_global":false,"is_invisible":false,"is_primary":true,"is_unique":true,"state":5}],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}', 0))stmt", // }, StmtCase{ 32, // 0, R"json({"id":1,"db_name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // - R"stmt(CREATE TABLE `db_1`.`t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"keyspace_id":4294967295,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}', 0))stmt", // + R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // + R"stmt(CREATE TABLE `db_1`.`t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"keyspace_id":4294967295,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}', 0))stmt", // }, StmtCase{ 32, // 1700815239, R"json({"id":1,"db_name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // - R"stmt(CREATE TABLE `db_1`.`t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"keyspace_id":4294967295,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}', 1700815239))stmt", // + R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // + R"stmt(CREATE TABLE `db_1`.`t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"keyspace_id":4294967295,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}', 1700815239))stmt", // }, }; diff --git a/tests/delta-merge-test/raft/bugs/FLASH-484.test b/tests/delta-merge-test/raft/bugs/FLASH-484.test index 045b93351be..0fe9bc7f9a1 100644 --- a/tests/delta-merge-test/raft/bugs/FLASH-484.test +++ b/tests/delta-merge-test/raft/bugs/FLASH-484.test @@ -22,7 +22,7 @@ ## create a DeltaMerge table -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) diff --git a/tests/delta-merge-test/raft/ingest_sst.test b/tests/delta-merge-test/raft/ingest_sst.test index 0963aeaf974..279724eed85 100644 --- a/tests/delta-merge-test/raft/ingest_sst.test +++ b/tests/delta-merge-test/raft/ingest_sst.test @@ -18,7 +18,7 @@ => DBGInvoke __drop_tidb_table(default, test) => DBGInvoke __refresh_schemas() -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64') => DBGInvoke __region_snapshot(4, 0, 1000, default, test) => DBGInvoke __region_ingest_sst(default, test, 4, 3, 6) => DBGInvoke query_mapped('select * from \$d.\$t', default, test) diff --git a/tests/delta-merge-test/raft/read_with_specify_tso.test b/tests/delta-merge-test/raft/read_with_specify_tso.test index 54806395be0..5d714fb1370 100644 --- a/tests/delta-merge-test/raft/read_with_specify_tso.test +++ b/tests/delta-merge-test/raft/read_with_specify_tso.test @@ -19,7 +19,7 @@ => DBGInvoke __refresh_schemas() ## create table -=> DBGInvoke __mock_tidb_table(default, test_dm, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test_dm, 'col_1 Int64') => DBGInvoke __refresh_mapped_table_schema(default, test_dm) => DBGInvoke query_mapped('select tidb_database,tidb_name,engine from system.tables where database=''\$d'' and name=''\$t''', default, test_dm) diff --git a/tests/delta-merge-test/raft/region_merge.test b/tests/delta-merge-test/raft/region_merge.test index 2450c83c8b5..9c4004b2dae 100644 --- a/tests/delta-merge-test/raft/region_merge.test +++ b/tests/delta-merge-test/raft/region_merge.test @@ -17,7 +17,7 @@ => DBGInvoke __drop_tidb_table(default, test) => DBGInvoke __refresh_schemas() -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '') => DBGInvoke __region_snapshot(4, 0, 100, default, test) diff --git a/tests/delta-merge-test/raft/region_merge_common_handle.test b/tests/delta-merge-test/raft/region_merge_common_handle.test index eb05fc51b63..145de8a0020 100644 --- a/tests/delta-merge-test/raft/region_merge_common_handle.test +++ b/tests/delta-merge-test/raft/region_merge_common_handle.test @@ -17,7 +17,7 @@ => DBGInvoke __drop_tidb_table(default, test) => DBGInvoke __refresh_schemas() -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2') => DBGInvoke __region_snapshot(4, 'test0', 0, 'test100', 100, default, test) diff --git a/tests/delta-merge-test/raft/remove_region.test b/tests/delta-merge-test/raft/remove_region.test index 3a9fb1fc469..d08ded7bcaa 100644 --- a/tests/delta-merge-test/raft/remove_region.test +++ b/tests/delta-merge-test/raft/remove_region.test @@ -22,7 +22,7 @@ ## create a DeltaMerge table -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke query_mapped('select tidb_database,tidb_name,engine from system.tables where database=''\$d'' and name=''\$t''', default, test) ┌─database─┬─name─┬─engine─────┐ diff --git a/tests/delta-merge-test/raft/remove_region_common_handle.test b/tests/delta-merge-test/raft/remove_region_common_handle.test index 6487f73c0fb..2df5014ad25 100644 --- a/tests/delta-merge-test/raft/remove_region_common_handle.test +++ b/tests/delta-merge-test/raft/remove_region_common_handle.test @@ -21,7 +21,7 @@ => DBGInvoke __refresh_schemas() ## create a DeltaMerge table -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke query_mapped('select tidb_database,tidb_name,engine from system.tables where database=''\$d'' and name=''\$t''', default, test) ┌─database─┬─name─┬─engine─────┐ diff --git a/tests/delta-merge-test/raft/schema/alter_for_nullable.test b/tests/delta-merge-test/raft/schema/alter_for_nullable.test index 0738c1fa99d..da466eab5ef 100644 --- a/tests/delta-merge-test/raft/schema/alter_for_nullable.test +++ b/tests/delta-merge-test/raft/schema/alter_for_nullable.test @@ -23,7 +23,7 @@ # Sync add column by reading. -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8, col_3 Int32', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8, col_3 Int32') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1, 3) @@ -61,7 +61,7 @@ => DBGInvoke __drop_tidb_table(default, test) # Test convert nullable type to not-null type. -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)') => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) => DBGInvoke __raft_insert_row(default, test, 4, 51, 'test', 2) diff --git a/tests/delta-merge-test/raft/schema/alter_on_read.test b/tests/delta-merge-test/raft/schema/alter_on_read.test index 35cbd6ed2c8..a573a6f5757 100644 --- a/tests/delta-merge-test/raft/schema/alter_on_read.test +++ b/tests/delta-merge-test/raft/schema/alter_on_read.test @@ -20,7 +20,7 @@ # Sync add column by reading. -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke __add_column_to_tidb_table(default, test, 'col_2 Nullable(Int8)') diff --git a/tests/delta-merge-test/raft/schema/alter_on_write.test b/tests/delta-merge-test/raft/schema/alter_on_write.test index 4dd269beb58..1c1f5c51267 100644 --- a/tests/delta-merge-test/raft/schema/alter_on_write.test +++ b/tests/delta-merge-test/raft/schema/alter_on_write.test @@ -20,7 +20,7 @@ # Sync add column by checking missing column in CH when flushing. -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke query_mapped('select col_1 from \$d.\$t', default, test) diff --git a/tests/delta-merge-test/raft/schema/concurrent_ddl_conflict.test b/tests/delta-merge-test/raft/schema/concurrent_ddl_conflict.test index 5f9eaacd62a..379fb6d9c17 100644 --- a/tests/delta-merge-test/raft/schema/concurrent_ddl_conflict.test +++ b/tests/delta-merge-test/raft/schema/concurrent_ddl_conflict.test @@ -17,7 +17,7 @@ => DBGInvoke __drop_tidb_table(default, test) -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke __skip_schema_version() diff --git a/tests/delta-merge-test/raft/schema/default_value.test b/tests/delta-merge-test/raft/schema/default_value.test index ba1908121ba..25a21dd2139 100644 --- a/tests/delta-merge-test/raft/schema/default_value.test +++ b/tests/delta-merge-test/raft/schema/default_value.test @@ -15,7 +15,7 @@ => DBGInvoke __enable_schema_sync_service('false') => DBGInvoke __drop_tidb_table(default, test) -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __put_region(4, 0, 100, default, test) diff --git a/tests/delta-merge-test/raft/schema/drop_on_read.test b/tests/delta-merge-test/raft/schema/drop_on_read.test index bdb9069b9fa..8d35750ae1f 100644 --- a/tests/delta-merge-test/raft/schema/drop_on_read.test +++ b/tests/delta-merge-test/raft/schema/drop_on_read.test @@ -19,7 +19,7 @@ -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1') @@ -32,7 +32,7 @@ └──────────────────────────────┘ => DBGInvoke __clean_up_region() -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Nullable(Int8)') => DBGInvoke query_mapped('select * from \$d.\$t', default, test) => DBGInvoke __drop_tidb_table(default, test) diff --git a/tests/delta-merge-test/raft/schema/drop_on_restart.test b/tests/delta-merge-test/raft/schema/drop_on_restart.test index b4d2f0c8e8c..5b796d7a234 100644 --- a/tests/delta-merge-test/raft/schema/drop_on_restart.test +++ b/tests/delta-merge-test/raft/schema/drop_on_restart.test @@ -28,8 +28,8 @@ # share the same data path => DBGInvoke __mock_tidb_db(db3) => DBGInvoke __mock_tidb_db(db4) -=> DBGInvoke __mock_tidb_table(db3, test3, 'col_1 String', '', 'dt') -=> DBGInvoke __mock_tidb_table(db4, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(db3, test3, 'col_1 String') +=> DBGInvoke __mock_tidb_table(db4, test, 'col_1 String') => DBGInvoke __refresh_schemas() => DBGInvoke __refresh_mapped_table_schema(db3, test3) => DBGInvoke __refresh_mapped_table_schema(db4, test) diff --git a/tests/delta-merge-test/raft/schema/mydate.test b/tests/delta-merge-test/raft/schema/mydate.test index c592e1e8e00..7c39f496242 100644 --- a/tests/delta-merge-test/raft/schema/mydate.test +++ b/tests/delta-merge-test/raft/schema/mydate.test @@ -16,7 +16,7 @@ => DBGInvoke __drop_tidb_table(default, test) => DBGInvoke __refresh_schemas() -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDateTime(1)', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDateTime(1)') => DBGInvoke __put_region(4, 0, 100, default, test) @@ -34,7 +34,7 @@ => DBGInvoke __drop_tidb_table(default, test) -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDate', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDate') => DBGInvoke __put_region(4, 0, 100, default, test) @@ -51,7 +51,7 @@ └────────────┴─────────────┘ => DBGInvoke __drop_tidb_table(default, test) -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDateTime(1)', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 MyDateTime(1)') => DBGInvoke __put_region(4, 0, 100, default, test) diff --git a/tests/delta-merge-test/raft/schema/rename_column.test b/tests/delta-merge-test/raft/schema/rename_column.test index b336c623b6c..aabd5162351 100644 --- a/tests/delta-merge-test/raft/schema/rename_column.test +++ b/tests/delta-merge-test/raft/schema/rename_column.test @@ -21,7 +21,7 @@ # create table and insert some rows -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke __raft_insert_row(default, test, 4, 50, 'test', 1) @@ -61,7 +61,7 @@ => DBGInvoke __drop_tidb_table(default, test) => DBGInvoke __refresh_schemas() -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) @@ -109,7 +109,7 @@ => DBGInvoke __refresh_schemas() -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) @@ -151,7 +151,7 @@ # do a lot of change and sync at once. -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int8') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) @@ -192,7 +192,7 @@ => DBGInvoke __refresh_schemas() ## test for partial-linked rename -=> DBGInvoke __mock_tidb_table(default, test, 'a String, b Int8', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'a String, b Int8') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) diff --git a/tests/delta-merge-test/raft/schema/truncate_on_read.test b/tests/delta-merge-test/raft/schema/truncate_on_read.test index b0bbc82eba0..f499d5390b0 100644 --- a/tests/delta-merge-test/raft/schema/truncate_on_read.test +++ b/tests/delta-merge-test/raft/schema/truncate_on_read.test @@ -20,7 +20,7 @@ -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 0, 100, default, test) => DBGInvoke __raft_insert_row(default, test, 4, 50, 'test1') diff --git a/tests/delta-merge-test/raft/snapshot.test b/tests/delta-merge-test/raft/snapshot.test index 1ecf098ccfa..621898f4c43 100644 --- a/tests/delta-merge-test/raft/snapshot.test +++ b/tests/delta-merge-test/raft/snapshot.test @@ -20,7 +20,7 @@ => DBGInvoke __refresh_schemas() ## create table and apply an empty snapshot -=> DBGInvoke __mock_tidb_table(default, test_dm, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test_dm, 'col_1 Int64') => DBGInvoke __refresh_mapped_table_schema(default, test_dm) => DBGInvoke query_mapped('select tidb_database,tidb_name,engine from system.tables where database=''\$d'' and name=''\$t''', default, test_dm) ┌─database─┬─name────┬─engine─────┐ diff --git a/tests/delta-merge-test/raft/snapshot_common_handle.test b/tests/delta-merge-test/raft/snapshot_common_handle.test index 00ed51398da..584e4f81e73 100644 --- a/tests/delta-merge-test/raft/snapshot_common_handle.test +++ b/tests/delta-merge-test/raft/snapshot_common_handle.test @@ -19,7 +19,7 @@ => DBGInvoke __refresh_schemas() ## create table and apply an empty snapshot -=> DBGInvoke __mock_tidb_table(default, test_dm, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2', 'dt') +=> DBGInvoke __mock_tidb_table(default, test_dm, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2') => DBGInvoke __refresh_mapped_table_schema(default, test_dm) => DBGInvoke query_mapped('select tidb_database,tidb_name,engine from system.tables where database=''\$d'' and name=''\$t''', default, test_dm) ┌─database─┬─name────┬─engine─────┐ diff --git a/tests/delta-merge-test/raft/snapshot_dtfile.test b/tests/delta-merge-test/raft/snapshot_dtfile.test index 1cdd2b75078..e24ba36800c 100644 --- a/tests/delta-merge-test/raft/snapshot_dtfile.test +++ b/tests/delta-merge-test/raft/snapshot_dtfile.test @@ -20,7 +20,7 @@ ##### ## Pre-handle region to dt files then apply -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __region_snapshot(4, 0, 10000, default, test) => DBGInvoke region_snapshot_pre_handle_file(default, test, 4, 3, 6, 'col_1 Int64', '') @@ -84,7 +84,7 @@ ## Recreate test table => DBGInvoke __drop_tidb_table(default, test) => drop table if exists default.test -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64') => DBGInvoke __refresh_schemas() => DBGInvoke __region_snapshot(4, 0, 10000, default, test) ## Mock to test apply snapshot with an older schema (case 1 - add column) @@ -119,7 +119,7 @@ ## Recreate test table => DBGInvoke __drop_tidb_table(default, test) => drop table if exists default.test -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64') => DBGInvoke __refresh_schemas() => DBGInvoke __region_snapshot(4, 0, 10000, default, test) ## Mock to test apply snapshot with an older schema (case 2 - drop column) @@ -153,7 +153,7 @@ ## Test that BR/lightning may only ingest sst files of write cf into tikv without default cf => DBGInvoke __drop_tidb_table(default, test) => drop table if exists default.test -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64') => DBGInvoke __refresh_schemas() => DBGInvoke __region_snapshot(4, 0, 10000, default, test) # Only the sst files of write cf ingested and send to TiFlash as a snapshot, no panic, and those uncommitted data remain in memory @@ -183,7 +183,7 @@ ## selraw will ignore 'delete_range', so rebuild the table first => DBGInvoke __drop_tidb_table(default, test) => drop table if exists default.test -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64') => DBGInvoke __refresh_schemas() => DBGInvoke __region_snapshot(4, 0, 10000, default, test) => DBGInvoke __region_snapshot_pre_handle_file_pks(default, test, 4, 'col_1 Int64', '', 3,4,5, 5,5,5, 5,6,7, 8,9) diff --git a/tests/delta-merge-test/raft/sync_table_from_raft.test b/tests/delta-merge-test/raft/sync_table_from_raft.test index fce7cba76e9..7d1635664a1 100644 --- a/tests/delta-merge-test/raft/sync_table_from_raft.test +++ b/tests/delta-merge-test/raft/sync_table_from_raft.test @@ -22,7 +22,7 @@ ## create a DeltaMerge table -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String', '', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke query_mapped('select tidb_database,tidb_name,engine from system.tables where database=''\$d'' and name=''\$t''', default, test) ┌─database─┬─name─┬─engine─────┐ diff --git a/tests/delta-merge-test/raft/sync_table_from_raft_common_handle.test b/tests/delta-merge-test/raft/sync_table_from_raft_common_handle.test index e7a7f27a371..7b20942cd1d 100644 --- a/tests/delta-merge-test/raft/sync_table_from_raft_common_handle.test +++ b/tests/delta-merge-test/raft/sync_table_from_raft_common_handle.test @@ -21,7 +21,7 @@ => DBGInvoke __refresh_schemas() ## create a DeltaMerge table -=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2', 'dt') +=> DBGInvoke __mock_tidb_table(default, test, 'col_1 String, col_2 Int64, col_3 Int64', 'col_1,col_2') => DBGInvoke __refresh_mapped_table_schema(default, test) => DBGInvoke __put_region(4, 'test0',0,'test100', 100, default, test)