chore: LockTable tracks fingerprints of keys (#2839)

* chore: LockTable tracks fingerprints of keys

It's a first step that will probably simplify dependencies in many places
where we need to keep key strings for that. A second step will be to reduce the CPU load
of multi-key operations like MSET and precompute Fingerprints once.

---------

Signed-off-by: Roman Gershman <roman@dragonflydb.io>
This commit is contained in:
Roman Gershman 2024-04-10 17:52:53 +03:00 committed by GitHub
parent 56a7f85e39
commit da5c51d1dd
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
11 changed files with 59 additions and 57 deletions

View file

@ -1050,19 +1050,12 @@ void DbSlice::Release(IntentLock::Mode mode, const KeyLockArgs& lock_args) {
} }
bool DbSlice::CheckLock(IntentLock::Mode mode, DbIndex dbid, string_view key) const { bool DbSlice::CheckLock(IntentLock::Mode mode, DbIndex dbid, string_view key) const {
KeyLockArgs args; const auto& lt = db_arr_[dbid]->trans_locks;
args.db_index = dbid; string_view s = KeyLockArgs::GetLockKey(key);
args.args = ArgSlice{&key, 1};
args.key_step = 1;
return CheckLock(mode, args);
}
bool DbSlice::CheckLock(IntentLock::Mode mode, const KeyLockArgs& lock_args) const { auto lock = lt.Find(s);
const auto& lt = db_arr_[lock_args.db_index]->trans_locks; if (lock) {
for (size_t i = 0; i < lock_args.args.size(); i += lock_args.key_step) { return lock->Check(mode);
string_view s = KeyLockArgs::GetLockKey(lock_args.args[i]);
if (auto lock = lt.Find(s); lock && !lock->Check(mode))
return false;
} }
return true; return true;
} }
@ -1327,7 +1320,7 @@ void DbSlice::FreeMemWithEvictionStep(DbIndex db_ind, size_t increase_goal_bytes
continue; continue;
// check if the key is locked by looking up transaction table. // check if the key is locked by looking up transaction table.
auto& lt = db_table->trans_locks; const auto& lt = db_table->trans_locks;
string_view key = evict_it->first.GetSlice(&tmp); string_view key = evict_it->first.GetSlice(&tmp);
if (lt.Find(KeyLockArgs::GetLockKey(key)).has_value()) if (lt.Find(KeyLockArgs::GetLockKey(key)).has_value())
continue; continue;

View file

@ -8,6 +8,7 @@
#include "core/string_or_view.h" #include "core/string_or_view.h"
#include "facade/dragonfly_connection.h" #include "facade/dragonfly_connection.h"
#include "facade/op_status.h" #include "facade/op_status.h"
#include "server/cluster/slot_set.h"
#include "server/common.h" #include "server/common.h"
#include "server/conn_context.h" #include "server/conn_context.h"
#include "server/table.h" #include "server/table.h"
@ -367,9 +368,6 @@ class DbSlice {
// Returns true if the key can be locked under m. Does not lock. // Returns true if the key can be locked under m. Does not lock.
bool CheckLock(IntentLock::Mode m, DbIndex dbid, std::string_view key) const; bool CheckLock(IntentLock::Mode m, DbIndex dbid, std::string_view key) const;
// Returns true if all keys can be locked under m. Does not lock.
bool CheckLock(IntentLock::Mode m, const KeyLockArgs& lock_args) const;
size_t db_array_size() const { size_t db_array_size() const {
return db_arr_.size(); return db_arr_.size();
} }

View file

@ -315,13 +315,9 @@ ObjInfo InspectOp(string_view key, DbIndex db_index) {
} }
} }
KeyLockArgs lock_args; if (!db_slice.CheckLock(IntentLock::EXCLUSIVE, db_index, key)) {
lock_args.args = ArgSlice{&key, 1}; oinfo.lock_status =
lock_args.key_step = 1; db_slice.CheckLock(IntentLock::SHARED, db_index, key) ? ObjInfo::S : ObjInfo::X;
lock_args.db_index = db_index;
if (!db_slice.CheckLock(IntentLock::EXCLUSIVE, lock_args)) {
oinfo.lock_status = db_slice.CheckLock(IntentLock::SHARED, lock_args) ? ObjInfo::S : ObjInfo::X;
} }
return oinfo; return oinfo;
@ -890,7 +886,7 @@ void DebugCmd::TxAnalysis() {
StrAppend(&result, " locks total:", info.total_locks, ",contended:", info.contended_locks, StrAppend(&result, " locks total:", info.total_locks, ",contended:", info.contended_locks,
"\n"); "\n");
StrAppend(&result, " max contention score: ", info.max_contention_score, StrAppend(&result, " max contention score: ", info.max_contention_score,
",lock_name:", info.max_contention_lock_name, "\n"); ",lock_name:", info.max_contention_lock, "\n");
} }
auto* rb = static_cast<RedisReplyBuilder*>(cntx_->reply_builder()); auto* rb = static_cast<RedisReplyBuilder*>(cntx_->reply_builder());
rb->SendVerbatimString(result); rb->SendVerbatimString(result);

