mirror of
https://github.com/XRPLF/clio.git
synced 2026-01-22 15:45:30 +00:00
Compare commits
3 Commits
nightly-20
...
develop
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
59d07fab64 | ||
|
|
3bb3e0b9f9 | ||
|
|
a72e5a180f |
@@ -29,6 +29,8 @@
|
||||
#include "etl/ETLService.hpp"
|
||||
#include "etl/LoadBalancer.hpp"
|
||||
#include "etl/NetworkValidatedLedgers.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/WriterState.hpp"
|
||||
#include "feed/SubscriptionManager.hpp"
|
||||
#include "migration/MigrationInspectorFactory.hpp"
|
||||
#include "rpc/Counters.hpp"
|
||||
@@ -121,7 +123,11 @@ ClioApplication::run(bool const useNgWebServer)
|
||||
// Interface to the database
|
||||
auto backend = data::makeBackend(config_, cache);
|
||||
|
||||
cluster::ClusterCommunicationService clusterCommunicationService{backend};
|
||||
auto systemState = etl::SystemState::makeSystemState(config_);
|
||||
|
||||
cluster::ClusterCommunicationService clusterCommunicationService{
|
||||
backend, std::make_unique<etl::WriterState>(systemState)
|
||||
};
|
||||
clusterCommunicationService.run();
|
||||
|
||||
auto const amendmentCenter = std::make_shared<data::AmendmentCenter const>(backend);
|
||||
@@ -151,7 +157,9 @@ ClioApplication::run(bool const useNgWebServer)
|
||||
);
|
||||
|
||||
// ETL is responsible for writing and publishing to streams. In read-only mode, ETL only publishes
|
||||
auto etl = etl::ETLService::makeETLService(config_, ctx, backend, subscriptions, balancer, ledgers);
|
||||
auto etl = etl::ETLService::makeETLService(
|
||||
config_, std::move(systemState), ctx, backend, subscriptions, balancer, ledgers
|
||||
);
|
||||
|
||||
auto workQueue = rpc::WorkQueue::makeWorkQueue(config_);
|
||||
auto counters = rpc::Counters::makeCounters(workQueue);
|
||||
@@ -197,7 +205,16 @@ ClioApplication::run(bool const useNgWebServer)
|
||||
}
|
||||
|
||||
appStopper_.setOnStop(
|
||||
Stopper::makeOnStopCallback(httpServer.value(), *balancer, *etl, *subscriptions, *backend, cacheSaver, ioc)
|
||||
Stopper::makeOnStopCallback(
|
||||
httpServer.value(),
|
||||
*balancer,
|
||||
*etl,
|
||||
*subscriptions,
|
||||
*backend,
|
||||
cacheSaver,
|
||||
clusterCommunicationService,
|
||||
ioc
|
||||
)
|
||||
);
|
||||
|
||||
// Blocks until stopped.
|
||||
@@ -213,7 +230,9 @@ ClioApplication::run(bool const useNgWebServer)
|
||||
|
||||
auto const httpServer = web::makeHttpServer(config_, ioc, dosGuard, handler, cache);
|
||||
appStopper_.setOnStop(
|
||||
Stopper::makeOnStopCallback(*httpServer, *balancer, *etl, *subscriptions, *backend, cacheSaver, ioc)
|
||||
Stopper::makeOnStopCallback(
|
||||
*httpServer, *balancer, *etl, *subscriptions, *backend, cacheSaver, clusterCommunicationService, ioc
|
||||
)
|
||||
);
|
||||
|
||||
// Blocks until stopped.
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "cluster/Concepts.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/LedgerCacheSaver.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
@@ -82,10 +83,14 @@ public:
|
||||
* @param subscriptions The subscription manager to stop.
|
||||
* @param backend The backend to stop.
|
||||
* @param cacheSaver The ledger cache saver
|
||||
* @param clusterCommunicationService The cluster communication service to stop.
|
||||
* @param ioc The io_context to stop.
|
||||
* @return The callback to be called on application stop.
|
||||
*/
|
||||
template <web::SomeServer ServerType, data::SomeLedgerCacheSaver LedgerCacheSaverType>
|
||||
template <
|
||||
web::SomeServer ServerType,
|
||||
data::SomeLedgerCacheSaver LedgerCacheSaverType,
|
||||
cluster::SomeClusterCommunicationService ClusterCommunicationServiceType>
|
||||
static std::function<void(boost::asio::yield_context)>
|
||||
makeOnStopCallback(
|
||||
ServerType& server,
|
||||
@@ -94,6 +99,7 @@ public:
|
||||
feed::SubscriptionManagerInterface& subscriptions,
|
||||
data::BackendInterface& backend,
|
||||
LedgerCacheSaverType& cacheSaver,
|
||||
ClusterCommunicationServiceType& clusterCommunicationService,
|
||||
boost::asio::io_context& ioc
|
||||
)
|
||||
{
|
||||
@@ -111,6 +117,8 @@ public:
|
||||
});
|
||||
coroutineGroup.asyncWait(yield);
|
||||
|
||||
clusterCommunicationService.stop();
|
||||
|
||||
etl.stop();
|
||||
LOG(util::LogService::info()) << "ETL stopped";
|
||||
|
||||
|
||||
134
src/cluster/Backend.cpp
Normal file
134
src/cluster/Backend.cpp
Normal file
@@ -0,0 +1,134 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/WriterState.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/thread_pool.hpp>
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <boost/json/serialize.hpp>
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/json/value_from.hpp>
|
||||
#include <boost/json/value_to.hpp>
|
||||
#include <boost/uuid/random_generator.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <fmt/format.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
Backend::Backend(
|
||||
boost::asio::thread_pool& ctx,
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::unique_ptr<etl::WriterStateInterface const> writerState,
|
||||
std::chrono::steady_clock::duration readInterval,
|
||||
std::chrono::steady_clock::duration writeInterval
|
||||
)
|
||||
: backend_(std::move(backend))
|
||||
, writerState_(std::move(writerState))
|
||||
, readerTask_(readInterval, ctx)
|
||||
, writerTask_(writeInterval, ctx)
|
||||
, selfUuid_(std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator{}()))
|
||||
{
|
||||
}
|
||||
|
||||
void
|
||||
Backend::run()
|
||||
{
|
||||
readerTask_.run([this](boost::asio::yield_context yield) {
|
||||
auto clusterData = doRead(yield);
|
||||
onNewState_(selfUuid_, std::make_shared<ClusterData>(std::move(clusterData)));
|
||||
});
|
||||
|
||||
writerTask_.run([this]() { doWrite(); });
|
||||
}
|
||||
|
||||
Backend::~Backend()
|
||||
{
|
||||
stop();
|
||||
}
|
||||
|
||||
void
|
||||
Backend::stop()
|
||||
{
|
||||
readerTask_.stop();
|
||||
writerTask_.stop();
|
||||
}
|
||||
|
||||
ClioNode::CUuid
|
||||
Backend::selfId() const
|
||||
{
|
||||
return selfUuid_;
|
||||
}
|
||||
|
||||
Backend::ClusterData
|
||||
Backend::doRead(boost::asio::yield_context yield)
|
||||
{
|
||||
BackendInterface::ClioNodesDataFetchResult expectedResult;
|
||||
try {
|
||||
expectedResult = backend_->fetchClioNodesData(yield);
|
||||
} catch (...) {
|
||||
expectedResult = std::unexpected{"Failed to fetch Clio nodes data"};
|
||||
}
|
||||
|
||||
if (!expectedResult.has_value()) {
|
||||
return std::unexpected{std::move(expectedResult).error()};
|
||||
}
|
||||
|
||||
std::vector<ClioNode> otherNodesData;
|
||||
for (auto const& [uuid, nodeDataStr] : expectedResult.value()) {
|
||||
if (uuid == *selfUuid_) {
|
||||
continue;
|
||||
}
|
||||
|
||||
boost::system::error_code errorCode;
|
||||
auto const json = boost::json::parse(nodeDataStr, errorCode);
|
||||
if (errorCode.failed()) {
|
||||
return std::unexpected{fmt::format("Error parsing json from DB: {}", nodeDataStr)};
|
||||
}
|
||||
|
||||
auto expectedNodeData = boost::json::try_value_to<ClioNode>(json);
|
||||
if (expectedNodeData.has_error()) {
|
||||
return std::unexpected{fmt::format("Error converting json to ClioNode: {}", nodeDataStr)};
|
||||
}
|
||||
*expectedNodeData->uuid = uuid;
|
||||
otherNodesData.push_back(std::move(expectedNodeData).value());
|
||||
}
|
||||
otherNodesData.push_back(ClioNode::from(selfUuid_, *writerState_));
|
||||
return otherNodesData;
|
||||
}
|
||||
|
||||
void
|
||||
Backend::doWrite()
|
||||
{
|
||||
auto const selfData = ClioNode::from(selfUuid_, *writerState_);
|
||||
boost::json::value jsonValue{};
|
||||
boost::json::value_from(selfData, jsonValue);
|
||||
backend_->writeNodeMessage(*selfData.uuid, boost::json::serialize(jsonValue.as_object()));
|
||||
}
|
||||
|
||||
} // namespace cluster
|
||||
147
src/cluster/Backend.hpp
Normal file
147
src/cluster/Backend.hpp
Normal file
@@ -0,0 +1,147 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "cluster/impl/RepeatedTask.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/WriterState.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/any_io_executor.hpp>
|
||||
#include <boost/asio/cancellation_signal.hpp>
|
||||
#include <boost/asio/execution_context.hpp>
|
||||
#include <boost/asio/executor.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <boost/asio/thread_pool.hpp>
|
||||
#include <boost/signals2/connection.hpp>
|
||||
#include <boost/signals2/signal.hpp>
|
||||
#include <boost/signals2/variadic_signal.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <concepts>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
/**
|
||||
* @brief Backend communication handler for cluster state synchronization.
|
||||
*
|
||||
* This class manages reading and writing cluster state information to/from the backend database.
|
||||
* It periodically reads the state of other nodes in the cluster and writes the current node's state,
|
||||
* enabling cluster-wide coordination and awareness.
|
||||
*/
|
||||
class Backend {
|
||||
public:
|
||||
/** @brief Type representing cluster data result - either a vector of nodes or an error message */
|
||||
using ClusterData = std::expected<std::vector<ClioNode>, std::string>;
|
||||
|
||||
private:
|
||||
util::Logger log_{"ClusterCommunication"};
|
||||
|
||||
std::shared_ptr<data::BackendInterface> backend_;
|
||||
std::unique_ptr<etl::WriterStateInterface const> writerState_;
|
||||
|
||||
impl::RepeatedTask<boost::asio::thread_pool> readerTask_;
|
||||
impl::RepeatedTask<boost::asio::thread_pool> writerTask_;
|
||||
|
||||
ClioNode::Uuid selfUuid_;
|
||||
|
||||
boost::signals2::signal<void(ClioNode::CUuid, std::shared_ptr<ClusterData const>)> onNewState_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Construct a Backend communication handler.
|
||||
*
|
||||
* @param ctx The execution context for asynchronous operations
|
||||
* @param backend Interface to the backend database
|
||||
* @param writerState State indicating whether this node is writing to the database
|
||||
* @param readInterval How often to read cluster state from the backend
|
||||
* @param writeInterval How often to write this node's state to the backend
|
||||
*/
|
||||
Backend(
|
||||
boost::asio::thread_pool& ctx,
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::unique_ptr<etl::WriterStateInterface const> writerState,
|
||||
std::chrono::steady_clock::duration readInterval,
|
||||
std::chrono::steady_clock::duration writeInterval
|
||||
);
|
||||
|
||||
~Backend();
|
||||
|
||||
Backend(Backend&&) = delete;
|
||||
Backend&
|
||||
operator=(Backend&&) = delete;
|
||||
Backend(Backend const&) = delete;
|
||||
Backend&
|
||||
operator=(Backend const&) = delete;
|
||||
|
||||
/**
|
||||
* @brief Start the backend read and write tasks.
|
||||
*
|
||||
* Begins periodic reading of cluster state from the backend and writing of this node's state.
|
||||
*/
|
||||
void
|
||||
run();
|
||||
|
||||
/**
|
||||
* @brief Stop the backend read and write tasks.
|
||||
*
|
||||
* Stops all periodic tasks and waits for them to complete.
|
||||
*/
|
||||
void
|
||||
stop();
|
||||
|
||||
/**
|
||||
* @brief Subscribe to new cluster state notifications.
|
||||
*
|
||||
* @tparam S Callable type accepting (ClioNode::cUUID, ClusterData)
|
||||
* @param s Subscriber callback to be invoked when new cluster state is available
|
||||
* @return A connection object that can be used to unsubscribe
|
||||
*/
|
||||
template <typename S>
|
||||
requires std::invocable<S, ClioNode::CUuid, std::shared_ptr<ClusterData const>>
|
||||
boost::signals2::connection
|
||||
subscribeToNewState(S&& s)
|
||||
{
|
||||
return onNewState_.connect(s);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get the UUID of this node in the cluster.
|
||||
*
|
||||
* @return The UUID of this node.
|
||||
*/
|
||||
ClioNode::CUuid
|
||||
selfId() const;
|
||||
|
||||
private:
|
||||
ClusterData
|
||||
doRead(boost::asio::yield_context yield);
|
||||
|
||||
void
|
||||
doWrite();
|
||||
};
|
||||
|
||||
} // namespace cluster
|
||||
@@ -1,5 +1,7 @@
|
||||
add_library(clio_cluster)
|
||||
|
||||
target_sources(clio_cluster PRIVATE ClioNode.cpp ClusterCommunicationService.cpp)
|
||||
target_sources(
|
||||
clio_cluster PRIVATE Backend.cpp ClioNode.cpp ClusterCommunicationService.cpp Metrics.cpp WriterDecider.cpp
|
||||
)
|
||||
|
||||
target_link_libraries(clio_cluster PRIVATE clio_util clio_data)
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
|
||||
#include "cluster/ClioNode.hpp"
|
||||
|
||||
#include "etl/WriterState.hpp"
|
||||
#include "util/TimeUtils.hpp"
|
||||
|
||||
#include <boost/json/conversion.hpp>
|
||||
@@ -26,39 +27,72 @@
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
namespace {
|
||||
|
||||
struct Fields {
|
||||
struct JsonFields {
|
||||
static constexpr std::string_view const kUPDATE_TIME = "update_time";
|
||||
static constexpr std::string_view const kDB_ROLE = "db_role";
|
||||
};
|
||||
|
||||
} // namespace
|
||||
|
||||
ClioNode
|
||||
ClioNode::from(ClioNode::Uuid uuid, etl::WriterStateInterface const& writerState)
|
||||
{
|
||||
auto const dbRole = [&writerState]() {
|
||||
if (writerState.isReadOnly()) {
|
||||
return ClioNode::DbRole::ReadOnly;
|
||||
}
|
||||
if (writerState.isFallback()) {
|
||||
return ClioNode::DbRole::Fallback;
|
||||
}
|
||||
if (writerState.isLoadingCache()) {
|
||||
return ClioNode::DbRole::LoadingCache;
|
||||
}
|
||||
|
||||
return writerState.isWriting() ? ClioNode::DbRole::Writer : ClioNode::DbRole::NotWriter;
|
||||
}();
|
||||
return ClioNode{.uuid = std::move(uuid), .updateTime = std::chrono::system_clock::now(), .dbRole = dbRole};
|
||||
}
|
||||
|
||||
void
|
||||
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, ClioNode const& node)
|
||||
{
|
||||
jv = {
|
||||
{Fields::kUPDATE_TIME, util::systemTpToUtcStr(node.updateTime, ClioNode::kTIME_FORMAT)},
|
||||
{JsonFields::kUPDATE_TIME, util::systemTpToUtcStr(node.updateTime, ClioNode::kTIME_FORMAT)},
|
||||
{JsonFields::kDB_ROLE, static_cast<int64_t>(node.dbRole)}
|
||||
};
|
||||
}
|
||||
|
||||
ClioNode
|
||||
tag_invoke(boost::json::value_to_tag<ClioNode>, boost::json::value const& jv)
|
||||
{
|
||||
auto const& updateTimeStr = jv.as_object().at(Fields::kUPDATE_TIME).as_string();
|
||||
auto const& updateTimeStr = jv.as_object().at(JsonFields::kUPDATE_TIME).as_string();
|
||||
auto const updateTime = util::systemTpFromUtcStr(std::string(updateTimeStr), ClioNode::kTIME_FORMAT);
|
||||
if (!updateTime.has_value()) {
|
||||
throw std::runtime_error("Failed to parse update time");
|
||||
}
|
||||
|
||||
return ClioNode{.uuid = std::make_shared<boost::uuids::uuid>(), .updateTime = updateTime.value()};
|
||||
auto const dbRoleValue = jv.as_object().at(JsonFields::kDB_ROLE).as_int64();
|
||||
if (dbRoleValue > static_cast<int64_t>(ClioNode::DbRole::MAX))
|
||||
throw std::runtime_error("Invalid db_role value");
|
||||
|
||||
return ClioNode{
|
||||
// Json data doesn't contain uuid so leaving it empty here. It will be filled outside of this parsing
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(),
|
||||
.updateTime = updateTime.value(),
|
||||
.dbRole = static_cast<ClioNode::DbRole>(dbRoleValue)
|
||||
};
|
||||
}
|
||||
|
||||
} // namespace cluster
|
||||
|
||||
@@ -19,6 +19,8 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/WriterState.hpp"
|
||||
|
||||
#include <boost/json/conversion.hpp>
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
@@ -37,16 +39,37 @@ struct ClioNode {
|
||||
*/
|
||||
static constexpr char const* kTIME_FORMAT = "%Y-%m-%dT%H:%M:%SZ";
|
||||
|
||||
// enum class WriterRole {
|
||||
// ReadOnly,
|
||||
// NotWriter,
|
||||
// Writer
|
||||
// };
|
||||
/**
|
||||
* @brief Database role of a node in the cluster.
|
||||
*
|
||||
* Roles are used to coordinate which node writes to the database:
|
||||
* - ReadOnly: Node is configured to never write (strict read-only mode)
|
||||
* - NotWriter: Node can write but is currently not the designated writer
|
||||
* - Writer: Node is actively writing to the database
|
||||
* - Fallback: Node is using the fallback writer decision mechanism
|
||||
*
|
||||
* When any node in the cluster is in Fallback mode, the entire cluster switches
|
||||
* from the cluster communication mechanism to the slower but more reliable
|
||||
* database-based conflict detection mechanism.
|
||||
*/
|
||||
enum class DbRole { ReadOnly = 0, LoadingCache = 1, NotWriter = 2, Writer = 3, Fallback = 4, MAX = 4 };
|
||||
|
||||
std::shared_ptr<boost::uuids::uuid> uuid; ///< The UUID of the node.
|
||||
using Uuid = std::shared_ptr<boost::uuids::uuid>;
|
||||
using CUuid = std::shared_ptr<boost::uuids::uuid const>;
|
||||
|
||||
Uuid uuid; ///< The UUID of the node.
|
||||
std::chrono::system_clock::time_point updateTime; ///< The time the data about the node was last updated.
|
||||
DbRole dbRole; ///< The database role of the node
|
||||
|
||||
// WriterRole writerRole;
|
||||
/**
|
||||
* @brief Create a ClioNode from writer state.
|
||||
*
|
||||
* @param uuid The UUID of the node
|
||||
* @param writerState The writer state to determine the node's database role
|
||||
* @return A ClioNode with the current time and role derived from writerState
|
||||
*/
|
||||
static ClioNode
|
||||
from(Uuid uuid, etl::WriterStateInterface const& writerState);
|
||||
};
|
||||
|
||||
void
|
||||
|
||||
@@ -19,98 +19,37 @@
|
||||
|
||||
#include "cluster/ClusterCommunicationService.hpp"
|
||||
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Spawn.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/bind_cancellation_slot.hpp>
|
||||
#include <boost/asio/cancellation_type.hpp>
|
||||
#include <boost/asio/error.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/steady_timer.hpp>
|
||||
#include <boost/asio/use_future.hpp>
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <boost/json/serialize.hpp>
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/json/value_from.hpp>
|
||||
#include <boost/json/value_to.hpp>
|
||||
#include <boost/uuid/random_generator.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include "etl/WriterState.hpp"
|
||||
|
||||
#include <chrono>
|
||||
#include <ctime>
|
||||
#include <latch>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace {
|
||||
constexpr auto kTOTAL_WORKERS = 2uz; // 1 reading and 1 writing worker (coroutines)
|
||||
} // namespace
|
||||
|
||||
namespace cluster {
|
||||
|
||||
ClusterCommunicationService::ClusterCommunicationService(
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::unique_ptr<etl::WriterStateInterface> writerState,
|
||||
std::chrono::steady_clock::duration readInterval,
|
||||
std::chrono::steady_clock::duration writeInterval
|
||||
)
|
||||
: backend_(std::move(backend))
|
||||
, readInterval_(readInterval)
|
||||
, writeInterval_(writeInterval)
|
||||
, finishedCountdown_(kTOTAL_WORKERS)
|
||||
, selfData_{ClioNode{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator{}()),
|
||||
.updateTime = std::chrono::system_clock::time_point{}
|
||||
}}
|
||||
: backend_(ctx_, std::move(backend), writerState->clone(), readInterval, writeInterval)
|
||||
, writerDecider_(ctx_, std::move(writerState))
|
||||
{
|
||||
nodesInClusterMetric_.set(1); // The node always sees itself
|
||||
isHealthy_ = true;
|
||||
}
|
||||
|
||||
void
|
||||
ClusterCommunicationService::run()
|
||||
{
|
||||
ASSERT(not running_ and not stopped_, "Can only be ran once");
|
||||
running_ = true;
|
||||
|
||||
util::spawn(strand_, [this](boost::asio::yield_context yield) {
|
||||
boost::asio::steady_timer timer(yield.get_executor());
|
||||
boost::system::error_code ec;
|
||||
|
||||
while (running_) {
|
||||
timer.expires_after(readInterval_);
|
||||
auto token = cancelSignal_.slot();
|
||||
timer.async_wait(boost::asio::bind_cancellation_slot(token, yield[ec]));
|
||||
|
||||
if (ec == boost::asio::error::operation_aborted or not running_)
|
||||
break;
|
||||
|
||||
doRead(yield);
|
||||
}
|
||||
|
||||
finishedCountdown_.count_down(1);
|
||||
backend_.subscribeToNewState([this](auto&&... args) {
|
||||
metrics_.onNewState(std::forward<decltype(args)>(args)...);
|
||||
});
|
||||
|
||||
util::spawn(strand_, [this](boost::asio::yield_context yield) {
|
||||
boost::asio::steady_timer timer(yield.get_executor());
|
||||
boost::system::error_code ec;
|
||||
|
||||
while (running_) {
|
||||
doWrite();
|
||||
timer.expires_after(writeInterval_);
|
||||
auto token = cancelSignal_.slot();
|
||||
timer.async_wait(boost::asio::bind_cancellation_slot(token, yield[ec]));
|
||||
|
||||
if (ec == boost::asio::error::operation_aborted or not running_)
|
||||
break;
|
||||
}
|
||||
|
||||
finishedCountdown_.count_down(1);
|
||||
backend_.subscribeToNewState([this](auto&&... args) {
|
||||
writerDecider_.onNewState(std::forward<decltype(args)>(args)...);
|
||||
});
|
||||
backend_.run();
|
||||
}
|
||||
|
||||
ClusterCommunicationService::~ClusterCommunicationService()
|
||||
@@ -121,107 +60,7 @@ ClusterCommunicationService::~ClusterCommunicationService()
|
||||
void
|
||||
ClusterCommunicationService::stop()
|
||||
{
|
||||
if (stopped_)
|
||||
return;
|
||||
|
||||
stopped_ = true;
|
||||
|
||||
// for ASAN to see through concurrency correctly we need to exit all coroutines before joining the ctx
|
||||
running_ = false;
|
||||
|
||||
// cancelSignal_ is not thread safe so we execute emit on the same strand
|
||||
boost::asio::spawn(
|
||||
strand_, [this](auto&&) { cancelSignal_.emit(boost::asio::cancellation_type::all); }, boost::asio::use_future
|
||||
)
|
||||
.wait();
|
||||
finishedCountdown_.wait();
|
||||
|
||||
ctx_.join();
|
||||
}
|
||||
|
||||
std::shared_ptr<boost::uuids::uuid>
|
||||
ClusterCommunicationService::selfUuid() const
|
||||
{
|
||||
// Uuid never changes so it is safe to copy it without using strand_
|
||||
return selfData_.uuid;
|
||||
}
|
||||
|
||||
ClioNode
|
||||
ClusterCommunicationService::selfData() const
|
||||
{
|
||||
ClioNode result{};
|
||||
util::spawn(strand_, [this, &result](boost::asio::yield_context) { result = selfData_; });
|
||||
return result;
|
||||
}
|
||||
|
||||
std::expected<std::vector<ClioNode>, std::string>
|
||||
ClusterCommunicationService::clusterData() const
|
||||
{
|
||||
if (not isHealthy_) {
|
||||
return std::unexpected{"Service is not healthy"};
|
||||
}
|
||||
std::vector<ClioNode> result;
|
||||
util::spawn(strand_, [this, &result](boost::asio::yield_context) {
|
||||
result = otherNodesData_;
|
||||
result.push_back(selfData_);
|
||||
});
|
||||
return result;
|
||||
}
|
||||
|
||||
void
|
||||
ClusterCommunicationService::doRead(boost::asio::yield_context yield)
|
||||
{
|
||||
otherNodesData_.clear();
|
||||
|
||||
BackendInterface::ClioNodesDataFetchResult expectedResult;
|
||||
try {
|
||||
expectedResult = backend_->fetchClioNodesData(yield);
|
||||
} catch (...) {
|
||||
expectedResult = std::unexpected{"Failed to fecth Clio nodes data"};
|
||||
}
|
||||
|
||||
if (!expectedResult.has_value()) {
|
||||
LOG(log_.error()) << "Failed to fetch nodes data";
|
||||
isHealthy_ = false;
|
||||
return;
|
||||
}
|
||||
|
||||
// Create a new vector here to not have partially parsed data in otherNodesData_
|
||||
std::vector<ClioNode> otherNodesData;
|
||||
for (auto const& [uuid, nodeDataStr] : expectedResult.value()) {
|
||||
if (uuid == *selfData_.uuid) {
|
||||
continue;
|
||||
}
|
||||
|
||||
boost::system::error_code errorCode;
|
||||
auto const json = boost::json::parse(nodeDataStr, errorCode);
|
||||
if (errorCode.failed()) {
|
||||
LOG(log_.error()) << "Error parsing json from DB: " << nodeDataStr;
|
||||
isHealthy_ = false;
|
||||
return;
|
||||
}
|
||||
|
||||
auto expectedNodeData = boost::json::try_value_to<ClioNode>(json);
|
||||
if (expectedNodeData.has_error()) {
|
||||
LOG(log_.error()) << "Error converting json to ClioNode: " << json;
|
||||
isHealthy_ = false;
|
||||
return;
|
||||
}
|
||||
*expectedNodeData->uuid = uuid;
|
||||
otherNodesData.push_back(std::move(expectedNodeData).value());
|
||||
}
|
||||
otherNodesData_ = std::move(otherNodesData);
|
||||
nodesInClusterMetric_.set(otherNodesData_.size() + 1);
|
||||
isHealthy_ = true;
|
||||
}
|
||||
|
||||
void
|
||||
ClusterCommunicationService::doWrite()
|
||||
{
|
||||
selfData_.updateTime = std::chrono::system_clock::now();
|
||||
boost::json::value jsonValue{};
|
||||
boost::json::value_from(selfData_, jsonValue);
|
||||
backend_->writeNodeMessage(*selfData_.uuid, boost::json::serialize(jsonValue.as_object()));
|
||||
backend_.stop();
|
||||
}
|
||||
|
||||
} // namespace cluster
|
||||
|
||||
@@ -19,13 +19,12 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "cluster/ClusterCommunicationServiceInterface.hpp"
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/Concepts.hpp"
|
||||
#include "cluster/Metrics.hpp"
|
||||
#include "cluster/WriterDecider.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/prometheus/Bool.hpp"
|
||||
#include "util/prometheus/Gauge.hpp"
|
||||
#include "util/prometheus/Prometheus.hpp"
|
||||
#include "etl/WriterState.hpp"
|
||||
|
||||
#include <boost/asio/cancellation_signal.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
@@ -33,67 +32,49 @@
|
||||
#include <boost/asio/thread_pool.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <latch>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
/**
|
||||
* @brief Service to post and read messages to/from the cluster. It uses a backend to communicate with the cluster.
|
||||
*/
|
||||
class ClusterCommunicationService : public ClusterCommunicationServiceInterface {
|
||||
util::prometheus::GaugeInt& nodesInClusterMetric_ = PrometheusService::gaugeInt(
|
||||
"cluster_nodes_total_number",
|
||||
{},
|
||||
"Total number of nodes this node can detect in the cluster."
|
||||
);
|
||||
util::prometheus::Bool isHealthy_ = PrometheusService::boolMetric(
|
||||
"cluster_communication_is_healthy",
|
||||
{},
|
||||
"Whether cluster communication service is operating healthy (1 - healthy, 0 - we have a problem)"
|
||||
);
|
||||
|
||||
class ClusterCommunicationService : public ClusterCommunicationServiceTag {
|
||||
// TODO: Use util::async::CoroExecutionContext after https://github.com/XRPLF/clio/issues/1973 is implemented
|
||||
boost::asio::thread_pool ctx_{1};
|
||||
boost::asio::strand<boost::asio::thread_pool::executor_type> strand_ = boost::asio::make_strand(ctx_);
|
||||
|
||||
util::Logger log_{"ClusterCommunication"};
|
||||
|
||||
std::shared_ptr<data::BackendInterface> backend_;
|
||||
|
||||
std::chrono::steady_clock::duration readInterval_;
|
||||
std::chrono::steady_clock::duration writeInterval_;
|
||||
|
||||
boost::asio::cancellation_signal cancelSignal_;
|
||||
std::latch finishedCountdown_;
|
||||
std::atomic_bool running_ = false;
|
||||
bool stopped_ = false;
|
||||
|
||||
ClioNode selfData_;
|
||||
std::vector<ClioNode> otherNodesData_;
|
||||
Backend backend_;
|
||||
Metrics metrics_;
|
||||
WriterDecider writerDecider_;
|
||||
|
||||
public:
|
||||
static constexpr std::chrono::milliseconds kDEFAULT_READ_INTERVAL{2100};
|
||||
static constexpr std::chrono::milliseconds kDEFAULT_WRITE_INTERVAL{1200};
|
||||
static constexpr std::chrono::milliseconds kDEFAULT_READ_INTERVAL{1000};
|
||||
static constexpr std::chrono::milliseconds kDEFAULT_WRITE_INTERVAL{1000};
|
||||
|
||||
/**
|
||||
* @brief Construct a new Cluster Communication Service object.
|
||||
*
|
||||
* @param backend The backend to use for communication.
|
||||
* @param writerState The state showing whether clio is writing to the database.
|
||||
* @param readInterval The interval to read messages from the cluster.
|
||||
* @param writeInterval The interval to write messages to the cluster.
|
||||
*/
|
||||
ClusterCommunicationService(
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::unique_ptr<etl::WriterStateInterface> writerState,
|
||||
std::chrono::steady_clock::duration readInterval = kDEFAULT_READ_INTERVAL,
|
||||
std::chrono::steady_clock::duration writeInterval = kDEFAULT_WRITE_INTERVAL
|
||||
);
|
||||
|
||||
~ClusterCommunicationService() override;
|
||||
|
||||
ClusterCommunicationService(ClusterCommunicationService&&) = delete;
|
||||
ClusterCommunicationService(ClusterCommunicationService const&) = delete;
|
||||
ClusterCommunicationService&
|
||||
operator=(ClusterCommunicationService&&) = delete;
|
||||
ClusterCommunicationService&
|
||||
operator=(ClusterCommunicationService const&) = delete;
|
||||
|
||||
/**
|
||||
* @brief Start the service.
|
||||
*/
|
||||
@@ -105,44 +86,6 @@ public:
|
||||
*/
|
||||
void
|
||||
stop();
|
||||
|
||||
ClusterCommunicationService(ClusterCommunicationService&&) = delete;
|
||||
ClusterCommunicationService(ClusterCommunicationService const&) = delete;
|
||||
ClusterCommunicationService&
|
||||
operator=(ClusterCommunicationService&&) = delete;
|
||||
ClusterCommunicationService&
|
||||
operator=(ClusterCommunicationService const&) = delete;
|
||||
|
||||
/**
|
||||
* @brief Get the UUID of the current node.
|
||||
*
|
||||
* @return The UUID of the current node.
|
||||
*/
|
||||
std::shared_ptr<boost::uuids::uuid>
|
||||
selfUuid() const;
|
||||
|
||||
/**
|
||||
* @brief Get the data of the current node.
|
||||
*
|
||||
* @return The data of the current node.
|
||||
*/
|
||||
ClioNode
|
||||
selfData() const override;
|
||||
|
||||
/**
|
||||
* @brief Get the data of all nodes in the cluster (including self).
|
||||
*
|
||||
* @return The data of all nodes in the cluster or error if the service is not healthy.
|
||||
*/
|
||||
std::expected<std::vector<ClioNode>, std::string>
|
||||
clusterData() const override;
|
||||
|
||||
private:
|
||||
void
|
||||
doRead(boost::asio::yield_context yield);
|
||||
|
||||
void
|
||||
doWrite();
|
||||
};
|
||||
|
||||
} // namespace cluster
|
||||
|
||||
39
src/cluster/Concepts.hpp
Normal file
39
src/cluster/Concepts.hpp
Normal file
@@ -0,0 +1,39 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <concepts>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
/**
|
||||
* @brief Tag type for cluster communication service implementations.
|
||||
*
|
||||
* This tag is used to identify types that implement cluster communication functionality.
|
||||
* Types should inherit from this tag to be recognized as cluster communication services.
|
||||
*/
|
||||
struct ClusterCommunicationServiceTag {
|
||||
virtual ~ClusterCommunicationServiceTag() = default;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept SomeClusterCommunicationService = std::derived_from<T, ClusterCommunicationServiceTag>;
|
||||
|
||||
} // namespace cluster
|
||||
@@ -17,38 +17,31 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
#include "cluster/Metrics.hpp"
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/ClioNode.hpp"
|
||||
|
||||
#include <expected>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
/**
|
||||
* @brief Interface for the cluster communication service.
|
||||
*/
|
||||
class ClusterCommunicationServiceInterface {
|
||||
public:
|
||||
virtual ~ClusterCommunicationServiceInterface() = default;
|
||||
Metrics::Metrics()
|
||||
{
|
||||
nodesInClusterMetric_.set(1); // The node always sees itself
|
||||
isHealthy_ = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get the data of the current node.
|
||||
*
|
||||
* @return The data of the current node.
|
||||
*/
|
||||
[[nodiscard]] virtual ClioNode
|
||||
selfData() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Get the data of all nodes in the cluster (including self).
|
||||
*
|
||||
* @return The data of all nodes in the cluster or error if the service is not healthy.
|
||||
*/
|
||||
[[nodiscard]] virtual std::expected<std::vector<ClioNode>, std::string>
|
||||
clusterData() const = 0;
|
||||
};
|
||||
void
|
||||
Metrics::onNewState(ClioNode::CUuid, std::shared_ptr<Backend::ClusterData const> clusterData)
|
||||
{
|
||||
if (clusterData->has_value()) {
|
||||
isHealthy_ = true;
|
||||
nodesInClusterMetric_.set(clusterData->value().size());
|
||||
} else {
|
||||
isHealthy_ = false;
|
||||
nodesInClusterMetric_.set(1);
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace cluster
|
||||
76
src/cluster/Metrics.hpp
Normal file
76
src/cluster/Metrics.hpp
Normal file
@@ -0,0 +1,76 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "util/prometheus/Bool.hpp"
|
||||
#include "util/prometheus/Gauge.hpp"
|
||||
#include "util/prometheus/Prometheus.hpp"
|
||||
|
||||
#include <memory>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
/**
|
||||
* @brief Manages Prometheus metrics for cluster communication and node tracking.
|
||||
*
|
||||
* This class tracks cluster-related metrics including:
|
||||
* - Total number of nodes detected in the cluster
|
||||
* - Health status of cluster communication
|
||||
*/
|
||||
class Metrics {
|
||||
/** @brief Gauge tracking the total number of nodes visible in the cluster */
|
||||
util::prometheus::GaugeInt& nodesInClusterMetric_ = PrometheusService::gaugeInt(
|
||||
"cluster_nodes_total_number",
|
||||
{},
|
||||
"Total number of nodes this node can detect in the cluster."
|
||||
);
|
||||
|
||||
/** @brief Boolean metric indicating whether cluster communication is healthy */
|
||||
util::prometheus::Bool isHealthy_ = PrometheusService::boolMetric(
|
||||
"cluster_communication_is_healthy",
|
||||
{},
|
||||
"Whether cluster communication service is operating healthy (1 - healthy, 0 - we have a problem)"
|
||||
);
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Constructs a Metrics instance and initializes metrics.
|
||||
*
|
||||
* Sets the initial node count to 1 (self) and marks communication as healthy.
|
||||
*/
|
||||
Metrics();
|
||||
|
||||
/**
|
||||
* @brief Updates metrics based on new cluster state.
|
||||
*
|
||||
* This callback is invoked when cluster state changes. It updates:
|
||||
* - Health status based on whether cluster data is available
|
||||
* - Node count to reflect the current cluster size
|
||||
*
|
||||
* @param uuid The UUID of the node (unused in current implementation)
|
||||
* @param clusterData Shared pointer to the current cluster data; may be empty if communication failed
|
||||
*/
|
||||
void
|
||||
onNewState(ClioNode::CUuid uuid, std::shared_ptr<Backend::ClusterData const> clusterData);
|
||||
};
|
||||
|
||||
} // namespace cluster
|
||||
98
src/cluster/WriterDecider.cpp
Normal file
98
src/cluster/WriterDecider.cpp
Normal file
@@ -0,0 +1,98 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "cluster/WriterDecider.hpp"
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "etl/WriterState.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Spawn.hpp"
|
||||
|
||||
#include <boost/asio/thread_pool.hpp>
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
WriterDecider::WriterDecider(boost::asio::thread_pool& ctx, std::unique_ptr<etl::WriterStateInterface> writerState)
|
||||
: ctx_(ctx), writerState_(std::move(writerState))
|
||||
{
|
||||
}
|
||||
|
||||
void
|
||||
WriterDecider::onNewState(ClioNode::CUuid selfId, std::shared_ptr<Backend::ClusterData const> clusterData)
|
||||
{
|
||||
if (not clusterData->has_value())
|
||||
return;
|
||||
|
||||
util::spawn(
|
||||
ctx_,
|
||||
[writerState = writerState_->clone(),
|
||||
selfId = std::move(selfId),
|
||||
clusterData = clusterData->value()](auto&&) mutable {
|
||||
auto const selfData =
|
||||
std::ranges::find_if(clusterData, [&selfId](ClioNode const& node) { return node.uuid == selfId; });
|
||||
ASSERT(selfData != clusterData.end(), "Self data should always be in the cluster data");
|
||||
|
||||
if (selfData->dbRole == ClioNode::DbRole::Fallback) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (selfData->dbRole == ClioNode::DbRole::ReadOnly) {
|
||||
writerState->giveUpWriting();
|
||||
return;
|
||||
}
|
||||
|
||||
// If any node in the cluster is in Fallback mode, the entire cluster must switch
|
||||
// to the fallback writer decision mechanism for consistency
|
||||
if (std::ranges::any_of(clusterData, [](ClioNode const& node) {
|
||||
return node.dbRole == ClioNode::DbRole::Fallback;
|
||||
})) {
|
||||
writerState->setWriterDecidingFallback();
|
||||
return;
|
||||
}
|
||||
|
||||
// We are not ReadOnly and there is no Fallback in the cluster
|
||||
std::ranges::sort(clusterData, [](ClioNode const& lhs, ClioNode const& rhs) {
|
||||
return *lhs.uuid < *rhs.uuid;
|
||||
});
|
||||
|
||||
auto const it = std::ranges::find_if(clusterData, [](ClioNode const& node) {
|
||||
return node.dbRole == ClioNode::DbRole::NotWriter or node.dbRole == ClioNode::DbRole::Writer;
|
||||
});
|
||||
|
||||
if (it == clusterData.end()) {
|
||||
// No writer nodes in the cluster yet
|
||||
return;
|
||||
}
|
||||
|
||||
if (*it->uuid == *selfId) {
|
||||
writerState->startWriting();
|
||||
} else {
|
||||
writerState->giveUpWriting();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
} // namespace cluster
|
||||
75
src/cluster/WriterDecider.hpp
Normal file
75
src/cluster/WriterDecider.hpp
Normal file
@@ -0,0 +1,75 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "etl/WriterState.hpp"
|
||||
|
||||
#include <boost/asio/thread_pool.hpp>
|
||||
|
||||
#include <memory>
|
||||
|
||||
namespace cluster {
|
||||
|
||||
/**
|
||||
* @brief Decides which node in the cluster should be the writer based on cluster state.
|
||||
*
|
||||
* This class monitors cluster state changes and determines whether the current node
|
||||
* should act as the writer to the database. The decision is made by:
|
||||
* 1. Sorting all nodes by UUID for deterministic ordering
|
||||
* 2. Selecting the first node that is allowed to write (not ReadOnly)
|
||||
* 3. Activating writing on this node if it's the current node, otherwise deactivating
|
||||
*
|
||||
* This ensures only one node in the cluster actively writes to the database at a time.
|
||||
*/
|
||||
class WriterDecider {
|
||||
/** @brief Thread pool for spawning asynchronous tasks */
|
||||
boost::asio::thread_pool& ctx_;
|
||||
|
||||
/** @brief Interface for controlling the writer state of this node */
|
||||
std::unique_ptr<etl::WriterStateInterface> writerState_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Constructs a WriterDecider.
|
||||
*
|
||||
* @param ctx Thread pool for executing asynchronous operations
|
||||
* @param writerState Writer state interface for controlling write operations
|
||||
*/
|
||||
WriterDecider(boost::asio::thread_pool& ctx, std::unique_ptr<etl::WriterStateInterface> writerState);
|
||||
|
||||
/**
|
||||
* @brief Handles cluster state changes and decides whether this node should be the writer.
|
||||
*
|
||||
* This method is called when cluster state changes. It asynchronously:
|
||||
* - Sorts all nodes by UUID to establish a deterministic order
|
||||
* - Identifies the first node allowed to write (not ReadOnly)
|
||||
* - Activates writing if this node is selected, otherwise deactivates writing
|
||||
* - Logs a warning if no nodes in the cluster are allowed to write
|
||||
*
|
||||
* @param selfId The UUID of the current node
|
||||
* @param clusterData Shared pointer to current cluster data; may be empty if communication failed
|
||||
*/
|
||||
void
|
||||
onNewState(ClioNode::CUuid selfId, std::shared_ptr<Backend::ClusterData const> clusterData);
|
||||
};
|
||||
|
||||
} // namespace cluster
|
||||
104
src/cluster/impl/RepeatedTask.hpp
Normal file
104
src/cluster/impl/RepeatedTask.hpp
Normal file
@@ -0,0 +1,104 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Spawn.hpp"
|
||||
|
||||
#include <boost/asio/bind_cancellation_slot.hpp>
|
||||
#include <boost/asio/cancellation_signal.hpp>
|
||||
#include <boost/asio/cancellation_type.hpp>
|
||||
#include <boost/asio/error.hpp>
|
||||
#include <boost/asio/executor.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/steady_timer.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <boost/asio/use_future.hpp>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <concepts>
|
||||
#include <semaphore>
|
||||
|
||||
namespace cluster::impl {
|
||||
|
||||
// TODO: Try to replace util::Repeat by this. https://github.com/XRPLF/clio/issues/2926
|
||||
template <typename Context>
|
||||
class RepeatedTask {
|
||||
std::chrono::steady_clock::duration interval_;
|
||||
boost::asio::strand<typename Context::executor_type> strand_;
|
||||
|
||||
enum class State { Running, Stopped };
|
||||
std::atomic<State> state_ = State::Stopped;
|
||||
|
||||
std::binary_semaphore semaphore_{0};
|
||||
boost::asio::steady_timer timer_;
|
||||
|
||||
public:
|
||||
RepeatedTask(std::chrono::steady_clock::duration interval, Context& ctx)
|
||||
: interval_(interval), strand_(boost::asio::make_strand(ctx)), timer_(strand_)
|
||||
{
|
||||
}
|
||||
|
||||
~RepeatedTask()
|
||||
{
|
||||
stop();
|
||||
}
|
||||
|
||||
template <typename Fn>
|
||||
requires std::invocable<Fn, boost::asio::yield_context> or std::invocable<Fn>
|
||||
void
|
||||
run(Fn&& f)
|
||||
{
|
||||
ASSERT(state_ == State::Stopped, "Can only be ran once");
|
||||
state_ = State::Running;
|
||||
util::spawn(strand_, [this, f = std::forward<Fn>(f)](boost::asio::yield_context yield) {
|
||||
boost::system::error_code ec;
|
||||
|
||||
while (state_ == State::Running) {
|
||||
timer_.expires_after(interval_);
|
||||
timer_.async_wait(yield[ec]);
|
||||
|
||||
if (ec or state_ != State::Running)
|
||||
break;
|
||||
|
||||
if constexpr (std::invocable<decltype(f), boost::asio::yield_context>) {
|
||||
f(yield);
|
||||
} else {
|
||||
f();
|
||||
}
|
||||
}
|
||||
|
||||
semaphore_.release();
|
||||
});
|
||||
}
|
||||
|
||||
void
|
||||
stop()
|
||||
{
|
||||
if (auto expected = State::Running; not state_.compare_exchange_strong(expected, State::Stopped))
|
||||
return; // Already stopped or not started
|
||||
|
||||
boost::asio::spawn(strand_, [this](auto&&) { timer_.cancel(); }, boost::asio::use_future).wait();
|
||||
semaphore_.acquire();
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace cluster::impl
|
||||
@@ -115,6 +115,11 @@ LedgerCacheFile::write(DataView dataView)
|
||||
auto const hash = file.hash();
|
||||
file.write(hash.data(), decltype(hash)::bytes);
|
||||
|
||||
// flush internal buffer explicitly before renaming
|
||||
if (auto const expectedSuccess = file.close(); not expectedSuccess.has_value()) {
|
||||
return expectedSuccess;
|
||||
}
|
||||
|
||||
try {
|
||||
std::filesystem::rename(newFilePath, path_);
|
||||
} catch (std::exception const& e) {
|
||||
|
||||
@@ -23,6 +23,7 @@
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstring>
|
||||
#include <expected>
|
||||
#include <ios>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
@@ -59,4 +60,14 @@ OutputFile::hash() const
|
||||
return std::move(sum).finalize();
|
||||
}
|
||||
|
||||
std::expected<void, std::string>
|
||||
OutputFile::close()
|
||||
{
|
||||
file_.close();
|
||||
if (not file_) {
|
||||
return std::unexpected{"Error closing cache file"};
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
} // namespace data::impl
|
||||
|
||||
@@ -25,6 +25,7 @@
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstring>
|
||||
#include <expected>
|
||||
#include <fstream>
|
||||
#include <string>
|
||||
|
||||
@@ -60,6 +61,9 @@ public:
|
||||
ripple::uint256
|
||||
hash() const;
|
||||
|
||||
std::expected<void, std::string>
|
||||
close();
|
||||
|
||||
private:
|
||||
void
|
||||
writeToFile(char const* data, size_t size);
|
||||
|
||||
@@ -11,6 +11,7 @@ target_sources(
|
||||
NetworkValidatedLedgers.cpp
|
||||
NFTHelpers.cpp
|
||||
Source.cpp
|
||||
WriterState.cpp
|
||||
impl/AmendmentBlockHandler.cpp
|
||||
impl/AsyncGrpcCall.cpp
|
||||
impl/Extraction.cpp
|
||||
|
||||
@@ -78,6 +78,7 @@ namespace etl {
|
||||
std::shared_ptr<ETLServiceInterface>
|
||||
ETLService::makeETLService(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<SystemState> state,
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
@@ -87,9 +88,6 @@ ETLService::makeETLService(
|
||||
{
|
||||
std::shared_ptr<ETLServiceInterface> ret;
|
||||
|
||||
auto state = std::make_shared<SystemState>();
|
||||
state->isStrictReadonly = config.get<bool>("read_only");
|
||||
|
||||
auto fetcher = std::make_shared<impl::LedgerFetcher>(backend, balancer);
|
||||
auto extractor = std::make_shared<impl::Extractor>(fetcher);
|
||||
auto publisher = std::make_shared<impl::LedgerPublisher>(ctx, backend, subscriptions, *state);
|
||||
@@ -173,6 +171,7 @@ ETLService::ETLService(
|
||||
, state_(std::move(state))
|
||||
, startSequence_(config.get().maybeValue<uint32_t>("start_sequence"))
|
||||
, finishSequence_(config.get().maybeValue<uint32_t>("finish_sequence"))
|
||||
, writeCommandStrand_(ctx_.makeStrand())
|
||||
{
|
||||
ASSERT(not state_->isWriting, "ETL should never start in writer mode");
|
||||
|
||||
@@ -213,14 +212,13 @@ ETLService::run()
|
||||
return;
|
||||
}
|
||||
|
||||
auto nextSequence = rng->maxSequence + 1;
|
||||
if (backend_->cache().latestLedgerSequence() != 0) {
|
||||
nextSequence = backend_->cache().latestLedgerSequence();
|
||||
}
|
||||
|
||||
auto const nextSequence = syncCacheWithDb();
|
||||
LOG(log_.debug()) << "Database is populated. Starting monitor loop. sequence = " << nextSequence;
|
||||
|
||||
startMonitor(nextSequence);
|
||||
|
||||
state_->isLoadingCache = false;
|
||||
|
||||
// If we are a writer as the result of loading the initial ledger - start loading
|
||||
if (state_->isWriting)
|
||||
startLoading(nextSequence);
|
||||
@@ -232,6 +230,13 @@ ETLService::stop()
|
||||
{
|
||||
LOG(log_.info()) << "Stop called";
|
||||
|
||||
systemStateWriteCommandSubscription_.disconnect();
|
||||
auto count = runningWriteCommandHandlers_.load();
|
||||
while (count != 0) {
|
||||
runningWriteCommandHandlers_.wait(count); // Blocks until value changes
|
||||
count = runningWriteCommandHandlers_.load();
|
||||
}
|
||||
|
||||
if (mainLoop_)
|
||||
mainLoop_->wait();
|
||||
if (taskMan_)
|
||||
@@ -343,35 +348,77 @@ ETLService::loadInitialLedgerIfNeeded()
|
||||
return rng;
|
||||
}
|
||||
|
||||
uint32_t
|
||||
ETLService::syncCacheWithDb()
|
||||
{
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
|
||||
while (not backend_->cache().isDisabled() and rng->maxSequence > backend_->cache().latestLedgerSequence()) {
|
||||
LOG(log_.info()) << "Syncing cache with DB. DB latest seq: " << rng->maxSequence
|
||||
<< ". Cache latest seq: " << backend_->cache().latestLedgerSequence();
|
||||
for (auto seq = backend_->cache().latestLedgerSequence(); seq <= rng->maxSequence; ++seq) {
|
||||
LOG(log_.info()) << "ETLService (via syncCacheWithDb) got new seq from db: " << seq;
|
||||
updateCache(seq);
|
||||
}
|
||||
rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
}
|
||||
return rng->maxSequence + 1;
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::updateCache(uint32_t seq)
|
||||
{
|
||||
auto const cacheNeedsUpdate = backend_->cache().latestLedgerSequence() < seq;
|
||||
auto const backendRange = backend_->fetchLedgerRange();
|
||||
auto const backendNeedsUpdate = backendRange.has_value() and backendRange->maxSequence < seq;
|
||||
|
||||
if (cacheNeedsUpdate) {
|
||||
auto const diff = data::synchronousAndRetryOnTimeout([this, seq](auto yield) {
|
||||
return backend_->fetchLedgerDiff(seq, yield);
|
||||
});
|
||||
cacheUpdater_->update(seq, diff);
|
||||
}
|
||||
|
||||
if (backendNeedsUpdate)
|
||||
backend_->updateRange(seq);
|
||||
|
||||
publisher_->publish(seq, {});
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::startMonitor(uint32_t seq)
|
||||
{
|
||||
monitor_ = monitorProvider_->make(ctx_, backend_, ledgers_, seq);
|
||||
|
||||
systemStateWriteCommandSubscription_ =
|
||||
state_->writeCommandSignal.connect([this](SystemState::WriteCommand command) {
|
||||
++runningWriteCommandHandlers_;
|
||||
writeCommandStrand_.submit([this, command]() {
|
||||
switch (command) {
|
||||
case etl::SystemState::WriteCommand::StartWriting:
|
||||
attemptTakeoverWriter();
|
||||
break;
|
||||
case etl::SystemState::WriteCommand::StopWriting:
|
||||
giveUpWriter();
|
||||
break;
|
||||
}
|
||||
--runningWriteCommandHandlers_;
|
||||
runningWriteCommandHandlers_.notify_one();
|
||||
});
|
||||
});
|
||||
|
||||
monitorNewSeqSubscription_ = monitor_->subscribeToNewSequence([this](uint32_t seq) {
|
||||
LOG(log_.info()) << "ETLService (via Monitor) got new seq from db: " << seq;
|
||||
|
||||
if (state_->writeConflict) {
|
||||
LOG(log_.info()) << "Got a write conflict; Giving up writer seat immediately";
|
||||
giveUpWriter();
|
||||
}
|
||||
|
||||
if (not state_->isWriting) {
|
||||
auto const diff = data::synchronousAndRetryOnTimeout([this, seq](auto yield) {
|
||||
return backend_->fetchLedgerDiff(seq, yield);
|
||||
});
|
||||
|
||||
cacheUpdater_->update(seq, diff);
|
||||
backend_->updateRange(seq);
|
||||
}
|
||||
|
||||
publisher_->publish(seq, {});
|
||||
updateCache(seq);
|
||||
});
|
||||
|
||||
monitorDbStalledSubscription_ = monitor_->subscribeToDbStalled([this]() {
|
||||
LOG(log_.warn()) << "ETLService received DbStalled signal from Monitor";
|
||||
// Database stall detected - no writer has been active for 10 seconds
|
||||
// This triggers the fallback mechanism and attempts to become the writer
|
||||
if (not state_->isStrictReadonly and not state_->isWriting)
|
||||
attemptTakeoverWriter();
|
||||
state_->writeCommandSignal(SystemState::WriteCommand::StartWriting);
|
||||
state_->isWriterDecidingFallback = true;
|
||||
});
|
||||
|
||||
monitor_->run();
|
||||
@@ -394,6 +441,13 @@ ETLService::attemptTakeoverWriter()
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
ASSERT(rng.has_value(), "Ledger range can't be null");
|
||||
|
||||
if (backend_->cache().latestLedgerSequence() != rng->maxSequence) {
|
||||
LOG(log_.info()) << "Wanted to take over the ETL writer seat but LedgerCache is outdated";
|
||||
// Give ETL time to update LedgerCache. This method will be called because ClusterCommunication will likely to
|
||||
// continue sending StartWriting signal every 1 second
|
||||
return;
|
||||
}
|
||||
|
||||
state_->isWriting = true; // switch to writer
|
||||
LOG(log_.info()) << "Taking over the ETL writer seat";
|
||||
startLoading(rng->maxSequence + 1);
|
||||
@@ -404,7 +458,7 @@ ETLService::giveUpWriter()
|
||||
{
|
||||
ASSERT(not state_->isStrictReadonly, "This should only happen on writer nodes");
|
||||
state_->isWriting = false;
|
||||
state_->writeConflict = false;
|
||||
LOG(log_.info()) << "Giving up writer seat";
|
||||
taskMan_ = nullptr;
|
||||
}
|
||||
|
||||
|
||||
@@ -52,6 +52,7 @@
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/async/AnyOperation.hpp"
|
||||
#include "util/async/AnyStrand.hpp"
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
@@ -69,12 +70,12 @@
|
||||
#include <xrpl/protocol/TxFormats.h>
|
||||
#include <xrpl/protocol/TxMeta.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
|
||||
namespace etl {
|
||||
|
||||
@@ -117,6 +118,9 @@ class ETLService : public ETLServiceInterface {
|
||||
|
||||
boost::signals2::scoped_connection monitorNewSeqSubscription_;
|
||||
boost::signals2::scoped_connection monitorDbStalledSubscription_;
|
||||
boost::signals2::scoped_connection systemStateWriteCommandSubscription_;
|
||||
util::async::AnyStrand writeCommandStrand_;
|
||||
std::atomic<size_t> runningWriteCommandHandlers_{0};
|
||||
|
||||
std::optional<util::async::AnyOperation<void>> mainLoop_;
|
||||
|
||||
@@ -127,6 +131,7 @@ public:
|
||||
* Creates and runs the ETL service.
|
||||
*
|
||||
* @param config The configuration to use
|
||||
* @param state The system state tracking object
|
||||
* @param ctx Execution context for asynchronous operations
|
||||
* @param backend BackendInterface implementation
|
||||
* @param subscriptions Subscription manager
|
||||
@@ -137,6 +142,7 @@ public:
|
||||
static std::shared_ptr<ETLServiceInterface>
|
||||
makeETLService(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<SystemState> state,
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
@@ -160,7 +166,7 @@ public:
|
||||
* @param initialLoadObserver The observer for initial data loading
|
||||
* @param taskManagerProvider The provider of the task manager instance
|
||||
* @param monitorProvider The provider of the monitor instance
|
||||
* @param state System state tracking object
|
||||
* @param state The system state tracking object
|
||||
*/
|
||||
ETLService(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
@@ -206,6 +212,12 @@ private:
|
||||
std::optional<data::LedgerRange>
|
||||
loadInitialLedgerIfNeeded();
|
||||
|
||||
[[nodiscard]] uint32_t
|
||||
syncCacheWithDb();
|
||||
|
||||
void
|
||||
updateCache(uint32_t seq);
|
||||
|
||||
void
|
||||
startMonitor(uint32_t seq);
|
||||
|
||||
|
||||
@@ -19,11 +19,16 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/prometheus/Bool.hpp"
|
||||
#include "util/prometheus/Label.hpp"
|
||||
#include "util/prometheus/Prometheus.hpp"
|
||||
|
||||
#include <atomic>
|
||||
#include <boost/signals2/signal.hpp>
|
||||
#include <boost/signals2/variadic_signal.hpp>
|
||||
|
||||
#include <memory>
|
||||
|
||||
namespace etl {
|
||||
|
||||
@@ -31,6 +36,25 @@ namespace etl {
|
||||
* @brief Represents the state of the ETL subsystem.
|
||||
*/
|
||||
struct SystemState {
|
||||
SystemState()
|
||||
{
|
||||
isLoadingCache = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Factory method to create a SystemState instance.
|
||||
*
|
||||
* @param config The configuration to use for initializing the system state
|
||||
* @return A shared pointer to the newly created SystemState
|
||||
*/
|
||||
static std::shared_ptr<SystemState>
|
||||
makeSystemState(util::config::ClioConfigDefinition const& config)
|
||||
{
|
||||
auto state = std::make_shared<SystemState>();
|
||||
state->isStrictReadonly = config.get<bool>("read_only");
|
||||
return state;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Whether the process is in strict read-only mode.
|
||||
*
|
||||
@@ -50,8 +74,31 @@ struct SystemState {
|
||||
"Whether the process is writing to the database"
|
||||
);
|
||||
|
||||
std::atomic_bool isStopping = false; /**< @brief Whether the software is stopping. */
|
||||
std::atomic_bool writeConflict = false; /**< @brief Whether a write conflict was detected. */
|
||||
/** @brief Whether the process is still loading cache after startup. */
|
||||
util::prometheus::Bool isLoadingCache = PrometheusService::boolMetric(
|
||||
"etl_loading_cache",
|
||||
util::prometheus::Labels{},
|
||||
"Whether etl is loading cache after clio startup"
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Commands for controlling the ETL writer state.
|
||||
*
|
||||
* These commands are emitted via writeCommandSignal to coordinate writer state transitions across components.
|
||||
*/
|
||||
enum class WriteCommand {
|
||||
StartWriting, /**< Request to attempt taking over as the ETL writer */
|
||||
StopWriting /**< Request to give up the ETL writer role (e.g., due to write conflict) */
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Signal for coordinating ETL writer state transitions.
|
||||
*
|
||||
* This signal allows components to request changes to the writer state without direct coupling.
|
||||
* - Emitted with StartWriting when database stalls and node should attempt to become writer
|
||||
* - Emitted with StopWriting when write conflicts are detected
|
||||
*/
|
||||
boost::signals2::signal<void(WriteCommand)> writeCommandSignal;
|
||||
|
||||
/**
|
||||
* @brief Whether clio detected an amendment block.
|
||||
@@ -77,6 +124,24 @@ struct SystemState {
|
||||
util::prometheus::Labels{},
|
||||
"Whether clio detected a corruption that needs manual attention"
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Whether the cluster is using the fallback writer decision mechanism.
|
||||
*
|
||||
* The fallback mechanism is triggered when:
|
||||
* - The database stalls for 10 seconds (detected by Monitor), indicating no active writer
|
||||
* - A write conflict is detected, indicating multiple nodes attempting to write simultaneously
|
||||
*
|
||||
* When fallback mode is active, the cluster stops using the cluster communication mechanism
|
||||
* (TTL-based role announcements) and relies on the slower but more reliable database-based
|
||||
* conflict detection. This flag propagates across the cluster - if any node enters fallback
|
||||
* mode, all nodes in the cluster will switch to fallback mode.
|
||||
*/
|
||||
util::prometheus::Bool isWriterDecidingFallback = PrometheusService::boolMetric(
|
||||
"etl_writing_deciding_fallback",
|
||||
util::prometheus::Labels{},
|
||||
"Whether the cluster is using the fallback writer decision mechanism"
|
||||
);
|
||||
};
|
||||
|
||||
} // namespace etl
|
||||
|
||||
88
src/etl/WriterState.cpp
Normal file
88
src/etl/WriterState.cpp
Normal file
@@ -0,0 +1,88 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etl/WriterState.hpp"
|
||||
|
||||
#include "etl/SystemState.hpp"
|
||||
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace etl {
|
||||
|
||||
WriterState::WriterState(std::shared_ptr<SystemState> state) : systemState_(std::move(state))
|
||||
{
|
||||
}
|
||||
|
||||
bool
|
||||
WriterState::isReadOnly() const
|
||||
{
|
||||
return systemState_->isStrictReadonly;
|
||||
}
|
||||
|
||||
bool
|
||||
WriterState::isWriting() const
|
||||
{
|
||||
return systemState_->isWriting;
|
||||
}
|
||||
|
||||
void
|
||||
WriterState::startWriting()
|
||||
{
|
||||
if (isWriting())
|
||||
return;
|
||||
|
||||
systemState_->writeCommandSignal(SystemState::WriteCommand::StartWriting);
|
||||
}
|
||||
|
||||
void
|
||||
WriterState::giveUpWriting()
|
||||
{
|
||||
if (not isWriting())
|
||||
return;
|
||||
|
||||
systemState_->writeCommandSignal(SystemState::WriteCommand::StopWriting);
|
||||
}
|
||||
|
||||
void
|
||||
WriterState::setWriterDecidingFallback()
|
||||
{
|
||||
systemState_->isWriterDecidingFallback = true;
|
||||
}
|
||||
|
||||
bool
|
||||
WriterState::isFallback() const
|
||||
{
|
||||
return systemState_->isWriterDecidingFallback;
|
||||
}
|
||||
|
||||
bool
|
||||
WriterState::isLoadingCache() const
|
||||
{
|
||||
return systemState_->isLoadingCache;
|
||||
}
|
||||
|
||||
std::unique_ptr<WriterStateInterface>
|
||||
WriterState::clone() const
|
||||
{
|
||||
auto c = WriterState(*this);
|
||||
return std::make_unique<WriterState>(std::move(c));
|
||||
}
|
||||
|
||||
} // namespace etl
|
||||
193
src/etl/WriterState.hpp
Normal file
193
src/etl/WriterState.hpp
Normal file
@@ -0,0 +1,193 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/SystemState.hpp"
|
||||
|
||||
#include <memory>
|
||||
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief Interface for managing writer state in the ETL subsystem.
|
||||
*
|
||||
* This interface provides methods to query and control whether the ETL process
|
||||
* is actively writing to the database. Implementations should coordinate with
|
||||
* the ETL system state to manage write responsibilities.
|
||||
*/
|
||||
class WriterStateInterface {
|
||||
public:
|
||||
virtual ~WriterStateInterface() = default;
|
||||
|
||||
/**
|
||||
* @brief Check if the ETL process is in strict read-only mode.
|
||||
* @return true if the process is in strict read-only mode, false otherwise
|
||||
*/
|
||||
[[nodiscard]] virtual bool
|
||||
isReadOnly() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Check if the ETL process is currently writing to the database.
|
||||
* @return true if the process is writing, false otherwise
|
||||
*/
|
||||
[[nodiscard]] virtual bool
|
||||
isWriting() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Request to start writing to the database.
|
||||
*
|
||||
* This method signals that the process should take over writing responsibilities.
|
||||
* The actual transition to writing state may not be immediate.
|
||||
*/
|
||||
virtual void
|
||||
startWriting() = 0;
|
||||
|
||||
/**
|
||||
* @brief Request to stop writing to the database.
|
||||
*
|
||||
* This method signals that the process should give up writing responsibilities.
|
||||
* The actual transition from writing state may not be immediate.
|
||||
*/
|
||||
virtual void
|
||||
giveUpWriting() = 0;
|
||||
|
||||
/**
|
||||
* @brief Check if the cluster is using the fallback writer decision mechanism.
|
||||
*
|
||||
* @return true if the cluster has switched to fallback mode, false otherwise
|
||||
*/
|
||||
[[nodiscard]] virtual bool
|
||||
isFallback() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Switch the cluster to the fallback writer decision mechanism.
|
||||
*
|
||||
* This method is called when the cluster needs to transition from the cluster
|
||||
* communication mechanism to the slower but more reliable fallback mechanism.
|
||||
* Once set, this flag propagates to all nodes in the cluster through the
|
||||
* ClioNode DbRole::Fallback state.
|
||||
*/
|
||||
virtual void
|
||||
setWriterDecidingFallback() = 0;
|
||||
|
||||
/**
|
||||
* @brief Whether clio is still loading cache after startup.
|
||||
*
|
||||
* @return true if clio is still loading cache, false otherwise.
|
||||
*/
|
||||
[[nodiscard]] virtual bool
|
||||
isLoadingCache() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Create a clone of this writer state.
|
||||
*
|
||||
* Creates a new instance of the writer state with the same underlying system state.
|
||||
* This is used when spawning operations that need their own writer state instance
|
||||
* while sharing the same system state.
|
||||
*
|
||||
* @return A unique pointer to the cloned writer state.
|
||||
*/
|
||||
[[nodiscard]] virtual std::unique_ptr<WriterStateInterface>
|
||||
clone() const = 0;
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Implementation of WriterStateInterface that manages ETL writer state.
|
||||
*
|
||||
* This class coordinates with SystemState to manage whether the ETL process
|
||||
* is actively writing to the database. It provides methods to query the current
|
||||
* writing state and request transitions between writing and non-writing states.
|
||||
*/
|
||||
class WriterState : public WriterStateInterface {
|
||||
private:
|
||||
std::shared_ptr<SystemState> systemState_; /**< @brief Shared system state for ETL coordination */
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Construct a WriterState with the given system state.
|
||||
* @param state Shared pointer to the system state for coordination
|
||||
*/
|
||||
WriterState(std::shared_ptr<SystemState> state);
|
||||
|
||||
bool
|
||||
isReadOnly() const override;
|
||||
|
||||
/**
|
||||
* @brief Check if the ETL process is currently writing to the database.
|
||||
* @return true if the process is writing, false otherwise
|
||||
*/
|
||||
bool
|
||||
isWriting() const override;
|
||||
|
||||
/**
|
||||
* @brief Request to start writing to the database.
|
||||
*
|
||||
* If already writing, this method does nothing. Otherwise, it sets the
|
||||
* shouldTakeoverWriting flag in the system state to signal the request.
|
||||
*/
|
||||
void
|
||||
startWriting() override;
|
||||
|
||||
/**
|
||||
* @brief Request to stop writing to the database.
|
||||
*
|
||||
* If not currently writing, this method does nothing. Otherwise, it sets the
|
||||
* shouldGiveUpWriter flag in the system state to signal the request.
|
||||
*/
|
||||
void
|
||||
giveUpWriting() override;
|
||||
|
||||
/**
|
||||
* @brief Switch the cluster to the fallback writer decision mechanism.
|
||||
*
|
||||
* Sets the isWriterDecidingFallback flag in the system state, which will be
|
||||
* propagated to other nodes in the cluster through the ClioNode DbRole::Fallback state.
|
||||
*/
|
||||
void
|
||||
setWriterDecidingFallback() override;
|
||||
|
||||
/**
|
||||
* @brief Check if the cluster is using the fallback writer decision mechanism.
|
||||
*
|
||||
* @return true if the cluster has switched to fallback mode, false otherwise
|
||||
*/
|
||||
bool
|
||||
isFallback() const override;
|
||||
|
||||
/**
|
||||
* @brief Whether clio is still loading cache after startup.
|
||||
*
|
||||
* @return true if clio is still loading cache, false otherwise.
|
||||
*/
|
||||
bool
|
||||
isLoadingCache() const override;
|
||||
|
||||
/**
|
||||
* @brief Create a clone of this writer state.
|
||||
*
|
||||
* Creates a new WriterState instance sharing the same system state.
|
||||
*
|
||||
* @return A unique pointer to the cloned writer state.
|
||||
*/
|
||||
std::unique_ptr<WriterStateInterface>
|
||||
clone() const override;
|
||||
};
|
||||
|
||||
} // namespace etl
|
||||
@@ -45,6 +45,7 @@
|
||||
#include <xrpl/protocol/Serializer.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
@@ -76,6 +77,8 @@ class LedgerPublisher : public LedgerPublisherInterface {
|
||||
|
||||
util::async::AnyStrand publishStrand_;
|
||||
|
||||
std::atomic_bool stop_{false};
|
||||
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions_;
|
||||
std::reference_wrapper<SystemState const> state_; // shared state for ETL
|
||||
@@ -125,7 +128,7 @@ public:
|
||||
{
|
||||
LOG(log_.info()) << "Attempting to publish ledger = " << ledgerSequence;
|
||||
size_t numAttempts = 0;
|
||||
while (not state_.get().isStopping) {
|
||||
while (not stop_) {
|
||||
auto range = backend_->hardFetchLedgerRangeNoThrow();
|
||||
|
||||
if (!range || range->maxSequence < ledgerSequence) {
|
||||
@@ -258,6 +261,18 @@ public:
|
||||
return *lastPublishedSequence_.lock();
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Stops publishing
|
||||
*
|
||||
* @note This is a basic implementation to satisfy tests. This will be improved in
|
||||
* https://github.com/XRPLF/clio/issues/2833
|
||||
*/
|
||||
void
|
||||
stop()
|
||||
{
|
||||
stop_ = true;
|
||||
}
|
||||
|
||||
private:
|
||||
void
|
||||
setLastClose(std::chrono::time_point<ripple::NetClock> lastCloseTime)
|
||||
|
||||
@@ -75,7 +75,10 @@ Loader::load(model::LedgerData const& data)
|
||||
<< "; took " << duration << "ms";
|
||||
|
||||
if (not success) {
|
||||
state_->writeConflict = true;
|
||||
// Write conflict detected - another node wrote to the database
|
||||
// This triggers the fallback mechanism and stops this node from writing
|
||||
state_->writeCommandSignal(SystemState::WriteCommand::StopWriting);
|
||||
state_->isWriterDecidingFallback = true;
|
||||
LOG(log_.warn()) << "Another node wrote a ledger into the DB - we have a write conflict";
|
||||
return std::unexpected(LoaderError::WriteConflict);
|
||||
}
|
||||
|
||||
@@ -19,6 +19,8 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "util/async/Concepts.hpp"
|
||||
|
||||
#include <boost/asio/any_io_executor.hpp>
|
||||
#include <boost/asio/experimental/channel.hpp>
|
||||
#include <boost/asio/experimental/concurrent_channel.hpp>
|
||||
@@ -42,15 +44,36 @@ struct ChannelInstantiated;
|
||||
} // namespace detail
|
||||
#endif
|
||||
|
||||
/**
|
||||
* @brief Specifies the producer concurrency model for a Channel.
|
||||
*/
|
||||
enum class ProducerType {
|
||||
Single, /**< Only one Sender can exist (non-copyable). Uses direct Guard ownership for zero overhead. */
|
||||
Multi /**< Multiple Senders can exist (copyable). Uses shared_ptr<Guard> for shared ownership. */
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Specifies the consumer concurrency model for a Channel.
|
||||
*/
|
||||
enum class ConsumerType {
|
||||
Single, /**< Only one Receiver can exist (non-copyable). Uses direct Guard ownership for zero overhead. */
|
||||
Multi /**< Multiple Receivers can exist (copyable). Uses shared_ptr<Guard> for shared ownership. */
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Represents a go-like channel, a multi-producer (Sender) multi-consumer (Receiver) thread-safe data pipe.
|
||||
* @note Use INSTANTIATE_CHANNEL_FOR_CLANG macro when using this class. See docs at the bottom of the file for more
|
||||
* details.
|
||||
*
|
||||
* @tparam T The type of data the channel transfers
|
||||
* @tparam P ProducerType::Multi (default) for multi-producer or ProducerType::Single for single-producer
|
||||
* @tparam C ConsumerType::Multi (default) for multi-consumer or ConsumerType::Single for single-consumer
|
||||
*/
|
||||
template <typename T>
|
||||
template <typename T, ProducerType P = ProducerType::Multi, ConsumerType C = ConsumerType::Multi>
|
||||
class Channel {
|
||||
static constexpr bool kIS_MULTI_PRODUCER = (P == ProducerType::Multi);
|
||||
static constexpr bool kIS_MULTI_CONSUMER = (C == ConsumerType::Multi);
|
||||
|
||||
private:
|
||||
class ControlBlock {
|
||||
using InternalChannelType = boost::asio::experimental::concurrent_channel<void(boost::system::error_code, T)>;
|
||||
@@ -58,7 +81,16 @@ private:
|
||||
InternalChannelType ch_;
|
||||
|
||||
public:
|
||||
ControlBlock(auto&& context, std::size_t capacity) : executor_(context.get_executor()), ch_(context, capacity)
|
||||
template <typename ContextType>
|
||||
requires(not async::SomeExecutionContext<ContextType>)
|
||||
ControlBlock(ContextType&& context, std::size_t capacity)
|
||||
: executor_(context.get_executor()), ch_(context, capacity)
|
||||
{
|
||||
}
|
||||
|
||||
template <async::SomeExecutionContext ContextType>
|
||||
ControlBlock(ContextType&& context, std::size_t capacity)
|
||||
: executor_(context.getExecutor().get_executor()), ch_(context.getExecutor(), capacity)
|
||||
{
|
||||
}
|
||||
|
||||
@@ -101,30 +133,54 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief The sending end of a channel.
|
||||
*
|
||||
* Sender is copyable and movable. The channel remains open as long as at least one Sender exists.
|
||||
* Sender is movable. For multi-producer channels, Sender is also copyable.
|
||||
* The channel remains open as long as at least one Sender exists.
|
||||
* When all Sender instances are destroyed, the channel is closed and receivers will receive std::nullopt.
|
||||
*/
|
||||
class Sender {
|
||||
std::shared_ptr<ControlBlock> shared_;
|
||||
std::shared_ptr<Guard> guard_;
|
||||
std::conditional_t<kIS_MULTI_PRODUCER, std::shared_ptr<Guard>, Guard> guard_;
|
||||
|
||||
friend class Channel<T, P, C>;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Constructs a Sender from a shared control block.
|
||||
* @param shared The shared control block managing the channel state
|
||||
*/
|
||||
Sender(std::shared_ptr<ControlBlock> shared)
|
||||
: shared_(std::move(shared)), guard_(std::make_shared<Guard>(shared_)) {};
|
||||
: shared_(shared), guard_([shared = std::move(shared)]() {
|
||||
if constexpr (kIS_MULTI_PRODUCER) {
|
||||
return std::make_shared<Guard>(std::move(shared));
|
||||
} else {
|
||||
return Guard{std::move(shared)};
|
||||
}
|
||||
}())
|
||||
{
|
||||
}
|
||||
|
||||
public:
|
||||
Sender(Sender&&) = default;
|
||||
Sender(Sender const&) = default;
|
||||
Sender(Sender const&)
|
||||
requires kIS_MULTI_PRODUCER
|
||||
= default;
|
||||
Sender(Sender const&)
|
||||
requires(!kIS_MULTI_PRODUCER)
|
||||
= delete;
|
||||
|
||||
Sender&
|
||||
operator=(Sender&&) = default;
|
||||
Sender&
|
||||
operator=(Sender const&) = default;
|
||||
operator=(Sender const&)
|
||||
requires kIS_MULTI_PRODUCER
|
||||
= default;
|
||||
Sender&
|
||||
operator=(Sender const&)
|
||||
requires(!kIS_MULTI_PRODUCER)
|
||||
= delete;
|
||||
|
||||
/**
|
||||
* @brief Asynchronously sends data through the channel using a coroutine.
|
||||
@@ -202,27 +258,50 @@ private:
|
||||
/**
|
||||
* @brief The receiving end of a channel.
|
||||
*
|
||||
* Receiver is copyable and movable. Multiple receivers can consume from the same channel concurrently.
|
||||
* Receiver is movable. For multi-consumer channels, Receiver is also copyable.
|
||||
* Multiple receivers can consume from the same multi-consumer channel concurrently.
|
||||
* When all Receiver instances are destroyed, the channel is closed and senders will fail to send.
|
||||
*/
|
||||
class Receiver {
|
||||
std::shared_ptr<ControlBlock> shared_;
|
||||
std::shared_ptr<Guard> guard_;
|
||||
std::conditional_t<kIS_MULTI_CONSUMER, std::shared_ptr<Guard>, Guard> guard_;
|
||||
|
||||
friend class Channel<T, P, C>;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Constructs a Receiver from a shared control block.
|
||||
* @param shared The shared control block managing the channel state
|
||||
*/
|
||||
Receiver(std::shared_ptr<ControlBlock> shared)
|
||||
: shared_(std::move(shared)), guard_(std::make_shared<Guard>(shared_)) {};
|
||||
: shared_(shared), guard_([shared = std::move(shared)]() {
|
||||
if constexpr (kIS_MULTI_CONSUMER) {
|
||||
return std::make_shared<Guard>(std::move(shared));
|
||||
} else {
|
||||
return Guard{std::move(shared)};
|
||||
}
|
||||
}())
|
||||
{
|
||||
}
|
||||
|
||||
public:
|
||||
Receiver(Receiver&&) = default;
|
||||
Receiver(Receiver const&) = default;
|
||||
Receiver(Receiver const&)
|
||||
requires kIS_MULTI_CONSUMER
|
||||
= default;
|
||||
Receiver(Receiver const&)
|
||||
requires(!kIS_MULTI_CONSUMER)
|
||||
= delete;
|
||||
|
||||
Receiver&
|
||||
operator=(Receiver&&) = default;
|
||||
Receiver&
|
||||
operator=(Receiver const&) = default;
|
||||
operator=(Receiver const&)
|
||||
requires kIS_MULTI_CONSUMER
|
||||
= default;
|
||||
Receiver&
|
||||
operator=(Receiver const&)
|
||||
requires(!kIS_MULTI_CONSUMER)
|
||||
= delete;
|
||||
|
||||
/**
|
||||
* @brief Attempts to receive data from the channel without blocking.
|
||||
@@ -297,7 +376,6 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Factory function to create channel components.
|
||||
* @param context A supported context type (either io_context or thread_pool)
|
||||
|
||||
@@ -22,6 +22,7 @@
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
|
||||
#include <concepts>
|
||||
#include <exception>
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
@@ -29,6 +29,27 @@
|
||||
|
||||
namespace util::async {
|
||||
|
||||
/**
|
||||
* @brief Tag type for identifying execution context types.
|
||||
*
|
||||
* Types that inherit from this tag can be detected using the SomeExecutionContext concept.
|
||||
* This allows generic code to differentiate between raw Boost.Asio contexts and wrapped execution contexts.
|
||||
*/
|
||||
struct ExecutionContextTag {
|
||||
virtual ~ExecutionContextTag() = default;
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Concept that identifies types derived from ExecutionContextTag.
|
||||
*
|
||||
* This concept is used to detect custom execution context wrappers (like BasicExecutionContext)
|
||||
* and distinguish them from raw Boost.Asio contexts (io_context, thread_pool, etc.).
|
||||
*
|
||||
* @tparam T The type to check
|
||||
*/
|
||||
template <typename T>
|
||||
concept SomeExecutionContext = std::derived_from<std::remove_cvref_t<T>, ExecutionContextTag>;
|
||||
|
||||
/**
|
||||
* @brief Specifies the interface for an entity that can be stopped
|
||||
*/
|
||||
|
||||
@@ -129,7 +129,7 @@ template <
|
||||
typename DispatcherType,
|
||||
typename TimerContextProvider = impl::SelfContextProvider,
|
||||
typename ErrorHandlerType = impl::DefaultErrorHandler>
|
||||
class BasicExecutionContext {
|
||||
class BasicExecutionContext : public ExecutionContextTag {
|
||||
ContextType context_;
|
||||
|
||||
/** @cond */
|
||||
@@ -182,7 +182,7 @@ public:
|
||||
/**
|
||||
* @brief Stops the underlying thread pool.
|
||||
*/
|
||||
~BasicExecutionContext()
|
||||
~BasicExecutionContext() override
|
||||
{
|
||||
stop();
|
||||
}
|
||||
@@ -402,6 +402,20 @@ public:
|
||||
{
|
||||
context_.join();
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get the underlying executor.
|
||||
*
|
||||
* Provides access to the wrapped executor for cases where the execution context
|
||||
* needs to interact with components that require explicit executor access (like Channel).
|
||||
*
|
||||
* @return Reference to the underlying executor
|
||||
*/
|
||||
typename ContextType::Executor&
|
||||
getExecutor()
|
||||
{
|
||||
return context_.getExecutor();
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
|
||||
@@ -36,17 +36,26 @@ struct SpawnDispatchStrategy {
|
||||
{
|
||||
auto op = outcome.getOperation();
|
||||
|
||||
util::spawn(
|
||||
ctx.getExecutor(),
|
||||
[outcome = std::forward<OutcomeType>(outcome), fn = std::forward<FnType>(fn)](auto yield) mutable {
|
||||
if constexpr (SomeStoppableOutcome<OutcomeType>) {
|
||||
auto& stopSource = outcome.getStopSource();
|
||||
std::invoke(std::forward<decltype(fn)>(fn), outcome, stopSource, stopSource[yield]);
|
||||
} else {
|
||||
if constexpr (SomeStoppableOutcome<OutcomeType>) {
|
||||
util::spawn(
|
||||
ctx.getExecutor(),
|
||||
[outcome = std::forward<OutcomeType>(outcome), fn = std::forward<FnType>(fn)](auto yield) mutable {
|
||||
if constexpr (SomeStoppableOutcome<OutcomeType>) {
|
||||
auto& stopSource = outcome.getStopSource();
|
||||
std::invoke(std::forward<decltype(fn)>(fn), outcome, stopSource, stopSource[yield]);
|
||||
} else {
|
||||
std::invoke(std::forward<decltype(fn)>(fn), outcome);
|
||||
}
|
||||
}
|
||||
);
|
||||
} else {
|
||||
boost::asio::post(
|
||||
ctx.getExecutor(),
|
||||
[outcome = std::forward<OutcomeType>(outcome), fn = std::forward<FnType>(fn)]() mutable {
|
||||
std::invoke(std::forward<decltype(fn)>(fn), outcome);
|
||||
}
|
||||
}
|
||||
);
|
||||
);
|
||||
}
|
||||
|
||||
return op;
|
||||
}
|
||||
@@ -55,7 +64,7 @@ struct SpawnDispatchStrategy {
|
||||
static void
|
||||
post(ContextType& ctx, FnType&& fn)
|
||||
{
|
||||
util::spawn(ctx.getExecutor(), [fn = std::forward<FnType>(fn)](auto) mutable {
|
||||
boost::asio::post(ctx.getExecutor(), [fn = std::forward<FnType>(fn)]() mutable {
|
||||
std::invoke(std::forward<decltype(fn)>(fn));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -25,6 +25,7 @@
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
|
||||
40
tests/common/util/MockWriterState.hpp
Normal file
40
tests/common/util/MockWriterState.hpp
Normal file
@@ -0,0 +1,40 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/WriterState.hpp"
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
struct MockWriterStateBase : public etl::WriterStateInterface {
|
||||
MOCK_METHOD(bool, isReadOnly, (), (const, override));
|
||||
MOCK_METHOD(bool, isWriting, (), (const, override));
|
||||
MOCK_METHOD(void, startWriting, (), (override));
|
||||
MOCK_METHOD(void, giveUpWriting, (), (override));
|
||||
MOCK_METHOD(void, setWriterDecidingFallback, (), (override));
|
||||
MOCK_METHOD(bool, isFallback, (), (const, override));
|
||||
MOCK_METHOD(bool, isLoadingCache, (), (const, override));
|
||||
MOCK_METHOD(std::unique_ptr<etl::WriterStateInterface>, clone, (), (const, override));
|
||||
};
|
||||
|
||||
using MockWriterState = testing::StrictMock<MockWriterStateBase>;
|
||||
using NiceMockWriterState = testing::NiceMock<MockWriterStateBase>;
|
||||
@@ -21,8 +21,12 @@ target_sources(
|
||||
data/impl/LedgerCacheFileTests.cpp
|
||||
data/impl/OutputFileTests.cpp
|
||||
# Cluster
|
||||
cluster/BackendTests.cpp
|
||||
cluster/ClioNodeTests.cpp
|
||||
cluster/ClusterCommunicationServiceTests.cpp
|
||||
cluster/MetricsTests.cpp
|
||||
cluster/RepeatedTaskTests.cpp
|
||||
cluster/WriterDeciderTests.cpp
|
||||
# ETL
|
||||
etl/AmendmentBlockHandlerTests.cpp
|
||||
etl/CacheLoaderSettingsTests.cpp
|
||||
@@ -46,7 +50,9 @@ target_sources(
|
||||
etl/SchedulingTests.cpp
|
||||
etl/SourceImplTests.cpp
|
||||
etl/SubscriptionSourceTests.cpp
|
||||
etl/SystemStateTests.cpp
|
||||
etl/TaskManagerTests.cpp
|
||||
etl/WriterStateTests.cpp
|
||||
etl/ext/CoreTests.cpp
|
||||
etl/ext/CacheTests.cpp
|
||||
etl/ext/MPTTests.cpp
|
||||
|
||||
@@ -17,6 +17,7 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
#include "app/Stopper.hpp"
|
||||
#include "cluster/Concepts.hpp"
|
||||
#include "util/AsioContextTestFixture.hpp"
|
||||
#include "util/MockBackend.hpp"
|
||||
#include "util/MockETLService.hpp"
|
||||
@@ -87,6 +88,10 @@ struct StopperMakeCallbackTest : util::prometheus::WithPrometheus, SyncAsioConte
|
||||
MOCK_METHOD(void, waitToFinish, ());
|
||||
};
|
||||
|
||||
struct MockClusterCommunicationService : cluster::ClusterCommunicationServiceTag {
|
||||
MOCK_METHOD(void, stop, (), ());
|
||||
};
|
||||
|
||||
protected:
|
||||
testing::StrictMock<ServerMock> serverMock_;
|
||||
testing::StrictMock<MockLoadBalancer> loadBalancerMock_;
|
||||
@@ -94,6 +99,7 @@ protected:
|
||||
testing::StrictMock<MockSubscriptionManager> subscriptionManagerMock_;
|
||||
testing::StrictMock<MockBackend> backendMock_{util::config::ClioConfigDefinition{}};
|
||||
testing::StrictMock<MockLedgerCacheSaver> cacheSaverMock_;
|
||||
testing::StrictMock<MockClusterCommunicationService> clusterCommunicationServiceMock_;
|
||||
boost::asio::io_context ioContextToStop_;
|
||||
|
||||
bool
|
||||
@@ -115,6 +121,7 @@ TEST_F(StopperMakeCallbackTest, makeCallbackTest)
|
||||
subscriptionManagerMock_,
|
||||
backendMock_,
|
||||
cacheSaverMock_,
|
||||
clusterCommunicationServiceMock_,
|
||||
ioContextToStop_
|
||||
);
|
||||
|
||||
@@ -122,6 +129,9 @@ TEST_F(StopperMakeCallbackTest, makeCallbackTest)
|
||||
EXPECT_CALL(cacheSaverMock_, save).InSequence(s1).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); });
|
||||
EXPECT_CALL(serverMock_, stop).InSequence(s1).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); });
|
||||
EXPECT_CALL(loadBalancerMock_, stop).InSequence(s2).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); });
|
||||
EXPECT_CALL(clusterCommunicationServiceMock_, stop).InSequence(s1, s2).WillOnce([this]() {
|
||||
EXPECT_FALSE(isContextStopped());
|
||||
});
|
||||
EXPECT_CALL(etlServiceMock_, stop).InSequence(s1, s2).WillOnce([this]() { EXPECT_FALSE(isContextStopped()); });
|
||||
EXPECT_CALL(subscriptionManagerMock_, stop).InSequence(s1, s2).WillOnce([this]() {
|
||||
EXPECT_FALSE(isContextStopped());
|
||||
|
||||
347
tests/unit/cluster/BackendTests.cpp
Normal file
347
tests/unit/cluster/BackendTests.cpp
Normal file
@@ -0,0 +1,347 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "util/MockBackendTestFixture.hpp"
|
||||
#include "util/MockPrometheus.hpp"
|
||||
#include "util/MockWriterState.hpp"
|
||||
|
||||
#include <boost/asio/thread_pool.hpp>
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <boost/json/value_to.hpp>
|
||||
#include <boost/uuid/random_generator.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <memory>
|
||||
#include <semaphore>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
using namespace cluster;
|
||||
|
||||
struct ClusterBackendTest : util::prometheus::WithPrometheus, MockBackendTestStrict {
|
||||
~ClusterBackendTest() override
|
||||
{
|
||||
ctx.stop();
|
||||
ctx.join();
|
||||
}
|
||||
|
||||
boost::asio::thread_pool ctx;
|
||||
std::unique_ptr<MockWriterState> writerState = std::make_unique<MockWriterState>();
|
||||
MockWriterState& writerStateRef = *writerState;
|
||||
testing::StrictMock<testing::MockFunction<void(ClioNode::CUuid, std::shared_ptr<Backend::ClusterData const>)>>
|
||||
callbackMock;
|
||||
std::binary_semaphore semaphore{0};
|
||||
|
||||
class SemaphoreReleaseGuard {
|
||||
std::binary_semaphore& semaphore_;
|
||||
|
||||
public:
|
||||
SemaphoreReleaseGuard(std::binary_semaphore& s) : semaphore_(s)
|
||||
{
|
||||
}
|
||||
~SemaphoreReleaseGuard()
|
||||
{
|
||||
semaphore_.release();
|
||||
}
|
||||
};
|
||||
};
|
||||
|
||||
TEST_F(ClusterBackendTest, SubscribeToNewState)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
clusterBackend.subscribeToNewState(callbackMock.AsStdFunction());
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly(testing::Return(BackendInterface::ClioNodesDataFetchResult{}));
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(testing::AtLeast(1));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(true));
|
||||
EXPECT_CALL(callbackMock, Call)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly([this](ClioNode::CUuid selfId, std::shared_ptr<Backend::ClusterData const> clusterData) {
|
||||
SemaphoreReleaseGuard const guard{semaphore};
|
||||
ASSERT_TRUE(clusterData->has_value());
|
||||
EXPECT_EQ(clusterData->value().size(), 1);
|
||||
auto const& nodeData = clusterData->value().front();
|
||||
EXPECT_EQ(nodeData.uuid, selfId);
|
||||
EXPECT_EQ(nodeData.dbRole, ClioNode::DbRole::ReadOnly);
|
||||
EXPECT_LE(nodeData.updateTime, std::chrono::system_clock::now());
|
||||
});
|
||||
|
||||
clusterBackend.run();
|
||||
semaphore.acquire();
|
||||
}
|
||||
|
||||
TEST_F(ClusterBackendTest, Stop)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly(testing::Return(BackendInterface::ClioNodesDataFetchResult{}));
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(testing::AtLeast(1));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(true));
|
||||
|
||||
clusterBackend.run();
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds{20});
|
||||
clusterBackend.stop();
|
||||
|
||||
testing::Mock::VerifyAndClearExpectations(&(*backend_));
|
||||
// Wait to make sure there is no new calls of mockDbBackend
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds{20});
|
||||
}
|
||||
|
||||
TEST_F(ClusterBackendTest, FetchClioNodesDataThrowsException)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
clusterBackend.subscribeToNewState(callbackMock.AsStdFunction());
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly(testing::Throw(std::runtime_error("Database connection failed")));
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(testing::AtLeast(1));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(true));
|
||||
EXPECT_CALL(callbackMock, Call)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly([this](ClioNode::CUuid, std::shared_ptr<Backend::ClusterData const> clusterData) {
|
||||
SemaphoreReleaseGuard const guard{semaphore};
|
||||
ASSERT_FALSE(clusterData->has_value());
|
||||
EXPECT_EQ(clusterData->error(), "Failed to fetch Clio nodes data");
|
||||
});
|
||||
|
||||
clusterBackend.run();
|
||||
semaphore.acquire();
|
||||
}
|
||||
|
||||
TEST_F(ClusterBackendTest, FetchClioNodesDataReturnsDataWithOtherNodes)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
clusterBackend.subscribeToNewState(callbackMock.AsStdFunction());
|
||||
|
||||
auto const otherUuid = boost::uuids::random_generator{}();
|
||||
auto const otherNodeJson = R"JSON({
|
||||
"db_role": 3,
|
||||
"update_time": "2025-01-15T10:30:00Z"
|
||||
})JSON";
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly(
|
||||
testing::Return(
|
||||
BackendInterface::ClioNodesDataFetchResult{
|
||||
std::vector<std::pair<boost::uuids::uuid, std::string>>{{otherUuid, otherNodeJson}}
|
||||
}
|
||||
)
|
||||
);
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(testing::AtLeast(1));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(writerStateRef, isFallback).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(writerStateRef, isLoadingCache).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(writerStateRef, isWriting).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(callbackMock, Call)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly([&](ClioNode::CUuid selfId, std::shared_ptr<Backend::ClusterData const> clusterData) {
|
||||
SemaphoreReleaseGuard const guard{semaphore};
|
||||
ASSERT_TRUE(clusterData->has_value()) << clusterData->error();
|
||||
EXPECT_EQ(clusterData->value().size(), 2);
|
||||
EXPECT_EQ(selfId, clusterBackend.selfId());
|
||||
|
||||
bool foundSelf = false;
|
||||
bool foundOther = false;
|
||||
|
||||
for (auto const& node : clusterData->value()) {
|
||||
if (*node.uuid == *selfId) {
|
||||
foundSelf = true;
|
||||
EXPECT_EQ(node.dbRole, ClioNode::DbRole::NotWriter);
|
||||
} else if (*node.uuid == otherUuid) {
|
||||
foundOther = true;
|
||||
EXPECT_EQ(node.dbRole, ClioNode::DbRole::Writer);
|
||||
}
|
||||
EXPECT_LE(node.updateTime, std::chrono::system_clock::now());
|
||||
}
|
||||
|
||||
EXPECT_TRUE(foundSelf);
|
||||
EXPECT_TRUE(foundOther);
|
||||
});
|
||||
|
||||
clusterBackend.run();
|
||||
semaphore.acquire();
|
||||
}
|
||||
|
||||
TEST_F(ClusterBackendTest, FetchClioNodesDataReturnsOnlySelfData)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
clusterBackend.subscribeToNewState(callbackMock.AsStdFunction());
|
||||
|
||||
auto const selfNodeJson = R"JSON({
|
||||
"db_role": 1,
|
||||
"update_time": "2025-01-16T10:30:00Z"
|
||||
})JSON";
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData).Times(testing::AtLeast(1)).WillRepeatedly([&]() {
|
||||
return BackendInterface::ClioNodesDataFetchResult{
|
||||
std::vector<std::pair<boost::uuids::uuid, std::string>>{{*clusterBackend.selfId(), selfNodeJson}}
|
||||
};
|
||||
});
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(testing::AtLeast(1));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(true));
|
||||
EXPECT_CALL(callbackMock, Call)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly([this](ClioNode::CUuid selfId, std::shared_ptr<Backend::ClusterData const> clusterData) {
|
||||
SemaphoreReleaseGuard const guard{semaphore};
|
||||
ASSERT_TRUE(clusterData->has_value());
|
||||
EXPECT_EQ(clusterData->value().size(), 1);
|
||||
auto const& nodeData = clusterData->value().front();
|
||||
EXPECT_EQ(nodeData.uuid, selfId);
|
||||
EXPECT_EQ(nodeData.dbRole, ClioNode::DbRole::ReadOnly);
|
||||
EXPECT_LE(nodeData.updateTime, std::chrono::system_clock::now());
|
||||
});
|
||||
|
||||
clusterBackend.run();
|
||||
semaphore.acquire();
|
||||
}
|
||||
|
||||
TEST_F(ClusterBackendTest, FetchClioNodesDataReturnsInvalidJson)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
clusterBackend.subscribeToNewState(callbackMock.AsStdFunction());
|
||||
|
||||
auto const otherUuid = boost::uuids::random_generator{}();
|
||||
auto const invalidJson = "{ invalid json";
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly(
|
||||
testing::Return(
|
||||
BackendInterface::ClioNodesDataFetchResult{
|
||||
std::vector<std::pair<boost::uuids::uuid, std::string>>{{otherUuid, invalidJson}}
|
||||
}
|
||||
)
|
||||
);
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(testing::AtLeast(1));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(true));
|
||||
EXPECT_CALL(callbackMock, Call)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly([this, invalidJson](ClioNode::CUuid, std::shared_ptr<Backend::ClusterData const> clusterData) {
|
||||
SemaphoreReleaseGuard const guard{semaphore};
|
||||
ASSERT_FALSE(clusterData->has_value());
|
||||
EXPECT_THAT(clusterData->error(), testing::HasSubstr("Error parsing json from DB"));
|
||||
EXPECT_THAT(clusterData->error(), testing::HasSubstr(invalidJson));
|
||||
});
|
||||
|
||||
clusterBackend.run();
|
||||
semaphore.acquire();
|
||||
}
|
||||
|
||||
TEST_F(ClusterBackendTest, FetchClioNodesDataReturnsValidJsonButCannotConvertToClioNode)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
clusterBackend.subscribeToNewState(callbackMock.AsStdFunction());
|
||||
|
||||
auto const otherUuid = boost::uuids::random_generator{}();
|
||||
// Valid JSON but missing required field 'db_role'
|
||||
auto const validJsonMissingField = R"JSON({
|
||||
"update_time": "2025-01-16T10:30:00Z"
|
||||
})JSON";
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly(
|
||||
testing::Return(
|
||||
BackendInterface::ClioNodesDataFetchResult{
|
||||
std::vector<std::pair<boost::uuids::uuid, std::string>>{{otherUuid, validJsonMissingField}}
|
||||
}
|
||||
)
|
||||
);
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(testing::AtLeast(1));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(true));
|
||||
EXPECT_CALL(callbackMock, Call)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly([this](ClioNode::CUuid, std::shared_ptr<Backend::ClusterData const> clusterData) {
|
||||
SemaphoreReleaseGuard const guard{semaphore};
|
||||
ASSERT_FALSE(clusterData->has_value());
|
||||
EXPECT_THAT(clusterData->error(), testing::HasSubstr("Error converting json to ClioNode"));
|
||||
});
|
||||
|
||||
clusterBackend.run();
|
||||
semaphore.acquire();
|
||||
}
|
||||
|
||||
TEST_F(ClusterBackendTest, WriteNodeMessageWritesSelfDataWithRecentTimestampAndDbRole)
|
||||
{
|
||||
Backend clusterBackend{
|
||||
ctx, backend_, std::move(writerState), std::chrono::milliseconds(1), std::chrono::milliseconds(1)
|
||||
};
|
||||
|
||||
auto const beforeRun = std::chrono::floor<std::chrono::seconds>(std::chrono::system_clock::now());
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly(testing::Return(BackendInterface::ClioNodesDataFetchResult{}));
|
||||
EXPECT_CALL(writerStateRef, isReadOnly).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(writerStateRef, isFallback).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(writerStateRef, isLoadingCache).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(writerStateRef, isWriting).Times(testing::AtLeast(1)).WillRepeatedly(testing::Return(false));
|
||||
EXPECT_CALL(*backend_, writeNodeMessage)
|
||||
.Times(testing::AtLeast(1))
|
||||
.WillRepeatedly([&](boost::uuids::uuid const& uuid, std::string message) {
|
||||
SemaphoreReleaseGuard const guard{semaphore};
|
||||
auto const afterWrite = std::chrono::system_clock::now();
|
||||
|
||||
EXPECT_EQ(uuid, *clusterBackend.selfId());
|
||||
auto const json = boost::json::parse(message);
|
||||
auto const node = boost::json::try_value_to<ClioNode>(json);
|
||||
ASSERT_TRUE(node.has_value());
|
||||
EXPECT_EQ(node->dbRole, ClioNode::DbRole::NotWriter);
|
||||
EXPECT_GE(node->updateTime, beforeRun);
|
||||
EXPECT_LE(node->updateTime, afterWrite);
|
||||
});
|
||||
|
||||
clusterBackend.run();
|
||||
semaphore.acquire();
|
||||
}
|
||||
@@ -18,6 +18,8 @@
|
||||
//==============================================================================
|
||||
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "util/MockWriterState.hpp"
|
||||
#include "util/NameGenerator.hpp"
|
||||
#include "util/TimeUtils.hpp"
|
||||
|
||||
#include <boost/json/object.hpp>
|
||||
@@ -26,9 +28,11 @@
|
||||
#include <boost/json/value_to.hpp>
|
||||
#include <boost/uuid/random_generator.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <ctime>
|
||||
#include <memory>
|
||||
#include <stdexcept>
|
||||
@@ -44,44 +48,44 @@ struct ClioNodeTest : testing::Test {
|
||||
|
||||
TEST_F(ClioNodeTest, Serialization)
|
||||
{
|
||||
// Create a ClioNode with test data
|
||||
ClioNode const node{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()()), .updateTime = updateTime
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()()),
|
||||
.updateTime = updateTime,
|
||||
.dbRole = ClioNode::DbRole::Writer
|
||||
};
|
||||
|
||||
// Serialize to JSON
|
||||
boost::json::value jsonValue;
|
||||
EXPECT_NO_THROW(boost::json::value_from(node, jsonValue));
|
||||
|
||||
// Verify JSON structure
|
||||
ASSERT_TRUE(jsonValue.is_object()) << jsonValue;
|
||||
auto const& obj = jsonValue.as_object();
|
||||
|
||||
// Check update_time exists and is a string
|
||||
EXPECT_TRUE(obj.contains("update_time"));
|
||||
EXPECT_TRUE(obj.at("update_time").is_string());
|
||||
|
||||
EXPECT_TRUE(obj.contains("db_role"));
|
||||
EXPECT_TRUE(obj.at("db_role").is_number());
|
||||
EXPECT_EQ(obj.at("db_role").as_int64(), static_cast<int64_t>(node.dbRole));
|
||||
}
|
||||
|
||||
TEST_F(ClioNodeTest, Deserialization)
|
||||
{
|
||||
boost::json::value const jsonValue = {{"update_time", updateTimeStr}};
|
||||
boost::json::value const jsonValue = {{"update_time", updateTimeStr}, {"db_role", 1}};
|
||||
|
||||
// Deserialize to ClioNode
|
||||
ClioNode node{.uuid = std::make_shared<boost::uuids::uuid>(), .updateTime = {}};
|
||||
EXPECT_NO_THROW(node = boost::json::value_to<ClioNode>(jsonValue));
|
||||
ClioNode node{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(), .updateTime = {}, .dbRole = ClioNode::DbRole::ReadOnly
|
||||
};
|
||||
ASSERT_NO_THROW(node = boost::json::value_to<ClioNode>(jsonValue));
|
||||
|
||||
// Verify deserialized data
|
||||
EXPECT_NE(node.uuid, nullptr);
|
||||
EXPECT_EQ(*node.uuid, boost::uuids::uuid{});
|
||||
EXPECT_EQ(node.updateTime, updateTime);
|
||||
EXPECT_EQ(node.dbRole, ClioNode::DbRole::LoadingCache);
|
||||
}
|
||||
|
||||
TEST_F(ClioNodeTest, DeserializationInvalidTime)
|
||||
{
|
||||
// Prepare an invalid time format
|
||||
boost::json::value const jsonValue{"update_time", "invalid_format"};
|
||||
|
||||
// Expect an exception during deserialization
|
||||
EXPECT_THROW(boost::json::value_to<ClioNode>(jsonValue), std::runtime_error);
|
||||
}
|
||||
|
||||
@@ -93,3 +97,145 @@ TEST_F(ClioNodeTest, DeserializationMissingTime)
|
||||
// Expect an exception
|
||||
EXPECT_THROW(boost::json::value_to<ClioNode>(jsonValue), std::runtime_error);
|
||||
}
|
||||
|
||||
struct ClioNodeDbRoleTestBundle {
|
||||
std::string testName;
|
||||
ClioNode::DbRole role;
|
||||
};
|
||||
|
||||
struct ClioNodeDbRoleTest : ClioNodeTest, testing::WithParamInterface<ClioNodeDbRoleTestBundle> {};
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
AllDbRoles,
|
||||
ClioNodeDbRoleTest,
|
||||
testing::Values(
|
||||
ClioNodeDbRoleTestBundle{.testName = "ReadOnly", .role = ClioNode::DbRole::ReadOnly},
|
||||
ClioNodeDbRoleTestBundle{.testName = "LoadingCache", .role = ClioNode::DbRole::LoadingCache},
|
||||
ClioNodeDbRoleTestBundle{.testName = "NotWriter", .role = ClioNode::DbRole::NotWriter},
|
||||
ClioNodeDbRoleTestBundle{.testName = "Writer", .role = ClioNode::DbRole::Writer},
|
||||
ClioNodeDbRoleTestBundle{.testName = "Fallback", .role = ClioNode::DbRole::Fallback}
|
||||
),
|
||||
tests::util::kNAME_GENERATOR
|
||||
);
|
||||
|
||||
TEST_P(ClioNodeDbRoleTest, Serialization)
|
||||
{
|
||||
auto const param = GetParam();
|
||||
ClioNode const node{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()()),
|
||||
.updateTime = updateTime,
|
||||
.dbRole = param.role
|
||||
};
|
||||
auto const jsonValue = boost::json::value_from(node);
|
||||
EXPECT_EQ(jsonValue.as_object().at("db_role").as_int64(), static_cast<int64_t>(param.role));
|
||||
}
|
||||
|
||||
TEST_P(ClioNodeDbRoleTest, Deserialization)
|
||||
{
|
||||
auto const param = GetParam();
|
||||
boost::json::value const jsonValue = {
|
||||
{"update_time", updateTimeStr}, {"db_role", static_cast<int64_t>(param.role)}
|
||||
};
|
||||
auto const node = boost::json::value_to<ClioNode>(jsonValue);
|
||||
EXPECT_EQ(node.dbRole, param.role);
|
||||
}
|
||||
|
||||
TEST_F(ClioNodeDbRoleTest, DeserializationInvalidDbRole)
|
||||
{
|
||||
boost::json::value const jsonValue = {{"update_time", updateTimeStr}, {"db_role", 10}};
|
||||
EXPECT_THROW(boost::json::value_to<ClioNode>(jsonValue), std::runtime_error);
|
||||
}
|
||||
|
||||
TEST_F(ClioNodeDbRoleTest, DeserializationMissingDbRole)
|
||||
{
|
||||
boost::json::value const jsonValue = {{"update_time", updateTimeStr}};
|
||||
EXPECT_THROW(boost::json::value_to<ClioNode>(jsonValue), std::runtime_error);
|
||||
}
|
||||
|
||||
struct ClioNodeFromTestBundle {
|
||||
std::string testName;
|
||||
bool readOnly;
|
||||
bool fallback;
|
||||
bool loadingCache;
|
||||
bool writing;
|
||||
ClioNode::DbRole expectedRole;
|
||||
};
|
||||
|
||||
struct ClioNodeFromTest : ClioNodeTest, testing::WithParamInterface<ClioNodeFromTestBundle> {
|
||||
std::shared_ptr<boost::uuids::uuid> uuid = std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()());
|
||||
|
||||
MockWriterState writerState;
|
||||
};
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
AllWriterStates,
|
||||
ClioNodeFromTest,
|
||||
testing::Values(
|
||||
ClioNodeFromTestBundle{
|
||||
.testName = "ReadOnly",
|
||||
.readOnly = true,
|
||||
.fallback = false,
|
||||
.loadingCache = false,
|
||||
.writing = false,
|
||||
.expectedRole = ClioNode::DbRole::ReadOnly
|
||||
},
|
||||
ClioNodeFromTestBundle{
|
||||
.testName = "Fallback",
|
||||
.readOnly = false,
|
||||
.fallback = true,
|
||||
.loadingCache = false,
|
||||
.writing = false,
|
||||
.expectedRole = ClioNode::DbRole::Fallback
|
||||
},
|
||||
ClioNodeFromTestBundle{
|
||||
.testName = "LoadingCache",
|
||||
.readOnly = false,
|
||||
.fallback = false,
|
||||
.loadingCache = true,
|
||||
.writing = false,
|
||||
.expectedRole = ClioNode::DbRole::LoadingCache
|
||||
},
|
||||
ClioNodeFromTestBundle{
|
||||
.testName = "NotWriterNotReadOnly",
|
||||
.readOnly = false,
|
||||
.fallback = false,
|
||||
.loadingCache = false,
|
||||
.writing = false,
|
||||
.expectedRole = ClioNode::DbRole::NotWriter
|
||||
},
|
||||
ClioNodeFromTestBundle{
|
||||
.testName = "Writer",
|
||||
.readOnly = false,
|
||||
.fallback = false,
|
||||
.loadingCache = false,
|
||||
.writing = true,
|
||||
.expectedRole = ClioNode::DbRole::Writer
|
||||
}
|
||||
),
|
||||
tests::util::kNAME_GENERATOR
|
||||
);
|
||||
|
||||
TEST_P(ClioNodeFromTest, FromWriterState)
|
||||
{
|
||||
auto const& param = GetParam();
|
||||
|
||||
EXPECT_CALL(writerState, isReadOnly()).WillOnce(testing::Return(param.readOnly));
|
||||
if (not param.readOnly) {
|
||||
EXPECT_CALL(writerState, isFallback()).WillOnce(testing::Return(param.fallback));
|
||||
if (not param.fallback) {
|
||||
EXPECT_CALL(writerState, isLoadingCache()).WillOnce(testing::Return(param.loadingCache));
|
||||
if (not param.loadingCache) {
|
||||
EXPECT_CALL(writerState, isWriting()).WillOnce(testing::Return(param.writing));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
auto const beforeTime = std::chrono::system_clock::now();
|
||||
auto const node = ClioNode::from(uuid, writerState);
|
||||
auto const afterTime = std::chrono::system_clock::now();
|
||||
|
||||
EXPECT_EQ(node.uuid, uuid);
|
||||
EXPECT_EQ(node.dbRole, param.expectedRole);
|
||||
EXPECT_GE(node.updateTime, beforeTime);
|
||||
EXPECT_LE(node.updateTime, afterTime);
|
||||
}
|
||||
|
||||
@@ -22,207 +22,197 @@
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "util/MockBackendTestFixture.hpp"
|
||||
#include "util/MockPrometheus.hpp"
|
||||
#include "util/TimeUtils.hpp"
|
||||
#include "util/prometheus/Bool.hpp"
|
||||
#include "util/prometheus/Gauge.hpp"
|
||||
#include "util/MockWriterState.hpp"
|
||||
#include "util/prometheus/Prometheus.hpp"
|
||||
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/serialize.hpp>
|
||||
#include <boost/json/string.hpp>
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/json/value_from.hpp>
|
||||
#include <boost/uuid/random_generator.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <boost/uuid/uuid_io.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <condition_variable>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <semaphore>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
using namespace cluster;
|
||||
|
||||
namespace {
|
||||
std::vector<ClioNode> const kOTHER_NODES_DATA = {
|
||||
ClioNode{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()()),
|
||||
.updateTime = util::systemTpFromUtcStr("2015-05-15T12:00:00Z", ClioNode::kTIME_FORMAT).value()
|
||||
},
|
||||
ClioNode{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()()),
|
||||
.updateTime = util::systemTpFromUtcStr("2015-05-15T12:00:01Z", ClioNode::kTIME_FORMAT).value()
|
||||
},
|
||||
};
|
||||
} // namespace
|
||||
struct ClusterCommunicationServiceTest : util::prometheus::WithPrometheus, MockBackendTest {
|
||||
std::unique_ptr<NiceMockWriterState> writerState = std::make_unique<NiceMockWriterState>();
|
||||
NiceMockWriterState& writerStateRef = *writerState;
|
||||
|
||||
struct ClusterCommunicationServiceTest : util::prometheus::WithPrometheus, MockBackendTestStrict {
|
||||
ClusterCommunicationService clusterCommunicationService{
|
||||
backend_,
|
||||
std::chrono::milliseconds{5},
|
||||
std::chrono::milliseconds{9}
|
||||
};
|
||||
static constexpr std::chrono::milliseconds kSHORT_INTERVAL{1};
|
||||
|
||||
util::prometheus::GaugeInt& nodesInClusterMetric = PrometheusService::gaugeInt("cluster_nodes_total_number", {});
|
||||
util::prometheus::Bool isHealthyMetric = PrometheusService::boolMetric("cluster_communication_is_healthy", {});
|
||||
|
||||
std::mutex mtx;
|
||||
std::condition_variable cv;
|
||||
|
||||
void
|
||||
notify()
|
||||
static boost::uuids::uuid
|
||||
makeUuid(uint8_t value)
|
||||
{
|
||||
std::unique_lock const lock{mtx};
|
||||
cv.notify_one();
|
||||
boost::uuids::uuid uuid{};
|
||||
std::ranges::fill(uuid, value);
|
||||
return uuid;
|
||||
}
|
||||
|
||||
void
|
||||
wait()
|
||||
static ClioNode
|
||||
makeNode(boost::uuids::uuid const& uuid, ClioNode::DbRole role)
|
||||
{
|
||||
std::unique_lock lock{mtx};
|
||||
cv.wait_until(lock, std::chrono::steady_clock::now() + std::chrono::milliseconds{100});
|
||||
return ClioNode{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(uuid),
|
||||
.updateTime = std::chrono::system_clock::now(),
|
||||
.dbRole = role
|
||||
};
|
||||
}
|
||||
|
||||
static std::string
|
||||
nodeToJson(ClioNode const& node)
|
||||
{
|
||||
boost::json::value const v = boost::json::value_from(node);
|
||||
return boost::json::serialize(v);
|
||||
}
|
||||
|
||||
ClusterCommunicationServiceTest()
|
||||
{
|
||||
ON_CALL(writerStateRef, clone()).WillByDefault(testing::Invoke([]() {
|
||||
auto state = std::make_unique<NiceMockWriterState>();
|
||||
ON_CALL(*state, isReadOnly()).WillByDefault(testing::Return(false));
|
||||
ON_CALL(*state, isWriting()).WillByDefault(testing::Return(true));
|
||||
return state;
|
||||
}));
|
||||
ON_CALL(writerStateRef, isReadOnly()).WillByDefault(testing::Return(false));
|
||||
ON_CALL(writerStateRef, isWriting()).WillByDefault(testing::Return(true));
|
||||
}
|
||||
|
||||
static bool
|
||||
waitForSignal(std::binary_semaphore& sem, std::chrono::milliseconds timeout = std::chrono::milliseconds{1000})
|
||||
{
|
||||
return sem.try_acquire_for(timeout);
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(ClusterCommunicationServiceTest, Write)
|
||||
TEST_F(ClusterCommunicationServiceTest, BackendReadsAndWritesData)
|
||||
{
|
||||
auto const selfUuid = *clusterCommunicationService.selfUuid();
|
||||
auto const otherUuid = makeUuid(0x02);
|
||||
std::binary_semaphore fetchSemaphore{0};
|
||||
std::binary_semaphore writeSemaphore{0};
|
||||
|
||||
auto const nowStr = util::systemTpToUtcStr(std::chrono::system_clock::now(), ClioNode::kTIME_FORMAT);
|
||||
auto const nowStrPrefix = nowStr.substr(0, nowStr.size() - 3);
|
||||
BackendInterface::ClioNodesDataFetchResult fetchResult{std::vector<std::pair<boost::uuids::uuid, std::string>>{
|
||||
{otherUuid, nodeToJson(makeNode(otherUuid, ClioNode::DbRole::Writer))}
|
||||
}};
|
||||
|
||||
EXPECT_CALL(*backend_, writeNodeMessage(selfUuid, testing::_)).WillOnce([&](auto&&, std::string const& jsonStr) {
|
||||
auto const jv = boost::json::parse(jsonStr);
|
||||
ASSERT_TRUE(jv.is_object());
|
||||
auto const& obj = jv.as_object();
|
||||
ASSERT_TRUE(obj.contains("update_time"));
|
||||
ASSERT_TRUE(obj.at("update_time").is_string());
|
||||
EXPECT_THAT(std::string{obj.at("update_time").as_string()}, testing::StartsWith(nowStrPrefix));
|
||||
ON_CALL(*backend_, fetchClioNodesData).WillByDefault(testing::Invoke([&](auto) {
|
||||
fetchSemaphore.release();
|
||||
return fetchResult;
|
||||
}));
|
||||
|
||||
notify();
|
||||
});
|
||||
ON_CALL(*backend_, writeNodeMessage).WillByDefault(testing::Invoke([&](auto, auto) { writeSemaphore.release(); }));
|
||||
|
||||
clusterCommunicationService.run();
|
||||
wait();
|
||||
// destructor of clusterCommunicationService calls .stop()
|
||||
ClusterCommunicationService service{backend_, std::move(writerState), kSHORT_INTERVAL, kSHORT_INTERVAL};
|
||||
|
||||
service.run();
|
||||
|
||||
EXPECT_TRUE(waitForSignal(fetchSemaphore));
|
||||
EXPECT_TRUE(waitForSignal(writeSemaphore));
|
||||
|
||||
service.stop();
|
||||
}
|
||||
|
||||
TEST_F(ClusterCommunicationServiceTest, Read_FetchFailed)
|
||||
TEST_F(ClusterCommunicationServiceTest, MetricsGetsNewStateFromBackend)
|
||||
{
|
||||
EXPECT_TRUE(isHealthyMetric);
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(2).WillOnce([](auto&&, auto&&) {}).WillOnce([this](auto&&, auto&&) {
|
||||
notify();
|
||||
});
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData).WillRepeatedly([](auto&&) { return std::unexpected{"Failed"}; });
|
||||
auto const otherUuid = makeUuid(0x02);
|
||||
std::binary_semaphore writerActionSemaphore{0};
|
||||
|
||||
clusterCommunicationService.run();
|
||||
wait();
|
||||
// call .stop() manually so that workers exit before expectations are called more times than we want
|
||||
clusterCommunicationService.stop();
|
||||
BackendInterface::ClioNodesDataFetchResult fetchResult{std::vector<std::pair<boost::uuids::uuid, std::string>>{
|
||||
{otherUuid, nodeToJson(makeNode(otherUuid, ClioNode::DbRole::Writer))}
|
||||
}};
|
||||
|
||||
EXPECT_FALSE(isHealthyMetric);
|
||||
ON_CALL(*backend_, fetchClioNodesData).WillByDefault(testing::Invoke([&](auto) { return fetchResult; }));
|
||||
|
||||
ON_CALL(writerStateRef, clone()).WillByDefault(testing::Invoke([&]() mutable {
|
||||
auto state = std::make_unique<NiceMockWriterState>();
|
||||
ON_CALL(*state, startWriting()).WillByDefault(testing::Invoke([&]() { writerActionSemaphore.release(); }));
|
||||
ON_CALL(*state, giveUpWriting()).WillByDefault(testing::Invoke([&]() { writerActionSemaphore.release(); }));
|
||||
return state;
|
||||
}));
|
||||
|
||||
auto& nodesInClusterMetric = PrometheusService::gaugeInt("cluster_nodes_total_number", {});
|
||||
auto isHealthyMetric = PrometheusService::boolMetric("cluster_communication_is_healthy", {});
|
||||
|
||||
ClusterCommunicationService service{backend_, std::move(writerState), kSHORT_INTERVAL, kSHORT_INTERVAL};
|
||||
|
||||
service.run();
|
||||
|
||||
// WriterDecider is called after metrics are updated so we could use it as a signal to stop
|
||||
EXPECT_TRUE(waitForSignal(writerActionSemaphore));
|
||||
|
||||
service.stop();
|
||||
|
||||
EXPECT_EQ(nodesInClusterMetric.value(), 2);
|
||||
EXPECT_TRUE(static_cast<bool>(isHealthyMetric));
|
||||
}
|
||||
|
||||
TEST_F(ClusterCommunicationServiceTest, Read_FetchThrew)
|
||||
TEST_F(ClusterCommunicationServiceTest, WriterDeciderCallsWriterStateMethodsAccordingly)
|
||||
{
|
||||
EXPECT_TRUE(isHealthyMetric);
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(2).WillOnce([](auto&&, auto&&) {}).WillOnce([this](auto&&, auto&&) {
|
||||
notify();
|
||||
});
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData).WillRepeatedly(testing::Throw(data::DatabaseTimeout{}));
|
||||
auto const smallerUuid = makeUuid(0x00);
|
||||
std::binary_semaphore fetchSemaphore{0};
|
||||
std::binary_semaphore writerActionSemaphore{0};
|
||||
|
||||
clusterCommunicationService.run();
|
||||
wait();
|
||||
clusterCommunicationService.stop();
|
||||
BackendInterface::ClioNodesDataFetchResult fetchResult{std::vector<std::pair<boost::uuids::uuid, std::string>>{
|
||||
{smallerUuid, nodeToJson(makeNode(smallerUuid, ClioNode::DbRole::Writer))}
|
||||
}};
|
||||
|
||||
EXPECT_FALSE(isHealthyMetric);
|
||||
EXPECT_FALSE(clusterCommunicationService.clusterData().has_value());
|
||||
ON_CALL(*backend_, fetchClioNodesData).WillByDefault(testing::Invoke([&](auto) {
|
||||
fetchSemaphore.release();
|
||||
return fetchResult;
|
||||
}));
|
||||
|
||||
ON_CALL(*backend_, writeNodeMessage).WillByDefault(testing::Return());
|
||||
|
||||
ON_CALL(writerStateRef, clone()).WillByDefault(testing::Invoke([&]() mutable {
|
||||
auto state = std::make_unique<NiceMockWriterState>();
|
||||
ON_CALL(*state, startWriting()).WillByDefault(testing::Invoke([&]() { writerActionSemaphore.release(); }));
|
||||
ON_CALL(*state, giveUpWriting()).WillByDefault(testing::Invoke([&]() { writerActionSemaphore.release(); }));
|
||||
return state;
|
||||
}));
|
||||
|
||||
ClusterCommunicationService service{backend_, std::move(writerState), kSHORT_INTERVAL, kSHORT_INTERVAL};
|
||||
|
||||
service.run();
|
||||
|
||||
EXPECT_TRUE(waitForSignal(fetchSemaphore));
|
||||
EXPECT_TRUE(waitForSignal(writerActionSemaphore));
|
||||
|
||||
service.stop();
|
||||
}
|
||||
|
||||
TEST_F(ClusterCommunicationServiceTest, Read_GotInvalidJson)
|
||||
TEST_F(ClusterCommunicationServiceTest, StopHaltsBackendOperations)
|
||||
{
|
||||
EXPECT_TRUE(isHealthyMetric);
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(2).WillOnce([](auto&&, auto&&) {}).WillOnce([this](auto&&, auto&&) {
|
||||
notify();
|
||||
});
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData).WillRepeatedly([](auto&&) {
|
||||
return std::vector<std::pair<boost::uuids::uuid, std::string>>{
|
||||
{boost::uuids::random_generator()(), "invalid json"}
|
||||
};
|
||||
});
|
||||
std::atomic<int> backendOperationsCount{0};
|
||||
std::binary_semaphore fetchSemaphore{0};
|
||||
|
||||
clusterCommunicationService.run();
|
||||
wait();
|
||||
clusterCommunicationService.stop();
|
||||
BackendInterface::ClioNodesDataFetchResult fetchResult{std::vector<std::pair<boost::uuids::uuid, std::string>>{}};
|
||||
|
||||
EXPECT_FALSE(isHealthyMetric);
|
||||
EXPECT_FALSE(clusterCommunicationService.clusterData().has_value());
|
||||
}
|
||||
|
||||
TEST_F(ClusterCommunicationServiceTest, Read_GotInvalidNodeData)
|
||||
{
|
||||
EXPECT_TRUE(isHealthyMetric);
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(2).WillOnce([](auto&&, auto&&) {}).WillOnce([this](auto&&, auto&&) {
|
||||
notify();
|
||||
});
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData).WillRepeatedly([](auto&&) {
|
||||
return std::vector<std::pair<boost::uuids::uuid, std::string>>{{boost::uuids::random_generator()(), "{}"}};
|
||||
});
|
||||
|
||||
clusterCommunicationService.run();
|
||||
wait();
|
||||
clusterCommunicationService.stop();
|
||||
|
||||
EXPECT_FALSE(isHealthyMetric);
|
||||
EXPECT_FALSE(clusterCommunicationService.clusterData().has_value());
|
||||
}
|
||||
|
||||
TEST_F(ClusterCommunicationServiceTest, Read_Success)
|
||||
{
|
||||
EXPECT_TRUE(isHealthyMetric);
|
||||
EXPECT_EQ(nodesInClusterMetric.value(), 1);
|
||||
|
||||
EXPECT_CALL(*backend_, writeNodeMessage).Times(2).WillOnce([](auto&&, auto&&) {}).WillOnce([this](auto&&, auto&&) {
|
||||
auto const clusterData = clusterCommunicationService.clusterData();
|
||||
ASSERT_TRUE(clusterData.has_value());
|
||||
ASSERT_EQ(clusterData->size(), kOTHER_NODES_DATA.size() + 1);
|
||||
for (auto const& node : kOTHER_NODES_DATA) {
|
||||
auto const it =
|
||||
std::ranges::find_if(*clusterData, [&](ClioNode const& n) { return *(n.uuid) == *(node.uuid); });
|
||||
EXPECT_NE(it, clusterData->cend()) << boost::uuids::to_string(*node.uuid);
|
||||
}
|
||||
auto const selfUuid = clusterCommunicationService.selfUuid();
|
||||
auto const it =
|
||||
std::ranges::find_if(*clusterData, [&selfUuid](ClioNode const& node) { return node.uuid == selfUuid; });
|
||||
EXPECT_NE(it, clusterData->end());
|
||||
|
||||
notify();
|
||||
});
|
||||
|
||||
EXPECT_CALL(*backend_, fetchClioNodesData).WillRepeatedly([this](auto&&) {
|
||||
auto const selfUuid = clusterCommunicationService.selfUuid();
|
||||
std::vector<std::pair<boost::uuids::uuid, std::string>> result = {
|
||||
{*selfUuid, R"JSON({"update_time": "2015-05-15:12:00:00"})JSON"},
|
||||
};
|
||||
|
||||
for (auto const& node : kOTHER_NODES_DATA) {
|
||||
boost::json::value jsonValue;
|
||||
boost::json::value_from(node, jsonValue);
|
||||
result.emplace_back(*node.uuid, boost::json::serialize(jsonValue));
|
||||
}
|
||||
return result;
|
||||
});
|
||||
|
||||
clusterCommunicationService.run();
|
||||
wait();
|
||||
clusterCommunicationService.stop();
|
||||
|
||||
EXPECT_TRUE(isHealthyMetric);
|
||||
EXPECT_EQ(nodesInClusterMetric.value(), 3);
|
||||
ON_CALL(*backend_, fetchClioNodesData).WillByDefault(testing::Invoke([&](auto) {
|
||||
backendOperationsCount++;
|
||||
fetchSemaphore.release();
|
||||
return fetchResult;
|
||||
}));
|
||||
ON_CALL(*backend_, writeNodeMessage).WillByDefault(testing::Invoke([&](auto&&, auto&&) {
|
||||
backendOperationsCount++;
|
||||
}));
|
||||
|
||||
ClusterCommunicationService service{backend_, std::move(writerState), kSHORT_INTERVAL, kSHORT_INTERVAL};
|
||||
|
||||
service.run();
|
||||
EXPECT_TRUE(waitForSignal(fetchSemaphore));
|
||||
service.stop();
|
||||
|
||||
auto const countAfterStop = backendOperationsCount.load();
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds{50});
|
||||
EXPECT_EQ(backendOperationsCount.load(), countAfterStop);
|
||||
}
|
||||
|
||||
189
tests/unit/cluster/MetricsTests.cpp
Normal file
189
tests/unit/cluster/MetricsTests.cpp
Normal file
@@ -0,0 +1,189 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "cluster/Metrics.hpp"
|
||||
#include "util/MockPrometheus.hpp"
|
||||
#include "util/prometheus/Gauge.hpp"
|
||||
|
||||
#include <boost/uuid/random_generator.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <expected>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
using namespace cluster;
|
||||
using namespace util::prometheus;
|
||||
using namespace testing;
|
||||
|
||||
struct MetricsTest : WithMockPrometheus {
|
||||
std::shared_ptr<boost::uuids::uuid> uuid1 =
|
||||
std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()());
|
||||
std::shared_ptr<boost::uuids::uuid> uuid2 =
|
||||
std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()());
|
||||
std::shared_ptr<boost::uuids::uuid> uuid3 =
|
||||
std::make_shared<boost::uuids::uuid>(boost::uuids::random_generator()());
|
||||
};
|
||||
|
||||
TEST_F(MetricsTest, InitializesMetricsOnConstruction)
|
||||
{
|
||||
auto& nodesInClusterMock = makeMock<GaugeInt>("cluster_nodes_total_number", "");
|
||||
auto& isHealthyMock = makeMock<GaugeInt>("cluster_communication_is_healthy", "");
|
||||
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
|
||||
Metrics const metrics;
|
||||
}
|
||||
|
||||
TEST_F(MetricsTest, OnNewStateWithValidClusterData)
|
||||
{
|
||||
auto& nodesInClusterMock = makeMock<GaugeInt>("cluster_nodes_total_number", "");
|
||||
auto& isHealthyMock = makeMock<GaugeInt>("cluster_communication_is_healthy", "");
|
||||
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
|
||||
Metrics metrics;
|
||||
|
||||
ClioNode const node1{
|
||||
.uuid = uuid1, .updateTime = std::chrono::system_clock::now(), .dbRole = ClioNode::DbRole::Writer
|
||||
};
|
||||
ClioNode const node2{
|
||||
.uuid = uuid2, .updateTime = std::chrono::system_clock::now(), .dbRole = ClioNode::DbRole::ReadOnly
|
||||
};
|
||||
ClioNode const node3{
|
||||
.uuid = uuid3, .updateTime = std::chrono::system_clock::now(), .dbRole = ClioNode::DbRole::NotWriter
|
||||
};
|
||||
|
||||
std::vector<ClioNode> const nodes = {node1, node2, node3};
|
||||
Backend::ClusterData const clusterData = std::expected<std::vector<ClioNode>, std::string>(nodes);
|
||||
auto sharedClusterData = std::make_shared<Backend::ClusterData>(clusterData);
|
||||
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
EXPECT_CALL(nodesInClusterMock, set(3));
|
||||
|
||||
metrics.onNewState(uuid1, sharedClusterData);
|
||||
}
|
||||
|
||||
TEST_F(MetricsTest, OnNewStateWithEmptyClusterData)
|
||||
{
|
||||
auto& nodesInClusterMock = makeMock<GaugeInt>("cluster_nodes_total_number", "");
|
||||
auto& isHealthyMock = makeMock<GaugeInt>("cluster_communication_is_healthy", "");
|
||||
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
|
||||
Metrics metrics;
|
||||
|
||||
std::vector<ClioNode> const nodes = {};
|
||||
Backend::ClusterData const clusterData = std::expected<std::vector<ClioNode>, std::string>(nodes);
|
||||
auto sharedClusterData = std::make_shared<Backend::ClusterData>(clusterData);
|
||||
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
EXPECT_CALL(nodesInClusterMock, set(0));
|
||||
|
||||
metrics.onNewState(uuid1, sharedClusterData);
|
||||
}
|
||||
|
||||
TEST_F(MetricsTest, OnNewStateWithFailedClusterData)
|
||||
{
|
||||
auto& nodesInClusterMock = makeMock<GaugeInt>("cluster_nodes_total_number", "");
|
||||
auto& isHealthyMock = makeMock<GaugeInt>("cluster_communication_is_healthy", "");
|
||||
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
|
||||
Metrics metrics;
|
||||
|
||||
Backend::ClusterData const clusterData =
|
||||
std::expected<std::vector<ClioNode>, std::string>(std::unexpected("Connection failed"));
|
||||
auto sharedClusterData = std::make_shared<Backend::ClusterData>(clusterData);
|
||||
|
||||
EXPECT_CALL(isHealthyMock, set(0));
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
|
||||
metrics.onNewState(uuid1, sharedClusterData);
|
||||
}
|
||||
|
||||
TEST_F(MetricsTest, OnNewStateWithSingleNode)
|
||||
{
|
||||
auto& nodesInClusterMock = makeMock<GaugeInt>("cluster_nodes_total_number", "");
|
||||
auto& isHealthyMock = makeMock<GaugeInt>("cluster_communication_is_healthy", "");
|
||||
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
|
||||
Metrics metrics;
|
||||
|
||||
ClioNode const node1{
|
||||
.uuid = uuid1, .updateTime = std::chrono::system_clock::now(), .dbRole = ClioNode::DbRole::Writer
|
||||
};
|
||||
|
||||
std::vector<ClioNode> const nodes = {node1};
|
||||
Backend::ClusterData const clusterData = std::expected<std::vector<ClioNode>, std::string>(nodes);
|
||||
auto sharedClusterData = std::make_shared<Backend::ClusterData>(clusterData);
|
||||
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
|
||||
metrics.onNewState(uuid1, sharedClusterData);
|
||||
}
|
||||
|
||||
TEST_F(MetricsTest, OnNewStateRecoveryFromFailure)
|
||||
{
|
||||
auto& nodesInClusterMock = makeMock<GaugeInt>("cluster_nodes_total_number", "");
|
||||
auto& isHealthyMock = makeMock<GaugeInt>("cluster_communication_is_healthy", "");
|
||||
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
|
||||
Metrics metrics;
|
||||
|
||||
Backend::ClusterData const clusterData1 =
|
||||
std::expected<std::vector<ClioNode>, std::string>(std::unexpected("Connection timeout"));
|
||||
auto sharedClusterData1 = std::make_shared<Backend::ClusterData>(clusterData1);
|
||||
|
||||
EXPECT_CALL(isHealthyMock, set(0));
|
||||
EXPECT_CALL(nodesInClusterMock, set(1));
|
||||
|
||||
metrics.onNewState(uuid1, sharedClusterData1);
|
||||
|
||||
ClioNode const node1{
|
||||
.uuid = uuid1, .updateTime = std::chrono::system_clock::now(), .dbRole = ClioNode::DbRole::Writer
|
||||
};
|
||||
ClioNode const node2{
|
||||
.uuid = uuid2, .updateTime = std::chrono::system_clock::now(), .dbRole = ClioNode::DbRole::ReadOnly
|
||||
};
|
||||
|
||||
std::vector<ClioNode> const nodes = {node1, node2};
|
||||
Backend::ClusterData const clusterData2 = std::expected<std::vector<ClioNode>, std::string>(nodes);
|
||||
auto sharedClusterData2 = std::make_shared<Backend::ClusterData>(clusterData2);
|
||||
|
||||
EXPECT_CALL(isHealthyMock, set(1));
|
||||
EXPECT_CALL(nodesInClusterMock, set(2));
|
||||
|
||||
metrics.onNewState(uuid2, sharedClusterData2);
|
||||
}
|
||||
223
tests/unit/cluster/RepeatedTaskTests.cpp
Normal file
223
tests/unit/cluster/RepeatedTaskTests.cpp
Normal file
@@ -0,0 +1,223 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "cluster/impl/RepeatedTask.hpp"
|
||||
#include "util/AsioContextTestFixture.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/steady_timer.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <semaphore>
|
||||
#include <thread>
|
||||
|
||||
using namespace cluster::impl;
|
||||
using namespace testing;
|
||||
|
||||
struct RepeatedTaskTest : AsyncAsioContextTest {
|
||||
static constexpr auto kTIMEOUT = std::chrono::seconds{5};
|
||||
};
|
||||
|
||||
template <typename MockFunctionType>
|
||||
struct RepeatedTaskTypedTest : RepeatedTaskTest {
|
||||
std::atomic_int32_t callCount{0};
|
||||
std::binary_semaphore semaphore{0};
|
||||
testing::StrictMock<MockFunctionType> mockFn;
|
||||
|
||||
void
|
||||
expectCalls(int const expectedCalls)
|
||||
{
|
||||
callCount = 0;
|
||||
|
||||
EXPECT_CALL(mockFn, Call).Times(AtLeast(expectedCalls)).WillRepeatedly([this, expectedCalls](auto&&...) {
|
||||
++callCount;
|
||||
if (callCount >= expectedCalls) {
|
||||
semaphore.release();
|
||||
}
|
||||
});
|
||||
}
|
||||
};
|
||||
|
||||
namespace {
|
||||
|
||||
using TypesToTest = Types<MockFunction<void()>, MockFunction<void(boost::asio::yield_context)>>;
|
||||
|
||||
} // namespace
|
||||
|
||||
TYPED_TEST_SUITE(RepeatedTaskTypedTest, TypesToTest);
|
||||
|
||||
TYPED_TEST(RepeatedTaskTypedTest, CallsFunctionRepeatedly)
|
||||
{
|
||||
RepeatedTask<boost::asio::io_context> task(std::chrono::milliseconds(1), this->ctx_);
|
||||
|
||||
this->expectCalls(3);
|
||||
|
||||
task.run(this->mockFn.AsStdFunction());
|
||||
|
||||
EXPECT_TRUE(this->semaphore.try_acquire_for(TestFixture::kTIMEOUT));
|
||||
|
||||
task.stop();
|
||||
}
|
||||
|
||||
TYPED_TEST(RepeatedTaskTypedTest, StopsImmediately)
|
||||
{
|
||||
auto const interval = std::chrono::seconds(5);
|
||||
RepeatedTask<boost::asio::io_context> task(interval, this->ctx_);
|
||||
|
||||
task.run(this->mockFn.AsStdFunction());
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(5));
|
||||
|
||||
auto start = std::chrono::steady_clock::now();
|
||||
task.stop();
|
||||
EXPECT_LT(std::chrono::steady_clock::now() - start, interval);
|
||||
}
|
||||
|
||||
TYPED_TEST(RepeatedTaskTypedTest, MultipleStops)
|
||||
{
|
||||
RepeatedTask<boost::asio::io_context> task(std::chrono::milliseconds(1), this->ctx_);
|
||||
|
||||
this->expectCalls(3);
|
||||
|
||||
task.run(this->mockFn.AsStdFunction());
|
||||
|
||||
EXPECT_TRUE(this->semaphore.try_acquire_for(TestFixture::kTIMEOUT));
|
||||
|
||||
task.stop();
|
||||
task.stop();
|
||||
task.stop();
|
||||
}
|
||||
|
||||
TYPED_TEST(RepeatedTaskTypedTest, DestructorStopsTask)
|
||||
{
|
||||
this->expectCalls(3);
|
||||
|
||||
{
|
||||
RepeatedTask<boost::asio::io_context> task(std::chrono::milliseconds(1), this->ctx_);
|
||||
|
||||
task.run(this->mockFn.AsStdFunction());
|
||||
|
||||
EXPECT_TRUE(this->semaphore.try_acquire_for(TestFixture::kTIMEOUT));
|
||||
|
||||
// Destructor will call stop()
|
||||
}
|
||||
|
||||
auto const countAfterDestruction = this->callCount.load();
|
||||
|
||||
// Wait a bit - no more calls should happen
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||
|
||||
EXPECT_EQ(this->callCount, countAfterDestruction);
|
||||
}
|
||||
|
||||
TYPED_TEST(RepeatedTaskTypedTest, StopWithoutRunIsNoOp)
|
||||
{
|
||||
RepeatedTask<boost::asio::io_context> task(std::chrono::milliseconds(1), this->ctx_);
|
||||
|
||||
// Should not crash or hang
|
||||
task.stop();
|
||||
}
|
||||
|
||||
TEST_F(RepeatedTaskTest, MultipleTasksRunConcurrently)
|
||||
{
|
||||
StrictMock<MockFunction<void()>> mockFn1;
|
||||
StrictMock<MockFunction<void()>> mockFn2;
|
||||
|
||||
RepeatedTask<boost::asio::io_context> task1(std::chrono::milliseconds(1), ctx_);
|
||||
RepeatedTask<boost::asio::io_context> task2(std::chrono::milliseconds(2), ctx_);
|
||||
|
||||
std::atomic_int32_t callCount1{0};
|
||||
std::atomic_int32_t callCount2{0};
|
||||
std::binary_semaphore semaphore1{0};
|
||||
std::binary_semaphore semaphore2{0};
|
||||
|
||||
EXPECT_CALL(mockFn1, Call).Times(AtLeast(10)).WillRepeatedly([&]() {
|
||||
if (++callCount1 >= 10) {
|
||||
semaphore1.release();
|
||||
}
|
||||
});
|
||||
|
||||
EXPECT_CALL(mockFn2, Call).Times(AtLeast(5)).WillRepeatedly([&]() {
|
||||
if (++callCount2 >= 5) {
|
||||
semaphore2.release();
|
||||
}
|
||||
});
|
||||
|
||||
task1.run(mockFn1.AsStdFunction());
|
||||
task2.run(mockFn2.AsStdFunction());
|
||||
|
||||
EXPECT_TRUE(semaphore1.try_acquire_for(kTIMEOUT));
|
||||
EXPECT_TRUE(semaphore2.try_acquire_for(kTIMEOUT));
|
||||
|
||||
task1.stop();
|
||||
task2.stop();
|
||||
}
|
||||
|
||||
TYPED_TEST(RepeatedTaskTypedTest, TaskStateTransitionsCorrectly)
|
||||
{
|
||||
RepeatedTask<boost::asio::io_context> task(std::chrono::milliseconds(1), this->ctx_);
|
||||
|
||||
task.stop(); // Should be no-op
|
||||
|
||||
this->expectCalls(3);
|
||||
|
||||
task.run(this->mockFn.AsStdFunction());
|
||||
|
||||
EXPECT_TRUE(this->semaphore.try_acquire_for(TestFixture::kTIMEOUT));
|
||||
|
||||
task.stop();
|
||||
|
||||
// Stop again should be no-op
|
||||
task.stop();
|
||||
}
|
||||
|
||||
TEST_F(RepeatedTaskTest, FunctionCanAccessYieldContext)
|
||||
{
|
||||
StrictMock<MockFunction<void(boost::asio::yield_context)>> mockFn;
|
||||
std::atomic_bool yieldContextUsed = false;
|
||||
std::binary_semaphore semaphore{0};
|
||||
|
||||
RepeatedTask<boost::asio::io_context> task(std::chrono::milliseconds(1), ctx_);
|
||||
|
||||
EXPECT_CALL(mockFn, Call).Times(AtLeast(1)).WillRepeatedly([&](boost::asio::yield_context yield) {
|
||||
if (yieldContextUsed)
|
||||
return;
|
||||
|
||||
// Use the yield context to verify it's valid
|
||||
boost::asio::steady_timer timer(yield.get_executor());
|
||||
timer.expires_after(std::chrono::milliseconds(1));
|
||||
boost::system::error_code ec;
|
||||
timer.async_wait(yield[ec]);
|
||||
EXPECT_FALSE(ec) << ec.message();
|
||||
yieldContextUsed = true;
|
||||
semaphore.release();
|
||||
});
|
||||
|
||||
task.run(mockFn.AsStdFunction());
|
||||
|
||||
EXPECT_TRUE(semaphore.try_acquire_for(kTIMEOUT));
|
||||
|
||||
task.stop();
|
||||
|
||||
EXPECT_TRUE(yieldContextUsed);
|
||||
}
|
||||
314
tests/unit/cluster/WriterDeciderTests.cpp
Normal file
314
tests/unit/cluster/WriterDeciderTests.cpp
Normal file
@@ -0,0 +1,314 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "cluster/Backend.hpp"
|
||||
#include "cluster/ClioNode.hpp"
|
||||
#include "cluster/WriterDecider.hpp"
|
||||
#include "util/MockWriterState.hpp"
|
||||
|
||||
#include <boost/asio/thread_pool.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
using namespace cluster;
|
||||
|
||||
enum class ExpectedAction { StartWriting, GiveUpWriting, NoAction, SetFallback };
|
||||
|
||||
struct WriterDeciderTestParams {
|
||||
std::string testName;
|
||||
uint8_t selfUuidValue;
|
||||
std::vector<std::pair<uint8_t, ClioNode::DbRole>> nodes;
|
||||
ExpectedAction expectedAction;
|
||||
bool useEmptyClusterData = false;
|
||||
};
|
||||
|
||||
struct WriterDeciderTest : testing::TestWithParam<WriterDeciderTestParams> {
|
||||
~WriterDeciderTest() override
|
||||
{
|
||||
ctx.stop();
|
||||
ctx.join();
|
||||
}
|
||||
|
||||
boost::asio::thread_pool ctx{1};
|
||||
std::unique_ptr<MockWriterState> writerState = std::make_unique<MockWriterState>();
|
||||
MockWriterState& writerStateRef = *writerState;
|
||||
|
||||
static ClioNode
|
||||
makeNode(boost::uuids::uuid const& uuid, ClioNode::DbRole role)
|
||||
{
|
||||
return ClioNode{
|
||||
.uuid = std::make_shared<boost::uuids::uuid>(uuid),
|
||||
.updateTime = std::chrono::system_clock::now(),
|
||||
.dbRole = role
|
||||
};
|
||||
}
|
||||
|
||||
static boost::uuids::uuid
|
||||
makeUuid(uint8_t value)
|
||||
{
|
||||
boost::uuids::uuid uuid{};
|
||||
std::ranges::fill(uuid, value);
|
||||
return uuid;
|
||||
}
|
||||
};
|
||||
|
||||
TEST_P(WriterDeciderTest, WriterSelection)
|
||||
{
|
||||
auto const& params = GetParam();
|
||||
|
||||
auto const selfUuid = makeUuid(params.selfUuidValue);
|
||||
|
||||
WriterDecider decider{ctx, std::move(writerState)};
|
||||
|
||||
auto clonedState = std::make_unique<MockWriterState>();
|
||||
|
||||
// Set up expectations based on expected action
|
||||
switch (params.expectedAction) {
|
||||
case ExpectedAction::StartWriting:
|
||||
EXPECT_CALL(*clonedState, startWriting());
|
||||
EXPECT_CALL(writerStateRef, clone()).WillOnce(testing::Return(testing::ByMove(std::move(clonedState))));
|
||||
break;
|
||||
case ExpectedAction::GiveUpWriting:
|
||||
EXPECT_CALL(*clonedState, giveUpWriting());
|
||||
EXPECT_CALL(writerStateRef, clone()).WillOnce(testing::Return(testing::ByMove(std::move(clonedState))));
|
||||
break;
|
||||
case ExpectedAction::SetFallback:
|
||||
EXPECT_CALL(*clonedState, setWriterDecidingFallback());
|
||||
EXPECT_CALL(writerStateRef, clone()).WillOnce(testing::Return(testing::ByMove(std::move(clonedState))));
|
||||
break;
|
||||
case ExpectedAction::NoAction:
|
||||
if (not params.useEmptyClusterData) {
|
||||
// For all-ReadOnly case, we still clone but don't call any action
|
||||
EXPECT_CALL(writerStateRef, clone()).WillOnce(testing::Return(testing::ByMove(std::move(clonedState))));
|
||||
}
|
||||
// For empty cluster data, clone is never called
|
||||
break;
|
||||
}
|
||||
|
||||
std::shared_ptr<Backend::ClusterData> clusterData;
|
||||
ClioNode::CUuid selfIdPtr;
|
||||
|
||||
if (params.useEmptyClusterData) {
|
||||
clusterData = std::make_shared<Backend::ClusterData>(std::unexpected(std::string("Communication failed")));
|
||||
selfIdPtr = std::make_shared<boost::uuids::uuid>(selfUuid);
|
||||
} else {
|
||||
std::vector<ClioNode> nodes;
|
||||
nodes.reserve(params.nodes.size());
|
||||
for (auto const& [uuidValue, role] : params.nodes) {
|
||||
auto node = makeNode(makeUuid(uuidValue), role);
|
||||
if (uuidValue == params.selfUuidValue) {
|
||||
selfIdPtr = node.uuid; // Use the same shared_ptr as in the node
|
||||
}
|
||||
nodes.push_back(std::move(node));
|
||||
}
|
||||
clusterData = std::make_shared<Backend::ClusterData>(std::move(nodes));
|
||||
}
|
||||
|
||||
decider.onNewState(selfIdPtr, clusterData);
|
||||
|
||||
ctx.join();
|
||||
}
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(
|
||||
WriterDeciderTests,
|
||||
WriterDeciderTest,
|
||||
testing::Values(
|
||||
WriterDeciderTestParams{
|
||||
.testName = "SelfNodeIsSelectedAsWriter",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::Writer}, {0x02, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::StartWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "OtherNodeIsSelectedAsWriter",
|
||||
.selfUuidValue = 0x02,
|
||||
.nodes = {{0x01, ClioNode::DbRole::Writer}, {0x02, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "NodesAreSortedByUUID",
|
||||
.selfUuidValue = 0x02,
|
||||
.nodes =
|
||||
{{0x03, ClioNode::DbRole::Writer}, {0x02, ClioNode::DbRole::Writer}, {0x01, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "FirstNodeAfterReadOnlyIsNotSelf",
|
||||
.selfUuidValue = 0x03,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::ReadOnly},
|
||||
{0x02, ClioNode::DbRole::Writer},
|
||||
{0x03, ClioNode::DbRole::NotWriter}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "FirstNodeAfterReadOnlyIsSelf",
|
||||
.selfUuidValue = 0x02,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::ReadOnly},
|
||||
{0x02, ClioNode::DbRole::Writer},
|
||||
{0x03, ClioNode::DbRole::NotWriter}},
|
||||
.expectedAction = ExpectedAction::StartWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "AllNodesReadOnlyGiveUpWriting",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::ReadOnly}, {0x02, ClioNode::DbRole::ReadOnly}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "EmptyClusterDataNoActionTaken",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {},
|
||||
.expectedAction = ExpectedAction::NoAction,
|
||||
.useEmptyClusterData = true
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "SingleNodeClusterSelfIsWriter",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::StartWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "NotWriterRoleIsSelectedWhenNoWriterRole",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::NotWriter}, {0x02, ClioNode::DbRole::NotWriter}},
|
||||
.expectedAction = ExpectedAction::StartWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "MixedRolesFirstNonReadOnlyIsSelected",
|
||||
.selfUuidValue = 0x03,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::ReadOnly},
|
||||
{0x02, ClioNode::DbRole::Writer},
|
||||
{0x03, ClioNode::DbRole::NotWriter},
|
||||
{0x04, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "ShuffledNodesAreSortedCorrectly",
|
||||
.selfUuidValue = 0x04,
|
||||
.nodes =
|
||||
{{0x04, ClioNode::DbRole::Writer},
|
||||
{0x01, ClioNode::DbRole::Writer},
|
||||
{0x03, ClioNode::DbRole::Writer},
|
||||
{0x02, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "ShuffledNodesWithReadOnlySelfIsSelected",
|
||||
.selfUuidValue = 0x03,
|
||||
.nodes =
|
||||
{{0x05, ClioNode::DbRole::Writer},
|
||||
{0x01, ClioNode::DbRole::ReadOnly},
|
||||
{0x04, ClioNode::DbRole::Writer},
|
||||
{0x03, ClioNode::DbRole::Writer},
|
||||
{0x02, ClioNode::DbRole::ReadOnly}},
|
||||
.expectedAction = ExpectedAction::StartWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "SelfIsFallbackNoActionTaken",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::Fallback}, {0x02, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::NoAction
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "OtherNodeIsFallbackSetsFallbackMode",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::Writer}, {0x02, ClioNode::DbRole::Fallback}},
|
||||
.expectedAction = ExpectedAction::SetFallback
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "SelfIsReadOnlyOthersAreFallbackGiveUpWriting",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::ReadOnly}, {0x02, ClioNode::DbRole::Fallback}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "MultipleFallbackNodesSelfNotFallbackSetsFallback",
|
||||
.selfUuidValue = 0x03,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::Fallback},
|
||||
{0x02, ClioNode::DbRole::Fallback},
|
||||
{0x03, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::SetFallback
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "MixedRolesWithOneFallbackSetsFallback",
|
||||
.selfUuidValue = 0x02,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::Writer},
|
||||
{0x02, ClioNode::DbRole::NotWriter},
|
||||
{0x03, ClioNode::DbRole::Fallback},
|
||||
{0x04, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::SetFallback
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "SelfIsLoadingCacheOtherIsWriter",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::LoadingCache}, {0x02, ClioNode::DbRole::Writer}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "OtherNodeIsLoadingCacheSkipToNextWriter",
|
||||
.selfUuidValue = 0x02,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::LoadingCache},
|
||||
{0x02, ClioNode::DbRole::Writer},
|
||||
{0x03, ClioNode::DbRole::NotWriter}},
|
||||
.expectedAction = ExpectedAction::StartWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "AllNodesLoadingCacheNoActionTaken",
|
||||
.selfUuidValue = 0x01,
|
||||
.nodes = {{0x01, ClioNode::DbRole::LoadingCache}, {0x02, ClioNode::DbRole::LoadingCache}},
|
||||
.expectedAction = ExpectedAction::NoAction
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "MixedWithLoadingCacheReadOnlyFirstNonReadOnlyNonLoadingCacheSelected",
|
||||
.selfUuidValue = 0x03,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::ReadOnly},
|
||||
{0x02, ClioNode::DbRole::LoadingCache},
|
||||
{0x03, ClioNode::DbRole::Writer},
|
||||
{0x04, ClioNode::DbRole::NotWriter}},
|
||||
.expectedAction = ExpectedAction::StartWriting
|
||||
},
|
||||
WriterDeciderTestParams{
|
||||
.testName = "LoadingCacheBeforeWriterSkipsLoadingCache",
|
||||
.selfUuidValue = 0x04,
|
||||
.nodes =
|
||||
{{0x01, ClioNode::DbRole::LoadingCache},
|
||||
{0x02, ClioNode::DbRole::LoadingCache},
|
||||
{0x03, ClioNode::DbRole::Writer},
|
||||
{0x04, ClioNode::DbRole::NotWriter}},
|
||||
.expectedAction = ExpectedAction::GiveUpWriting
|
||||
}
|
||||
),
|
||||
[](testing::TestParamInfo<WriterDeciderTestParams> const& info) { return info.param.testName; }
|
||||
);
|
||||
@@ -216,6 +216,10 @@ protected:
|
||||
std::shared_ptr<testing::NiceMock<MockMonitorProvider>> monitorProvider_ =
|
||||
std::make_shared<testing::NiceMock<MockMonitorProvider>>();
|
||||
std::shared_ptr<etl::SystemState> systemState_ = std::make_shared<etl::SystemState>();
|
||||
testing::StrictMock<testing::MockFunction<void(etl::SystemState::WriteCommand)>> mockWriteSignalCommandCallback_;
|
||||
boost::signals2::scoped_connection writeCommandConnection_{
|
||||
systemState_->writeCommandSignal.connect(mockWriteSignalCommandCallback_.AsStdFunction())
|
||||
};
|
||||
|
||||
etl::ETLService service_{
|
||||
ctx_,
|
||||
@@ -300,6 +304,7 @@ TEST_F(ETLServiceTests, RunWithEmptyDatabase)
|
||||
auto mockTaskManager = std::make_unique<testing::NiceMock<MockTaskManager>>();
|
||||
auto& mockTaskManagerRef = *mockTaskManager;
|
||||
auto ledgerData = createTestData(kSEQ);
|
||||
EXPECT_TRUE(systemState_->isLoadingCache);
|
||||
|
||||
testing::Sequence const s;
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange).InSequence(s).WillOnce(testing::Return(std::nullopt));
|
||||
@@ -308,25 +313,61 @@ TEST_F(ETLServiceTests, RunWithEmptyDatabase)
|
||||
EXPECT_CALL(*balancer_, loadInitialLedger(kSEQ, testing::_, testing::_))
|
||||
.WillOnce(testing::Return(std::vector<std::string>{}));
|
||||
EXPECT_CALL(*loader_, loadInitialLedger).WillOnce(testing::Return(ripple::LedgerHeader{}));
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.InSequence(s)
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
// In syncCacheWithDb()
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange).Times(2).InSequence(s).WillRepeatedly([this]() {
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
return data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ};
|
||||
});
|
||||
EXPECT_CALL(mockTaskManagerRef, run);
|
||||
EXPECT_CALL(*taskManagerProvider_, make(testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce(testing::Return(std::unique_ptr<etl::TaskManagerInterface>(mockTaskManager.release())));
|
||||
EXPECT_CALL(*monitorProvider_, make(testing::_, testing::_, testing::_, testing::_, testing::_))
|
||||
.WillOnce([](auto, auto, auto, auto, auto) { return std::make_unique<testing::NiceMock<MockMonitor>>(); });
|
||||
EXPECT_CALL(*taskManagerProvider_, make(testing::_, testing::_, kSEQ + 1, testing::_)).WillOnce([&](auto&&...) {
|
||||
EXPECT_FALSE(systemState_->isLoadingCache);
|
||||
return std::unique_ptr<etl::TaskManagerInterface>(mockTaskManager.release());
|
||||
});
|
||||
EXPECT_CALL(*monitorProvider_, make(testing::_, testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce([this](auto, auto, auto, auto, auto) {
|
||||
EXPECT_TRUE(systemState_->isLoadingCache);
|
||||
return std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
});
|
||||
|
||||
service_.run();
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, RunWithPopulatedDatabase)
|
||||
{
|
||||
EXPECT_TRUE(systemState_->isLoadingCache);
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([](auto, auto, auto, auto, auto) {
|
||||
return std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
});
|
||||
EXPECT_CALL(*monitorProvider_, make(testing::_, testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce([this](auto, auto, auto, auto, auto) {
|
||||
EXPECT_TRUE(systemState_->isLoadingCache);
|
||||
return std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
});
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillRepeatedly(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, SyncCacheWithDbBeforeStartingMonitor)
|
||||
{
|
||||
EXPECT_TRUE(systemState_->isLoadingCache);
|
||||
backend_->cache().update({}, kSEQ - 2, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
|
||||
EXPECT_CALL(*backend_, fetchLedgerDiff(kSEQ - 1, testing::_));
|
||||
EXPECT_CALL(*cacheUpdater_, update(kSEQ - 1, std::vector<data::LedgerObject>()))
|
||||
.WillOnce([this](auto const seq, auto&&...) { backend_->cache().update({}, seq, false); });
|
||||
EXPECT_CALL(*backend_, fetchLedgerDiff(kSEQ, testing::_));
|
||||
EXPECT_CALL(*cacheUpdater_, update(kSEQ, std::vector<data::LedgerObject>()))
|
||||
.WillOnce([this](auto const seq, auto&&...) { backend_->cache().update({}, seq, false); });
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make(testing::_, testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce([this](auto, auto, auto, auto, auto) {
|
||||
EXPECT_TRUE(systemState_->isLoadingCache);
|
||||
return std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
});
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillRepeatedly(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
@@ -364,19 +405,22 @@ TEST_F(ETLServiceTests, HandlesWriteConflictInMonitorSubscription)
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled);
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
.Times(2)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->writeConflict = true;
|
||||
writeCommandConnection_.disconnect();
|
||||
systemState_->writeCommandSignal(etl::SystemState::WriteCommand::StopWriting);
|
||||
|
||||
EXPECT_CALL(*publisher_, publish(kSEQ + 1, testing::_, testing::_));
|
||||
ASSERT_TRUE(capturedCallback);
|
||||
capturedCallback(kSEQ + 1);
|
||||
|
||||
EXPECT_FALSE(systemState_->writeConflict);
|
||||
EXPECT_FALSE(systemState_->isWriting);
|
||||
}
|
||||
|
||||
@@ -397,8 +441,11 @@ TEST_F(ETLServiceTests, NormalFlowInMonitorSubscription)
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled);
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
.Times(2)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
@@ -424,13 +471,19 @@ TEST_F(ETLServiceTests, AttemptTakeoverWriter)
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence);
|
||||
std::function<void(uint32_t)> onNewSeqCallback;
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence).WillOnce([&onNewSeqCallback](auto cb) {
|
||||
onNewSeqCallback = std::move(cb);
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled).WillOnce([&capturedDbStalledCallback](auto callback) {
|
||||
capturedDbStalledCallback = callback;
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
@@ -447,10 +500,14 @@ TEST_F(ETLServiceTests, AttemptTakeoverWriter)
|
||||
EXPECT_CALL(*taskManagerProvider_, make(testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce(testing::Return(std::move(mockTaskManager)));
|
||||
|
||||
ASSERT_TRUE(capturedDbStalledCallback);
|
||||
capturedDbStalledCallback();
|
||||
EXPECT_CALL(mockWriteSignalCommandCallback_, Call(etl::SystemState::WriteCommand::StartWriting));
|
||||
|
||||
EXPECT_TRUE(systemState_->isWriting); // should attempt to become writer
|
||||
ASSERT_TRUE(capturedDbStalledCallback);
|
||||
EXPECT_FALSE(systemState_->isWriting); // will attempt to become writer after new sequence appears but not yet
|
||||
EXPECT_FALSE(systemState_->isWriterDecidingFallback);
|
||||
capturedDbStalledCallback();
|
||||
EXPECT_TRUE(systemState_->isWriting); // should attempt to become writer
|
||||
EXPECT_TRUE(systemState_->isWriterDecidingFallback); // fallback mode activated
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, GiveUpWriterAfterWriteConflict)
|
||||
@@ -470,22 +527,25 @@ TEST_F(ETLServiceTests, GiveUpWriterAfterWriteConflict)
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled);
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
.Times(2)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isWriting = true;
|
||||
systemState_->writeConflict = true; // got a write conflict along the way
|
||||
writeCommandConnection_.disconnect();
|
||||
systemState_->writeCommandSignal(etl::SystemState::WriteCommand::StopWriting);
|
||||
|
||||
EXPECT_CALL(*publisher_, publish(kSEQ + 1, testing::_, testing::_));
|
||||
|
||||
ASSERT_TRUE(capturedCallback);
|
||||
capturedCallback(kSEQ + 1);
|
||||
|
||||
EXPECT_FALSE(systemState_->isWriting); // gives up writing
|
||||
EXPECT_FALSE(systemState_->writeConflict); // and removes write conflict flag
|
||||
EXPECT_FALSE(systemState_->isWriting); // gives up writing
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, CancelledLoadInitialLedger)
|
||||
@@ -539,3 +599,327 @@ TEST_F(ETLServiceTests, RunStopsIfInitialLoadIsCancelledByBalancer)
|
||||
EXPECT_FALSE(service_.isAmendmentBlocked());
|
||||
EXPECT_FALSE(service_.isCorruptionDetected());
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, DbStalledDoesNotTriggerSignalWhenStrictReadonly)
|
||||
{
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
auto& mockMonitorRef = *mockMonitor;
|
||||
std::function<void()> capturedDbStalledCallback;
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence);
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled).WillOnce([&capturedDbStalledCallback](auto callback) {
|
||||
capturedDbStalledCallback = callback;
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isStrictReadonly = true; // strict readonly mode
|
||||
systemState_->isWriting = false;
|
||||
|
||||
// No signal should be emitted because node is in strict readonly mode
|
||||
// But fallback flag should still be set
|
||||
|
||||
ASSERT_TRUE(capturedDbStalledCallback);
|
||||
EXPECT_FALSE(systemState_->isWriterDecidingFallback);
|
||||
capturedDbStalledCallback();
|
||||
EXPECT_TRUE(systemState_->isWriterDecidingFallback); // fallback mode activated even in readonly
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, DbStalledDoesNotTriggerSignalWhenAlreadyWriting)
|
||||
{
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
auto& mockMonitorRef = *mockMonitor;
|
||||
std::function<void()> capturedDbStalledCallback;
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence);
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled).WillOnce([&capturedDbStalledCallback](auto callback) {
|
||||
capturedDbStalledCallback = callback;
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isStrictReadonly = false;
|
||||
systemState_->isWriting = true; // already writing
|
||||
|
||||
// No signal should be emitted because node is already writing
|
||||
// But fallback flag should still be set
|
||||
|
||||
ASSERT_TRUE(capturedDbStalledCallback);
|
||||
EXPECT_FALSE(systemState_->isWriterDecidingFallback);
|
||||
capturedDbStalledCallback();
|
||||
EXPECT_TRUE(systemState_->isWriterDecidingFallback); // fallback mode activated
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, CacheUpdatesDependOnActualCacheState_WriterMode)
|
||||
{
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
auto& mockMonitorRef = *mockMonitor;
|
||||
std::function<void(uint32_t)> capturedCallback;
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence).WillOnce([&capturedCallback](auto callback) {
|
||||
capturedCallback = callback;
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled);
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB initially to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isWriting = true; // In writer mode
|
||||
|
||||
// Simulate cache is behind (e.g., update failed previously)
|
||||
// Cache latestLedgerSequence returns kSEQ (behind the new seq kSEQ + 1)
|
||||
std::vector<data::LedgerObject> const emptyObjs = {};
|
||||
backend_->cache().update(emptyObjs, kSEQ); // Set cache to kSEQ
|
||||
|
||||
std::vector<data::LedgerObject> const dummyDiff = {};
|
||||
EXPECT_CALL(*backend_, fetchLedgerDiff(kSEQ + 1, testing::_)).WillOnce(testing::Return(dummyDiff));
|
||||
|
||||
// Cache should be updated even though we're in writer mode
|
||||
EXPECT_CALL(*cacheUpdater_, update(kSEQ + 1, testing::A<std::vector<data::LedgerObject> const&>()));
|
||||
|
||||
EXPECT_CALL(*publisher_, publish(kSEQ + 1, testing::_, testing::_));
|
||||
|
||||
ASSERT_TRUE(capturedCallback);
|
||||
capturedCallback(kSEQ + 1);
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, OnlyCacheUpdatesWhenBackendIsCurrent)
|
||||
{
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
auto& mockMonitorRef = *mockMonitor;
|
||||
std::function<void(uint32_t)> capturedCallback;
|
||||
// Set cache to be in sync with DB initially to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence).WillOnce([&capturedCallback](auto callback) {
|
||||
capturedCallback = callback;
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled);
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set backend range to be at kSEQ + 1 (already current)
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}))
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}))
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ + 1}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isWriting = false;
|
||||
|
||||
// Cache is behind (at kSEQ)
|
||||
std::vector<data::LedgerObject> const emptyObjs = {};
|
||||
backend_->cache().update(emptyObjs, kSEQ);
|
||||
|
||||
std::vector<data::LedgerObject> const dummyDiff = {};
|
||||
EXPECT_CALL(*backend_, fetchLedgerDiff(kSEQ + 1, testing::_)).WillOnce(testing::Return(dummyDiff));
|
||||
EXPECT_CALL(*cacheUpdater_, update(kSEQ + 1, testing::A<std::vector<data::LedgerObject> const&>()));
|
||||
|
||||
EXPECT_CALL(*publisher_, publish(kSEQ + 1, testing::_, testing::_));
|
||||
|
||||
ASSERT_TRUE(capturedCallback);
|
||||
capturedCallback(kSEQ + 1);
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, NoUpdatesWhenBothCacheAndBackendAreCurrent)
|
||||
{
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
auto& mockMonitorRef = *mockMonitor;
|
||||
std::function<void(uint32_t)> capturedCallback;
|
||||
// Set cache to be in sync with DB initially to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence).WillOnce([&capturedCallback](auto callback) {
|
||||
capturedCallback = callback;
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled);
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set backend range to be at kSEQ + 1 (already current)
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}))
|
||||
.WillOnce(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}))
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ + 1}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
|
||||
// Cache is current (at kSEQ + 1)
|
||||
std::vector<data::LedgerObject> const emptyObjs = {};
|
||||
backend_->cache().update(emptyObjs, kSEQ + 1);
|
||||
|
||||
// Neither should be updated
|
||||
EXPECT_CALL(*backend_, fetchLedgerDiff).Times(0);
|
||||
EXPECT_CALL(*cacheUpdater_, update(testing::_, testing::A<std::vector<data::LedgerObject> const&>())).Times(0);
|
||||
|
||||
EXPECT_CALL(*publisher_, publish(kSEQ + 1, testing::_, testing::_));
|
||||
|
||||
ASSERT_TRUE(capturedCallback);
|
||||
capturedCallback(kSEQ + 1);
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, StopWaitsForWriteCommandHandlersToComplete)
|
||||
{
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isStrictReadonly = false;
|
||||
|
||||
auto mockTaskManager = std::make_unique<testing::NiceMock<MockTaskManager>>();
|
||||
|
||||
EXPECT_CALL(mockWriteSignalCommandCallback_, Call(etl::SystemState::WriteCommand::StartWriting));
|
||||
EXPECT_CALL(*taskManagerProvider_, make(testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce(testing::Return(std::move(mockTaskManager)));
|
||||
|
||||
// Emit a command
|
||||
systemState_->writeCommandSignal(etl::SystemState::WriteCommand::StartWriting);
|
||||
|
||||
// The test context processes operations synchronously, so the handler should have run
|
||||
// Stop should wait for the handler to complete and disconnect the subscription
|
||||
service_.stop();
|
||||
|
||||
// Verify stop() returned, meaning all handlers completed
|
||||
SUCCEED();
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, WriteConflictIsHandledImmediately_NotDelayed)
|
||||
{
|
||||
// This test verifies that write conflicts are handled immediately via signal,
|
||||
// not delayed until the next sequence notification (the old behavior)
|
||||
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
auto& mockMonitorRef = *mockMonitor;
|
||||
std::function<void(uint32_t)> capturedNewSeqCallback;
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToNewSequence).WillOnce([&capturedNewSeqCallback](auto callback) {
|
||||
capturedNewSeqCallback = callback;
|
||||
return boost::signals2::scoped_connection{};
|
||||
});
|
||||
EXPECT_CALL(mockMonitorRef, subscribeToDbStalled);
|
||||
EXPECT_CALL(mockMonitorRef, run);
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isWriting = true;
|
||||
|
||||
// Emit StopWriting signal (simulating write conflict from Loader)
|
||||
EXPECT_CALL(mockWriteSignalCommandCallback_, Call(etl::SystemState::WriteCommand::StopWriting));
|
||||
systemState_->writeCommandSignal(etl::SystemState::WriteCommand::StopWriting);
|
||||
|
||||
// The test context processes operations synchronously, so the handler should have run immediately
|
||||
// Verify that isWriting is immediately set to false
|
||||
EXPECT_FALSE(systemState_->isWriting);
|
||||
}
|
||||
|
||||
TEST_F(ETLServiceTests, WriteCommandsAreSerializedOnStrand)
|
||||
{
|
||||
auto mockMonitor = std::make_unique<testing::NiceMock<MockMonitor>>();
|
||||
|
||||
EXPECT_CALL(*monitorProvider_, make).WillOnce([&mockMonitor](auto, auto, auto, auto, auto) {
|
||||
return std::move(mockMonitor);
|
||||
});
|
||||
|
||||
// Set cache to be in sync with DB to avoid syncCacheWithDb loop
|
||||
backend_->cache().update({}, kSEQ, false);
|
||||
EXPECT_CALL(*backend_, hardFetchLedgerRange)
|
||||
.WillRepeatedly(testing::Return(data::LedgerRange{.minSequence = 1, .maxSequence = kSEQ}));
|
||||
EXPECT_CALL(*ledgers_, getMostRecent()).WillOnce(testing::Return(kSEQ));
|
||||
EXPECT_CALL(*cacheLoader_, load(kSEQ));
|
||||
|
||||
service_.run();
|
||||
systemState_->isStrictReadonly = false;
|
||||
systemState_->isWriting = false;
|
||||
|
||||
auto mockTaskManager1 = std::make_unique<testing::NiceMock<MockTaskManager>>();
|
||||
auto mockTaskManager2 = std::make_unique<testing::NiceMock<MockTaskManager>>();
|
||||
|
||||
// Set up expectations for the sequence of write commands
|
||||
// The signals should be processed in order: StartWriting, StopWriting, StartWriting
|
||||
{
|
||||
testing::InSequence const seq;
|
||||
|
||||
// First StartWriting
|
||||
EXPECT_CALL(mockWriteSignalCommandCallback_, Call(etl::SystemState::WriteCommand::StartWriting));
|
||||
EXPECT_CALL(*taskManagerProvider_, make(testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce(testing::Return(std::move(mockTaskManager1)));
|
||||
|
||||
// Then StopWriting
|
||||
EXPECT_CALL(mockWriteSignalCommandCallback_, Call(etl::SystemState::WriteCommand::StopWriting));
|
||||
|
||||
// Finally second StartWriting
|
||||
EXPECT_CALL(mockWriteSignalCommandCallback_, Call(etl::SystemState::WriteCommand::StartWriting));
|
||||
EXPECT_CALL(*taskManagerProvider_, make(testing::_, testing::_, kSEQ + 1, testing::_))
|
||||
.WillOnce(testing::Return(std::move(mockTaskManager2)));
|
||||
}
|
||||
|
||||
// Emit multiple signals rapidly - they should be serialized on the strand
|
||||
systemState_->writeCommandSignal(etl::SystemState::WriteCommand::StartWriting);
|
||||
systemState_->writeCommandSignal(etl::SystemState::WriteCommand::StopWriting);
|
||||
systemState_->writeCommandSignal(etl::SystemState::WriteCommand::StartWriting);
|
||||
|
||||
// The test context processes operations synchronously, so all signals should have been processed
|
||||
// Final state should be writing (last signal was StartWriting)
|
||||
EXPECT_TRUE(systemState_->isWriting);
|
||||
}
|
||||
|
||||
@@ -216,15 +216,14 @@ TEST_F(ETLLedgerPublisherTest, PublishLedgerHeaderCloseTimeGreaterThanNow)
|
||||
TEST_F(ETLLedgerPublisherTest, PublishLedgerSeqStopIsTrue)
|
||||
{
|
||||
auto dummyState = etl::SystemState{};
|
||||
dummyState.isStopping = true;
|
||||
auto publisher = impl::LedgerPublisher(ctx, backend_, mockSubscriptionManagerPtr, dummyState);
|
||||
publisher.stop();
|
||||
EXPECT_FALSE(publisher.publish(kSEQ, {}));
|
||||
}
|
||||
|
||||
TEST_F(ETLLedgerPublisherTest, PublishLedgerSeqMaxAttempt)
|
||||
{
|
||||
auto dummyState = etl::SystemState{};
|
||||
dummyState.isStopping = false;
|
||||
auto publisher = impl::LedgerPublisher(ctx, backend_, mockSubscriptionManagerPtr, dummyState);
|
||||
|
||||
static constexpr auto kMAX_ATTEMPT = 2;
|
||||
@@ -238,7 +237,6 @@ TEST_F(ETLLedgerPublisherTest, PublishLedgerSeqMaxAttempt)
|
||||
TEST_F(ETLLedgerPublisherTest, PublishLedgerSeqStopIsFalse)
|
||||
{
|
||||
auto dummyState = etl::SystemState{};
|
||||
dummyState.isStopping = false;
|
||||
auto publisher = impl::LedgerPublisher(ctx, backend_, mockSubscriptionManagerPtr, dummyState);
|
||||
|
||||
LedgerRange const range{.minSequence = kSEQ, .maxSequence = kSEQ};
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/RegistryInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
@@ -188,3 +189,59 @@ TEST_F(LoadingAssertTest, LoadInitialLedgerHasDataInDB)
|
||||
|
||||
EXPECT_CLIO_ASSERT_FAIL({ [[maybe_unused]] auto unused = loader_.loadInitialLedger(data); });
|
||||
}
|
||||
|
||||
TEST_F(LoadingTests, LoadWriteConflictEmitsStopWritingSignal)
|
||||
{
|
||||
state_->isWriting = true; // writer is active
|
||||
auto const data = createTestData();
|
||||
testing::StrictMock<testing::MockFunction<void(etl::SystemState::WriteCommand)>> mockSignalCallback;
|
||||
|
||||
auto connection = state_->writeCommandSignal.connect(mockSignalCallback.AsStdFunction());
|
||||
|
||||
EXPECT_CALL(*mockRegistryPtr_, dispatch(data));
|
||||
EXPECT_CALL(*backend_, doFinishWrites()).WillOnce(testing::Return(false)); // simulate write conflict
|
||||
EXPECT_CALL(mockSignalCallback, Call(etl::SystemState::WriteCommand::StopWriting));
|
||||
|
||||
EXPECT_FALSE(state_->isWriterDecidingFallback);
|
||||
|
||||
auto result = loader_.load(data);
|
||||
EXPECT_FALSE(result.has_value());
|
||||
EXPECT_EQ(result.error(), etl::LoaderError::WriteConflict);
|
||||
EXPECT_TRUE(state_->isWriterDecidingFallback);
|
||||
}
|
||||
|
||||
TEST_F(LoadingTests, LoadSuccessDoesNotEmitSignal)
|
||||
{
|
||||
state_->isWriting = true; // writer is active
|
||||
auto const data = createTestData();
|
||||
testing::StrictMock<testing::MockFunction<void(etl::SystemState::WriteCommand)>> mockSignalCallback;
|
||||
|
||||
auto connection = state_->writeCommandSignal.connect(mockSignalCallback.AsStdFunction());
|
||||
|
||||
EXPECT_CALL(*mockRegistryPtr_, dispatch(data));
|
||||
EXPECT_CALL(*backend_, doFinishWrites()).WillOnce(testing::Return(true)); // success
|
||||
// No signal should be emitted on success
|
||||
|
||||
EXPECT_FALSE(state_->isWriterDecidingFallback);
|
||||
|
||||
auto result = loader_.load(data);
|
||||
EXPECT_TRUE(result.has_value());
|
||||
EXPECT_FALSE(state_->isWriterDecidingFallback);
|
||||
}
|
||||
|
||||
TEST_F(LoadingTests, LoadWhenNotWritingDoesNotCheckConflict)
|
||||
{
|
||||
state_->isWriting = false; // not a writer
|
||||
auto const data = createTestData();
|
||||
testing::StrictMock<testing::MockFunction<void(etl::SystemState::WriteCommand)>> mockSignalCallback;
|
||||
|
||||
auto connection = state_->writeCommandSignal.connect(mockSignalCallback.AsStdFunction());
|
||||
|
||||
EXPECT_CALL(*mockRegistryPtr_, dispatch(data));
|
||||
// doFinishWrites should not be called when not writing
|
||||
EXPECT_CALL(*backend_, doFinishWrites()).Times(0);
|
||||
// No signal should be emitted
|
||||
|
||||
auto result = loader_.load(data);
|
||||
EXPECT_TRUE(result.has_value());
|
||||
}
|
||||
|
||||
@@ -260,7 +260,7 @@ struct RegistryTest : util::prometheus::WithPrometheus {
|
||||
}
|
||||
|
||||
protected:
|
||||
etl::SystemState state_{};
|
||||
etl::SystemState state_;
|
||||
};
|
||||
|
||||
} // namespace
|
||||
|
||||
73
tests/unit/etl/SystemStateTests.cpp
Normal file
73
tests/unit/etl/SystemStateTests.cpp
Normal file
@@ -0,0 +1,73 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2026, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "util/MockPrometheus.hpp"
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/config/ConfigFileJson.hpp"
|
||||
#include "util/config/ConfigValue.hpp"
|
||||
#include "util/config/Types.hpp"
|
||||
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <fmt/format.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
using namespace etl;
|
||||
using namespace util::config;
|
||||
|
||||
struct SystemStateTest : util::prometheus::WithPrometheus {};
|
||||
|
||||
TEST_F(SystemStateTest, InitialValuesAreCorrect)
|
||||
{
|
||||
auto state = SystemState{};
|
||||
|
||||
EXPECT_FALSE(state.isStrictReadonly);
|
||||
EXPECT_FALSE(state.isWriting);
|
||||
EXPECT_TRUE(state.isLoadingCache);
|
||||
EXPECT_FALSE(state.isAmendmentBlocked);
|
||||
EXPECT_FALSE(state.isCorruptionDetected);
|
||||
EXPECT_FALSE(state.isWriterDecidingFallback);
|
||||
}
|
||||
|
||||
struct SystemStateReadOnlyTest : util::prometheus::WithPrometheus, testing::WithParamInterface<bool> {};
|
||||
|
||||
TEST_P(SystemStateReadOnlyTest, MakeSystemStateWithReadOnly)
|
||||
{
|
||||
auto const readOnlyValue = GetParam();
|
||||
auto const configJson = boost::json::parse(fmt::format(R"JSON({{"read_only": {}}})JSON", readOnlyValue));
|
||||
|
||||
auto config = ClioConfigDefinition{{{"read_only", ConfigValue{ConfigType::Boolean}}}};
|
||||
auto const configFile = ConfigFileJson{configJson.as_object()};
|
||||
auto const errors = config.parse(configFile);
|
||||
ASSERT_FALSE(errors.has_value());
|
||||
|
||||
auto state = SystemState::makeSystemState(config);
|
||||
|
||||
EXPECT_EQ(state->isStrictReadonly, readOnlyValue);
|
||||
EXPECT_FALSE(state->isWriting);
|
||||
EXPECT_TRUE(state->isLoadingCache);
|
||||
EXPECT_FALSE(state->isAmendmentBlocked);
|
||||
EXPECT_FALSE(state->isCorruptionDetected);
|
||||
EXPECT_FALSE(state->isWriterDecidingFallback);
|
||||
}
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(SystemStateTest, SystemStateReadOnlyTest, testing::Values(true, false));
|
||||
162
tests/unit/etl/WriterStateTests.cpp
Normal file
162
tests/unit/etl/WriterStateTests.cpp
Normal file
@@ -0,0 +1,162 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/WriterState.hpp"
|
||||
#include "util/MockPrometheus.hpp"
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
using namespace etl;
|
||||
using namespace testing;
|
||||
|
||||
struct WriterStateTest : util::prometheus::WithPrometheus {
|
||||
std::shared_ptr<SystemState> systemState = std::make_shared<SystemState>();
|
||||
StrictMock<MockFunction<void(SystemState::WriteCommand)>> mockWriteCommand;
|
||||
WriterState writerState{systemState};
|
||||
|
||||
WriterStateTest()
|
||||
{
|
||||
systemState->writeCommandSignal.connect(mockWriteCommand.AsStdFunction());
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(WriterStateTest, IsWritingReturnsSystemStateValue)
|
||||
{
|
||||
systemState->isWriting = false;
|
||||
EXPECT_FALSE(writerState.isWriting());
|
||||
|
||||
systemState->isWriting = true;
|
||||
EXPECT_TRUE(writerState.isWriting());
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, StartWritingEmitsStartWritingCommand)
|
||||
{
|
||||
systemState->isWriting = false;
|
||||
|
||||
EXPECT_CALL(mockWriteCommand, Call(SystemState::WriteCommand::StartWriting));
|
||||
|
||||
writerState.startWriting();
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, StartWritingDoesNothingWhenAlreadyWriting)
|
||||
{
|
||||
systemState->isWriting = true;
|
||||
|
||||
// No EXPECT_CALL - StrictMock will fail if any command is emitted
|
||||
|
||||
writerState.startWriting();
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, GiveUpWritingEmitsStopWritingCommand)
|
||||
{
|
||||
systemState->isWriting = true;
|
||||
|
||||
EXPECT_CALL(mockWriteCommand, Call(SystemState::WriteCommand::StopWriting));
|
||||
|
||||
writerState.giveUpWriting();
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, GiveUpWritingDoesNothingWhenNotWriting)
|
||||
{
|
||||
systemState->isWriting = false;
|
||||
|
||||
// No EXPECT_CALL - StrictMock will fail if any command is emitted
|
||||
|
||||
writerState.giveUpWriting();
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, IsFallbackReturnsFalseByDefault)
|
||||
{
|
||||
EXPECT_FALSE(writerState.isFallback());
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, SetWriterDecidingFallbackSetsFlag)
|
||||
{
|
||||
EXPECT_FALSE(systemState->isWriterDecidingFallback);
|
||||
|
||||
writerState.setWriterDecidingFallback();
|
||||
|
||||
EXPECT_TRUE(systemState->isWriterDecidingFallback);
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, IsFallbackReturnsSystemStateValue)
|
||||
{
|
||||
systemState->isWriterDecidingFallback = false;
|
||||
EXPECT_FALSE(writerState.isFallback());
|
||||
|
||||
systemState->isWriterDecidingFallback = true;
|
||||
EXPECT_TRUE(writerState.isFallback());
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, IsReadOnlyReturnsSystemStateValue)
|
||||
{
|
||||
systemState->isStrictReadonly = false;
|
||||
EXPECT_FALSE(writerState.isReadOnly());
|
||||
|
||||
systemState->isStrictReadonly = true;
|
||||
EXPECT_TRUE(writerState.isReadOnly());
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, IsLoadingCacheReturnsSystemStateValue)
|
||||
{
|
||||
systemState->isLoadingCache = false;
|
||||
EXPECT_FALSE(writerState.isLoadingCache());
|
||||
|
||||
systemState->isLoadingCache = true;
|
||||
EXPECT_TRUE(writerState.isLoadingCache());
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, CloneCreatesNewInstanceWithSameSystemState)
|
||||
{
|
||||
systemState->isWriting = true;
|
||||
systemState->isStrictReadonly = true;
|
||||
systemState->isLoadingCache = false;
|
||||
|
||||
auto cloned = writerState.clone();
|
||||
|
||||
ASSERT_NE(cloned.get(), &writerState);
|
||||
EXPECT_TRUE(cloned->isWriting());
|
||||
EXPECT_TRUE(cloned->isReadOnly());
|
||||
EXPECT_FALSE(cloned->isLoadingCache());
|
||||
}
|
||||
|
||||
TEST_F(WriterStateTest, ClonedInstanceSharesSystemState)
|
||||
{
|
||||
auto cloned = writerState.clone();
|
||||
|
||||
systemState->isWriting = true;
|
||||
|
||||
EXPECT_TRUE(writerState.isWriting());
|
||||
EXPECT_TRUE(cloned->isWriting());
|
||||
|
||||
systemState->isWriting = false;
|
||||
|
||||
EXPECT_FALSE(writerState.isWriting());
|
||||
EXPECT_FALSE(cloned->isWriting());
|
||||
|
||||
EXPECT_FALSE(writerState.isFallback());
|
||||
EXPECT_FALSE(cloned->isFallback());
|
||||
cloned->setWriterDecidingFallback();
|
||||
EXPECT_TRUE(writerState.isFallback());
|
||||
EXPECT_TRUE(cloned->isFallback());
|
||||
}
|
||||
@@ -187,7 +187,7 @@ TEST_P(ChannelSpawnTest, MultipleSendersMultipleReceivers)
|
||||
context_.withExecutor([this](auto& executor) {
|
||||
auto [sender, receiver] = util::Channel<int>::create(executor, 10);
|
||||
util::Mutex<std::vector<int>> receivedValues;
|
||||
std::vector<decltype(receiver)> receivers(kNUM_RECEIVERS, receiver);
|
||||
std::vector receivers(kNUM_RECEIVERS, receiver);
|
||||
|
||||
for (auto receiverId = 0uz; receiverId < kNUM_RECEIVERS; ++receiverId) {
|
||||
util::spawn(
|
||||
@@ -402,7 +402,7 @@ TEST_P(ChannelCallbackTest, MultipleSendersMultipleReceivers)
|
||||
context_.withExecutor([this](auto& executor) {
|
||||
auto [sender, receiver] = util::Channel<int>::create(executor, 10);
|
||||
util::Mutex<std::vector<int>> receivedValues;
|
||||
std::vector<decltype(receiver)> receivers(kNUM_RECEIVERS, receiver);
|
||||
std::vector receivers(kNUM_RECEIVERS, receiver);
|
||||
|
||||
for (auto receiverId = 0uz; receiverId < kNUM_RECEIVERS; ++receiverId) {
|
||||
auto& receiverRef = receivers[receiverId];
|
||||
@@ -528,8 +528,8 @@ TEST_P(ChannelCallbackTest, TryMethodsWithClosedChannel)
|
||||
context_.withExecutor([this](auto& executor) {
|
||||
std::atomic_bool testCompleted{false};
|
||||
auto [sender, receiver] = util::Channel<int>::create(executor, 3);
|
||||
auto receiverPtr = std::make_shared<decltype(receiver)>(std::move(receiver));
|
||||
auto senderPtr = std::make_shared<std::optional<decltype(sender)>>(std::move(sender));
|
||||
auto receiverPtr = std::make_shared<util::Channel<int>::Receiver>(std::move(receiver));
|
||||
auto senderPtr = std::make_shared<std::optional<util::Channel<int>::Sender>>(std::move(sender));
|
||||
|
||||
boost::asio::post(executor, [receiverPtr, senderPtr, &testCompleted]() {
|
||||
EXPECT_TRUE(senderPtr->value().trySend(100));
|
||||
|
||||
Reference in New Issue
Block a user