Skip to content

Commit

Permalink
ddl: Remove field "comment" from TiDB::TableInfo (#9339)
Browse files Browse the repository at this point in the history
ref #6233

ddl: Remove field "comment" from TiDB::TableInfo

  * Remove `comment` field from `TableInfo`, `ColumnInfo`, etc
  * Remove `table_name` from `IndexInfo` because it is useless
  * Remove storage_engine_type from `TableInfo`. We used to use it to create "StorageDeltaMerge" or "StorageTransactionMergeTree" before v4.0 is released. But the later one has been totally removed from TiFlash.
  • Loading branch information
JaySon-Huang authored Aug 22, 2024
1 parent 0d0463a commit e139b95
Show file tree
Hide file tree
Showing 40 changed files with 153 additions and 249 deletions.
4 changes: 2 additions & 2 deletions dbms/src/Debug/MockKVStore/MockRaftStoreProxy.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
31 changes: 6 additions & 25 deletions dbms/src/Debug/MockTiDB.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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());
Expand All @@ -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<TiDB::TableInfo>(std::move(table_info));
}

Expand All @@ -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);

Expand All @@ -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));
Expand All @@ -346,7 +329,6 @@ std::tuple<TableID, std::vector<TableID>> 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);
Expand All @@ -364,7 +346,7 @@ std::tuple<TableID, std::vector<TableID>> MockTiDB::newPartitionTable(

std::vector<TableID> 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;
Expand All @@ -385,8 +367,7 @@ std::tuple<TableID, std::vector<TableID>> MockTiDB::newPartitionTable(
std::vector<TableID> MockTiDB::newTables(
const String & database_name,
const std::vector<std::tuple<String, ColumnsDescription, String>> & tables,
Timestamp tso,
const String & engine_type)
Timestamp tso)
{
std::scoped_lock lock(tables_mutex);
std::vector<TableID> table_ids;
Expand All @@ -407,7 +388,7 @@ std::vector<TableID> 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;

Expand Down
10 changes: 3 additions & 7 deletions dbms/src/Debug/MockTiDB.h
Original file line number Diff line number Diff line change
Expand Up @@ -84,8 +84,7 @@ class MockTiDB : public ext::Singleton<MockTiDB>
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 <logical_table_id, [physical_table_id0, physical_table_id1, ...]>
Expand All @@ -95,22 +94,19 @@ class MockTiDB : public ext::Singleton<MockTiDB>
const ColumnsDescription & columns,
Timestamp tso,
const String & handle_pk_name,
const String & engine_type,
const Strings & part_names);

std::vector<TableID> newTables(
const String & database_name,
const std::vector<std::tuple<String, ColumnsDescription, String>> & 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);

Expand Down
17 changes: 3 additions & 14 deletions dbms/src/Debug/dbgFuncMockTiDBTable.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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<const ASTIdentifier &>(*args[0]).name;
Expand All @@ -65,16 +64,9 @@ void MockTiDBTable::dbgFuncMockTiDBTable(Context & context, const ASTs & args, D
ColumnsDescription columns
= InterpreterCreateQuery::getColumnsDescription(typeid_cast<const ASTExpressionList &>(*columns_ast), context);

String engine_type("dt");
if (context.getTMTContext().getEngineType() == ::TiDB::StorageEngine::TMT)
engine_type = "tmt";
if (args.size() == 5)
engine_type = safeGet<String>(typeid_cast<const ASTLiteral &>(*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));
}
Expand Down Expand Up @@ -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("");
}

Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Debug/dbgFuncMockTiDBTable.h
Original file line number Diff line number Diff line change
Expand Up @@ -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);

Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Debug/dbgKVStore/dbgFuncMockRaftSnapshot.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -727,7 +727,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFiles(
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(
typeid_cast<const ASTExpressionList &>(*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);
Expand Down Expand Up @@ -830,7 +830,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFilesWithHandles(
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(
typeid_cast<const ASTExpressionList &>(*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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -167,4 +166,4 @@ try
}
CATCH
} // namespace tests
} // namespace DB
} // namespace DB
1 change: 0 additions & 1 deletion dbms/src/Server/DTTool/DTTool.h
Original file line number Diff line number Diff line change
Expand Up @@ -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*/);
Expand Down
14 changes: 0 additions & 14 deletions dbms/src/Server/RaftConfigParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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<Int64>(res.engine));

return res;
}

Expand Down
3 changes: 0 additions & 3 deletions dbms/src/Server/RaftConfigParser.h
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down
1 change: 0 additions & 1 deletion dbms/src/Storages/KVStore/TMTContext.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
4 changes: 0 additions & 4 deletions dbms/src/Storages/KVStore/TMTContext.h
Original file line number Diff line number Diff line change
Expand Up @@ -126,8 +126,6 @@ class TMTContext : private boost::noncopyable

const std::unordered_set<std::string> & getIgnoreDatabases() const;

::TiDB::StorageEngine getEngineType() const { return engine; }

void reloadConfig(const Poco::Util::AbstractConfiguration & config);

bool isInitialized() const;
Expand Down Expand Up @@ -172,8 +170,6 @@ class TMTContext : private boost::noncopyable
std::shared_ptr<TiDBSchemaSyncerManager> 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;
Expand Down
1 change: 0 additions & 1 deletion dbms/src/TestUtils/TiFlashTestEnv.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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());

Expand Down
33 changes: 8 additions & 25 deletions dbms/src/TiDB/Schema/SchemaBuilder.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down Expand Up @@ -1138,12 +1127,6 @@ void SchemaBuilder<Getter, NameMapper>::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
Expand Down
Loading

0 comments on commit e139b95

Please sign in to comment.