View file

@ -758,7 +758,7 @@ auto EngineShard::AnalyzeTxQueue() const -> TxQueueInfo {
info.contended_locks++; info.contended_locks++;
if (lock.ContentionScore() > info.max_contention_score) { if (lock.ContentionScore() > info.max_contention_score) {
info.max_contention_score = lock.ContentionScore(); info.max_contention_score = lock.ContentionScore();
info.max_contention_lock_name = key.view(); info.max_contention_lock = key;
} }
} }
} }

View file

@ -181,8 +181,8 @@ class EngineShard {
// The score of the lock with maximum contention (see IntentLock::ContetionScore for details). // The score of the lock with maximum contention (see IntentLock::ContetionScore for details).
unsigned max_contention_score = 0; unsigned max_contention_score = 0;
// the lock name with maximum contention // the lock fingerprint with maximum contention score.
std::string max_contention_lock_name; uint64_t max_contention_lock;
}; };
TxQueueInfo AnalyzeTxQueue() const; TxQueueInfo AnalyzeTxQueue() const;

View file

@ -2103,9 +2103,9 @@ void ServerFamily::Info(CmdArgList args, ConnectionContext* cntx) {
} }
if (should_enter("TRANSACTION", true)) { if (should_enter("TRANSACTION", true)) {
append("tx_shard_polls", m.shard_stats.poll_execution_total);
append("tx_shard_immediate_total", m.shard_stats.tx_immediate_total); append("tx_shard_immediate_total", m.shard_stats.tx_immediate_total);
append("tx_shard_ooo_total", m.shard_stats.tx_ooo_total); append("tx_shard_ooo_total", m.shard_stats.tx_ooo_total);
append("tx_global_total", m.coordinator_stats.tx_global_cnt); append("tx_global_total", m.coordinator_stats.tx_global_cnt);
append("tx_normal_total", m.coordinator_stats.tx_normal_cnt); append("tx_normal_total", m.coordinator_stats.tx_normal_cnt);
append("tx_inline_runs_total", m.coordinator_stats.tx_inline_runs); append("tx_inline_runs_total", m.coordinator_stats.tx_inline_runs);

View file

@ -4,6 +4,8 @@
#include "server/table.h" #include "server/table.h"
#include <xxhash.h>
#include "base/flags.h" #include "base/flags.h"
#include "base/logging.h" #include "base/logging.h"
#include "server/cluster/cluster_config.h" #include "server/cluster/cluster_config.h"
@ -12,8 +14,8 @@
ABSL_FLAG(bool, enable_top_keys_tracking, false, ABSL_FLAG(bool, enable_top_keys_tracking, false,
"Enables / disables tracking of hot keys debugging feature"); "Enables / disables tracking of hot keys debugging feature");
using namespace std;
namespace dfly { namespace dfly {
#define ADD(x) (x) += o.x #define ADD(x) (x) += o.x
// It should be const, but we override this variable in our tests so that they run faster. // It should be const, but we override this variable in our tests so that they run faster.
@ -62,29 +64,32 @@ size_t LockTable::Size() const {
return locks_.size(); return locks_.size();
} }
std::optional<const IntentLock> LockTable::Find(std::string_view key) const { LockFp LockTable::Fingerprint(string_view key) {
return XXH64(key.data(), key.size(), 0x1C69B3F74AC4AE35UL);
}
std::optional<const IntentLock> LockTable::Find(string_view key) const {
DCHECK_EQ(KeyLockArgs::GetLockKey(key), key); DCHECK_EQ(KeyLockArgs::GetLockKey(key), key);
if (auto it = locks_.find(Key::FromView(key)); it != locks_.end()) LockFp fp = Fingerprint(key);
if (auto it = locks_.find(fp); it != locks_.end())
return it->second; return it->second;
return std::nullopt; return std::nullopt;
} }
bool LockTable::Acquire(std::string_view key, IntentLock::Mode mode) { bool LockTable::Acquire(string_view key, IntentLock::Mode mode) {
DCHECK_EQ(KeyLockArgs::GetLockKey(key), key); DCHECK_EQ(KeyLockArgs::GetLockKey(key), key);
LockFp fp = Fingerprint(key);
auto [it, inserted] = locks_.try_emplace(Key::FromView(key)); auto [it, inserted] = locks_.try_emplace(fp);
if (!inserted) // If more than one transaction refers to a key
const_cast<Key&>(it->first).MakeOwned(); // we must fall back to using a self-contained string
return it->second.Acquire(mode); return it->second.Acquire(mode);
} }
void LockTable::Release(std::string_view key, IntentLock::Mode mode) { void LockTable::Release(string_view key, IntentLock::Mode mode) {
DCHECK_EQ(KeyLockArgs::GetLockKey(key), key); DCHECK_EQ(KeyLockArgs::GetLockKey(key), key);
auto it = locks_.find(Key::FromView(key)); LockFp fp = Fingerprint(key);
CHECK(it != locks_.end()) << key; auto it = locks_.find(fp);
DCHECK(it != locks_.end()) << key;
it->second.Release(mode); it->second.Release(mode);
if (it->second.IsFree()) if (it->second.IsFree())
@ -115,7 +120,7 @@ void DbTable::Clear() {
stats = DbTableStats{}; stats = DbTableStats{};
} }
PrimeIterator DbTable::Launder(PrimeIterator it, std::string_view key) { PrimeIterator DbTable::Launder(PrimeIterator it, string_view key) {
if (!it.IsOccupied() || it->first != key) { if (!it.IsOccupied() || it->first != key) {
it = prime.Find(key); it = prime.Find(key);
} }

View file

@ -11,9 +11,6 @@
#include "core/expire_period.h" #include "core/expire_period.h"
#include "core/intent_lock.h" #include "core/intent_lock.h"
#include "core/string_or_view.h"
#include "server/cluster/cluster_config.h"
#include "server/cluster/slot_set.h"
#include "server/conn_context.h" #include "server/conn_context.h"
#include "server/detail/table.h" #include "server/detail/table.h"
#include "server/top_keys.h" #include "server/top_keys.h"
@ -28,6 +25,7 @@ using PrimeValue = detail::PrimeValue;
using PrimeTable = DashTable<PrimeKey, PrimeValue, detail::PrimeTablePolicy>; using PrimeTable = DashTable<PrimeKey, PrimeValue, detail::PrimeTablePolicy>;
using ExpireTable = DashTable<PrimeKey, ExpirePeriod, detail::ExpireTablePolicy>; using ExpireTable = DashTable<PrimeKey, ExpirePeriod, detail::ExpireTablePolicy>;
using LockFp = uint64_t; // a key fingerprint used by the LockTable.
/// Iterators are invalidated when new keys are added to the table or some entries are deleted. /// Iterators are invalidated when new keys are added to the table or some entries are deleted.
/// Iterators are still valid if a different entry in the table was mutated. /// Iterators are still valid if a different entry in the table was mutated.
@ -92,17 +90,24 @@ class LockTable {
bool Acquire(std::string_view key, IntentLock::Mode mode); bool Acquire(std::string_view key, IntentLock::Mode mode);
void Release(std::string_view key, IntentLock::Mode mode); void Release(std::string_view key, IntentLock::Mode mode);
static LockFp Fingerprint(std::string_view key);
auto begin() const { auto begin() const {
return locks_.cbegin(); return locks_.cbegin();
} }
auto end() const { auto end() const {
return locks_.cbegin(); return locks_.cend();
} }
private: private:
using Key = StringOrView; // We use fingerprinting before accessing locks - no need to mix more.
absl::flat_hash_map<Key, IntentLock> locks_; struct Hasher {
size_t operator()(LockFp val) const {
return val;
}
};
absl::flat_hash_map<LockFp, IntentLock, Hasher> locks_;
}; };
// A single Db table that represents a table that can be chosen with "SELECT" command. // A single Db table that represents a table that can be chosen with "SELECT" command.

View file

@ -552,12 +552,10 @@ BaseFamilyTest::TestConnWrapper::GetInvalidationMessage(size_t index) const {
bool BaseFamilyTest::IsLocked(DbIndex db_index, std::string_view key) const { bool BaseFamilyTest::IsLocked(DbIndex db_index, std::string_view key) const {
ShardId sid = Shard(key, shard_set->size()); ShardId sid = Shard(key, shard_set->size());
KeyLockArgs args;
args.db_index = db_index; bool is_open = pp_->at(sid)->AwaitBrief([db_index, key] {
args.args = ArgSlice{&key, 1}; return EngineShard::tlocal()->db_slice().CheckLock(IntentLock::EXCLUSIVE, db_index, key);
args.key_step = 1; });
bool is_open = pp_->at(sid)->AwaitBrief(
[args] { return EngineShard::tlocal()->db_slice().CheckLock(IntentLock::EXCLUSIVE, args); });
return !is_open; return !is_open;
} }

View file

@ -51,7 +51,7 @@ void AnalyzeTxQueue(const EngineShard* shard, const TxQueue* txq) {
", runnable:", info.tx_runnable, ", total locks: ", info.total_locks, ", runnable:", info.tx_runnable, ", total locks: ", info.total_locks,
", contended locks: ", info.contended_locks, "\n"); ", contended locks: ", info.contended_locks, "\n");
absl::StrAppend(&msg, "max contention score: ", info.max_contention_score, absl::StrAppend(&msg, "max contention score: ", info.max_contention_score,
", lock: ", info.max_contention_lock_name, ", lock: ", info.max_contention_lock,
", poll_executions:", shard->stats().poll_execution_total); ", poll_executions:", shard->stats().poll_execution_total);
const Transaction* cont_tx = shard->GetContTx(); const Transaction* cont_tx = shard->GetContTx();
if (cont_tx) { if (cont_tx) {
@ -82,6 +82,15 @@ uint16_t trans_id(const Transaction* ptr) {
return (intptr_t(ptr) >> 8) & 0xFFFF; return (intptr_t(ptr) >> 8) & 0xFFFF;
} }
bool CheckLocks(const DbSlice& db_slice, IntentLock::Mode mode, const KeyLockArgs& lock_args) {
for (size_t i = 0; i < lock_args.args.size(); i += lock_args.key_step) {
string_view s = KeyLockArgs::GetLockKey(lock_args.args[i]);
if (!db_slice.CheckLock(mode, lock_args.db_index, s))
return false;
}
return true;
}
} // namespace } // namespace
bool Transaction::BatonBarrier::IsClaimed() const { bool Transaction::BatonBarrier::IsClaimed() const {
@ -1076,7 +1085,7 @@ bool Transaction::ScheduleInShard(EngineShard* shard, bool can_run_immediately)
bool shard_unlocked = shard->shard_lock()->Check(mode); bool shard_unlocked = shard->shard_lock()->Check(mode);
// Check if we can run immediately // Check if we can run immediately
if (shard_unlocked && can_run_immediately && shard->db_slice().CheckLock(mode, lock_args)) { if (shard_unlocked && can_run_immediately && CheckLocks(shard->db_slice(), mode, lock_args)) {
sd.local_mask |= RAN_IMMEDIATELY; sd.local_mask |= RAN_IMMEDIATELY;
shard->stats().tx_immediate_total++; shard->stats().tx_immediate_total++;

View file

@ -170,7 +170,6 @@ class Transaction {
RAN_IMMEDIATELY = 1 << 7, // Whether the shard executed immediately (during schedule) RAN_IMMEDIATELY = 1 << 7, // Whether the shard executed immediately (during schedule)
}; };
public:
explicit Transaction(const CommandId* cid); explicit Transaction(const CommandId* cid);
// Initialize transaction for squashing placed on a specific shard with a given parent tx // Initialize transaction for squashing placed on a specific shard with a given parent tx
@ -475,7 +474,6 @@ class Transaction {
util::fb2::EventCount ec_{}; util::fb2::EventCount ec_{};
}; };
private:
// Init basic fields and reset re-usable. // Init basic fields and reset re-usable.
void InitBase(DbIndex dbid, CmdArgList args); void InitBase(DbIndex dbid, CmdArgList args);