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:
parent
a426ef9cc3
commit
4bc5d749b2
@ -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) {}
|
||||
|
||||
|
@ -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) {
|
||||
|
@ -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 };
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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_;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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_;
|
||||
};
|
||||
|
566
src/storage/v2/inmemory/replication/replication_client.cpp
Normal file
566
src/storage/v2/inmemory/replication/replication_client.cpp
Normal 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
|
68
src/storage/v2/inmemory/replication/replication_client.hpp
Normal file
68
src/storage/v2/inmemory/replication/replication_client.hpp
Normal 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
|
575
src/storage/v2/inmemory/replication/replication_server.cpp
Normal file
575
src/storage/v2/inmemory/replication/replication_server.cpp
Normal 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
|
47
src/storage/v2/inmemory/replication/replication_server.hpp
Normal file
47
src/storage/v2/inmemory/replication/replication_server.hpp
Normal 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
|
@ -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
|
||||
|
@ -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_;
|
||||
};
|
||||
|
||||
|
65
src/storage/v2/replication/global.hpp
Normal file
65
src/storage/v2/replication/global.hpp
Normal 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
|
402
src/storage/v2/replication/replication.cpp
Normal file
402
src/storage/v2/replication/replication.cpp
Normal 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
|
127
src/storage/v2/replication/replication.hpp
Normal file
127
src/storage/v2/replication/replication.hpp
Normal 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
|
@ -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
|
||||
|
@ -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};
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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) {
|
||||
|
@ -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);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user