Refactor replication, part 3 (#1177)

Changes to make replication code agnostic of the storage kind being used.

Co-authored-by: Andreja Tonev <andreja.tonev@memgraph.io>
This commit is contained in:
Gareth Andrew Lloyd 2023-08-25 10:52:07 +01:00 committed by GitHub
parent a426ef9cc3
commit 4bc5d749b2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 2116 additions and 1837 deletions

View File

@ -58,11 +58,11 @@ class Server final {
* Constructs and binds server to endpoint, operates on session data and
* invokes workers_count workers
*/
Server(const io::network::Endpoint &endpoint, TSessionContext *session_context, ServerContext *context,
Server(io::network::Endpoint endpoint, TSessionContext *session_context, ServerContext *context,
int inactivity_timeout_sec, const std::string &service_name,
size_t workers_count = std::thread::hardware_concurrency())
: alive_(false),
endpoint_(endpoint),
endpoint_(std::move(endpoint)),
listener_(session_context, context, inactivity_timeout_sec, service_name, workers_count),
service_name_(service_name) {}

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -91,7 +91,6 @@ std::string Endpoint::SocketAddress() const {
return ip_address + ":" + std::to_string(port);
}
Endpoint::Endpoint() {}
Endpoint::Endpoint(std::string ip_address, uint16_t port) : address(std::move(ip_address)), port(port) {
IpFamily ip_family = GetIpFamily(address);
if (ip_family == IpFamily::NONE) {

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -25,8 +25,13 @@ namespace memgraph::io::network {
* connection address.
*/
struct Endpoint {
Endpoint();
Endpoint() = default;
Endpoint(std::string ip_address, uint16_t port);
Endpoint(Endpoint const &) = default;
Endpoint(Endpoint &&) noexcept = default;
Endpoint &operator=(Endpoint const &) = default;
Endpoint &operator=(Endpoint &&) noexcept = default;
~Endpoint() = default;
enum class IpFamily : std::uint8_t { NONE, IP4, IP6 };

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -13,8 +13,8 @@
namespace memgraph::rpc {
Client::Client(const io::network::Endpoint &endpoint, communication::ClientContext *context)
: endpoint_(endpoint), context_(context) {}
Client::Client(io::network::Endpoint endpoint, communication::ClientContext *context)
: endpoint_(std::move(endpoint)), context_(context) {}
void Client::Abort() {
if (!client_) return;

View File

@ -30,7 +30,7 @@ namespace memgraph::rpc {
/// Client is thread safe, but it is recommended to use thread_local clients.
class Client {
public:
Client(const io::network::Endpoint &endpoint, communication::ClientContext *context);
Client(io::network::Endpoint endpoint, communication::ClientContext *context);
/// Object used to handle streaming of request data to the RPC server.
template <class TRequestResponse>
@ -194,7 +194,7 @@ class Client {
/// Call this function from another thread to abort a pending RPC call.
void Abort();
const auto &Endpoint() const { return endpoint_; }
auto Endpoint() const -> io::network::Endpoint const & { return endpoint_; }
private:
io::network::Endpoint endpoint_;

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -13,8 +13,8 @@
namespace memgraph::rpc {
Server::Server(const io::network::Endpoint &endpoint, communication::ServerContext *context, size_t workers_count)
: server_(endpoint, this, context, -1, context->use_ssl() ? "RPCS" : "RPC", workers_count) {}
Server::Server(io::network::Endpoint endpoint, communication::ServerContext *context, size_t workers_count)
: server_(std::move(endpoint), this, context, -1, context->use_ssl() ? "RPCS" : "RPC", workers_count) {}
bool Server::Start() { return server_.Start(); }
@ -22,5 +22,7 @@ void Server::Shutdown() { server_.Shutdown(); }
void Server::AwaitShutdown() { server_.AwaitShutdown(); }
bool Server::IsRunning() { return server_.IsRunning(); }
const io::network::Endpoint &Server::endpoint() const { return server_.endpoint(); }
} // namespace memgraph::rpc

View File

@ -25,7 +25,7 @@ namespace memgraph::rpc {
class Server {
public:
Server(const io::network::Endpoint &endpoint, communication::ServerContext *context,
Server(io::network::Endpoint endpoint, communication::ServerContext *context,
size_t workers_count = std::thread::hardware_concurrency());
Server(const Server &) = delete;
Server(Server &&) = delete;
@ -35,6 +35,7 @@ class Server {
bool Start();
void Shutdown();
void AwaitShutdown();
bool IsRunning();
const io::network::Endpoint &endpoint() const;

View File

@ -1,51 +1,47 @@
set(storage_v2_src_files
commit_log.cpp
constraints/existence_constraints.cpp
temporal.cpp
durability/durability.cpp
durability/serialization.cpp
durability/snapshot.cpp
durability/wal.cpp
edge_accessor.cpp
property_store.cpp
vertex_accessor.cpp
vertex_info_cache_fwd.hpp
vertex_info_cache.hpp
vertex_info_cache.cpp
storage.cpp
indices/indices.cpp
all_vertices_iterable.cpp
vertices_iterable.cpp
inmemory/storage.cpp
inmemory/label_index.cpp
inmemory/label_property_index.cpp
inmemory/unique_constraints.cpp
disk/storage.cpp
disk/rocksdb_storage.cpp
disk/label_index.cpp
disk/label_property_index.cpp
disk/unique_constraints.cpp
storage_mode.cpp
isolation_level.cpp)
set(storage_v2_src_files
${storage_v2_src_files}
replication/replication_client.cpp
replication/replication_server.cpp
replication/serialization.cpp
replication/slk.cpp
replication/replication_persistence_helper.cpp
replication/rpc.cpp)
#######################
find_package(gflags REQUIRED)
find_package(Threads REQUIRED)
add_library(mg-storage-v2 STATIC ${storage_v2_src_files})
target_link_libraries(mg-storage-v2 Threads::Threads mg-utils gflags absl::flat_hash_map)
target_link_libraries(mg-storage-v2 mg-rpc mg-slk)
add_library(mg-storage-v2 STATIC
commit_log.cpp
constraints/existence_constraints.cpp
temporal.cpp
durability/durability.cpp
durability/serialization.cpp
durability/snapshot.cpp
durability/wal.cpp
edge_accessor.cpp
property_store.cpp
vertex_accessor.cpp
vertex_info_cache_fwd.hpp
vertex_info_cache.hpp
vertex_info_cache.cpp
storage.cpp
indices/indices.cpp
all_vertices_iterable.cpp
vertices_iterable.cpp
inmemory/storage.cpp
inmemory/label_index.cpp
inmemory/label_property_index.cpp
inmemory/unique_constraints.cpp
disk/storage.cpp
disk/rocksdb_storage.cpp
disk/label_index.cpp
disk/label_property_index.cpp
disk/unique_constraints.cpp
storage_mode.cpp
isolation_level.cpp
replication/replication_client.cpp
replication/replication_server.cpp
replication/serialization.cpp
replication/slk.cpp
replication/replication_persistence_helper.cpp
replication/rpc.cpp
replication/replication.cpp
inmemory/replication/replication_server.cpp
inmemory/replication/replication_client.cpp
)
target_link_libraries(mg-storage-v2 Threads::Threads mg-utils gflags absl::flat_hash_map mg-rpc mg-slk)
# Until we get LTO there is an advantage to do some unity builds
set_target_properties(mg-storage-v2

View File

@ -336,6 +336,18 @@ class DiskStorage final : public Storage {
void EstablishNewEpoch() override { throw utils::BasicException("Disk storage mode does not support replication."); }
auto CreateReplicationClient(std::string name, io::network::Endpoint endpoint, replication::ReplicationMode mode,
const replication::ReplicationClientConfig &config)
-> std::unique_ptr<ReplicationClient> override {
throw utils::BasicException("Disk storage mode does not support replication.");
}
auto CreateReplicationServer(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config)
-> std::unique_ptr<ReplicationServer> override {
throw utils::BasicException("Disk storage mode does not support replication.");
}
private:
std::unique_ptr<RocksDBStorage> kvstore_;
std::unique_ptr<kvstore::KVStore> durability_kvstore_;
};

View File

@ -0,0 +1,566 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/inmemory/replication/replication_client.hpp"
#include "storage/v2/durability/durability.hpp"
namespace memgraph::storage {
namespace {
template <typename>
[[maybe_unused]] inline constexpr bool always_false_v = false;
} // namespace
// Handler for transfering the current WAL file whose data is
// contained in the internal buffer and the file.
class CurrentWalHandler {
public:
explicit CurrentWalHandler(ReplicationClient *self);
void AppendFilename(const std::string &filename);
void AppendSize(size_t size);
void AppendFileData(utils::InputFile *file);
void AppendBufferData(const uint8_t *buffer, size_t buffer_size);
/// @throw rpc::RpcFailedException
replication::CurrentWalRes Finalize();
private:
ReplicationClient *self_;
rpc::Client::StreamHandler<replication::CurrentWalRpc> stream_;
};
////// CurrentWalHandler //////
CurrentWalHandler::CurrentWalHandler(ReplicationClient *self)
: self_(self), stream_(self_->rpc_client_.Stream<replication::CurrentWalRpc>()) {}
void CurrentWalHandler::AppendFilename(const std::string &filename) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteString(filename);
}
void CurrentWalHandler::AppendSize(const size_t size) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteUint(size);
}
void CurrentWalHandler::AppendFileData(utils::InputFile *file) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteFileData(file);
}
void CurrentWalHandler::AppendBufferData(const uint8_t *buffer, const size_t buffer_size) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteBuffer(buffer, buffer_size);
}
replication::CurrentWalRes CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); }
////// ReplicationClient //////
InMemoryReplicationClient::InMemoryReplicationClient(InMemoryStorage *storage, std::string name,
io::network::Endpoint endpoint, replication::ReplicationMode mode,
const replication::ReplicationClientConfig &config)
: ReplicationClient{std::move(name), std::move(endpoint), mode, config}, storage_{storage} {}
void InMemoryReplicationClient::TryInitializeClientAsync() {
thread_pool_.AddTask([this] {
rpc_client_.Abort();
this->TryInitializeClientSync();
});
}
void InMemoryReplicationClient::FrequentCheck() {
const auto is_success = std::invoke([this]() {
try {
auto stream{rpc_client_.Stream<replication::FrequentHeartbeatRpc>()};
const auto response = stream.AwaitResponse();
return response.success;
} catch (const rpc::RpcFailedException &) {
return false;
}
});
// States: READY, REPLICATING, RECOVERY, INVALID
// If success && ready, replicating, recovery -> stay the same because something good is going on.
// If success && INVALID -> [it's possible that replica came back to life] -> TryInitializeClient.
// If fail -> [replica is not reachable at all] -> INVALID state.
// NOTE: TryInitializeClient might return nothing if there is a branching point.
// NOTE: The early return pattern simplified the code, but the behavior should be as explained.
if (!is_success) {
replica_state_.store(replication::ReplicaState::INVALID);
return;
}
if (replica_state_.load() == replication::ReplicaState::INVALID) {
TryInitializeClientAsync();
}
} /// @throws rpc::RpcFailedException
void InMemoryReplicationClient::Start() {
auto const &endpoint = rpc_client_.Endpoint();
spdlog::trace("Replication client started at: {}:{}", endpoint.address, endpoint.port);
TryInitializeClientSync();
// Help the user to get the most accurate replica state possible.
if (replica_check_frequency_ > std::chrono::seconds(0)) {
replica_checker_.Run("Replica Checker", replica_check_frequency_, [this] { FrequentCheck(); });
}
}
void InMemoryReplicationClient::InitializeClient() {
uint64_t current_commit_timestamp{kTimestampInitialId};
const auto &main_epoch = storage_->replication_state_.GetEpoch();
auto stream{rpc_client_.Stream<replication::HeartbeatRpc>(storage_->replication_state_.last_commit_timestamp_,
main_epoch.id)};
const auto replica = stream.AwaitResponse();
std::optional<uint64_t> branching_point;
if (replica.epoch_id != main_epoch.id && replica.current_commit_timestamp != kTimestampInitialId) {
auto const &history = storage_->replication_state_.history;
const auto epoch_info_iter = std::find_if(history.crbegin(), history.crend(), [&](const auto &main_epoch_info) {
return main_epoch_info.first == replica.epoch_id;
});
if (epoch_info_iter == history.crend()) {
branching_point = 0;
} else if (epoch_info_iter->second != replica.current_commit_timestamp) {
branching_point = epoch_info_iter->second;
}
}
if (branching_point) {
spdlog::error(
"You cannot register Replica {} to this Main because at one point "
"Replica {} acted as the Main instance. Both the Main and Replica {} "
"now hold unique data. Please resolve data conflicts and start the "
"replication on a clean instance.",
name_, name_, name_);
return;
}
current_commit_timestamp = replica.current_commit_timestamp;
spdlog::trace("Current timestamp on replica {}: {}", name_, current_commit_timestamp);
spdlog::trace("Current timestamp on main: {}", storage_->replication_state_.last_commit_timestamp_.load());
if (current_commit_timestamp == storage_->replication_state_.last_commit_timestamp_.load()) {
spdlog::debug("Replica '{}' up to date", name_);
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::READY);
} else {
spdlog::debug("Replica '{}' is behind", name_);
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::RECOVERY);
}
thread_pool_.AddTask([=, this] { this->RecoverReplica(current_commit_timestamp); });
}
}
void InMemoryReplicationClient::TryInitializeClientSync() {
try {
InitializeClient();
} catch (const rpc::RpcFailedException &) {
std::unique_lock client_guarde{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
spdlog::error(utils::MessageWithLink("Failed to connect to replica {} at the endpoint {}.", name_,
rpc_client_.Endpoint(), "https://memgr.ph/replication"));
}
}
void InMemoryReplicationClient::HandleRpcFailure() {
spdlog::error(utils::MessageWithLink("Couldn't replicate data to {}.", name_, "https://memgr.ph/replication"));
TryInitializeClientAsync();
}
void InMemoryReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) {
std::unique_lock guard(client_lock_);
const auto status = replica_state_.load();
switch (status) {
case replication::ReplicaState::RECOVERY:
spdlog::debug("Replica {} is behind MAIN instance", name_);
return;
case replication::ReplicaState::REPLICATING:
spdlog::debug("Replica {} missed a transaction", name_);
// We missed a transaction because we're still replicating
// the previous transaction so we need to go to RECOVERY
// state to catch up with the missing transaction
// We cannot queue the recovery process here because
// an error can happen while we're replicating the previous
// transaction after which the client should go to
// INVALID state before starting the recovery process
replica_state_.store(replication::ReplicaState::RECOVERY);
return;
case replication::ReplicaState::INVALID:
HandleRpcFailure();
return;
case replication::ReplicaState::READY:
MG_ASSERT(!replica_stream_);
try {
replica_stream_.emplace(
ReplicaStream{this, storage_->replication_state_.last_commit_timestamp_.load(), current_wal_seq_num});
replica_state_.store(replication::ReplicaState::REPLICATING);
} catch (const rpc::RpcFailedException &) {
replica_state_.store(replication::ReplicaState::INVALID);
HandleRpcFailure();
}
return;
}
}
void InMemoryReplicationClient::IfStreamingTransaction(const std::function<void(ReplicaStream &)> &callback) {
// We can only check the state because it guarantees to be only
// valid during a single transaction replication (if the assumption
// that this and other transaction replication functions can only be
// called from a one thread stands)
if (replica_state_ != replication::ReplicaState::REPLICATING) {
return;
}
try {
callback(*replica_stream_);
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
}
}
bool InMemoryReplicationClient::FinalizeTransactionReplication() {
// We can only check the state because it guarantees to be only
// valid during a single transaction replication (if the assumption
// that this and other transaction replication functions can only be
// called from a one thread stands)
if (replica_state_ != replication::ReplicaState::REPLICATING) {
return false;
}
if (mode_ == replication::ReplicationMode::ASYNC) {
thread_pool_.AddTask([this] { static_cast<void>(this->FinalizeTransactionReplicationInternal()); });
return true;
}
return FinalizeTransactionReplicationInternal();
}
bool InMemoryReplicationClient::FinalizeTransactionReplicationInternal() {
MG_ASSERT(replica_stream_, "Missing stream for transaction deltas");
try {
auto response = replica_stream_->Finalize();
replica_stream_.reset();
std::unique_lock client_guard(client_lock_);
if (!response.success || replica_state_ == replication::ReplicaState::RECOVERY) {
replica_state_.store(replication::ReplicaState::RECOVERY);
thread_pool_.AddTask([&, this] { this->RecoverReplica(response.current_commit_timestamp); });
} else {
replica_state_.store(replication::ReplicaState::READY);
return true;
}
} catch (const rpc::RpcFailedException &) {
replica_stream_.reset();
{
std::unique_lock client_guard(client_lock_);
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
}
return false;
}
void InMemoryReplicationClient::RecoverReplica(uint64_t replica_commit) {
spdlog::debug("Starting replica recover");
while (true) {
auto file_locker = storage_->file_retainer_.AddLocker();
const auto steps = GetRecoverySteps(replica_commit, &file_locker);
int i = 0;
for (const auto &recovery_step : steps) {
spdlog::trace("Recovering in step: {}", i++);
try {
std::visit(
[&, this]<typename T>(T &&arg) {
using StepType = std::remove_cvref_t<T>;
if constexpr (std::is_same_v<StepType, RecoverySnapshot>) {
spdlog::debug("Sending the latest snapshot file: {}", arg);
auto response = TransferSnapshot(arg);
replica_commit = response.current_commit_timestamp;
} else if constexpr (std::is_same_v<StepType, RecoveryWals>) {
spdlog::debug("Sending the latest wal files");
auto response = TransferWalFiles(arg);
replica_commit = response.current_commit_timestamp;
spdlog::debug("Wal files successfully transferred.");
} else if constexpr (std::is_same_v<StepType, RecoveryCurrentWal>) {
std::unique_lock transaction_guard(storage_->engine_lock_);
if (storage_->wal_file_ && storage_->wal_file_->SequenceNumber() == arg.current_wal_seq_num) {
storage_->wal_file_->DisableFlushing();
transaction_guard.unlock();
spdlog::debug("Sending current wal file");
replica_commit = ReplicateCurrentWal();
storage_->wal_file_->EnableFlushing();
} else {
spdlog::debug("Cannot recover using current wal file");
}
} else {
static_assert(always_false_v<T>, "Missing type from variant visitor");
}
},
recovery_step);
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
return;
}
}
spdlog::trace("Current timestamp on replica: {}", replica_commit);
// To avoid the situation where we read a correct commit timestamp in
// one thread, and after that another thread commits a different a
// transaction and THEN we set the state to READY in the first thread,
// we set this lock before checking the timestamp.
// We will detect that the state is invalid during the next commit,
// because replication::AppendDeltasRpc sends the last commit timestamp which
// replica checks if it's the same last commit timestamp it received
// and we will go to recovery.
// By adding this lock, we can avoid that, and go to RECOVERY immediately.
std::unique_lock client_guard{client_lock_};
SPDLOG_INFO("Replica timestamp: {}", replica_commit);
SPDLOG_INFO("Last commit: {}", storage_->replication_state_.last_commit_timestamp_);
if (storage_->replication_state_.last_commit_timestamp_.load() == replica_commit) {
replica_state_.store(replication::ReplicaState::READY);
return;
}
}
}
uint64_t InMemoryReplicationClient::ReplicateCurrentWal() {
const auto &wal_file = storage_->wal_file_;
auto stream = CurrentWalHandler{this};
stream.AppendFilename(wal_file->Path().filename());
utils::InputFile file;
MG_ASSERT(file.Open(storage_->wal_file_->Path()), "Failed to open current WAL file!");
const auto [buffer, buffer_size] = wal_file->CurrentFileBuffer();
stream.AppendSize(file.GetSize() + buffer_size);
stream.AppendFileData(&file);
stream.AppendBufferData(buffer, buffer_size);
auto response = stream.Finalize();
return response.current_commit_timestamp;
} /// This method tries to find the optimal path for recoverying a single replica.
/// Based on the last commit transfered to replica it tries to update the
/// replica using durability files - WALs and Snapshots. WAL files are much
/// smaller in size as they contain only the Deltas (changes) made during the
/// transactions while Snapshots contain all the data. For that reason we prefer
/// WALs as much as possible. As the WAL file that is currently being updated
/// can change during the process we ignore it as much as possible. Also, it
/// uses the transaction lock so locking it can be really expensive. After we
/// fetch the list of finalized WALs, we try to find the longest chain of
/// sequential WALs, starting from the latest one, that will update the recovery
/// with the all missed updates. If the WAL chain cannot be created, replica is
/// behind by a lot, so we use the regular recovery process, we send the latest
/// snapshot and all the necessary WAL files, starting from the newest WAL that
/// contains a timestamp before the snapshot. If we registered the existence of
/// the current WAL, we add the sequence number we read from it to the recovery
/// process. After all the other steps are finished, if the current WAL contains
/// the same sequence number, it's the same WAL we read while fetching the
/// recovery steps, so we can safely send it to the replica.
/// We assume that the property of preserving at least 1 WAL before the snapshot
/// is satisfied as we extract the timestamp information from it.
std::vector<InMemoryReplicationClient::RecoveryStep> InMemoryReplicationClient::GetRecoverySteps(
const uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker) {
// First check if we can recover using the current wal file only
// otherwise save the seq_num of the current wal file
// This lock is also necessary to force the missed transaction to finish.
std::optional<uint64_t> current_wal_seq_num;
std::optional<uint64_t> current_wal_from_timestamp;
if (std::unique_lock transtacion_guard(storage_->engine_lock_); storage_->wal_file_) {
current_wal_seq_num.emplace(storage_->wal_file_->SequenceNumber());
current_wal_from_timestamp.emplace(storage_->wal_file_->FromTimestamp());
}
auto locker_acc = file_locker->Access();
auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_, current_wal_seq_num);
MG_ASSERT(wal_files, "Wal files could not be loaded");
auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
std::optional<durability::SnapshotDurabilityInfo> latest_snapshot;
if (!snapshot_files.empty()) {
std::sort(snapshot_files.begin(), snapshot_files.end());
latest_snapshot.emplace(std::move(snapshot_files.back()));
}
std::vector<RecoveryStep> recovery_steps;
// No finalized WAL files were found. This means the difference is contained
// inside the current WAL or the snapshot.
if (wal_files->empty()) {
if (current_wal_from_timestamp && replica_commit >= *current_wal_from_timestamp) {
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
// Without the finalized WAL containing the current timestamp of replica,
// we cannot know if the difference is only in the current WAL or we need
// to send the snapshot.
if (latest_snapshot) {
const auto lock_success = locker_acc.AddPath(latest_snapshot->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_steps.emplace_back(std::in_place_type_t<RecoverySnapshot>{}, std::move(latest_snapshot->path));
}
// if there are no finalized WAL files, snapshot left the current WAL
// as the WAL file containing a transaction before snapshot creation
// so we can be sure that the current WAL is present
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
// Find the longest chain of WALs for recovery.
// The chain consists ONLY of sequential WALs.
auto rwal_it = wal_files->rbegin();
// if the last finalized WAL is before the replica commit
// then we can recovery only from current WAL
if (rwal_it->to_timestamp <= replica_commit) {
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
uint64_t previous_seq_num{rwal_it->seq_num};
for (; rwal_it != wal_files->rend(); ++rwal_it) {
// If the difference between two consecutive wal files is not 0 or 1
// we have a missing WAL in our chain
if (previous_seq_num - rwal_it->seq_num > 1) {
break;
}
// Find first WAL that contains up to replica commit, i.e. WAL
// that is before the replica commit or conatins the replica commit
// as the last committed transaction OR we managed to find the first WAL
// file.
if (replica_commit >= rwal_it->from_timestamp || rwal_it->seq_num == 0) {
if (replica_commit >= rwal_it->to_timestamp) {
// We want the WAL after because the replica already contains all the
// commits from this WAL
--rwal_it;
}
std::vector<std::filesystem::path> wal_chain;
auto distance_from_first = std::distance(rwal_it, wal_files->rend() - 1);
// We have managed to create WAL chain
// We need to lock these files and add them to the chain
for (auto result_wal_it = wal_files->begin() + distance_from_first; result_wal_it != wal_files->end();
++result_wal_it) {
const auto lock_success = locker_acc.AddPath(result_wal_it->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
wal_chain.push_back(std::move(result_wal_it->path));
}
recovery_steps.emplace_back(std::in_place_type_t<RecoveryWals>{}, std::move(wal_chain));
if (current_wal_seq_num) {
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
}
return recovery_steps;
}
previous_seq_num = rwal_it->seq_num;
}
MG_ASSERT(latest_snapshot, "Invalid durability state, missing snapshot");
// We didn't manage to find a WAL chain, we need to send the latest snapshot
// with its WALs
const auto lock_success = locker_acc.AddPath(latest_snapshot->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_steps.emplace_back(std::in_place_type_t<RecoverySnapshot>{}, std::move(latest_snapshot->path));
std::vector<std::filesystem::path> recovery_wal_files;
auto wal_it = wal_files->begin();
for (; wal_it != wal_files->end(); ++wal_it) {
// Assuming recovery process is correct the snashpot should
// always retain a single WAL that contains a transaction
// before its creation
if (latest_snapshot->start_timestamp < wal_it->to_timestamp) {
if (latest_snapshot->start_timestamp < wal_it->from_timestamp) {
MG_ASSERT(wal_it != wal_files->begin(), "Invalid durability files state");
--wal_it;
}
break;
}
}
for (; wal_it != wal_files->end(); ++wal_it) {
const auto lock_success = locker_acc.AddPath(wal_it->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_wal_files.push_back(std::move(wal_it->path));
}
// We only have a WAL before the snapshot
if (recovery_wal_files.empty()) {
const auto lock_success = locker_acc.AddPath(wal_files->back().path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_wal_files.push_back(std::move(wal_files->back().path));
}
recovery_steps.emplace_back(std::in_place_type_t<RecoveryWals>{}, std::move(recovery_wal_files));
if (current_wal_seq_num) {
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
}
return recovery_steps;
}
TimestampInfo InMemoryReplicationClient::GetTimestampInfo() {
TimestampInfo info;
info.current_timestamp_of_replica = 0;
info.current_number_of_timestamp_behind_master = 0;
try {
auto stream{rpc_client_.Stream<replication::TimestampRpc>()};
const auto response = stream.AwaitResponse();
const auto is_success = response.success;
if (!is_success) {
replica_state_.store(replication::ReplicaState::INVALID);
HandleRpcFailure();
}
auto main_time_stamp = storage_->replication_state_.last_commit_timestamp_.load();
info.current_timestamp_of_replica = response.current_commit_timestamp;
info.current_number_of_timestamp_behind_master = response.current_commit_timestamp - main_time_stamp;
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard(client_lock_);
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure(); // mutex already unlocked, if the new enqueued task dispatches immediately it probably won't
// block
}
return info;
}
std::string const &InMemoryReplicationClient::GetEpochId() const { return storage_->replication_state_.GetEpoch().id; }
auto InMemoryReplicationClient::GetStorage() -> Storage * { return storage_; }
replication::SnapshotRes InMemoryReplicationClient::TransferSnapshot(const std::filesystem::path &path) {
auto stream{rpc_client_.Stream<replication::SnapshotRpc>()};
replication::Encoder encoder(stream.GetBuilder());
encoder.WriteFile(path);
return stream.AwaitResponse();
}
replication::WalFilesRes InMemoryReplicationClient::TransferWalFiles(
const std::vector<std::filesystem::path> &wal_files) {
MG_ASSERT(!wal_files.empty(), "Wal files list is empty!");
auto stream{rpc_client_.Stream<replication::WalFilesRpc>(wal_files.size())};
replication::Encoder encoder(stream.GetBuilder());
for (const auto &wal : wal_files) {
spdlog::debug("Sending wal file: {}", wal);
encoder.WriteFile(wal);
}
return stream.AwaitResponse();
}
} // namespace memgraph::storage

View File

@ -0,0 +1,68 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/replication/replication_client.hpp"
namespace memgraph::storage {
class InMemoryReplicationClient : public ReplicationClient {
public:
InMemoryReplicationClient(InMemoryStorage *storage, std::string name, io::network::Endpoint endpoint,
replication::ReplicationMode mode, const replication::ReplicationClientConfig &config = {});
void StartTransactionReplication(uint64_t current_wal_seq_num) override;
// Replication clients can be removed at any point
// so to avoid any complexity of checking if the client was removed whenever
// we want to send part of transaction and to avoid adding some GC logic this
// function will run a callback if, after previously callling
// StartTransactionReplication, stream is created.
void IfStreamingTransaction(const std::function<void(ReplicaStream &)> &callback) override;
auto GetEpochId() const -> std::string const & override;
auto GetStorage() -> Storage * override;
void Start() override;
// Return whether the transaction could be finalized on the replication client or not.
[[nodiscard]] bool FinalizeTransactionReplication() override;
TimestampInfo GetTimestampInfo() override;
private:
void TryInitializeClientAsync();
void FrequentCheck();
void InitializeClient();
void TryInitializeClientSync();
void HandleRpcFailure();
[[nodiscard]] bool FinalizeTransactionReplicationInternal();
void RecoverReplica(uint64_t replica_commit);
uint64_t ReplicateCurrentWal();
using RecoverySnapshot = std::filesystem::path;
using RecoveryWals = std::vector<std::filesystem::path>;
struct RecoveryCurrentWal {
explicit RecoveryCurrentWal(const uint64_t current_wal_seq_num) : current_wal_seq_num(current_wal_seq_num) {}
uint64_t current_wal_seq_num;
};
using RecoveryStep = std::variant<RecoverySnapshot, RecoveryWals, RecoveryCurrentWal>;
std::vector<RecoveryStep> GetRecoverySteps(uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker);
// Transfer the snapshot file.
// @param path Path of the snapshot file.
replication::SnapshotRes TransferSnapshot(const std::filesystem::path &path);
// Transfer the WAL files
replication::WalFilesRes TransferWalFiles(const std::vector<std::filesystem::path> &wal_files);
InMemoryStorage *storage_;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,575 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/inmemory/replication/replication_server.hpp"
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/version.hpp"
#include "storage/v2/inmemory/storage.hpp"
namespace memgraph::storage {
namespace {
std::pair<uint64_t, durability::WalDeltaData> ReadDelta(durability::BaseDecoder *decoder) {
try {
auto timestamp = ReadWalDeltaHeader(decoder);
SPDLOG_INFO(" Timestamp {}", timestamp);
auto delta = ReadWalDeltaData(decoder);
return {timestamp, delta};
} catch (const slk::SlkReaderException &) {
throw utils::BasicException("Missing data!");
} catch (const durability::RecoveryFailure &) {
throw utils::BasicException("Invalid data!");
}
};
} // namespace
InMemoryReplicationServer::InMemoryReplicationServer(InMemoryStorage *storage, memgraph::io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config)
: ReplicationServer{std::move(endpoint), config}, storage_(storage) {
rpc_server_.Register<replication::HeartbeatRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received HeartbeatRpc");
this->HeartbeatHandler(req_reader, res_builder);
});
rpc_server_.Register<replication::AppendDeltasRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received AppendDeltasRpc");
this->AppendDeltasHandler(req_reader, res_builder);
});
rpc_server_.Register<replication::SnapshotRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received SnapshotRpc");
this->SnapshotHandler(req_reader, res_builder);
});
rpc_server_.Register<replication::WalFilesRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received WalFilesRpc");
this->WalFilesHandler(req_reader, res_builder);
});
rpc_server_.Register<replication::CurrentWalRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received CurrentWalRpc");
this->CurrentWalHandler(req_reader, res_builder);
});
rpc_server_.Register<replication::TimestampRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received TimestampRpc");
this->TimestampHandler(req_reader, res_builder);
});
}
void InMemoryReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::HeartbeatReq req;
slk::Load(&req, req_reader);
replication::HeartbeatRes res{true, storage_->replication_state_.last_commit_timestamp_.load(),
storage_->replication_state_.GetEpoch().id};
slk::Save(res, res_builder);
}
void InMemoryReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::AppendDeltasReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
auto maybe_epoch_id = decoder.ReadString();
MG_ASSERT(maybe_epoch_id, "Invalid replication message");
if (*maybe_epoch_id != storage_->replication_state_.GetEpoch().id) {
storage_->replication_state_.AppendEpoch(*maybe_epoch_id);
}
if (storage_->wal_file_) {
if (req.seq_num > storage_->wal_file_->SequenceNumber() ||
*maybe_epoch_id != storage_->replication_state_.GetEpoch().id) {
storage_->wal_file_->FinalizeWal();
storage_->wal_file_.reset();
storage_->wal_seq_num_ = req.seq_num;
spdlog::trace("Finalized WAL file");
} else {
MG_ASSERT(storage_->wal_file_->SequenceNumber() == req.seq_num, "Invalid sequence number of current wal file");
storage_->wal_seq_num_ = req.seq_num + 1;
}
} else {
storage_->wal_seq_num_ = req.seq_num;
}
if (req.previous_commit_timestamp != storage_->replication_state_.last_commit_timestamp_.load()) {
// Empty the stream
bool transaction_complete = false;
while (!transaction_complete) {
SPDLOG_INFO("Skipping delta");
const auto [timestamp, delta] = ReadDelta(&decoder);
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
}
replication::AppendDeltasRes res{false, storage_->replication_state_.last_commit_timestamp_.load()};
slk::Save(res, res_builder);
return;
}
ReadAndApplyDelta(storage_, &decoder);
replication::AppendDeltasRes res{true, storage_->replication_state_.last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::debug("Replication recovery from append deltas finished, replica is now up to date!");
}
void InMemoryReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::SnapshotReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->snapshot_directory_);
const auto maybe_snapshot_path = decoder.ReadFile(storage_->snapshot_directory_);
MG_ASSERT(maybe_snapshot_path, "Failed to load snapshot!");
spdlog::info("Received snapshot saved to {}", *maybe_snapshot_path);
std::unique_lock<utils::RWLock> storage_guard(storage_->main_lock_);
spdlog::trace("Clearing database since recovering from snapshot.");
// Clear the database
storage_->vertices_.clear();
storage_->edges_.clear();
storage_->constraints_.existence_constraints_ = std::make_unique<ExistenceConstraints>();
storage_->constraints_.unique_constraints_ = std::make_unique<InMemoryUniqueConstraints>();
storage_->indices_.label_index_ =
std::make_unique<InMemoryLabelIndex>(&storage_->indices_, &storage_->constraints_, storage_->config_);
storage_->indices_.label_property_index_ =
std::make_unique<InMemoryLabelPropertyIndex>(&storage_->indices_, &storage_->constraints_, storage_->config_);
try {
spdlog::debug("Loading snapshot");
auto &epoch =
storage_->replication_state_
.GetEpoch(); // This needs to be a non-const ref since we are updating it in LoadSnapshot TODO fix
auto recovered_snapshot = durability::LoadSnapshot(
*maybe_snapshot_path, &storage_->vertices_, &storage_->edges_, &storage_->replication_state_.history,
storage_->name_id_mapper_.get(), &storage_->edge_count_, storage_->config_);
spdlog::debug("Snapshot loaded successfully");
// If this step is present it should always be the first step of
// the recovery so we use the UUID we read from snasphost
storage_->uuid_ = std::move(recovered_snapshot.snapshot_info.uuid);
epoch.id = std::move(recovered_snapshot.snapshot_info.epoch_id);
const auto &recovery_info = recovered_snapshot.recovery_info;
storage_->vertex_id_ = recovery_info.next_vertex_id;
storage_->edge_id_ = recovery_info.next_edge_id;
storage_->timestamp_ = std::max(storage_->timestamp_, recovery_info.next_timestamp);
spdlog::trace("Recovering indices and constraints from snapshot.");
durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &storage_->indices_,
&storage_->constraints_, &storage_->vertices_);
} catch (const durability::RecoveryFailure &e) {
LOG_FATAL("Couldn't load the snapshot because of: {}", e.what());
}
storage_guard.unlock();
replication::SnapshotRes res{true, storage_->replication_state_.last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::trace("Deleting old snapshot files due to snapshot recovery.");
// Delete other durability files
auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
for (const auto &[path, uuid, _] : snapshot_files) {
if (path != *maybe_snapshot_path) {
spdlog::trace("Deleting snapshot file {}", path);
storage_->file_retainer_.DeleteFile(path);
}
}
spdlog::trace("Deleting old WAL files due to snapshot recovery.");
auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_);
if (wal_files) {
for (const auto &wal_file : *wal_files) {
spdlog::trace("Deleting WAL file {}", wal_file.path);
storage_->file_retainer_.DeleteFile(wal_file.path);
}
storage_->wal_file_.reset();
}
spdlog::debug("Replication recovery from snapshot finished!");
}
void InMemoryReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::WalFilesReq req;
slk::Load(&req, req_reader);
const auto wal_file_number = req.file_number;
spdlog::debug("Received WAL files: {}", wal_file_number);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->wal_directory_);
for (auto i = 0; i < wal_file_number; ++i) {
LoadWal(storage_, &decoder);
}
replication::WalFilesRes res{true, storage_->replication_state_.last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::debug("Replication recovery from WAL files ended successfully, replica is now up to date!");
}
void InMemoryReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::CurrentWalReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->wal_directory_);
LoadWal(storage_, &decoder);
replication::CurrentWalRes res{true, storage_->replication_state_.last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::debug("Replication recovery from current WAL ended successfully, replica is now up to date!");
}
void InMemoryReplicationServer::LoadWal(InMemoryStorage *storage, replication::Decoder *decoder) {
const auto temp_wal_directory = std::filesystem::temp_directory_path() / "memgraph" / durability::kWalDirectory;
utils::EnsureDir(temp_wal_directory);
auto maybe_wal_path = decoder->ReadFile(temp_wal_directory);
MG_ASSERT(maybe_wal_path, "Failed to load WAL!");
spdlog::trace("Received WAL saved to {}", *maybe_wal_path);
try {
auto wal_info = durability::ReadWalInfo(*maybe_wal_path);
if (wal_info.seq_num == 0) {
storage->uuid_ = wal_info.uuid;
}
if (wal_info.epoch_id != storage->replication_state_.GetEpoch().id) {
storage->replication_state_.AppendEpoch(wal_info.epoch_id);
}
if (storage->wal_file_) {
if (storage->wal_file_->SequenceNumber() != wal_info.seq_num) {
storage->wal_file_->FinalizeWal();
storage->wal_seq_num_ = wal_info.seq_num;
storage->wal_file_.reset();
spdlog::trace("WAL file {} finalized successfully", *maybe_wal_path);
}
} else {
storage->wal_seq_num_ = wal_info.seq_num;
}
spdlog::trace("Loading WAL deltas from {}", *maybe_wal_path);
durability::Decoder wal;
const auto version = wal.Initialize(*maybe_wal_path, durability::kWalMagic);
spdlog::debug("WAL file {} loaded successfully", *maybe_wal_path);
if (!version) throw durability::RecoveryFailure("Couldn't read WAL magic and/or version!");
if (!durability::IsVersionSupported(*version)) throw durability::RecoveryFailure("Invalid WAL version!");
wal.SetPosition(wal_info.offset_deltas);
for (size_t i = 0; i < wal_info.num_deltas;) {
i += ReadAndApplyDelta(storage, &wal);
}
spdlog::debug("Replication from current WAL successful!");
} catch (const durability::RecoveryFailure &e) {
LOG_FATAL("Couldn't recover WAL deltas from {} because of: {}", *maybe_wal_path, e.what());
}
}
void InMemoryReplicationServer::TimestampHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::TimestampReq req;
slk::Load(&req, req_reader);
replication::TimestampRes res{true, storage_->replication_state_.last_commit_timestamp_.load()};
slk::Save(res, res_builder);
}
uint64_t InMemoryReplicationServer::ReadAndApplyDelta(InMemoryStorage *storage, durability::BaseDecoder *decoder) {
auto edge_acc = storage->edges_.access();
auto vertex_acc = storage->vertices_.access();
std::optional<std::pair<uint64_t, InMemoryStorage::ReplicationAccessor>> commit_timestamp_and_accessor;
auto get_transaction = [storage, &commit_timestamp_and_accessor](uint64_t commit_timestamp) {
if (!commit_timestamp_and_accessor) {
auto acc = storage->Access(std::nullopt);
auto inmem_acc = std::unique_ptr<InMemoryStorage::InMemoryAccessor>(
static_cast<InMemoryStorage::InMemoryAccessor *>(acc.release()));
commit_timestamp_and_accessor.emplace(commit_timestamp, std::move(*inmem_acc));
} else if (commit_timestamp_and_accessor->first != commit_timestamp) {
throw utils::BasicException("Received more than one transaction!");
}
return &commit_timestamp_and_accessor->second;
};
uint64_t applied_deltas = 0;
auto max_commit_timestamp = storage->replication_state_.last_commit_timestamp_.load();
for (bool transaction_complete = false; !transaction_complete; ++applied_deltas) {
const auto [timestamp, delta] = ReadDelta(decoder);
if (timestamp > max_commit_timestamp) {
max_commit_timestamp = timestamp;
}
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
if (timestamp < storage->timestamp_) {
continue;
}
SPDLOG_INFO(" Delta {}", applied_deltas);
switch (delta.type) {
case durability::WalDeltaData::Type::VERTEX_CREATE: {
spdlog::trace(" Create vertex {}", delta.vertex_create_delete.gid.AsUint());
auto *transaction = get_transaction(timestamp);
transaction->CreateVertexEx(delta.vertex_create_delete.gid);
break;
}
case durability::WalDeltaData::Type::VERTEX_DELETE: {
spdlog::trace(" Delete vertex {}", delta.vertex_create_delete.gid.AsUint());
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_create_delete.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = transaction->DeleteVertex(&*vertex);
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_ADD_LABEL: {
spdlog::trace(" Vertex {} add label {}", delta.vertex_add_remove_label.gid.AsUint(),
delta.vertex_add_remove_label.label);
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_add_remove_label.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->AddLabel(transaction->NameToLabel(delta.vertex_add_remove_label.label));
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_REMOVE_LABEL: {
spdlog::trace(" Vertex {} remove label {}", delta.vertex_add_remove_label.gid.AsUint(),
delta.vertex_add_remove_label.label);
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_add_remove_label.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->RemoveLabel(transaction->NameToLabel(delta.vertex_add_remove_label.label));
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_SET_PROPERTY: {
spdlog::trace(" Vertex {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(),
delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value);
auto *transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_edge_set_property.gid, View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property),
delta.vertex_edge_set_property.value);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_CREATE: {
spdlog::trace(" Create edge {} of type {} from vertex {} to vertex {}",
delta.edge_create_delete.gid.AsUint(), delta.edge_create_delete.edge_type,
delta.edge_create_delete.from_vertex.AsUint(), delta.edge_create_delete.to_vertex.AsUint());
auto *transaction = get_transaction(timestamp);
auto from_vertex = transaction->FindVertex(delta.edge_create_delete.from_vertex, View::NEW);
if (!from_vertex) throw utils::BasicException("Invalid transaction!");
auto to_vertex = transaction->FindVertex(delta.edge_create_delete.to_vertex, View::NEW);
if (!to_vertex) throw utils::BasicException("Invalid transaction!");
auto edge = transaction->CreateEdgeEx(&*from_vertex, &*to_vertex,
transaction->NameToEdgeType(delta.edge_create_delete.edge_type),
delta.edge_create_delete.gid);
if (edge.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_DELETE: {
spdlog::trace(" Delete edge {} of type {} from vertex {} to vertex {}",
delta.edge_create_delete.gid.AsUint(), delta.edge_create_delete.edge_type,
delta.edge_create_delete.from_vertex.AsUint(), delta.edge_create_delete.to_vertex.AsUint());
auto *transaction = get_transaction(timestamp);
auto from_vertex = transaction->FindVertex(delta.edge_create_delete.from_vertex, View::NEW);
if (!from_vertex) throw utils::BasicException("Invalid transaction!");
auto to_vertex = transaction->FindVertex(delta.edge_create_delete.to_vertex, View::NEW);
if (!to_vertex) throw utils::BasicException("Invalid transaction!");
auto edges = from_vertex->OutEdges(View::NEW, {transaction->NameToEdgeType(delta.edge_create_delete.edge_type)},
&*to_vertex);
if (edges.HasError()) throw utils::BasicException("Invalid transaction!");
if (edges->size() != 1) throw utils::BasicException("Invalid transaction!");
auto &edge = (*edges)[0];
auto ret = transaction->DeleteEdge(&edge);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_SET_PROPERTY: {
spdlog::trace(" Edge {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(),
delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value);
if (!storage->config_.items.properties_on_edges)
throw utils::BasicException(
"Can't set properties on edges because properties on edges "
"are disabled!");
auto *transaction = get_transaction(timestamp);
// The following block of code effectively implements `FindEdge` and
// yields an accessor that is only valid for managing the edge's
// properties.
auto edge = edge_acc.find(delta.vertex_edge_set_property.gid);
if (edge == edge_acc.end()) throw utils::BasicException("Invalid transaction!");
// The edge visibility check must be done here manually because we
// don't allow direct access to the edges through the public API.
{
bool is_visible = true;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge->lock);
is_visible = !edge->deleted;
delta = edge->delta;
}
ApplyDeltasForRead(&transaction->GetTransaction(), delta, View::NEW, [&is_visible](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
case Delta::Action::RECREATE_OBJECT: {
is_visible = true;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
is_visible = false;
break;
}
}
});
if (!is_visible) throw utils::BasicException("Invalid transaction!");
}
EdgeRef edge_ref(&*edge);
// Here we create an edge accessor that we will use to get the
// properties of the edge. The accessor is created with an invalid
// type and invalid from/to pointers because we don't know them
// here, but that isn't an issue because we won't use that part of
// the API here.
auto ea = EdgeAccessor{edge_ref,
EdgeTypeId::FromUint(0UL),
nullptr,
nullptr,
&transaction->GetTransaction(),
&storage->indices_,
&storage->constraints_,
storage->config_.items};
auto ret = ea.SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property),
delta.vertex_edge_set_property.value);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::TRANSACTION_END: {
spdlog::trace(" Transaction end");
if (!commit_timestamp_and_accessor || commit_timestamp_and_accessor->first != timestamp)
throw utils::BasicException("Invalid data!");
auto ret = commit_timestamp_and_accessor->second.Commit(commit_timestamp_and_accessor->first);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
commit_timestamp_and_accessor = std::nullopt;
break;
}
case durability::WalDeltaData::Type::LABEL_INDEX_CREATE: {
spdlog::trace(" Create label index on :{}", delta.operation_label.label);
// Need to send the timestamp
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage->CreateIndex(storage->NameToLabel(delta.operation_label.label), timestamp).HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_INDEX_DROP: {
spdlog::trace(" Drop label index on :{}", delta.operation_label.label);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage->DropIndex(storage->NameToLabel(delta.operation_label.label), timestamp).HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE: {
spdlog::trace(" Create label+property index on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage
->CreateIndex(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property), timestamp)
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP: {
spdlog::trace(" Drop label+property index on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage
->DropIndex(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property), timestamp)
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: {
spdlog::trace(" Create existence constraint on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
auto ret = storage->CreateExistenceConstraint(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property),
timestamp);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP: {
spdlog::trace(" Drop existence constraint on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage
->DropExistenceConstraint(storage->NameToLabel(delta.operation_label_property.label),
storage->NameToProperty(delta.operation_label_property.property), timestamp)
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE: {
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Create unique constraint on :{} ({})", delta.operation_label_properties.label, ss.str());
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
std::set<PropertyId> properties;
for (const auto &prop : delta.operation_label_properties.properties) {
properties.emplace(storage->NameToProperty(prop));
}
auto ret = storage->CreateUniqueConstraint(storage->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
if (!ret.HasValue() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS)
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP: {
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Drop unique constraint on :{} ({})", delta.operation_label_properties.label, ss.str());
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
std::set<PropertyId> properties;
for (const auto &prop : delta.operation_label_properties.properties) {
properties.emplace(storage->NameToProperty(prop));
}
auto ret = storage->DropUniqueConstraint(storage->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
if (ret.HasError() || ret.GetValue() != UniqueConstraints::DeletionStatus::SUCCESS)
throw utils::BasicException("Invalid transaction!");
break;
}
}
}
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid data!");
storage->replication_state_.last_commit_timestamp_ = max_commit_timestamp;
spdlog::debug("Applied {} deltas", applied_deltas);
return applied_deltas;
}
} // namespace memgraph::storage

View File

@ -0,0 +1,47 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include "storage/v2/replication/replication_server.hpp"
#include "storage/v2/replication/serialization.hpp"
namespace memgraph::storage {
class InMemoryStorage;
class InMemoryReplicationServer : public ReplicationServer {
public:
explicit InMemoryReplicationServer(InMemoryStorage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config);
private:
// RPC handlers
void HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void TimestampHandler(slk::Reader *req_reader, slk::Builder *res_builder);
static void LoadWal(InMemoryStorage *storage, replication::Decoder *decoder);
static uint64_t ReadAndApplyDelta(InMemoryStorage *storage, durability::BaseDecoder *decoder);
InMemoryStorage *storage_;
};
} // namespace memgraph::storage

View File

@ -21,41 +21,22 @@
#include "utils/stat.hpp"
/// REPLICATION ///
#include "storage/v2/replication/replication_client.hpp"
#include "storage/v2/replication/replication_server.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/storage_error.hpp"
#include "storage/v2/inmemory/replication/replication_client.hpp"
#include "storage/v2/inmemory/replication/replication_server.hpp"
namespace memgraph::storage {
using OOMExceptionEnabler = utils::MemoryTracker::OutOfMemoryExceptionEnabler;
namespace {
inline constexpr uint16_t kEpochHistoryRetention = 1000;
std::string RegisterReplicaErrorToString(InMemoryStorage::ReplicationState::RegisterReplicaError error) {
using enum InMemoryStorage::ReplicationState::RegisterReplicaError;
switch (error) {
case NAME_EXISTS:
return "NAME_EXISTS";
case END_POINT_EXISTS:
return "END_POINT_EXISTS";
case CONNECTION_FAILED:
return "CONNECTION_FAILED";
case COULD_NOT_BE_PERSISTED:
return "COULD_NOT_BE_PERSISTED";
}
}
} // namespace
InMemoryStorage::InMemoryStorage(Config config)
: Storage(config, StorageMode::IN_MEMORY_TRANSACTIONAL),
snapshot_directory_(config.durability.storage_directory / durability::kSnapshotDirectory),
lock_file_path_(config.durability.storage_directory / durability::kLockFile),
wal_directory_(config.durability.storage_directory / durability::kWalDirectory),
uuid_(utils::GenerateUUID()),
epoch_id_(utils::GenerateUUID()),
global_locker_(file_retainer_.AddLocker()) {
global_locker_(file_retainer_.AddLocker()),
replication_state_(config_.durability.restore_replication_state_on_startup,
config_.durability.storage_directory) {
if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED ||
config_.durability.snapshot_on_exit || config_.durability.recover_on_startup) {
// Create the directory initially to crash the database in case of
@ -82,15 +63,16 @@ InMemoryStorage::InMemoryStorage(Config config)
config_.durability.storage_directory);
}
if (config_.durability.recover_on_startup) {
auto info = durability::RecoverData(snapshot_directory_, wal_directory_, &uuid_, &epoch_id_, &epoch_history_,
&vertices_, &edges_, &edge_count_, name_id_mapper_.get(), &indices_,
&constraints_, config_, &wal_seq_num_);
auto &epoch = replication_state_.GetEpoch();
auto info = durability::RecoverData(snapshot_directory_, wal_directory_, &uuid_, &epoch.id,
&replication_state_.history, &vertices_, &edges_, &edge_count_,
name_id_mapper_.get(), &indices_, &constraints_, config_, &wal_seq_num_);
if (info) {
vertex_id_ = info->next_vertex_id;
edge_id_ = info->next_edge_id;
timestamp_ = std::max(timestamp_, info->next_timestamp);
if (info->last_commit_timestamp) {
last_commit_timestamp_ = *info->last_commit_timestamp;
replication_state_.last_commit_timestamp_ = *info->last_commit_timestamp;
}
}
} else if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::DISABLED ||
@ -153,11 +135,6 @@ InMemoryStorage::InMemoryStorage(Config config)
if (config_.durability.restore_replication_state_on_startup) {
spdlog::info("Replication configuration will be stored and will be automatically restored in case of a crash.");
utils::EnsureDirOrDie(config_.durability.storage_directory / durability::kReplicationDirectory);
// TODO: Move this to replication
replication_state_.durability_ =
std::make_unique<kvstore::KVStore>(config_.durability.storage_directory / durability::kReplicationDirectory);
RestoreReplicationRole();
if (replication_state_.GetRole() == replication::ReplicationRole::MAIN) {
@ -722,7 +699,7 @@ utils::BasicResult<StorageDataManipulationError, void> InMemoryStorage::InMemory
if (mem_storage->replication_state_.GetRole() == replication::ReplicationRole::MAIN ||
desired_commit_timestamp.has_value()) {
// Update the last commit timestamp
mem_storage->last_commit_timestamp_.store(*commit_timestamp_);
mem_storage->replication_state_.last_commit_timestamp_.store(*commit_timestamp_);
}
// Release engine lock because we don't have to hold it anymore
// and emplace back could take a long time.
@ -938,7 +915,7 @@ utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::CreateInd
const auto success =
AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_INDEX_CREATE, label, {}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::IncrementCounter(memgraph::metrics::ActiveLabelIndices);
@ -961,7 +938,7 @@ utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::CreateInd
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_CREATE, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::IncrementCounter(memgraph::metrics::ActiveLabelPropertyIndices);
@ -983,7 +960,7 @@ utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::DropIndex
auto success =
AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_INDEX_DROP, label, {}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::DecrementCounter(memgraph::metrics::ActiveLabelIndices);
@ -1007,7 +984,7 @@ utils::BasicResult<StorageIndexDefinitionError, void> InMemoryStorage::DropIndex
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::LABEL_PROPERTY_INDEX_DROP, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
// We don't care if there is a replication error because on main node the change will go through
memgraph::metrics::DecrementCounter(memgraph::metrics::ActiveLabelPropertyIndices);
@ -1038,7 +1015,7 @@ utils::BasicResult<StorageExistenceConstraintDefinitionError, void> InMemoryStor
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_CREATE, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return {};
@ -1057,7 +1034,7 @@ utils::BasicResult<StorageExistenceConstraintDroppingError, void> InMemoryStorag
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::EXISTENCE_CONSTRAINT_DROP, label,
{property}, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return {};
@ -1082,7 +1059,7 @@ InMemoryStorage::CreateUniqueConstraint(LabelId label, const std::set<PropertyId
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_CREATE, label,
properties, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return UniqueConstraints::CreationStatus::SUCCESS;
@ -1103,7 +1080,7 @@ InMemoryStorage::DropUniqueConstraint(LabelId label, const std::set<PropertyId>
auto success = AppendToWalDataDefinition(durability::StorageGlobalOperation::UNIQUE_CONSTRAINT_DROP, label,
properties, commit_timestamp);
commit_log_->MarkFinished(commit_timestamp);
last_commit_timestamp_ = commit_timestamp;
replication_state_.last_commit_timestamp_ = commit_timestamp;
if (success) {
return UniqueConstraints::DeletionStatus::SUCCESS;
@ -1487,8 +1464,8 @@ bool InMemoryStorage::InitializeWalFile() {
if (config_.durability.snapshot_wal_mode != Config::Durability::SnapshotWalMode::PERIODIC_SNAPSHOT_WITH_WAL)
return false;
if (!wal_file_) {
wal_file_.emplace(wal_directory_, uuid_, epoch_id_, config_.items, name_id_mapper_.get(), wal_seq_num_++,
&file_retainer_);
wal_file_.emplace(wal_directory_, uuid_, replication_state_.GetEpoch().id, config_.items, name_id_mapper_.get(),
wal_seq_num_++, &file_retainer_);
}
return true;
}
@ -1703,12 +1680,13 @@ utils::BasicResult<InMemoryStorage::CreateSnapshotError> InMemoryStorage::Create
auto snapshot_creator = [this]() {
utils::Timer timer;
const auto &epoch = replication_state_.GetEpoch();
auto transaction = CreateTransaction(IsolationLevel::SNAPSHOT_ISOLATION, storage_mode_);
// Create snapshot.
durability::CreateSnapshot(&transaction, snapshot_directory_, wal_directory_,
config_.durability.snapshot_retention_count, &vertices_, &edges_, name_id_mapper_.get(),
&indices_, &constraints_, config_, uuid_, epoch_id_, epoch_history_, &file_retainer_);
&indices_, &constraints_, config_, uuid_, epoch.id, replication_state_.history,
&file_retainer_);
// Finalize snapshot transaction.
commit_log_->MarkFinished(transaction.start_timestamp);
@ -1769,12 +1747,8 @@ void InMemoryStorage::EstablishNewEpoch() {
wal_file_->FinalizeWal();
wal_file_.reset();
}
// Generate new epoch id and save the last one to the history.
if (epoch_history_.size() == kEpochHistoryRetention) {
epoch_history_.pop_front();
}
epoch_history_.emplace_back(std::exchange(epoch_id_, utils::GenerateUUID()), last_commit_timestamp_);
// TODO: Move out of storage (no need for the lock) <- missing commit_timestamp at a higher level
replication_state_.NewEpoch();
}
utils::FileRetainer::FileLockerAccessor::ret_type InMemoryStorage::IsPathLocked() {
@ -1802,327 +1776,16 @@ utils::FileRetainer::FileLockerAccessor::ret_type InMemoryStorage::UnlockPath()
return true;
}
void storage::InMemoryStorage::ReplicationState::Reset() {
replication_server_.reset();
replication_clients_.WithLock([&](auto &clients) { clients.clear(); });
auto InMemoryStorage::CreateReplicationClient(std::string name, io::network::Endpoint endpoint,
replication::ReplicationMode mode,
replication::ReplicationClientConfig const &config)
-> std::unique_ptr<ReplicationClient> {
return std::make_unique<InMemoryReplicationClient>(this, std::move(name), endpoint, mode, config);
}
bool storage::InMemoryStorage::ReplicationState::SetMainReplicationRole(storage::Storage *storage) {
// We don't want to generate new epoch_id and do the
// cleanup if we're already a MAIN
if (GetRole() == replication::ReplicationRole::MAIN) {
return false;
}
// Main instance does not need replication server
// This should be always called first so we finalize everything
replication_server_.reset(nullptr);
storage->EstablishNewEpoch();
if (ShouldStoreAndRestoreReplicationState()) {
// Only thing that matters here is the role saved as MAIN
auto data = replication::ReplicationStatusToJSON(
replication::ReplicationStatus{.name = replication::kReservedReplicationRoleName,
.ip_address = "",
.port = 0,
.sync_mode = replication::ReplicationMode::SYNC,
.replica_check_frequency = std::chrono::seconds(0),
.ssl = std::nullopt,
.role = replication::ReplicationRole::MAIN});
if (!durability_->Put(replication::kReservedReplicationRoleName, data.dump())) {
spdlog::error("Error when saving MAIN replication role in settings.");
return false;
}
}
SetRole(replication::ReplicationRole::MAIN);
return true;
}
bool storage::InMemoryStorage::ReplicationState::AppendToWalDataDefinition(const uint64_t seq_num,
durability::StorageGlobalOperation operation,
LabelId label,
const std::set<PropertyId> &properties,
uint64_t final_commit_timestamp) {
bool finalized_on_all_replicas = true;
// TODO Should we return true if not MAIN?
if (GetRole() == replication::ReplicationRole::MAIN) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->StartTransactionReplication(seq_num);
client->IfStreamingTransaction(
[&](auto &stream) { stream.AppendOperation(operation, label, properties, final_commit_timestamp); });
const auto finalized = client->FinalizeTransactionReplication();
if (client->Mode() == replication::ReplicationMode::SYNC) {
finalized_on_all_replicas = finalized && finalized_on_all_replicas;
}
}
});
}
return finalized_on_all_replicas;
}
void storage::InMemoryStorage::ReplicationState::InitializeTransaction(uint64_t seq_num) {
if (GetRole() == replication::ReplicationRole::MAIN) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->StartTransactionReplication(seq_num);
}
});
}
}
void storage::InMemoryStorage::ReplicationState::AppendDelta(const Delta &delta, const Edge &parent,
uint64_t timestamp) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->IfStreamingTransaction([&](auto &stream) { stream.AppendDelta(delta, parent, timestamp); });
}
});
}
void storage::InMemoryStorage::ReplicationState::AppendDelta(const Delta &delta, const Vertex &parent,
uint64_t timestamp) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->IfStreamingTransaction([&](auto &stream) { stream.AppendDelta(delta, parent, timestamp); });
}
});
}
bool storage::InMemoryStorage::ReplicationState::FinalizeTransaction(uint64_t timestamp) {
bool finalized_on_all_replicas = true;
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->IfStreamingTransaction([&](auto &stream) { stream.AppendTransactionEnd(timestamp); });
const auto finalized = client->FinalizeTransactionReplication();
if (client->Mode() == replication::ReplicationMode::SYNC) {
finalized_on_all_replicas = finalized && finalized_on_all_replicas;
}
}
});
return finalized_on_all_replicas;
}
utils::BasicResult<InMemoryStorage::ReplicationState::RegisterReplicaError>
InMemoryStorage::ReplicationState::RegisterReplica(std::string name, io::network::Endpoint endpoint,
const replication::ReplicationMode replication_mode,
const replication::RegistrationMode registration_mode,
const replication::ReplicationClientConfig &config,
InMemoryStorage *storage) {
MG_ASSERT(GetRole() == replication::ReplicationRole::MAIN, "Only main instance can register a replica!");
const bool name_exists = replication_clients_.WithLock([&](auto &clients) {
return std::any_of(clients.begin(), clients.end(), [&name](const auto &client) { return client->Name() == name; });
});
if (name_exists) {
return RegisterReplicaError::NAME_EXISTS;
}
const auto end_point_exists = replication_clients_.WithLock([&endpoint](auto &clients) {
return std::any_of(clients.begin(), clients.end(),
[&endpoint](const auto &client) { return client->Endpoint() == endpoint; });
});
if (end_point_exists) {
return RegisterReplicaError::END_POINT_EXISTS;
}
if (ShouldStoreAndRestoreReplicationState()) {
auto data = replication::ReplicationStatusToJSON(
replication::ReplicationStatus{.name = name,
.ip_address = endpoint.address,
.port = endpoint.port,
.sync_mode = replication_mode,
.replica_check_frequency = config.replica_check_frequency,
.ssl = config.ssl,
.role = replication::ReplicationRole::REPLICA});
if (!durability_->Put(name, data.dump())) {
spdlog::error("Error when saving replica {} in settings.", name);
return RegisterReplicaError::COULD_NOT_BE_PERSISTED;
}
}
auto client = std::make_unique<ReplicationClient>(std::move(name), storage, endpoint, replication_mode, config);
if (client->State() == replication::ReplicaState::INVALID) {
if (replication::RegistrationMode::CAN_BE_INVALID != registration_mode) {
return RegisterReplicaError::CONNECTION_FAILED;
}
spdlog::warn("Connection failed when registering replica {}. Replica will still be registered.", client->Name());
}
return replication_clients_.WithLock(
[&](auto &clients) -> utils::BasicResult<InMemoryStorage::ReplicationState::RegisterReplicaError> {
// Another thread could have added a client with same name while
// we were connecting to this client.
if (std::any_of(clients.begin(), clients.end(),
[&](const auto &other_client) { return client->Name() == other_client->Name(); })) {
return RegisterReplicaError::NAME_EXISTS;
}
if (std::any_of(clients.begin(), clients.end(), [&client](const auto &other_client) {
return client->Endpoint() == other_client->Endpoint();
})) {
return RegisterReplicaError::END_POINT_EXISTS;
}
clients.push_back(std::move(client));
return {};
});
}
bool InMemoryStorage::ReplicationState::SetReplicaRole(io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config,
InMemoryStorage *storage) {
// We don't want to restart the server if we're already a REPLICA
if (GetRole() == replication::ReplicationRole::REPLICA) {
return false;
}
auto port = endpoint.port; // assigning because we will move the endpoint
replication_server_ = std::make_unique<ReplicationServer>(storage, std::move(endpoint), config);
if (ShouldStoreAndRestoreReplicationState()) {
// Only thing that matters here is the role saved as REPLICA and the listening port
auto data = replication::ReplicationStatusToJSON(
replication::ReplicationStatus{.name = replication::kReservedReplicationRoleName,
.ip_address = "",
.port = port,
.sync_mode = replication::ReplicationMode::SYNC,
.replica_check_frequency = std::chrono::seconds(0),
.ssl = std::nullopt,
.role = replication::ReplicationRole::REPLICA});
if (!durability_->Put(replication::kReservedReplicationRoleName, data.dump())) {
spdlog::error("Error when saving REPLICA replication role in settings.");
return false;
}
}
SetRole(replication::ReplicationRole::REPLICA);
return true;
}
bool InMemoryStorage::ReplicationState::UnregisterReplica(std::string_view name) {
MG_ASSERT(GetRole() == replication::ReplicationRole::MAIN, "Only main instance can unregister a replica!");
if (ShouldStoreAndRestoreReplicationState()) {
if (!durability_->Delete(name)) {
spdlog::error("Error when removing replica {} from settings.", name);
return false;
}
}
return replication_clients_.WithLock([&](auto &clients) {
return std::erase_if(clients, [&](const auto &client) { return client->Name() == name; });
});
}
std::optional<replication::ReplicaState> InMemoryStorage::ReplicationState::GetReplicaState(
const std::string_view name) {
return replication_clients_.WithLock([&](auto &clients) -> std::optional<replication::ReplicaState> {
const auto client_it =
std::find_if(clients.cbegin(), clients.cend(), [name](auto &client) { return client->Name() == name; });
if (client_it == clients.cend()) {
return std::nullopt;
}
return (*client_it)->State();
});
}
std::vector<InMemoryStorage::ReplicaInfo> InMemoryStorage::ReplicationState::ReplicasInfo() {
return replication_clients_.WithLock([](auto &clients) {
std::vector<InMemoryStorage::ReplicaInfo> replica_info;
replica_info.reserve(clients.size());
std::transform(
clients.begin(), clients.end(), std::back_inserter(replica_info), [](const auto &client) -> ReplicaInfo {
return {client->Name(), client->Mode(), client->Endpoint(), client->State(), client->GetTimestampInfo()};
});
return replica_info;
});
}
void InMemoryStorage::ReplicationState::RestoreReplicationRole(InMemoryStorage *storage) {
if (!ShouldStoreAndRestoreReplicationState()) {
return;
}
spdlog::info("Restoring replication role.");
uint16_t port = replication::kDefaultReplicationPort;
const auto replication_data = durability_->Get(replication::kReservedReplicationRoleName);
if (!replication_data.has_value()) {
spdlog::debug("Cannot find data needed for restore replication role in persisted metadata.");
return;
}
const auto maybe_replication_status = replication::JSONToReplicationStatus(nlohmann::json::parse(*replication_data));
if (!maybe_replication_status.has_value()) {
LOG_FATAL("Cannot parse previously saved configuration of replication role {}.",
replication::kReservedReplicationRoleName);
}
const auto replication_status = *maybe_replication_status;
if (!replication_status.role.has_value()) {
SetRole(replication::ReplicationRole::MAIN);
} else {
SetRole(*replication_status.role);
port = replication_status.port;
}
if (GetRole() == replication::ReplicationRole::REPLICA) {
io::network::Endpoint endpoint(replication::kDefaultReplicationServerIp, port);
replication_server_ =
std::make_unique<ReplicationServer>(storage, std::move(endpoint), replication::ReplicationServerConfig{});
}
spdlog::info("Replication role restored to {}.",
GetRole() == replication::ReplicationRole::MAIN ? "MAIN" : "REPLICA");
}
void InMemoryStorage::ReplicationState::RestoreReplicas(InMemoryStorage *storage) {
if (!ShouldStoreAndRestoreReplicationState()) {
return;
}
spdlog::info("Restoring replicas.");
for (const auto &[replica_name, replica_data] : *durability_) {
spdlog::info("Restoring replica {}.", replica_name);
const auto maybe_replica_status = replication::JSONToReplicationStatus(nlohmann::json::parse(replica_data));
if (!maybe_replica_status.has_value()) {
LOG_FATAL("Cannot parse previously saved configuration of replica {}.", replica_name);
}
auto replica_status = *maybe_replica_status;
MG_ASSERT(replica_status.name == replica_name, "Expected replica name is '{}', but got '{}'", replica_status.name,
replica_name);
if (replica_name == replication::kReservedReplicationRoleName) {
continue;
}
auto ret =
RegisterReplica(std::move(replica_status.name), {std::move(replica_status.ip_address), replica_status.port},
replica_status.sync_mode, replication::RegistrationMode::CAN_BE_INVALID,
{
.replica_check_frequency = replica_status.replica_check_frequency,
.ssl = replica_status.ssl,
},
storage);
if (ret.HasError()) {
MG_ASSERT(RegisterReplicaError::CONNECTION_FAILED != ret.GetError());
LOG_FATAL("Failure when restoring replica {}: {}.", replica_name, RegisterReplicaErrorToString(ret.GetError()));
}
spdlog::info("Replica {} restored.", replica_name);
}
std::unique_ptr<ReplicationServer> InMemoryStorage::CreateReplicationServer(
io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config) {
return std::make_unique<InMemoryReplicationServer>(this, std::move(endpoint), config);
}
} // namespace memgraph::storage

