Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions src/server/conn_context.h
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,7 @@ struct ConnectionState {
std::string repl_ip_address;
uint32_t repl_listening_port = 0;
DflyVersion repl_version = DflyVersion::VER1;
bool is_valkey = false;
};

struct SquashingInfo {
Expand Down
182 changes: 182 additions & 0 deletions src/server/dflycmd.cc
Original file line number Diff line number Diff line change
Expand Up @@ -649,6 +649,183 @@ void DflyCmd::Load(CmdArgList args, RedisReplyBuilder* rb, ConnectionContext* cn
rb->SendOk();
}

namespace {

struct ShardJournalChannel : journal::JournalConsumerInterface, io::Source {
void ConsumeJournalChange(const journal::JournalChangeItem& item) override {
// TODO don't block journal slice
ec.await([&] { return wpos < cap; });
buffer[wpos++] = item.journal_item.data;
ec.notifyAll();
parent.notifyAll();
}

io::Result<unsigned long> ReadSome(const iovec* v, uint32_t len) override {
ec.await([&] { return rpos < wpos; });

auto bytes_read = 0;
size_t chunk_size = 0;
bool partial_read = false;
while (rpos < wpos && len > 0) {
auto& item = buffer[rpos];
chunk_size = std::min(v->iov_len, item.size());
std::copy_n(item.data(), chunk_size, static_cast<char*>(v->iov_base));

bytes_read += chunk_size;
++v;
--len;

// If we were not able to fully consume the string, erase the front of it but remain at same
// index
partial_read = chunk_size < item.size();
if (!partial_read) {
++rpos;
} else {
item.erase(0, chunk_size);
}
}

if (rpos == wpos && wpos == cap) {
auto new_size = 0;
if (partial_read) {
buffer[0] = std::move(buffer[rpos]);
new_size = 1;
}

rpos = 0;
wpos = new_size;
buffer.resize(new_size);
}

ec.notifyAll();
return bytes_read;
}

bool HasData() const {
return rpos > 0;
}

fb2::EventCount ec;
size_t rpos{0};
size_t wpos{0};
size_t cap{5};
std::vector<std::string> buffer;
fb2::EventCount& parent;
};

struct Pipe final : io::Source, io::Sink {
io::Result<unsigned long> ReadSome(const iovec* v, uint32_t len) override {
if (done) {
return 0;
}

ec.await([&] { return rpos < wpos; });
auto bytes_read = 0;

while (rpos < wpos && len > 0) {
const auto chunk_size = min(wpos - rpos, v->iov_len);
std::copy_n(buffer.begin() + rpos, chunk_size, static_cast<char*>(v->iov_base));
bytes_read += chunk_size;
rpos += chunk_size;
++v;
--len;
}

if (rpos == wpos && wpos == cap) {
rpos = 0;
wpos = 0;
ec.notifyAll();
}

return bytes_read;
}

io::Result<unsigned long> WriteSome(const iovec* v, uint32_t len) override {
CHECK(!done);
ec.await([&] { return wpos < cap; });
int bytes_written = 0;

while (wpos < cap && len > 0) {
const auto chunk_size = std::min(cap - wpos, v->iov_len);
auto p = static_cast<const char*>(v->iov_base);
std::copy_n(p, chunk_size, buffer.begin() + wpos);
bytes_written += chunk_size;
wpos += chunk_size;
++v;
--len;
}

std::string debugging{reinterpret_cast<const char*>(buffer.data() + 5), wpos};
VLOG(1) << "debugging: " << debugging;
ec.notifyAll();
return bytes_written;
}

std::array<uint8_t, 1024> buffer;
size_t rpos{0};
size_t wpos{0};
size_t cap{1024};
std::atomic_bool done{false};
fb2::EventCount ec;
};

} // namespace

void DflyCmd::StartValkeySync() {
auto Write = [this](auto v) {
const auto buf = io::Bytes(reinterpret_cast<const unsigned char*>(v.data()), v.size());
CHECK(!_valkey_replica->conn->socket()->Write(buf));
};

CHECK(_valkey_replica.has_value()) << "There is no valkey replica to sync with";

// Since we do not know the size of rdb up front, use the EOF protocol, send
// "$EOF:<40-random-chars>\n" first, then the same 40 chars at the end
std::string eof_mark(40, 'X');
std::string eof_mark_with_prefix = absl::StrCat("$EOF:", eof_mark, "\n");

Write(eof_mark_with_prefix);

for (unsigned i = 0; i < shard_set->size(); ++i) {
Pipe p;
auto cb = [&] {
std::array<uint8_t, 128> backing;
const io::MutableBytes mb{backing};
while (!p.done) {
if (auto n = p.Read(mb); !n.has_value() || n.value() == 0) {
break;
}
CHECK(!_valkey_replica->conn->socket()->Write(mb));
}

if (auto n = p.Read(mb); n.has_value() && n.value()) {
CHECK(!_valkey_replica->conn->socket()->Write(mb));
}
};
auto drain_fb = fb2::Fiber("replica-drain-fb", cb);

shard_set->Await(i, [&p, this, i] {
auto shard = EngineShard::tlocal();
auto mode = i == 0 ? SaveMode::SINGLE_SHARD_WITH_SUMMARY : SaveMode::SINGLE_SHARD;
RdbSaver saver{&p, mode, false, ""};
if (mode == SaveMode::SINGLE_SHARD_WITH_SUMMARY) {
CHECK(!saver.SaveHeader(saver.GetGlobalData(&sf_->service())));
}

saver.StartSnapshotInShard(false, &_valkey_replica->exec_st, shard);
CHECK(!saver.WaitSnapshotInShard(shard));
p.done = true;
VLOG(1) << "finished writing snapshot for shard " << shard->shard_id();
});

drain_fb.JoinIfNeeded();
}

Write(eof_mark);

// Stable sync
}

OpStatus DflyCmd::StartFullSyncInThread(FlowInfo* flow, ExecutionState* exec_st,
EngineShard* shard) {
DCHECK(shard);
Expand Down Expand Up @@ -730,6 +907,11 @@ void DflyCmd::StartStableSyncInThread(FlowInfo* flow, ExecutionState* exec_st, E
};
}

void DflyCmd::CreateValkeySyncSession(facade::Connection* conn) {
fb2::LockGuard lk(mu_);
_valkey_replica.emplace(conn, [](const GenericError&) {});
}

auto DflyCmd::CreateSyncSession(ConnectionState* state) -> std::pair<uint32_t, unsigned> {
util::fb2::LockGuard lk(mu_);
unsigned sync_id = next_sync_id_++;
Expand Down
11 changes: 11 additions & 0 deletions src/server/dflycmd.h
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,13 @@ class DflyCmd {
util::fb2::SharedMutex shared_mu; // See top of header for locking levels.
};

struct ValkeyReplica {
ValkeyReplica(facade::Connection* conn, ExecutionState::ErrHandler h) : conn{conn}, exec_st{h} {
}
facade::Connection* conn = nullptr;
ExecutionState exec_st;
};

public:
DflyCmd(ServerFamily* server_family);

Expand All @@ -142,6 +149,7 @@ class DflyCmd {

// Create new sync session. Returns (session_id, number of flows)
std::pair<uint32_t, unsigned> CreateSyncSession(ConnectionState* state) ABSL_LOCKS_EXCLUDED(mu_);
void CreateValkeySyncSession(facade::Connection* conn);

// Master side access method to replication info of that connection.
std::shared_ptr<ReplicaInfo> GetReplicaInfoFromConnection(ConnectionState* state);
Expand All @@ -156,6 +164,7 @@ class DflyCmd {

// Tries to break those flows that stuck on socket write for too long time.
void BreakStalledFlowsInShard() ABSL_NO_THREAD_SAFETY_ANALYSIS;
void StartValkeySync();

private:
using RedisReplyBuilder = facade::RedisReplyBuilder;
Expand Down Expand Up @@ -238,6 +247,8 @@ class DflyCmd {
using ReplicaInfoMap = absl::btree_map<uint32_t, std::shared_ptr<ReplicaInfo>>;
ReplicaInfoMap replica_infos_ ABSL_GUARDED_BY(mu_);

std::optional<ValkeyReplica> _valkey_replica = std::nullopt;

mutable util::fb2::Mutex mu_; // Guard global operations. See header top for locking levels.
};

Expand Down
14 changes: 13 additions & 1 deletion src/server/server_family.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3771,6 +3771,14 @@ void ServerFamily::ReplTakeOver(CmdArgList args, const CommandContext& cmd_cntx)
return builder->SendOk();
}

void ServerFamily::PSync(CmdArgList args, const CommandContext& cmd_cntx) {
auto* rb = static_cast<RedisReplyBuilder*>(cmd_cntx.rb);
auto response = absl::StrFormat("FULLRESYNC %s %ld", master_replid_, 0);
rb->SendSimpleString(response);

dfly_cmd_->StartValkeySync();
}

void ServerFamily::ReplConf(CmdArgList args, const CommandContext& cmd_cntx) {
auto* builder = cmd_cntx.rb;
{
Expand Down Expand Up @@ -3856,6 +3864,9 @@ void ServerFamily::ReplConf(CmdArgList args, const CommandContext& cmd_cntx) {
VLOG(2) << "Received client ACK=" << ack;
cntx->replication_flow->last_acked_lsn = ack;
return;
} else if (cmd == "VERSION" && args.size() == 2) {
cntx->conn_state.replication_info.is_valkey = true;
dfly_cmd_->CreateValkeySyncSession(cntx->conn());
} else {
VLOG(1) << "Error " << cmd << " " << arg << " " << args.size();
return err_cb();
Expand Down Expand Up @@ -4161,7 +4172,8 @@ void ServerFamily::Register(CommandRegistry* registry) {
<< CI{"SLOWLOG", CO::ADMIN | CO::FAST, -2, 0, 0, acl::kSlowLog}.HFUNC(SlowLog)
<< CI{"SCRIPT", CO::NOSCRIPT | CO::NO_KEY_TRANSACTIONAL, -2, 0, 0, acl::kScript}.HFUNC(Script)
<< CI{"DFLY", CO::ADMIN | CO::GLOBAL_TRANS | CO::HIDDEN, -2, 0, 0, acl::kDfly}.HFUNC(Dfly)
<< CI{"MODULE", CO::ADMIN, 2, 0, 0, acl::kModule}.HFUNC(Module);
<< CI{"MODULE", CO::ADMIN, 2, 0, 0, acl::kModule}.HFUNC(Module)
<< CI{"PSYNC", CO::ADMIN | CO::GLOBAL_TRANS, -2, 0, 0, acl::kDfly}.HFUNC(PSync);
}

} // namespace dfly
1 change: 1 addition & 0 deletions src/server/server_family.h
Original file line number Diff line number Diff line change
Expand Up @@ -360,6 +360,7 @@ class ServerFamily {
void Script(CmdArgList args, const CommandContext& cmd_cntx);
void SlowLog(CmdArgList args, const CommandContext& cmd_cntx);
void Module(CmdArgList args, const CommandContext& cmd_cntx);
void PSync(CmdArgList args, const CommandContext& cmd_cntx);

void SyncGeneric(std::string_view repl_master_id, uint64_t offs, ConnectionContext* cntx);

Expand Down
Loading