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:
Roman Gershman 2022-01-24 16:56:25 +02:00
parent 3f80b89e19
commit 65f35f2cac
7 changed files with 204 additions and 196 deletions

View File

@ -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;

View File

@ -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

View File

@ -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());
} }

View File

@ -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() {

View File

@ -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

179
server/snapshot.cc Normal file
View File

@ -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

View File

@ -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