mirror of
https://github.com/dragonflydb/dragonfly.git
synced 2025-05-10 18:05:44 +02:00
chore: change Namespaces to be a global pointer (#4032)
* chore: change Namespaces to be a global pointer Before the namespaces object was defined globally. However it has non-trivial d'tor that is being called after main exits. It's quite dangerous to have global non-POD objects being defined globally. For example, if we used LOG(INFO) inside the Clear function , that would crash dragonfly on exit. Ths PR changes it to be a global pointer. --------- Signed-off-by: Roman Gershman <roman@dragonflydb.io>
This commit is contained in:
parent
9366c67464
commit
be96e6cf99
22 changed files with 76 additions and 71 deletions
|
@ -61,7 +61,7 @@ void BlockingControllerTest::SetUp() {
|
||||||
arg_vec_.emplace_back(s);
|
arg_vec_.emplace_back(s);
|
||||||
}
|
}
|
||||||
|
|
||||||
trans_->InitByArgs(&namespaces.GetDefaultNamespace(), 0, {arg_vec_.data(), arg_vec_.size()});
|
trans_->InitByArgs(&namespaces->GetDefaultNamespace(), 0, {arg_vec_.data(), arg_vec_.size()});
|
||||||
CHECK_EQ(0u, Shard("x", shard_set->size()));
|
CHECK_EQ(0u, Shard("x", shard_set->size()));
|
||||||
CHECK_EQ(2u, Shard("z", shard_set->size()));
|
CHECK_EQ(2u, Shard("z", shard_set->size()));
|
||||||
|
|
||||||
|
@ -71,7 +71,6 @@ void BlockingControllerTest::SetUp() {
|
||||||
|
|
||||||
void BlockingControllerTest::TearDown() {
|
void BlockingControllerTest::TearDown() {
|
||||||
shard_set->PreShutdown();
|
shard_set->PreShutdown();
|
||||||
namespaces.Clear();
|
|
||||||
shard_set->Shutdown();
|
shard_set->Shutdown();
|
||||||
delete shard_set;
|
delete shard_set;
|
||||||
|
|
||||||
|
@ -81,7 +80,7 @@ void BlockingControllerTest::TearDown() {
|
||||||
|
|
||||||
TEST_F(BlockingControllerTest, Basic) {
|
TEST_F(BlockingControllerTest, Basic) {
|
||||||
trans_->ScheduleSingleHop([&](Transaction* t, EngineShard* shard) {
|
trans_->ScheduleSingleHop([&](Transaction* t, EngineShard* shard) {
|
||||||
BlockingController bc(shard, &namespaces.GetDefaultNamespace());
|
BlockingController bc(shard, &namespaces->GetDefaultNamespace());
|
||||||
auto keys = t->GetShardArgs(shard->shard_id());
|
auto keys = t->GetShardArgs(shard->shard_id());
|
||||||
bc.AddWatched(
|
bc.AddWatched(
|
||||||
keys, [](auto...) { return true; }, t);
|
keys, [](auto...) { return true; }, t);
|
||||||
|
@ -107,7 +106,7 @@ TEST_F(BlockingControllerTest, Timeout) {
|
||||||
unsigned num_watched = shard_set->Await(
|
unsigned num_watched = shard_set->Await(
|
||||||
|
|
||||||
0, [&] {
|
0, [&] {
|
||||||
return namespaces.GetDefaultNamespace()
|
return namespaces->GetDefaultNamespace()
|
||||||
.GetBlockingController(EngineShard::tlocal()->shard_id())
|
.GetBlockingController(EngineShard::tlocal()->shard_id())
|
||||||
->NumWatched(0);
|
->NumWatched(0);
|
||||||
});
|
});
|
||||||
|
|
|
@ -477,7 +477,7 @@ void DeleteSlots(const SlotRanges& slots_ranges) {
|
||||||
if (shard == nullptr)
|
if (shard == nullptr)
|
||||||
return;
|
return;
|
||||||
|
|
||||||
namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id()).FlushSlots(slots_ranges);
|
namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id()).FlushSlots(slots_ranges);
|
||||||
};
|
};
|
||||||
shard_set->pool()->AwaitFiberOnAll(std::move(cb));
|
shard_set->pool()->AwaitFiberOnAll(std::move(cb));
|
||||||
}
|
}
|
||||||
|
@ -633,7 +633,7 @@ void ClusterFamily::DflyClusterGetSlotInfo(CmdArgList args, SinkReplyBuilder* bu
|
||||||
|
|
||||||
util::fb2::LockGuard lk(mu);
|
util::fb2::LockGuard lk(mu);
|
||||||
for (auto& [slot, data] : slots_stats) {
|
for (auto& [slot, data] : slots_stats) {
|
||||||
data += namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id()).GetSlotStats(slot);
|
data += namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id()).GetSlotStats(slot);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
|
@ -50,7 +50,7 @@ uint64_t GetKeyCount(const SlotRanges& slots) {
|
||||||
uint64_t shard_keys = 0;
|
uint64_t shard_keys = 0;
|
||||||
for (const SlotRange& range : slots) {
|
for (const SlotRange& range : slots) {
|
||||||
for (SlotId slot = range.start; slot <= range.end; slot++) {
|
for (SlotId slot = range.start; slot <= range.end; slot++) {
|
||||||
shard_keys += namespaces.GetDefaultNamespace()
|
shard_keys += namespaces->GetDefaultNamespace()
|
||||||
.GetDbSlice(shard->shard_id())
|
.GetDbSlice(shard->shard_id())
|
||||||
.GetSlotStats(slot)
|
.GetSlotStats(slot)
|
||||||
.key_count;
|
.key_count;
|
||||||
|
|
|
@ -101,7 +101,7 @@ OutgoingMigration::OutgoingMigration(MigrationInfo info, ClusterFamily* cf, Serv
|
||||||
server_family_(sf),
|
server_family_(sf),
|
||||||
cf_(cf),
|
cf_(cf),
|
||||||
tx_(new Transaction{sf->service().FindCmd("DFLYCLUSTER")}) {
|
tx_(new Transaction{sf->service().FindCmd("DFLYCLUSTER")}) {
|
||||||
tx_->InitByArgs(&namespaces.GetDefaultNamespace(), 0, {});
|
tx_->InitByArgs(&namespaces->GetDefaultNamespace(), 0, {});
|
||||||
}
|
}
|
||||||
|
|
||||||
OutgoingMigration::~OutgoingMigration() {
|
OutgoingMigration::~OutgoingMigration() {
|
||||||
|
@ -218,7 +218,7 @@ void OutgoingMigration::SyncFb() {
|
||||||
}
|
}
|
||||||
|
|
||||||
OnAllShards([this](auto& migration) {
|
OnAllShards([this](auto& migration) {
|
||||||
DbSlice& db_slice = namespaces.GetDefaultNamespace().GetCurrentDbSlice();
|
DbSlice& db_slice = namespaces->GetDefaultNamespace().GetCurrentDbSlice();
|
||||||
server_family_->journal()->StartInThread();
|
server_family_->journal()->StartInThread();
|
||||||
migration = std::make_unique<SliceSlotMigration>(
|
migration = std::make_unique<SliceSlotMigration>(
|
||||||
&db_slice, server(), migration_info_.slot_ranges, server_family_->journal());
|
&db_slice, server(), migration_info_.slot_ranges, server_family_->journal());
|
||||||
|
@ -291,8 +291,8 @@ bool OutgoingMigration::FinalizeMigration(long attempt) {
|
||||||
bool is_block_active = true;
|
bool is_block_active = true;
|
||||||
auto is_pause_in_progress = [&is_block_active] { return is_block_active; };
|
auto is_pause_in_progress = [&is_block_active] { return is_block_active; };
|
||||||
auto pause_fb_opt =
|
auto pause_fb_opt =
|
||||||
Pause(server_family_->GetNonPriviligedListeners(), &namespaces.GetDefaultNamespace(), nullptr,
|
Pause(server_family_->GetNonPriviligedListeners(), &namespaces->GetDefaultNamespace(),
|
||||||
ClientPause::WRITE, is_pause_in_progress);
|
nullptr, ClientPause::WRITE, is_pause_in_progress);
|
||||||
|
|
||||||
if (!pause_fb_opt) {
|
if (!pause_fb_opt) {
|
||||||
LOG(WARNING) << "Cluster migration finalization time out";
|
LOG(WARNING) << "Cluster migration finalization time out";
|
||||||
|
|
|
@ -120,6 +120,7 @@ atomic_uint64_t rss_mem_peak(0);
|
||||||
unsigned kernel_version = 0;
|
unsigned kernel_version = 0;
|
||||||
size_t max_memory_limit = 0;
|
size_t max_memory_limit = 0;
|
||||||
size_t serialization_max_chunk_size = 0;
|
size_t serialization_max_chunk_size = 0;
|
||||||
|
Namespaces* namespaces = nullptr;
|
||||||
|
|
||||||
const char* GlobalStateName(GlobalState s) {
|
const char* GlobalStateName(GlobalState s) {
|
||||||
switch (s) {
|
switch (s) {
|
||||||
|
|
|
@ -49,6 +49,7 @@ class Transaction;
|
||||||
class EngineShard;
|
class EngineShard;
|
||||||
class ConnectionState;
|
class ConnectionState;
|
||||||
class Interpreter;
|
class Interpreter;
|
||||||
|
class Namespaces;
|
||||||
|
|
||||||
struct LockTagOptions {
|
struct LockTagOptions {
|
||||||
bool enabled = false;
|
bool enabled = false;
|
||||||
|
@ -132,6 +133,8 @@ extern std::atomic_uint64_t rss_mem_peak;
|
||||||
|
|
||||||
extern size_t max_memory_limit;
|
extern size_t max_memory_limit;
|
||||||
|
|
||||||
|
extern Namespaces* namespaces;
|
||||||
|
|
||||||
// version 5.11 maps to 511 etc.
|
// version 5.11 maps to 511 etc.
|
||||||
// set upon server start.
|
// set upon server start.
|
||||||
extern unsigned kernel_version;
|
extern unsigned kernel_version;
|
||||||
|
|
|
@ -434,7 +434,7 @@ void SaveStagesController::CloseCb(unsigned index) {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (auto* es = EngineShard::tlocal(); use_dfs_format_ && es)
|
if (auto* es = EngineShard::tlocal(); use_dfs_format_ && es)
|
||||||
namespaces.GetDefaultNamespace().GetDbSlice(es->shard_id()).ResetUpdateEvents();
|
namespaces->GetDefaultNamespace().GetDbSlice(es->shard_id()).ResetUpdateEvents();
|
||||||
}
|
}
|
||||||
|
|
||||||
void SaveStagesController::RunStage(void (SaveStagesController::*cb)(unsigned)) {
|
void SaveStagesController::RunStage(void (SaveStagesController::*cb)(unsigned)) {
|
||||||
|
|
|
@ -77,7 +77,7 @@ bool WaitReplicaFlowToCatchup(absl::Time end_time, const DflyCmd::ReplicaInfo* r
|
||||||
EngineShard* shard) {
|
EngineShard* shard) {
|
||||||
// We don't want any writes to the journal after we send the `PING`,
|
// We don't want any writes to the journal after we send the `PING`,
|
||||||
// and expirations could ruin that.
|
// and expirations could ruin that.
|
||||||
namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id()).SetExpireAllowed(false);
|
namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id()).SetExpireAllowed(false);
|
||||||
shard->journal()->RecordEntry(0, journal::Op::PING, 0, 0, nullopt, {}, true);
|
shard->journal()->RecordEntry(0, journal::Op::PING, 0, 0, nullopt, {}, true);
|
||||||
|
|
||||||
const FlowInfo* flow = &replica->flows[shard->shard_id()];
|
const FlowInfo* flow = &replica->flows[shard->shard_id()];
|
||||||
|
@ -455,7 +455,7 @@ void DflyCmd::TakeOver(CmdArgList args, RedisReplyBuilder* rb, ConnectionContext
|
||||||
absl::Cleanup cleanup([] {
|
absl::Cleanup cleanup([] {
|
||||||
VLOG(2) << "Enabling expiration";
|
VLOG(2) << "Enabling expiration";
|
||||||
shard_set->RunBriefInParallel([](EngineShard* shard) {
|
shard_set->RunBriefInParallel([](EngineShard* shard) {
|
||||||
namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id()).SetExpireAllowed(true);
|
namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id()).SetExpireAllowed(true);
|
||||||
});
|
});
|
||||||
});
|
});
|
||||||
|
|
||||||
|
|
|
@ -379,7 +379,7 @@ TEST_F(DflyEngineTest, MemcacheFlags) {
|
||||||
ASSERT_EQ(Run("resp", {"flushdb"}), "OK");
|
ASSERT_EQ(Run("resp", {"flushdb"}), "OK");
|
||||||
pp_->AwaitFiberOnAll([](auto*) {
|
pp_->AwaitFiberOnAll([](auto*) {
|
||||||
if (auto* shard = EngineShard::tlocal(); shard) {
|
if (auto* shard = EngineShard::tlocal(); shard) {
|
||||||
EXPECT_EQ(namespaces.GetDefaultNamespace()
|
EXPECT_EQ(namespaces->GetDefaultNamespace()
|
||||||
.GetDbSlice(shard->shard_id())
|
.GetDbSlice(shard->shard_id())
|
||||||
.GetDBTable(0)
|
.GetDBTable(0)
|
||||||
->mcflag.size(),
|
->mcflag.size(),
|
||||||
|
@ -600,7 +600,7 @@ TEST_F(DflyEngineTest, Bug468) {
|
||||||
|
|
||||||
TEST_F(DflyEngineTest, Bug496) {
|
TEST_F(DflyEngineTest, Bug496) {
|
||||||
shard_set->RunBlockingInParallel([](EngineShard* shard) {
|
shard_set->RunBlockingInParallel([](EngineShard* shard) {
|
||||||
auto& db = namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id());
|
auto& db = namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id());
|
||||||
|
|
||||||
int cb_hits = 0;
|
int cb_hits = 0;
|
||||||
uint32_t cb_id =
|
uint32_t cb_id =
|
||||||
|
|
|
@ -309,7 +309,7 @@ bool EngineShard::DoDefrag() {
|
||||||
const float threshold = GetFlag(FLAGS_mem_defrag_page_utilization_threshold);
|
const float threshold = GetFlag(FLAGS_mem_defrag_page_utilization_threshold);
|
||||||
|
|
||||||
// TODO: enable tiered storage on non-default db slice
|
// TODO: enable tiered storage on non-default db slice
|
||||||
DbSlice& slice = namespaces.GetDefaultNamespace().GetDbSlice(shard_->shard_id());
|
DbSlice& slice = namespaces->GetDefaultNamespace().GetDbSlice(shard_->shard_id());
|
||||||
|
|
||||||
// If we moved to an invalid db, skip as long as it's not the last one
|
// If we moved to an invalid db, skip as long as it's not the last one
|
||||||
while (!slice.IsDbValid(defrag_state_.dbid) && defrag_state_.dbid + 1 < slice.db_array_size())
|
while (!slice.IsDbValid(defrag_state_.dbid) && defrag_state_.dbid + 1 < slice.db_array_size())
|
||||||
|
@ -339,7 +339,7 @@ bool EngineShard::DoDefrag() {
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
traverses_count++;
|
traverses_count++;
|
||||||
} while (traverses_count < kMaxTraverses && cur && namespaces.IsInitialized());
|
} while (traverses_count < kMaxTraverses && cur && namespaces);
|
||||||
|
|
||||||
defrag_state_.UpdateScanState(cur.value());
|
defrag_state_.UpdateScanState(cur.value());
|
||||||
|
|
||||||
|
@ -370,7 +370,7 @@ bool EngineShard::DoDefrag() {
|
||||||
// priority.
|
// priority.
|
||||||
// otherwise lower the task priority so that it would not use the CPU when not required
|
// otherwise lower the task priority so that it would not use the CPU when not required
|
||||||
uint32_t EngineShard::DefragTask() {
|
uint32_t EngineShard::DefragTask() {
|
||||||
if (!namespaces.IsInitialized()) {
|
if (!namespaces) {
|
||||||
return util::ProactorBase::kOnIdleMaxLevel;
|
return util::ProactorBase::kOnIdleMaxLevel;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -392,7 +392,6 @@ EngineShard::EngineShard(util::ProactorBase* pb, mi_heap_t* heap)
|
||||||
txq_([](const Transaction* t) { return t->txid(); }),
|
txq_([](const Transaction* t) { return t->txid(); }),
|
||||||
mi_resource_(heap),
|
mi_resource_(heap),
|
||||||
shard_id_(pb->GetPoolIndex()) {
|
shard_id_(pb->GetPoolIndex()) {
|
||||||
defrag_task_ = pb->AddOnIdleTask([this]() { return DefragTask(); });
|
|
||||||
queue_.Start(absl::StrCat("shard_queue_", shard_id()));
|
queue_.Start(absl::StrCat("shard_queue_", shard_id()));
|
||||||
queue2_.Start(absl::StrCat("l2_queue_", shard_id()));
|
queue2_.Start(absl::StrCat("l2_queue_", shard_id()));
|
||||||
}
|
}
|
||||||
|
@ -452,6 +451,7 @@ void EngineShard::StartPeriodicHeartbeatFiber(util::ProactorBase* pb) {
|
||||||
ThisFiber::SetName(absl::StrCat("heartbeat_periodic", index));
|
ThisFiber::SetName(absl::StrCat("heartbeat_periodic", index));
|
||||||
RunFPeriodically(heartbeat, period_ms, "heartbeat", &fiber_heartbeat_periodic_done_);
|
RunFPeriodically(heartbeat, period_ms, "heartbeat", &fiber_heartbeat_periodic_done_);
|
||||||
});
|
});
|
||||||
|
defrag_task_ = pb->AddOnIdleTask([this]() { return DefragTask(); });
|
||||||
}
|
}
|
||||||
|
|
||||||
void EngineShard::StartPeriodicShardHandlerFiber(util::ProactorBase* pb,
|
void EngineShard::StartPeriodicShardHandlerFiber(util::ProactorBase* pb,
|
||||||
|
@ -492,7 +492,7 @@ void EngineShard::InitTieredStorage(ProactorBase* pb, size_t max_file_size) {
|
||||||
<< "Only ioring based backing storage is supported. Exiting...";
|
<< "Only ioring based backing storage is supported. Exiting...";
|
||||||
|
|
||||||
// TODO: enable tiered storage on non-default namespace
|
// TODO: enable tiered storage on non-default namespace
|
||||||
DbSlice& db_slice = namespaces.GetDefaultNamespace().GetDbSlice(shard_id());
|
DbSlice& db_slice = namespaces->GetDefaultNamespace().GetDbSlice(shard_id());
|
||||||
auto* shard = EngineShard::tlocal();
|
auto* shard = EngineShard::tlocal();
|
||||||
shard->tiered_storage_ = make_unique<TieredStorage>(max_file_size, &db_slice);
|
shard->tiered_storage_ = make_unique<TieredStorage>(max_file_size, &db_slice);
|
||||||
error_code ec = shard->tiered_storage_->Open(backing_prefix);
|
error_code ec = shard->tiered_storage_->Open(backing_prefix);
|
||||||
|
@ -657,7 +657,7 @@ void EngineShard::RemoveContTx(Transaction* tx) {
|
||||||
|
|
||||||
void EngineShard::Heartbeat() {
|
void EngineShard::Heartbeat() {
|
||||||
DVLOG(2) << " Hearbeat";
|
DVLOG(2) << " Hearbeat";
|
||||||
DCHECK(namespaces.IsInitialized());
|
DCHECK(namespaces);
|
||||||
|
|
||||||
CacheStats();
|
CacheStats();
|
||||||
|
|
||||||
|
@ -666,7 +666,7 @@ void EngineShard::Heartbeat() {
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: iterate over all namespaces
|
// TODO: iterate over all namespaces
|
||||||
DbSlice& db_slice = namespaces.GetDefaultNamespace().GetDbSlice(shard_id());
|
DbSlice& db_slice = namespaces->GetDefaultNamespace().GetDbSlice(shard_id());
|
||||||
|
|
||||||
// Offset CoolMemoryUsage when consider background offloading.
|
// Offset CoolMemoryUsage when consider background offloading.
|
||||||
// TODO: Another approach could be is to align the approach similarly to how we do with
|
// TODO: Another approach could be is to align the approach similarly to how we do with
|
||||||
|
@ -692,7 +692,7 @@ void EngineShard::Heartbeat() {
|
||||||
|
|
||||||
void EngineShard::RetireExpiredAndEvict() {
|
void EngineShard::RetireExpiredAndEvict() {
|
||||||
// TODO: iterate over all namespaces
|
// TODO: iterate over all namespaces
|
||||||
DbSlice& db_slice = namespaces.GetDefaultNamespace().GetDbSlice(shard_id());
|
DbSlice& db_slice = namespaces->GetDefaultNamespace().GetDbSlice(shard_id());
|
||||||
// Some of the functions below might acquire the same lock again so we need to unlock it
|
// Some of the functions below might acquire the same lock again so we need to unlock it
|
||||||
// asap. We won't yield before we relock the mutex again, so the code below is atomic
|
// asap. We won't yield before we relock the mutex again, so the code below is atomic
|
||||||
// in respect to preemptions of big values. An example of that is the call to
|
// in respect to preemptions of big values. An example of that is the call to
|
||||||
|
@ -758,7 +758,7 @@ void EngineShard::CacheStats() {
|
||||||
cache_stats_time_ = now;
|
cache_stats_time_ = now;
|
||||||
// Used memory for this shard.
|
// Used memory for this shard.
|
||||||
size_t used_mem = UsedMemory();
|
size_t used_mem = UsedMemory();
|
||||||
DbSlice& db_slice = namespaces.GetDefaultNamespace().GetDbSlice(shard_id());
|
DbSlice& db_slice = namespaces->GetDefaultNamespace().GetDbSlice(shard_id());
|
||||||
|
|
||||||
// delta can wrap if used_memory is smaller than last_cached_used_memory_ and it's fine.
|
// delta can wrap if used_memory is smaller than last_cached_used_memory_ and it's fine.
|
||||||
size_t delta = used_mem - last_cached_used_memory_;
|
size_t delta = used_mem - last_cached_used_memory_;
|
||||||
|
@ -808,7 +808,7 @@ EngineShard::TxQueueInfo EngineShard::AnalyzeTxQueue() const {
|
||||||
info.tx_total = queue->size();
|
info.tx_total = queue->size();
|
||||||
unsigned max_db_id = 0;
|
unsigned max_db_id = 0;
|
||||||
|
|
||||||
auto& db_slice = namespaces.GetDefaultNamespace().GetCurrentDbSlice();
|
auto& db_slice = namespaces->GetDefaultNamespace().GetCurrentDbSlice();
|
||||||
|
|
||||||
{
|
{
|
||||||
auto value = queue->At(cur);
|
auto value = queue->At(cur);
|
||||||
|
|
|
@ -103,7 +103,10 @@ EngineShardSet* shard_set = nullptr;
|
||||||
|
|
||||||
void EngineShardSet::Init(uint32_t sz, std::function<void()> shard_handler) {
|
void EngineShardSet::Init(uint32_t sz, std::function<void()> shard_handler) {
|
||||||
CHECK_EQ(0u, size());
|
CHECK_EQ(0u, size());
|
||||||
|
CHECK(namespaces == nullptr);
|
||||||
|
|
||||||
shards_.reset(new EngineShard*[sz]);
|
shards_.reset(new EngineShard*[sz]);
|
||||||
|
|
||||||
size_ = sz;
|
size_ = sz;
|
||||||
size_t max_shard_file_size = GetTieredFileLimit(sz);
|
size_t max_shard_file_size = GetTieredFileLimit(sz);
|
||||||
pp_->AwaitFiberOnAll([this](uint32_t index, ProactorBase* pb) {
|
pp_->AwaitFiberOnAll([this](uint32_t index, ProactorBase* pb) {
|
||||||
|
@ -112,7 +115,8 @@ void EngineShardSet::Init(uint32_t sz, std::function<void()> shard_handler) {
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
namespaces.Init();
|
// The order is important here. We must initialize namespaces after shards_.
|
||||||
|
namespaces = new Namespaces();
|
||||||
|
|
||||||
pp_->AwaitFiberOnAll([&](uint32_t index, ProactorBase* pb) {
|
pp_->AwaitFiberOnAll([&](uint32_t index, ProactorBase* pb) {
|
||||||
if (index < size_) {
|
if (index < size_) {
|
||||||
|
@ -139,7 +143,13 @@ void EngineShardSet::PreShutdown() {
|
||||||
}
|
}
|
||||||
|
|
||||||
void EngineShardSet::Shutdown() {
|
void EngineShardSet::Shutdown() {
|
||||||
|
// Calling Namespaces::Clear before destroying engine shards, because it accesses them
|
||||||
|
// internally.
|
||||||
|
namespaces->Clear();
|
||||||
RunBlockingInParallel([](EngineShard*) { EngineShard::DestroyThreadLocal(); });
|
RunBlockingInParallel([](EngineShard*) { EngineShard::DestroyThreadLocal(); });
|
||||||
|
|
||||||
|
delete namespaces;
|
||||||
|
namespaces = nullptr;
|
||||||
}
|
}
|
||||||
|
|
||||||
void EngineShardSet::InitThreadLocal(ProactorBase* pb) {
|
void EngineShardSet::InitThreadLocal(ProactorBase* pb) {
|
||||||
|
@ -150,7 +160,7 @@ void EngineShardSet::InitThreadLocal(ProactorBase* pb) {
|
||||||
|
|
||||||
void EngineShardSet::TEST_EnableCacheMode() {
|
void EngineShardSet::TEST_EnableCacheMode() {
|
||||||
RunBlockingInParallel([](EngineShard* shard) {
|
RunBlockingInParallel([](EngineShard* shard) {
|
||||||
namespaces.GetDefaultNamespace().GetCurrentDbSlice().TEST_EnableCacheMode();
|
namespaces->GetDefaultNamespace().GetCurrentDbSlice().TEST_EnableCacheMode();
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -42,7 +42,7 @@ JournalExecutor::JournalExecutor(Service* service)
|
||||||
conn_context_.is_replicating = true;
|
conn_context_.is_replicating = true;
|
||||||
conn_context_.journal_emulated = true;
|
conn_context_.journal_emulated = true;
|
||||||
conn_context_.skip_acl_validation = true;
|
conn_context_.skip_acl_validation = true;
|
||||||
conn_context_.ns = &namespaces.GetDefaultNamespace();
|
conn_context_.ns = &namespaces->GetDefaultNamespace();
|
||||||
}
|
}
|
||||||
|
|
||||||
JournalExecutor::~JournalExecutor() {
|
JournalExecutor::~JournalExecutor() {
|
||||||
|
|
|
@ -33,7 +33,7 @@ class ListFamilyTest : public BaseFamilyTest {
|
||||||
static unsigned NumWatched() {
|
static unsigned NumWatched() {
|
||||||
atomic_uint32_t sum{0};
|
atomic_uint32_t sum{0};
|
||||||
|
|
||||||
auto ns = &namespaces.GetDefaultNamespace();
|
auto ns = &namespaces->GetDefaultNamespace();
|
||||||
shard_set->RunBriefInParallel([&](EngineShard* es) {
|
shard_set->RunBriefInParallel([&](EngineShard* es) {
|
||||||
auto* bc = ns->GetBlockingController(es->shard_id());
|
auto* bc = ns->GetBlockingController(es->shard_id());
|
||||||
if (bc)
|
if (bc)
|
||||||
|
@ -45,7 +45,7 @@ class ListFamilyTest : public BaseFamilyTest {
|
||||||
|
|
||||||
static bool HasAwakened() {
|
static bool HasAwakened() {
|
||||||
atomic_uint32_t sum{0};
|
atomic_uint32_t sum{0};
|
||||||
auto ns = &namespaces.GetDefaultNamespace();
|
auto ns = &namespaces->GetDefaultNamespace();
|
||||||
shard_set->RunBriefInParallel([&](EngineShard* es) {
|
shard_set->RunBriefInParallel([&](EngineShard* es) {
|
||||||
auto* bc = ns->GetBlockingController(es->shard_id());
|
auto* bc = ns->GetBlockingController(es->shard_id());
|
||||||
if (bc)
|
if (bc)
|
||||||
|
|
|
@ -468,7 +468,7 @@ void Topkeys(const http::QueryArgs& args, HttpContext* send) {
|
||||||
|
|
||||||
shard_set->RunBriefInParallel([&](EngineShard* shard) {
|
shard_set->RunBriefInParallel([&](EngineShard* shard) {
|
||||||
for (const auto& db :
|
for (const auto& db :
|
||||||
namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id()).databases()) {
|
namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id()).databases()) {
|
||||||
if (db->top_keys.IsEnabled()) {
|
if (db->top_keys.IsEnabled()) {
|
||||||
is_enabled = true;
|
is_enabled = true;
|
||||||
for (const auto& [key, count] : db->top_keys.GetTopKeys()) {
|
for (const auto& [key, count] : db->top_keys.GetTopKeys()) {
|
||||||
|
@ -823,7 +823,7 @@ void Service::Init(util::AcceptServer* acceptor, std::vector<facade::Listener*>
|
||||||
auto* shard = EngineShard::tlocal();
|
auto* shard = EngineShard::tlocal();
|
||||||
if (shard) {
|
if (shard) {
|
||||||
auto shard_id = shard->shard_id();
|
auto shard_id = shard->shard_id();
|
||||||
auto& db_slice = namespaces.GetDefaultNamespace().GetDbSlice(shard_id);
|
auto& db_slice = namespaces->GetDefaultNamespace().GetDbSlice(shard_id);
|
||||||
db_slice.SetNotifyKeyspaceEvents(*res);
|
db_slice.SetNotifyKeyspaceEvents(*res);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -897,7 +897,6 @@ void Service::Shutdown() {
|
||||||
ChannelStore::Destroy();
|
ChannelStore::Destroy();
|
||||||
|
|
||||||
shard_set->PreShutdown();
|
shard_set->PreShutdown();
|
||||||
namespaces.Clear();
|
|
||||||
shard_set->Shutdown();
|
shard_set->Shutdown();
|
||||||
Transaction::Shutdown();
|
Transaction::Shutdown();
|
||||||
|
|
||||||
|
@ -1586,7 +1585,7 @@ facade::ConnectionContext* Service::CreateContext(util::FiberSocketBase* peer,
|
||||||
facade::Connection* owner) {
|
facade::Connection* owner) {
|
||||||
auto cred = user_registry_.GetCredentials("default");
|
auto cred = user_registry_.GetCredentials("default");
|
||||||
ConnectionContext* res = new ConnectionContext{peer, owner, std::move(cred)};
|
ConnectionContext* res = new ConnectionContext{peer, owner, std::move(cred)};
|
||||||
res->ns = &namespaces.GetOrInsert("");
|
res->ns = &namespaces->GetOrInsert("");
|
||||||
|
|
||||||
if (peer->IsUDS()) {
|
if (peer->IsUDS()) {
|
||||||
res->req_auth = false;
|
res->req_auth = false;
|
||||||
|
@ -2449,7 +2448,7 @@ void Service::Command(CmdArgList args, Transaction* tx, SinkReplyBuilder* builde
|
||||||
VarzValue::Map Service::GetVarzStats() {
|
VarzValue::Map Service::GetVarzStats() {
|
||||||
VarzValue::Map res;
|
VarzValue::Map res;
|
||||||
|
|
||||||
Metrics m = server_family_.GetMetrics(&namespaces.GetDefaultNamespace());
|
Metrics m = server_family_.GetMetrics(&namespaces->GetDefaultNamespace());
|
||||||
DbStats db_stats;
|
DbStats db_stats;
|
||||||
for (const auto& s : m.db_stats) {
|
for (const auto& s : m.db_stats) {
|
||||||
db_stats += s;
|
db_stats += s;
|
||||||
|
|
|
@ -1,3 +1,7 @@
|
||||||
|
// Copyright 2024, DragonflyDB authors. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
#include "server/namespaces.h"
|
#include "server/namespaces.h"
|
||||||
|
|
||||||
#include "base/flags.h"
|
#include "base/flags.h"
|
||||||
|
@ -45,19 +49,12 @@ BlockingController* Namespace::GetBlockingController(ShardId sid) {
|
||||||
return shard_blocking_controller_[sid].get();
|
return shard_blocking_controller_[sid].get();
|
||||||
}
|
}
|
||||||
|
|
||||||
Namespaces namespaces;
|
Namespaces::Namespaces() {
|
||||||
|
|
||||||
Namespaces::~Namespaces() {
|
|
||||||
Clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
void Namespaces::Init() {
|
|
||||||
DCHECK(default_namespace_ == nullptr);
|
|
||||||
default_namespace_ = &GetOrInsert("");
|
default_namespace_ = &GetOrInsert("");
|
||||||
}
|
}
|
||||||
|
|
||||||
bool Namespaces::IsInitialized() const {
|
Namespaces::~Namespaces() {
|
||||||
return default_namespace_ != nullptr;
|
Clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
void Namespaces::Clear() {
|
void Namespaces::Clear() {
|
||||||
|
|
|
@ -49,11 +49,9 @@ class Namespace {
|
||||||
// mutual dependencies.
|
// mutual dependencies.
|
||||||
class Namespaces {
|
class Namespaces {
|
||||||
public:
|
public:
|
||||||
Namespaces() = default;
|
Namespaces();
|
||||||
~Namespaces();
|
~Namespaces();
|
||||||
|
|
||||||
void Init();
|
|
||||||
bool IsInitialized() const;
|
|
||||||
void Clear() ABSL_LOCKS_EXCLUDED(mu_); // Thread unsafe, use in tear-down or tests
|
void Clear() ABSL_LOCKS_EXCLUDED(mu_); // Thread unsafe, use in tear-down or tests
|
||||||
|
|
||||||
Namespace& GetDefaultNamespace() const; // No locks
|
Namespace& GetDefaultNamespace() const; // No locks
|
||||||
|
@ -65,6 +63,4 @@ class Namespaces {
|
||||||
Namespace* default_namespace_ = nullptr;
|
Namespace* default_namespace_ = nullptr;
|
||||||
};
|
};
|
||||||
|
|
||||||
extern Namespaces namespaces;
|
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace dfly
|
||||||
|
|
|
@ -2258,7 +2258,7 @@ error_code RdbLoader::Load(io::Source* src) {
|
||||||
|
|
||||||
// Active database if not existed before.
|
// Active database if not existed before.
|
||||||
shard_set->Add(
|
shard_set->Add(
|
||||||
i, [dbid] { namespaces.GetDefaultNamespace().GetCurrentDbSlice().ActivateDb(dbid); });
|
i, [dbid] { namespaces->GetDefaultNamespace().GetCurrentDbSlice().ActivateDb(dbid); });
|
||||||
}
|
}
|
||||||
|
|
||||||
cur_db_index_ = dbid;
|
cur_db_index_ = dbid;
|
||||||
|
@ -2656,7 +2656,7 @@ std::error_code RdbLoaderBase::FromOpaque(const OpaqueObj& opaque, LoadConfig co
|
||||||
|
|
||||||
void RdbLoader::LoadItemsBuffer(DbIndex db_ind, const ItemsBuf& ib) {
|
void RdbLoader::LoadItemsBuffer(DbIndex db_ind, const ItemsBuf& ib) {
|
||||||
EngineShard* es = EngineShard::tlocal();
|
EngineShard* es = EngineShard::tlocal();
|
||||||
DbContext db_cntx{&namespaces.GetDefaultNamespace(), db_ind, GetCurrentTimeMs()};
|
DbContext db_cntx{&namespaces->GetDefaultNamespace(), db_ind, GetCurrentTimeMs()};
|
||||||
DbSlice& db_slice = db_cntx.GetDbSlice(es->shard_id());
|
DbSlice& db_slice = db_cntx.GetDbSlice(es->shard_id());
|
||||||
|
|
||||||
auto error_msg = [](const auto* item, auto db_ind) {
|
auto error_msg = [](const auto* item, auto db_ind) {
|
||||||
|
@ -2860,7 +2860,7 @@ void RdbLoader::LoadSearchIndexDefFromAux(string&& def) {
|
||||||
cntx.is_replicating = true;
|
cntx.is_replicating = true;
|
||||||
cntx.journal_emulated = true;
|
cntx.journal_emulated = true;
|
||||||
cntx.skip_acl_validation = true;
|
cntx.skip_acl_validation = true;
|
||||||
cntx.ns = &namespaces.GetDefaultNamespace();
|
cntx.ns = &namespaces->GetDefaultNamespace();
|
||||||
|
|
||||||
uint32_t consumed = 0;
|
uint32_t consumed = 0;
|
||||||
facade::RespVec resp_vec;
|
facade::RespVec resp_vec;
|
||||||
|
@ -2897,7 +2897,7 @@ void RdbLoader::PerformPostLoad(Service* service) {
|
||||||
// Rebuild all search indices as only their definitions are extracted from the snapshot
|
// Rebuild all search indices as only their definitions are extracted from the snapshot
|
||||||
shard_set->AwaitRunningOnShardQueue([](EngineShard* es) {
|
shard_set->AwaitRunningOnShardQueue([](EngineShard* es) {
|
||||||
es->search_indices()->RebuildAllIndices(
|
es->search_indices()->RebuildAllIndices(
|
||||||
OpArgs{es, nullptr, DbContext{&namespaces.GetDefaultNamespace(), 0, GetCurrentTimeMs()}});
|
OpArgs{es, nullptr, DbContext{&namespaces->GetDefaultNamespace(), 0, GetCurrentTimeMs()}});
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1327,7 +1327,7 @@ void RdbSaver::Impl::FinalizeSnapshotWriting() {
|
||||||
|
|
||||||
void RdbSaver::Impl::StartSnapshotting(bool stream_journal, Context* cntx, EngineShard* shard) {
|
void RdbSaver::Impl::StartSnapshotting(bool stream_journal, Context* cntx, EngineShard* shard) {
|
||||||
auto& s = GetSnapshot(shard);
|
auto& s = GetSnapshot(shard);
|
||||||
auto& db_slice = namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id());
|
auto& db_slice = namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id());
|
||||||
auto on_snapshot_finish = std::bind(&RdbSaver::Impl::FinalizeSnapshotWriting, this);
|
auto on_snapshot_finish = std::bind(&RdbSaver::Impl::FinalizeSnapshotWriting, this);
|
||||||
auto push_cb = std::bind(&RdbSaver::Impl::PushSnapshotData, this, cntx, std::placeholders::_1);
|
auto push_cb = std::bind(&RdbSaver::Impl::PushSnapshotData, this, cntx, std::placeholders::_1);
|
||||||
|
|
||||||
|
@ -1341,7 +1341,7 @@ void RdbSaver::Impl::StartSnapshotting(bool stream_journal, Context* cntx, Engin
|
||||||
|
|
||||||
void RdbSaver::Impl::StartIncrementalSnapshotting(Context* cntx, EngineShard* shard,
|
void RdbSaver::Impl::StartIncrementalSnapshotting(Context* cntx, EngineShard* shard,
|
||||||
LSN start_lsn) {
|
LSN start_lsn) {
|
||||||
auto& db_slice = namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id());
|
auto& db_slice = namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id());
|
||||||
auto& s = GetSnapshot(shard);
|
auto& s = GetSnapshot(shard);
|
||||||
auto on_finalize_cb = std::bind(&RdbSaver::Impl::FinalizeSnapshotWriting, this);
|
auto on_finalize_cb = std::bind(&RdbSaver::Impl::FinalizeSnapshotWriting, this);
|
||||||
auto push_cb = std::bind(&RdbSaver::Impl::PushSnapshotData, this, cntx, std::placeholders::_1);
|
auto push_cb = std::bind(&RdbSaver::Impl::PushSnapshotData, this, cntx, std::placeholders::_1);
|
||||||
|
|
|
@ -597,7 +597,7 @@ error_code Replica::ConsumeRedisStream() {
|
||||||
conn_context.is_replicating = true;
|
conn_context.is_replicating = true;
|
||||||
conn_context.journal_emulated = true;
|
conn_context.journal_emulated = true;
|
||||||
conn_context.skip_acl_validation = true;
|
conn_context.skip_acl_validation = true;
|
||||||
conn_context.ns = &namespaces.GetDefaultNamespace();
|
conn_context.ns = &namespaces->GetDefaultNamespace();
|
||||||
|
|
||||||
// we never reply back on the commands.
|
// we never reply back on the commands.
|
||||||
facade::CapturingReplyBuilder null_builder{facade::ReplyMode::NONE};
|
facade::CapturingReplyBuilder null_builder{facade::ReplyMode::NONE};
|
||||||
|
|
|
@ -1533,7 +1533,7 @@ void ServerFamily::ConfigureMetrics(util::HttpListenerBase* http_base) {
|
||||||
|
|
||||||
auto cb = [this](const util::http::QueryArgs& args, util::HttpContext* send) {
|
auto cb = [this](const util::http::QueryArgs& args, util::HttpContext* send) {
|
||||||
StringResponse resp = util::http::MakeStringResponse(boost::beast::http::status::ok);
|
StringResponse resp = util::http::MakeStringResponse(boost::beast::http::status::ok);
|
||||||
PrintPrometheusMetrics(this->GetMetrics(&namespaces.GetDefaultNamespace()),
|
PrintPrometheusMetrics(this->GetMetrics(&namespaces->GetDefaultNamespace()),
|
||||||
this->dfly_cmd_.get(), &resp);
|
this->dfly_cmd_.get(), &resp);
|
||||||
|
|
||||||
return send->Invoke(std::move(resp));
|
return send->Invoke(std::move(resp));
|
||||||
|
@ -1608,7 +1608,7 @@ void ServerFamily::StatsMC(std::string_view section, SinkReplyBuilder* builder)
|
||||||
double utime = dbl_time(ru.ru_utime);
|
double utime = dbl_time(ru.ru_utime);
|
||||||
double systime = dbl_time(ru.ru_stime);
|
double systime = dbl_time(ru.ru_stime);
|
||||||
|
|
||||||
Metrics m = GetMetrics(&namespaces.GetDefaultNamespace());
|
Metrics m = GetMetrics(&namespaces->GetDefaultNamespace());
|
||||||
|
|
||||||
ADD_LINE(pid, getpid());
|
ADD_LINE(pid, getpid());
|
||||||
ADD_LINE(uptime, m.uptime);
|
ADD_LINE(uptime, m.uptime);
|
||||||
|
@ -1638,7 +1638,7 @@ GenericError ServerFamily::DoSave(bool ignore_state) {
|
||||||
const CommandId* cid = service().FindCmd("SAVE");
|
const CommandId* cid = service().FindCmd("SAVE");
|
||||||
CHECK_NOTNULL(cid);
|
CHECK_NOTNULL(cid);
|
||||||
boost::intrusive_ptr<Transaction> trans(new Transaction{cid});
|
boost::intrusive_ptr<Transaction> trans(new Transaction{cid});
|
||||||
trans->InitByArgs(&namespaces.GetDefaultNamespace(), 0, {});
|
trans->InitByArgs(&namespaces->GetDefaultNamespace(), 0, {});
|
||||||
return DoSave(absl::GetFlag(FLAGS_df_snapshot_format), {}, trans.get(), ignore_state);
|
return DoSave(absl::GetFlag(FLAGS_df_snapshot_format), {}, trans.get(), ignore_state);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1826,7 +1826,7 @@ bool ServerFamily::DoAuth(ConnectionContext* cntx, std::string_view username,
|
||||||
cntx->acl_commands = cred.acl_commands;
|
cntx->acl_commands = cred.acl_commands;
|
||||||
cntx->keys = std::move(cred.keys);
|
cntx->keys = std::move(cred.keys);
|
||||||
cntx->pub_sub = std::move(cred.pub_sub);
|
cntx->pub_sub = std::move(cred.pub_sub);
|
||||||
cntx->ns = &namespaces.GetOrInsert(cred.ns);
|
cntx->ns = &namespaces->GetOrInsert(cred.ns);
|
||||||
cntx->authenticated = true;
|
cntx->authenticated = true;
|
||||||
}
|
}
|
||||||
return is_authorized;
|
return is_authorized;
|
||||||
|
|
|
@ -82,7 +82,7 @@ void TransactionSuspension::Start() {
|
||||||
|
|
||||||
transaction_ = new dfly::Transaction{&cid};
|
transaction_ = new dfly::Transaction{&cid};
|
||||||
|
|
||||||
auto st = transaction_->InitByArgs(&namespaces.GetDefaultNamespace(), 0, {});
|
auto st = transaction_->InitByArgs(&namespaces->GetDefaultNamespace(), 0, {});
|
||||||
CHECK_EQ(st, OpStatus::OK);
|
CHECK_EQ(st, OpStatus::OK);
|
||||||
|
|
||||||
transaction_->Execute([](Transaction* t, EngineShard* shard) { return OpStatus::OK; }, false);
|
transaction_->Execute([](Transaction* t, EngineShard* shard) { return OpStatus::OK; }, false);
|
||||||
|
@ -109,7 +109,7 @@ class BaseFamilyTest::TestConnWrapper {
|
||||||
|
|
||||||
ConnectionContext* cmd_cntx() {
|
ConnectionContext* cmd_cntx() {
|
||||||
auto cntx = static_cast<ConnectionContext*>(dummy_conn_->cntx());
|
auto cntx = static_cast<ConnectionContext*>(dummy_conn_->cntx());
|
||||||
cntx->ns = &namespaces.GetDefaultNamespace();
|
cntx->ns = &namespaces->GetDefaultNamespace();
|
||||||
return cntx;
|
return cntx;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -213,7 +213,7 @@ void BaseFamilyTest::ResetService() {
|
||||||
used_mem_current = 0;
|
used_mem_current = 0;
|
||||||
|
|
||||||
TEST_current_time_ms = absl::GetCurrentTimeNanos() / 1000000;
|
TEST_current_time_ms = absl::GetCurrentTimeNanos() / 1000000;
|
||||||
auto default_ns = &namespaces.GetDefaultNamespace();
|
auto default_ns = &namespaces->GetDefaultNamespace();
|
||||||
auto cb = [&](EngineShard* s) {
|
auto cb = [&](EngineShard* s) {
|
||||||
default_ns->GetDbSlice(s->shard_id()).UpdateExpireBase(TEST_current_time_ms - 1000, 0);
|
default_ns->GetDbSlice(s->shard_id()).UpdateExpireBase(TEST_current_time_ms - 1000, 0);
|
||||||
};
|
};
|
||||||
|
@ -250,7 +250,7 @@ void BaseFamilyTest::ResetService() {
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG(ERROR) << "TxLocks for shard " << es->shard_id();
|
LOG(ERROR) << "TxLocks for shard " << es->shard_id();
|
||||||
for (const auto& k_v : namespaces.GetDefaultNamespace()
|
for (const auto& k_v : namespaces->GetDefaultNamespace()
|
||||||
.GetDbSlice(es->shard_id())
|
.GetDbSlice(es->shard_id())
|
||||||
.GetDBTable(0)
|
.GetDBTable(0)
|
||||||
->trans_locks) {
|
->trans_locks) {
|
||||||
|
@ -305,7 +305,7 @@ void BaseFamilyTest::CleanupSnapshots() {
|
||||||
|
|
||||||
unsigned BaseFamilyTest::NumLocked() {
|
unsigned BaseFamilyTest::NumLocked() {
|
||||||
atomic_uint count = 0;
|
atomic_uint count = 0;
|
||||||
auto default_ns = &namespaces.GetDefaultNamespace();
|
auto default_ns = &namespaces->GetDefaultNamespace();
|
||||||
shard_set->RunBriefInParallel([&](EngineShard* shard) {
|
shard_set->RunBriefInParallel([&](EngineShard* shard) {
|
||||||
for (const auto& db : default_ns->GetDbSlice(shard->shard_id()).databases()) {
|
for (const auto& db : default_ns->GetDbSlice(shard->shard_id()).databases()) {
|
||||||
if (db == nullptr) {
|
if (db == nullptr) {
|
||||||
|
@ -386,7 +386,7 @@ RespExpr BaseFamilyTest::Run(std::string_view id, ArgSlice slice) {
|
||||||
CmdArgVec args = conn_wrapper->Args(slice);
|
CmdArgVec args = conn_wrapper->Args(slice);
|
||||||
|
|
||||||
auto* context = conn_wrapper->cmd_cntx();
|
auto* context = conn_wrapper->cmd_cntx();
|
||||||
context->ns = &namespaces.GetDefaultNamespace();
|
context->ns = &namespaces->GetDefaultNamespace();
|
||||||
|
|
||||||
DCHECK(context->transaction == nullptr) << id;
|
DCHECK(context->transaction == nullptr) << id;
|
||||||
|
|
||||||
|
@ -566,7 +566,7 @@ 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 {
|
||||||
return service_->IsLocked(&namespaces.GetDefaultNamespace(), db_index, key);
|
return service_->IsLocked(&namespaces->GetDefaultNamespace(), db_index, key);
|
||||||
}
|
}
|
||||||
|
|
||||||
string BaseFamilyTest::GetId() const {
|
string BaseFamilyTest::GetId() const {
|
||||||
|
@ -654,7 +654,7 @@ vector<LockFp> BaseFamilyTest::GetLastFps() {
|
||||||
|
|
||||||
lock_guard lk(mu);
|
lock_guard lk(mu);
|
||||||
for (auto fp :
|
for (auto fp :
|
||||||
namespaces.GetDefaultNamespace().GetDbSlice(shard->shard_id()).TEST_GetLastLockedFps()) {
|
namespaces->GetDefaultNamespace().GetDbSlice(shard->shard_id()).TEST_GetLastLockedFps()) {
|
||||||
result.push_back(fp);
|
result.push_back(fp);
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -117,7 +117,7 @@ class BaseFamilyTest : public ::testing::Test {
|
||||||
static std::vector<std::string> StrArray(const RespExpr& expr);
|
static std::vector<std::string> StrArray(const RespExpr& expr);
|
||||||
|
|
||||||
Metrics GetMetrics() const {
|
Metrics GetMetrics() const {
|
||||||
return service_->server_family().GetMetrics(&namespaces.GetDefaultNamespace());
|
return service_->server_family().GetMetrics(&namespaces->GetDefaultNamespace());
|
||||||
}
|
}
|
||||||
|
|
||||||
void ClearMetrics();
|
void ClearMetrics();
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue