Skip to content

Commit

Permalink
KVStore: Fix unreleased Region instance (#9763)
Browse files Browse the repository at this point in the history
close #9722

Signed-off-by: Calvin Neo <[email protected]>

Co-authored-by: ti-chi-bot[bot] <108142056+ti-chi-bot[bot]@users.noreply.github.com>
  • Loading branch information
CalvinNeo and ti-chi-bot[bot] authored Jan 7, 2025
1 parent 330a709 commit 1b23623
Show file tree
Hide file tree
Showing 16 changed files with 109 additions and 64 deletions.
5 changes: 5 additions & 0 deletions dbms/src/Common/TiFlashMetrics.h
Original file line number Diff line number Diff line change
Expand Up @@ -576,6 +576,11 @@ static_assert(RAFT_REGION_BIG_WRITE_THRES * 4 < RAFT_REGION_BIG_WRITE_MAX, "Inva
F(type_tikv_server_issue, {{"type", "tikv_server_issue"}}), \
F(type_tikv_lock, {{"type", "tikv_lock"}}), \
F(type_other, {{"type", "other"}})) \
M(tiflash_raft_classes_count, \
"Raft classes counter", \
Gauge, \
F(type_region, {{"type", "region"}}), \
F(type_fully_decoded_lockcf, {{"type", "fully_decoded_lockcf"}})) \
/* required by DBaaS */ \
M(tiflash_server_info, \
"Indicate the tiflash server info, and the value is the start timestamp (s).", \
Expand Down
3 changes: 1 addition & 2 deletions dbms/src/Storages/KVStore/KVStore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,6 @@ KVStore::KVStore(Context & context)
: region_persister(
context.getSharedContextDisagg()->isDisaggregatedComputeMode() ? nullptr
: std::make_unique<RegionPersister>(context))
, raft_cmd_res(std::make_unique<RaftCommandResult>())
, log(Logger::get())
, region_compact_log_min_rows(40 * 1024)
, region_compact_log_min_bytes(32 * 1024 * 1024)
Expand Down Expand Up @@ -363,7 +362,7 @@ void KVStore::handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTas
LOG_INFO(log, "region_id={} not found, might be removed already", region_id);
return;
}
LOG_INFO(log, "Handle destroy {}", region->toString());
LOG_INFO(log, "Handle destroy {}, refCount {}", region->toString(), region.use_count());
region->setPendingRemove();
removeRegion(
region_id,
Expand Down
3 changes: 0 additions & 3 deletions dbms/src/Storages/KVStore/KVStore.h
Original file line number Diff line number Diff line change
Expand Up @@ -379,9 +379,6 @@ class KVStore final : private boost::noncopyable

mutable std::mutex task_mutex;

// raft_cmd_res stores the result of applying raft cmd. It must be protected by task_mutex.
std::unique_ptr<RaftCommandResult> raft_cmd_res;

LoggerPtr log;

std::atomic<UInt64> region_compact_log_min_rows;
Expand Down
4 changes: 2 additions & 2 deletions dbms/src/Storages/KVStore/MultiRaft/RaftCommandsKVS.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -216,9 +216,9 @@ EngineStoreApplyRes KVStore::handleAdminRaftCmd(
curr_region.orphanKeysInfo().advanceAppliedIndex(index);
}

RaftCommandResult result;
curr_region.makeRaftCommandDelegate(task_lock)
.handleAdminRaftCmd(request, response, index, term, *this, region_table, *raft_cmd_res);
RaftCommandResult & result = *raft_cmd_res;
.handleAdminRaftCmd(request, response, index, term, *this, region_table, result);

// After region split / merge, try to flush it
const auto try_to_flush_region = [&tmt](const RegionPtr & region) {
Expand Down
10 changes: 9 additions & 1 deletion dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -310,7 +310,8 @@ void RegionData::deserialize(ReadBuffer & buf, RegionData & region_data)
total_size += RegionWriteCFData::deserialize(buf, region_data.write_cf);
total_size += RegionLockCFData::deserialize(buf, region_data.lock_cf);

region_data.cf_data_size += total_size;
region_data.cf_data_size = total_size;
reportAlloc(total_size);
}

RegionWriteCFData & RegionData::writeCF()
Expand Down Expand Up @@ -348,6 +349,13 @@ RegionData::RegionData(RegionData && data)
, cf_data_size(data.cf_data_size.load())
{}


RegionData::~RegionData()
{
reportDealloc(cf_data_size);
cf_data_size = 0;
}

RegionData & RegionData::operator=(RegionData && rhs)
{
write_cf = std::move(rhs.write_cf);
Expand Down
1 change: 1 addition & 0 deletions dbms/src/Storages/KVStore/MultiRaft/RegionData.h
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ class RegionData
const RegionLockCFData & lockCF() const;

RegionData() = default;
~RegionData();

RegionData(RegionData && data);
RegionData & operator=(RegionData &&);
Expand Down
71 changes: 71 additions & 0 deletions dbms/src/Storages/KVStore/MultiRaft/RegionRangeKeys.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
// Copyright 2025 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#include <Storages/KVStore/MultiRaft/RegionRangeKeys.h>
#include <Storages/KVStore/TiKVHelpers/TiKVRecordFormat.h>

namespace DB
{
RegionRangeKeys::RegionRangeKeys(TiKVKey && start_key, TiKVKey && end_key)
: ori(RegionRangeKeys::makeComparableKeys(std::move(start_key), std::move(end_key)))
, raw(std::make_shared<DecodedTiKVKey>(
ori.first.key.empty() ? DecodedTiKVKey() : RecordKVFormat::decodeTiKVKey(ori.first.key)),
std::make_shared<DecodedTiKVKey>(
ori.second.key.empty() ? DecodedTiKVKey() : RecordKVFormat::decodeTiKVKey(ori.second.key)))
{
keyspace_id = raw.first->getKeyspaceID();
if (!computeMappedTableID(*raw.first, mapped_table_id) || ori.first.compare(ori.second) >= 0)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Illegal region range, should not happen, start_key={} end_key={}",
ori.first.key.toDebugString(),
ori.second.key.toDebugString());
}
}

TableID RegionRangeKeys::getMappedTableID() const
{
return mapped_table_id;
}

KeyspaceID RegionRangeKeys::getKeyspaceID() const
{
return keyspace_id;
}

const std::pair<DecodedTiKVKeyPtr, DecodedTiKVKeyPtr> & RegionRangeKeys::rawKeys() const
{
return raw;
}

const RegionRangeKeys::RegionRange & RegionRangeKeys::comparableKeys() const
{
return ori;
}

RegionRangeKeys::RegionRange RegionRangeKeys::makeComparableKeys(TiKVKey && start_key, TiKVKey && end_key)
{
return std::make_pair(
TiKVRangeKey::makeTiKVRangeKey<true>(std::move(start_key)),
TiKVRangeKey::makeTiKVRangeKey<false>(std::move(end_key)));
}

RegionRangeKeys::RegionRange RegionRangeKeys::cloneRange(const RegionRange & from)
{
return std::make_pair(from.first.copy(), from.second.copy());
}


} // namespace DB
2 changes: 2 additions & 0 deletions dbms/src/Storages/KVStore/MultiRaft/RegionRangeKeys.h
Original file line number Diff line number Diff line change
Expand Up @@ -88,4 +88,6 @@ class RegionRangeKeys : boost::noncopyable
KeyspaceID keyspace_id = NullspaceID;
};

