HA: Add coordinator to replication cluster (#1608)

This commit is contained in:
Andi 2024-01-24 13:07:51 +01:00 committed by GitHub
parent 6706ebfa2b
commit 38ade99652
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
96 changed files with 3087 additions and 319 deletions

View File

@ -337,7 +337,53 @@ jobs:
# multiple paths could be defined
build/logs
experimental_build:
experimental_build_ha:
name: "High availability build"
runs-on: [self-hosted, Linux, X64, Diff]
env:
THREADS: 24
MEMGRAPH_ENTERPRISE_LICENSE: ${{ secrets.MEMGRAPH_ENTERPRISE_LICENSE }}
MEMGRAPH_ORGANIZATION_NAME: ${{ secrets.MEMGRAPH_ORGANIZATION_NAME }}
steps:
- name: Set up repository
uses: actions/checkout@v3
with:
# Number of commits to fetch. `0` indicates all history for all
# branches and tags. (default: 1)
fetch-depth: 0
- name: Build release binaries
run: |
source /opt/toolchain-v4/activate
./init
cd build
cmake -DCMAKE_BUILD_TYPE=Release -DMG_EXPERIMENTAL_HIGH_AVAILABILITY=ON ..
make -j$THREADS
- name: Run unit tests
run: |
source /opt/toolchain-v4/activate
cd build
ctest -R memgraph__unit --output-on-failure -j$THREADS
- name: Run e2e tests
run: |
cd tests
./setup.sh /opt/toolchain-v4/activate
source ve3/bin/activate_e2e
cd e2e
./run.sh "Coordinator"
./run.sh "Client initiated failover"
./run.sh "Uninitialized cluster"
- name: Save test data
uses: actions/upload-artifact@v3
if: always()
with:
name: "Test data"
path: |
# multiple paths could be defined
build/logs
experimental_build_mt:
name: "MultiTenancy replication build"
runs-on: [self-hosted, Linux, X64, Diff]
env:

View File

@ -271,6 +271,17 @@ endif()
set(libs_dir ${CMAKE_SOURCE_DIR}/libs)
add_subdirectory(libs EXCLUDE_FROM_ALL)
option(MG_EXPERIMENTAL_HIGH_AVAILABILITY "Feature flag for experimental high availability" OFF)
if (NOT MG_ENTERPRISE AND MG_EXPERIMENTAL_HIGH_AVAILABILITY)
set(MG_EXPERIMENTAL_HIGH_AVAILABILITY OFF)
message(FATAL_ERROR "MG_EXPERIMENTAL_HIGH_AVAILABILITY must be used with enterpise version of the code.")
endif ()
if (MG_EXPERIMENTAL_HIGH_AVAILABILITY)
add_compile_definitions(MG_EXPERIMENTAL_HIGH_AVAILABILITY)
endif ()
# Optional subproject configuration -------------------------------------------
option(TEST_COVERAGE "Generate coverage reports from running memgraph" OFF)
option(TOOLS "Build tools binaries" ON)

View File

@ -22,6 +22,8 @@ add_subdirectory(dbms)
add_subdirectory(flags)
add_subdirectory(distributed)
add_subdirectory(replication)
add_subdirectory(coordination)
add_subdirectory(replication_coordination_glue)
string(TOLOWER ${CMAKE_BUILD_TYPE} lower_build_type)

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Licensed as a Memgraph Enterprise file under the Memgraph Enterprise
// License (the "License"); by using this file, you agree to be bound by the terms of the License, and you may not use
@ -62,7 +62,8 @@ const std::vector<Permission> kPermissionsAll = {Permission::MATCH,
Permission::TRANSACTION_MANAGEMENT,
Permission::STORAGE_MODE,
Permission::MULTI_DATABASE_EDIT,
Permission::MULTI_DATABASE_USE};
Permission::MULTI_DATABASE_USE,
Permission::COORDINATOR};
} // namespace
@ -118,6 +119,8 @@ std::string PermissionToString(Permission permission) {
return "MULTI_DATABASE_EDIT";
case Permission::MULTI_DATABASE_USE:
return "MULTI_DATABASE_USE";
case Permission::COORDINATOR:
return "COORDINATOR";
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 Memgraph Ltd.
//
// Licensed as a Memgraph Enterprise file under the Memgraph Enterprise
// License (the "License"); by using this file, you agree to be bound by the terms of the License, and you may not use
@ -48,6 +48,7 @@ enum class Permission : uint64_t {
STORAGE_MODE = 1U << 22U,
MULTI_DATABASE_EDIT = 1U << 23U,
MULTI_DATABASE_USE = 1U << 24U,
COORDINATOR = 1U << 25U,
};
// clang-format on

View File

@ -0,0 +1,25 @@
add_library(mg-coordination STATIC)
add_library(mg::coordination ALIAS mg-coordination)
target_sources(mg-coordination
PUBLIC
include/coordination/coordinator_client.hpp
include/coordination/coordinator_state.hpp
include/coordination/coordinator_rpc.hpp
include/coordination/coordinator_server.hpp
include/coordination/coordinator_config.hpp
include/coordination/coordinator_entity_info.hpp
include/coordination/coordinator_exceptions.hpp
include/coordination/coordinator_slk.hpp
include/coordination/constants.hpp
PRIVATE
coordinator_client.cpp
coordinator_state.cpp
coordinator_rpc.cpp
coordinator_server.cpp
)
target_include_directories(mg-coordination PUBLIC include)
target_link_libraries(mg-coordination
PUBLIC mg::utils mg::rpc mg::slk mg::io mg::repl_coord_glue
)

View File

@ -0,0 +1,113 @@
// Copyright 2024 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.
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_client.hpp"
#include "coordination/coordinator_config.hpp"
#include "coordination/coordinator_rpc.hpp"
#include "replication_coordination_glue/messages.hpp"
namespace memgraph::coordination {
namespace {
auto CreateClientContext(const memgraph::coordination::CoordinatorClientConfig &config)
-> communication::ClientContext {
return (config.ssl) ? communication::ClientContext{config.ssl->key_file, config.ssl->cert_file}
: communication::ClientContext{};
}
} // namespace
CoordinatorClient::CoordinatorClient(const CoordinatorClientConfig &config)
: rpc_context_{CreateClientContext(config)},
rpc_client_{io::network::Endpoint(io::network::Endpoint::needs_resolving, config.ip_address, config.port),
&rpc_context_},
config_{config} {}
CoordinatorClient::~CoordinatorClient() {
auto exit_job = utils::OnScopeExit([&] {
StopFrequentCheck();
thread_pool_.Shutdown();
});
const auto endpoint = rpc_client_.Endpoint();
// Logging can throw
spdlog::trace("Closing replication client on {}:{}", endpoint.address, endpoint.port);
}
void CoordinatorClient::StartFrequentCheck() {
MG_ASSERT(config_.health_check_frequency_sec > std::chrono::seconds(0),
"Health check frequency must be greater than 0");
replica_checker_.Run(
"Coord checker", config_.health_check_frequency_sec,
[last_response_time = &last_response_time_, rpc_client = &rpc_client_] {
try {
{
auto stream{rpc_client->Stream<memgraph::replication_coordination_glue::FrequentHeartbeatRpc>()};
stream.AwaitResponse();
last_response_time->store(std::chrono::system_clock::now(), std::memory_order_acq_rel);
}
} catch (const rpc::RpcFailedException &) {
// Nothing to do...wait for a reconnect
}
});
}
void CoordinatorClient::StopFrequentCheck() { replica_checker_.Stop(); }
bool CoordinatorClient::DoHealthCheck() const {
auto current_time = std::chrono::system_clock::now();
auto duration = std::chrono::duration_cast<std::chrono::seconds>(current_time -
last_response_time_.load(std::memory_order_acquire));
return duration.count() <= alive_response_time_difference_sec_;
}
auto CoordinatorClient::InstanceName() const -> std::string_view { return config_.instance_name; }
auto CoordinatorClient::Endpoint() const -> io::network::Endpoint const & { return rpc_client_.Endpoint(); }
auto CoordinatorClient::Config() const -> CoordinatorClientConfig const & { return config_; }
auto CoordinatorClient::ReplicationClientInfo() const -> CoordinatorClientConfig::ReplicationClientInfo const & {
MG_ASSERT(config_.replication_client_info.has_value(), "No ReplicationClientInfo for MAIN instance!");
return *config_.replication_client_info;
}
////// AF design choice
auto CoordinatorClient::ReplicationClientInfo() -> std::optional<CoordinatorClientConfig::ReplicationClientInfo> & {
MG_ASSERT(config_.replication_client_info.has_value(), "No ReplicationClientInfo for MAIN instance!");
return config_.replication_client_info;
}
void CoordinatorClient::UpdateTimeCheck(const std::chrono::system_clock::time_point &last_checked_time) {
last_response_time_.store(last_checked_time, std::memory_order_acq_rel);
}
auto CoordinatorClient::GetLastTimeResponse() -> std::chrono::system_clock::time_point { return last_response_time_; }
auto CoordinatorClient::SendPromoteReplicaToMainRpc(
std::vector<CoordinatorClientConfig::ReplicationClientInfo> replication_clients_info) const -> bool {
try {
{
auto stream{rpc_client_.Stream<PromoteReplicaToMainRpc>(std::move(replication_clients_info))};
if (!stream.AwaitResponse().success) {
spdlog::error("Failed to perform failover!");
return false;
}
spdlog::info("Sent failover RPC from coordinator to new main!");
return true;
}
} catch (const rpc::RpcFailedException &) {
spdlog::error("Failed to send failover RPC from coordinator to new main!");
}
return false;
}
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,69 @@
// Copyright 2024 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.
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_rpc.hpp"
#include "coordination/coordinator_slk.hpp"
#include "slk/serialization.hpp"
namespace memgraph {
namespace coordination {
void PromoteReplicaToMainReq::Save(const PromoteReplicaToMainReq &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void PromoteReplicaToMainReq::Load(PromoteReplicaToMainReq *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void PromoteReplicaToMainRes::Save(const PromoteReplicaToMainRes &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void PromoteReplicaToMainRes::Load(PromoteReplicaToMainRes *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
} // namespace coordination
constexpr utils::TypeInfo coordination::PromoteReplicaToMainReq::kType{utils::TypeId::COORD_FAILOVER_REQ,
"CoordPromoteReplicaToMainReq", nullptr};
constexpr utils::TypeInfo coordination::PromoteReplicaToMainRes::kType{utils::TypeId::COORD_FAILOVER_RES,
"CoordPromoteReplicaToMainRes", nullptr};
namespace slk {
void Save(const memgraph::coordination::PromoteReplicaToMainRes &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self.success, builder);
}
void Load(memgraph::coordination::PromoteReplicaToMainRes *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(&self->success, reader);
}
void Save(const memgraph::coordination::PromoteReplicaToMainReq &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self.replication_clients_info, builder);
}
void Load(memgraph::coordination::PromoteReplicaToMainReq *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(&self->replication_clients_info, reader);
}
} // namespace slk
} // namespace memgraph
#endif

View File

@ -0,0 +1,57 @@
// Copyright 2024 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.
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_server.hpp"
#include "replication_coordination_glue/messages.hpp"
namespace memgraph::coordination {
namespace {
auto CreateServerContext(const memgraph::coordination::CoordinatorServerConfig &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 coordinator server doesn't more than 1 processing thread - each replica can
// have only a single coordinator server. Also, the single-threaded guarantee
// simplifies the rest of the implementation.
constexpr auto kCoordinatorServerThreads = 1;
} // namespace
CoordinatorServer::CoordinatorServer(const CoordinatorServerConfig &config)
: rpc_server_context_{CreateServerContext(config)},
rpc_server_{io::network::Endpoint{config.ip_address, config.port}, &rpc_server_context_,
kCoordinatorServerThreads} {
rpc_server_.Register<replication_coordination_glue::FrequentHeartbeatRpc>([](auto *req_reader, auto *res_builder) {
spdlog::debug("Received FrequentHeartbeatRpc on coordinator server");
replication_coordination_glue::FrequentHeartbeatHandler(req_reader, res_builder);
});
}
CoordinatorServer::~CoordinatorServer() {
if (rpc_server_.IsRunning()) {
auto const &endpoint = rpc_server_.endpoint();
spdlog::trace("Closing coordinator server on {}:{}", endpoint.address, endpoint.port);
rpc_server_.Shutdown();
}
rpc_server_.AwaitShutdown();
}
bool CoordinatorServer::Start() { return rpc_server_.Start(); }
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,221 @@
// Copyright 2024 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 "coordination/coordinator_state.hpp"
#include <span>
#include "coordination/coordinator_client.hpp"
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_config.hpp"
#include "coordination/coordinator_entity_info.hpp"
#include "flags/replication.hpp"
#include "spdlog/spdlog.h"
#include "utils/logging.hpp"
#include "utils/variant_helpers.hpp"
#include <atomic>
#include <exception>
#include <optional>
namespace memgraph::coordination {
namespace {
bool CheckName(const std::list<CoordinatorClient> &replicas, const CoordinatorClientConfig &config) {
auto name_matches = [&instance_name = config.instance_name](auto const &replica) {
return replica.InstanceName() == instance_name;
};
return std::any_of(replicas.begin(), replicas.end(), name_matches);
};
} // namespace
CoordinatorState::CoordinatorState() {
MG_ASSERT(!(FLAGS_coordinator && FLAGS_coordinator_server_port),
"Instance cannot be a coordinator and have registered coordinator server.");
if (FLAGS_coordinator_server_port) {
auto const config = CoordinatorServerConfig{
.ip_address = kDefaultReplicationServerIp,
.port = static_cast<uint16_t>(FLAGS_coordinator_server_port),
};
data_ = CoordinatorMainReplicaData{.coordinator_server_ = std::make_unique<CoordinatorServer>(config)};
}
}
auto CoordinatorState::RegisterReplica(const CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus, CoordinatorClient *> {
const auto name_endpoint_status =
std::visit(memgraph::utils::Overloaded{[](const CoordinatorMainReplicaData & /*coordinator_main_replica_data*/) {
return RegisterMainReplicaCoordinatorStatus::NOT_COORDINATOR;
},
[&config](const CoordinatorData &coordinator_data) {
if (memgraph::coordination::CheckName(
coordinator_data.registered_replicas_, config)) {
return RegisterMainReplicaCoordinatorStatus::NAME_EXISTS;
}
return RegisterMainReplicaCoordinatorStatus::SUCCESS;
}},
data_);
if (name_endpoint_status != RegisterMainReplicaCoordinatorStatus::SUCCESS) {
return name_endpoint_status;
}
// Maybe no need to return client if you can start replica client here
return &std::get<CoordinatorData>(data_).registered_replicas_.emplace_back(config);
}
auto CoordinatorState::RegisterMain(const CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus, CoordinatorClient *> {
const auto endpoint_status = std::visit(
memgraph::utils::Overloaded{
[](const CoordinatorMainReplicaData & /*coordinator_main_replica_data*/) {
return RegisterMainReplicaCoordinatorStatus::NOT_COORDINATOR;
},
[](const CoordinatorData & /*coordinator_data*/) { return RegisterMainReplicaCoordinatorStatus::SUCCESS; }},
data_);
if (endpoint_status != RegisterMainReplicaCoordinatorStatus::SUCCESS) {
return endpoint_status;
}
auto &registered_main = std::get<CoordinatorData>(data_).registered_main_;
registered_main = std::make_unique<CoordinatorClient>(config);
return registered_main.get();
}
auto CoordinatorState::ShowReplicas() const -> std::vector<CoordinatorEntityInfo> {
MG_ASSERT(std::holds_alternative<CoordinatorData>(data_),
"Can't call show replicas on data_, as variant holds wrong alternative");
std::vector<CoordinatorEntityInfo> result;
const auto &registered_replicas = std::get<CoordinatorData>(data_).registered_replicas_;
result.reserve(registered_replicas.size());
std::ranges::transform(registered_replicas, std::back_inserter(result), [](const auto &replica) {
return CoordinatorEntityInfo{replica.InstanceName(), replica.Endpoint()};
});
return result;
}
auto CoordinatorState::ShowMain() const -> std::optional<CoordinatorEntityInfo> {
MG_ASSERT(std::holds_alternative<CoordinatorData>(data_),
"Can't call show main on data_, as variant holds wrong alternative");
const auto &registered_main = std::get<CoordinatorData>(data_).registered_main_;
if (registered_main) {
return CoordinatorEntityInfo{registered_main->InstanceName(), registered_main->Endpoint()};
}
return std::nullopt;
}
auto CoordinatorState::PingReplicas() const -> std::unordered_map<std::string_view, bool> {
MG_ASSERT(std::holds_alternative<CoordinatorData>(data_),
"Can't call ping replicas on data_, as variant holds wrong alternative");
std::unordered_map<std::string_view, bool> result;
const auto &registered_replicas = std::get<CoordinatorData>(data_).registered_replicas_;
result.reserve(registered_replicas.size());
for (const CoordinatorClient &replica_client : registered_replicas) {
result.emplace(replica_client.InstanceName(), replica_client.DoHealthCheck());
}
return result;
}
auto CoordinatorState::PingMain() const -> std::optional<CoordinatorEntityHealthInfo> {
MG_ASSERT(std::holds_alternative<CoordinatorData>(data_),
"Can't call show main on data_, as variant holds wrong alternative");
const auto &registered_main = std::get<CoordinatorData>(data_).registered_main_;
if (registered_main) {
return CoordinatorEntityHealthInfo{registered_main->InstanceName(), registered_main->DoHealthCheck()};
}
return std::nullopt;
}
auto CoordinatorState::DoFailover() -> DoFailoverStatus {
// 1. MAIN is already down, stop sending frequent checks
// 2. find new replica (coordinator)
// 3. make copy replica's client as potential new main client (coordinator)
// 4. send failover RPC to new main (coordinator and new main)
// 5. exchange old main to new main (coordinator)
// 6. remove replica which was promoted to main from all replicas -> this will shut down RPC frequent check client
// (coordinator)
// 7. for new main start frequent checks (coordinator)
MG_ASSERT(std::holds_alternative<CoordinatorData>(data_), "Cannot do failover since variant holds wrong alternative");
using ReplicationClientInfo = CoordinatorClientConfig::ReplicationClientInfo;
// 1.
auto &current_main = std::get<CoordinatorData>(data_).registered_main_;
if (!current_main) {
return DoFailoverStatus::CLUSTER_UNINITIALIZED;
}
if (current_main->DoHealthCheck()) {
return DoFailoverStatus::MAIN_ALIVE;
}
current_main->StopFrequentCheck();
// 2.
// Get all replicas and find new main
auto &registered_replicas = std::get<CoordinatorData>(data_).registered_replicas_;
const auto chosen_replica = std::ranges::find_if(
registered_replicas, [](const CoordinatorClient &replica) { return replica.DoHealthCheck(); });
if (chosen_replica == registered_replicas.end()) {
return DoFailoverStatus::ALL_REPLICAS_DOWN;
}
std::vector<ReplicationClientInfo> repl_clients_info;
repl_clients_info.reserve(registered_replicas.size() - 1);
std::ranges::for_each(registered_replicas, [&chosen_replica, &repl_clients_info](const CoordinatorClient &replica) {
if (replica != *chosen_replica) {
repl_clients_info.emplace_back(replica.ReplicationClientInfo());
}
});
// 3.
// Set on coordinator data of new main
// allocate resources for new main, clear replication info on this replica as main
// set last response time
auto potential_new_main = std::make_unique<CoordinatorClient>(chosen_replica->Config());
potential_new_main->ReplicationClientInfo().reset();
potential_new_main->UpdateTimeCheck(chosen_replica->GetLastTimeResponse());
// 4.
if (!chosen_replica->SendPromoteReplicaToMainRpc(std::move(repl_clients_info))) {
spdlog::error("Sent RPC message, but exception was caught, aborting Failover");
// TODO: new status and rollback all changes that were done...
MG_ASSERT(false, "RPC message failed");
}
// 5.
current_main = std::move(potential_new_main);
// 6. remove old replica
// TODO: Stop pinging chosen_replica before failover.
// Check that it doesn't fail when you call StopFrequentCheck if it is already stopped
registered_replicas.erase(chosen_replica);
// 7.
current_main->StartFrequentCheck();
return DoFailoverStatus::SUCCESS;
}
auto CoordinatorState::GetCoordinatorServer() const -> CoordinatorServer & {
MG_ASSERT(std::holds_alternative<CoordinatorMainReplicaData>(data_),
"Cannot get coordinator server since variant holds wrong alternative");
return *std::get<CoordinatorMainReplicaData>(data_).coordinator_server_;
}
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,22 @@
// Copyright 2024 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
namespace memgraph::coordination {
#ifdef MG_EXPERIMENTAL_HIGH_AVAILABILITY
constexpr bool allow_ha = true;
#else
constexpr bool allow_ha = false;
#endif
} // namespace memgraph::coordination

View File

@ -0,0 +1,69 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_config.hpp"
#include "rpc/client.hpp"
#include "utils/scheduler.hpp"
#include "utils/thread_pool.hpp"
#include <string_view>
namespace memgraph::coordination {
class CoordinatorClient {
public:
explicit CoordinatorClient(const CoordinatorClientConfig &config);
~CoordinatorClient();
CoordinatorClient(CoordinatorClient &other) = delete;
CoordinatorClient &operator=(CoordinatorClient const &other) = delete;
CoordinatorClient(CoordinatorClient &&) noexcept = delete;
CoordinatorClient &operator=(CoordinatorClient &&) noexcept = delete;
void StartFrequentCheck();
void StopFrequentCheck();
auto DoHealthCheck() const -> bool;
auto SendPromoteReplicaToMainRpc(
std::vector<CoordinatorClientConfig::ReplicationClientInfo> replication_clients_info) const -> bool;
auto InstanceName() const -> std::string_view;
auto Endpoint() const -> io::network::Endpoint const &;
auto Config() const -> CoordinatorClientConfig const &;
auto ReplicationClientInfo() const -> CoordinatorClientConfig::ReplicationClientInfo const &;
auto ReplicationClientInfo() -> std::optional<CoordinatorClientConfig::ReplicationClientInfo> &;
void UpdateTimeCheck(const std::chrono::system_clock::time_point &last_checked_time);
auto GetLastTimeResponse() -> std::chrono::system_clock::time_point;
friend bool operator==(CoordinatorClient const &first, CoordinatorClient const &second) {
return first.config_ == second.config_;
}
private:
utils::ThreadPool thread_pool_{1};
utils::Scheduler replica_checker_;
communication::ClientContext rpc_context_;
mutable rpc::Client rpc_client_;
CoordinatorClientConfig config_;
std::atomic<std::chrono::system_clock::time_point> last_response_time_{};
static constexpr int alive_response_time_difference_sec_{5};
};
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,77 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "replication_coordination_glue/mode.hpp"
#include <chrono>
#include <cstdint>
#include <optional>
#include <string>
namespace memgraph::coordination {
inline constexpr auto *kDefaultReplicationServerIp = "0.0.0.0";
struct CoordinatorClientConfig {
const std::string instance_name;
const std::string ip_address;
const uint16_t port{};
// Frequency with which coordinator pings main/replicas about it status
const std::chrono::seconds health_check_frequency_sec{1};
// Info which coordinator will send to new main when performing failover
struct ReplicationClientInfo {
// Should be the same as CoordinatorClientConfig's instance_name
std::string instance_name;
replication_coordination_glue::ReplicationMode replication_mode{};
std::string replication_ip_address;
uint16_t replication_port{};
friend bool operator==(ReplicationClientInfo const &, ReplicationClientInfo const &) = default;
};
std::optional<ReplicationClientInfo> replication_client_info;
struct SSL {
const std::string key_file;
const std::string cert_file;
friend bool operator==(const SSL &, const SSL &) = default;
};
const std::optional<SSL> ssl;
friend bool operator==(CoordinatorClientConfig const &, CoordinatorClientConfig const &) = default;
};
struct CoordinatorServerConfig {
std::string ip_address;
uint16_t port{};
struct SSL {
std::string key_file;
std::string cert_file;
std::string ca_file;
bool verify_peer{};
friend bool operator==(SSL const &, SSL const &) = default;
};
std::optional<SSL> ssl;
friend bool operator==(CoordinatorServerConfig const &, CoordinatorServerConfig const &) = default;
};
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,34 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "io/network/endpoint.hpp"
#include <string>
namespace memgraph::coordination {
struct CoordinatorEntityInfo {
std::string_view name;
const io::network::Endpoint &endpoint;
};
struct CoordinatorEntityHealthInfo {
std::string_view name;
bool alive;
};
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,32 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "utils/exceptions.hpp"
namespace memgraph::coordination {
class CoordinatorFailoverException final : public utils::BasicException {
public:
explicit CoordinatorFailoverException(const std::string_view what) noexcept
: BasicException("Failover didn't complete successfully: " + std::string(what)) {}
template <class... Args>
explicit CoordinatorFailoverException(fmt::format_string<Args...> fmt, Args &&...args) noexcept
: CoordinatorFailoverException(fmt::format(fmt, std::forward<Args>(args)...)) {}
SPECIALIZE_GET_EXCEPTION_NAME(CoordinatorFailoverException)
};
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,66 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_config.hpp"
#include "rpc/messages.hpp"
#include "slk/serialization.hpp"
namespace memgraph::coordination {
struct PromoteReplicaToMainReq {
static const utils::TypeInfo kType;
static const utils::TypeInfo &GetTypeInfo() { return kType; }
static void Load(PromoteReplicaToMainReq *self, memgraph::slk::Reader *reader);
static void Save(const PromoteReplicaToMainReq &self, memgraph::slk::Builder *builder);
explicit PromoteReplicaToMainReq(std::vector<CoordinatorClientConfig::ReplicationClientInfo> replication_clients_info)
: replication_clients_info(std::move(replication_clients_info)) {}
PromoteReplicaToMainReq() = default;
std::vector<CoordinatorClientConfig::ReplicationClientInfo> replication_clients_info;
};
struct PromoteReplicaToMainRes {
static const utils::TypeInfo kType;
static const utils::TypeInfo &GetTypeInfo() { return kType; }
static void Load(PromoteReplicaToMainRes *self, memgraph::slk::Reader *reader);
static void Save(const PromoteReplicaToMainRes &self, memgraph::slk::Builder *builder);
explicit PromoteReplicaToMainRes(bool success) : success(success) {}
PromoteReplicaToMainRes() = default;
bool success;
};
using PromoteReplicaToMainRpc = rpc::RequestResponse<PromoteReplicaToMainReq, PromoteReplicaToMainRes>;
} // namespace memgraph::coordination
// SLK serialization declarations
namespace memgraph::slk {
void Save(const memgraph::coordination::PromoteReplicaToMainRes &self, memgraph::slk::Builder *builder);
void Load(memgraph::coordination::PromoteReplicaToMainRes *self, memgraph::slk::Reader *reader);
void Save(const memgraph::coordination::PromoteReplicaToMainReq &self, memgraph::slk::Builder *builder);
void Load(memgraph::coordination::PromoteReplicaToMainReq *self, memgraph::slk::Reader *reader);
} // namespace memgraph::slk
#endif

View File

@ -0,0 +1,44 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_config.hpp"
#include "rpc/server.hpp"
namespace memgraph::coordination {
class CoordinatorServer {
public:
explicit CoordinatorServer(const CoordinatorServerConfig &config);
CoordinatorServer(const CoordinatorServer &) = delete;
CoordinatorServer(CoordinatorServer &&) = delete;
CoordinatorServer &operator=(const CoordinatorServer &) = delete;
CoordinatorServer &operator=(CoordinatorServer &&) = delete;
virtual ~CoordinatorServer();
bool Start();
template <typename TRequestResponse, typename F>
void Register(F &&callback) {
rpc_server_.Register<TRequestResponse>(std::forward<F>(callback));
}
private:
communication::ServerContext rpc_server_context_;
rpc::Server rpc_server_;
};
} // namespace memgraph::coordination
#endif

View File

@ -0,0 +1,38 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_config.hpp"
#include "slk/serialization.hpp"
#include "slk/streams.hpp"
namespace memgraph::slk {
using ReplicationClientInfo = coordination::CoordinatorClientConfig::ReplicationClientInfo;
inline void Save(const ReplicationClientInfo &obj, Builder *builder) {
Save(obj.instance_name, builder);
Save(obj.replication_mode, builder);
Save(obj.replication_ip_address, builder);
Save(obj.replication_port, builder);
}
inline void Load(ReplicationClientInfo *obj, Reader *reader) {
Load(&obj->instance_name, reader);
Load(&obj->replication_mode, reader);
Load(&obj->replication_ip_address, reader);
Load(&obj->replication_port, reader);
}
} // namespace memgraph::slk
#endif

View File

@ -0,0 +1,94 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_client.hpp"
#include "coordination/coordinator_entity_info.hpp"
#include "coordination/coordinator_server.hpp"
#include "rpc/server.hpp"
#include "utils/result.hpp"
#include "utils/rw_spin_lock.hpp"
#include "utils/synchronized.hpp"
#include <list>
#include <variant>
namespace memgraph::coordination {
enum class RegisterMainReplicaCoordinatorStatus : uint8_t {
NAME_EXISTS,
END_POINT_EXISTS,
COULD_NOT_BE_PERSISTED,
NOT_COORDINATOR,
SUCCESS
};
enum class DoFailoverStatus : uint8_t { SUCCESS, ALL_REPLICAS_DOWN, MAIN_ALIVE, CLUSTER_UNINITIALIZED };
class CoordinatorState {
public:
CoordinatorState();
~CoordinatorState() = default;
CoordinatorState(const CoordinatorState &) = delete;
CoordinatorState &operator=(const CoordinatorState &) = delete;
CoordinatorState(CoordinatorState &&other) noexcept : data_(std::move(other.data_)) {}
CoordinatorState &operator=(CoordinatorState &&other) noexcept {
if (this == &other) {
return *this;
}
data_ = std::move(other.data_);
return *this;
}
auto RegisterReplica(const CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus, CoordinatorClient *>;
auto RegisterMain(const CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus, CoordinatorClient *>;
auto ShowReplicas() const -> std::vector<CoordinatorEntityInfo>;
auto PingReplicas() const -> std::unordered_map<std::string_view, bool>;
auto ShowMain() const -> std::optional<CoordinatorEntityInfo>;
auto PingMain() const -> std::optional<CoordinatorEntityHealthInfo>;
// The client code must check that the server exists before calling this method.
auto GetCoordinatorServer() const -> CoordinatorServer &;
auto DoFailover() -> DoFailoverStatus;
private:
// TODO: Data is not thread safe
// Coordinator stores registered replicas and main
struct CoordinatorData {
std::list<CoordinatorClient> registered_replicas_;
std::unique_ptr<CoordinatorClient> registered_main_;
};
// Data which each main and replica stores
struct CoordinatorMainReplicaData {
std::unique_ptr<CoordinatorServer> coordinator_server_;
};
std::variant<CoordinatorData, CoordinatorMainReplicaData> data_;
};
} // namespace memgraph::coordination
#endif

View File

@ -1,3 +1,2 @@
add_library(mg-dbms STATIC dbms_handler.cpp database.cpp replication_handler.cpp replication_client.cpp inmemory/replication_handlers.cpp)
target_link_libraries(mg-dbms mg-utils mg-storage-v2 mg-query)
add_library(mg-dbms STATIC dbms_handler.cpp database.cpp replication_handler.cpp coordinator_handler.cpp replication_client.cpp inmemory/replication_handlers.cpp coordinator_handlers.cpp)
target_link_libraries(mg-dbms mg-utils mg-storage-v2 mg-query mg-replication mg-coordination)

View File

@ -0,0 +1,99 @@
// Copyright 2024 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.
#ifdef MG_ENTERPRISE
#include "dbms/coordinator_handler.hpp"
#include "dbms/dbms_handler.hpp"
namespace memgraph::dbms {
CoordinatorHandler::CoordinatorHandler(DbmsHandler &dbms_handler) : dbms_handler_(dbms_handler) {}
auto CoordinatorHandler::RegisterReplicaOnCoordinator(const memgraph::coordination::CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus> {
auto instance_client = dbms_handler_.CoordinatorState().RegisterReplica(config);
using repl_status = memgraph::coordination::RegisterMainReplicaCoordinatorStatus;
using dbms_status = memgraph::dbms::RegisterMainReplicaCoordinatorStatus;
if (instance_client.HasError()) {
switch (instance_client.GetError()) {
case memgraph::coordination::RegisterMainReplicaCoordinatorStatus::NOT_COORDINATOR:
MG_ASSERT(false, "Only coordinator instance can register main and replica!");
return {};
case repl_status::NAME_EXISTS:
return dbms_status::NAME_EXISTS;
case repl_status::END_POINT_EXISTS:
return dbms_status::END_POINT_EXISTS;
case repl_status::COULD_NOT_BE_PERSISTED:
return dbms_status::COULD_NOT_BE_PERSISTED;
case repl_status::SUCCESS:
break;
}
}
instance_client.GetValue()->StartFrequentCheck();
return {};
}
auto CoordinatorHandler::RegisterMainOnCoordinator(const memgraph::coordination::CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus> {
auto instance_client = dbms_handler_.CoordinatorState().RegisterMain(config);
if (instance_client.HasError()) switch (instance_client.GetError()) {
case memgraph::coordination::RegisterMainReplicaCoordinatorStatus::NOT_COORDINATOR:
MG_ASSERT(false, "Only coordinator instance can register main and replica!");
case memgraph::coordination::RegisterMainReplicaCoordinatorStatus::NAME_EXISTS:
return memgraph::dbms::RegisterMainReplicaCoordinatorStatus::NAME_EXISTS;
case memgraph::coordination::RegisterMainReplicaCoordinatorStatus::END_POINT_EXISTS:
return memgraph::dbms::RegisterMainReplicaCoordinatorStatus::END_POINT_EXISTS;
case memgraph::coordination::RegisterMainReplicaCoordinatorStatus::COULD_NOT_BE_PERSISTED:
return memgraph::dbms::RegisterMainReplicaCoordinatorStatus::COULD_NOT_BE_PERSISTED;
case memgraph::coordination::RegisterMainReplicaCoordinatorStatus::SUCCESS:
break;
}
instance_client.GetValue()->StartFrequentCheck();
return {};
}
auto CoordinatorHandler::ShowReplicasOnCoordinator() const -> std::vector<coordination::CoordinatorEntityInfo> {
return dbms_handler_.CoordinatorState().ShowReplicas();
}
auto CoordinatorHandler::PingReplicasOnCoordinator() const -> std::unordered_map<std::string_view, bool> {
return dbms_handler_.CoordinatorState().PingReplicas();
}
auto CoordinatorHandler::ShowMainOnCoordinator() const -> std::optional<coordination::CoordinatorEntityInfo> {
return dbms_handler_.CoordinatorState().ShowMain();
}
auto CoordinatorHandler::PingMainOnCoordinator() const -> std::optional<coordination::CoordinatorEntityHealthInfo> {
return dbms_handler_.CoordinatorState().PingMain();
}
auto CoordinatorHandler::DoFailover() const -> DoFailoverStatus {
auto status = dbms_handler_.CoordinatorState().DoFailover();
switch (status) {
case memgraph::coordination::DoFailoverStatus::ALL_REPLICAS_DOWN:
return memgraph::dbms::DoFailoverStatus::ALL_REPLICAS_DOWN;
case memgraph::coordination::DoFailoverStatus::SUCCESS:
return memgraph::dbms::DoFailoverStatus::SUCCESS;
case memgraph::coordination::DoFailoverStatus::MAIN_ALIVE:
return memgraph::dbms::DoFailoverStatus::MAIN_ALIVE;
case memgraph::coordination::DoFailoverStatus::CLUSTER_UNINITIALIZED:
return memgraph::dbms::DoFailoverStatus::CLUSTER_UNINITIALIZED;
}
}
} // namespace memgraph::dbms
#endif

View File

@ -0,0 +1,67 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "utils/result.hpp"
#include <cstdint>
#include <optional>
#include <vector>
namespace memgraph::coordination {
struct CoordinatorEntityInfo;
struct CoordinatorEntityHealthInfo;
struct CoordinatorClientConfig;
} // namespace memgraph::coordination
namespace memgraph::dbms {
enum class RegisterMainReplicaCoordinatorStatus : uint8_t {
NAME_EXISTS,
END_POINT_EXISTS,
COULD_NOT_BE_PERSISTED,
NOT_COORDINATOR,
SUCCESS
};
enum class DoFailoverStatus : uint8_t { SUCCESS, ALL_REPLICAS_DOWN, MAIN_ALIVE, CLUSTER_UNINITIALIZED };
class DbmsHandler;
class CoordinatorHandler {
public:
explicit CoordinatorHandler(DbmsHandler &dbms_handler);
auto RegisterReplicaOnCoordinator(const memgraph::coordination::CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus>;
auto RegisterMainOnCoordinator(const memgraph::coordination::CoordinatorClientConfig &config)
-> utils::BasicResult<RegisterMainReplicaCoordinatorStatus>;
auto ShowReplicasOnCoordinator() const -> std::vector<memgraph::coordination::CoordinatorEntityInfo>;
auto ShowMainOnCoordinator() const -> std::optional<memgraph::coordination::CoordinatorEntityInfo>;
auto PingReplicasOnCoordinator() const -> std::unordered_map<std::string_view, bool>;
auto PingMainOnCoordinator() const -> std::optional<memgraph::coordination::CoordinatorEntityHealthInfo>;
auto DoFailover() const -> DoFailoverStatus;
private:
DbmsHandler &dbms_handler_;
};
} // namespace memgraph::dbms
#endif

View File

@ -0,0 +1,106 @@
// Copyright 2024 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.
#ifdef MG_ENTERPRISE
#include "dbms/coordinator_handlers.hpp"
#include "dbms/utils.hpp"
#include "coordination/coordinator_exceptions.hpp"
#include "coordination/coordinator_rpc.hpp"
#include "dbms/dbms_handler.hpp"
#include "dbms/replication_client.hpp"
namespace memgraph::dbms {
void CoordinatorHandlers::Register(DbmsHandler &dbms_handler) {
auto &server = dbms_handler.CoordinatorState().GetCoordinatorServer();
server.Register<coordination::PromoteReplicaToMainRpc>(
[&dbms_handler](slk::Reader *req_reader, slk::Builder *res_builder) -> void {
spdlog::info("Received PromoteReplicaToMainRpc from coordinator server");
CoordinatorHandlers::PromoteReplicaToMainHandler(dbms_handler, req_reader, res_builder);
});
}
void CoordinatorHandlers::PromoteReplicaToMainHandler(DbmsHandler &dbms_handler, slk::Reader *req_reader,
slk::Builder *res_builder) {
auto &repl_state = dbms_handler.ReplicationState();
if (!repl_state.IsReplica()) {
spdlog::error("Failover must be performed on replica!");
slk::Save(coordination::PromoteReplicaToMainRes{false}, res_builder);
return;
}
if (bool success = memgraph::dbms::DoReplicaToMainPromotion(dbms_handler); !success) {
spdlog::error("Promoting replica to main failed!");
slk::Save(coordination::PromoteReplicaToMainRes{false}, res_builder);
return;
}
coordination::PromoteReplicaToMainReq req;
slk::Load(&req, req_reader);
std::vector<replication::ReplicationClientConfig> clients_config;
clients_config.reserve(req.replication_clients_info.size());
std::ranges::transform(req.replication_clients_info, std::back_inserter(clients_config),
[](const auto &repl_info_config) {
return replication::ReplicationClientConfig{
.name = repl_info_config.instance_name,
.mode = repl_info_config.replication_mode,
.ip_address = repl_info_config.replication_ip_address,
.port = repl_info_config.replication_port,
};
});
std::ranges::for_each(clients_config, [&dbms_handler, &repl_state, &res_builder](const auto &config) {
auto instance_client = repl_state.RegisterReplica(config);
if (instance_client.HasError()) {
switch (instance_client.GetError()) {
case memgraph::replication::RegisterReplicaError::NOT_MAIN:
spdlog::error("Failover must be performed to main!");
slk::Save(coordination::PromoteReplicaToMainRes{false}, res_builder);
return;
case memgraph::replication::RegisterReplicaError::NAME_EXISTS:
spdlog::error("Replica with the same name already exists!");
slk::Save(coordination::PromoteReplicaToMainRes{false}, res_builder);
return;
case memgraph::replication::RegisterReplicaError::END_POINT_EXISTS:
spdlog::error("Replica with the same endpoint already exists!");
slk::Save(coordination::PromoteReplicaToMainRes{false}, res_builder);
return;
case memgraph::replication::RegisterReplicaError::COULD_NOT_BE_PERSISTED:
spdlog::error("Registered replica could not be persisted!");
slk::Save(coordination::PromoteReplicaToMainRes{false}, res_builder);
return;
case memgraph::replication::RegisterReplicaError::SUCCESS:
break;
}
}
auto &instance_client_ref = *instance_client.GetValue();
const bool all_clients_good = memgraph::dbms::RegisterAllDatabasesClients(dbms_handler, instance_client_ref);
if (!all_clients_good) {
spdlog::error("Failed to register all databases to the REPLICA \"{}\"", config.name);
slk::Save(coordination::PromoteReplicaToMainRes{false}, res_builder);
return;
}
StartReplicaClient(dbms_handler, instance_client_ref);
});
slk::Save(coordination::PromoteReplicaToMainRes{true}, res_builder);
}
} // namespace memgraph::dbms
#endif

View File

@ -0,0 +1,33 @@
// Copyright 2024 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
#ifdef MG_ENTERPRISE
#include "slk/serialization.hpp"
namespace memgraph::dbms {
class DbmsHandler;
class CoordinatorHandlers {
public:
static void Register(DbmsHandler &dbms_handler);
private:
static void PromoteReplicaToMainHandler(DbmsHandler &dbms_handler, slk::Reader *req_reader,
slk::Builder *res_builder);
};
} // namespace memgraph::dbms
#endif

View File

@ -11,6 +11,9 @@
#include "dbms/dbms_handler.hpp"
#include "dbms/coordinator_handlers.hpp"
#include "flags/replication.hpp"
#include <cstdint>
#include <filesystem>
@ -225,6 +228,12 @@ DbmsHandler::DbmsHandler(
"enabling durability by using --storage-snapshot-interval-sec and --storage-wal-enabled flags because "
"without write-ahead logs this instance is not replicating any data.");
}
// MAIN or REPLICA instance
if (FLAGS_coordinator_server_port) {
CoordinatorHandlers::Register(*this);
MG_ASSERT(coordinator_state_.GetCoordinatorServer().Start(), "Failed to start coordinator server!");
}
}
DbmsHandler::DeleteResult DbmsHandler::TryDelete(std::string_view db_name) {
@ -375,7 +384,7 @@ AllSyncReplicaStatus DbmsHandler::Commit() {
std::string(main_data.epoch_.id()), last_commited_system_timestamp_,
system_transaction_->system_timestamp, delta.config);
// TODO: reduce duplicate code
if (!completed && client.mode_ == replication::ReplicationMode::SYNC) {
if (!completed && client.mode_ == replication_coordination_glue::ReplicationMode::SYNC) {
sync_status = AllSyncReplicaStatus::SomeCommitsUnconfirmed;
}
}
@ -399,7 +408,7 @@ AllSyncReplicaStatus DbmsHandler::Commit() {
std::string(main_data.epoch_.id()), last_commited_system_timestamp_,
system_transaction_->system_timestamp, delta.uuid);
// TODO: reduce duplicate code
if (!completed && client.mode_ == replication::ReplicationMode::SYNC) {
if (!completed && client.mode_ == replication_coordination_glue::ReplicationMode::SYNC) {
sync_status = AllSyncReplicaStatus::SomeCommitsUnconfirmed;
}
}

View File

@ -34,6 +34,7 @@
#include "storage/v2/transaction.hpp"
#include "utils/thread_pool.hpp"
#ifdef MG_ENTERPRISE
#include "coordination/coordinator_state.hpp"
#include "dbms/database_handler.hpp"
#endif
#include "dbms/transaction.hpp"
@ -272,6 +273,10 @@ class DbmsHandler {
bool IsMain() const { return repl_state_.IsMain(); }
bool IsReplica() const { return repl_state_.IsReplica(); }
#ifdef MG_ENTERPRISE
coordination::CoordinatorState &CoordinatorState() { return coordinator_state_; }
#endif
/**
* @brief Return the statistics all databases.
*
@ -436,7 +441,7 @@ class DbmsHandler {
return false;
};
if (client.mode_ == memgraph::replication::ReplicationMode::ASYNC) {
if (client.mode_ == memgraph::replication_coordination_glue::ReplicationMode::ASYNC) {
client.thread_pool_.AddTask([task = utils::CopyMovableFunctionWrapper{std::move(task)}]() mutable { task(); });
return true;
}
@ -672,6 +677,7 @@ class DbmsHandler {
storage::Config default_config_; //!< Storage configuration used when creating new databases
DatabaseHandler db_handler_; //!< multi-tenancy storage handler
std::unique_ptr<kvstore::KVStore> durability_; //!< list of active dbs (pointer so we can postpone its creation)
coordination::CoordinatorState coordinator_state_; //!< Replication coordinator
#endif
// TODO: Make an api
public:

View File

@ -18,14 +18,14 @@
#include "dbms/global.hpp"
#include "dbms/inmemory/replication_handlers.hpp"
#include "dbms/replication_client.hpp"
#include "dbms/utils.hpp"
#include "replication/messages.hpp"
#include "replication/state.hpp"
#include "spdlog/spdlog.h"
#include "storage/v2/config.hpp"
#include "storage/v2/replication/rpc.hpp"
#include "utils/on_scope_exit.hpp"
using memgraph::replication::ReplicationClientConfig;
using memgraph::replication::ReplicationState;
using memgraph::replication::RoleMainData;
using memgraph::replication::RoleReplicaData;
@ -51,34 +51,13 @@ std::string RegisterReplicaErrorToString(RegisterReplicaError error) {
ReplicationHandler::ReplicationHandler(DbmsHandler &dbms_handler) : dbms_handler_(dbms_handler) {}
bool ReplicationHandler::SetReplicationRoleMain() {
auto const main_handler = [](RoleMainData const &) {
auto const main_handler = [](RoleMainData &) {
// If we are already MAIN, we don't want to change anything
return false;
};
auto const replica_handler = [this](RoleReplicaData const &) {
// STEP 1) bring down all REPLICA servers
dbms_handler_.ForEach([](DatabaseAccess db_acc) {
auto *storage = db_acc->storage();
// Remember old epoch + storage timestamp association
storage->PrepareForNewEpoch();
});
// STEP 2) Change to MAIN
// TODO: restore replication servers if false?
if (!dbms_handler_.ReplicationState().SetReplicationRoleMain()) {
// TODO: Handle recovery on failure???
return false;
}
// STEP 3) We are now MAIN, update storage local epoch
const auto &epoch =
std::get<RoleMainData>(std::as_const(dbms_handler_.ReplicationState()).ReplicationData()).epoch_;
dbms_handler_.ForEach([&](DatabaseAccess db_acc) {
auto *storage = db_acc->storage();
storage->repl_storage_state_.epoch_ = epoch;
});
return true;
return memgraph::dbms::DoReplicaToMainPromotion(dbms_handler_);
};
// TODO: under lock
@ -147,32 +126,12 @@ auto ReplicationHandler::RegisterReplica(const memgraph::replication::Replicatio
dbms_handler_.SystemRestore(*instance_client.GetValue());
#endif
bool all_clients_good = true;
// Add database specific clients (NOTE Currently all databases are connected to each replica)
dbms_handler_.ForEach([&](DatabaseAccess db_acc) {
auto *storage = db_acc->storage();
if (!allow_mt_repl && storage->name() != kDefaultDB) {
return;
}
// TODO: ATM only IN_MEMORY_TRANSACTIONAL, fix other modes
if (storage->storage_mode_ != storage::StorageMode::IN_MEMORY_TRANSACTIONAL) return;
all_clients_good &= storage->repl_storage_state_.replication_clients_.WithLock(
[storage, &instance_client, db_acc = std::move(db_acc)](auto &storage_clients) mutable { // NOLINT
auto client = std::make_unique<storage::ReplicationStorageClient>(*instance_client.GetValue());
// All good, start replica client
client->Start(storage, std::move(db_acc));
// After start the storage <-> replica state should be READY or RECOVERING (if correctly started)
// MAYBE_BEHIND isn't a statement of the current state, this is the default value
// Failed to start due an error like branching of MAIN and REPLICA
if (client->State() == storage::replication::ReplicaState::MAYBE_BEHIND) {
return false;
}
storage_clients.push_back(std::move(client));
return true;
});
});
const auto dbms_error = memgraph::dbms::HandleErrorOnReplicaClient(instance_client);
if (dbms_error.has_value()) {
return *dbms_error;
}
auto &instance_client_ptr = instance_client.GetValue();
const bool all_clients_good = memgraph::dbms::RegisterAllDatabasesClients(dbms_handler_, *instance_client_ptr);
// NOTE Currently if any databases fails, we revert back
if (!all_clients_good) {

View File

@ -15,7 +15,6 @@
#include "replication/role.hpp"
#include "utils/result.hpp"
// BEGIN fwd declares
namespace memgraph::replication {
struct ReplicationState;
struct ReplicationServerConfig;
@ -23,9 +22,11 @@ struct ReplicationClientConfig;
} // namespace memgraph::replication
namespace memgraph::dbms {
class DbmsHandler;
enum class RegisterReplicaError : uint8_t { NAME_EXISTS, END_POINT_EXISTS, CONNECTION_FAILED, COULD_NOT_BE_PERSISTED };
enum class UnregisterReplicaResult : uint8_t {
NOT_MAIN,
COULD_NOT_BE_PERSISTED,

100
src/dbms/utils.hpp Normal file
View File

@ -0,0 +1,100 @@
// Copyright 2024 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 "dbms/dbms_handler.hpp"
#include "dbms/replication_handler.hpp"
#include "replication/include/replication/state.hpp"
#include "utils/result.hpp"
namespace memgraph::dbms {
inline bool DoReplicaToMainPromotion(dbms::DbmsHandler &dbms_handler) {
// STEP 1) bring down all REPLICA servers
dbms_handler.ForEach([](DatabaseAccess db_acc) {
auto *storage = db_acc->storage();
// Remember old epoch + storage timestamp association
storage->PrepareForNewEpoch();
});
// STEP 2) Change to MAIN
// TODO: restore replication servers if false?
if (!dbms_handler.ReplicationState().SetReplicationRoleMain()) {
// TODO: Handle recovery on failure???
return false;
}
// STEP 3) We are now MAIN, update storage local epoch
const auto &epoch =
std::get<replication::RoleMainData>(std::as_const(dbms_handler.ReplicationState()).ReplicationData()).epoch_;
dbms_handler.ForEach([&](DatabaseAccess db_acc) {
auto *storage = db_acc->storage();
storage->repl_storage_state_.epoch_ = epoch;
});
return true;
};
inline bool RegisterAllDatabasesClients(dbms::DbmsHandler &dbms_handler,
replication::ReplicationClient &instance_client) {
if (!allow_mt_repl && dbms_handler.All().size() > 1) {
spdlog::warn("Multi-tenant replication is currently not supported!");
}
bool all_clients_good = true;
// Add database specific clients (NOTE Currently all databases are connected to each replica)
dbms_handler.ForEach([&](DatabaseAccess db_acc) {
auto *storage = db_acc->storage();
if (!allow_mt_repl && storage->name() != kDefaultDB) {
return;
}
// TODO: ATM only IN_MEMORY_TRANSACTIONAL, fix other modes
if (storage->storage_mode_ != storage::StorageMode::IN_MEMORY_TRANSACTIONAL) return;
all_clients_good &= storage->repl_storage_state_.replication_clients_.WithLock(
[storage, &instance_client, db_acc = std::move(db_acc)](auto &storage_clients) mutable { // NOLINT
auto client = std::make_unique<storage::ReplicationStorageClient>(instance_client);
// All good, start replica client
client->Start(storage, std::move(db_acc));
// After start the storage <-> replica state should be READY or RECOVERING (if correctly started)
// MAYBE_BEHIND isn't a statement of the current state, this is the default value
// Failed to start due an error like branching of MAIN and REPLICA
if (client->State() == storage::replication::ReplicaState::MAYBE_BEHIND) {
return false;
}
storage_clients.push_back(std::move(client));
return true;
});
});
return all_clients_good;
}
inline std::optional<RegisterReplicaError> HandleErrorOnReplicaClient(
utils::BasicResult<replication::RegisterReplicaError, replication::ReplicationClient *> &instance_client) {
if (instance_client.HasError()) switch (instance_client.GetError()) {
case replication::RegisterReplicaError::NOT_MAIN:
MG_ASSERT(false, "Only main instance can register a replica!");
return {};
case replication::RegisterReplicaError::NAME_EXISTS:
return dbms::RegisterReplicaError::NAME_EXISTS;
case replication::RegisterReplicaError::END_POINT_EXISTS:
return dbms::RegisterReplicaError::END_POINT_EXISTS;
case replication::RegisterReplicaError::COULD_NOT_BE_PERSISTED:
return dbms::RegisterReplicaError::COULD_NOT_BE_PERSISTED;
case replication::RegisterReplicaError::SUCCESS:
break;
}
return {};
}
} // namespace memgraph::dbms

View File

@ -6,6 +6,7 @@ add_library(mg-flags STATIC audit.cpp
memory_limit.cpp
run_time_configurable.cpp
storage_mode.cpp
query.cpp)
query.cpp
replication.cpp)
target_include_directories(mg-flags PUBLIC ${CMAKE_SOURCE_DIR}/include)
target_link_libraries(mg-flags PUBLIC spdlog::spdlog mg-settings mg-utils)

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -17,5 +17,6 @@
#include "flags/log_level.hpp"
#include "flags/memory_limit.hpp"
#include "flags/query.hpp"
#include "flags/replication.hpp"
#include "flags/run_time_configurable.hpp"
#include "flags/storage_mode.hpp"

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -156,13 +156,6 @@ DEFINE_string(pulsar_service_url, "", "Default URL used while connecting to Puls
// Query flags.
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_uint64(replication_replica_check_frequency_sec, 1,
"The time duration between two replica checks/pings. If < 1, replicas will NOT be checked at all. NOTE: "
"The MAIN instance allocates a new thread for each REPLICA.");
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_bool(replication_restore_state_on_startup, false, "Restore replication state on startup, e.g. recover replica");
DEFINE_VALIDATED_string(query_modules_directory, "",
"Directory where modules with custom query procedures are stored. "
"NOTE: Multiple comma-separated directories can be defined.",

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -112,11 +112,6 @@ namespace memgraph::flags {
auto ParseQueryModulesDirectory() -> std::vector<std::filesystem::path>;
} // namespace memgraph::flags
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DECLARE_uint64(replication_replica_check_frequency_sec);
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DECLARE_bool(replication_restore_state_on_startup);
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DECLARE_string(license_key);
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)

26
src/flags/replication.cpp Normal file
View File

@ -0,0 +1,26 @@
// Copyright 2024 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 "replication.hpp"
#ifdef MG_ENTERPRISE
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_bool(coordinator, false, "Controls whether the instance is a replication coordinator.");
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_uint32(coordinator_server_port, 0, "Port on which coordinator servers will be started.");
#endif
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_uint64(replication_replica_check_frequency_sec, 1,
"The time duration between two replica checks/pings. If < 1, replicas will NOT be checked at all. NOTE: "
"The MAIN instance allocates a new thread for each REPLICA.");
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DEFINE_bool(replication_restore_state_on_startup, false, "Restore replication state on startup, e.g. recover replica");

26
src/flags/replication.hpp Normal file
View File

@ -0,0 +1,26 @@
// Copyright 2024 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 "gflags/gflags.h"
#ifdef MG_ENTERPRISE
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DECLARE_bool(coordinator);
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DECLARE_uint32(coordinator_server_port);
#endif
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DECLARE_uint64(replication_replica_check_frequency_sec);
// NOLINTNEXTLINE(cppcoreguidelines-avoid-non-const-global-variables)
DECLARE_bool(replication_restore_state_on_startup);

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -193,12 +193,13 @@ std::pair<std::vector<std::string>, std::optional<int>> SessionHL::Interpret(
for (const auto &[key, bolt_param] : params) {
params_pv.emplace(key, ToPropertyValue(bolt_param));
}
#ifdef MG_ENTERPRISE
const std::string *username{nullptr};
if (user_) {
username = &user_->username();
}
#ifdef MG_ENTERPRISE
if (memgraph::license::global_license_checker.IsEnterpriseValidFast()) {
auto &db = interpreter_.current_db_.db_acc_;
audit_log_->Record(endpoint_.address().to_string(), user_ ? *username : "", query,

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -66,6 +66,8 @@ auth::Permission PrivilegeToPermission(query::AuthQuery::Privilege privilege) {
return auth::Permission::MULTI_DATABASE_EDIT;
case query::AuthQuery::Privilege::MULTI_DATABASE_USE:
return auth::Permission::MULTI_DATABASE_USE;
case query::AuthQuery::Privilege::COORDINATOR:
return auth::Permission::COORDINATOR;
}
}

View File

@ -8,4 +8,5 @@ find_package(fmt REQUIRED)
find_package(Threads REQUIRED)
add_library(mg-io STATIC ${io_src_files})
add_library(mg::io ALIAS mg-io)
target_link_libraries(mg-io stdc++fs Threads::Threads fmt::fmt mg-utils)

View File

@ -166,7 +166,7 @@ bool Endpoint::IsResolvableAddress(const std::string &address, uint16_t port) {
}
std::optional<std::pair<std::string, uint16_t>> Endpoint::ParseSocketOrAddress(
const std::string &address, const std::optional<uint16_t> default_port = {}) {
const std::string &address, const std::optional<uint16_t> default_port) {
const std::string delimiter = ":";
std::vector<std::string> parts = utils::Split(address, delimiter);
if (parts.size() == 1) {

View File

@ -16,7 +16,6 @@
#include "dbms/constants.hpp"
#include "dbms/inmemory/replication_handlers.hpp"
#include "flags/all.hpp"
#include "flags/run_time_configurable.hpp"
#include "glue/MonitoringServerT.hpp"
#include "glue/ServerT.hpp"
#include "glue/auth_checker.hpp"

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -259,6 +259,13 @@ class ReplicationModificationInMulticommandTxException : public QueryException {
SPECIALIZE_GET_EXCEPTION_NAME(ReplicationModificationInMulticommandTxException)
};
class CoordinatorModificationInMulticommandTxException : public QueryException {
public:
CoordinatorModificationInMulticommandTxException()
: QueryException("Coordinator clause not allowed in multicommand transactions.") {}
SPECIALIZE_GET_EXCEPTION_NAME(CoordinatorModificationInMulticommandTxException)
};
class ReplicationDisabledOnDiskStorage : public QueryException {
public:
ReplicationDisabledOnDiskStorage() : QueryException("Replication is not supported while in on-disk storage mode.") {}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -239,6 +239,9 @@ constexpr utils::TypeInfo query::DumpQuery::kType{utils::TypeId::AST_DUMP_QUERY,
constexpr utils::TypeInfo query::ReplicationQuery::kType{utils::TypeId::AST_REPLICATION_QUERY, "ReplicationQuery",
&query::Query::kType};
constexpr utils::TypeInfo query::CoordinatorQuery::kType{utils::TypeId::AST_COORDINATOR_QUERY, "CoordinatorQuery",
&query::Query::kType};
constexpr utils::TypeInfo query::LockPathQuery::kType{utils::TypeId::AST_LOCK_PATH_QUERY, "LockPathQuery",
&query::Query::kType};

View File

@ -2849,6 +2849,7 @@ class AuthQuery : public memgraph::query::Query {
TRANSACTION_MANAGEMENT,
MULTI_DATABASE_EDIT,
MULTI_DATABASE_USE,
COORDINATOR
};
enum class FineGrainedPrivilege { NOTHING, READ, UPDATE, CREATE_DELETE };
@ -2927,7 +2928,8 @@ const std::vector<AuthQuery::Privilege> kPrivilegesAll = {AuthQuery::Privilege::
AuthQuery::Privilege::TRANSACTION_MANAGEMENT,
AuthQuery::Privilege::STORAGE_MODE,
AuthQuery::Privilege::MULTI_DATABASE_EDIT,
AuthQuery::Privilege::MULTI_DATABASE_USE};
AuthQuery::Privilege::MULTI_DATABASE_USE,
AuthQuery::Privilege::COORDINATOR};
class DatabaseInfoQuery : public memgraph::query::Query {
public:
@ -3039,8 +3041,9 @@ class ReplicationQuery : public memgraph::query::Query {
memgraph::query::ReplicationQuery::Action action_;
memgraph::query::ReplicationQuery::ReplicationRole role_;
std::string replica_name_;
std::string instance_name_;
memgraph::query::Expression *socket_address_{nullptr};
memgraph::query::Expression *coordinator_socket_address_{nullptr};
memgraph::query::Expression *port_{nullptr};
memgraph::query::ReplicationQuery::SyncMode sync_mode_;
@ -3048,10 +3051,57 @@ class ReplicationQuery : public memgraph::query::Query {
ReplicationQuery *object = storage->Create<ReplicationQuery>();
object->action_ = action_;
object->role_ = role_;
object->replica_name_ = replica_name_;
object->instance_name_ = instance_name_;
object->socket_address_ = socket_address_ ? socket_address_->Clone(storage) : nullptr;
object->port_ = port_ ? port_->Clone(storage) : nullptr;
object->sync_mode_ = sync_mode_;
object->coordinator_socket_address_ =
coordinator_socket_address_ ? coordinator_socket_address_->Clone(storage) : nullptr;
return object;
}
private:
friend class AstStorage;
};
class CoordinatorQuery : public memgraph::query::Query {
public:
static const utils::TypeInfo kType;
const utils::TypeInfo &GetTypeInfo() const override { return kType; }
enum class Action {
REGISTER_MAIN_COORDINATOR_SERVER,
REGISTER_REPLICA_COORDINATOR_SERVER,
SHOW_REPLICATION_CLUSTER,
DO_FAILOVER
};
enum class ReplicationRole { MAIN, REPLICA };
enum class SyncMode { SYNC, ASYNC };
CoordinatorQuery() = default;
DEFVISITABLE(QueryVisitor<void>);
memgraph::query::CoordinatorQuery::Action action_;
memgraph::query::CoordinatorQuery::ReplicationRole role_;
std::string instance_name_;
memgraph::query::Expression *socket_address_{nullptr};
memgraph::query::Expression *coordinator_socket_address_{nullptr};
memgraph::query::CoordinatorQuery::SyncMode sync_mode_;
CoordinatorQuery *Clone(AstStorage *storage) const override {
auto *object = storage->Create<CoordinatorQuery>();
object->action_ = action_;
object->role_ = role_;
object->instance_name_ = instance_name_;
object->socket_address_ = socket_address_ ? socket_address_->Clone(storage) : nullptr;
object->sync_mode_ = sync_mode_;
object->coordinator_socket_address_ =
coordinator_socket_address_ ? coordinator_socket_address_->Clone(storage) : nullptr;
return object;
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -108,6 +108,7 @@ class MultiDatabaseQuery;
class ShowDatabasesQuery;
class EdgeImportModeQuery;
class PatternComprehension;
class CoordinatorQuery;
using TreeCompositeVisitor = utils::CompositeVisitor<
SingleQuery, CypherUnion, NamedExpression, OrOperator, XorOperator, AndOperator, NotOperator, AdditionOperator,
@ -146,6 +147,7 @@ class QueryVisitor
SystemInfoQuery, ConstraintQuery, DumpQuery, ReplicationQuery, LockPathQuery,
FreeMemoryQuery, TriggerQuery, IsolationLevelQuery, CreateSnapshotQuery, StreamQuery,
SettingQuery, VersionQuery, ShowConfigQuery, TransactionQueueQuery, StorageModeQuery,
AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery, EdgeImportModeQuery> {};
AnalyzeGraphQuery, MultiDatabaseQuery, ShowDatabasesQuery, EdgeImportModeQuery,
CoordinatorQuery> {};
} // namespace memgraph::query

View File

@ -302,6 +302,13 @@ antlrcpp::Any CypherMainVisitor::visitReplicationQuery(MemgraphCypher::Replicati
return replication_query;
}
antlrcpp::Any CypherMainVisitor::visitCoordinatorQuery(MemgraphCypher::CoordinatorQueryContext *ctx) {
MG_ASSERT(ctx->children.size() == 1, "CoordinatorQuery should have exactly one child!");
auto *coordinator_query = std::any_cast<CoordinatorQuery *>(ctx->children[0]->accept(this));
query_ = coordinator_query;
return coordinator_query;
}
antlrcpp::Any CypherMainVisitor::visitEdgeImportModeQuery(MemgraphCypher::EdgeImportModeQueryContext *ctx) {
auto *edge_import_mode_query = storage_->Create<EdgeImportModeQuery>();
if (ctx->ACTIVE()) {
@ -316,24 +323,34 @@ antlrcpp::Any CypherMainVisitor::visitEdgeImportModeQuery(MemgraphCypher::EdgeIm
antlrcpp::Any CypherMainVisitor::visitSetReplicationRole(MemgraphCypher::SetReplicationRoleContext *ctx) {
auto *replication_query = storage_->Create<ReplicationQuery>();
replication_query->action_ = ReplicationQuery::Action::SET_REPLICATION_ROLE;
auto set_replication_port = [replication_query, ctx, this]() -> void {
if (ctx->port->numberLiteral() && ctx->port->numberLiteral()->integerLiteral()) {
replication_query->port_ = std::any_cast<Expression *>(ctx->port->accept(this));
} else {
throw SyntaxException("Port must be an integer literal!");
}
};
if (ctx->MAIN()) {
replication_query->role_ = ReplicationQuery::ReplicationRole::MAIN;
if (ctx->WITH() || ctx->PORT()) {
throw SemanticException("Main can't set a port!");
}
replication_query->role_ = ReplicationQuery::ReplicationRole::MAIN;
} else if (ctx->REPLICA()) {
replication_query->role_ = ReplicationQuery::ReplicationRole::REPLICA;
if (ctx->WITH() && ctx->PORT()) {
if (ctx->port->numberLiteral() && ctx->port->numberLiteral()->integerLiteral()) {
replication_query->port_ = std::any_cast<Expression *>(ctx->port->accept(this));
} else {
throw SyntaxException("Port must be an integer literal!");
}
set_replication_port();
} else {
throw SemanticException("Replica must set a port!");
}
}
return replication_query;
}
antlrcpp::Any CypherMainVisitor::visitShowReplicationRole(MemgraphCypher::ShowReplicationRoleContext *ctx) {
antlrcpp::Any CypherMainVisitor::visitShowReplicationRole(MemgraphCypher::ShowReplicationRoleContext * /*ctx*/) {
auto *replication_query = storage_->Create<ReplicationQuery>();
replication_query->action_ = ReplicationQuery::Action::SHOW_REPLICATION_ROLE;
return replication_query;
@ -342,7 +359,7 @@ antlrcpp::Any CypherMainVisitor::visitShowReplicationRole(MemgraphCypher::ShowRe
antlrcpp::Any CypherMainVisitor::visitRegisterReplica(MemgraphCypher::RegisterReplicaContext *ctx) {
auto *replication_query = storage_->Create<ReplicationQuery>();
replication_query->action_ = ReplicationQuery::Action::REGISTER_REPLICA;
replication_query->replica_name_ = std::any_cast<std::string>(ctx->replicaName()->symbolicName()->accept(this));
replication_query->instance_name_ = std::any_cast<std::string>(ctx->instanceName()->symbolicName()->accept(this));
if (ctx->SYNC()) {
replication_query->sync_mode_ = memgraph::query::ReplicationQuery::SyncMode::SYNC;
} else if (ctx->ASYNC()) {
@ -351,26 +368,90 @@ antlrcpp::Any CypherMainVisitor::visitRegisterReplica(MemgraphCypher::RegisterRe
if (!ctx->socketAddress()->literal()->StringLiteral()) {
throw SemanticException("Socket address should be a string literal!");
} else {
replication_query->socket_address_ = std::any_cast<Expression *>(ctx->socketAddress()->accept(this));
}
replication_query->socket_address_ = std::any_cast<Expression *>(ctx->socketAddress()->accept(this));
return replication_query;
}
// License check is done in the interpreter.
antlrcpp::Any CypherMainVisitor::visitRegisterCoordinatorServer(MemgraphCypher::RegisterCoordinatorServerContext *ctx) {
MG_ASSERT(ctx->children.size() == 1, "RegisterCoordinatorServerQuery should have exactly one child!");
auto *coordinator_query = std::any_cast<CoordinatorQuery *>(ctx->children[0]->accept(this));
query_ = coordinator_query;
return coordinator_query;
}
// License check is done in the interpreter
antlrcpp::Any CypherMainVisitor::visitShowReplicationCluster(MemgraphCypher::ShowReplicationClusterContext * /*ctx*/) {
auto *coordinator_query = storage_->Create<CoordinatorQuery>();
coordinator_query->action_ = CoordinatorQuery::Action::SHOW_REPLICATION_CLUSTER;
return coordinator_query;
}
// License check is done in the interpreter
antlrcpp::Any CypherMainVisitor::visitRegisterReplicaCoordinatorServer(
MemgraphCypher::RegisterReplicaCoordinatorServerContext *ctx) {
auto *coordinator_query = storage_->Create<CoordinatorQuery>();
if (!ctx->socketAddress()->literal()->StringLiteral()) {
throw SemanticException("Socket address should be a string literal!");
}
if (!ctx->coordinatorSocketAddress()->literal()->StringLiteral()) {
throw SemanticException("Coordinator socket address should be a string literal!");
}
coordinator_query->action_ = CoordinatorQuery::Action::REGISTER_REPLICA_COORDINATOR_SERVER;
coordinator_query->role_ = CoordinatorQuery::ReplicationRole::REPLICA;
coordinator_query->socket_address_ = std::any_cast<Expression *>(ctx->socketAddress()->accept(this));
coordinator_query->coordinator_socket_address_ =
std::any_cast<Expression *>(ctx->coordinatorSocketAddress()->accept(this));
coordinator_query->instance_name_ = std::any_cast<std::string>(ctx->instanceName()->symbolicName()->accept(this));
if (ctx->SYNC()) {
coordinator_query->sync_mode_ = memgraph::query::CoordinatorQuery::SyncMode::SYNC;
} else if (ctx->ASYNC()) {
coordinator_query->sync_mode_ = memgraph::query::CoordinatorQuery::SyncMode::ASYNC;
}
return coordinator_query;
}
// License check is done in the interpreter
antlrcpp::Any CypherMainVisitor::visitRegisterMainCoordinatorServer(
MemgraphCypher::RegisterMainCoordinatorServerContext *ctx) {
if (!ctx->coordinatorSocketAddress()->literal()->StringLiteral()) {
throw SemanticException("Coordinator socket address should be a string literal!");
}
auto *coordinator_query = storage_->Create<CoordinatorQuery>();
coordinator_query->action_ = CoordinatorQuery::Action::REGISTER_MAIN_COORDINATOR_SERVER;
coordinator_query->role_ = CoordinatorQuery::ReplicationRole::MAIN;
coordinator_query->coordinator_socket_address_ =
std::any_cast<Expression *>(ctx->coordinatorSocketAddress()->accept(this));
coordinator_query->instance_name_ = std::any_cast<std::string>(ctx->instanceName()->symbolicName()->accept(this));
return coordinator_query;
}
antlrcpp::Any CypherMainVisitor::visitDropReplica(MemgraphCypher::DropReplicaContext *ctx) {
auto *replication_query = storage_->Create<ReplicationQuery>();
replication_query->action_ = ReplicationQuery::Action::DROP_REPLICA;
replication_query->replica_name_ = std::any_cast<std::string>(ctx->replicaName()->symbolicName()->accept(this));
replication_query->instance_name_ = std::any_cast<std::string>(ctx->instanceName()->symbolicName()->accept(this));
return replication_query;
}
antlrcpp::Any CypherMainVisitor::visitShowReplicas(MemgraphCypher::ShowReplicasContext *ctx) {
antlrcpp::Any CypherMainVisitor::visitShowReplicas(MemgraphCypher::ShowReplicasContext * /*ctx*/) {
auto *replication_query = storage_->Create<ReplicationQuery>();
replication_query->action_ = ReplicationQuery::Action::SHOW_REPLICAS;
return replication_query;
}
// License check is done in the interpreter
antlrcpp::Any CypherMainVisitor::visitDoFailover(MemgraphCypher::DoFailoverContext * /*ctx*/) {
auto *coordinator_query = storage_->Create<CoordinatorQuery>();
coordinator_query->action_ = CoordinatorQuery::Action::DO_FAILOVER;
query_ = coordinator_query;
return coordinator_query;
}
antlrcpp::Any CypherMainVisitor::visitLockPathQuery(MemgraphCypher::LockPathQueryContext *ctx) {
auto *lock_query = storage_->Create<LockPathQuery>();
if (ctx->STATUS()) {
@ -1638,6 +1719,7 @@ antlrcpp::Any CypherMainVisitor::visitPrivilege(MemgraphCypher::PrivilegeContext
if (ctx->STORAGE_MODE()) return AuthQuery::Privilege::STORAGE_MODE;
if (ctx->MULTI_DATABASE_EDIT()) return AuthQuery::Privilege::MULTI_DATABASE_EDIT;
if (ctx->MULTI_DATABASE_USE()) return AuthQuery::Privilege::MULTI_DATABASE_USE;
if (ctx->COORDINATOR()) return AuthQuery::Privilege::COORDINATOR;
LOG_FATAL("Should not get here - unknown privilege!");
}

View File

@ -233,6 +233,37 @@ class CypherMainVisitor : public antlropencypher::MemgraphCypherBaseVisitor {
*/
antlrcpp::Any visitShowReplicas(MemgraphCypher::ShowReplicasContext *ctx) override;
/**
* @return CoordinatorQuery*
*/
antlrcpp::Any visitCoordinatorQuery(MemgraphCypher::CoordinatorQueryContext *ctx) override;
/**
* @return CoordinatorQuery*
*/
antlrcpp::Any visitRegisterCoordinatorServer(MemgraphCypher::RegisterCoordinatorServerContext *ctx) override;
/**
* @return CoordinatorQuery*
*/
antlrcpp::Any visitRegisterMainCoordinatorServer(MemgraphCypher::RegisterMainCoordinatorServerContext *ctx) override;
/**
* @return CoordinatorQuery*
*/
antlrcpp::Any visitRegisterReplicaCoordinatorServer(
MemgraphCypher::RegisterReplicaCoordinatorServerContext *ctx) override;
/**
* @return CoordinatorQuery*
*/
antlrcpp::Any visitShowReplicationCluster(MemgraphCypher::ShowReplicationClusterContext *ctx) override;
/**
* @return CoordinatorQuery*
*/
antlrcpp::Any visitDoFailover(MemgraphCypher::DoFailoverContext *ctx) override;
/**
* @return LockPathQuery*
*/

View File

@ -48,10 +48,12 @@ memgraphCypherKeyword : cypherKeyword
| DATABASE
| DENY
| DROP
| DO
| DUMP
| EDGE
| EDGE_TYPES
| EXECUTE
| FAILOVER
| FOR
| FOREACH
| FREE
@ -151,6 +153,7 @@ query : cypherQuery
| multiDatabaseQuery
| showDatabases
| edgeImportModeQuery
| coordinatorQuery
;
cypherQuery : ( indexHints )? singleQuery ( cypherUnion )* ( queryMemoryLimit )? ;
@ -183,6 +186,11 @@ replicationQuery : setReplicationRole
| showReplicas
;
coordinatorQuery : registerCoordinatorServer
| showReplicationCluster
| doFailover
;
triggerQuery : createTrigger
| dropTrigger
| showTriggers
@ -244,6 +252,8 @@ transactionQueueQuery : showTransactions
showTransactions : SHOW TRANSACTIONS ;
doFailover : DO FAILOVER ;
terminateTransactions : TERMINATE TRANSACTIONS transactionIdList;
loadCsv : LOAD CSV FROM csvFile ( WITH | NO ) HEADER
@ -323,6 +333,7 @@ privilege : CREATE
| STORAGE_MODE
| MULTI_DATABASE_EDIT
| MULTI_DATABASE_USE
| COORDINATOR
;
granularPrivilege : NOTHING | READ | UPDATE | CREATE_DELETE ;
@ -364,14 +375,24 @@ setReplicationRole : SET REPLICATION ROLE TO ( MAIN | REPLICA )
showReplicationRole : SHOW REPLICATION ROLE ;
replicaName : symbolicName ;
showReplicationCluster : SHOW REPLICATION CLUSTER ;
instanceName : symbolicName ;
socketAddress : literal ;
registerReplica : REGISTER REPLICA replicaName ( SYNC | ASYNC )
coordinatorSocketAddress : literal ;
registerReplica : REGISTER REPLICA instanceName ( SYNC | ASYNC )
TO socketAddress ;
dropReplica : DROP REPLICA replicaName ;
registerReplicaCoordinatorServer: REGISTER REPLICA instanceName ( ASYNC | SYNC ) TO socketAddress WITH COORDINATOR SERVER ON coordinatorSocketAddress ;
registerMainCoordinatorServer: REGISTER MAIN instanceName WITH COORDINATOR SERVER ON coordinatorSocketAddress ;
registerCoordinatorServer : registerMainCoordinatorServer | registerReplicaCoordinatorServer ;
dropReplica : DROP REPLICA instanceName ;
showReplicas : SHOW REPLICAS ;

View File

@ -39,15 +39,18 @@ BOOTSTRAP_SERVERS : B O O T S T R A P UNDERSCORE S E R V E R S ;
CALL : C A L L ;
CHECK : C H E C K ;
CLEAR : C L E A R ;
CLUSTER : C L U S T E R ;
COMMIT : C O M M I T ;
COMMITTED : C O M M I T T E D ;
CONFIG : C O N F I G ;
CONFIGS : C O N F I G S;
CONSUMER_GROUP : C O N S U M E R UNDERSCORE G R O U P ;
COORDINATOR : C O O R D I N A T O R ;
CREATE_DELETE : C R E A T E UNDERSCORE D E L E T E ;
CREDENTIALS : C R E D E N T I A L S ;
CSV : C S V ;
DATA : D A T A ;
DO : D O ;
DELIMITER : D E L I M I T E R ;
DATABASE : D A T A B A S E ;
DATABASES : D A T A B A S E S ;
@ -59,6 +62,7 @@ DURABILITY : D U R A B I L I T Y ;
EDGE : E D G E ;
EDGE_TYPES : E D G E UNDERSCORE T Y P E S ;
EXECUTE : E X E C U T E ;
FAILOVER : F A I L O V E R ;
FOR : F O R ;
FOREACH : F O R E A C H;
FREE : F R E E ;
@ -107,6 +111,7 @@ REVOKE : R E V O K E ;
ROLE : R O L E ;
ROLES : R O L E S ;
QUOTE : Q U O T E ;
SERVER : S E R V E R ;
SERVICE_URL : S E R V I C E UNDERSCORE U R L ;
SESSION : S E S S I O N ;
SETTING : S E T T I N G ;

View File

@ -116,6 +116,8 @@ class PrivilegeExtractor : public QueryVisitor<void>, public HierarchicalTreeVis
AddPrivilege(AuthQuery::Privilege::MULTI_DATABASE_USE); /* OR EDIT */
}
void Visit(CoordinatorQuery & /*coordinator_query*/) override { AddPrivilege(AuthQuery::Privilege::COORDINATOR); }
bool PreVisit(Create & /*unused*/) override {
AddPrivilege(AuthQuery::Privilege::CREATE);
return false;

View File

@ -35,9 +35,9 @@
#include "auth/models.hpp"
#include "csv/parsing.hpp"
#include "dbms/database.hpp"
#include "dbms/dbms_handler.hpp"
#include "dbms/global.hpp"
#include "dbms/inmemory/storage_helper.hpp"
#include "flags/replication.hpp"
#include "flags/run_time_configurable.hpp"
#include "glue/communication.hpp"
#include "license/license.hpp"
@ -101,12 +101,18 @@
#include "utils/typeinfo.hpp"
#include "utils/variant_helpers.hpp"
#include "dbms/coordinator_handler.hpp"
#include "dbms/dbms_handler.hpp"
#include "dbms/replication_handler.hpp"
#include "query/auth_query_handler.hpp"
#include "query/interpreter_context.hpp"
#include "replication/state.hpp"
#ifdef MG_ENTERPRISE
#include "coordination/constants.hpp"
#include "coordination/coordinator_entity_info.hpp"
#endif
namespace memgraph::metrics {
extern Event ReadQuery;
extern Event WriteQuery;
@ -121,6 +127,7 @@ extern const Event CommitedTransactions;
extern const Event RollbackedTransactions;
extern const Event ActiveTransactions;
} // namespace memgraph::metrics
void memgraph::query::CurrentDB::SetupDatabaseTransaction(
std::optional<storage::IsolationLevel> override_isolation_level, bool could_commit, bool unique) {
auto &db_acc = *db_acc_;
@ -260,17 +267,32 @@ bool IsAllShortestPathsQuery(const std::vector<memgraph::query::Clause *> &claus
return false;
}
inline auto convertToReplicationMode(const ReplicationQuery::SyncMode &sync_mode) -> replication::ReplicationMode {
inline auto convertFromCoordinatorToReplicationMode(const CoordinatorQuery::SyncMode &sync_mode)
-> replication_coordination_glue::ReplicationMode {
switch (sync_mode) {
case ReplicationQuery::SyncMode::ASYNC: {
return replication::ReplicationMode::ASYNC;
case CoordinatorQuery::SyncMode::ASYNC: {
return replication_coordination_glue::ReplicationMode::ASYNC;
}
case ReplicationQuery::SyncMode::SYNC: {
return replication::ReplicationMode::SYNC;
case CoordinatorQuery::SyncMode::SYNC: {
return replication_coordination_glue::ReplicationMode::SYNC;
}
}
// TODO: C++23 std::unreachable()
return replication::ReplicationMode::ASYNC;
return replication_coordination_glue::ReplicationMode::ASYNC;
}
inline auto convertToReplicationMode(const ReplicationQuery::SyncMode &sync_mode)
-> replication_coordination_glue::ReplicationMode {
switch (sync_mode) {
case ReplicationQuery::SyncMode::ASYNC: {
return replication_coordination_glue::ReplicationMode::ASYNC;
}
case ReplicationQuery::SyncMode::SYNC: {
return replication_coordination_glue::ReplicationMode::SYNC;
}
}
// TODO: C++23 std::unreachable()
return replication_coordination_glue::ReplicationMode::ASYNC;
}
class ReplQueryHandler {
@ -289,14 +311,17 @@ class ReplQueryHandler {
/// @throw QueryRuntimeException if an error ocurred.
void SetReplicationRole(ReplicationQuery::ReplicationRole replication_role, std::optional<int64_t> port) {
if (replication_role == ReplicationQuery::ReplicationRole::MAIN) {
if (!handler_.SetReplicationRoleMain()) {
throw QueryRuntimeException("Couldn't set role to main!");
}
} else {
if (!port || *port < 0 || *port > std::numeric_limits<uint16_t>::max()) {
auto ValidatePort = [](std::optional<int64_t> port) -> void {
if (*port < 0 || *port > std::numeric_limits<uint16_t>::max()) {
throw QueryRuntimeException("Port number invalid!");
}
};
if (replication_role == ReplicationQuery::ReplicationRole::MAIN) {
if (!handler_.SetReplicationRoleMain()) {
throw QueryRuntimeException("Couldn't set replication role to main!");
}
} else {
ValidatePort(port);
auto const config = memgraph::replication::ReplicationServerConfig{
.ip_address = memgraph::replication::kDefaultReplicationServerIp,
@ -323,27 +348,33 @@ class ReplQueryHandler {
/// @throw QueryRuntimeException if an error ocurred.
void RegisterReplica(const std::string &name, const std::string &socket_address,
const ReplicationQuery::SyncMode sync_mode, const std::chrono::seconds replica_check_frequency) {
// Coordinator is main by default so this check is OK although it should actually be nothing (neither main nor
// replica)
if (handler_.IsReplica()) {
// replica can't register another replica
throw QueryRuntimeException("Replica can't register another replica!");
}
auto repl_mode = convertToReplicationMode(sync_mode);
const auto repl_mode = convertToReplicationMode(sync_mode);
auto maybe_ip_and_port =
const auto maybe_ip_and_port =
io::network::Endpoint::ParseSocketOrAddress(socket_address, memgraph::replication::kDefaultReplicationPort);
if (maybe_ip_and_port) {
auto [ip, port] = *maybe_ip_and_port;
auto config = replication::ReplicationClientConfig{.name = name,
.mode = repl_mode,
.ip_address = ip,
.port = port,
.replica_check_frequency = replica_check_frequency,
.ssl = std::nullopt};
auto ret = handler_.RegisterReplica(config);
if (ret.HasError()) {
const auto [ip, port] = *maybe_ip_and_port;
const auto replication_config =
replication::ReplicationClientConfig{.name = name,
.mode = repl_mode,
.ip_address = ip,
.port = port,
.replica_check_frequency = replica_check_frequency,
.ssl = std::nullopt};
const auto error = handler_.RegisterReplica(replication_config).HasError();
if (error) {
throw QueryRuntimeException(fmt::format("Couldn't register replica '{}'!", name));
}
} else {
throw QueryRuntimeException("Invalid socket address!");
}
@ -382,10 +413,10 @@ class ReplQueryHandler {
replica.name = repl_info.name;
replica.socket_address = repl_info.endpoint.SocketAddress();
switch (repl_info.mode) {
case memgraph::replication::ReplicationMode::SYNC:
case replication_coordination_glue::ReplicationMode::SYNC:
replica.sync_mode = ReplicationQuery::SyncMode::SYNC;
break;
case memgraph::replication::ReplicationMode::ASYNC:
case replication_coordination_glue::ReplicationMode::ASYNC:
replica.sync_mode = ReplicationQuery::SyncMode::ASYNC;
break;
}
@ -420,6 +451,140 @@ class ReplQueryHandler {
dbms::ReplicationHandler handler_;
};
class CoordQueryHandler final : public query::CoordinatorQueryHandler {
public:
explicit CoordQueryHandler(dbms::DbmsHandler *dbms_handler) : handler_ { *dbms_handler }
#ifdef MG_ENTERPRISE
, coordinator_handler_(*dbms_handler)
#endif
{
}
#ifdef MG_ENTERPRISE
/// @throw QueryRuntimeException if an error ocurred.
void RegisterReplicaCoordinatorServer(const std::string &replication_socket_address,
const std::string &coordinator_socket_address,
const std::chrono::seconds instance_check_frequency,
const std::string &instance_name,
CoordinatorQuery::SyncMode sync_mode) override {
const auto maybe_replication_ip_port =
io::network::Endpoint::ParseSocketOrAddress(replication_socket_address, std::nullopt);
if (!maybe_replication_ip_port) {
throw QueryRuntimeException("Invalid replication socket address!");
}
const auto maybe_coordinator_ip_port =
io::network::Endpoint::ParseSocketOrAddress(coordinator_socket_address, std::nullopt);
if (!maybe_replication_ip_port) {
throw QueryRuntimeException("Invalid replication socket address!");
}
const auto [replication_ip, replication_port] = *maybe_replication_ip_port;
const auto [coordinator_server_ip, coordinator_server_port] = *maybe_coordinator_ip_port;
const auto repl_config = coordination::CoordinatorClientConfig::ReplicationClientInfo{
.instance_name = instance_name,
.replication_mode = convertFromCoordinatorToReplicationMode(sync_mode),
.replication_ip_address = replication_ip,
.replication_port = replication_port};
const auto coordinator_client_config =
coordination::CoordinatorClientConfig{.instance_name = instance_name,
.ip_address = coordinator_server_ip,
.port = coordinator_server_port,
.health_check_frequency_sec = instance_check_frequency,
.replication_client_info = repl_config,
.ssl = std::nullopt};
if (const auto ret = coordinator_handler_.RegisterReplicaOnCoordinator(coordinator_client_config); ret.HasError()) {
throw QueryRuntimeException("Couldn't register replica on coordinator!");
}
}
void RegisterMainCoordinatorServer(const std::string &coordinator_socket_address,
const std::chrono::seconds instance_check_frequency,
const std::string &instance_name) override {
const auto maybe_ip_and_port =
io::network::Endpoint::ParseSocketOrAddress(coordinator_socket_address, std::nullopt);
if (!maybe_ip_and_port) {
throw QueryRuntimeException("Invalid socket address!");
}
const auto [ip, port] = *maybe_ip_and_port;
const auto config = coordination::CoordinatorClientConfig{.instance_name = instance_name,
.ip_address = ip,
.port = port,
.health_check_frequency_sec = instance_check_frequency,
.ssl = std::nullopt};
if (const auto ret = coordinator_handler_.RegisterMainOnCoordinator(config); ret.HasError()) {
throw QueryRuntimeException("Couldn't register main on coordinator!");
}
}
/// @throw QueryRuntimeException if an error ocurred.
void DoFailover() const override {
if (!FLAGS_coordinator) {
throw QueryRuntimeException("Only coordinator can register coordinator server!");
}
auto status = coordinator_handler_.DoFailover();
switch (status) {
using enum memgraph::dbms::DoFailoverStatus;
case ALL_REPLICAS_DOWN:
throw QueryRuntimeException("Failover aborted since all replicas are down!");
case MAIN_ALIVE:
throw QueryRuntimeException("Failover aborted since main is alive!");
case CLUSTER_UNINITIALIZED:
throw QueryRuntimeException("Failover aborted since cluster is uninitialized!");
case SUCCESS:
break;
}
}
std::vector<MainReplicaStatus> ShowMainReplicaStatus(
const std::vector<coordination::CoordinatorEntityInfo> &replicas,
const std::unordered_map<std::string_view, bool> &health_check_replicas,
const std::optional<coordination::CoordinatorEntityInfo> &main,
const std::optional<coordination::CoordinatorEntityHealthInfo> &health_check_main) const override {
std::vector<MainReplicaStatus> result{};
result.reserve(replicas.size() + 1); // replicas + 1 main
std::ranges::transform(
replicas, std::back_inserter(result), [&health_check_replicas](const auto &replica) -> MainReplicaStatus {
return {replica.name, replica.endpoint.SocketAddress(), health_check_replicas.at(replica.name), false};
});
if (main) {
bool is_main_alive = health_check_main.has_value() ? health_check_main.value().alive : false;
result.emplace_back(main->name, main->endpoint.SocketAddress(), is_main_alive, true);
}
return result;
}
#endif
#ifdef MG_ENTERPRISE
std::vector<coordination::CoordinatorEntityInfo> ShowReplicasOnCoordinator() const override {
return coordinator_handler_.ShowReplicasOnCoordinator();
}
std::unordered_map<std::string_view, bool> PingReplicasOnCoordinator() const override {
return coordinator_handler_.PingReplicasOnCoordinator();
}
std::optional<coordination::CoordinatorEntityInfo> ShowMainOnCoordinator() const override {
return coordinator_handler_.ShowMainOnCoordinator();
}
std::optional<coordination::CoordinatorEntityHealthInfo> PingMainOnCoordinator() const override {
return coordinator_handler_.PingMainOnCoordinator();
}
#endif
private:
dbms::ReplicationHandler handler_;
#ifdef MG_ENTERPRISE
dbms::CoordinatorHandler coordinator_handler_;
#endif
};
/// returns false if the replication role can't be set
/// @throw QueryRuntimeException if an error ocurred.
@ -723,6 +888,15 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
Callback callback;
switch (repl_query->action_) {
case ReplicationQuery::Action::SET_REPLICATION_ROLE: {
#ifdef MG_ENTERPRISE
if (FLAGS_coordinator) {
if (repl_query->role_ == ReplicationQuery::ReplicationRole::REPLICA) {
throw QueryRuntimeException("Coordinator cannot become a replica!");
}
throw QueryRuntimeException("Coordinator cannot become main!");
}
#endif
auto port = EvaluateOptionalExpression(repl_query->port_, evaluator);
std::optional<int64_t> maybe_port;
if (port.IsInt()) {
@ -743,6 +917,12 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
return callback;
}
case ReplicationQuery::Action::SHOW_REPLICATION_ROLE: {
#ifdef MG_ENTERPRISE
if (FLAGS_coordinator) {
throw QueryRuntimeException("Coordinator doesn't have a replication role!");
}
#endif
callback.header = {"replication role"};
callback.fn = [handler = ReplQueryHandler{dbms_handler}] {
auto mode = handler.ShowReplicationRole();
@ -758,7 +938,7 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
return callback;
}
case ReplicationQuery::Action::REGISTER_REPLICA: {
const auto &name = repl_query->replica_name_;
const auto &name = repl_query->instance_name_;
const auto &sync_mode = repl_query->sync_mode_;
auto socket_address = repl_query->socket_address_->Accept(evaluator);
const auto replica_check_frequency = config.replication_replica_check_frequency;
@ -769,20 +949,27 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
return std::vector<std::vector<TypedValue>>();
};
notifications->emplace_back(SeverityLevel::INFO, NotificationCode::REGISTER_REPLICA,
fmt::format("Replica {} is registered.", repl_query->replica_name_));
fmt::format("Replica {} is registered.", repl_query->instance_name_));
return callback;
}
case ReplicationQuery::Action::DROP_REPLICA: {
const auto &name = repl_query->replica_name_;
const auto &name = repl_query->instance_name_;
callback.fn = [handler = ReplQueryHandler{dbms_handler}, name]() mutable {
handler.DropReplica(name);
return std::vector<std::vector<TypedValue>>();
};
notifications->emplace_back(SeverityLevel::INFO, NotificationCode::DROP_REPLICA,
fmt::format("Replica {} is dropped.", repl_query->replica_name_));
fmt::format("Replica {} is dropped.", repl_query->instance_name_));
return callback;
}
case ReplicationQuery::Action::SHOW_REPLICAS: {
#ifdef MG_ENTERPRISE
if (FLAGS_coordinator) {
throw QueryRuntimeException("Coordinator cannot call SHOW REPLICAS! Use SHOW REPLICATION CLUSTER instead.");
}
#endif
callback.header = {
"name", "socket_address", "sync_mode", "current_timestamp_of_replica", "number_of_timestamp_behind_master",
"state"};
@ -834,6 +1021,154 @@ Callback HandleReplicationQuery(ReplicationQuery *repl_query, const Parameters &
}
}
Callback HandleCoordinatorQuery(CoordinatorQuery *coordinator_query, const Parameters &parameters,
dbms::DbmsHandler *dbms_handler, const query::InterpreterConfig &config,
std::vector<Notification> *notifications) {
Callback callback;
switch (coordinator_query->action_) {
case CoordinatorQuery::Action::REGISTER_MAIN_COORDINATOR_SERVER: {
if (!license::global_license_checker.IsEnterpriseValidFast()) {
throw QueryException("Trying to use enterprise feature without a valid license.");
}
#ifdef MG_ENTERPRISE
if constexpr (!coordination::allow_ha) {
throw QueryRuntimeException(
"High availability is experimental feature. Please set MG_EXPERIMENTAL_HIGH_AVAILABILITY compile flag to "
"be able to use this functionality.");
}
if (!FLAGS_coordinator) {
throw QueryRuntimeException("Only coordinator can register coordinator server!");
}
// TODO: MemoryResource for EvaluationContext, it should probably be passed as
// the argument to Callback.
EvaluationContext evaluation_context{.timestamp = QueryTimestamp(), .parameters = parameters};
auto evaluator = PrimitiveLiteralExpressionEvaluator{evaluation_context};
auto coordinator_socket_address_tv = coordinator_query->coordinator_socket_address_->Accept(evaluator);
callback.fn = [handler = CoordQueryHandler{dbms_handler}, coordinator_socket_address_tv,
main_check_frequency = config.replication_replica_check_frequency,
instance_name = coordinator_query->instance_name_]() mutable {
handler.RegisterMainCoordinatorServer(std::string(coordinator_socket_address_tv.ValueString()),
main_check_frequency, instance_name);
return std::vector<std::vector<TypedValue>>();
};
notifications->emplace_back(
SeverityLevel::INFO, NotificationCode::REGISTER_COORDINATOR_SERVER,
fmt::format("Coordinator has registered coordinator server on {} for instance {}.",
coordinator_socket_address_tv.ValueString(), coordinator_query->instance_name_));
return callback;
#endif
}
case CoordinatorQuery::Action::REGISTER_REPLICA_COORDINATOR_SERVER: {
if (!license::global_license_checker.IsEnterpriseValidFast()) {
throw QueryException("Trying to use enterprise feature without a valid license.");
}
#ifdef MG_ENTERPRISE
if constexpr (!coordination::allow_ha) {
throw QueryRuntimeException(
"High availability is experimental feature. Please set MG_EXPERIMENTAL_HIGH_AVAILABILITY compile flag to "
"be able to use this functionality.");
}
if (!FLAGS_coordinator) {
throw QueryRuntimeException("Only coordinator can register coordinator server!");
}
// TODO: MemoryResource for EvaluationContext, it should probably be passed as
// the argument to Callback.
EvaluationContext evaluation_context{.timestamp = QueryTimestamp(), .parameters = parameters};
auto evaluator = PrimitiveLiteralExpressionEvaluator{evaluation_context};
auto coordinator_socket_address_tv = coordinator_query->coordinator_socket_address_->Accept(evaluator);
auto replication_socket_address_tv = coordinator_query->socket_address_->Accept(evaluator);
callback.fn = [handler = CoordQueryHandler{dbms_handler}, coordinator_socket_address_tv,
replication_socket_address_tv, main_check_frequency = config.replication_replica_check_frequency,
instance_name = coordinator_query->instance_name_,
sync_mode = coordinator_query->sync_mode_]() mutable {
handler.RegisterReplicaCoordinatorServer(std::string(replication_socket_address_tv.ValueString()),
std::string(coordinator_socket_address_tv.ValueString()),
main_check_frequency, instance_name, sync_mode);
return std::vector<std::vector<TypedValue>>();
};
notifications->emplace_back(
SeverityLevel::INFO, NotificationCode::REGISTER_COORDINATOR_SERVER,
fmt::format("Coordinator has registered coordinator server on {} for instance {}.",
coordinator_socket_address_tv.ValueString(), coordinator_query->instance_name_));
return callback;
#endif
}
case CoordinatorQuery::Action::SHOW_REPLICATION_CLUSTER: {
if (!license::global_license_checker.IsEnterpriseValidFast()) {
throw QueryException("Trying to use enterprise feature without a valid license.");
}
#ifdef MG_ENTERPRISE
if constexpr (!coordination::allow_ha) {
throw QueryRuntimeException(
"High availability is experimental feature. Please set MG_EXPERIMENTAL_HIGH_AVAILABILITY compile flag to "
"be able to use this functionality.");
}
if (!FLAGS_coordinator) {
throw QueryRuntimeException("Only coordinator can run SHOW REPLICATION CLUSTER.");
}
callback.header = {"name", "socket_address", "alive", "role"};
callback.fn = [handler = CoordQueryHandler{dbms_handler}, replica_nfields = callback.header.size()]() mutable {
const auto main = handler.ShowMainOnCoordinator();
const auto health_check_main = main ? handler.PingMainOnCoordinator() : std::nullopt;
const auto result_status = handler.ShowMainReplicaStatus(
handler.ShowReplicasOnCoordinator(), handler.PingReplicasOnCoordinator(), main, health_check_main);
std::vector<std::vector<TypedValue>> result{};
result.reserve(result_status.size());
std::ranges::transform(result_status, std::back_inserter(result),
[](const auto &status) -> std::vector<TypedValue> {
return {TypedValue{status.name}, TypedValue{status.socket_address},
TypedValue{status.alive}, TypedValue{status.is_main ? "main" : "replica"}};
});
return result;
};
return callback;
#endif
}
case CoordinatorQuery::Action::DO_FAILOVER: {
if (!license::global_license_checker.IsEnterpriseValidFast()) {
throw QueryException("Trying to use enterprise feature without a valid license.");
}
#ifdef MG_ENTERPRISE
if constexpr (!coordination::allow_ha) {
throw QueryRuntimeException(
"High availability is experimental feature. Please set MG_EXPERIMENTAL_HIGH_AVAILABILITY compile flag to "
"be able to use this functionality.");
}
if (!FLAGS_coordinator) {
throw QueryRuntimeException("Only coordinator can run DO FAILOVER!");
}
callback.header = {"name", "socket_address", "alive", "role"};
callback.fn = [handler = CoordQueryHandler{dbms_handler}]() mutable {
handler.DoFailover();
const auto main = handler.ShowMainOnCoordinator();
const auto health_check_main = main ? handler.PingMainOnCoordinator() : std::nullopt;
const auto result_status = handler.ShowMainReplicaStatus(
handler.ShowReplicasOnCoordinator(), handler.PingReplicasOnCoordinator(), main, health_check_main);
std::vector<std::vector<TypedValue>> result{};
result.reserve(result_status.size());
std::ranges::transform(result_status, std::back_inserter(result),
[](const auto &status) -> std::vector<TypedValue> {
return {TypedValue{status.name}, TypedValue{status.socket_address},
TypedValue{status.alive}, TypedValue{status.is_main ? "main" : "replica"}};
});
return result;
};
notifications->emplace_back(SeverityLevel::INFO, NotificationCode::DO_FAILOVER,
"DO FAILOVER called on coordinator.");
return callback;
#endif
}
return callback;
}
}
stream::CommonStreamInfo GetCommonStreamInfo(StreamQuery *stream_query, ExpressionVisitor<TypedValue> &evaluator) {
return {
.batch_interval = GetOptionalValue<std::chrono::milliseconds>(stream_query->batch_interval_, evaluator)
@ -2293,6 +2628,34 @@ PreparedQuery PrepareReplicationQuery(ParsedQuery parsed_query, bool in_explicit
// NOLINTNEXTLINE(clang-analyzer-cplusplus.NewDeleteLeaks)
}
PreparedQuery PrepareCoordinatorQuery(ParsedQuery parsed_query, bool in_explicit_transaction,
std::vector<Notification> *notifications, dbms::DbmsHandler &dbms_handler,
const InterpreterConfig &config) {
if (in_explicit_transaction) {
throw CoordinatorModificationInMulticommandTxException();
}
auto *coordinator_query = utils::Downcast<CoordinatorQuery>(parsed_query.query);
auto callback =
HandleCoordinatorQuery(coordinator_query, parsed_query.parameters, &dbms_handler, config, notifications);
return PreparedQuery{callback.header, std::move(parsed_query.required_privileges),
[callback_fn = std::move(callback.fn), pull_plan = std::shared_ptr<PullPlanVector>{nullptr}](
AnyStream *stream, std::optional<int> n) mutable -> std::optional<QueryHandlerResult> {
if (UNLIKELY(!pull_plan)) {
pull_plan = std::make_shared<PullPlanVector>(callback_fn());
}
if (pull_plan->Pull(stream, n)) [[likely]] {
return QueryHandlerResult::COMMIT;
}
return std::nullopt;
},
RWType::NONE};
// False positive report for the std::make_shared above
// NOLINTNEXTLINE(clang-analyzer-cplusplus.NewDeleteLeaks)
}
PreparedQuery PrepareLockPathQuery(ParsedQuery parsed_query, bool in_explicit_transaction, CurrentDB &current_db) {
if (in_explicit_transaction) {
throw LockPathModificationInMulticommandTxException();
@ -3795,6 +4158,13 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
SetupDatabaseTransaction(could_commit, unique);
}
#ifdef MG_ENTERPRISE
if (FLAGS_coordinator && !utils::Downcast<CoordinatorQuery>(parsed_query.query) &&
!utils::Downcast<SettingQuery>(parsed_query.query)) {
throw QueryRuntimeException("Coordinator can run only coordinator queries!");
}
#endif
utils::Timer planning_timer;
PreparedQuery prepared_query;
utils::MemoryResource *memory_resource =
@ -3837,6 +4207,10 @@ Interpreter::PrepareResult Interpreter::Prepare(const std::string &query_string,
prepared_query =
PrepareReplicationQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->notifications,
*interpreter_context_->dbms_handler, current_db_, interpreter_context_->config);
} else if (utils::Downcast<CoordinatorQuery>(parsed_query.query)) {
prepared_query =
PrepareCoordinatorQuery(std::move(parsed_query), in_explicit_transaction_, &query_execution->notifications,
*interpreter_context_->dbms_handler, interpreter_context_->config);
} else if (utils::Downcast<LockPathQuery>(parsed_query.query)) {
prepared_query = PrepareLockPathQuery(std::move(parsed_query), in_explicit_transaction_, current_db_);
} else if (utils::Downcast<FreeMemoryQuery>(parsed_query.query)) {

View File

@ -15,6 +15,7 @@
#include <gflags/gflags.h>
#include "coordination/coordinator_entity_info.hpp"
#include "dbms/database.hpp"
#include "dbms/dbms_handler.hpp"
#include "memory/query_memory_control.hpp"
@ -68,6 +69,75 @@ inline constexpr size_t kExecutionPoolMaxBlockSize = 1024UL; // 2 ^ 10
enum class QueryHandlerResult { COMMIT, ABORT, NOTHING };
class CoordinatorQueryHandler {
public:
CoordinatorQueryHandler() = default;
virtual ~CoordinatorQueryHandler() = default;
CoordinatorQueryHandler(const CoordinatorQueryHandler &) = default;
CoordinatorQueryHandler &operator=(const CoordinatorQueryHandler &) = default;
CoordinatorQueryHandler(CoordinatorQueryHandler &&) = default;
CoordinatorQueryHandler &operator=(CoordinatorQueryHandler &&) = default;
struct Replica {
std::string name;
std::string socket_address;
ReplicationQuery::SyncMode sync_mode;
std::optional<double> timeout;
uint64_t current_timestamp_of_replica;
uint64_t current_number_of_timestamp_behind_master;
ReplicationQuery::ReplicaState state;
};
#ifdef MG_ENTERPRISE
struct MainReplicaStatus {
std::string_view name;
std::string socket_address;
bool alive;
bool is_main;
MainReplicaStatus(std::string_view name, std::string socket_address, bool alive, bool is_main)
: name{name}, socket_address{std::move(socket_address)}, alive{alive}, is_main{is_main} {}
};
#endif
#ifdef MG_ENTERPRISE
/// @throw QueryRuntimeException if an error ocurred.
virtual void RegisterReplicaCoordinatorServer(const std::string &replication_socket_address,
const std::string &coordinator_socket_address,
const std::chrono::seconds instance_check_frequency,
const std::string &instance_name,
CoordinatorQuery::SyncMode sync_mode) = 0;
virtual void RegisterMainCoordinatorServer(const std::string &socket_address,
const std::chrono::seconds instance_check_frequency,
const std::string &instance_name) = 0;
/// @throw QueryRuntimeException if an error ocurred.
virtual std::vector<coordination::CoordinatorEntityInfo> ShowReplicasOnCoordinator() const = 0;
/// @throw QueryRuntimeException if an error ocurred.
virtual std::optional<coordination::CoordinatorEntityInfo> ShowMainOnCoordinator() const = 0;
/// @throw QueryRuntimeException if an error ocurred.
virtual std::unordered_map<std::string_view, bool> PingReplicasOnCoordinator() const = 0;
/// @throw QueryRuntimeException if an error ocurred.
virtual std::optional<coordination::CoordinatorEntityHealthInfo> PingMainOnCoordinator() const = 0;
/// @throw QueryRuntimeException if an error ocurred.
virtual void DoFailover() const = 0;
/// @throw QueryRuntimeException if an error ocurred.
virtual std::vector<MainReplicaStatus> ShowMainReplicaStatus(
const std::vector<coordination::CoordinatorEntityInfo> &replicas,
const std::unordered_map<std::string_view, bool> &health_check_replicas,
const std::optional<coordination::CoordinatorEntityInfo> &main,
const std::optional<coordination::CoordinatorEntityHealthInfo> &health_check_main) const = 0;
#endif
};
class AnalyzeGraphQueryHandler {
public:
AnalyzeGraphQueryHandler() = default;

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -66,6 +66,12 @@ constexpr std::string_view GetCodeString(const NotificationCode code) {
return "PlanHinting"sv;
case NotificationCode::REGISTER_REPLICA:
return "RegisterReplica"sv;
#ifdef MG_ENTERPRISE
case NotificationCode::REGISTER_COORDINATOR_SERVER:
return "RegisterCoordinatorServer"sv;
case NotificationCode::DO_FAILOVER:
return "DoFailover"sv;
#endif
case NotificationCode::REPLICA_PORT_WARNING:
return "ReplicaPortWarning"sv;
case NotificationCode::SET_REPLICA:

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -42,6 +42,10 @@ enum class NotificationCode : uint8_t {
PLAN_HINTING,
REPLICA_PORT_WARNING,
REGISTER_REPLICA,
#ifdef MG_ENTERPRISE
REGISTER_COORDINATOR_SERVER,
DO_FAILOVER,
#endif
SET_REPLICA,
START_STREAM,
START_ALL_STREAMS,

View File

@ -5,10 +5,9 @@ target_sources(mg-replication
include/replication/state.hpp
include/replication/epoch.hpp
include/replication/config.hpp
include/replication/mode.hpp
include/replication/messages.hpp
include/replication/role.hpp
include/replication/status.hpp
include/replication/messages.hpp
include/replication/replication_client.hpp
include/replication/replication_server.hpp
@ -25,6 +24,6 @@ target_include_directories(mg-replication PUBLIC include)
find_package(fmt REQUIRED)
target_link_libraries(mg-replication
PUBLIC mg::utils mg::kvstore lib::json mg::rpc mg::slk
PUBLIC mg::utils mg::kvstore lib::json mg::rpc mg::slk mg::io mg::repl_coord_glue
PRIVATE fmt::fmt
)

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -15,7 +15,7 @@
#include <cstdint>
#include <optional>
#include <string>
#include "replication/mode.hpp"
#include "replication_coordination_glue/mode.hpp"
namespace memgraph::replication {
@ -24,7 +24,7 @@ inline constexpr auto *kDefaultReplicationServerIp = "0.0.0.0";
struct ReplicationClientConfig {
std::string name;
ReplicationMode mode{};
replication_coordination_glue::ReplicationMode mode{};
std::string ip_address;
uint16_t port{};

View File

@ -15,27 +15,6 @@
#include "slk/serialization.hpp"
namespace memgraph::replication {
struct FrequentHeartbeatReq {
static const utils::TypeInfo kType; // TODO: make constexpr?
static const utils::TypeInfo &GetTypeInfo() { return kType; } // WHAT?
static void Load(FrequentHeartbeatReq *self, memgraph::slk::Reader *reader);
static void Save(const FrequentHeartbeatReq &self, memgraph::slk::Builder *builder);
FrequentHeartbeatReq() = default;
};
struct FrequentHeartbeatRes {
static const utils::TypeInfo kType;
static const utils::TypeInfo &GetTypeInfo() { return kType; }
static void Load(FrequentHeartbeatRes *self, memgraph::slk::Reader *reader);
static void Save(const FrequentHeartbeatRes &self, memgraph::slk::Builder *builder);
FrequentHeartbeatRes() = default;
};
using FrequentHeartbeatRpc = rpc::RequestResponse<FrequentHeartbeatReq, FrequentHeartbeatRes>;
struct SystemHeartbeatReq {
static const utils::TypeInfo kType;
static const utils::TypeInfo &GetTypeInfo() { return kType; }
@ -58,16 +37,9 @@ struct SystemHeartbeatRes {
};
using SystemHeartbeatRpc = rpc::RequestResponse<SystemHeartbeatReq, SystemHeartbeatRes>;
void FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder);
} // namespace memgraph::replication
namespace memgraph::slk {
void Save(const memgraph::replication::FrequentHeartbeatRes &self, memgraph::slk::Builder *builder);
void Load(memgraph::replication::FrequentHeartbeatRes *self, memgraph::slk::Reader *reader);
void Save(const memgraph::replication::FrequentHeartbeatReq & /*self*/, memgraph::slk::Builder * /*builder*/);
void Load(memgraph::replication::FrequentHeartbeatReq * /*self*/, memgraph::slk::Reader * /*reader*/);
void Save(const memgraph::replication::SystemHeartbeatRes &self, memgraph::slk::Builder *builder);
void Load(memgraph::replication::SystemHeartbeatRes *self, memgraph::slk::Reader *reader);
void Save(const memgraph::replication::SystemHeartbeatReq & /*self*/, memgraph::slk::Builder * /*builder*/);

View File

@ -12,7 +12,7 @@
#pragma once
#include "replication/config.hpp"
#include "replication/messages.hpp"
#include "replication_coordination_glue/messages.hpp"
#include "rpc/client.hpp"
#include "utils/scheduler.hpp"
#include "utils/synchronized.hpp"
@ -45,7 +45,7 @@ struct ReplicationClient {
[this, cb = std::forward<F>(callback), reconnect = false]() mutable {
try {
{
auto stream{rpc_client_.Stream<memgraph::replication::FrequentHeartbeatRpc>()};
auto stream{rpc_client_.Stream<memgraph::replication_coordination_glue::FrequentHeartbeatRpc>()};
stream.AwaitResponse();
}
cb(reconnect, *this);
@ -73,7 +73,7 @@ struct ReplicationClient {
};
utils::Synchronized<State> state_{State::BEHIND};
memgraph::replication::ReplicationMode mode_{memgraph::replication::ReplicationMode::SYNC};
replication_coordination_glue::ReplicationMode mode_{replication_coordination_glue::ReplicationMode::SYNC};
// This thread pool is used for background tasks so we don't
// block the main storage thread
// We use only 1 thread for 2 reasons:

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -14,9 +14,9 @@
#include "kvstore/kvstore.hpp"
#include "replication/config.hpp"
#include "replication/epoch.hpp"
#include "replication/mode.hpp"
#include "replication/replication_client.hpp"
#include "replication/role.hpp"
#include "replication_coordination_glue/mode.hpp"
#include "replication_server.hpp"
#include "status.hpp"
#include "utils/result.hpp"

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -42,7 +42,7 @@ struct MainRole {
// fragment of key: "__replication_role"
struct ReplicaRole {
ReplicationServerConfig config;
ReplicationServerConfig config{};
friend bool operator==(ReplicaRole const &, ReplicaRole const &) = default;
};

View File

@ -8,25 +8,32 @@
// 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 "replication/messages.hpp"
#include "rpc/messages.hpp"
#include "slk/serialization.hpp"
#include "slk/streams.hpp"
namespace memgraph::replication {
constexpr utils::TypeInfo SystemHeartbeatReq::kType{utils::TypeId::REP_SYSTEM_HEARTBEAT_REQ, "SystemHeartbeatReq",
nullptr};
constexpr utils::TypeInfo SystemHeartbeatRes::kType{utils::TypeId::REP_SYSTEM_HEARTBEAT_RES, "SystemHeartbeatRes",
nullptr};
void SystemHeartbeatReq::Save(const SystemHeartbeatReq &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void SystemHeartbeatReq::Load(SystemHeartbeatReq *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void SystemHeartbeatRes::Save(const SystemHeartbeatRes &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void SystemHeartbeatRes::Load(SystemHeartbeatRes *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
} // namespace memgraph::replication
namespace memgraph::slk {
// Serialize code for FrequentHeartbeatRes
void Save(const memgraph::replication::FrequentHeartbeatRes &self, memgraph::slk::Builder *builder) {}
void Load(memgraph::replication::FrequentHeartbeatRes *self, memgraph::slk::Reader *reader) {}
// Serialize code for FrequentHeartbeatReq
void Save(const memgraph::replication::FrequentHeartbeatReq & /*self*/, memgraph::slk::Builder * /*builder*/) {
/* Nothing to serialize */
}
void Load(memgraph::replication::FrequentHeartbeatReq * /*self*/, memgraph::slk::Reader * /*reader*/) {
/* Nothing to serialize */
}
// Serialize code for SystemHeartbeatRes
void Save(const memgraph::replication::SystemHeartbeatRes &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self.system_timestamp, builder);
@ -42,55 +49,4 @@ void Save(const memgraph::replication::SystemHeartbeatReq & /*self*/, memgraph::
void Load(memgraph::replication::SystemHeartbeatReq * /*self*/, memgraph::slk::Reader * /*reader*/) {
/* Nothing to serialize */
}
} // namespace memgraph::slk
namespace memgraph::replication {
constexpr utils::TypeInfo FrequentHeartbeatReq::kType{utils::TypeId::REP_FREQUENT_HEARTBEAT_REQ, "FrequentHeartbeatReq",
nullptr};
constexpr utils::TypeInfo FrequentHeartbeatRes::kType{utils::TypeId::REP_FREQUENT_HEARTBEAT_RES, "FrequentHeartbeatRes",
nullptr};
constexpr utils::TypeInfo SystemHeartbeatReq::kType{utils::TypeId::REP_SYSTEM_HEARTBEAT_REQ, "SystemHeartbeatReq",
nullptr};
constexpr utils::TypeInfo SystemHeartbeatRes::kType{utils::TypeId::REP_SYSTEM_HEARTBEAT_RES, "SystemHeartbeatRes",
nullptr};
void FrequentHeartbeatReq::Save(const FrequentHeartbeatReq &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void FrequentHeartbeatReq::Load(FrequentHeartbeatReq *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void FrequentHeartbeatRes::Save(const FrequentHeartbeatRes &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void FrequentHeartbeatRes::Load(FrequentHeartbeatRes *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void SystemHeartbeatReq::Save(const SystemHeartbeatReq &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void SystemHeartbeatReq::Load(SystemHeartbeatReq *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void SystemHeartbeatRes::Save(const SystemHeartbeatRes &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void SystemHeartbeatRes::Load(SystemHeartbeatRes *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
FrequentHeartbeatReq req;
FrequentHeartbeatReq::Load(&req, req_reader);
memgraph::slk::Load(&req, req_reader);
FrequentHeartbeatRes res{};
memgraph::slk::Save(res, res_builder);
}
} // namespace memgraph::replication

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -28,8 +28,8 @@ ReplicationClient::ReplicationClient(const memgraph::replication::ReplicationCli
mode_{config.mode} {}
ReplicationClient::~ReplicationClient() {
auto endpoint = rpc_client_.Endpoint();
try {
auto const &endpoint = rpc_client_.Endpoint();
spdlog::trace("Closing replication client on {}:{}", endpoint.address, endpoint.port);
} catch (...) {
// Logging can throw. Not a big deal, just ignore.

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -10,7 +10,7 @@
// licenses/APL.txt.
#include "replication/replication_server.hpp"
#include "replication/messages.hpp"
#include "replication_coordination_glue/messages.hpp"
namespace memgraph::replication {
namespace {
@ -32,9 +32,9 @@ ReplicationServer::ReplicationServer(const memgraph::replication::ReplicationSer
: rpc_server_context_{CreateServerContext(config)},
rpc_server_{io::network::Endpoint{config.ip_address, config.port}, &rpc_server_context_,
kReplicationServerThreads} {
rpc_server_.Register<FrequentHeartbeatRpc>([](auto *req_reader, auto *res_builder) {
rpc_server_.Register<replication_coordination_glue::FrequentHeartbeatRpc>([](auto *req_reader, auto *res_builder) {
spdlog::debug("Received FrequentHeartbeatRpc");
FrequentHeartbeatHandler(req_reader, res_builder);
replication_coordination_glue::FrequentHeartbeatHandler(req_reader, res_builder);
});
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -219,10 +219,12 @@ bool ReplicationState::TryPersistRegisteredReplica(const ReplicationClientConfig
bool ReplicationState::SetReplicationRoleMain() {
auto new_epoch = utils::GenerateUUID();
if (!TryPersistRoleMain(new_epoch)) {
return false;
}
replication_data_ = RoleMainData{ReplicationEpoch{new_epoch}};
return true;
}
@ -237,6 +239,7 @@ bool ReplicationState::SetReplicationRoleReplica(const ReplicationServerConfig &
utils::BasicResult<RegisterReplicaError, ReplicationClient *> ReplicationState::RegisterReplica(
const ReplicationClientConfig &config) {
auto const replica_handler = [](RoleReplicaData const &) { return RegisterReplicaError::NOT_MAIN; };
ReplicationClient *client{nullptr};
auto const main_handler = [&client, &config, this](RoleMainData &mainData) -> RegisterReplicaError {
// name check

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -95,7 +95,7 @@ void from_json(const nlohmann::json &j, ReplicationReplicaEntry &p) {
auto seconds = j.at(kCheckFrequency).get<std::chrono::seconds::rep>();
auto config = ReplicationClientConfig{
.name = j.at(kReplicaName).get<std::string>(),
.mode = j.at(kSyncMode).get<ReplicationMode>(),
.mode = j.at(kSyncMode).get<replication_coordination_glue::ReplicationMode>(),
.ip_address = j.at(kIpAddress).get<std::string>(),
.port = j.at(kPort).get<uint16_t>(),
.replica_check_frequency = std::chrono::seconds{seconds},

View File

@ -0,0 +1,13 @@
add_library(mg-repl_coord_glue STATIC )
add_library(mg::repl_coord_glue ALIAS mg-repl_coord_glue)
target_sources(mg-repl_coord_glue
PUBLIC
messages.hpp
mode.hpp
PRIVATE
messages.cpp
)
target_link_libraries(mg-repl_coord_glue mg-rpc mg-slk)

View File

@ -0,0 +1,63 @@
// Copyright 2024 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 "replication_coordination_glue/messages.hpp"
#include "rpc/messages.hpp"
#include "slk/serialization.hpp"
#include "slk/streams.hpp"
namespace memgraph::slk {
// Serialize code for FrequentHeartbeatRes
void Save(const memgraph::replication_coordination_glue::FrequentHeartbeatRes &self, memgraph::slk::Builder *builder) {}
void Load(memgraph::replication_coordination_glue::FrequentHeartbeatRes *self, memgraph::slk::Reader *reader) {}
// Serialize code for FrequentHeartbeatReq
void Save(const memgraph::replication_coordination_glue::FrequentHeartbeatReq & /*self*/,
memgraph::slk::Builder * /*builder*/) {
/* Nothing to serialize */
}
void Load(memgraph::replication_coordination_glue::FrequentHeartbeatReq * /*self*/,
memgraph::slk::Reader * /*reader*/) {
/* Nothing to serialize */
}
} // namespace memgraph::slk
namespace memgraph::replication_coordination_glue {
constexpr utils::TypeInfo FrequentHeartbeatReq::kType{utils::TypeId::REP_FREQUENT_HEARTBEAT_REQ, "FrequentHeartbeatReq",
nullptr};
constexpr utils::TypeInfo FrequentHeartbeatRes::kType{utils::TypeId::REP_FREQUENT_HEARTBEAT_RES, "FrequentHeartbeatRes",
nullptr};
void FrequentHeartbeatReq::Save(const FrequentHeartbeatReq &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void FrequentHeartbeatReq::Load(FrequentHeartbeatReq *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void FrequentHeartbeatRes::Save(const FrequentHeartbeatRes &self, memgraph::slk::Builder *builder) {
memgraph::slk::Save(self, builder);
}
void FrequentHeartbeatRes::Load(FrequentHeartbeatRes *self, memgraph::slk::Reader *reader) {
memgraph::slk::Load(self, reader);
}
void FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder) {
FrequentHeartbeatReq req;
FrequentHeartbeatReq::Load(&req, req_reader);
memgraph::slk::Load(&req, req_reader);
FrequentHeartbeatRes res{};
memgraph::slk::Save(res, res_builder);
}
} // namespace memgraph::replication_coordination_glue

View File

@ -0,0 +1,49 @@
// Copyright 2024 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 "rpc/messages.hpp"
#include "slk/serialization.hpp"
namespace memgraph::replication_coordination_glue {
struct FrequentHeartbeatReq {
static const utils::TypeInfo kType; // TODO: make constexpr?
static const utils::TypeInfo &GetTypeInfo() { return kType; }
static void Load(FrequentHeartbeatReq *self, memgraph::slk::Reader *reader);
static void Save(const FrequentHeartbeatReq &self, memgraph::slk::Builder *builder);
FrequentHeartbeatReq() = default;
};
struct FrequentHeartbeatRes {
static const utils::TypeInfo kType;
static const utils::TypeInfo &GetTypeInfo() { return kType; }
static void Load(FrequentHeartbeatRes *self, memgraph::slk::Reader *reader);
static void Save(const FrequentHeartbeatRes &self, memgraph::slk::Builder *builder);
FrequentHeartbeatRes() = default;
};
using FrequentHeartbeatRpc = rpc::RequestResponse<FrequentHeartbeatReq, FrequentHeartbeatRes>;
void FrequentHeartbeatHandler(slk::Reader *req_reader, slk::Builder *res_builder);
} // namespace memgraph::replication_coordination_glue
namespace memgraph::slk {
void Save(const memgraph::replication_coordination_glue::FrequentHeartbeatRes &self, memgraph::slk::Builder *builder);
void Load(memgraph::replication_coordination_glue::FrequentHeartbeatRes *self, memgraph::slk::Reader *reader);
void Save(const memgraph::replication_coordination_glue::FrequentHeartbeatReq & /*self*/,
memgraph::slk::Builder * /*builder*/);
void Load(memgraph::replication_coordination_glue::FrequentHeartbeatReq * /*self*/, memgraph::slk::Reader * /*reader*/);
} // namespace memgraph::slk

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -10,7 +10,9 @@
// licenses/APL.txt.
#pragma once
#include <cstdint>
namespace memgraph::replication {
namespace memgraph::replication_coordination_glue {
enum class ReplicationMode : std::uint8_t { SYNC, ASYNC };
}
} // namespace memgraph::replication_coordination_glue

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -55,6 +55,12 @@ class SlkDecodeException : public utils::BasicException {
// here because C++ doesn't know how to resolve the function call if it isn't in
// the global namespace.
template <typename T>
inline void Save(const std::vector<T> &obj, Builder *builder,
std::function<void(const T &, Builder *)> item_save_function);
template <typename T>
inline void Load(std::vector<T> *obj, Reader *reader, std::function<void(T *, Reader *)> item_load_function);
template <typename T>
void Save(const std::vector<T> &obj, Builder *builder);
template <typename T>
@ -509,4 +515,17 @@ inline void Load(utils::TypeId *obj, Reader *reader) {
*obj = utils::TypeId(utils::MemcpyCast<enum_type>(obj_encoded));
}
template <utils::Enum T>
void Save(const T &enum_value, slk::Builder *builder) {
slk::Save(utils::UnderlyingCast(enum_value), builder);
}
template <utils::Enum T>
void Load(T *enum_value, slk::Reader *reader) {
using UnderlyingType = std::underlying_type_t<T>;
UnderlyingType value;
slk::Load(&value, reader);
*enum_value = static_cast<T>(value);
}
} // namespace memgraph::slk

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -1050,14 +1050,11 @@ bool PropertyStore::HasProperty(PropertyId property) const {
return ExistsSpecificProperty(&reader, property) == ExpectedPropertyStatus::EQUAL;
}
/// TODO: andi write a unit test for it
bool PropertyStore::HasAllProperties(const std::set<PropertyId> &properties) const {
return std::all_of(properties.begin(), properties.end(), [this](const auto &prop) { return HasProperty(prop); });
}
/// TODO: andi write a unit test for it
bool PropertyStore::HasAllPropertyValues(const std::vector<PropertyValue> &property_values) const {
/// TODO: andi extract this into a private method
auto property_map = Properties();
std::vector<PropertyValue> all_property_values;
transform(property_map.begin(), property_map.end(), back_inserter(all_property_values),

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -55,7 +55,6 @@ class PropertyStore {
/// Checks whether all property values in the vector `property_values` exist in the store. The time
/// complexity of this function is O(n^2).
/// TODO: andi Not so sure it is quadratic complexity
bool HasAllPropertyValues(const std::vector<PropertyValue> &property_values) const;
/// Extracts property values for all property ids in the set `properties`. The time

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -31,7 +31,7 @@ struct TimestampInfo {
struct ReplicaInfo {
std::string name;
memgraph::replication::ReplicationMode mode;
replication_coordination_glue::ReplicationMode mode;
io::network::Endpoint endpoint;
replication::ReplicaState state;
TimestampInfo timestamp_info;

View File

@ -226,7 +226,7 @@ bool ReplicationStorageClient::FinalizeTransactionReplication(Storage *storage,
}
};
if (client_.mode_ == memgraph::replication::ReplicationMode::ASYNC) {
if (client_.mode_ == replication_coordination_glue::ReplicationMode::ASYNC) {
client_.thread_pool_.AddTask([task = std::move(task)]() mutable { (void)task(); });
return true;
}

View File

@ -13,8 +13,8 @@
#include "replication/config.hpp"
#include "replication/epoch.hpp"
#include "replication/messages.hpp"
#include "replication/replication_client.hpp"
#include "replication_coordination_glue/messages.hpp"
#include "rpc/client.hpp"
#include "storage/v2/database_access.hpp"
#include "storage/v2/durability/storage_global_operation.hpp"
@ -94,7 +94,7 @@ class ReplicationStorageClient {
~ReplicationStorageClient() = default;
// TODO Remove the client related functions
auto Mode() const -> memgraph::replication::ReplicationMode { return client_.mode_; }
auto Mode() const -> memgraph::replication_coordination_glue::ReplicationMode { return client_.mode_; }
auto Name() const -> std::string const & { return client_.name_; }
auto Endpoint() const -> io::network::Endpoint const & { return client_.rpc_client_.Endpoint(); }

View File

@ -64,7 +64,7 @@ bool ReplicationStorageState::FinalizeTransaction(uint64_t timestamp, Storage *s
client->IfStreamingTransaction([&](auto &stream) { stream.AppendTransactionEnd(timestamp); });
const auto finalized = client->FinalizeTransactionReplication(storage, std::move(db_acc));
if (client->Mode() == memgraph::replication::ReplicationMode::SYNC) {
if (client->Mode() == replication_coordination_glue::ReplicationMode::SYNC) {
finalized_on_all_replicas = finalized && finalized_on_all_replicas;
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2024 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,17 +25,4 @@ void Load(storage::Gid *gid, slk::Reader *reader);
void Save(const storage::PropertyValue &value, slk::Builder *builder);
void Load(storage::PropertyValue *value, slk::Reader *reader);
template <utils::Enum T>
void Save(const T &enum_value, slk::Builder *builder) {
slk::Save(utils::UnderlyingCast(enum_value), builder);
}
template <utils::Enum T>
void Load(T *enum_value, slk::Reader *reader) {
using UnderlyingType = std::underlying_type_t<T>;
UnderlyingType value;
slk::Load(&value, reader);
*enum_value = static_cast<T>(value);
}
} // namespace memgraph::slk

View File

@ -94,6 +94,10 @@ enum class TypeId : uint64_t {
REP_SYSTEM_RECOVERY_REQ,
REP_SYSTEM_RECOVERY_RES,
// Coordinator
COORD_FAILOVER_REQ,
COORD_FAILOVER_RES,
// AST
AST_LABELIX = 3000,
AST_PROPERTYIX,
@ -201,6 +205,7 @@ enum class TypeId : uint64_t {
AST_SHOW_DATABASES,
AST_EDGE_IMPORT_MODE_QUERY,
AST_PATTERN_COMPREHENSION,
AST_COORDINATOR_QUERY,
// Symbol
SYMBOL = 4000,

View File

@ -77,8 +77,13 @@ add_subdirectory(query_modules_storage_modes)
add_subdirectory(garbage_collection)
add_subdirectory(query_planning)
if (MG_EXPERIMENTAL_HIGH_AVAILABILITY)
add_subdirectory(high_availability_experimental)
endif ()
if (MG_EXPERIMENTAL_REPLICATION_MULTITENANCY)
add_subdirectory(replication_experimental)
add_subdirectory(replication_experimental)
endif ()
copy_e2e_python_files(pytest_runner pytest_runner.sh "")

View File

@ -66,6 +66,8 @@ startup_config_dict = {
"Time in seconds after which inactive Bolt sessions will be closed.",
),
"cartesian_product_enabled": ("true", "true", "Enable cartesian product expansion."),
"coordinator": ("false", "false", "Controls whether the instance is a replication coordinator."),
"coordinator_server_port": ("0", "0", "Port on which coordinator servers will be started."),
"data_directory": ("mg_data", "mg_data", "Path to directory in which to save all permanent data."),
"data_recovery_on_startup": (
"false",

View File

@ -11,7 +11,6 @@
import sys
import default_config
import mgclient
import pytest

View File

@ -0,0 +1,12 @@
find_package(gflags REQUIRED)
copy_e2e_python_files(ha_experimental coordinator.py)
copy_e2e_python_files(ha_experimental client_initiated_failover.py)
copy_e2e_python_files(ha_experimental uninitialized_cluster.py)
copy_e2e_python_files(ha_experimental common.py)
copy_e2e_python_files(ha_experimental conftest.py)
copy_e2e_python_files(ha_experimental workloads.yaml)
copy_e2e_python_files_from_parent_folder(ha_experimental ".." memgraph.py)
copy_e2e_python_files_from_parent_folder(ha_experimental ".." interactive_mg_runner.py)
copy_e2e_python_files_from_parent_folder(ha_experimental ".." mg_utils.py)

View File

@ -0,0 +1,224 @@
# Copyright 2022 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.
import os
import sys
import interactive_mg_runner
import pytest
from common import execute_and_fetch_all
from mg_utils import mg_sleep_and_assert
interactive_mg_runner.SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
interactive_mg_runner.PROJECT_DIR = os.path.normpath(
os.path.join(interactive_mg_runner.SCRIPT_DIR, "..", "..", "..", "..")
)
interactive_mg_runner.BUILD_DIR = os.path.normpath(os.path.join(interactive_mg_runner.PROJECT_DIR, "build"))
interactive_mg_runner.MEMGRAPH_BINARY = os.path.normpath(os.path.join(interactive_mg_runner.BUILD_DIR, "memgraph"))
MEMGRAPH_INSTANCES_DESCRIPTION = {
"instance_1": {
"args": ["--bolt-port", "7688", "--log-level", "TRACE", "--coordinator-server-port", "10011"],
"log_file": "replica1.log",
"setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10001;"],
},
"instance_2": {
"args": ["--bolt-port", "7689", "--log-level", "TRACE", "--coordinator-server-port", "10012"],
"log_file": "replica2.log",
"setup_queries": ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"],
},
"instance_3": {
"args": ["--bolt-port", "7687", "--log-level", "TRACE", "--coordinator-server-port", "10013"],
"log_file": "main.log",
"setup_queries": [
"REGISTER REPLICA instance_1 SYNC TO '127.0.0.1:10001'",
"REGISTER REPLICA instance_2 SYNC TO '127.0.0.1:10002'",
],
},
"coordinator": {
"args": ["--bolt-port", "7690", "--log-level=TRACE", "--coordinator"],
"log_file": "replica3.log",
"setup_queries": [
"REGISTER REPLICA instance_1 SYNC TO '127.0.0.1:10001' WITH COORDINATOR SERVER ON '127.0.0.1:10011';",
"REGISTER REPLICA instance_2 SYNC TO '127.0.0.1:10002' WITH COORDINATOR SERVER ON '127.0.0.1:10012';",
"REGISTER MAIN instance_3 WITH COORDINATOR SERVER ON '127.0.0.1:10013';",
],
},
}
def test_show_replication_cluster(connection):
# Goal of this test is to check the SHOW REPLICATION CLUSTER command.
# 1. We start all replicas, main and coordinator manually: we want to be able to kill them ourselves without relying on external tooling to kill processes.
# 2. We check that all replicas and main have the correct state: they should all be alive.
# 3. We kill one replica. It should not appear anymore in the SHOW REPLICATION CLUSTER command.
# 4. We kill main. It should not appear anymore in the SHOW REPLICATION CLUSTER command.
# 1.
interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION)
cursor = connection(7690, "coordinator").cursor()
# 2.
# We leave some time for the coordinator to realise the replicas are down.
def retrieve_data():
return set(execute_and_fetch_all(cursor, "SHOW REPLICATION CLUSTER;"))
expected_data = {
("instance_1", "127.0.0.1:10011", True, "replica"),
("instance_2", "127.0.0.1:10012", True, "replica"),
("instance_3", "127.0.0.1:10013", True, "main"),
}
mg_sleep_and_assert(expected_data, retrieve_data)
# 3.
interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_1")
expected_data = {
("instance_2", "127.0.0.1:10012", True, "replica"),
("instance_3", "127.0.0.1:10013", True, "main"),
("instance_1", "127.0.0.1:10011", False, "replica"),
}
mg_sleep_and_assert(expected_data, retrieve_data)
# 4.
interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3")
expected_data = {
("instance_2", "127.0.0.1:10012", True, "replica"),
("instance_1", "127.0.0.1:10011", False, "replica"),
("instance_3", "127.0.0.1:10013", False, "main"),
}
mg_sleep_and_assert(expected_data, retrieve_data)
def test_simple_client_initiated_failover(connection):
# 1. Start all instances
# 2. Kill main
# 3. Run DO FAILOVER on COORDINATOR
# 4. Assert new config on coordinator by running show replication cluster
# 5. Assert replicas on new main
# 1.
interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION)
# 2.
main_cursor = connection(7687, "instance_3").cursor()
expected_data_on_main = {
("instance_1", "127.0.0.1:10001", "sync", 0, 0, "ready"),
("instance_2", "127.0.0.1:10002", "sync", 0, 0, "ready"),
}
actual_data_on_main = set(execute_and_fetch_all(main_cursor, "SHOW REPLICAS;"))
assert actual_data_on_main == expected_data_on_main
interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3")
coord_cursor = connection(7690, "coordinator").cursor()
def retrieve_data_show_repl_cluster():
return set(execute_and_fetch_all(coord_cursor, "SHOW REPLICATION CLUSTER;"))
expected_data_on_coord = {
("instance_1", "127.0.0.1:10011", True, "replica"),
("instance_2", "127.0.0.1:10012", True, "replica"),
("instance_3", "127.0.0.1:10013", False, "main"),
}
mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster)
# 3.
execute_and_fetch_all(coord_cursor, "DO FAILOVER")
expected_data_on_coord = {
("instance_1", "127.0.0.1:10011", True, "main"),
("instance_2", "127.0.0.1:10012", True, "replica"),
}
mg_sleep_and_assert(expected_data_on_coord, retrieve_data_show_repl_cluster)
# 4.
new_main_cursor = connection(7688, "instance_1").cursor()
def retrieve_data_show_replicas():
return set(execute_and_fetch_all(new_main_cursor, "SHOW REPLICAS;"))
expected_data_on_new_main = {
("instance_2", "127.0.0.1:10002", "sync", 0, 0, "ready"),
}
mg_sleep_and_assert(expected_data_on_new_main, retrieve_data_show_replicas)
def test_failover_fails_all_replicas_down(connection):
# 1. Start all instances
# 2. Kill all replicas
# 3. Kill main
# 4. Run DO FAILOVER on COORDINATOR. Assert exception is being thrown due to all replicas being down
# 5. Assert cluster status didn't change
# 1.
interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION)
# 2.
interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_1")
interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_2")
# 3.
interactive_mg_runner.kill(MEMGRAPH_INSTANCES_DESCRIPTION, "instance_3")
coord_cursor = connection(7690, "coordinator").cursor()
# 4.
with pytest.raises(Exception) as e:
execute_and_fetch_all(coord_cursor, "DO FAILOVER;")
assert str(e.value) == "Failover aborted since all replicas are down!"
# 5.
def retrieve_data():
return set(execute_and_fetch_all(coord_cursor, "SHOW REPLICATION CLUSTER;"))
expected_data_on_coord = {
("instance_1", "127.0.0.1:10011", False, "replica"),
("instance_2", "127.0.0.1:10012", False, "replica"),
("instance_3", "127.0.0.1:10013", False, "main"),
}
mg_sleep_and_assert(expected_data_on_coord, retrieve_data)
def test_failover_fails_main_is_alive(connection):
# 1. Start all instances
# 2. Run DO FAILOVER on COORDINATOR. Assert exception is being thrown due to main is still live.
# 3. Assert cluster status didn't change
# 1.
interactive_mg_runner.start_all(MEMGRAPH_INSTANCES_DESCRIPTION)
# 2.
coord_cursor = connection(7690, "coordinator").cursor()
def retrieve_data():
return set(execute_and_fetch_all(coord_cursor, "SHOW REPLICATION CLUSTER;"))
expected_data_on_coord = {
("instance_1", "127.0.0.1:10011", True, "replica"),
("instance_2", "127.0.0.1:10012", True, "replica"),
("instance_3", "127.0.0.1:10013", True, "main"),
}
mg_sleep_and_assert(expected_data_on_coord, retrieve_data)
# 4.
with pytest.raises(Exception) as e:
execute_and_fetch_all(coord_cursor, "DO FAILOVER;")
assert str(e.value) == "Failover aborted since main is alive!"
# 5.
mg_sleep_and_assert(expected_data_on_coord, retrieve_data)
if __name__ == "__main__":
sys.exit(pytest.main([__file__, "-rA"]))

View File

@ -0,0 +1,25 @@
# Copyright 2022 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.
import typing
import mgclient
def execute_and_fetch_all(cursor: mgclient.Cursor, query: str, params: dict = {}) -> typing.List[tuple]:
cursor.execute(query, params)
return cursor.fetchall()
def connect(**kwargs) -> mgclient.Connection:
connection = mgclient.connect(**kwargs)
connection.autocommit = True
return connection

View File

@ -0,0 +1,43 @@
# Copyright 2022 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.
import pytest
from common import connect, execute_and_fetch_all
# The fixture here is more complex because the connection has to be
# parameterized based on the test parameters (info has to be available on both
# sides).
#
# https://docs.pytest.org/en/latest/example/parametrize.html#indirect-parametrization
# is not an elegant/feasible solution here.
#
# The solution was independently developed and then I stumbled upon the same
# approach here https://stackoverflow.com/a/68286553/4888809 which I think is
# optimal.
@pytest.fixture(scope="function")
def connection():
connection_holder = None
role_holder = None
def inner_connection(port, role):
nonlocal connection_holder, role_holder
connection_holder = connect(host="localhost", port=port)
role_holder = role
return connection_holder
yield inner_connection
# Only main instance can be cleaned up because replicas do NOT accept
# writes.
if role_holder == "main":
cursor = connection_holder.cursor()
execute_and_fetch_all(cursor, "MATCH (n) DETACH DELETE n;")

View File

@ -0,0 +1,98 @@
# Copyright 2022 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.
import sys
import pytest
from common import execute_and_fetch_all
from mg_utils import mg_sleep_and_assert
def test_disable_cypher_queries(connection):
cursor = connection(7690, "coordinator").cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(cursor, "CREATE (n:TestNode {prop: 'test'})")
assert str(e.value) == "Coordinator can run only coordinator queries!"
def test_coordinator_cannot_be_replica_role(connection):
cursor = connection(7690, "coordinator").cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(cursor, "SET REPLICATION ROLE TO REPLICA WITH PORT 10001;")
assert str(e.value) == "Coordinator can run only coordinator queries!"
def test_coordinator_cannot_run_show_repl_role(connection):
cursor = connection(7690, "coordinator").cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(cursor, "SHOW REPLICATION ROLE;")
assert str(e.value) == "Coordinator can run only coordinator queries!"
def test_coordinator_show_replication_cluster(connection):
cursor = connection(7690, "coordinator").cursor()
def retrieve_data():
return set(execute_and_fetch_all(cursor, "SHOW REPLICATION CLUSTER;"))
expected_data = {
("main", "127.0.0.1:10013", True, "main"),
("replica_1", "127.0.0.1:10011", True, "replica"),
("replica_2", "127.0.0.1:10012", True, "replica"),
}
mg_sleep_and_assert(expected_data, retrieve_data)
def test_coordinator_cannot_call_show_replicas(connection):
cursor = connection(7690, "coordinator").cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(cursor, "SHOW REPLICAS;")
assert str(e.value) == "Coordinator can run only coordinator queries!"
@pytest.mark.parametrize(
"port, role",
[(7687, "main"), (7688, "replica"), (7689, "replica")],
)
def test_main_and_replicas_cannot_call_show_repl_cluster(port, role, connection):
cursor = connection(port, role).cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(cursor, "SHOW REPLICATION CLUSTER;")
assert str(e.value) == "Only coordinator can run SHOW REPLICATION CLUSTER."
@pytest.mark.parametrize(
"port, role",
[(7687, "main"), (7688, "replica"), (7689, "replica")],
)
def test_main_and_replicas_cannot_register_coord_server(port, role, connection):
cursor = connection(port, role).cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(
cursor,
"REGISTER REPLICA instance_1 SYNC TO '127.0.0.1:10001' WITH COORDINATOR SERVER ON '127.0.0.1:10011';",
)
assert str(e.value) == "Only coordinator can register coordinator server!"
@pytest.mark.parametrize(
"port, role",
[(7687, "main"), (7688, "replica"), (7689, "replica")],
)
def test_main_and_replicas_cannot_run_do_failover(port, role, connection):
cursor = connection(port, role).cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(cursor, "DO FAILOVER;")
assert str(e.value) == "Only coordinator can run DO FAILOVER!"
if __name__ == "__main__":
sys.exit(pytest.main([__file__, "-rA"]))

View File

@ -0,0 +1,26 @@
# Copyright 2022 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.
import sys
import pytest
from common import execute_and_fetch_all
def test_failover_on_non_setup_cluster(connection):
cursor = connection(7690, "coordinator").cursor()
with pytest.raises(Exception) as e:
execute_and_fetch_all(cursor, "DO FAILOVER;")
assert str(e.value) == "Failover aborted since cluster is uninitialized!"
if __name__ == "__main__":
sys.exit(pytest.main([__file__, "-rA"]))

View File

@ -0,0 +1,63 @@
ha_cluster: &ha_cluster
cluster:
replica_1:
args: ["--bolt-port", "7688", "--log-level=TRACE", "--coordinator-server-port=10011"]
log_file: "replication-e2e-replica1.log"
setup_queries: ["SET REPLICATION ROLE TO REPLICA WITH PORT 10001;"]
replica_2:
args: ["--bolt-port", "7689", "--log-level=TRACE", "--coordinator-server-port=10012"]
log_file: "replication-e2e-replica2.log"
setup_queries: ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"]
main:
args: ["--bolt-port", "7687", "--log-level=TRACE", "--coordinator-server-port=10013"]
log_file: "replication-e2e-main.log"
setup_queries: [
"REGISTER REPLICA replica_1 SYNC TO '127.0.0.1:10001'",
"REGISTER REPLICA replica_2 SYNC TO '127.0.0.1:10002'",
]
coordinator:
args: ["--bolt-port", "7690", "--log-level=TRACE", "--coordinator"]
log_file: "replication-e2e-coordinator.log"
setup_queries: [
"REGISTER MAIN main WITH COORDINATOR SERVER ON '127.0.0.1:10013'",
"REGISTER REPLICA replica_1 SYNC TO '127.0.0.1:10001' WITH COORDINATOR SERVER ON '127.0.0.1:10011'",
"REGISTER REPLICA replica_2 SYNC TO '127.0.0.1:10002' WITH COORDINATOR SERVER ON '127.0.0.1:10012'",
]
noninitialized_cluster: &noninitialized_cluster
cluster:
replica_1:
args: ["--bolt-port", "7688", "--log-level=TRACE", "--coordinator-server-port=10011"]
log_file: "replication-e2e-replica1.log"
setup_queries: ["SET REPLICATION ROLE TO REPLICA WITH PORT 10001;"]
replica_2:
args: ["--bolt-port", "7689", "--log-level=TRACE", "--coordinator-server-port=10012"]
log_file: "replication-e2e-replica2.log"
setup_queries: ["SET REPLICATION ROLE TO REPLICA WITH PORT 10002;"]
main:
args: ["--bolt-port", "7687", "--log-level=TRACE", "--coordinator-server-port=10013"]
log_file: "replication-e2e-main.log"
setup_queries: [
"REGISTER REPLICA replica_1 SYNC TO '127.0.0.1:10001'",
"REGISTER REPLICA replica_2 SYNC TO '127.0.0.1:10002'",
]
coordinator:
args: ["--bolt-port", "7690", "--log-level=TRACE", "--coordinator"]
log_file: "replication-e2e-coordinator.log"
setup_queries: []
workloads:
- name: "Coordinator"
binary: "tests/e2e/pytest_runner.sh"
args: ["high_availability_experimental/coordinator.py"]
<<: *ha_cluster
- name: "Uninitialized cluster"
binary: "tests/e2e/pytest_runner.sh"
args: ["high_availability_experimental/uninitialized_cluster.py"]
<<: *noninitialized_cluster
- name: "Client initiated failover"
binary: "tests/e2e/pytest_runner.sh"
args: ["high_availability_experimental/client_initiated_failover.py"]

View File

@ -40,6 +40,7 @@ BASIC_PRIVILEGES = [
"STORAGE_MODE",
"MULTI_DATABASE_EDIT",
"MULTI_DATABASE_USE",
"COORDINATOR",
]
@ -63,7 +64,7 @@ def test_lba_procedures_show_privileges_first_user():
cursor = connect(username="Josip", password="").cursor()
result = execute_and_fetch_all(cursor, "SHOW PRIVILEGES FOR Josip;")
assert len(result) == 34
assert len(result) == 35
fine_privilege_results = [res for res in result if res[0] not in BASIC_PRIVILEGES]

View File

@ -9,13 +9,13 @@ target_link_libraries(memgraph__e2e__replication__indices gflags mgclient mg-uti
add_executable(memgraph__e2e__replication__read_write_benchmark read_write_benchmark.cpp)
target_link_libraries(memgraph__e2e__replication__read_write_benchmark gflags json mgclient mg-utils mg-io Threads::Threads)
copy_e2e_python_files(replication_show common.py)
copy_e2e_python_files(replication_show conftest.py)
copy_e2e_python_files(replication_show show.py)
copy_e2e_python_files(replication_show show_while_creating_invalid_state.py)
copy_e2e_python_files(replication_show edge_delete.py)
copy_e2e_python_files_from_parent_folder(replication_show ".." memgraph.py)
copy_e2e_python_files_from_parent_folder(replication_show ".." interactive_mg_runner.py)
copy_e2e_python_files_from_parent_folder(replication_show ".." mg_utils.py)
copy_e2e_python_files(replication common.py)
copy_e2e_python_files(replication conftest.py)
copy_e2e_python_files(replication show.py)
copy_e2e_python_files(replication show_while_creating_invalid_state.py)
copy_e2e_python_files(replication edge_delete.py)
copy_e2e_python_files_from_parent_folder(replication ".." memgraph.py)
copy_e2e_python_files_from_parent_folder(replication ".." interactive_mg_runner.py)
copy_e2e_python_files_from_parent_folder(replication ".." mg_utils.py)
copy_e2e_files(replication workloads.yaml)

View File

@ -11,7 +11,7 @@ template_validation_queries: &template_validation_queries
template_simple_cluster: &template_simple_cluster
cluster:
replica_1:
args: [ "--bolt-port", "7688", "--log-level=TRACE" ]
args: [ "--bolt-port", "7688", "--log-level=TRACE"]
log_file: "replication-e2e-replica1.log"
setup_queries: [ "SET REPLICATION ROLE TO REPLICA WITH PORT 10001;" ]
replica_2:
@ -25,6 +25,7 @@ template_simple_cluster: &template_simple_cluster
"REGISTER REPLICA replica_1 SYNC TO '127.0.0.1:10001'",
"REGISTER REPLICA replica_2 ASYNC TO '127.0.0.1:10002'",
]
template_cluster: &template_cluster
cluster:
replica_1:
@ -50,7 +51,6 @@ template_cluster: &template_cluster
"REGISTER REPLICA replica_2 SYNC TO '127.0.0.1:10002'",
"REGISTER REPLICA replica_3 ASYNC TO '127.0.0.1:10003'"
]
<<: *template_validation_queries
workloads:
- name: "Constraints"

View File

@ -350,7 +350,7 @@ add_unit_test(storage_v2_wal_file.cpp)
target_link_libraries(${test_prefix}storage_v2_wal_file mg-storage-v2 storage_test_utils fmt)
add_unit_test(storage_v2_replication.cpp)
target_link_libraries(${test_prefix}storage_v2_replication mg-storage-v2 mg-dbms fmt)
target_link_libraries(${test_prefix}storage_v2_replication mg-storage-v2 mg-dbms fmt mg-repl_coord_glue)
add_unit_test(storage_v2_isolation_level.cpp)
target_link_libraries(${test_prefix}storage_v2_isolation_level mg-storage-v2)
@ -368,7 +368,7 @@ add_unit_test(storage_v2_storage_mode.cpp)
target_link_libraries(${test_prefix}storage_v2_storage_mode mg-storage-v2 storage_test_utils mg-query mg-glue)
add_unit_test(replication_persistence_helper.cpp)
target_link_libraries(${test_prefix}replication_persistence_helper mg-storage-v2)
target_link_libraries(${test_prefix}replication_persistence_helper mg-storage-v2 mg-repl_coord_glue)
add_unit_test(auth_checker.cpp)
target_link_libraries(${test_prefix}auth_checker mg-glue mg-auth)
@ -389,7 +389,7 @@ endif()
# Test mg-slk
if(MG_ENTERPRISE)
add_unit_test(slk_advanced.cpp)
target_link_libraries(${test_prefix}slk_advanced mg-storage-v2)
target_link_libraries(${test_prefix}slk_advanced mg-storage-v2 mg-replication mg-coordination mg-repl_coord_glue)
endif()
add_unit_test(slk_core.cpp)

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -43,6 +43,7 @@
#include "query/procedure/module.hpp"
#include "query/typed_value.hpp"
#include "utils/logging.hpp"
#include "utils/string.hpp"
#include "utils/variant_helpers.hpp"
@ -2550,7 +2551,7 @@ TEST_P(CypherMainVisitorTest, ShowUsersForRole) {
void check_replication_query(Base *ast_generator, const ReplicationQuery *query, const std::string name,
const std::optional<TypedValue> socket_address, const ReplicationQuery::SyncMode sync_mode,
const std::optional<TypedValue> port = {}) {
EXPECT_EQ(query->replica_name_, name);
EXPECT_EQ(query->instance_name_, name);
EXPECT_EQ(query->sync_mode_, sync_mode);
ASSERT_EQ(static_cast<bool>(query->socket_address_), static_cast<bool>(socket_address));
if (socket_address) {
@ -2597,7 +2598,7 @@ TEST_P(CypherMainVisitorTest, TestSetReplicationMode) {
}
{
const std::string query = "SET REPLICATION ROLE TO MAIN WITH PORT 10000";
const std::string query = "SET REPLICATION ROLE TO REPLICA";
ASSERT_THROW(ast_generator.ParseQuery(query), SemanticException);
}
@ -2631,6 +2632,77 @@ TEST_P(CypherMainVisitorTest, TestRegisterReplicationQuery) {
ReplicationQuery::SyncMode::SYNC);
}
#ifdef MG_ENTERPRISE
TEST_P(CypherMainVisitorTest, TestRegisterCoordinatorServer) {
auto &ast_generator = *GetParam();
{
const std::string faulty_query_1 = "REGISTER MAIN COORDINATOR SERVER TO";
ASSERT_THROW(ast_generator.ParseQuery(faulty_query_1), SyntaxException);
}
{
const std::string faulty_query_2 = "REGISTER MAIN COORDINATOR SERVER TO MAIN";
ASSERT_THROW(ast_generator.ParseQuery(faulty_query_2), SyntaxException);
}
{
std::string full_query = "REGISTER MAIN main WITH COORDINATOR SERVER ON '127.0.0.1:10011';";
auto *full_query_parsed = dynamic_cast<CoordinatorQuery *>(ast_generator.ParseQuery(full_query));
ASSERT_TRUE(full_query_parsed);
EXPECT_EQ(full_query_parsed->action_, CoordinatorQuery::Action::REGISTER_MAIN_COORDINATOR_SERVER);
EXPECT_EQ(full_query_parsed->role_, CoordinatorQuery::ReplicationRole::MAIN);
EXPECT_EQ(full_query_parsed->instance_name_, "main");
ast_generator.CheckLiteral(full_query_parsed->coordinator_socket_address_, "127.0.0.1:10011");
ASSERT_EQ(full_query_parsed->socket_address_, nullptr);
}
{
std::string full_query =
R"(REGISTER REPLICA replica_1 SYNC TO "127.0.0.1:10002" WITH COORDINATOR SERVER ON "127.0.0.1:10012")";
auto *full_query_parsed = dynamic_cast<CoordinatorQuery *>(ast_generator.ParseQuery(full_query));
ASSERT_TRUE(full_query_parsed);
EXPECT_EQ(full_query_parsed->action_, CoordinatorQuery::Action::REGISTER_REPLICA_COORDINATOR_SERVER);
EXPECT_EQ(full_query_parsed->role_, CoordinatorQuery::ReplicationRole::REPLICA);
ast_generator.CheckLiteral(full_query_parsed->socket_address_, "127.0.0.1:10002");
ast_generator.CheckLiteral(full_query_parsed->coordinator_socket_address_, "127.0.0.1:10012");
EXPECT_EQ(full_query_parsed->instance_name_, "replica_1");
EXPECT_EQ(full_query_parsed->sync_mode_, CoordinatorQuery::SyncMode::SYNC);
}
{
std::string full_query =
R"(REGISTER REPLICA replica_1 ASYNC TO '127.0.0.1:10002' WITH COORDINATOR SERVER ON '127.0.0.1:10012')";
auto *full_query_parsed = dynamic_cast<CoordinatorQuery *>(ast_generator.ParseQuery(full_query));
ASSERT_TRUE(full_query_parsed);
EXPECT_EQ(full_query_parsed->action_, CoordinatorQuery::Action::REGISTER_REPLICA_COORDINATOR_SERVER);
EXPECT_EQ(full_query_parsed->role_, CoordinatorQuery::ReplicationRole::REPLICA);
ast_generator.CheckLiteral(full_query_parsed->socket_address_, "127.0.0.1:10002");
ast_generator.CheckLiteral(full_query_parsed->coordinator_socket_address_, "127.0.0.1:10012");
EXPECT_EQ(full_query_parsed->instance_name_, "replica_1");
EXPECT_EQ(full_query_parsed->sync_mode_, CoordinatorQuery::SyncMode::ASYNC);
}
}
TEST_P(CypherMainVisitorTest, TestDoFailover) {
auto &ast_generator = *GetParam();
{
std::string invalid_query = "DO FAILO";
ASSERT_THROW(ast_generator.ParseQuery(invalid_query), SyntaxException);
}
{
std::string correct_query = "DO FAILOVER";
auto *correct_query_parsed = dynamic_cast<CoordinatorQuery *>(ast_generator.ParseQuery(correct_query));
ASSERT_TRUE(correct_query_parsed);
EXPECT_EQ(correct_query_parsed->action_, CoordinatorQuery::Action::DO_FAILOVER);
}
}
#endif
TEST_P(CypherMainVisitorTest, TestDeleteReplica) {
auto &ast_generator = *GetParam();
@ -2640,7 +2712,7 @@ TEST_P(CypherMainVisitorTest, TestDeleteReplica) {
std::string correct_query = "DROP REPLICA replica1";
auto *correct_query_parsed = dynamic_cast<ReplicationQuery *>(ast_generator.ParseQuery(correct_query));
ASSERT_TRUE(correct_query_parsed);
EXPECT_EQ(correct_query_parsed->replica_name_, "replica1");
EXPECT_EQ(correct_query_parsed->instance_name_, "replica1");
}
TEST_P(CypherMainVisitorTest, TestExplainRegularQuery) {
@ -3645,7 +3717,7 @@ TEST_P(CypherMainVisitorTest, MemoryLimit) {
ASSERT_TRUE(query->single_query_);
auto *single_query = query->single_query_;
ASSERT_EQ(single_query->clauses_.size(), 2U);
auto *call_proc = dynamic_cast<CallProcedure *>(single_query->clauses_[0]);
[[maybe_unused]] auto *call_proc = dynamic_cast<CallProcedure *>(single_query->clauses_[0]);
}
{
@ -3705,7 +3777,7 @@ TEST_P(CypherMainVisitorTest, MemoryLimit) {
ASSERT_TRUE(query->single_query_);
auto *single_query = query->single_query_;
ASSERT_EQ(single_query->clauses_.size(), 1U);
auto *call_proc = dynamic_cast<CallProcedure *>(single_query->clauses_[0]);
[[maybe_unused]] auto *call_proc = dynamic_cast<CallProcedure *>(single_query->clauses_[0]);
}
}

View File

@ -1,4 +1,4 @@
// Copyright 2023 Memgraph Ltd.
// Copyright 2024 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
@ -22,12 +22,7 @@
using namespace memgraph::replication::durability;
using namespace memgraph::replication;
static_assert(sizeof(ReplicationRoleEntry) == 168,
"Most likely you modified ReplicationRoleEntry without updating the tests. ");
static_assert(sizeof(ReplicationReplicaEntry) == 160,
"Most likely you modified ReplicationReplicaEntry without updating the tests.");
using namespace memgraph::replication_coordination_glue;
TEST(ReplicationDurability, V1Main) {
auto const role_entry = ReplicationRoleEntry{.version = DurabilityVersion::V1,

View File

@ -1,4 +1,4 @@
// Copyright 2022 Memgraph Ltd.
// Copyright 2024 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
@ -11,10 +11,13 @@
#include <gtest/gtest.h>
#include "coordination/coordinator_config.hpp"
#include "coordination/coordinator_slk.hpp"
#include "replication/config.hpp"
#include "replication_coordination_glue/mode.hpp"
#include "slk_common.hpp"
#include "storage/v2/property_value.hpp"
#include "storage/v2/replication/slk.hpp"
#include "slk_common.hpp"
#include "storage/v2/temporal.hpp"
TEST(SlkAdvanced, PropertyValueList) {
@ -114,3 +117,34 @@ TEST(SlkAdvanced, PropertyValueComplex) {
ASSERT_EQ(original, decoded);
}
TEST(SlkAdvanced, ReplicationClientConfigs) {
using ReplicationClientInfo = memgraph::coordination::CoordinatorClientConfig::ReplicationClientInfo;
using ReplicationClientInfoVec = std::vector<ReplicationClientInfo>;
using ReplicationMode = memgraph::replication_coordination_glue::ReplicationMode;
ReplicationClientInfoVec original{ReplicationClientInfo{.instance_name = "replica1",
.replication_mode = ReplicationMode::SYNC,
.replication_ip_address = "127.0.0.1",
.replication_port = 10000},
ReplicationClientInfo{.instance_name = "replica2",
.replication_mode = ReplicationMode::ASYNC,
.replication_ip_address = "127.0.1.1",
.replication_port = 10010},
ReplicationClientInfo{
.instance_name = "replica3",
.replication_mode = ReplicationMode::ASYNC,
.replication_ip_address = "127.1.1.1",
.replication_port = 1110,
}};
memgraph::slk::Loopback loopback;
auto builder = loopback.GetBuilder();
memgraph::slk::Save(original, builder);
ReplicationClientInfoVec decoded;
auto reader = loopback.GetReader();
memgraph::slk::Load(&decoded, reader);
ASSERT_EQ(original, decoded);
}

View File

@ -39,9 +39,9 @@ using memgraph::dbms::RegisterReplicaError;
using memgraph::dbms::ReplicationHandler;
using memgraph::dbms::UnregisterReplicaResult;
using memgraph::replication::ReplicationClientConfig;
using memgraph::replication::ReplicationMode;
using memgraph::replication::ReplicationRole;
using memgraph::replication::ReplicationServerConfig;
using memgraph::replication_coordination_glue::ReplicationMode;
using memgraph::storage::Config;
using memgraph::storage::EdgeAccessor;
using memgraph::storage::Gid;