View File

@ -24,10 +24,9 @@
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
namespace memgraph::storage {
#include "storage/v2/replication/replication.hpp"
class ReplicationServer;
class ReplicationClient;
namespace memgraph::storage {
// The storage is based on this paper:
// https://db.in.tum.de/~muehlbau/papers/mvcc.pdf
@ -35,23 +34,11 @@ class ReplicationClient;
// only implement snapshot isolation for transactions.
class InMemoryStorage final : public Storage {
friend class ReplicationServer;
friend class InMemoryReplicationServer;
friend class InMemoryReplicationClient;
friend class ReplicationClient;
public:
struct TimestampInfo {
uint64_t current_timestamp_of_replica;
uint64_t current_number_of_timestamp_behind_master;
};
struct ReplicaInfo {
std::string name;
replication::ReplicationMode mode;
io::network::Endpoint endpoint;
replication::ReplicaState state;
TimestampInfo timestamp_info;
};
enum class CreateSnapshotError : uint8_t {
DisabledForReplica,
DisabledForAnalyticsPeriodicCommit,
@ -413,6 +400,13 @@ class InMemoryStorage final : public Storage {
Transaction CreateTransaction(IsolationLevel isolation_level, StorageMode storage_mode) override;
auto CreateReplicationClient(std::string name, io::network::Endpoint endpoint, replication::ReplicationMode mode,
replication::ReplicationClientConfig const &config)
-> std::unique_ptr<ReplicationClient> override;
auto CreateReplicationServer(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config)
-> std::unique_ptr<ReplicationServer> override;
private:
/// The force parameter determines the behaviour of the garbage collector.
/// If it's set to true, it will behave as a global operation, i.e. it can't
@ -465,29 +459,6 @@ class InMemoryStorage final : public Storage {
// Sequence number used to keep track of the chain of WALs.
uint64_t wal_seq_num_{0};
// UUID to distinguish different main instance runs for replication process
// on SAME storage.
// Multiple instances can have same storage UUID and be MAIN at the same time.
// We cannot compare commit timestamps of those instances if one of them
// becomes the replica of the other so we use epoch_id_ as additional
// discriminating property.
// Example of this:
// We have 2 instances of the same storage, S1 and S2.
// S1 and S2 are MAIN and accept their own commits and write them to the WAL.
// At the moment when S1 commited a transaction with timestamp 20, and S2
// a different transaction with timestamp 15, we change S2's role to REPLICA
// and register it on S1.
// Without using the epoch_id, we don't know that S1 and S2 have completely
// different transactions, we think that the S2 is behind only by 5 commits.
std::string epoch_id_; // TODO: Move to replication level
// Questions:
// - storage durability <- databases/*name*/wal and snapshots (where this for epoch_id)
// - multi-tenant durability <- databases/.durability (there is a list of all active tenants)
// History of the previous epoch ids.
// Each value consists of the epoch id along the last commit belonging to that
// epoch.
std::deque<std::pair<std::string, uint64_t>> epoch_history_;
std::optional<durability::WalFile> wal_file_;
uint64_t wal_unsynced_transactions_{0};
@ -524,81 +495,6 @@ class InMemoryStorage final : public Storage {
std::atomic<bool> gc_full_scan_vertices_delete_ = false;
std::atomic<bool> gc_full_scan_edges_delete_ = false;
std::atomic<uint64_t> last_commit_timestamp_{kTimestampInitialId};
public:
struct ReplicationState {
enum class RegisterReplicaError : uint8_t {
NAME_EXISTS,
END_POINT_EXISTS,
CONNECTION_FAILED,
COULD_NOT_BE_PERSISTED
};
// TODO Move to private (needed for Storage construction)
std::unique_ptr<kvstore::KVStore> durability_;
// Generic API
void Reset();
// TODO: Just check if server exists -> you are REPLICA
replication::ReplicationRole GetRole() const { return replication_role_.load(); }
bool SetMainReplicationRole(Storage *storage); // Set the instance to MAIN
// TODO: ReplicationServer/Client uses InMemoryStorage* for RPC callbacks
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config,
InMemoryStorage *storage); // Sets the instance to REPLICA
// Generic restoration
void RestoreReplicationRole(InMemoryStorage *storage);
// MAIN actually doing the replication
bool AppendToWalDataDefinition(uint64_t seq_num, durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
void InitializeTransaction(uint64_t seq_num);
void AppendDelta(const Delta &delta, const Vertex &parent, uint64_t timestamp);
void AppendDelta(const Delta &delta, const Edge &parent, uint64_t timestamp);
bool FinalizeTransaction(uint64_t timestamp);
// MAIN connecting to replicas
utils::BasicResult<RegisterReplicaError> RegisterReplica(std::string name, io::network::Endpoint endpoint,
replication::ReplicationMode replication_mode,
replication::RegistrationMode registration_mode,
const replication::ReplicationClientConfig &config,
InMemoryStorage *storage);
bool UnregisterReplica(std::string_view name);
// MAIN reconnecting to replicas
void RestoreReplicas(InMemoryStorage *storage);
// MAIN getting info from replicas
// TODO make into const (problem with SpinLock and WithReadLock)
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
std::vector<InMemoryStorage::ReplicaInfo> ReplicasInfo();
private:
bool ShouldStoreAndRestoreReplicationState() const { return nullptr != durability_; }
void SetRole(replication::ReplicationRole role) { return replication_role_.store(role); }
// NOTE: Server is not in MAIN it is in REPLICA
std::unique_ptr<ReplicationServer> replication_server_{nullptr};
// We create ReplicationClient using unique_ptr so we can move
// newly created client into the vector.
// We cannot move the client directly because it contains ThreadPool
// which cannot be moved. Also, the move is necessary because
// we don't want to create the client directly inside the vector
// because that would require the lock on the list putting all
// commits (they iterate list of clients) to halt.
// This way we can initialize client in main thread which means
// that we can immediately notify the user if the initialization
// failed.
using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
ReplicationClientList replication_clients_;
std::atomic<replication::ReplicationRole> replication_role_{replication::ReplicationRole::MAIN};
};
private:
ReplicationState replication_state_;
};

View File

@ -0,0 +1,65 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include <cstdint>
#include <deque>
#include <string>
#include <utility>
#include "io/network/endpoint.hpp"
#include "storage/v2/replication/enums.hpp"
#include "utils/uuid.hpp"
// TODO: move to replication namespace and unify
namespace memgraph::storage {
// TODO: Should be at MAIN instance level; shouldn't be connected to storage
struct ReplicationEpoch {
ReplicationEpoch() : id(utils::GenerateUUID()) {}
// UUID to distinguish different main instance runs for replication process
// on SAME storage.
// Multiple instances can have same storage UUID and be MAIN at the same time.
// We cannot compare commit timestamps of those instances if one of them
// becomes the replica of the other so we use epoch_id_ as additional
// discriminating property.
// Example of this:
// We have 2 instances of the same storage, S1 and S2.
// S1 and S2 are MAIN and accept their own commits and write them to the WAL.
// At the moment when S1 commited a transaction with timestamp 20, and S2
// a different transaction with timestamp 15, we change S2's role to REPLICA
// and register it on S1.
// Without using the epoch_id, we don't know that S1 and S2 have completely
// different transactions, we think that the S2 is behind only by 5 commits.
std::string id; // TODO: Move to replication level
// Generates a new epoch id, returning the old one
std::string NewEpoch() { return std::exchange(id, utils::GenerateUUID()); }
std::string SetEpoch(std::string new_epoch) { return std::exchange(id, std::move(new_epoch)); }
};
struct TimestampInfo {
uint64_t current_timestamp_of_replica;
uint64_t current_number_of_timestamp_behind_master;
};
struct ReplicaInfo {
std::string name;
replication::ReplicationMode mode;
io::network::Endpoint endpoint;
replication::ReplicaState state;
TimestampInfo timestamp_info;
};
} // namespace memgraph::storage

View File

@ -0,0 +1,402 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include "storage/v2/replication/replication.hpp"
#include "storage/v2/constraints/constraints.hpp"
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/edge_direction.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/storage_mode.hpp"
#include "storage/v2/vertex_accessor.hpp"
#include "utils/stat.hpp"
/// REPLICATION ///
#include "storage/v2/replication/replication_client.hpp"
#include "storage/v2/replication/replication_server.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/storage_error.hpp"
#include "storage/v2/inmemory/replication/replication_client.hpp"
#include "storage/v2/inmemory/replication/replication_server.hpp"
namespace memgraph::storage {
using OOMExceptionEnabler = utils::MemoryTracker::OutOfMemoryExceptionEnabler;
namespace {
std::string RegisterReplicaErrorToString(ReplicationState::RegisterReplicaError error) {
using enum ReplicationState::RegisterReplicaError;
switch (error) {
case NAME_EXISTS:
return "NAME_EXISTS";
case END_POINT_EXISTS:
return "END_POINT_EXISTS";
case CONNECTION_FAILED:
return "CONNECTION_FAILED";
case COULD_NOT_BE_PERSISTED:
return "COULD_NOT_BE_PERSISTED";
}
}
} // namespace
storage::ReplicationState::ReplicationState(bool restore, std::filesystem::path durability_dir) {
if (restore) {
utils::EnsureDirOrDie(durability_dir / durability::kReplicationDirectory);
durability_ = std::make_unique<kvstore::KVStore>(durability_dir / durability::kReplicationDirectory);
}
}
void storage::ReplicationState::Reset() {
replication_server_.reset();
replication_clients_.WithLock([&](auto &clients) { clients.clear(); });
}
bool storage::ReplicationState::SetMainReplicationRole(storage::Storage *storage) {
// We don't want to generate new epoch_id and do the
// cleanup if we're already a MAIN
if (GetRole() == replication::ReplicationRole::MAIN) {
return false;
}
// Main instance does not need replication server
// This should be always called first so we finalize everything
replication_server_.reset(nullptr);
storage->EstablishNewEpoch();
if (ShouldStoreAndRestoreReplicationState()) {
// Only thing that matters here is the role saved as MAIN
auto data = replication::ReplicationStatusToJSON(
replication::ReplicationStatus{.name = replication::kReservedReplicationRoleName,
.ip_address = "",
.port = 0,
.sync_mode = replication::ReplicationMode::SYNC,
.replica_check_frequency = std::chrono::seconds(0),
.ssl = std::nullopt,
.role = replication::ReplicationRole::MAIN});
if (!durability_->Put(replication::kReservedReplicationRoleName, data.dump())) {
spdlog::error("Error when saving MAIN replication role in settings.");
return false;
}
}
SetRole(replication::ReplicationRole::MAIN);
return true;
}
bool storage::ReplicationState::AppendToWalDataDefinition(const uint64_t seq_num,
durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties,
uint64_t final_commit_timestamp) {
bool finalized_on_all_replicas = true;
// TODO Should we return true if not MAIN?
if (GetRole() == replication::ReplicationRole::MAIN) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->StartTransactionReplication(seq_num);
client->IfStreamingTransaction(
[&](auto &stream) { stream.AppendOperation(operation, label, properties, final_commit_timestamp); });
const auto finalized = client->FinalizeTransactionReplication();
if (client->Mode() == replication::ReplicationMode::SYNC) {
finalized_on_all_replicas = finalized && finalized_on_all_replicas;
}
}
});
}
return finalized_on_all_replicas;
}
void storage::ReplicationState::InitializeTransaction(uint64_t seq_num) {
if (GetRole() == replication::ReplicationRole::MAIN) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->StartTransactionReplication(seq_num);
}
});
}
}
void storage::ReplicationState::AppendDelta(const Delta &delta, const Edge &parent, uint64_t timestamp) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->IfStreamingTransaction([&](auto &stream) { stream.AppendDelta(delta, parent, timestamp); });
}
});
}
void storage::ReplicationState::AppendDelta(const Delta &delta, const Vertex &parent, uint64_t timestamp) {
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->IfStreamingTransaction([&](auto &stream) { stream.AppendDelta(delta, parent, timestamp); });
}
});
}
bool storage::ReplicationState::FinalizeTransaction(uint64_t timestamp) {
bool finalized_on_all_replicas = true;
replication_clients_.WithLock([&](auto &clients) {
for (auto &client : clients) {
client->IfStreamingTransaction([&](auto &stream) { stream.AppendTransactionEnd(timestamp); });
const auto finalized = client->FinalizeTransactionReplication();
if (client->Mode() == replication::ReplicationMode::SYNC) {
finalized_on_all_replicas = finalized && finalized_on_all_replicas;
}
}
});
return finalized_on_all_replicas;
}
utils::BasicResult<ReplicationState::RegisterReplicaError> ReplicationState::RegisterReplica(
std::string name, io::network::Endpoint endpoint, const replication::ReplicationMode replication_mode,
const replication::RegistrationMode registration_mode, const replication::ReplicationClientConfig &config,
Storage *storage) {
MG_ASSERT(GetRole() == replication::ReplicationRole::MAIN, "Only main instance can register a replica!");
const bool name_exists = replication_clients_.WithLock([&](auto &clients) {
return std::any_of(clients.begin(), clients.end(), [&name](const auto &client) { return client->Name() == name; });
});
if (name_exists) {
return RegisterReplicaError::NAME_EXISTS;
}
const auto end_point_exists = replication_clients_.WithLock([&endpoint](auto &clients) {
return std::any_of(clients.begin(), clients.end(),
[&endpoint](const auto &client) { return client->Endpoint() == endpoint; });
});
if (end_point_exists) {
return RegisterReplicaError::END_POINT_EXISTS;
}
if (ShouldStoreAndRestoreReplicationState()) {
auto data = replication::ReplicationStatusToJSON(
replication::ReplicationStatus{.name = name,
.ip_address = endpoint.address,
.port = endpoint.port,
.sync_mode = replication_mode,
.replica_check_frequency = config.replica_check_frequency,
.ssl = config.ssl,
.role = replication::ReplicationRole::REPLICA});
if (!durability_->Put(name, data.dump())) {
spdlog::error("Error when saving replica {} in settings.", name);
return RegisterReplicaError::COULD_NOT_BE_PERSISTED;
}
}
auto client = storage->CreateReplicationClient(std::move(name), endpoint, replication_mode, config);
client->Start();
if (client->State() == replication::ReplicaState::INVALID) {
if (replication::RegistrationMode::CAN_BE_INVALID != registration_mode) {
return RegisterReplicaError::CONNECTION_FAILED;
}
spdlog::warn("Connection failed when registering replica {}. Replica will still be registered.", client->Name());
}
return replication_clients_.WithLock(
[&](auto &clients) -> utils::BasicResult<ReplicationState::RegisterReplicaError> {
// Another thread could have added a client with same name while
// we were connecting to this client.
if (std::any_of(clients.begin(), clients.end(),
[&](const auto &other_client) { return client->Name() == other_client->Name(); })) {
return RegisterReplicaError::NAME_EXISTS;
}
if (std::any_of(clients.begin(), clients.end(), [&client](const auto &other_client) {
return client->Endpoint() == other_client->Endpoint();
})) {
return RegisterReplicaError::END_POINT_EXISTS;
}
clients.push_back(std::move(client));
return {};
});
}
bool ReplicationState::SetReplicaRole(io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config, Storage *storage) {
// We don't want to restart the server if we're already a REPLICA
if (GetRole() == replication::ReplicationRole::REPLICA) {
return false;
}
auto port = endpoint.port; // assigning because we will move the endpoint
replication_server_ = storage->CreateReplicationServer(std::move(endpoint), config);
bool res = replication_server_->Start();
if (!res) {
spdlog::error("Unable to start the replication server.");
return false;
}
if (ShouldStoreAndRestoreReplicationState()) {
// Only thing that matters here is the role saved as REPLICA and the listening port
auto data = replication::ReplicationStatusToJSON(
replication::ReplicationStatus{.name = replication::kReservedReplicationRoleName,
.ip_address = "",
.port = port,
.sync_mode = replication::ReplicationMode::SYNC,
.replica_check_frequency = std::chrono::seconds(0),
.ssl = std::nullopt,
.role = replication::ReplicationRole::REPLICA});
if (!durability_->Put(replication::kReservedReplicationRoleName, data.dump())) {
spdlog::error("Error when saving REPLICA replication role in settings.");
return false;
}
}
SetRole(replication::ReplicationRole::REPLICA);
return true;
}
bool ReplicationState::UnregisterReplica(std::string_view name) {
MG_ASSERT(GetRole() == replication::ReplicationRole::MAIN, "Only main instance can unregister a replica!");
if (ShouldStoreAndRestoreReplicationState()) {
if (!durability_->Delete(name)) {
spdlog::error("Error when removing replica {} from settings.", name);
return false;
}
}
return replication_clients_.WithLock([&](auto &clients) {
return std::erase_if(clients, [&](const auto &client) { return client->Name() == name; });
});
}
std::optional<replication::ReplicaState> ReplicationState::GetReplicaState(const std::string_view name) {
return replication_clients_.WithLock([&](auto &clients) -> std::optional<replication::ReplicaState> {
const auto client_it =
std::find_if(clients.cbegin(), clients.cend(), [name](auto &client) { return client->Name() == name; });
if (client_it == clients.cend()) {
return std::nullopt;
}
return (*client_it)->State();
});
}
std::vector<ReplicaInfo> ReplicationState::ReplicasInfo() {
return replication_clients_.WithLock([](auto &clients) {
std::vector<ReplicaInfo> replica_info;
replica_info.reserve(clients.size());
std::transform(
clients.begin(), clients.end(), std::back_inserter(replica_info), [](const auto &client) -> ReplicaInfo {
return {client->Name(), client->Mode(), client->Endpoint(), client->State(), client->GetTimestampInfo()};
});
return replica_info;
});
}
void ReplicationState::RestoreReplicationRole(Storage *storage) {
if (!ShouldStoreAndRestoreReplicationState()) {
return;
}
spdlog::info("Restoring replication role.");
uint16_t port = replication::kDefaultReplicationPort;
const auto replication_data = durability_->Get(replication::kReservedReplicationRoleName);
if (!replication_data.has_value()) {
spdlog::debug("Cannot find data needed for restore replication role in persisted metadata.");
return;
}
const auto maybe_replication_status = replication::JSONToReplicationStatus(nlohmann::json::parse(*replication_data));
if (!maybe_replication_status.has_value()) {
LOG_FATAL("Cannot parse previously saved configuration of replication role {}.",
replication::kReservedReplicationRoleName);
}
const auto replication_status = *maybe_replication_status;
if (!replication_status.role.has_value()) {
SetRole(replication::ReplicationRole::MAIN);
} else {
SetRole(*replication_status.role);
port = replication_status.port;
}
if (GetRole() == replication::ReplicationRole::REPLICA) {
io::network::Endpoint endpoint(replication::kDefaultReplicationServerIp, port);
replication_server_ = storage->CreateReplicationServer(std::move(endpoint), {});
bool res = replication_server_->Start();
if (!res) {
LOG_FATAL("Unable to start the replication server.");
}
}
spdlog::info("Replication role restored to {}.",
GetRole() == replication::ReplicationRole::MAIN ? "MAIN" : "REPLICA");
}
void ReplicationState::RestoreReplicas(Storage *storage) {
if (!ShouldStoreAndRestoreReplicationState()) {
return;
}
spdlog::info("Restoring replicas.");
for (const auto &[replica_name, replica_data] : *durability_) {
spdlog::info("Restoring replica {}.", replica_name);
const auto maybe_replica_status = replication::JSONToReplicationStatus(nlohmann::json::parse(replica_data));
if (!maybe_replica_status.has_value()) {
LOG_FATAL("Cannot parse previously saved configuration of replica {}.", replica_name);
}
auto replica_status = *maybe_replica_status;
MG_ASSERT(replica_status.name == replica_name, "Expected replica name is '{}', but got '{}'", replica_status.name,
replica_name);
if (replica_name == replication::kReservedReplicationRoleName) {
continue;
}
auto ret =
RegisterReplica(std::move(replica_status.name), {std::move(replica_status.ip_address), replica_status.port},
replica_status.sync_mode, replication::RegistrationMode::CAN_BE_INVALID,
{
.replica_check_frequency = replica_status.replica_check_frequency,
.ssl = replica_status.ssl,
},
storage);
if (ret.HasError()) {
MG_ASSERT(RegisterReplicaError::CONNECTION_FAILED != ret.GetError());
LOG_FATAL("Failure when restoring replica {}: {}.", replica_name, RegisterReplicaErrorToString(ret.GetError()));
}
spdlog::info("Replica {} restored.", replica_name);
}
}
constexpr uint16_t kEpochHistoryRetention = 1000;
void ReplicationState::NewEpoch() {
// Generate new epoch id and save the last one to the history.
if (history.size() == kEpochHistoryRetention) {
history.pop_front();
}
auto prevEpoch = epoch_.NewEpoch();
history.emplace_back(std::move(prevEpoch), last_commit_timestamp_);
}
void ReplicationState::AppendEpoch(std::string new_epoch) {
auto prevEpoch = epoch_.SetEpoch(std::move(new_epoch));
history.emplace_back(std::move(prevEpoch), last_commit_timestamp_);
}
} // namespace memgraph::storage