bool computeMappedTableID(const DecodedTiKVKey & key, TableID & table_id);

} // namespace DB
3 changes: 1 addition & 2 deletions dbms/src/Storages/KVStore/MultiRaft/RegionSerde.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -158,12 +158,11 @@ RegionPtr Region::deserializeImpl(

// deserialize data
RegionData::deserialize(buf, region->data);
region->data.reportAlloc(region->data.cf_data_size);

// restore other var according to meta
region->last_restart_log_applied = region->appliedIndex();
region->setLastCompactLogApplied(region->appliedIndex());
return region;
}

} // namespace DB
} // namespace DB
49 changes: 0 additions & 49 deletions dbms/src/Storages/KVStore/MultiRaft/RegionState.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -136,38 +136,6 @@ bool computeMappedTableID(const DecodedTiKVKey & key, TableID & table_id)
return false;
}

RegionRangeKeys::RegionRangeKeys(TiKVKey && start_key, TiKVKey && end_key)
: ori(RegionRangeKeys::makeComparableKeys(std::move(start_key), std::move(end_key)))
, raw(std::make_shared<DecodedTiKVKey>(
ori.first.key.empty() ? DecodedTiKVKey() : RecordKVFormat::decodeTiKVKey(ori.first.key)),
std::make_shared<DecodedTiKVKey>(
ori.second.key.empty() ? DecodedTiKVKey() : RecordKVFormat::decodeTiKVKey(ori.second.key)))
{
keyspace_id = raw.first->getKeyspaceID();
if (!computeMappedTableID(*raw.first, mapped_table_id) || ori.first.compare(ori.second) >= 0)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Illegal region range, should not happen, start_key={} end_key={}",
ori.first.key.toDebugString(),
ori.second.key.toDebugString());
}
}

