Skip to content

Commit

Permalink
feat(server): Use new journal format (#563)
Browse files Browse the repository at this point in the history
  • Loading branch information
dranikpg authored Dec 20, 2022
1 parent cdc31fc commit 2386b02
Show file tree
Hide file tree
Showing 30 changed files with 302 additions and 179 deletions.
3 changes: 2 additions & 1 deletion src/facade/conn_context.h
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,8 @@ class ConnectionContext {
bool req_auth : 1;
bool replica_conn : 1;
bool authenticated : 1;
bool force_dispatch : 1; // whether we should route all requests to the dispatch fiber.
bool force_dispatch : 1; // whether we should route all requests to the dispatch fiber.
bool journal_emulated : 1; // whether it is used to dispatch journal commands.

private:
Connection* owner_;
Expand Down
1 change: 1 addition & 0 deletions src/facade/facade.cc
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ ConnectionContext::ConnectionContext(::io::Sink* stream, Connection* owner) : ow
replica_conn = false;
authenticated = false;
force_dispatch = false;
journal_emulated = false;
}

RedisReplyBuilder* ConnectionContext::operator->() {
Expand Down
2 changes: 1 addition & 1 deletion src/server/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ add_library(dragonfly_lib channel_slice.cc command_registry.cc
snapshot.cc script_mgr.cc server_family.cc malloc_stats.cc
set_family.cc stream_family.cc string_family.cc
zset_family.cc version.cc bitops_family.cc container_utils.cc
serializer_commons.cc journal/serializer.cc)
serializer_commons.cc journal/serializer.cc journal/executor.cc)

cxx_link(dragonfly_lib dfly_transaction dfly_facade redis_lib strings_lib html_lib
absl::random_random TRDP::jsoncons zstd TRDP::lz4)
Expand Down
5 changes: 4 additions & 1 deletion src/server/common.h
Original file line number Diff line number Diff line change
Expand Up @@ -268,7 +268,10 @@ class Context : protected Cancellation {
//
// Note: this function blocks when called from inside an error handler.
template <typename... T> GenericError Error(T... ts) {
std::lock_guard lk{mu_};
if (!mu_.try_lock()) // TODO: Maybe use two separate locks.
return GenericError{std::forward<T>(ts)...};

std::lock_guard lk{mu_, std::adopt_lock};
if (err_)
return err_;

Expand Down
13 changes: 7 additions & 6 deletions src/server/dflycmd.cc
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
#include "server/engine_shard_set.h"
#include "server/error.h"
#include "server/journal/journal.h"
#include "server/journal/serializer.h"
#include "server/rdb_save.h"
#include "server/script_mgr.h"
#include "server/server_family.h"
Expand Down Expand Up @@ -353,16 +354,16 @@ OpStatus DflyCmd::StartStableSyncInThread(FlowInfo* flow, EngineShard* shard) {
// Register journal listener and cleanup.
uint32_t cb_id = 0;
if (shard != nullptr) {
cb_id = sf_->journal()->RegisterOnChange([flow](const journal::Entry& je) {
// TODO: Serialize event.
ReqSerializer serializer{flow->conn->socket()};
serializer.SendCommand(absl::StrCat("SET ", je.key, " ", je.pval_ptr->ToString()));
});
JournalWriter writer{flow->conn->socket()};
auto journal_cb = [flow, writer = std::move(writer)](const journal::Entry& je) mutable {
writer.Write(je);
};
cb_id = sf_->journal()->RegisterOnChange(std::move(journal_cb));
}

flow->cleanup = [flow, this, cb_id]() {
if (cb_id)
sf_->journal()->Unregister(cb_id);
sf_->journal()->UnregisterOnChange(cb_id);
flow->TryShutdownSocket();
};

Expand Down
29 changes: 29 additions & 0 deletions src/server/journal/executor.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
// Copyright 2022, DragonflyDB authors. All rights reserved.
// See LICENSE for licensing terms.
//

#include "server/journal/executor.h"

#include "base/logging.h"
#include "server/main_service.h"

namespace dfly {

JournalExecutor::JournalExecutor(Service* service) : service_{service} {
}

void JournalExecutor::Execute(journal::ParsedEntry&& entry) {
if (entry.payload) {
io::NullSink null_sink;
ConnectionContext conn_context{&null_sink, nullptr};
conn_context.is_replicating = true;
conn_context.journal_emulated = true;
conn_context.conn_state.db_index = entry.dbid;

auto span = CmdArgList{entry.payload->data(), entry.payload->size()};

service_->DispatchCommand(span, &conn_context);
}
}

} // namespace dfly
23 changes: 23 additions & 0 deletions src/server/journal/executor.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
// Copyright 2022, DragonflyDB authors. All rights reserved.
// See LICENSE for licensing terms.
//

#pragma once

#include "server/journal/types.h"

namespace dfly {

class Service;

// JournalExecutor allows executing journal entries.
class JournalExecutor {
public:
JournalExecutor(Service* service);
void Execute(journal::ParsedEntry&& entry);

private:
Service* service_;
};

} // namespace dfly
18 changes: 9 additions & 9 deletions src/server/journal/journal.cc
Original file line number Diff line number Diff line change
Expand Up @@ -41,11 +41,11 @@ error_code Journal::OpenInThread(bool persistent, string_view dir) {
}
}

ServerState::tlocal()->set_journal(this);
EngineShard* shard = EngineShard::tlocal();
if (shard) {
shard->set_journal(this);
}
ServerState::tlocal()->set_journal(this);
EngineShard* shard = EngineShard::tlocal();
if (shard) {
shard->set_journal(this);
}

return ec;
}
Expand Down Expand Up @@ -83,16 +83,16 @@ uint32_t Journal::RegisterOnChange(ChangeCallback cb) {
return journal_slice.RegisterOnChange(cb);
}

void Journal::Unregister(uint32_t id) {
journal_slice.Unregister(id);
void Journal::UnregisterOnChange(uint32_t id) {
journal_slice.UnregisterOnChange(id);
}

bool Journal::SchedStartTx(TxId txid, unsigned num_keys, unsigned num_shards) {
if (!journal_slice.IsOpen() || lameduck_.load(memory_order_relaxed))
return false;

// TODO: to complete the metadata.
journal_slice.AddLogRecord(Entry::Sched(txid));
// TODO: Handle tx entries.
// journal_slice.AddLogRecord(Entry::Sched(txid));

return true;
}
Expand Down
5 changes: 1 addition & 4 deletions src/server/journal/journal.h
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ class Transaction;

namespace journal {


class Journal {
public:
using Span = absl::Span<const std::string_view>;
Expand All @@ -32,9 +31,8 @@ class Journal {

//******* The following functions must be called in the context of the owning shard *********//


uint32_t RegisterOnChange(ChangeCallback cb);
void Unregister(uint32_t id);
void UnregisterOnChange(uint32_t id);

// Returns true if transaction was scheduled, false if journal is inactive
// or in lameduck mode and does not log new transactions.
Expand All @@ -58,7 +56,6 @@ class Journal {
void RecordEntry(const Entry& entry);

private:

mutable boost::fibers::mutex state_mu_;

std::atomic_bool lameduck_{false};
Expand Down
2 changes: 1 addition & 1 deletion src/server/journal/journal_slice.cc
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ uint32_t JournalSlice::RegisterOnChange(ChangeCallback cb) {
return id;
}

void JournalSlice::Unregister(uint32_t id) {
void JournalSlice::UnregisterOnChange(uint32_t id) {
CHECK(!iterating_cb_arr_);

auto it = find_if(change_cb_arr_.begin(), change_cb_arr_.end(),
Expand Down
2 changes: 1 addition & 1 deletion src/server/journal/journal_slice.h
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ class JournalSlice {
void AddLogRecord(const Entry& entry);

uint32_t RegisterOnChange(ChangeCallback cb);
void Unregister(uint32_t);
void UnregisterOnChange(uint32_t);

private:
struct RingItem;
Expand Down
51 changes: 27 additions & 24 deletions src/server/journal/serializer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -56,10 +56,10 @@ error_code JournalWriter::Write(std::monostate) {
return std::error_code{};
}

error_code JournalWriter::Write(const journal::EntryNew& entry) {
error_code JournalWriter::Write(const journal::Entry& entry) {
// Check if entry has a new db index and we need to emit a SELECT entry.
if (entry.opcode != journal::Op::SELECT && (!cur_dbid_ || entry.dbid != *cur_dbid_)) {
RETURN_ON_ERR(Write(journal::EntryNew{journal::Op::SELECT, entry.dbid}));
RETURN_ON_ERR(Write(journal::Entry{journal::Op::SELECT, entry.dbid}));
cur_dbid_ = entry.dbid;
}

Expand All @@ -68,7 +68,7 @@ error_code JournalWriter::Write(const journal::EntryNew& entry) {
switch (entry.opcode) {
case journal::Op::SELECT:
return Write(entry.dbid);
case journal::Op::VAL:
case journal::Op::COMMAND:
RETURN_ON_ERR(Write(entry.txid));
return std::visit([this](const auto& payload) { return Write(payload); }, entry.payload);
default:
Expand All @@ -77,8 +77,11 @@ error_code JournalWriter::Write(const journal::EntryNew& entry) {
return std::error_code{};
}

JournalReader::JournalReader(io::Source* source, DbIndex dbid)
: source_{source}, buf_{}, dbid_{dbid} {
JournalReader::JournalReader(DbIndex dbid) : buf_{}, dbid_{dbid} {
}

void JournalReader::SetDb(DbIndex dbid) {
dbid_ = dbid;
}

template <typename UT> io::Result<UT> ReadPackedUIntTyped(io::Source* source) {
Expand All @@ -89,26 +92,26 @@ template <typename UT> io::Result<UT> ReadPackedUIntTyped(io::Source* source) {
return static_cast<UT>(v);
}

io::Result<uint8_t> JournalReader::ReadU8() {
return ReadPackedUIntTyped<uint8_t>(source_);
io::Result<uint8_t> JournalReader::ReadU8(io::Source* source) {
return ReadPackedUIntTyped<uint8_t>(source);
}

io::Result<uint16_t> JournalReader::ReadU16() {
return ReadPackedUIntTyped<uint16_t>(source_);
io::Result<uint16_t> JournalReader::ReadU16(io::Source* source) {
return ReadPackedUIntTyped<uint16_t>(source);
}

io::Result<uint64_t> JournalReader::ReadU64() {
return ReadPackedUIntTyped<uint64_t>(source_);
io::Result<uint64_t> JournalReader::ReadU64(io::Source* source) {
return ReadPackedUIntTyped<uint64_t>(source);
}

io::Result<size_t> JournalReader::ReadString() {
io::Result<size_t> JournalReader::ReadString(io::Source* source) {
size_t size = 0;
SET_OR_UNEXPECT(ReadU64(), size);
SET_OR_UNEXPECT(ReadU64(source), size);

buf_.EnsureCapacity(size);
auto dest = buf_.AppendBuffer().first(size);
uint64_t read = 0;
SET_OR_UNEXPECT(source_->Read(dest), read);
SET_OR_UNEXPECT(source->Read(dest), read);

buf_.CommitWrite(read);
if (read != size)
Expand All @@ -117,16 +120,16 @@ io::Result<size_t> JournalReader::ReadString() {
return size;
}

std::error_code JournalReader::Read(CmdArgVec* vec) {
std::error_code JournalReader::Read(io::Source* source, CmdArgVec* vec) {
buf_.ConsumeInput(buf_.InputBuffer().size());

size_t size = 0;
SET_OR_RETURN(ReadU64(), size);
SET_OR_RETURN(ReadU64(source), size);

vec->resize(size);
for (auto& span : *vec) {
size_t len;
SET_OR_RETURN(ReadString(), len);
SET_OR_RETURN(ReadString(source), len);
span = MutableSlice{nullptr, len};
}

Expand All @@ -141,22 +144,22 @@ std::error_code JournalReader::Read(CmdArgVec* vec) {
return std::error_code{};
}

io::Result<journal::ParsedEntry> JournalReader::ReadEntry() {
io::Result<journal::ParsedEntry> JournalReader::ReadEntry(io::Source* source) {
uint8_t opcode;
SET_OR_UNEXPECT(ReadU8(), opcode);
SET_OR_UNEXPECT(ReadU8(source), opcode);

journal::ParsedEntry entry{static_cast<journal::Op>(opcode), dbid_};

switch (entry.opcode) {
case journal::Op::VAL:
SET_OR_UNEXPECT(ReadU64(), entry.txid);
case journal::Op::COMMAND:
SET_OR_UNEXPECT(ReadU64(source), entry.txid);
entry.payload = CmdArgVec{};
if (auto ec = Read(&*entry.payload); ec)
if (auto ec = Read(source, &*entry.payload); ec)
return make_unexpected(ec);
break;
case journal::Op::SELECT:
SET_OR_UNEXPECT(ReadU16(), dbid_);
return ReadEntry();
SET_OR_UNEXPECT(ReadU16(source), dbid_);
return ReadEntry(source);
default:
break;
};
Expand Down
22 changes: 12 additions & 10 deletions src/server/journal/serializer.h
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ class JournalWriter {
JournalWriter(io::Sink* sink, std::optional<DbIndex> dbid = std::nullopt);

// Write single entry.
std::error_code Write(const journal::EntryNew& entry);
std::error_code Write(const journal::Entry& entry);

private:
std::error_code Write(uint64_t v); // Write packed unsigned integer.
Expand All @@ -42,27 +42,29 @@ class JournalWriter {
// Like the writer, it automatically keeps track of the database index.
struct JournalReader {
public:
// Initialize with source and start database index.
JournalReader(io::Source* source, DbIndex dbid);
// Initialize start database index.
JournalReader(DbIndex dbid);

// Overwrite current db index.
void SetDb(DbIndex dbid);

// Try reading entry from source.
io::Result<journal::ParsedEntry> ReadEntry();
io::Result<journal::ParsedEntry> ReadEntry(io::Source* source);

private:
// TODO: Templated endian encoding to not repeat...?
io::Result<uint8_t> ReadU8();
io::Result<uint16_t> ReadU16();
io::Result<uint64_t> ReadU64();
io::Result<uint8_t> ReadU8(io::Source* source);
io::Result<uint16_t> ReadU16(io::Source* source);
io::Result<uint64_t> ReadU64(io::Source* source);

// Read string into internal buffer and return size.
io::Result<size_t> ReadString();
io::Result<size_t> ReadString(io::Source* source);

// Read argument array into internal buffer and build slice.
// TODO: Inline store span data inside buffer to avoid alloaction
std::error_code Read(CmdArgVec* vec);
std::error_code Read(io::Source* source, CmdArgVec* vec);

private:
io::Source* source_;
base::IoBuf buf_;
DbIndex dbid_;
};
Expand Down
Loading

0 comments on commit 2386b02

Please sign in to comment.