feat(replica): Support FlushDB command for replication #580 (#591)

This commit is contained in:
adiholden 2022-12-25 14:03:49 +02:00 committed by GitHub
parent 8d86e9b014
commit 5d39521de3
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
10 changed files with 107 additions and 25 deletions

View file

@ -12,7 +12,7 @@ namespace dfly {
JournalExecutor::JournalExecutor(Service* service) : service_{service} { JournalExecutor::JournalExecutor(Service* service) : service_{service} {
} }
void JournalExecutor::Execute(journal::ParsedEntry&& entry) { void JournalExecutor::Execute(journal::ParsedEntry& entry) {
if (entry.payload) { if (entry.payload) {
io::NullSink null_sink; io::NullSink null_sink;
ConnectionContext conn_context{&null_sink, nullptr}; ConnectionContext conn_context{&null_sink, nullptr};

View file

@ -14,7 +14,7 @@ class Service;
class JournalExecutor { class JournalExecutor {
public: public:
JournalExecutor(Service* service); JournalExecutor(Service* service);
void Execute(journal::ParsedEntry&& entry); void Execute(journal::ParsedEntry& entry);
private: private:
Service* service_; Service* service_;

View file

@ -70,6 +70,7 @@ error_code JournalWriter::Write(const journal::Entry& entry) {
return Write(entry.dbid); return Write(entry.dbid);
case journal::Op::COMMAND: case journal::Op::COMMAND:
RETURN_ON_ERR(Write(entry.txid)); RETURN_ON_ERR(Write(entry.txid));
RETURN_ON_ERR(Write(entry.shard_cnt));
return std::visit([this](const auto& payload) { return Write(payload); }, entry.payload); return std::visit([this](const auto& payload) { return Write(payload); }, entry.payload);
default: default:
break; break;
@ -100,6 +101,10 @@ io::Result<uint16_t> JournalReader::ReadU16(io::Source* source) {
return ReadPackedUIntTyped<uint16_t>(source); return ReadPackedUIntTyped<uint16_t>(source);
} }
io::Result<uint32_t> JournalReader::ReadU32(io::Source* source) {
return ReadPackedUIntTyped<uint32_t>(source);
}
io::Result<uint64_t> JournalReader::ReadU64(io::Source* source) { io::Result<uint64_t> JournalReader::ReadU64(io::Source* source) {
return ReadPackedUIntTyped<uint64_t>(source); return ReadPackedUIntTyped<uint64_t>(source);
} }
@ -153,6 +158,7 @@ io::Result<journal::ParsedEntry> JournalReader::ReadEntry(io::Source* source) {
switch (entry.opcode) { switch (entry.opcode) {
case journal::Op::COMMAND: case journal::Op::COMMAND:
SET_OR_UNEXPECT(ReadU64(source), entry.txid); SET_OR_UNEXPECT(ReadU64(source), entry.txid);
SET_OR_UNEXPECT(ReadU32(source), entry.shard_cnt);
entry.payload = CmdArgVec{}; entry.payload = CmdArgVec{};
if (auto ec = Read(source, &*entry.payload); ec) if (auto ec = Read(source, &*entry.payload); ec)
return make_unexpected(ec); return make_unexpected(ec);

View file

@ -55,6 +55,7 @@ struct JournalReader {
// TODO: Templated endian encoding to not repeat...? // TODO: Templated endian encoding to not repeat...?
io::Result<uint8_t> ReadU8(io::Source* source); io::Result<uint8_t> ReadU8(io::Source* source);
io::Result<uint16_t> ReadU16(io::Source* source); io::Result<uint16_t> ReadU16(io::Source* source);
io::Result<uint32_t> ReadU32(io::Source* source);
io::Result<uint64_t> ReadU64(io::Source* source); io::Result<uint64_t> ReadU64(io::Source* source);
// Read string into internal buffer and return size. // Read string into internal buffer and return size.

View file

@ -22,6 +22,7 @@ struct EntryBase {
TxId txid; TxId txid;
Op opcode; Op opcode;
DbIndex dbid; DbIndex dbid;
uint32_t shard_cnt;
}; };
// This struct represents a single journal entry. // This struct represents a single journal entry.
@ -34,11 +35,11 @@ struct Entry : public EntryBase {
std::pair<std::string_view, ArgSlice> // Command and its shard parts. std::pair<std::string_view, ArgSlice> // Command and its shard parts.
>; >;
Entry(TxId txid, DbIndex dbid, Payload pl) Entry(TxId txid, DbIndex dbid, Payload pl, uint32_t shard_cnt)
: EntryBase{txid, journal::Op::COMMAND, dbid}, payload{pl} { : EntryBase{txid, journal::Op::COMMAND, dbid, shard_cnt}, payload{pl} {
} }
Entry(journal::Op opcode, DbIndex dbid) : EntryBase{0, opcode, dbid}, payload{} { Entry(journal::Op opcode, DbIndex dbid) : EntryBase{0, opcode, dbid, 0}, payload{} {
} }
Payload payload; Payload payload;
@ -50,11 +51,11 @@ struct ParsedEntry : public EntryBase {
ParsedEntry() = default; ParsedEntry() = default;
ParsedEntry(journal::Op opcode, DbIndex dbid) : EntryBase{0, opcode, dbid}, payload{} { ParsedEntry(journal::Op opcode, DbIndex dbid) : EntryBase{0, opcode, dbid, 0}, payload{} {
} }
ParsedEntry(TxId txid, DbIndex dbid, Payload pl) ParsedEntry(TxId txid, DbIndex dbid, Payload pl, uint32_t shard_cnt)
: EntryBase{txid, journal::Op::COMMAND, dbid}, payload{pl} { : EntryBase{txid, journal::Op::COMMAND, dbid, shard_cnt}, payload{pl} {
} }
Payload payload; Payload payload;

View file

@ -95,13 +95,14 @@ TEST(Journal, WriteRead) {
auto slice = [v = &slices](auto... ss) { return StoreSlice(v, ss...); }; auto slice = [v = &slices](auto... ss) { return StoreSlice(v, ss...); };
auto list = [v = &lists](auto... ss) { return StoreList(v, ss...); }; auto list = [v = &lists](auto... ss) { return StoreList(v, ss...); };
std::vector<journal::Entry> test_entries = {{0, 0, make_pair("MSET", slice("A", "1", "B", "2"))}, std::vector<journal::Entry> test_entries = {
{1, 0, make_pair("MSET", slice("C", "3"))}, {0, 0, make_pair("MSET", slice("A", "1", "B", "2")), 2},
{2, 0, list("DEL", "A", "B")}, {0, 0, make_pair("MSET", slice("C", "3")), 2},
{3, 1, list("LPUSH", "l", "v1", "v2")}, {1, 0, list("DEL", "A", "B"), 2},
{4, 0, make_pair("MSET", slice("D", "4"))}, {2, 1, list("LPUSH", "l", "v1", "v2"), 1},
{5, 1, list("DEL", "l1")}, {3, 0, make_pair("MSET", slice("D", "4")), 1},
{6, 2, list("SET", "E", "2")}}; {4, 1, list("DEL", "l1"), 1},
{5, 2, list("SET", "E", "2"), 1}};
// Write all entries to string file. // Write all entries to string file.
io::StringSink ss; io::StringSink ss;

View file

@ -1970,7 +1970,7 @@ error_code RdbLoaderBase::HandleJournalBlob(Service* service, DbIndex dbid) {
while (done < num_entries) { while (done < num_entries) {
journal::ParsedEntry entry{}; journal::ParsedEntry entry{};
SET_OR_RETURN(journal_reader_.ReadEntry(&bs), entry); SET_OR_RETURN(journal_reader_.ReadEntry(&bs), entry);
ex.Execute(std::move(entry)); ex.Execute(entry);
done++; done++;
} }

View file

@ -97,9 +97,11 @@ Replica::Replica(string host, uint16_t port, Service* se) : service_(*se) {
master_context_.port = port; master_context_.port = port;
} }
Replica::Replica(const MasterContext& context, uint32_t dfly_flow_id, Service* service) Replica::Replica(const MasterContext& context, uint32_t dfly_flow_id, Service* service,
std::shared_ptr<Replica::MultiShardExecution> shared_exe_data)
: service_(*service), master_context_(context) { : service_(*service), master_context_(context) {
master_context_.dfly_flow_id = dfly_flow_id; master_context_.dfly_flow_id = dfly_flow_id;
multi_shard_exe_ = shared_exe_data;
} }
Replica::~Replica() { Replica::~Replica() {
@ -427,13 +429,13 @@ error_code Replica::InitiatePSync() {
// Initialize and start sub-replica for each flow. // Initialize and start sub-replica for each flow.
error_code Replica::InitiateDflySync() { error_code Replica::InitiateDflySync() {
DCHECK_GT(num_df_flows_, 0u); DCHECK_GT(num_df_flows_, 0u);
multi_shard_exe_.reset(new MultiShardExecution());
shard_flows_.resize(num_df_flows_); shard_flows_.resize(num_df_flows_);
for (unsigned i = 0; i < num_df_flows_; ++i) { for (unsigned i = 0; i < num_df_flows_; ++i) {
shard_flows_[i].reset(new Replica(master_context_, i, &service_)); shard_flows_[i].reset(new Replica(master_context_, i, &service_, multi_shard_exe_));
} }
// Blocked on untill all flows got full sync cut. // Blocked on until all flows got full sync cut.
fibers_ext::BlockingCounter sync_block{num_df_flows_}; fibers_ext::BlockingCounter sync_block{num_df_flows_};
auto err_handler = [this, sync_block](const auto& ge) mutable { auto err_handler = [this, sync_block](const auto& ge) mutable {
@ -705,14 +707,63 @@ void Replica::StableSyncDflyFb(Context* cntx) {
cntx->Error(res.error(), "Journal format error"); cntx->Error(res.error(), "Journal format error");
return; return;
} }
ExecuteEntry(&executor, res.value());
executor.Execute(std::move(res.value()));
last_io_time_ = sock_->proactor()->GetMonotonicTimeNs(); last_io_time_ = sock_->proactor()->GetMonotonicTimeNs();
} }
return; return;
} }
void Replica::ExecuteEntry(JournalExecutor* executor, journal::ParsedEntry& entry) {
if (entry.shard_cnt <= 1) { // not multi shard cmd
executor->Execute(entry);
return;
}
// Multi shard command flow:
// step 1: Fiber wait until all the fibers that should execute this tranaction got
// to the journal entry of the transaction.
// step 2: execute the command (All fibers)
// step 3: Fiber wait until all fibers finished the execution
// By step 1 we enforce that replica will execute multi shard commands that finished on master
// By step 3 we ensures the correctness of flushall/flushdb commands
// TODO: this implemantaion does not support atomicity in replica
// Although multi shard transaction happen in step 2 very close to each other,
// user can query replica between executions.
// To support atomicity we should have one fiber in step 2 which will excute all the entries of
// the transaction together. In case of global comand such as flushdb the command can be executed
// by only one fiber.
// TODO: support error handler in this flow
// Only the first fiber to reach the transaction will create data for transaction in map
multi_shard_exe_->map_mu.lock();
auto [it, was_insert] = multi_shard_exe_->tx_sync_execution.emplace(entry.txid, entry.shard_cnt);
// Note: we must release the mutex befor calling wait on barrier
multi_shard_exe_->map_mu.unlock();
VLOG(2) << "txid: " << entry.txid << " unique_shard_cnt_: " << entry.shard_cnt
<< " was_insert: " << was_insert;
// step 1
it->second.barrier.wait();
// step 2
executor->Execute(entry);
// step 3
it->second.barrier.wait();
// Note: erase from map can be done only after all fibers returned from wait.
// The last fiber which will decrease the counter to 0 will be the one to erase the data from map
auto val = it->second.counter.fetch_sub(1, std::memory_order_relaxed);
VLOG(2) << "txid: " << entry.txid << " unique_shard_cnt_: " << entry.shard_cnt
<< " counter: " << val;
if (val == 1) {
std::lock_guard lg{multi_shard_exe_->map_mu};
multi_shard_exe_->tx_sync_execution.erase(entry.txid);
}
}
error_code Replica::ReadRespReply(base::IoBuf* io_buf, uint32_t* consumed) { error_code Replica::ReadRespReply(base::IoBuf* io_buf, uint32_t* consumed) {
DCHECK(parser_); DCHECK(parser_);

View file

@ -3,13 +3,16 @@
// //
#pragma once #pragma once
#include <boost/fiber/barrier.hpp>
#include <boost/fiber/fiber.hpp> #include <boost/fiber/fiber.hpp>
#include <boost/fiber/mutex.hpp>
#include <variant> #include <variant>
#include "base/io_buf.h" #include "base/io_buf.h"
#include "facade/facade_types.h" #include "facade/facade_types.h"
#include "facade/redis_parser.h" #include "facade/redis_parser.h"
#include "server/common.h" #include "server/common.h"
#include "server/journal/types.h"
#include "util/fiber_socket_base.h" #include "util/fiber_socket_base.h"
#include "util/fibers/fibers_ext.h" #include "util/fibers/fibers_ext.h"
@ -21,6 +24,7 @@ namespace dfly {
class Service; class Service;
class ConnectionContext; class ConnectionContext;
class JournalExecutor;
class Replica { class Replica {
private: private:
@ -46,6 +50,19 @@ class Replica {
R_SYNC_OK = 0x10, R_SYNC_OK = 0x10,
}; };
struct MultiShardExecution {
boost::fibers::mutex map_mu;
struct TxExecutionSync {
boost::fibers::barrier barrier;
std::atomic_uint32_t counter;
TxExecutionSync(uint32_t counter) : barrier(counter), counter(counter) {
}
};
std::unordered_map<TxId, TxExecutionSync> tx_sync_execution;
};
public: public:
Replica(std::string master_host, uint16_t port, Service* se); Replica(std::string master_host, uint16_t port, Service* se);
~Replica(); ~Replica();
@ -81,7 +98,8 @@ class Replica {
private: /* Main dlfly flow mode functions */ private: /* Main dlfly flow mode functions */
// Initialize as single dfly flow. // Initialize as single dfly flow.
Replica(const MasterContext& context, uint32_t dfly_flow_id, Service* service); Replica(const MasterContext& context, uint32_t dfly_flow_id, Service* service,
std::shared_ptr<MultiShardExecution> shared_exe_data);
// Start replica initialized as dfly flow. // Start replica initialized as dfly flow.
std::error_code StartFullSyncFlow(util::fibers_ext::BlockingCounter block, Context* cntx); std::error_code StartFullSyncFlow(util::fibers_ext::BlockingCounter block, Context* cntx);
@ -122,6 +140,8 @@ class Replica {
// Send command, update last_io_time, return error. // Send command, update last_io_time, return error.
std::error_code SendCommand(std::string_view command, facade::ReqSerializer* serializer); std::error_code SendCommand(std::string_view command, facade::ReqSerializer* serializer);
void ExecuteEntry(JournalExecutor* executor, journal::ParsedEntry& entry);
public: /* Utility */ public: /* Utility */
struct Info { struct Info {
std::string host; std::string host;
@ -154,6 +174,8 @@ class Replica {
MasterContext master_context_; MasterContext master_context_;
std::unique_ptr<util::LinuxSocketBase> sock_; std::unique_ptr<util::LinuxSocketBase> sock_;
std::shared_ptr<MultiShardExecution> multi_shard_exe_;
// MainReplicationFb in standalone mode, FullSyncDflyFb in flow mode. // MainReplicationFb in standalone mode, FullSyncDflyFb in flow mode.
::boost::fibers::fiber sync_fb_; ::boost::fibers::fiber sync_fb_;
std::vector<std::unique_ptr<Replica>> shard_flows_; std::vector<std::unique_ptr<Replica>> shard_flows_;

View file

@ -1221,7 +1221,7 @@ void Transaction::LogJournalOnShard(EngineShard* shard) {
entry_payload = entry_payload =
make_pair(facade::ToSV(cmd_with_full_args_.front()), ShardArgsInShard(shard->shard_id())); make_pair(facade::ToSV(cmd_with_full_args_.front()), ShardArgsInShard(shard->shard_id()));
} }
journal->RecordEntry(journal::Entry{txid_, db_index_, entry_payload}); journal->RecordEntry(journal::Entry{txid_, db_index_, entry_payload, unique_shard_cnt_});
} }
void Transaction::BreakOnShutdown() { void Transaction::BreakOnShutdown() {