Some improvements to snapshotting algo.
Rename rdb_snapshot.h to snapshot.h since it's not related to the underlying serialization format.
This commit is contained in:
parent
3f80b89e19
commit
65f35f2cac
|
@ -10,6 +10,10 @@
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
|
||||||
|
// DASH: Dynamic And Scalable Hashing.
|
||||||
|
// TODO: We could name it DACHE: Dynamic and Adaptive caCHE.
|
||||||
|
// After all, we added additionaly improvements we added as part of the dragonfly project,
|
||||||
|
// that probably justify a right to choose our own name for this data structure.
|
||||||
struct BasicDashPolicy {
|
struct BasicDashPolicy {
|
||||||
enum { kSlotNum = 12, kBucketNum = 64, kStashBucketNum = 2 };
|
enum { kSlotNum = 12, kBucketNum = 64, kStashBucketNum = 2 };
|
||||||
static constexpr bool kUseVersion = false;
|
static constexpr bool kUseVersion = false;
|
||||||
|
|
|
@ -4,7 +4,7 @@ cxx_link(dragonfly base dragonfly_lib)
|
||||||
add_library(dragonfly_lib command_registry.cc common.cc config_flags.cc
|
add_library(dragonfly_lib command_registry.cc common.cc config_flags.cc
|
||||||
conn_context.cc db_slice.cc debugcmd.cc dragonfly_listener.cc
|
conn_context.cc db_slice.cc debugcmd.cc dragonfly_listener.cc
|
||||||
dragonfly_connection.cc engine_shard_set.cc generic_family.cc
|
dragonfly_connection.cc engine_shard_set.cc generic_family.cc
|
||||||
list_family.cc main_service.cc memcache_parser.cc rdb_save.cc rdb_snapshot.cc
|
list_family.cc main_service.cc memcache_parser.cc rdb_save.cc snapshot.cc
|
||||||
redis_parser.cc reply_builder.cc server_family.cc string_family.cc transaction.cc)
|
redis_parser.cc reply_builder.cc server_family.cc string_family.cc transaction.cc)
|
||||||
|
|
||||||
cxx_link(dragonfly_lib dfly_core redis_lib uring_fiber_lib
|
cxx_link(dragonfly_lib dfly_core redis_lib uring_fiber_lib
|
||||||
|
|
|
@ -364,6 +364,7 @@ bool DbSlice::CheckLock(IntentLock::Mode mode, const KeyLockArgs& lock_args) con
|
||||||
|
|
||||||
void DbSlice::PreUpdate(DbIndex db_ind, MainIterator it) {
|
void DbSlice::PreUpdate(DbIndex db_ind, MainIterator it) {
|
||||||
auto& db = db_arr_[db_ind];
|
auto& db = db_arr_[db_ind];
|
||||||
|
|
||||||
db->stats.obj_memory_usage -= it->second.MallocUsed();
|
db->stats.obj_memory_usage -= it->second.MallocUsed();
|
||||||
it.SetVersion(NextVersion());
|
it.SetVersion(NextVersion());
|
||||||
}
|
}
|
||||||
|
|
|
@ -15,7 +15,7 @@ extern "C" {
|
||||||
#include "base/logging.h"
|
#include "base/logging.h"
|
||||||
#include "server/engine_shard_set.h"
|
#include "server/engine_shard_set.h"
|
||||||
#include "server/error.h"
|
#include "server/error.h"
|
||||||
#include "server/rdb_snapshot.h"
|
#include "server/snapshot.h"
|
||||||
#include "util/fibers/simple_channel.h"
|
#include "util/fibers/simple_channel.h"
|
||||||
|
|
||||||
namespace dfly {
|
namespace dfly {
|
||||||
|
@ -213,8 +213,7 @@ error_code RdbSerializer::SaveObject(const robj* o) {
|
||||||
if (quicklistNodeIsCompressed(node)) {
|
if (quicklistNodeIsCompressed(node)) {
|
||||||
void* data;
|
void* data;
|
||||||
size_t compress_len = quicklistGetLzf(node, &data);
|
size_t compress_len = quicklistGetLzf(node, &data);
|
||||||
RETURN_ON_ERR(
|
RETURN_ON_ERR(SaveLzfBlob(Bytes{reinterpret_cast<uint8_t*>(data), compress_len}, node->sz));
|
||||||
SaveLzfBlob(Bytes{reinterpret_cast<uint8_t*>(data), compress_len}, node->sz));
|
|
||||||
} else {
|
} else {
|
||||||
RETURN_ON_ERR(SaveString(node->entry, node->sz));
|
RETURN_ON_ERR(SaveString(node->entry, node->sz));
|
||||||
}
|
}
|
||||||
|
@ -352,12 +351,12 @@ error_code RdbSerializer::SaveLzfBlob(const io::Bytes& src, size_t uncompressed_
|
||||||
|
|
||||||
struct RdbSaver::Impl {
|
struct RdbSaver::Impl {
|
||||||
RdbSerializer serializer;
|
RdbSerializer serializer;
|
||||||
RdbSnapshot::StringChannel channel;
|
SliceSnapshot::StringChannel channel;
|
||||||
vector<unique_ptr<RdbSnapshot>> handles;
|
vector<unique_ptr<SliceSnapshot>> shard_snapshots;
|
||||||
|
|
||||||
// We pass K=sz to say how many producers are pushing data in order to maintain
|
// We pass K=sz to say how many producers are pushing data in order to maintain
|
||||||
// correct closing semantics - channel is closing when K producers marked it as closed.
|
// correct closing semantics - channel is closing when K producers marked it as closed.
|
||||||
Impl(unsigned sz) : channel{128, sz}, handles(sz) {
|
Impl(unsigned sz) : channel{128, sz}, shard_snapshots(sz) {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -407,7 +406,7 @@ error_code RdbSaver::SaveBody() {
|
||||||
vals.clear();
|
vals.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
for (auto& ptr : impl_->handles) {
|
for (auto& ptr : impl_->shard_snapshots) {
|
||||||
ptr->Join();
|
ptr->Join();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -420,10 +419,10 @@ error_code RdbSaver::SaveBody() {
|
||||||
|
|
||||||
void RdbSaver::StartSnapshotInShard(EngineShard* shard) {
|
void RdbSaver::StartSnapshotInShard(EngineShard* shard) {
|
||||||
auto pair = shard->db_slice().GetTables(0);
|
auto pair = shard->db_slice().GetTables(0);
|
||||||
auto s = make_unique<RdbSnapshot>(pair.first, pair.second, &impl_->channel);
|
auto s = make_unique<SliceSnapshot>(pair.first, pair.second, &impl_->channel);
|
||||||
|
|
||||||
s->Start(shard->db_slice().version());
|
s->Start(shard->db_slice().version());
|
||||||
impl_->handles[shard->shard_id()] = move(s);
|
impl_->shard_snapshots[shard->shard_id()] = move(s);
|
||||||
}
|
}
|
||||||
|
|
||||||
error_code RdbSaver::SaveAux() {
|
error_code RdbSaver::SaveAux() {
|
||||||
|
|
|
@ -1,181 +0,0 @@
|
||||||
// Copyright 2022, Roman Gershman. All rights reserved.
|
|
||||||
// See LICENSE for licensing terms.
|
|
||||||
//
|
|
||||||
|
|
||||||
#include "server/rdb_snapshot.h"
|
|
||||||
|
|
||||||
extern "C" {
|
|
||||||
#include "redis/object.h"
|
|
||||||
}
|
|
||||||
|
|
||||||
#include <bitset>
|
|
||||||
|
|
||||||
#include "base/logging.h"
|
|
||||||
#include "server/rdb_save.h"
|
|
||||||
#include "util/fiber_sched_algo.h"
|
|
||||||
|
|
||||||
namespace dfly {
|
|
||||||
using namespace boost;
|
|
||||||
using namespace std;
|
|
||||||
using namespace util;
|
|
||||||
using namespace chrono_literals;
|
|
||||||
|
|
||||||
RdbSnapshot::RdbSnapshot(PrimeTable* prime, ExpireTable* et, StringChannel* dest)
|
|
||||||
: prime_table_(prime), dest_(dest) {
|
|
||||||
}
|
|
||||||
|
|
||||||
RdbSnapshot::~RdbSnapshot() {
|
|
||||||
}
|
|
||||||
|
|
||||||
void RdbSnapshot::Start(uint64_t version) {
|
|
||||||
DCHECK(!fb_.joinable());
|
|
||||||
|
|
||||||
VLOG(1) << "DbSaver::Start";
|
|
||||||
sfile_.reset(new io::StringFile);
|
|
||||||
|
|
||||||
rdb_serializer_.reset(new RdbSerializer(sfile_.get()));
|
|
||||||
snapshot_version_ = version;
|
|
||||||
fb_ = fibers::fiber([this] { FiberFunc(); });
|
|
||||||
}
|
|
||||||
|
|
||||||
void RdbSnapshot::Join() {
|
|
||||||
fb_.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
static_assert(sizeof(PrimeTable::const_iterator) == 16);
|
|
||||||
|
|
||||||
void RdbSnapshot::PhysicalCb(MainIterator it) {
|
|
||||||
error_code ec;
|
|
||||||
|
|
||||||
string tmp;
|
|
||||||
|
|
||||||
auto key = it->first.GetSlice(&tmp);
|
|
||||||
// TODO: fetch expire.
|
|
||||||
if (it->second.ObjType() == OBJ_STRING) {
|
|
||||||
ec = rdb_serializer_->SaveKeyVal(key, it->second.ToString(), 0);
|
|
||||||
} else {
|
|
||||||
robj* obj = it->second.AsRObj();
|
|
||||||
ec = rdb_serializer_->SaveKeyVal(key, obj, 0);
|
|
||||||
}
|
|
||||||
CHECK(!ec); // we write to StringFile.
|
|
||||||
++processed_;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Serializes all the entries with version less than snapshot_version_.
|
|
||||||
void RdbSnapshot::FiberFunc() {
|
|
||||||
this_fiber::properties<FiberProps>().set_name("RdbSnapshot");
|
|
||||||
VLOG(1) << "Saving entries with version less than " << snapshot_version_;
|
|
||||||
|
|
||||||
uint64_t cursor = 0;
|
|
||||||
uint64_t skipped = 0;
|
|
||||||
bitset<128> physical;
|
|
||||||
vector<MainIterator> physical_list;
|
|
||||||
|
|
||||||
static_assert(physical.size() > PrimeTable::kPhysicalBucketNum);
|
|
||||||
|
|
||||||
// The algorithm is to go over all the buckets and serialize entries that
|
|
||||||
// have version < snapshot_version_. In order to serialize each entry exactly once we update its
|
|
||||||
// version to snapshot_version_ once it has been serialized.
|
|
||||||
// Due to how bucket versions work we can not update individual entries - they may affect their
|
|
||||||
// neighbours in the bucket. Instead we handle serialization at physical bucket granularity.
|
|
||||||
// To further complicate things, Table::Traverse covers a logical bucket that may comprise of
|
|
||||||
// several physical buckets. The reason for this complication is that we need to guarantee
|
|
||||||
// a stable traversal during prime table mutations. PrimeTable::Traverse guarantees an atomic
|
|
||||||
// traversal of a single logical bucket, it also guarantees 100% coverage of all items
|
|
||||||
// that existed when the traversal started and survived until it finished.
|
|
||||||
//
|
|
||||||
// It's important that cb will run atomically so we avoid anu I/O work inside it.
|
|
||||||
// Instead, we flush our string file to disk in the traverse loop below.
|
|
||||||
auto save_cb = [&](MainIterator it) {
|
|
||||||
uint64_t v = it.GetVersion();
|
|
||||||
|
|
||||||
if (v >= snapshot_version_) {
|
|
||||||
// either has been already serialized or added after snapshotting started.
|
|
||||||
DVLOG(2) << "Skipped " << it.segment_id() << ":" << it.bucket_id() << ":" << it.slot_id()
|
|
||||||
<< " at " << v;
|
|
||||||
++skipped;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
// if we touched that physical bucket - skip it.
|
|
||||||
// If DashTable interface would introduce TraversePhysicalBuckets - where it
|
|
||||||
// goes over each bucket once - we would not need to check for uniqueness here .
|
|
||||||
// But right now we must to make sure we TraverseBucket exactly once for each physical
|
|
||||||
// bucket.
|
|
||||||
if (physical.test(it.bucket_id())) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
physical.set(it.bucket_id());
|
|
||||||
physical_list.push_back(it);
|
|
||||||
|
|
||||||
// traverse physical bucket and write into string file.
|
|
||||||
// TODO: I think we can avoid using physical_list by calling here
|
|
||||||
// prime_table_->TraverseBucket(it, version_cb);
|
|
||||||
prime_table_->TraverseBucket(it, [this](auto&& it) { this->PhysicalCb(it); });
|
|
||||||
|
|
||||||
return false;
|
|
||||||
};
|
|
||||||
|
|
||||||
auto version_cb = [&](MainIterator it) {
|
|
||||||
DCHECK_LE(it.GetVersion(), snapshot_version_);
|
|
||||||
DVLOG(2) << "Bumping up version " << it.bucket_id() << ":" << it.slot_id();
|
|
||||||
|
|
||||||
it.SetVersion(snapshot_version_);
|
|
||||||
};
|
|
||||||
|
|
||||||
uint64_t last_yield = 0;
|
|
||||||
do {
|
|
||||||
DVLOG(2) << "traverse cusrsor " << cursor;
|
|
||||||
|
|
||||||
// Traverse a single logical bucket but do not update its versions.
|
|
||||||
// we can not update a version because entries in the same bucket share part of the version.
|
|
||||||
// Therefore we save first, and then update version in one atomic swipe.
|
|
||||||
uint64_t next = prime_table_->Traverse(cursor, save_cb);
|
|
||||||
|
|
||||||
// Traverse physical buckets that were touched and update their version.
|
|
||||||
for (auto it : physical_list) {
|
|
||||||
prime_table_->TraverseBucket(it, version_cb);
|
|
||||||
}
|
|
||||||
cursor = next;
|
|
||||||
physical.reset();
|
|
||||||
physical_list.clear();
|
|
||||||
|
|
||||||
// Flush if needed.
|
|
||||||
FlushSfile(false);
|
|
||||||
if (processed_ >= last_yield + 200) {
|
|
||||||
this_fiber::yield();
|
|
||||||
last_yield = processed_;
|
|
||||||
|
|
||||||
// flush in case other fibers (writes commands that pushed previous values) filled the file.
|
|
||||||
FlushSfile(false);
|
|
||||||
}
|
|
||||||
} while (cursor > 0);
|
|
||||||
|
|
||||||
FlushSfile(true);
|
|
||||||
dest_->StartClosing();
|
|
||||||
|
|
||||||
VLOG(1) << "Exit RdbProducer fiber with " << processed_ << " processed";
|
|
||||||
}
|
|
||||||
|
|
||||||
bool RdbSnapshot::FlushSfile(bool force) {
|
|
||||||
if (force) {
|
|
||||||
auto ec = rdb_serializer_->FlushMem();
|
|
||||||
CHECK(!ec);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (sfile_->val.empty() || (!force && sfile_->val.size() < 4096))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (!force) {
|
|
||||||
// Make sure we flush everything from membuffer in order to preserve the atomicity of keyvalue
|
|
||||||
// serializations.
|
|
||||||
auto ec = rdb_serializer_->FlushMem();
|
|
||||||
CHECK(!ec); // stringfile always succeeds.
|
|
||||||
}
|
|
||||||
string tmp = std::move(sfile_->val); // important to move before pushing!
|
|
||||||
dest_->Push(std::move(tmp));
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
} // namespace dfly
|
|
|
@ -0,0 +1,179 @@
|
||||||
|
// Copyright 2022, Roman Gershman. All rights reserved.
|
||||||
|
// See LICENSE for licensing terms.
|
||||||
|
//
|
||||||
|
|
||||||
|
#include "server/snapshot.h"
|
||||||
|
|
||||||
|
extern "C" {
|
||||||
|
#include "redis/object.h"
|
||||||
|
}
|
||||||
|
|
||||||
|
#include <absl/strings/str_cat.h>
|
||||||
|
|
||||||
|
#include "base/logging.h"
|
||||||
|
#include "server/rdb_save.h"
|
||||||
|
#include "util/fiber_sched_algo.h"
|
||||||
|
#include "util/proactor_base.h"
|
||||||
|
|
||||||
|
namespace dfly {
|
||||||
|
|
||||||
|
using namespace std;
|
||||||
|
using namespace util;
|
||||||
|
using namespace chrono_literals;
|
||||||
|
namespace this_fiber = ::boost::this_fiber;
|
||||||
|
using boost::fibers::fiber;
|
||||||
|
|
||||||
|
SliceSnapshot::SliceSnapshot(PrimeTable* prime, ExpireTable* et, StringChannel* dest)
|
||||||
|
: prime_table_(prime), dest_(dest) {
|
||||||
|
}
|
||||||
|
|
||||||
|
SliceSnapshot::~SliceSnapshot() {
|
||||||
|
}
|
||||||
|
|
||||||
|
void SliceSnapshot::Start(uint64_t version) {
|
||||||
|
DCHECK(!fb_.joinable());
|
||||||
|
|
||||||
|
VLOG(1) << "DbSaver::Start - saving entries with version less than " << version;
|
||||||
|
sfile_.reset(new io::StringFile);
|
||||||
|
|
||||||
|
rdb_serializer_.reset(new RdbSerializer(sfile_.get()));
|
||||||
|
snapshot_version_ = version;
|
||||||
|
fb_ = fiber([this] { FiberFunc(); });
|
||||||
|
}
|
||||||
|
|
||||||
|
void SliceSnapshot::Join() {
|
||||||
|
fb_.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
static_assert(sizeof(PrimeTable::const_iterator) == 16);
|
||||||
|
|
||||||
|
void SliceSnapshot::SerializeCb(MainIterator it) {
|
||||||
|
error_code ec;
|
||||||
|
|
||||||
|
string tmp;
|
||||||
|
|
||||||
|
auto key = it->first.GetSlice(&tmp);
|
||||||
|
|
||||||
|
// TODO: fetch expire.
|
||||||
|
if (it->second.ObjType() == OBJ_STRING) {
|
||||||
|
ec = rdb_serializer_->SaveKeyVal(key, it->second.ToString(), 0);
|
||||||
|
} else {
|
||||||
|
robj* obj = it->second.AsRObj();
|
||||||
|
ec = rdb_serializer_->SaveKeyVal(key, obj, 0);
|
||||||
|
}
|
||||||
|
CHECK(!ec); // we write to StringFile.
|
||||||
|
++serialized_;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Serializes all the entries with version less than snapshot_version_.
|
||||||
|
void SliceSnapshot::FiberFunc() {
|
||||||
|
this_fiber::properties<FiberProps>().set_name(
|
||||||
|
absl::StrCat("SliceSnapshot", ProactorBase::GetIndex()));
|
||||||
|
uint64_t cursor = 0;
|
||||||
|
static_assert(PHYSICAL_LEN > PrimeTable::kPhysicalBucketNum);
|
||||||
|
|
||||||
|
uint64_t last_yield = 0;
|
||||||
|
do {
|
||||||
|
// Traverse a single logical bucket but do not update its versions.
|
||||||
|
// we can not update a version because entries in the same bucket share part of the version.
|
||||||
|
// Therefore we save first, and then update version in one atomic swipe.
|
||||||
|
uint64_t next = prime_table_->Traverse(cursor, [this](auto it) { this->SaveCb(move(it)); });
|
||||||
|
|
||||||
|
cursor = next;
|
||||||
|
physical_mask_.reset();
|
||||||
|
|
||||||
|
// Flush if needed.
|
||||||
|
FlushSfile(false);
|
||||||
|
if (serialized_ >= last_yield + 100) {
|
||||||
|
DVLOG(2) << "Before sleep " << this_fiber::properties<FiberProps>().name();
|
||||||
|
this_fiber::yield();
|
||||||
|
last_yield = serialized_;
|
||||||
|
DVLOG(2) << "After sleep";
|
||||||
|
// flush in case other fibers (writes commands that pushed previous values) filled the file.
|
||||||
|
FlushSfile(false);
|
||||||
|
}
|
||||||
|
} while (cursor > 0);
|
||||||
|
|
||||||
|
DVLOG(1) << "after loop " << this_fiber::properties<FiberProps>().name();
|
||||||
|
FlushSfile(true);
|
||||||
|
dest_->StartClosing();
|
||||||
|
|
||||||
|
VLOG(1) << "Exit RdbProducer fiber with " << serialized_ << " serialized";
|
||||||
|
}
|
||||||
|
|
||||||
|
bool SliceSnapshot::FlushSfile(bool force) {
|
||||||
|
if (force) {
|
||||||
|
auto ec = rdb_serializer_->FlushMem();
|
||||||
|
CHECK(!ec);
|
||||||
|
if (sfile_->val.empty())
|
||||||
|
return false;
|
||||||
|
} else {
|
||||||
|
if (sfile_->val.size() < 4096) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Make sure we flush everything from membuffer in order to preserve the atomicity of keyvalue
|
||||||
|
// serializations.
|
||||||
|
auto ec = rdb_serializer_->FlushMem();
|
||||||
|
CHECK(!ec); // stringfile always succeeds.
|
||||||
|
}
|
||||||
|
VLOG(2) << "FlushSfile " << sfile_->val.size() << " bytes";
|
||||||
|
|
||||||
|
string tmp = std::move(sfile_->val); // important to move before pushing!
|
||||||
|
dest_->Push(std::move(tmp));
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
// The algorithm is to go over all the buckets and serialize entries that
|
||||||
|
// have version < snapshot_version_. In order to serialize each entry exactly once we update its
|
||||||
|
// version to snapshot_version_ once it has been serialized.
|
||||||
|
// Due to how bucket versions work we can not update individual entries - they may affect their
|
||||||
|
// neighbours in the bucket. Instead we handle serialization at physical bucket granularity.
|
||||||
|
// To further complicate things, Table::Traverse covers a logical bucket that may comprise of
|
||||||
|
// several physical buckets. The reason for this complication is that we need to guarantee
|
||||||
|
// a stable traversal during prime table mutations. PrimeTable::Traverse guarantees an atomic
|
||||||
|
// traversal of a single logical bucket, it also guarantees 100% coverage of all items
|
||||||
|
// that existed when the traversal started and survived until it finished.
|
||||||
|
//
|
||||||
|
// It's important that cb will run atomically so we avoid anu I/O work inside it.
|
||||||
|
// Instead, we flush our string file to disk in the traverse loop below.
|
||||||
|
bool SliceSnapshot::SaveCb(MainIterator it) {
|
||||||
|
// if we touched that physical bucket - skip it.
|
||||||
|
// We must to make sure we TraverseBucket exactly once for each physical bucket.
|
||||||
|
// This test is the first one because it's likely to be the fastest one:
|
||||||
|
// physical_mask_ is likely to be loaded in L1 and bucket_id() does not require accesing the
|
||||||
|
// prime_table.
|
||||||
|
if (physical_mask_.test(it.bucket_id())) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
uint64_t v = it.GetVersion();
|
||||||
|
if (v >= snapshot_version_) {
|
||||||
|
// either has been already serialized or added after snapshotting started.
|
||||||
|
DVLOG(2) << "Skipped " << it.segment_id() << ":" << it.bucket_id() << ":" << it.slot_id()
|
||||||
|
<< " at " << v;
|
||||||
|
++skipped_;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
physical_mask_.set(it.bucket_id());
|
||||||
|
|
||||||
|
// Both traversals below execute atomically.
|
||||||
|
// traverse physical bucket and write into string file.
|
||||||
|
prime_table_->TraverseBucket(it, [this](auto entry_it) { this->SerializeCb(move(entry_it)); });
|
||||||
|
|
||||||
|
// Theoretically we could merge version_cb into the traversal above but then would would need
|
||||||
|
// to give up on DCHECK.
|
||||||
|
auto version_cb = [this](MainIterator entry_it) {
|
||||||
|
DCHECK_LE(entry_it.GetVersion(), snapshot_version_);
|
||||||
|
DVLOG(3) << "Bumping up version " << entry_it.bucket_id() << ":" << entry_it.slot_id();
|
||||||
|
|
||||||
|
entry_it.SetVersion(snapshot_version_);
|
||||||
|
};
|
||||||
|
|
||||||
|
prime_table_->TraverseBucket(it, version_cb);
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
} // namespace dfly
|
|
@ -4,6 +4,8 @@
|
||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <bitset>
|
||||||
|
|
||||||
#include "io/file.h"
|
#include "io/file.h"
|
||||||
#include "server/table.h"
|
#include "server/table.h"
|
||||||
#include "util/fibers/simple_channel.h"
|
#include "util/fibers/simple_channel.h"
|
||||||
|
@ -12,13 +14,13 @@ namespace dfly {
|
||||||
|
|
||||||
class RdbSerializer;
|
class RdbSerializer;
|
||||||
|
|
||||||
class RdbSnapshot {
|
class SliceSnapshot {
|
||||||
public:
|
public:
|
||||||
using StringChannel =
|
using StringChannel =
|
||||||
::util::fibers_ext::SimpleChannel<std::string, base::mpmc_bounded_queue<std::string>>;
|
::util::fibers_ext::SimpleChannel<std::string, base::mpmc_bounded_queue<std::string>>;
|
||||||
|
|
||||||
RdbSnapshot(PrimeTable* prime, ExpireTable* et, StringChannel* dest);
|
SliceSnapshot(PrimeTable* prime, ExpireTable* et, StringChannel* dest);
|
||||||
~RdbSnapshot();
|
~SliceSnapshot();
|
||||||
|
|
||||||
void Start(uint64_t version);
|
void Start(uint64_t version);
|
||||||
void Join();
|
void Join();
|
||||||
|
@ -30,10 +32,14 @@ class RdbSnapshot {
|
||||||
private:
|
private:
|
||||||
void FiberFunc();
|
void FiberFunc();
|
||||||
bool FlushSfile(bool force);
|
bool FlushSfile(bool force);
|
||||||
void PhysicalCb(MainIterator it);
|
void SerializeCb(MainIterator it);
|
||||||
|
bool SaveCb(MainIterator it);
|
||||||
|
|
||||||
::boost::fibers::fiber fb_;
|
::boost::fibers::fiber fb_;
|
||||||
|
|
||||||
|
enum {PHYSICAL_LEN = 128};
|
||||||
|
std::bitset<PHYSICAL_LEN> physical_mask_;
|
||||||
|
|
||||||
std::unique_ptr<io::StringFile> sfile_;
|
std::unique_ptr<io::StringFile> sfile_;
|
||||||
std::unique_ptr<RdbSerializer> rdb_serializer_;
|
std::unique_ptr<RdbSerializer> rdb_serializer_;
|
||||||
|
|
||||||
|
@ -42,7 +48,7 @@ class RdbSnapshot {
|
||||||
PrimeTable* prime_table_;
|
PrimeTable* prime_table_;
|
||||||
StringChannel* dest_;
|
StringChannel* dest_;
|
||||||
|
|
||||||
uint64_t processed_ = 0;
|
size_t serialized_ = 0, skipped_ = 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace dfly
|
} // namespace dfly
|
Loading…
Reference in New Issue