View File

@ -0,0 +1,127 @@
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
// License, and you may not use this file except in compliance with the Business Source License.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#pragma once
#include "storage/v2/inmemory/label_index.hpp"
#include "storage/v2/inmemory/label_property_index.hpp"
#include "storage/v2/storage.hpp"
/// REPLICATION ///
#include "rpc/server.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/replication_persistence_helper.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/replication/global.hpp"
// TODO use replication namespace
namespace memgraph::storage {
class Storage;
class ReplicationServer;
class ReplicationClient;
struct ReplicationState {
enum class RegisterReplicaError : uint8_t {
NAME_EXISTS,
END_POINT_EXISTS,
CONNECTION_FAILED,
COULD_NOT_BE_PERSISTED
};
// TODO: This mirrors the logic in InMemoryConstructor; make it independent
ReplicationState(bool restore, std::filesystem::path durability_dir);
// Generic API
void Reset();
// TODO: Just check if server exists -> you are REPLICA
replication::ReplicationRole GetRole() const { return replication_role_.load(); }
bool SetMainReplicationRole(Storage *storage); // Set the instance to MAIN
// TODO: ReplicationServer/Client uses Storage* for RPC callbacks
bool SetReplicaRole(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config,
Storage *storage); // Sets the instance to REPLICA
// Generic restoration
void RestoreReplicationRole(Storage *storage);
// MAIN actually doing the replication
bool AppendToWalDataDefinition(uint64_t seq_num, durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t final_commit_timestamp);
void InitializeTransaction(uint64_t seq_num);
void AppendDelta(const Delta &delta, const Vertex &parent, uint64_t timestamp);
void AppendDelta(const Delta &delta, const Edge &parent, uint64_t timestamp);
bool FinalizeTransaction(uint64_t timestamp);
// MAIN connecting to replicas
utils::BasicResult<RegisterReplicaError> RegisterReplica(std::string name, io::network::Endpoint endpoint,
const replication::ReplicationMode replication_mode,
const replication::RegistrationMode registration_mode,
const replication::ReplicationClientConfig &config,
Storage *storage);
bool UnregisterReplica(std::string_view name);
// MAIN reconnecting to replicas
void RestoreReplicas(Storage *storage);
// MAIN getting info from replicas
// TODO make into const (problem with SpinLock and WithReadLock)
std::optional<replication::ReplicaState> GetReplicaState(std::string_view name);
std::vector<ReplicaInfo> ReplicasInfo();
const ReplicationEpoch &GetEpoch() const { return epoch_; }
ReplicationEpoch &GetEpoch() { return epoch_; }
// Questions:
// - storage durability <- databases/*name*/wal and snapshots (where this for epoch_id)
// - multi-tenant durability <- databases/.durability (there is a list of all active tenants)
// History of the previous epoch ids.
// Each value consists of the epoch id along the last commit belonging to that
// epoch.
std::deque<std::pair<std::string, uint64_t>> history;
// TODO: actually durability
std::atomic<uint64_t> last_commit_timestamp_{kTimestampInitialId};
void NewEpoch();
void AppendEpoch(std::string new_epoch);
private:
bool ShouldStoreAndRestoreReplicationState() const { return nullptr != durability_; }
void SetRole(replication::ReplicationRole role) { return replication_role_.store(role); }
// NOTE: Server is not in MAIN it is in REPLICA
std::unique_ptr<ReplicationServer> replication_server_{nullptr};
// We create ReplicationClient using unique_ptr so we can move
// newly created client into the vector.
// We cannot move the client directly because it contains ThreadPool
// which cannot be moved. Also, the move is necessary because
// we don't want to create the client directly inside the vector
// because that would require the lock on the list putting all
// commits (they iterate list of clients) to halt.
// This way we can initialize client in main thread which means
// that we can immediately notify the user if the initialization
// failed.
using ReplicationClientList = utils::Synchronized<std::vector<std::unique_ptr<ReplicationClient>>, utils::SpinLock>;
ReplicationClientList replication_clients_;
std::atomic<replication::ReplicationRole> replication_role_{replication::ReplicationRole::MAIN};
std::unique_ptr<kvstore::KVStore> durability_;
ReplicationEpoch epoch_;
};
} // namespace memgraph::storage

