Skip to content

Commit

Permalink
fix test
Browse files Browse the repository at this point in the history
Signed-off-by: CalvinNeo <calvinneo1995@gmail.com>
  • Loading branch information
CalvinNeo committed Jun 14, 2023
1 parent bc98195 commit d40931a
Show file tree
Hide file tree
Showing 5 changed files with 42 additions and 31 deletions.
17 changes: 14 additions & 3 deletions dbms/src/Storages/Transaction/KVStore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -295,7 +295,10 @@ EngineStoreApplyRes KVStore::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt
}

auto res = region->handleWriteRaftCmd(cmds, index, term, tmt);
region->orphanKeysInfo().advanceAppliedIndex(index);
if (region->getClusterRaftstoreVer() == RaftstoreVer::V2)
{
region->orphanKeysInfo().advanceAppliedIndex(index);
}
return res;
}

Expand Down Expand Up @@ -464,7 +467,10 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd(
term,
index);

curr_region.orphanKeysInfo().advanceAppliedIndex(index);
if (curr_region.getClusterRaftstoreVer() == RaftstoreVer::V2)
{
curr_region.orphanKeysInfo().advanceAppliedIndex(index);
}

if (cmd_type == raft_cmdpb::AdminCmdType::CompactLog)
{
Expand Down Expand Up @@ -534,8 +540,13 @@ EngineStoreApplyRes KVStore::handleAdminRaftCmd(raft_cmdpb::AdminRequest && requ
}

auto & curr_region = *curr_region_ptr;

// Admin cmd contains no normal data, we can advance orphan keys info just before handling.
curr_region.orphanKeysInfo().advanceAppliedIndex(index);
if (curr_region.getClusterRaftstoreVer() == RaftstoreVer::V2)
{
curr_region.orphanKeysInfo().advanceAppliedIndex(index);
}

curr_region.makeRaftCommandDelegate(task_lock).handleAdminRaftCmd(
request,
response,
Expand Down
17 changes: 12 additions & 5 deletions dbms/src/Storages/Transaction/Region.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -517,15 +517,22 @@ RaftstoreVer Region::getClusterRaftstoreVer()

void Region::beforePrehandleSnapshot(uint64_t region_id, std::optional<uint64_t> deadline_index)
{
data.orphan_keys_info.snapshot_index = appliedIndex();
data.orphan_keys_info.pre_handling = true;
data.orphan_keys_info.deadline_index = deadline_index;
data.orphan_keys_info.region_id = region_id;
if (getClusterRaftstoreVer() == RaftstoreVer::V2)
{
data.orphan_keys_info.snapshot_index = appliedIndex();
data.orphan_keys_info.pre_handling = true;
data.orphan_keys_info.deadline_index = deadline_index;
data.orphan_keys_info.region_id = region_id;
}
}

void Region::afterPrehandleSnapshot()
{
data.orphan_keys_info.pre_handling = false;
if (getClusterRaftstoreVer() == RaftstoreVer::V2)
{
data.orphan_keys_info.pre_handling = false;
LOG_INFO(log, "After prehandle, remains {} orphan keys [region_id={}]", data.orphan_keys_info.remainedKeyCount(), id());
}
}

kvrpcpb::ReadIndexRequest GenRegionReadIndexReq(const Region & region, UInt64 start_ts)
Expand Down
2 changes: 0 additions & 2 deletions dbms/src/Storages/Transaction/TiKVKeyValue.h
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@
#include <Storages/Transaction/SerializationHelper.h>
#include <Storages/Transaction/Types.h>

#include <string_view>

namespace DB
{
template <bool is_key>
Expand Down
36 changes: 16 additions & 20 deletions dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -295,7 +295,7 @@ void RegionKVStoreTest::testRaftMergeRollback(KVStore & kvs, TMTContext & tmt)
}
}

void RegionKVStoreTest::testRaftSplit(KVStore & kvs, TMTContext & tmt)
static void testRaftSplit(KVStore & kvs, TMTContext & tmt, std::unique_ptr<MockRaftStoreProxy> & proxy_instance)
{
{
auto region = kvs.getRegion(1);
Expand All @@ -310,7 +310,6 @@ void RegionKVStoreTest::testRaftSplit(KVStore & kvs, TMTContext & tmt)

ASSERT_EQ(region->dataInfo(), "[write 2 lock 2 default 2 ]");
}

// Split region
RegionID region_id = 1;
RegionID region_id2 = 7;
Expand All @@ -336,17 +335,13 @@ void RegionKVStoreTest::testRaftSplit(KVStore & kvs, TMTContext & tmt)
ASSERT_EQ(kvs.getRegion(1)->dataInfo(), "[write 1 lock 1 default 1 ]");
ASSERT_EQ(kvs.getRegion(7)->dataInfo(), "[lock 1 ]");
}
return;
// Rollback 1 to before split
// 7 is persisted
{
kvs.handleDestroy(1, tmt);
{
auto task_lock = kvs.genTaskLock();
auto lock = kvs.genRegionMgrWriteLock(task_lock);
auto region = makeRegion(1, ori_source_range.first.key, ori_source_range.second.key);
lock.regions.emplace(1, region);
lock.index.add(region);
}
proxy_instance->debugAddRegions(kvs, tmt, {1}, {{ori_source_range.first.key, ori_source_range.second.key}});

auto table_id = 1;
auto region = kvs.getRegion(1);
region->insert("lock", RecordKVFormat::genKey(table_id, 3), RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 3, 20));
Expand Down Expand Up @@ -481,7 +476,7 @@ void RegionKVStoreTest::testRaftMerge(KVStore & kvs, TMTContext & tmt)
// add 7 back
auto task_lock = kvs.genTaskLock();
auto lock = kvs.genRegionMgrWriteLock(task_lock);
auto region = makeRegion(7, RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5));
auto region = makeRegion(7, RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 5), kvs.getProxyHelper());
lock.regions.emplace(7, region);
lock.index.add(region);
}
Expand Down Expand Up @@ -780,7 +775,7 @@ TEST_F(RegionKVStoreTest, AdminSplit)
KVStore & kvs = getKVS();
proxy_instance->debugAddRegions(kvs, ctx.getTMTContext(), {1}, {{RecordKVFormat::genKey(1, 0), RecordKVFormat::genKey(1, 10)}});
{
testRaftSplit(kvs, ctx.getTMTContext());
testRaftSplit(kvs, ctx.getTMTContext(), proxy_instance);
ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{}, raft_cmdpb::AdminResponse{}, 8192, 5, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound);
}
}
Expand Down Expand Up @@ -888,7 +883,7 @@ try
});
// Initially region_19 range is [0, 10000)
{
auto region = makeRegion(region_id, RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, 10000));
auto region = makeRegion(region_id, RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, 10000), kvs.getProxyHelper());
GenMockSSTData(DMTestEnv::getMinimalTableInfo(table_id), table_id, region_id_str, 20, 100, 0);
std::vector<SSTView> sst_views{
SSTView{
Expand Down Expand Up @@ -923,7 +918,7 @@ try
}
// Later, its range is changed to [20000, 50000)
{
auto region = makeRegion(region_id, RecordKVFormat::genKey(table_id, 20000), RecordKVFormat::genKey(table_id, 50000));
auto region = makeRegion(region_id, RecordKVFormat::genKey(table_id, 20000), RecordKVFormat::genKey(table_id, 50000), kvs.getProxyHelper());
GenMockSSTData(DMTestEnv::getMinimalTableInfo(table_id), table_id, region_id_str, 20100, 20200, 0);
std::vector<SSTView> sst_views{
SSTView{
Expand Down Expand Up @@ -989,7 +984,7 @@ try
ASSERT_NE(proxy_helper->sst_reader_interfaces.fn_key, nullptr);
{
auto region_id = 19;
auto region = makeRegion(region_id, RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60));
auto region = makeRegion(region_id, RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60), kvs.getProxyHelper());
auto region_id_str = std::to_string(region_id);
auto & mmp = MockSSTReader::getMockSSTData();
MockSSTReader::getMockSSTData().clear();
Expand All @@ -1004,7 +999,7 @@ try
ColumnFamilyType::Default,
BaseBuffView{region_id_str.data(), region_id_str.length()},
});
// Will reject a snapshot with snaller index.
// Will reject a snapshot with smaller index.
{
// Pre-handle snapshot to DTFiles is ignored because the table is dropped.
kvs.handleApplySnapshot(
Expand Down Expand Up @@ -1037,7 +1032,7 @@ try
{
// Snapshot will be rejected if region overlaps.
{
auto region = makeRegion(22, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100));
auto region = makeRegion(22, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper());
auto ingest_ids = kvs.preHandleSnapshotToFiles(
region,
{},
Expand All @@ -1049,7 +1044,7 @@ try
}
try
{
auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100));
auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper());
auto ingest_ids = kvs.preHandleSnapshotToFiles(
region,
{},
Expand All @@ -1069,21 +1064,22 @@ try
{
// Applying snapshot will throw if proxy is not inited.
const auto * ori_ptr = proxy_helper->proxy_ptr.inner;
proxy_helper->proxy_ptr.inner = nullptr;
SCOPE_EXIT({
proxy_helper->proxy_ptr.inner = ori_ptr;
});

try
{
auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100));
auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper());
// preHandleSnapshotToFiles will assert proxy_ptr is not null.
auto ingest_ids = kvs.preHandleSnapshotToFiles(
region,
{},
10,
5,
std::nullopt,
ctx.getTMTContext());
proxy_helper->proxy_ptr.inner = nullptr;
kvs.checkAndApplyPreHandledSnapshot<RegionPtrWithSnapshotFiles>(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext());
ASSERT_TRUE(false);
}
Expand All @@ -1100,7 +1096,7 @@ try
s.set_state(::raft_serverpb::PeerState::Tombstone);
s;
}));
auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100));
auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100), kvs.getProxyHelper());
auto ingest_ids = kvs.preHandleSnapshotToFiles(
region,
{},
Expand Down
1 change: 0 additions & 1 deletion dbms/src/Storages/Transaction/tests/kvstore_helper.h
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,6 @@ class RegionKVStoreTest : public ::testing::Test
}

protected:
static void testRaftSplit(KVStore & kvs, TMTContext & tmt);
static void testRaftMerge(KVStore & kvs, TMTContext & tmt);
static void testRaftMergeRollback(KVStore & kvs, TMTContext & tmt);

Expand Down

0 comments on commit d40931a

Please sign in to comment.