TableID RegionRangeKeys::getMappedTableID() const
{
return mapped_table_id;
}

KeyspaceID RegionRangeKeys::getKeyspaceID() const
{
return keyspace_id;
}

const std::pair<DecodedTiKVKeyPtr, DecodedTiKVKeyPtr> & RegionRangeKeys::rawKeys() const
{
return raw;
}

HandleRange<HandleID> getHandleRangeByTable(
const std::pair<DecodedTiKVKeyPtr, DecodedTiKVKeyPtr> & rawKeys,
Expand All @@ -176,11 +144,6 @@ HandleRange<HandleID> getHandleRangeByTable(
return TiKVRange::getHandleRangeByTable(*rawKeys.first, *rawKeys.second, table_id);
}

const RegionRangeKeys::RegionRange & RegionRangeKeys::comparableKeys() const
{
return ori;
}

template <bool is_start>
TiKVRangeKey TiKVRangeKey::makeTiKVRangeKey(TiKVKey && key)
{
Expand All @@ -191,18 +154,6 @@ TiKVRangeKey TiKVRangeKey::makeTiKVRangeKey(TiKVKey && key)
template TiKVRangeKey TiKVRangeKey::makeTiKVRangeKey<true>(TiKVKey &&);
template TiKVRangeKey TiKVRangeKey::makeTiKVRangeKey<false>(TiKVKey &&);

RegionRangeKeys::RegionRange RegionRangeKeys::makeComparableKeys(TiKVKey && start_key, TiKVKey && end_key)
{
return std::make_pair(
TiKVRangeKey::makeTiKVRangeKey<true>(std::move(start_key)),
TiKVRangeKey::makeTiKVRangeKey<false>(std::move(end_key)));
}

RegionRangeKeys::RegionRange RegionRangeKeys::cloneRange(const RegionRange & from)
{
return std::make_pair(from.first.copy(), from.second.copy());
}

std::string TiKVRangeKey::toDebugString() const
{
if (this->state == MAX)
Expand Down
1 change: 0 additions & 1 deletion dbms/src/Storages/KVStore/MultiRaft/RegionState.h
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,5 @@ class RegionState : private boost::noncopyable
HandleRange<HandleID> getHandleRangeByTable(
const std::pair<DecodedTiKVKeyPtr, DecodedTiKVKeyPtr> & rawKeys,
TableID table_id);
bool computeMappedTableID(const DecodedTiKVKey & key, TableID & table_id);

} // namespace DB
6 changes: 4 additions & 2 deletions dbms/src/Storages/KVStore/Region.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -340,11 +340,13 @@ Region::Region(DB::RegionMeta && meta_, const TiFlashRaftProxyHelper * proxy_hel
, keyspace_id(meta.getRange()->getKeyspaceID())
, mapped_table_id(meta.getRange()->getMappedTableID())
, proxy_helper(proxy_helper_)
{}
{
GET_METRIC(tiflash_raft_classes_count, type_region).Increment(1);
}

Region::~Region()
{
data.reportDealloc(data.cf_data_size);
GET_METRIC(tiflash_raft_classes_count, type_region).Decrement();
}

TableID Region::getMappedTableID() const
Expand Down
7 changes: 7 additions & 0 deletions dbms/src/Storages/KVStore/TiKVHelpers/DecodedLockCFValue.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -164,10 +164,17 @@ DecodedLockCFValue::DecodedLockCFValue(std::shared_ptr<const TiKVKey> key_, std:
if (parsed->generation == 0)
{
// It is not a large txn, we cache the parsed lock.
GET_METRIC(tiflash_raft_classes_count, type_fully_decoded_lockcf).Increment(1);
inner = std::move(parsed);
}
}

DecodedLockCFValue::~DecodedLockCFValue()
{
if (inner != nullptr)
GET_METRIC(tiflash_raft_classes_count, type_fully_decoded_lockcf).Decrement(1);
}

void DecodedLockCFValue::withInner(std::function<void(const DecodedLockCFValue::Inner &)> && f) const
{
if likely (inner != nullptr)
Expand Down
1 change: 1 addition & 0 deletions dbms/src/Storages/KVStore/TiKVHelpers/DecodedLockCFValue.h
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ struct DecodedLockCFValue : boost::noncopyable
void intoLockInfo(const std::shared_ptr<const TiKVKey> & key, kvrpcpb::LockInfo &) const;
};
DecodedLockCFValue(std::shared_ptr<const TiKVKey> key_, std::shared_ptr<const TiKVValue> val_);
~DecodedLockCFValue();
std::unique_ptr<kvrpcpb::LockInfo> intoLockInfo() const;
void intoLockInfo(kvrpcpb::LockInfo &) const;
bool isLargeTxn() const;
Expand Down
2 changes: 1 addition & 1 deletion dbms/src/Storages/KVStore/tests/gtest_kvstore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -356,7 +356,7 @@ static void testRaftSplit(KVStore & kvs, TMTContext & tmt, std::unique_ptr<MockR
RegionID region_id2 = 7;
auto source_region = kvs.getRegion(region_id);
auto old_epoch = source_region->getMeta().getMetaRegion().region_epoch();
const auto & ori_source_range = source_region->getRange()->comparableKeys();
auto ori_source_range = RegionRangeKeys::cloneRange(source_region->getRange()->comparableKeys());
RegionRangeKeys::RegionRange new_source_range = RegionRangeKeys::makeComparableKeys( //
RecordKVFormat::genKey(table_id, 5),
RecordKVFormat::genKey(table_id, 10));
Expand Down
5 changes: 4 additions & 1 deletion dbms/src/Storages/KVStore/tests/gtest_new_kvstore.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -123,8 +123,11 @@ try
RegionPtr region = tests::makeRegion(702, start, end, proxy_helper.get());
region->insert("default", TiKVKey::copyFrom(str_key), TiKVValue::copyFrom(str_val_default));
ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size());
// 702 is not registed, so we persist as 1.
tryPersistRegion(kvs, 1);
reloadKVSFromDisk();
root_of_kvstore_mem_trackers->reset();
ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0);
reloadKVSFromDisk(false);
ASSERT_EQ(root_of_kvstore_mem_trackers->get(), str_key.dataSize() + str_val_default.size());
}
ASSERT_EQ(root_of_kvstore_mem_trackers->get(), 0);
Expand Down

0 comments on commit 1b23623

Please sign in to comment.