View File

@ -15,6 +15,7 @@
#include <type_traits>
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/transaction.hpp"
@ -24,581 +25,56 @@
namespace memgraph::storage {
namespace {
template <typename>
[[maybe_unused]] inline constexpr bool always_false_v = false;
} // namespace
////// ReplicationClient //////
ReplicationClient::ReplicationClient(std::string name, InMemoryStorage *storage, const io::network::Endpoint &endpoint,
const replication::ReplicationMode mode,
const replication::ReplicationClientConfig &config)
: name_(std::move(name)), storage_(storage), mode_(mode) {
spdlog::trace("Replication client started at: {}:{}", endpoint.address, endpoint.port);
if (config.ssl) {
rpc_context_.emplace(config.ssl->key_file, config.ssl->cert_file);
} else {
rpc_context_.emplace();
}
rpc_client_.emplace(endpoint, &*rpc_context_);
TryInitializeClientSync();
// Help the user to get the most accurate replica state possible.
if (config.replica_check_frequency > std::chrono::seconds(0)) {
replica_checker_.Run("Replica Checker", config.replica_check_frequency, [&] { FrequentCheck(); });
}
static auto CreateClientContext(const replication::ReplicationClientConfig &config) -> communication::ClientContext {
return (config.ssl) ? communication::ClientContext{config.ssl->key_file, config.ssl->cert_file}
: communication::ClientContext{};
}
void ReplicationClient::TryInitializeClientAsync() {
thread_pool_.AddTask([this] {
rpc_client_->Abort();
this->TryInitializeClientSync();
});
}
ReplicationClient::ReplicationClient(std::string name, memgraph::io::network::Endpoint endpoint,
replication::ReplicationMode mode,
replication::ReplicationClientConfig const &config)
: name_{std::move(name)},
rpc_context_{CreateClientContext(config)},
rpc_client_{std::move(endpoint), &rpc_context_},
replica_check_frequency_{config.replica_check_frequency},
mode_{mode} {}
void ReplicationClient::FrequentCheck() {
const auto is_success = std::invoke([this]() {
try {
auto stream{rpc_client_->Stream<replication::FrequentHeartbeatRpc>()};
const auto response = stream.AwaitResponse();
return response.success;
} catch (const rpc::RpcFailedException &) {
return false;
}
});
// States: READY, REPLICATING, RECOVERY, INVALID
// If success && ready, replicating, recovery -> stay the same because something good is going on.
// If success && INVALID -> [it's possible that replica came back to life] -> TryInitializeClient.
// If fail -> [replica is not reachable at all] -> INVALID state.
// NOTE: TryInitializeClient might return nothing if there is a branching point.
// NOTE: The early return pattern simplified the code, but the behavior should be as explained.
if (!is_success) {
replica_state_.store(replication::ReplicaState::INVALID);
return;
}
if (replica_state_.load() == replication::ReplicaState::INVALID) {
TryInitializeClientAsync();
}
}
/// @throws rpc::RpcFailedException
void ReplicationClient::InitializeClient() {
uint64_t current_commit_timestamp{kTimestampInitialId};
std::optional<std::string> epoch_id;
{
// epoch_id_ can be changed if we don't take this lock
std::unique_lock engine_guard(storage_->engine_lock_);
epoch_id.emplace(storage_->epoch_id_);
}
auto stream{rpc_client_->Stream<replication::HeartbeatRpc>(storage_->last_commit_timestamp_, std::move(*epoch_id))};
const auto response = stream.AwaitResponse();
std::optional<uint64_t> branching_point;
if (response.epoch_id != storage_->epoch_id_ && response.current_commit_timestamp != kTimestampInitialId) {
const auto &epoch_history = storage_->epoch_history_;
const auto epoch_info_iter =
std::find_if(epoch_history.crbegin(), epoch_history.crend(),
[&](const auto &epoch_info) { return epoch_info.first == response.epoch_id; });
if (epoch_info_iter == epoch_history.crend()) {
branching_point = 0;
} else if (epoch_info_iter->second != response.current_commit_timestamp) {
branching_point = epoch_info_iter->second;
}
}
if (branching_point) {
spdlog::error(
"You cannot register Replica {} to this Main because at one point "
"Replica {} acted as the Main instance. Both the Main and Replica {} "
"now hold unique data. Please resolve data conflicts and start the "
"replication on a clean instance.",
name_, name_, name_);
return;
}
current_commit_timestamp = response.current_commit_timestamp;
spdlog::trace("Current timestamp on replica {}: {}", name_, current_commit_timestamp);
spdlog::trace("Current timestamp on main: {}", storage_->last_commit_timestamp_.load());
if (current_commit_timestamp == storage_->last_commit_timestamp_.load()) {
spdlog::debug("Replica '{}' up to date", name_);
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::READY);
} else {
spdlog::debug("Replica '{}' is behind", name_);
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::RECOVERY);
}
thread_pool_.AddTask([=, this] { this->RecoverReplica(current_commit_timestamp); });
}
}
void ReplicationClient::TryInitializeClientSync() {
try {
InitializeClient();
} catch (const rpc::RpcFailedException &) {
std::unique_lock client_guarde{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
spdlog::error(utils::MessageWithLink("Failed to connect to replica {} at the endpoint {}.", name_,
rpc_client_->Endpoint(), "https://memgr.ph/replication"));
}
}
void ReplicationClient::HandleRpcFailure() {
spdlog::error(utils::MessageWithLink("Couldn't replicate data to {}.", name_, "https://memgr.ph/replication"));
TryInitializeClientAsync();
}
replication::SnapshotRes ReplicationClient::TransferSnapshot(const std::filesystem::path &path) {
auto stream{rpc_client_->Stream<replication::SnapshotRpc>()};
replication::Encoder encoder(stream.GetBuilder());
encoder.WriteFile(path);
return stream.AwaitResponse();
}
replication::WalFilesRes ReplicationClient::TransferWalFiles(const std::vector<std::filesystem::path> &wal_files) {
MG_ASSERT(!wal_files.empty(), "Wal files list is empty!");
auto stream{rpc_client_->Stream<replication::WalFilesRpc>(wal_files.size())};
replication::Encoder encoder(stream.GetBuilder());
for (const auto &wal : wal_files) {
spdlog::debug("Sending wal file: {}", wal);
encoder.WriteFile(wal);
}
return stream.AwaitResponse();
}
void ReplicationClient::StartTransactionReplication(const uint64_t current_wal_seq_num) {
std::unique_lock guard(client_lock_);
const auto status = replica_state_.load();
switch (status) {
case replication::ReplicaState::RECOVERY:
spdlog::debug("Replica {} is behind MAIN instance", name_);
return;
case replication::ReplicaState::REPLICATING:
spdlog::debug("Replica {} missed a transaction", name_);
// We missed a transaction because we're still replicating
// the previous transaction so we need to go to RECOVERY
// state to catch up with the missing transaction
// We cannot queue the recovery process here because
// an error can happen while we're replicating the previous
// transaction after which the client should go to
// INVALID state before starting the recovery process
replica_state_.store(replication::ReplicaState::RECOVERY);
return;
case replication::ReplicaState::INVALID:
HandleRpcFailure();
return;
case replication::ReplicaState::READY:
MG_ASSERT(!replica_stream_);
try {
replica_stream_.emplace(ReplicaStream{this, storage_->last_commit_timestamp_.load(), current_wal_seq_num});
replica_state_.store(replication::ReplicaState::REPLICATING);
} catch (const rpc::RpcFailedException &) {
replica_state_.store(replication::ReplicaState::INVALID);
HandleRpcFailure();
}
return;
}
}
void ReplicationClient::IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback) {
// We can only check the state because it guarantees to be only
// valid during a single transaction replication (if the assumption
// that this and other transaction replication functions can only be
// called from a one thread stands)
if (replica_state_ != replication::ReplicaState::REPLICATING) {
return;
}
try {
callback(*replica_stream_);
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
}
}
bool ReplicationClient::FinalizeTransactionReplication() {
// We can only check the state because it guarantees to be only
// valid during a single transaction replication (if the assumption
// that this and other transaction replication functions can only be
// called from a one thread stands)
if (replica_state_ != replication::ReplicaState::REPLICATING) {
return false;
}
if (mode_ == replication::ReplicationMode::ASYNC) {
thread_pool_.AddTask([this] { static_cast<void>(this->FinalizeTransactionReplicationInternal()); });
return true;
} else {
return FinalizeTransactionReplicationInternal();
}
}
bool ReplicationClient::FinalizeTransactionReplicationInternal() {
MG_ASSERT(replica_stream_, "Missing stream for transaction deltas");
try {
auto response = replica_stream_->Finalize();
replica_stream_.reset();
std::unique_lock client_guard(client_lock_);
if (!response.success || replica_state_ == replication::ReplicaState::RECOVERY) {
replica_state_.store(replication::ReplicaState::RECOVERY);
thread_pool_.AddTask([&, this] { this->RecoverReplica(response.current_commit_timestamp); });
} else {
replica_state_.store(replication::ReplicaState::READY);
return true;
}
} catch (const rpc::RpcFailedException &) {
replica_stream_.reset();
{
std::unique_lock client_guard(client_lock_);
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
}
return false;
}
void ReplicationClient::RecoverReplica(uint64_t replica_commit) {
spdlog::debug("Starting replica recover");
while (true) {
auto file_locker = storage_->file_retainer_.AddLocker();
const auto steps = GetRecoverySteps(replica_commit, &file_locker);
int i = 0;
for (const auto &recovery_step : steps) {
spdlog::trace("Recovering in step: {}", i++);
try {
std::visit(
[&, this]<typename T>(T &&arg) {
using StepType = std::remove_cvref_t<T>;
if constexpr (std::is_same_v<StepType, RecoverySnapshot>) {
spdlog::debug("Sending the latest snapshot file: {}", arg);
auto response = TransferSnapshot(arg);
replica_commit = response.current_commit_timestamp;
} else if constexpr (std::is_same_v<StepType, RecoveryWals>) {
spdlog::debug("Sending the latest wal files");
auto response = TransferWalFiles(arg);
replica_commit = response.current_commit_timestamp;
spdlog::debug("Wal files successfully transferred.");
} else if constexpr (std::is_same_v<StepType, RecoveryCurrentWal>) {
std::unique_lock transaction_guard(storage_->engine_lock_);
if (storage_->wal_file_ && storage_->wal_file_->SequenceNumber() == arg.current_wal_seq_num) {
storage_->wal_file_->DisableFlushing();
transaction_guard.unlock();
spdlog::debug("Sending current wal file");
replica_commit = ReplicateCurrentWal();
storage_->wal_file_->EnableFlushing();
} else {
spdlog::debug("Cannot recover using current wal file");
}
} else {
static_assert(always_false_v<T>, "Missing type from variant visitor");
}
},
recovery_step);
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard{client_lock_};
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure();
return;
}
}
spdlog::trace("Current timestamp on replica: {}", replica_commit);
// To avoid the situation where we read a correct commit timestamp in
// one thread, and after that another thread commits a different a
// transaction and THEN we set the state to READY in the first thread,
// we set this lock before checking the timestamp.
// We will detect that the state is invalid during the next commit,
// because replication::AppendDeltasRpc sends the last commit timestamp which
// replica checks if it's the same last commit timestamp it received
// and we will go to recovery.
// By adding this lock, we can avoid that, and go to RECOVERY immediately.
std::unique_lock client_guard{client_lock_};
SPDLOG_INFO("Replica timestamp: {}", replica_commit);
SPDLOG_INFO("Last commit: {}", storage_->last_commit_timestamp_);
if (storage_->last_commit_timestamp_.load() == replica_commit) {
replica_state_.store(replication::ReplicaState::READY);
return;
}
}
}
uint64_t ReplicationClient::ReplicateCurrentWal() {
const auto &wal_file = storage_->wal_file_;
auto stream = TransferCurrentWalFile();
stream.AppendFilename(wal_file->Path().filename());
utils::InputFile file;
MG_ASSERT(file.Open(storage_->wal_file_->Path()), "Failed to open current WAL file!");
const auto [buffer, buffer_size] = wal_file->CurrentFileBuffer();
stream.AppendSize(file.GetSize() + buffer_size);
stream.AppendFileData(&file);
stream.AppendBufferData(buffer, buffer_size);
auto response = stream.Finalize();
return response.current_commit_timestamp;
}
/// This method tries to find the optimal path for recoverying a single replica.
/// Based on the last commit transfered to replica it tries to update the
/// replica using durability files - WALs and Snapshots. WAL files are much
/// smaller in size as they contain only the Deltas (changes) made during the
/// transactions while Snapshots contain all the data. For that reason we prefer
/// WALs as much as possible. As the WAL file that is currently being updated
/// can change during the process we ignore it as much as possible. Also, it
/// uses the transaction lock so locking it can be really expensive. After we
/// fetch the list of finalized WALs, we try to find the longest chain of
/// sequential WALs, starting from the latest one, that will update the recovery
/// with the all missed updates. If the WAL chain cannot be created, replica is
/// behind by a lot, so we use the regular recovery process, we send the latest
/// snapshot and all the necessary WAL files, starting from the newest WAL that
/// contains a timestamp before the snapshot. If we registered the existence of
/// the current WAL, we add the sequence number we read from it to the recovery
/// process. After all the other steps are finished, if the current WAL contains
/// the same sequence number, it's the same WAL we read while fetching the
/// recovery steps, so we can safely send it to the replica.
/// We assume that the property of preserving at least 1 WAL before the snapshot
/// is satisfied as we extract the timestamp information from it.
std::vector<ReplicationClient::RecoveryStep> ReplicationClient::GetRecoverySteps(
const uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker) {
// First check if we can recover using the current wal file only
// otherwise save the seq_num of the current wal file
// This lock is also necessary to force the missed transaction to finish.
std::optional<uint64_t> current_wal_seq_num;
std::optional<uint64_t> current_wal_from_timestamp;
if (std::unique_lock transtacion_guard(storage_->engine_lock_); storage_->wal_file_) {
current_wal_seq_num.emplace(storage_->wal_file_->SequenceNumber());
current_wal_from_timestamp.emplace(storage_->wal_file_->FromTimestamp());
}
auto locker_acc = file_locker->Access();
auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_, current_wal_seq_num);
MG_ASSERT(wal_files, "Wal files could not be loaded");
auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
std::optional<durability::SnapshotDurabilityInfo> latest_snapshot;
if (!snapshot_files.empty()) {
std::sort(snapshot_files.begin(), snapshot_files.end());
latest_snapshot.emplace(std::move(snapshot_files.back()));
}
std::vector<RecoveryStep> recovery_steps;
// No finalized WAL files were found. This means the difference is contained
// inside the current WAL or the snapshot.
if (wal_files->empty()) {
if (current_wal_from_timestamp && replica_commit >= *current_wal_from_timestamp) {
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
// Without the finalized WAL containing the current timestamp of replica,
// we cannot know if the difference is only in the current WAL or we need
// to send the snapshot.
if (latest_snapshot) {
const auto lock_success = locker_acc.AddPath(latest_snapshot->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_steps.emplace_back(std::in_place_type_t<RecoverySnapshot>{}, std::move(latest_snapshot->path));
}
// if there are no finalized WAL files, snapshot left the current WAL
// as the WAL file containing a transaction before snapshot creation
// so we can be sure that the current WAL is present
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
// Find the longest chain of WALs for recovery.
// The chain consists ONLY of sequential WALs.
auto rwal_it = wal_files->rbegin();
// if the last finalized WAL is before the replica commit
// then we can recovery only from current WAL
if (rwal_it->to_timestamp <= replica_commit) {
MG_ASSERT(current_wal_seq_num);
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
return recovery_steps;
}
uint64_t previous_seq_num{rwal_it->seq_num};
for (; rwal_it != wal_files->rend(); ++rwal_it) {
// If the difference between two consecutive wal files is not 0 or 1
// we have a missing WAL in our chain
if (previous_seq_num - rwal_it->seq_num > 1) {
break;
}
// Find first WAL that contains up to replica commit, i.e. WAL
// that is before the replica commit or conatins the replica commit
// as the last committed transaction OR we managed to find the first WAL
// file.
if (replica_commit >= rwal_it->from_timestamp || rwal_it->seq_num == 0) {
if (replica_commit >= rwal_it->to_timestamp) {
// We want the WAL after because the replica already contains all the
// commits from this WAL
--rwal_it;
}
std::vector<std::filesystem::path> wal_chain;
auto distance_from_first = std::distance(rwal_it, wal_files->rend() - 1);
// We have managed to create WAL chain
// We need to lock these files and add them to the chain
for (auto result_wal_it = wal_files->begin() + distance_from_first; result_wal_it != wal_files->end();
++result_wal_it) {
const auto lock_success = locker_acc.AddPath(result_wal_it->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
wal_chain.push_back(std::move(result_wal_it->path));
}
recovery_steps.emplace_back(std::in_place_type_t<RecoveryWals>{}, std::move(wal_chain));
if (current_wal_seq_num) {
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
}
return recovery_steps;
}
previous_seq_num = rwal_it->seq_num;
}
MG_ASSERT(latest_snapshot, "Invalid durability state, missing snapshot");
// We didn't manage to find a WAL chain, we need to send the latest snapshot
// with its WALs
const auto lock_success = locker_acc.AddPath(latest_snapshot->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_steps.emplace_back(std::in_place_type_t<RecoverySnapshot>{}, std::move(latest_snapshot->path));
std::vector<std::filesystem::path> recovery_wal_files;
auto wal_it = wal_files->begin();
for (; wal_it != wal_files->end(); ++wal_it) {
// Assuming recovery process is correct the snashpot should
// always retain a single WAL that contains a transaction
// before its creation
if (latest_snapshot->start_timestamp < wal_it->to_timestamp) {
if (latest_snapshot->start_timestamp < wal_it->from_timestamp) {
MG_ASSERT(wal_it != wal_files->begin(), "Invalid durability files state");
--wal_it;
}
break;
}
}
for (; wal_it != wal_files->end(); ++wal_it) {
const auto lock_success = locker_acc.AddPath(wal_it->path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_wal_files.push_back(std::move(wal_it->path));
}
// We only have a WAL before the snapshot
if (recovery_wal_files.empty()) {
const auto lock_success = locker_acc.AddPath(wal_files->back().path);
MG_ASSERT(!lock_success.HasError(), "Tried to lock a nonexistant path.");
recovery_wal_files.push_back(std::move(wal_files->back().path));
}
recovery_steps.emplace_back(std::in_place_type_t<RecoveryWals>{}, std::move(recovery_wal_files));
if (current_wal_seq_num) {
recovery_steps.emplace_back(RecoveryCurrentWal{*current_wal_seq_num});
}
return recovery_steps;
}
InMemoryStorage::TimestampInfo ReplicationClient::GetTimestampInfo() {
InMemoryStorage::TimestampInfo info;
info.current_timestamp_of_replica = 0;
info.current_number_of_timestamp_behind_master = 0;
try {
auto stream{rpc_client_->Stream<replication::TimestampRpc>()};
const auto response = stream.AwaitResponse();
const auto is_success = response.success;
if (!is_success) {
replica_state_.store(replication::ReplicaState::INVALID);
HandleRpcFailure();
}
auto main_time_stamp = storage_->last_commit_timestamp_.load();
info.current_timestamp_of_replica = response.current_commit_timestamp;
info.current_number_of_timestamp_behind_master = response.current_commit_timestamp - main_time_stamp;
} catch (const rpc::RpcFailedException &) {
{
std::unique_lock client_guard(client_lock_);
replica_state_.store(replication::ReplicaState::INVALID);
}
HandleRpcFailure(); // mutex already unlocked, if the new enqueued task dispatches immediately it probably won't
// block
}
return info;
ReplicationClient::~ReplicationClient() {
auto endpoint = rpc_client_.Endpoint();
spdlog::trace("Closing replication client on {}:{}", endpoint.address, endpoint.port);
}
////// ReplicaStream //////
ReplicationClient::ReplicaStream::ReplicaStream(ReplicationClient *self, const uint64_t previous_commit_timestamp,
const uint64_t current_seq_num)
ReplicaStream::ReplicaStream(ReplicationClient *self, const uint64_t previous_commit_timestamp,
const uint64_t current_seq_num)
: self_(self),
stream_(self_->rpc_client_->Stream<replication::AppendDeltasRpc>(previous_commit_timestamp, current_seq_num)) {
stream_(self_->rpc_client_.Stream<replication::AppendDeltasRpc>(previous_commit_timestamp, current_seq_num)) {
replication::Encoder encoder{stream_.GetBuilder()};
encoder.WriteString(self_->storage_->epoch_id_);
encoder.WriteString(self_->GetEpochId());
}
void ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex,
uint64_t final_commit_timestamp) {
void ReplicaStream::AppendDelta(const Delta &delta, const Vertex &vertex, uint64_t final_commit_timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeDelta(&encoder, self_->storage_->name_id_mapper_.get(), self_->storage_->config_.items, delta, vertex,
final_commit_timestamp);
auto *storage = self_->GetStorage();
EncodeDelta(&encoder, storage->name_id_mapper_.get(), storage->config_.items, delta, vertex, final_commit_timestamp);
}
void ReplicationClient::ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge,
uint64_t final_commit_timestamp) {
void ReplicaStream::AppendDelta(const Delta &delta, const Edge &edge, uint64_t final_commit_timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeDelta(&encoder, self_->storage_->name_id_mapper_.get(), delta, edge, final_commit_timestamp);
EncodeDelta(&encoder, self_->GetStorage()->name_id_mapper_.get(), delta, edge, final_commit_timestamp);
}
void ReplicationClient::ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) {
void ReplicaStream::AppendTransactionEnd(uint64_t final_commit_timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeTransactionEnd(&encoder, final_commit_timestamp);
}
void ReplicationClient::ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t timestamp) {
void ReplicaStream::AppendOperation(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t timestamp) {
replication::Encoder encoder(stream_.GetBuilder());
EncodeOperation(&encoder, self_->storage_->name_id_mapper_.get(), operation, label, properties, timestamp);
EncodeOperation(&encoder, self_->GetStorage()->name_id_mapper_.get(), operation, label, properties, timestamp);
}
replication::AppendDeltasRes ReplicationClient::ReplicaStream::Finalize() { return stream_.AwaitResponse(); }
replication::AppendDeltasRes ReplicaStream::Finalize() { return stream_.AwaitResponse(); }
////// CurrentWalHandler //////
ReplicationClient::CurrentWalHandler::CurrentWalHandler(ReplicationClient *self)
: self_(self), stream_(self_->rpc_client_->Stream<replication::CurrentWalRpc>()) {}
void ReplicationClient::CurrentWalHandler::AppendFilename(const std::string &filename) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteString(filename);
}
void ReplicationClient::CurrentWalHandler::AppendSize(const size_t size) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteUint(size);
}
void ReplicationClient::CurrentWalHandler::AppendFileData(utils::InputFile *file) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteFileData(file);
}
void ReplicationClient::CurrentWalHandler::AppendBufferData(const uint8_t *buffer, const size_t buffer_size) {
replication::Encoder encoder(stream_.GetBuilder());
encoder.WriteBuffer(buffer, buffer_size);
}
replication::CurrentWalRes ReplicationClient::CurrentWalHandler::Finalize() { return stream_.AwaitResponse(); }
} // namespace memgraph::storage

View File

@ -11,114 +11,65 @@
#pragma once
#include <atomic>
#include <chrono>
#include <thread>
#include <variant>
#include "rpc/client.hpp"
#include "storage/v2/config.hpp"
#include "storage/v2/delta.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/id_types.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/name_id_mapper.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/global.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "storage/v2/replication/serialization.hpp"
#include "storage/v2/vertex.hpp"
#include "utils/file.hpp"
#include "utils/file_locker.hpp"
#include "utils/spin_lock.hpp"
#include "utils/synchronized.hpp"
#include "utils/scheduler.hpp"
#include "utils/thread_pool.hpp"
namespace memgraph::storage {
class ReplicationClient {
class Storage;
class ReplicationClient;
// Handler used for transfering the current transaction.
class ReplicaStream {
public:
ReplicationClient(std::string name, InMemoryStorage *storage, const io::network::Endpoint &endpoint,
replication::ReplicationMode mode, const replication::ReplicationClientConfig &config = {});
explicit ReplicaStream(ReplicationClient *self, uint64_t previous_commit_timestamp, uint64_t current_seq_num);
~ReplicationClient() {
auto endpoint = rpc_client_->Endpoint();
spdlog::trace("Closing replication client on {}:{}", endpoint.address, endpoint.port);
}
/// @throw rpc::RpcFailedException
void AppendDelta(const Delta &delta, const Vertex &vertex, uint64_t final_commit_timestamp);
// Handler used for transfering the current transaction.
class ReplicaStream {
private:
friend class ReplicationClient;
explicit ReplicaStream(ReplicationClient *self, uint64_t previous_commit_timestamp, uint64_t current_seq_num);
/// @throw rpc::RpcFailedException
void AppendDelta(const Delta &delta, const Edge &edge, uint64_t final_commit_timestamp);
public:
/// @throw rpc::RpcFailedException
void AppendDelta(const Delta &delta, const Vertex &vertex, uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
void AppendTransactionEnd(uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
void AppendDelta(const Delta &delta, const Edge &edge, uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
void AppendOperation(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t timestamp);
/// @throw rpc::RpcFailedException
void AppendTransactionEnd(uint64_t final_commit_timestamp);
/// @throw rpc::RpcFailedException
replication::AppendDeltasRes Finalize();
/// @throw rpc::RpcFailedException
void AppendOperation(durability::StorageGlobalOperation operation, LabelId label,
const std::set<PropertyId> &properties, uint64_t timestamp);
private:
ReplicationClient *self_;
rpc::Client::StreamHandler<replication::AppendDeltasRpc> stream_;
};
private:
/// @throw rpc::RpcFailedException
replication::AppendDeltasRes Finalize();
class ReplicationClient {
friend class CurrentWalHandler;
friend class ReplicaStream;
ReplicationClient *self_;
rpc::Client::StreamHandler<replication::AppendDeltasRpc> stream_;
};
public:
ReplicationClient(std::string name, memgraph::io::network::Endpoint endpoint, replication::ReplicationMode mode,
const replication::ReplicationClientConfig &config);
// Handler for transfering the current WAL file whose data is
// contained in the internal buffer and the file.
class CurrentWalHandler {
private:
friend class ReplicationClient;
explicit CurrentWalHandler(ReplicationClient *self);
ReplicationClient(ReplicationClient const &) = delete;
ReplicationClient &operator=(ReplicationClient const &) = delete;
ReplicationClient(ReplicationClient &&) noexcept = delete;
ReplicationClient &operator=(ReplicationClient &&) noexcept = delete;
public:
void AppendFilename(const std::string &filename);
virtual ~ReplicationClient();
void AppendSize(size_t size);
void AppendFileData(utils::InputFile *file);
void AppendBufferData(const uint8_t *buffer, size_t buffer_size);
/// @throw rpc::RpcFailedException
replication::CurrentWalRes Finalize();
private:
ReplicationClient *self_;
rpc::Client::StreamHandler<replication::CurrentWalRpc> stream_;
};
void StartTransactionReplication(uint64_t current_wal_seq_num);
// Replication clients can be removed at any point
// so to avoid any complexity of checking if the client was removed whenever
// we want to send part of transaction and to avoid adding some GC logic this
// function will run a callback if, after previously callling
// StartTransactionReplication, stream is created.
void IfStreamingTransaction(const std::function<void(ReplicaStream &handler)> &callback);
// Return whether the transaction could be finalized on the replication client or not.
[[nodiscard]] bool FinalizeTransactionReplication();
// Transfer the snapshot file.
// @param path Path of the snapshot file.
replication::SnapshotRes TransferSnapshot(const std::filesystem::path &path);
CurrentWalHandler TransferCurrentWalFile() { return CurrentWalHandler{this}; }
// Transfer the WAL files
replication::WalFilesRes TransferWalFiles(const std::vector<std::filesystem::path> &wal_files);
virtual void Start() = 0;
const auto &Name() const { return name_; }
@ -126,38 +77,20 @@ class ReplicationClient {
auto Mode() const { return mode_; }
const auto &Endpoint() const { return rpc_client_->Endpoint(); }
auto Endpoint() const -> io::network::Endpoint const & { return rpc_client_.Endpoint(); }
InMemoryStorage::TimestampInfo GetTimestampInfo();
private:
[[nodiscard]] bool FinalizeTransactionReplicationInternal();
void RecoverReplica(uint64_t replica_commit);
uint64_t ReplicateCurrentWal();
using RecoveryWals = std::vector<std::filesystem::path>;
struct RecoveryCurrentWal {
uint64_t current_wal_seq_num;
explicit RecoveryCurrentWal(const uint64_t current_wal_seq_num) : current_wal_seq_num(current_wal_seq_num) {}
};
using RecoverySnapshot = std::filesystem::path;
using RecoveryStep = std::variant<RecoverySnapshot, RecoveryWals, RecoveryCurrentWal>;
std::vector<RecoveryStep> GetRecoverySteps(uint64_t replica_commit, utils::FileRetainer::FileLocker *file_locker);
void FrequentCheck();
void InitializeClient();
void TryInitializeClientSync();
void TryInitializeClientAsync();
void HandleRpcFailure();
virtual void StartTransactionReplication(uint64_t current_wal_seq_num) = 0;
virtual void IfStreamingTransaction(const std::function<void(ReplicaStream &)> &callback) = 0;
virtual auto GetEpochId() const -> std::string const & = 0; // TODO: make non-virtual once epoch is moved to storage
virtual auto GetStorage() -> Storage * = 0;
[[nodiscard]] virtual bool FinalizeTransactionReplication() = 0;
virtual TimestampInfo GetTimestampInfo() = 0;
protected:
std::string name_;
InMemoryStorage *storage_;
std::optional<communication::ClientContext> rpc_context_;
std::optional<rpc::Client> rpc_client_;
communication::ClientContext rpc_context_;
rpc::Client rpc_client_;
std::chrono::seconds replica_check_frequency_;
std::optional<ReplicaStream> replica_stream_;
replication::ReplicationMode mode_{replication::ReplicationMode::SYNC};

View File

@ -9,96 +9,47 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
#include <atomic>
#include <filesystem>
#include "spdlog/spdlog.h"
#include "storage/v2/delta.hpp"
#include "storage/v2/durability/durability.hpp"
#include "storage/v2/durability/paths.hpp"
#include "storage/v2/durability/serialization.hpp"
#include "storage/v2/durability/snapshot.hpp"
#include "storage/v2/durability/version.hpp"
#include "storage/v2/durability/wal.hpp"
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/inmemory/unique_constraints.hpp"
#include "replication_server.hpp"
#include "io/network/endpoint.hpp"
#include "rpc.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/replication_server.hpp"
#include "storage/v2/transaction.hpp"
#include "utils/exceptions.hpp"
namespace memgraph::storage {
namespace {
std::pair<uint64_t, durability::WalDeltaData> ReadDelta(durability::BaseDecoder *decoder) {
try {
auto timestamp = ReadWalDeltaHeader(decoder);
SPDLOG_INFO(" Timestamp {}", timestamp);
auto delta = ReadWalDeltaData(decoder);
return {timestamp, delta};
} catch (const slk::SlkReaderException &) {
throw utils::BasicException("Missing data!");
} catch (const durability::RecoveryFailure &) {
throw utils::BasicException("Invalid data!");
}
};
auto CreateServerContext(const replication::ReplicationServerConfig &config) -> communication::ServerContext {
return (config.ssl) ? communication::ServerContext{config.ssl->key_file, config.ssl->cert_file, config.ssl->ca_file,
config.ssl->verify_peer}
: communication::ServerContext{};
}
// NOTE: The replication server must have a single thread for processing
// because there is no need for more processing threads - each replica can
// have only a single main server. Also, the single-threaded guarantee
// simplifies the rest of the implementation.
constexpr auto kReplictionServerThreads = 1;
} // namespace
ReplicationServer::ReplicationServer(InMemoryStorage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config)
: storage_(storage), endpoint_(endpoint) {
// Create RPC server.
if (config.ssl) {
rpc_server_context_.emplace(config.ssl->key_file, config.ssl->cert_file, config.ssl->ca_file,
config.ssl->verify_peer);
} else {
rpc_server_context_.emplace();
}
// NOTE: The replication server must have a single thread for processing
// because there is no need for more processing threads - each replica can
// have only a single main server. Also, the single-threaded guarantee
// simplifies the rest of the implementation.
rpc_server_.emplace(std::move(endpoint), &*rpc_server_context_,
/* workers_count = */ 1);
rpc_server_->Register<replication::HeartbeatRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received HeartbeatRpc");
this->HeartbeatHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::FrequentHeartbeatRpc>([](auto *req_reader, auto *res_builder) {
ReplicationServer::ReplicationServer(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config)
: rpc_server_context_{CreateServerContext(config)},
rpc_server_{std::move(endpoint), &rpc_server_context_, kReplictionServerThreads} {
rpc_server_.Register<replication::FrequentHeartbeatRpc>([](auto *req_reader, auto *res_builder) {
spdlog::debug("Received FrequentHeartbeatRpc");
FrequentHeartbeatHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::AppendDeltasRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received AppendDeltasRpc");
this->AppendDeltasHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::SnapshotRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received SnapshotRpc");
this->SnapshotHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::WalFilesRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received WalFilesRpc");
this->WalFilesHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::CurrentWalRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received CurrentWalRpc");
this->CurrentWalHandler(req_reader, res_builder);
});
rpc_server_->Register<replication::TimestampRpc>([this](auto *req_reader, auto *res_builder) {
spdlog::debug("Received TimestampRpc");
this->TimestampHandler(req_reader, res_builder);
});
rpc_server_->Start();
}
void ReplicationServer::HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::HeartbeatReq req;
slk::Load(&req, req_reader);
replication::HeartbeatRes res{true, storage_->last_commit_timestamp_.load(), storage_->epoch_id_};
slk::Save(res, res_builder);
ReplicationServer::~ReplicationServer() {
if (rpc_server_.IsRunning()) {
auto const &endpoint = rpc_server_.endpoint();
spdlog::trace("Closing replication server on {}:{}", endpoint.address, endpoint.port);
rpc_server_.Shutdown();
}
rpc_server_.AwaitShutdown();
}
bool ReplicationServer::Start() { return rpc_server_.Start(); }
void ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::FrequentHeartbeatReq req;
slk::Load(&req, req_reader);
@ -106,511 +57,4 @@ void ReplicationServer::FrequentHeartbeatHandler(slk::Reader *req_reader, slk::B
slk::Save(res, res_builder);
}
void ReplicationServer::AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::AppendDeltasReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
auto maybe_epoch_id = decoder.ReadString();
MG_ASSERT(maybe_epoch_id, "Invalid replication message");
if (*maybe_epoch_id != storage_->epoch_id_) {
storage_->epoch_history_.emplace_back(std::move(storage_->epoch_id_), storage_->last_commit_timestamp_);
storage_->epoch_id_ = std::move(*maybe_epoch_id);
}
if (storage_->wal_file_) {
if (req.seq_num > storage_->wal_file_->SequenceNumber() || *maybe_epoch_id != storage_->epoch_id_) {
storage_->wal_file_->FinalizeWal();
storage_->wal_file_.reset();
storage_->wal_seq_num_ = req.seq_num;
spdlog::trace("Finalized WAL file");
} else {
MG_ASSERT(storage_->wal_file_->SequenceNumber() == req.seq_num, "Invalid sequence number of current wal file");
storage_->wal_seq_num_ = req.seq_num + 1;
}
} else {
storage_->wal_seq_num_ = req.seq_num;
}
if (req.previous_commit_timestamp != storage_->last_commit_timestamp_.load()) {
// Empty the stream
bool transaction_complete = false;
while (!transaction_complete) {
SPDLOG_INFO("Skipping delta");
const auto [timestamp, delta] = ReadDelta(&decoder);
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
}
replication::AppendDeltasRes res{false, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
return;
}
ReadAndApplyDelta(&decoder);
replication::AppendDeltasRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::debug("Replication recovery from append deltas finished, replica is now up to date!");
}
void ReplicationServer::SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::SnapshotReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->snapshot_directory_);
const auto maybe_snapshot_path = decoder.ReadFile(storage_->snapshot_directory_);
MG_ASSERT(maybe_snapshot_path, "Failed to load snapshot!");
spdlog::info("Received snapshot saved to {}", *maybe_snapshot_path);
std::unique_lock<utils::RWLock> storage_guard(storage_->main_lock_);
spdlog::trace("Clearing database since recovering from snapshot.");
// Clear the database
storage_->vertices_.clear();
storage_->edges_.clear();
storage_->constraints_.existence_constraints_ = std::make_unique<ExistenceConstraints>();
storage_->constraints_.unique_constraints_ = std::make_unique<InMemoryUniqueConstraints>();
storage_->indices_.label_index_ =
std::make_unique<InMemoryLabelIndex>(&storage_->indices_, &storage_->constraints_, storage_->config_);
storage_->indices_.label_property_index_ =
std::make_unique<InMemoryLabelPropertyIndex>(&storage_->indices_, &storage_->constraints_, storage_->config_);
try {
spdlog::debug("Loading snapshot");
auto recovered_snapshot = durability::LoadSnapshot(*maybe_snapshot_path, &storage_->vertices_, &storage_->edges_,
&storage_->epoch_history_, storage_->name_id_mapper_.get(),
&storage_->edge_count_, storage_->config_);
spdlog::debug("Snapshot loaded successfully");
// If this step is present it should always be the first step of
// the recovery so we use the UUID we read from snasphost
storage_->uuid_ = std::move(recovered_snapshot.snapshot_info.uuid);
storage_->epoch_id_ = std::move(recovered_snapshot.snapshot_info.epoch_id);
const auto &recovery_info = recovered_snapshot.recovery_info;
storage_->vertex_id_ = recovery_info.next_vertex_id;
storage_->edge_id_ = recovery_info.next_edge_id;
storage_->timestamp_ = std::max(storage_->timestamp_, recovery_info.next_timestamp);
spdlog::trace("Recovering indices and constraints from snapshot.");
durability::RecoverIndicesAndConstraints(recovered_snapshot.indices_constraints, &storage_->indices_,
&storage_->constraints_, &storage_->vertices_);
} catch (const durability::RecoveryFailure &e) {
LOG_FATAL("Couldn't load the snapshot because of: {}", e.what());
}
storage_guard.unlock();
replication::SnapshotRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::trace("Deleting old snapshot files due to snapshot recovery.");
// Delete other durability files
auto snapshot_files = durability::GetSnapshotFiles(storage_->snapshot_directory_, storage_->uuid_);
for (const auto &[path, uuid, _] : snapshot_files) {
if (path != *maybe_snapshot_path) {
spdlog::trace("Deleting snapshot file {}", path);
storage_->file_retainer_.DeleteFile(path);
}
}
spdlog::trace("Deleting old WAL files due to snapshot recovery.");
auto wal_files = durability::GetWalFiles(storage_->wal_directory_, storage_->uuid_);
if (wal_files) {
for (const auto &wal_file : *wal_files) {
spdlog::trace("Deleting WAL file {}", wal_file.path);
storage_->file_retainer_.DeleteFile(wal_file.path);
}
storage_->wal_file_.reset();
}
spdlog::debug("Replication recovery from snapshot finished!");
}
void ReplicationServer::WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::WalFilesReq req;
slk::Load(&req, req_reader);
const auto wal_file_number = req.file_number;
spdlog::debug("Received WAL files: {}", wal_file_number);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->wal_directory_);
for (auto i = 0; i < wal_file_number; ++i) {
LoadWal(&decoder);
}
replication::WalFilesRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::debug("Replication recovery from WAL files ended successfully, replica is now up to date!");
}
void ReplicationServer::CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::CurrentWalReq req;
slk::Load(&req, req_reader);
replication::Decoder decoder(req_reader);
utils::EnsureDirOrDie(storage_->wal_directory_);
LoadWal(&decoder);
replication::CurrentWalRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
spdlog::debug("Replication recovery from current WAL ended successfully, replica is now up to date!");
}
void ReplicationServer::LoadWal(replication::Decoder *decoder) {
const auto temp_wal_directory = std::filesystem::temp_directory_path() / "memgraph" / durability::kWalDirectory;
utils::EnsureDir(temp_wal_directory);
auto maybe_wal_path = decoder->ReadFile(temp_wal_directory);
MG_ASSERT(maybe_wal_path, "Failed to load WAL!");
spdlog::trace("Received WAL saved to {}", *maybe_wal_path);
try {
auto wal_info = durability::ReadWalInfo(*maybe_wal_path);
if (wal_info.seq_num == 0) {
storage_->uuid_ = wal_info.uuid;
}
if (wal_info.epoch_id != storage_->epoch_id_) {
storage_->epoch_history_.emplace_back(wal_info.epoch_id, storage_->last_commit_timestamp_);
storage_->epoch_id_ = std::move(wal_info.epoch_id);
}
if (storage_->wal_file_) {
if (storage_->wal_file_->SequenceNumber() != wal_info.seq_num) {
storage_->wal_file_->FinalizeWal();
storage_->wal_seq_num_ = wal_info.seq_num;
storage_->wal_file_.reset();
spdlog::trace("WAL file {} finalized successfully", *maybe_wal_path);
}
} else {
storage_->wal_seq_num_ = wal_info.seq_num;
}
spdlog::trace("Loading WAL deltas from {}", *maybe_wal_path);
durability::Decoder wal;
const auto version = wal.Initialize(*maybe_wal_path, durability::kWalMagic);
spdlog::debug("WAL file {} loaded successfully", *maybe_wal_path);
if (!version) throw durability::RecoveryFailure("Couldn't read WAL magic and/or version!");
if (!durability::IsVersionSupported(*version)) throw durability::RecoveryFailure("Invalid WAL version!");
wal.SetPosition(wal_info.offset_deltas);
for (size_t i = 0; i < wal_info.num_deltas;) {
i += ReadAndApplyDelta(&wal);
}
spdlog::debug("Replication from current WAL successful!");
} catch (const durability::RecoveryFailure &e) {
LOG_FATAL("Couldn't recover WAL deltas from {} because of: {}", *maybe_wal_path, e.what());
}
}
void ReplicationServer::TimestampHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
replication::TimestampReq req;
slk::Load(&req, req_reader);
replication::TimestampRes res{true, storage_->last_commit_timestamp_.load()};
slk::Save(res, res_builder);
}
ReplicationServer::~ReplicationServer() {
if (rpc_server_) {
spdlog::trace("Closing replication server on {}:{}", endpoint_.address, endpoint_.port);
rpc_server_->Shutdown();
rpc_server_->AwaitShutdown();
}
}
uint64_t ReplicationServer::ReadAndApplyDelta(durability::BaseDecoder *decoder) {
auto edge_acc = storage_->edges_.access();
auto vertex_acc = storage_->vertices_.access();
std::optional<std::pair<uint64_t, storage::InMemoryStorage::ReplicationAccessor>> commit_timestamp_and_accessor;
auto get_transaction = [this, &commit_timestamp_and_accessor](uint64_t commit_timestamp) {
if (!commit_timestamp_and_accessor) {
auto acc = storage_->Access(std::nullopt);
auto inmem_acc = std::unique_ptr<storage::InMemoryStorage::InMemoryAccessor>(
static_cast<storage::InMemoryStorage::InMemoryAccessor *>(acc.release()));
commit_timestamp_and_accessor.emplace(commit_timestamp, std::move(*inmem_acc));
} else if (commit_timestamp_and_accessor->first != commit_timestamp) {
throw utils::BasicException("Received more than one transaction!");
}
return &commit_timestamp_and_accessor->second;
};
uint64_t applied_deltas = 0;
auto max_commit_timestamp = storage_->last_commit_timestamp_.load();
for (bool transaction_complete = false; !transaction_complete; ++applied_deltas) {
const auto [timestamp, delta] = ReadDelta(decoder);
if (timestamp > max_commit_timestamp) {
max_commit_timestamp = timestamp;
}
transaction_complete = durability::IsWalDeltaDataTypeTransactionEnd(delta.type);
if (timestamp < storage_->timestamp_) {
continue;
}
SPDLOG_INFO(" Delta {}", applied_deltas);
switch (delta.type) {
case durability::WalDeltaData::Type::VERTEX_CREATE: {
spdlog::trace(" Create vertex {}", delta.vertex_create_delete.gid.AsUint());
auto transaction = get_transaction(timestamp);
transaction->CreateVertexEx(delta.vertex_create_delete.gid);
break;
}
case durability::WalDeltaData::Type::VERTEX_DELETE: {
spdlog::trace(" Delete vertex {}", delta.vertex_create_delete.gid.AsUint());
auto transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_create_delete.gid, storage::View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = transaction->DeleteVertex(&*vertex);
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_ADD_LABEL: {
spdlog::trace(" Vertex {} add label {}", delta.vertex_add_remove_label.gid.AsUint(),
delta.vertex_add_remove_label.label);
auto transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_add_remove_label.gid, storage::View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->AddLabel(transaction->NameToLabel(delta.vertex_add_remove_label.label));
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_REMOVE_LABEL: {
spdlog::trace(" Vertex {} remove label {}", delta.vertex_add_remove_label.gid.AsUint(),
delta.vertex_add_remove_label.label);
auto transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_add_remove_label.gid, storage::View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->RemoveLabel(transaction->NameToLabel(delta.vertex_add_remove_label.label));
if (ret.HasError() || !ret.GetValue()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::VERTEX_SET_PROPERTY: {
spdlog::trace(" Vertex {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(),
delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value);
auto transaction = get_transaction(timestamp);
auto vertex = transaction->FindVertex(delta.vertex_edge_set_property.gid, storage::View::NEW);
if (!vertex) throw utils::BasicException("Invalid transaction!");
auto ret = vertex->SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property),
delta.vertex_edge_set_property.value);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_CREATE: {
spdlog::trace(" Create edge {} of type {} from vertex {} to vertex {}",
delta.edge_create_delete.gid.AsUint(), delta.edge_create_delete.edge_type,
delta.edge_create_delete.from_vertex.AsUint(), delta.edge_create_delete.to_vertex.AsUint());
auto transaction = get_transaction(timestamp);
auto from_vertex = transaction->FindVertex(delta.edge_create_delete.from_vertex, storage::View::NEW);
if (!from_vertex) throw utils::BasicException("Invalid transaction!");
auto to_vertex = transaction->FindVertex(delta.edge_create_delete.to_vertex, storage::View::NEW);
if (!to_vertex) throw utils::BasicException("Invalid transaction!");
auto edge = transaction->CreateEdgeEx(&*from_vertex, &*to_vertex,
transaction->NameToEdgeType(delta.edge_create_delete.edge_type),
delta.edge_create_delete.gid);
if (edge.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_DELETE: {
spdlog::trace(" Delete edge {} of type {} from vertex {} to vertex {}",
delta.edge_create_delete.gid.AsUint(), delta.edge_create_delete.edge_type,
delta.edge_create_delete.from_vertex.AsUint(), delta.edge_create_delete.to_vertex.AsUint());
auto transaction = get_transaction(timestamp);
auto from_vertex = transaction->FindVertex(delta.edge_create_delete.from_vertex, storage::View::NEW);
if (!from_vertex) throw utils::BasicException("Invalid transaction!");
auto to_vertex = transaction->FindVertex(delta.edge_create_delete.to_vertex, storage::View::NEW);
if (!to_vertex) throw utils::BasicException("Invalid transaction!");
auto edges = from_vertex->OutEdges(
storage::View::NEW, {transaction->NameToEdgeType(delta.edge_create_delete.edge_type)}, &*to_vertex);
if (edges.HasError()) throw utils::BasicException("Invalid transaction!");
if (edges->size() != 1) throw utils::BasicException("Invalid transaction!");
auto &edge = (*edges)[0];
auto ret = transaction->DeleteEdge(&edge);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EDGE_SET_PROPERTY: {
spdlog::trace(" Edge {} set property {} to {}", delta.vertex_edge_set_property.gid.AsUint(),
delta.vertex_edge_set_property.property, delta.vertex_edge_set_property.value);
if (!storage_->config_.items.properties_on_edges)
throw utils::BasicException(
"Can't set properties on edges because properties on edges "
"are disabled!");
auto transaction = get_transaction(timestamp);
// The following block of code effectively implements `FindEdge` and
// yields an accessor that is only valid for managing the edge's
// properties.
auto edge = edge_acc.find(delta.vertex_edge_set_property.gid);
if (edge == edge_acc.end()) throw utils::BasicException("Invalid transaction!");
// The edge visibility check must be done here manually because we
// don't allow direct access to the edges through the public API.
{
bool is_visible = true;
Delta *delta = nullptr;
{
std::lock_guard<utils::SpinLock> guard(edge->lock);
is_visible = !edge->deleted;
delta = edge->delta;
}
ApplyDeltasForRead(&transaction->GetTransaction(), delta, View::NEW, [&is_visible](const Delta &delta) {
switch (delta.action) {
case Delta::Action::ADD_LABEL:
case Delta::Action::REMOVE_LABEL:
case Delta::Action::SET_PROPERTY:
case Delta::Action::ADD_IN_EDGE:
case Delta::Action::ADD_OUT_EDGE:
case Delta::Action::REMOVE_IN_EDGE:
case Delta::Action::REMOVE_OUT_EDGE:
break;
case Delta::Action::RECREATE_OBJECT: {
is_visible = true;
break;
}
case Delta::Action::DELETE_DESERIALIZED_OBJECT:
case Delta::Action::DELETE_OBJECT: {
is_visible = false;
break;
}
}
});
if (!is_visible) throw utils::BasicException("Invalid transaction!");
}
EdgeRef edge_ref(&*edge);
// Here we create an edge accessor that we will use to get the
// properties of the edge. The accessor is created with an invalid
// type and invalid from/to pointers because we don't know them
// here, but that isn't an issue because we won't use that part of
// the API here.
auto ea = EdgeAccessor{edge_ref,
EdgeTypeId::FromUint(0UL),
nullptr,
nullptr,
&transaction->GetTransaction(),
&storage_->indices_,
&storage_->constraints_,
storage_->config_.items};
auto ret = ea.SetProperty(transaction->NameToProperty(delta.vertex_edge_set_property.property),
delta.vertex_edge_set_property.value);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::TRANSACTION_END: {
spdlog::trace(" Transaction end");
if (!commit_timestamp_and_accessor || commit_timestamp_and_accessor->first != timestamp)
throw utils::BasicException("Invalid data!");
auto ret = commit_timestamp_and_accessor->second.Commit(commit_timestamp_and_accessor->first);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
commit_timestamp_and_accessor = std::nullopt;
break;
}
case durability::WalDeltaData::Type::LABEL_INDEX_CREATE: {
spdlog::trace(" Create label index on :{}", delta.operation_label.label);
// Need to send the timestamp
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage_->CreateIndex(storage_->NameToLabel(delta.operation_label.label), timestamp).HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_INDEX_DROP: {
spdlog::trace(" Drop label index on :{}", delta.operation_label.label);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage_->DropIndex(storage_->NameToLabel(delta.operation_label.label), timestamp).HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_CREATE: {
spdlog::trace(" Create label+property index on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage_
->CreateIndex(storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property), timestamp)
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::LABEL_PROPERTY_INDEX_DROP: {
spdlog::trace(" Drop label+property index on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage_
->DropIndex(storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property), timestamp)
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_CREATE: {
spdlog::trace(" Create existence constraint on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
auto ret = storage_->CreateExistenceConstraint(
storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property), timestamp);
if (ret.HasError()) throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::EXISTENCE_CONSTRAINT_DROP: {
spdlog::trace(" Drop existence constraint on :{} ({})", delta.operation_label_property.label,
delta.operation_label_property.property);
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
if (storage_
->DropExistenceConstraint(storage_->NameToLabel(delta.operation_label_property.label),
storage_->NameToProperty(delta.operation_label_property.property), timestamp)
.HasError())
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_CREATE: {
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Create unique constraint on :{} ({})", delta.operation_label_properties.label, ss.str());
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
std::set<PropertyId> properties;
for (const auto &prop : delta.operation_label_properties.properties) {
properties.emplace(storage_->NameToProperty(prop));
}
auto ret = storage_->CreateUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
if (!ret.HasValue() || ret.GetValue() != UniqueConstraints::CreationStatus::SUCCESS)
throw utils::BasicException("Invalid transaction!");
break;
}
case durability::WalDeltaData::Type::UNIQUE_CONSTRAINT_DROP: {
std::stringstream ss;
utils::PrintIterable(ss, delta.operation_label_properties.properties);
spdlog::trace(" Drop unique constraint on :{} ({})", delta.operation_label_properties.label, ss.str());
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid transaction!");
std::set<PropertyId> properties;
for (const auto &prop : delta.operation_label_properties.properties) {
properties.emplace(storage_->NameToProperty(prop));
}
auto ret = storage_->DropUniqueConstraint(storage_->NameToLabel(delta.operation_label_properties.label),
properties, timestamp);
if (ret.HasError() || ret.GetValue() != UniqueConstraints::DeletionStatus::SUCCESS)
throw utils::BasicException("Invalid transaction!");
break;
}
}
}
if (commit_timestamp_and_accessor) throw utils::BasicException("Invalid data!");
storage_->last_commit_timestamp_ = max_commit_timestamp;
spdlog::debug("Applied {} deltas", applied_deltas);
return applied_deltas;
}
} // namespace memgraph::storage

View File

@ -13,40 +13,28 @@
#include "rpc/server.hpp"
#include "slk/streams.hpp"
#include "storage/v2/inmemory/storage.hpp"
#include "storage/v2/replication/replication_client.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/global.hpp"
namespace memgraph::storage {
class ReplicationServer {
public:
explicit ReplicationServer(InMemoryStorage *storage, io::network::Endpoint endpoint,
const replication::ReplicationServerConfig &config);
explicit ReplicationServer(io::network::Endpoint endpoint, const replication::ReplicationServerConfig &config);
ReplicationServer(const ReplicationServer &) = delete;
ReplicationServer(ReplicationServer &&) = delete;
ReplicationServer &operator=(const ReplicationServer &) = delete;
ReplicationServer &operator=(ReplicationServer &&) = delete;
~ReplicationServer();
virtual ~ReplicationServer();
private:
// RPC handlers
void HeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder);
bool Start();
protected:
static void FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void AppendDeltasHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void SnapshotHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void WalFilesHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void CurrentWalHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void TimestampHandler(slk::Reader *req_reader, slk::Builder *res_builder);
void LoadWal(replication::Decoder *decoder);
uint64_t ReadAndApplyDelta(durability::BaseDecoder *decoder);
std::optional<communication::ServerContext> rpc_server_context_;
std::optional<rpc::Server> rpc_server_;
InMemoryStorage *storage_;
io::network::Endpoint endpoint_;
communication::ServerContext rpc_server_context_;
rpc::Server rpc_server_;
};
} // namespace memgraph::storage

View File

@ -24,6 +24,10 @@
#include "storage/v2/edge_accessor.hpp"
#include "storage/v2/indices/indices.hpp"
#include "storage/v2/mvcc.hpp"
#include "storage/v2/replication/config.hpp"
#include "storage/v2/replication/enums.hpp"
#include "storage/v2/replication/replication_client.hpp"
#include "storage/v2/replication/replication_server.hpp"
#include "storage/v2/storage_error.hpp"
#include "storage/v2/storage_mode.hpp"
#include "storage/v2/vertices_iterable.hpp"
@ -72,7 +76,7 @@ class Storage {
Storage &operator=(const Storage &) = delete;
Storage &operator=(Storage &&) = delete;
virtual ~Storage() {}
virtual ~Storage() = default;
const std::string &id() const { return id_; }
@ -279,6 +283,15 @@ class Storage {
virtual void EstablishNewEpoch() = 0;
virtual auto CreateReplicationClient(std::string name, io::network::Endpoint endpoint,
replication::ReplicationMode mode,
replication::ReplicationClientConfig const &config)
-> std::unique_ptr<ReplicationClient> = 0;
virtual auto CreateReplicationServer(io::network::Endpoint endpoint,
replication::ReplicationServerConfig const &config)
-> std::unique_ptr<ReplicationServer> = 0;
// Main storage lock.
// Accessors take a shared lock when starting, so it is possible to block
// creation of new accessors by taking a unique lock. This is used when doing

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2023 Memgraph Ltd.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt; by using this file, you agree to be bound by the terms of the Business Source
@ -18,6 +18,7 @@
#include "rpc/client_pool.hpp"
#include "rpc/messages.hpp"
#include "rpc/server.hpp"
#include "utils/on_scope_exit.hpp"
#include "utils/timer.hpp"
#include "rpc_messages.hpp"
@ -57,6 +58,10 @@ void EchoMessage::Save(const EchoMessage &obj, memgraph::slk::Builder *builder)
TEST(Rpc, Call) {
memgraph::communication::ServerContext server_context;
Server server({"127.0.0.1", 0}, &server_context);
auto const on_exit = memgraph::utils::OnScopeExit{[&] {
server.Shutdown();
server.AwaitShutdown();
}};
server.Register<Sum>([](auto *req_reader, auto *res_builder) {
SumReq req;
memgraph::slk::Load(&req, req_reader);
@ -70,9 +75,6 @@ TEST(Rpc, Call) {
Client client(server.endpoint(), &client_context);
auto sum = client.Call<Sum>(10, 20);
EXPECT_EQ(sum.sum, 30);
server.Shutdown();
server.AwaitShutdown();
}
TEST(Rpc, Abort) {

View File

@ -747,12 +747,12 @@ TEST_F(ReplicationTest, ReplicationReplicaWithExistingName) {
.HasError());
const std::string replica2_name{replicas[0]};
ASSERT_TRUE(
main_mem_store
->RegisterReplica(replica2_name, replica2_endpoint, memgraph::storage::replication::ReplicationMode::ASYNC,
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID,
memgraph::storage::replication::ReplicationClientConfig{})
.GetError() == memgraph::storage::InMemoryStorage::ReplicationState::RegisterReplicaError::NAME_EXISTS);
ASSERT_TRUE(main_mem_store
->RegisterReplica(replica2_name, replica2_endpoint,
memgraph::storage::replication::ReplicationMode::ASYNC,
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID,
memgraph::storage::replication::ReplicationClientConfig{})
.GetError() == memgraph::storage::ReplicationState::RegisterReplicaError::NAME_EXISTS);
}
TEST_F(ReplicationTest, ReplicationReplicaWithExistingEndPoint) {
@ -779,12 +779,12 @@ TEST_F(ReplicationTest, ReplicationReplicaWithExistingEndPoint) {
.HasError());
const std::string replica2_name{replicas[1]};
ASSERT_TRUE(
main_mem_store
->RegisterReplica(replica2_name, replica2_endpoint, memgraph::storage::replication::ReplicationMode::ASYNC,
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID,
memgraph::storage::replication::ReplicationClientConfig{})
.GetError() == memgraph::storage::InMemoryStorage::ReplicationState::RegisterReplicaError::END_POINT_EXISTS);
ASSERT_TRUE(main_mem_store
->RegisterReplica(replica2_name, replica2_endpoint,
memgraph::storage::replication::ReplicationMode::ASYNC,
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID,
memgraph::storage::replication::ReplicationClientConfig{})
.GetError() == memgraph::storage::ReplicationState::RegisterReplicaError::END_POINT_EXISTS);
}
TEST_F(ReplicationTest, RestoringReplicationAtStartupAfterDroppingReplica) {
@ -907,6 +907,5 @@ TEST_F(ReplicationTest, AddingInvalidReplica) {
memgraph::storage::replication::ReplicationMode::SYNC,
memgraph::storage::replication::RegistrationMode::MUST_BE_INSTANTLY_VALID,
memgraph::storage::replication::ReplicationClientConfig{})
.GetError() ==
memgraph::storage::InMemoryStorage::ReplicationState::RegisterReplicaError::CONNECTION_FAILED);
.GetError() == memgraph::storage::ReplicationState::RegisterReplicaError::CONNECTION_FAILED);
}