Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#8721
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <[email protected]>
  • Loading branch information
JaySon-Huang authored and ti-chi-bot committed Jan 24, 2024
1 parent 0618c04 commit b23daac
Show file tree
Hide file tree
Showing 21 changed files with 656 additions and 100 deletions.
1 change: 1 addition & 0 deletions dbms/src/Common/FailPoint.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ namespace DB
M(skip_check_segment_update) \
M(force_set_page_file_write_errno) \
M(force_split_io_size_4k) \
M(force_set_num_regions_for_table) \
M(minimum_block_size_for_cross_join) \
M(random_exception_after_dt_write_done) \
M(random_slow_page_storage_write) \
Expand Down
5 changes: 5 additions & 0 deletions dbms/src/Debug/MockRaftStoreProxy.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -471,8 +471,13 @@ void MockRaftStoreProxy::unsafeInvokeForTest(std::function<void(MockRaftStorePro
void MockRaftStoreProxy::bootstrapWithRegion(
KVStore & kvs,
TMTContext & tmt,
<<<<<<< HEAD:dbms/src/Debug/MockRaftStoreProxy.cpp
UInt64 region_id,
std::optional<std::pair<std::string, std::string>> maybe_range)
=======
RegionID region_id,
std::optional<std::pair<std::string, std::string>> maybe_range) NO_THREAD_SAFETY_ANALYSIS
>>>>>>> 6058d19646 (ddl: Fix the storage instance may be physically dropped when the region is not removed (#8721)):dbms/src/Debug/MockKVStore/MockRaftStoreProxy.cpp
{
{
auto _ = genLockGuard();
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Debug/MockRaftStoreProxy.h
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ struct MockRaftStoreProxy : MutexLockWrap
void bootstrapWithRegion(
KVStore & kvs,
TMTContext & tmt,
UInt64 region_id,
RegionID region_id,
std::optional<std::pair<std::string, std::string>> maybe_range);

/// Boostrap a table.
Expand Down
59 changes: 53 additions & 6 deletions dbms/src/Debug/MockTiDB.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ MockTiDB::MockTiDB()
databases["default"] = 0;
}

TablePtr MockTiDB::dropTableInternal(
TablePtr MockTiDB::dropTableByNameImpl(
Context & context,
const String & database_name,
const String & table_name,
Expand All @@ -79,10 +79,41 @@ TablePtr MockTiDB::dropTableInternal(
if (it_by_name == tables_by_name.end())
return nullptr;

auto table = it_by_name->second;
dropTableInternal(context, table, drop_regions);

tables_by_name.erase(it_by_name);
return table;
}

TablePtr MockTiDB::dropTableByIdImpl(Context & context, const TableID table_id, bool drop_regions)
{
auto iter = tables_by_id.find(table_id);
if (iter == tables_by_id.end())
return nullptr;

auto table = iter->second;
dropTableInternal(context, table, drop_regions);

// erase from `tables_by_name`
for (auto iter_by_name = tables_by_name.begin(); iter_by_name != tables_by_name.end(); /* empty */)
{
if (table != iter_by_name->second)
{
++iter_by_name;
continue;
}
LOG_INFO(Logger::get(), "removing table from MockTiDB, name={} table_id={}", iter_by_name->first, table_id);
iter_by_name = tables_by_name.erase(iter_by_name);
}
return table;
}

TablePtr MockTiDB::dropTableInternal(Context & context, const TablePtr & table, bool drop_regions)
{
auto & kvstore = context.getTMTContext().getKVStore();
auto & region_table = context.getTMTContext().getRegionTable();

auto table = it_by_name->second;
if (table->isPartitionTable())
{
for (const auto & partition : table->table_info.partition.definitions)
Expand All @@ -98,8 +129,6 @@ TablePtr MockTiDB::dropTableInternal(
}
tables_by_id.erase(table->id());

tables_by_name.erase(it_by_name);

if (drop_regions)
{
for (auto & e : region_table.getRegionsByTable(NullspaceID, table->id()))
Expand All @@ -121,7 +150,7 @@ void MockTiDB::dropDB(Context & context, const String & database_name, bool drop
});

for (const auto & table_name : table_names)
dropTableInternal(context, database_name, table_name, drop_regions);
dropTableByNameImpl(context, database_name, table_name, drop_regions);

version++;

Expand All @@ -141,7 +170,25 @@ void MockTiDB::dropTable(Context & context, const String & database_name, const
{
std::lock_guard lock(tables_mutex);

auto table = dropTableInternal(context, database_name, table_name, drop_regions);
auto table = dropTableByNameImpl(context, database_name, table_name, drop_regions);
if (!table)
return;

version++;

SchemaDiff diff;
diff.type = SchemaActionType::DropTable;
diff.schema_id = table->database_id;
diff.table_id = table->id();
diff.version = version;
version_diff[version] = diff;
}

void MockTiDB::dropTableById(Context & context, const TableID & table_id, bool drop_regions)
{
std::lock_guard lock(tables_mutex);

auto table = dropTableByIdImpl(context, table_id, drop_regions);
if (!table)
return;

Expand Down
6 changes: 5 additions & 1 deletion dbms/src/Debug/MockTiDB.h
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
#pragma once

#include <Storages/ColumnsDescription.h>
#include <Storages/IStorage.h>
#include <Storages/KVStore/Types.h>
#include <TiDB/Schema/SchemaGetter.h>
#include <TiDB/Schema/SchemaSyncer.h>
Expand Down Expand Up @@ -113,6 +114,7 @@ class MockTiDB : public ext::Singleton<MockTiDB>
void dropPartition(const String & database_name, const String & table_name, TableID partition_id);

void dropTable(Context & context, const String & database_name, const String & table_name, bool drop_regions);
void dropTableById(Context & context, const TableID & table_id, bool drop_regions);

void dropDB(Context & context, const String & database_name, bool drop_regions);

Expand Down Expand Up @@ -169,11 +171,13 @@ class MockTiDB : public ext::Singleton<MockTiDB>
const String & partition_name,
Timestamp tso,
bool is_add_part);
TablePtr dropTableInternal(
TablePtr dropTableByNameImpl(
Context & context,
const String & database_name,
const String & table_name,
bool drop_regions);
TablePtr dropTableByIdImpl(Context & context, TableID table_id, bool drop_regions);
TablePtr dropTableInternal(Context & context, const TablePtr & table, bool drop_regions);
TablePtr getTableByNameInternal(const String & database_name, const String & table_name);
TablePtr getTableByID(TableID table_id);

Expand Down
10 changes: 7 additions & 3 deletions dbms/src/Debug/dbgFuncSchema.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -176,16 +176,20 @@ void dbgFuncRefreshMappedTableSchema(Context & context, const ASTs & args, DBGIn

// Trigger gc on all databases / tables.
// Usage:
// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point])"
// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point, ignore_remain_regions])"
void dbgFuncGcSchemas(Context & context, const ASTs & args, DBGInvoker::Printer output)
{
auto & service = context.getSchemaSyncService();
Timestamp gc_safe_point = 0;
bool ignore_remain_regions = false;
if (args.empty())
gc_safe_point = PDClientHelper::getGCSafePointWithRetry(context.getTMTContext().getPDClient(), NullspaceID);
else
if (!args.empty())
gc_safe_point = safeGet<Timestamp>(typeid_cast<const ASTLiteral &>(*args[0]).value);
service->gc(gc_safe_point, NullspaceID);
if (args.size() >= 2)
ignore_remain_regions = safeGet<String>(typeid_cast<const ASTLiteral &>(*args[1]).value) == "true";
// Note that only call it in tests, we need to ignore remain regions
service->gcImpl(gc_safe_point, NullspaceID, ignore_remain_regions);

output("schemas gc done");
}
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Debug/dbgFuncSchema.h
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ void dbgFuncRefreshMappedTableSchema(Context & context, const ASTs & args, DBGIn

// Trigger gc on all databases / tables.
// Usage:
// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point])"
// ./storage-client.sh "DBGInvoke gc_schemas([gc_safe_point, ignore_remain_regions])"
void dbgFuncGcSchemas(Context & context, const ASTs & args, DBGInvoker::Printer output);

// Reset schemas.
Expand Down
6 changes: 5 additions & 1 deletion dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
#include <TiDB/Schema/SchemaSyncer.h>
#include <TiDB/Schema/TiDBSchemaManager.h>
#include <common/logger_useful.h>
#include <fiu.h>

namespace DB
{
Expand Down Expand Up @@ -71,6 +72,10 @@ static DM::WriteResult writeRegionDataToStorage(
auto storage = tmt.getStorages().get(keyspace_id, table_id);
if (storage == nullptr)
{
// - force_decode == false and storage not exist, let upper level sync schema and retry.
// - force_decode == true and storage not exist. It could be the RaftLog or Snapshot comes
// after the schema is totally exceed the GC safepoint. And TiFlash know nothing about
// the schema. We can only throw away those committed rows.
return force_decode;
}

Expand Down Expand Up @@ -212,7 +217,6 @@ static DM::WriteResult writeRegionDataToStorage(
if (!atomic_read_write(true))
{
// Failure won't be tolerated this time.
// TODO: Enrich exception message.
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Write region failed! region_id={} keyspace={} table_id={}",
Expand Down
35 changes: 34 additions & 1 deletion dbms/src/Storages/KVStore/Decode/RegionTable.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,13 @@
#include <Storages/KVStore/MultiRaft/RegionManager.h>
#include <Storages/KVStore/Region.h>
#include <Storages/KVStore/TMTContext.h>
#include <Storages/KVStore/Types.h>
#include <Storages/StorageDeltaMerge.h>
#include <Storages/StorageDeltaMergeHelpers.h>
#include <TiDB/Schema/SchemaSyncer.h>
#include <fiu.h>

#include <any>

namespace DB
{
Expand All @@ -37,6 +41,10 @@ extern const int UNKNOWN_TABLE;
extern const int ILLFORMAT_RAFT_ROW;
extern const int TABLE_IS_DROPPED;
} // namespace ErrorCodes
namespace FailPoints
{
extern const char force_set_num_regions_for_table[];
} // namespace FailPoints

RegionTable::Table & RegionTable::getOrCreateTable(const KeyspaceID keyspace_id, const TableID table_id)
{
Expand Down Expand Up @@ -218,8 +226,8 @@ void RegionTable::removeRegion(const RegionID region_id, bool remove_data, const
{
tables.erase(ks_table_id);
}
LOG_INFO(log, "remove region in RegionTable done, region_id={}", region_id);
}
LOG_INFO(log, "remove region in RegionTable done, region_id={}", region_id);

// Sometime we don't need to remove data. e.g. remove region after region merge.
if (remove_data)
Expand Down Expand Up @@ -318,6 +326,31 @@ void RegionTable::handleInternalRegionsByTable(
}
}

std::vector<RegionID> RegionTable::getRegionIdsByTable(KeyspaceID keyspace_id, TableID table_id) const
{
fiu_do_on(FailPoints::force_set_num_regions_for_table, {
if (auto v = FailPointHelper::getFailPointVal(FailPoints::force_set_num_regions_for_table); v)
{
auto num_regions = std::any_cast<std::vector<RegionID>>(v.value());
return num_regions;
}
});

std::lock_guard lock(mutex);
if (auto iter = tables.find(KeyspaceTableID{keyspace_id, table_id}); //
unlikely(iter != tables.end()))
{
std::vector<RegionID> ret_regions;
ret_regions.reserve(iter->second.regions.size());
for (const auto & r : iter->second.regions)
{
ret_regions.emplace_back(r.first);
}
return ret_regions;
}
return {};
}

std::vector<std::pair<RegionID, RegionPtr>> RegionTable::getRegionsByTable(
const KeyspaceID keyspace_id,
const TableID table_id) const
Expand Down
8 changes: 5 additions & 3 deletions dbms/src/Storages/KVStore/Decode/RegionTable.h
Original file line number Diff line number Diff line change
Expand Up @@ -101,9 +101,6 @@ class RegionTable : private boost::noncopyable
InternalRegions regions;
};

using TableMap = std::unordered_map<KeyspaceTableID, Table, boost::hash<KeyspaceTableID>>;
using RegionInfoMap = std::unordered_map<RegionID, KeyspaceTableID>;

explicit RegionTable(Context & context_);
void restore();

Expand All @@ -126,6 +123,8 @@ class RegionTable : private boost::noncopyable
KeyspaceID keyspace_id,
TableID table_id,
std::function<void(const InternalRegions &)> && callback) const;

std::vector<RegionID> getRegionIdsByTable(KeyspaceID keyspace_id, TableID table_id) const;
std::vector<std::pair<RegionID, RegionPtr>> getRegionsByTable(KeyspaceID keyspace_id, TableID table_id) const;

/// Write the data of the given region into the table with the given table ID, fill the data list for outer to remove.
Expand Down Expand Up @@ -188,7 +187,10 @@ class RegionTable : private boost::noncopyable
InternalRegion & doGetInternalRegion(KeyspaceTableID ks_table_id, RegionID region_id);

private:
using TableMap = std::unordered_map<KeyspaceTableID, Table, boost::hash<KeyspaceTableID>>;
TableMap tables;

using RegionInfoMap = std::unordered_map<RegionID, KeyspaceTableID>;
RegionInfoMap regions;
SafeTsMap safe_ts_map;

Expand Down
Loading

0 comments on commit b23daac

Please sign in to comment.