mirror of
https://github.com/XRPLF/clio.git
synced 2025-11-27 07:05:54 +00:00
@@ -162,6 +162,7 @@ if(BUILD_TESTS)
|
|||||||
unittests/rpc/handlers/LedgerEntryTest.cpp
|
unittests/rpc/handlers/LedgerEntryTest.cpp
|
||||||
unittests/rpc/handlers/LedgerRangeTest.cpp
|
unittests/rpc/handlers/LedgerRangeTest.cpp
|
||||||
unittests/rpc/handlers/NoRippleCheckTest.cpp
|
unittests/rpc/handlers/NoRippleCheckTest.cpp
|
||||||
|
unittests/rpc/handlers/ServerInfoTest.cpp
|
||||||
unittests/rpc/handlers/PingTest.cpp
|
unittests/rpc/handlers/PingTest.cpp
|
||||||
unittests/rpc/handlers/RandomTest.cpp
|
unittests/rpc/handlers/RandomTest.cpp
|
||||||
unittests/rpc/handlers/NFTInfoTest.cpp
|
unittests/rpc/handlers/NFTInfoTest.cpp
|
||||||
|
|||||||
@@ -37,11 +37,11 @@ class SimpleCache
|
|||||||
};
|
};
|
||||||
|
|
||||||
// counters for fetchLedgerObject(s) hit rate
|
// counters for fetchLedgerObject(s) hit rate
|
||||||
mutable std::atomic_uint32_t objectReqCounter_;
|
mutable std::atomic_uint32_t objectReqCounter_ = 0;
|
||||||
mutable std::atomic_uint32_t objectHitCounter_;
|
mutable std::atomic_uint32_t objectHitCounter_ = 0;
|
||||||
// counters for fetchSuccessorKey hit rate
|
// counters for fetchSuccessorKey hit rate
|
||||||
mutable std::atomic_uint32_t successorReqCounter_;
|
mutable std::atomic_uint32_t successorReqCounter_ = 0;
|
||||||
mutable std::atomic_uint32_t successorHitCounter_;
|
mutable std::atomic_uint32_t successorHitCounter_ = 0;
|
||||||
|
|
||||||
std::map<ripple::uint256, CacheEntry> map_;
|
std::map<ripple::uint256, CacheEntry> map_;
|
||||||
mutable std::shared_mutex mtx_;
|
mutable std::shared_mutex mtx_;
|
||||||
|
|||||||
@@ -80,7 +80,7 @@ Counters::rpcForwarded(std::string const& method)
|
|||||||
}
|
}
|
||||||
|
|
||||||
boost::json::object
|
boost::json::object
|
||||||
Counters::report()
|
Counters::report() const
|
||||||
{
|
{
|
||||||
std::shared_lock lk(mutex_);
|
std::shared_lock lk(mutex_);
|
||||||
boost::json::object obj = {};
|
boost::json::object obj = {};
|
||||||
|
|||||||
@@ -47,7 +47,7 @@ private:
|
|||||||
void
|
void
|
||||||
initializeCounter(std::string const& method);
|
initializeCounter(std::string const& method);
|
||||||
|
|
||||||
std::shared_mutex mutex_;
|
mutable std::shared_mutex mutex_;
|
||||||
std::unordered_map<std::string, MethodInfo> methodInfo_;
|
std::unordered_map<std::string, MethodInfo> methodInfo_;
|
||||||
|
|
||||||
std::reference_wrapper<const WorkQueue> workQueue_;
|
std::reference_wrapper<const WorkQueue> workQueue_;
|
||||||
@@ -65,7 +65,7 @@ public:
|
|||||||
rpcForwarded(std::string const& method);
|
rpcForwarded(std::string const& method);
|
||||||
|
|
||||||
boost::json::object
|
boost::json::object
|
||||||
report();
|
report() const;
|
||||||
};
|
};
|
||||||
|
|
||||||
} // namespace RPC
|
} // namespace RPC
|
||||||
|
|||||||
@@ -49,29 +49,39 @@ concept Requirement = requires(T a) {
|
|||||||
*/
|
*/
|
||||||
// clang-format off
|
// clang-format off
|
||||||
template <typename T>
|
template <typename T>
|
||||||
concept ContextProcess = requires(T a, typename T::Input in, typename T::Output out, Context const& y) {
|
concept ContextProcessWithInput = requires(T a, typename T::Input in, typename T::Output out, Context const& ctx) {
|
||||||
{ a.process(in, y) } -> std::same_as<HandlerReturnType<decltype(out)>>; };
|
{ a.process(in, ctx) } -> std::same_as<HandlerReturnType<decltype(out)>>;
|
||||||
|
};
|
||||||
|
|
||||||
|
template <typename T>
|
||||||
|
concept ContextProcessWithoutInput = requires(T a, typename T::Output out, Context const& ctx) {
|
||||||
|
{ a.process(ctx) } -> std::same_as<HandlerReturnType<decltype(out)>>;
|
||||||
|
};
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
concept NonContextProcess = requires(T a, typename T::Input in, typename T::Output out) {
|
concept NonContextProcess = requires(T a, typename T::Input in, typename T::Output out) {
|
||||||
{ a.process(in) } -> std::same_as<HandlerReturnType<decltype(out)>>; };
|
{ a.process(in) } -> std::same_as<HandlerReturnType<decltype(out)>>;
|
||||||
|
};
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
concept HandlerWithInput = requires(T a, typename T::Input in, typename T::Output out) {
|
concept HandlerWithInput = requires(T a) {
|
||||||
{ a.spec() } -> std::same_as<RpcSpecConstRef>; }
|
{ a.spec() } -> std::same_as<RpcSpecConstRef>;
|
||||||
and (ContextProcess<T> or NonContextProcess<T>)
|
}
|
||||||
and boost::json::has_value_to<typename T::Input>::value;
|
and (ContextProcessWithInput<T> or NonContextProcess<T>)
|
||||||
|
and boost::json::has_value_to<typename T::Input>::value;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
concept HandlerWithoutInput = requires(T a, typename T::Output out) {
|
concept HandlerWithoutInput = requires(T a, typename T::Output out) {
|
||||||
{ a.process() } -> std::same_as<HandlerReturnType<decltype(out)>>; };
|
{ a.process() } -> std::same_as<HandlerReturnType<decltype(out)>>;
|
||||||
|
}
|
||||||
|
or ContextProcessWithoutInput<T>;
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
concept Handler =
|
concept Handler =
|
||||||
(HandlerWithInput<T>
|
(
|
||||||
or
|
HandlerWithInput<T> or
|
||||||
HandlerWithoutInput<T>)
|
HandlerWithoutInput<T>
|
||||||
|
)
|
||||||
and boost::json::has_value_from<typename T::Output>::value;
|
and boost::json::has_value_from<typename T::Output>::value;
|
||||||
// clang-format on
|
// clang-format on
|
||||||
|
|
||||||
} // namespace RPCng
|
} // namespace RPCng
|
||||||
|
|||||||
@@ -63,8 +63,10 @@ struct Context
|
|||||||
{
|
{
|
||||||
// TODO: we shall change yield_context to const yield_context after we
|
// TODO: we shall change yield_context to const yield_context after we
|
||||||
// update backend interfaces to use const& yield
|
// update backend interfaces to use const& yield
|
||||||
const std::reference_wrapper<boost::asio::yield_context> yield;
|
std::reference_wrapper<boost::asio::yield_context> yield;
|
||||||
const std::shared_ptr<WsBase> session;
|
std::shared_ptr<WsBase const> session;
|
||||||
|
bool isAdmin = false;
|
||||||
|
std::string clientIp;
|
||||||
};
|
};
|
||||||
|
|
||||||
inline void
|
inline void
|
||||||
|
|||||||
@@ -64,12 +64,24 @@ struct DefaultProcessor final
|
|||||||
}
|
}
|
||||||
else if constexpr (HandlerWithoutInput<HandlerType>)
|
else if constexpr (HandlerWithoutInput<HandlerType>)
|
||||||
{
|
{
|
||||||
|
using OutType = HandlerReturnType<typename HandlerType::Output>;
|
||||||
|
|
||||||
// no input to pass, ignore the value
|
// no input to pass, ignore the value
|
||||||
|
if constexpr (ContextProcessWithoutInput<HandlerType>)
|
||||||
|
{
|
||||||
|
if (auto const ret = handler.process(*ctx); not ret)
|
||||||
|
return Error{ret.error()}; // forward Status
|
||||||
|
else
|
||||||
|
return value_from(ret.value());
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
if (auto const ret = handler.process(); not ret)
|
if (auto const ret = handler.process(); not ret)
|
||||||
return Error{ret.error()}; // forward Status
|
return Error{ret.error()}; // forward Status
|
||||||
else
|
else
|
||||||
return value_from(ret.value());
|
return value_from(ret.value());
|
||||||
}
|
}
|
||||||
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
// when concept HandlerWithInput and HandlerWithoutInput not cover
|
// when concept HandlerWithInput and HandlerWithoutInput not cover
|
||||||
|
|||||||
243
src/rpc/ngHandlers/ServerInfo.h
Normal file
243
src/rpc/ngHandlers/ServerInfo.h
Normal file
@@ -0,0 +1,243 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of clio: https://github.com/XRPLF/clio
|
||||||
|
Copyright (c) 2023, 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 <backend/BackendInterface.h>
|
||||||
|
#include <backend/DBHelpers.h>
|
||||||
|
#include <main/Build.h>
|
||||||
|
#include <rpc/Errors.h>
|
||||||
|
#include <rpc/RPCHelpers.h>
|
||||||
|
#include <rpc/common/Types.h>
|
||||||
|
#include <rpc/common/Validators.h>
|
||||||
|
|
||||||
|
#include <fmt/core.h>
|
||||||
|
|
||||||
|
class SubscriptionManager;
|
||||||
|
class ReportingETL;
|
||||||
|
class ETLLoadBalancer;
|
||||||
|
|
||||||
|
namespace RPC {
|
||||||
|
class Counters;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace RPCng {
|
||||||
|
|
||||||
|
template <
|
||||||
|
typename SubscriptionManagerType,
|
||||||
|
typename ETLLoadBalancerType,
|
||||||
|
typename ReportingETLType,
|
||||||
|
typename CountersType>
|
||||||
|
class BaseServerInfoHandler
|
||||||
|
{
|
||||||
|
std::shared_ptr<BackendInterface> backend_;
|
||||||
|
std::shared_ptr<SubscriptionManagerType> subscriptions_;
|
||||||
|
std::shared_ptr<ETLLoadBalancerType> balancer_;
|
||||||
|
std::shared_ptr<ReportingETLType const> etl_;
|
||||||
|
std::reference_wrapper<CountersType const> counters_;
|
||||||
|
|
||||||
|
public:
|
||||||
|
struct AdminSection
|
||||||
|
{
|
||||||
|
boost::json::object counters = {};
|
||||||
|
boost::json::object subscriptions = {};
|
||||||
|
boost::json::object etl = {};
|
||||||
|
};
|
||||||
|
|
||||||
|
struct ValidatedLedgerSection
|
||||||
|
{
|
||||||
|
uint32_t age = 0;
|
||||||
|
std::string hash = {};
|
||||||
|
ripple::LedgerIndex seq = {};
|
||||||
|
std::optional<ripple::Fees> fees = std::nullopt;
|
||||||
|
};
|
||||||
|
|
||||||
|
struct CacheSection
|
||||||
|
{
|
||||||
|
std::size_t size = 0;
|
||||||
|
bool isFull = false;
|
||||||
|
ripple::LedgerIndex latestLedgerSeq = {};
|
||||||
|
float objectHitRate = 1.0;
|
||||||
|
float successorHitRate = 1.0;
|
||||||
|
};
|
||||||
|
|
||||||
|
struct InfoSection
|
||||||
|
{
|
||||||
|
std::optional<AdminSection> adminSection = std::nullopt;
|
||||||
|
std::string completeLedgers = {};
|
||||||
|
uint32_t loadFactor = 1u;
|
||||||
|
std::string clioVersion = Build::getClioVersionString();
|
||||||
|
std::optional<boost::json::object> rippledInfo = std::nullopt;
|
||||||
|
ValidatedLedgerSection validatedLedger = {};
|
||||||
|
CacheSection cache = {};
|
||||||
|
};
|
||||||
|
|
||||||
|
struct Output
|
||||||
|
{
|
||||||
|
InfoSection info = {};
|
||||||
|
|
||||||
|
// validated should be sent via framework
|
||||||
|
bool validated = true;
|
||||||
|
};
|
||||||
|
|
||||||
|
using Result = HandlerReturnType<Output>;
|
||||||
|
|
||||||
|
BaseServerInfoHandler(
|
||||||
|
std::shared_ptr<BackendInterface> const& backend,
|
||||||
|
std::shared_ptr<SubscriptionManagerType> const& subscriptions,
|
||||||
|
std::shared_ptr<ETLLoadBalancerType> const& balancer,
|
||||||
|
std::shared_ptr<ReportingETLType const> const& etl,
|
||||||
|
CountersType const& counters)
|
||||||
|
: backend_(backend)
|
||||||
|
, subscriptions_(subscriptions)
|
||||||
|
, balancer_(balancer)
|
||||||
|
, etl_(etl)
|
||||||
|
, counters_(std::cref(counters))
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
Result
|
||||||
|
process(Context const& ctx) const
|
||||||
|
{
|
||||||
|
using namespace RPC;
|
||||||
|
using namespace std::chrono;
|
||||||
|
|
||||||
|
auto const range = backend_->fetchLedgerRange();
|
||||||
|
|
||||||
|
// TODO: remove this check in https://github.com/XRPLF/clio/issues/592
|
||||||
|
// note: this should happen on framework level.
|
||||||
|
if (not range.has_value())
|
||||||
|
return Error{Status{RippledError::rpcNOT_READY, "emptyDatabase", "The server has no data in the database"}};
|
||||||
|
|
||||||
|
auto const lgrInfo = backend_->fetchLedgerBySequence(range->maxSequence, ctx.yield);
|
||||||
|
if (not lgrInfo.has_value())
|
||||||
|
return Error{Status{RippledError::rpcINTERNAL}};
|
||||||
|
|
||||||
|
auto const fees = backend_->fetchFees(lgrInfo->seq, ctx.yield);
|
||||||
|
if (not fees.has_value())
|
||||||
|
return Error{Status{RippledError::rpcINTERNAL}};
|
||||||
|
|
||||||
|
auto const sinceEpoch = duration_cast<seconds>(system_clock::now().time_since_epoch()).count();
|
||||||
|
auto const age = static_cast<int32_t>(sinceEpoch) -
|
||||||
|
static_cast<int32_t>(lgrInfo->closeTime.time_since_epoch().count()) -
|
||||||
|
static_cast<int32_t>(rippleEpochStart);
|
||||||
|
|
||||||
|
Output output;
|
||||||
|
|
||||||
|
output.info.completeLedgers = fmt::format("{}-{}", range->minSequence, range->maxSequence);
|
||||||
|
if (ctx.isAdmin)
|
||||||
|
output.info.adminSection = {counters_.get().report(), subscriptions_->report(), etl_->getInfo()};
|
||||||
|
|
||||||
|
auto const serverInfoRippled =
|
||||||
|
balancer_->forwardToRippled({{"command", "server_info"}}, ctx.clientIp, ctx.yield);
|
||||||
|
if (serverInfoRippled && !serverInfoRippled->contains(JS(error)))
|
||||||
|
{
|
||||||
|
if (serverInfoRippled->contains(JS(result)) &&
|
||||||
|
serverInfoRippled->at(JS(result)).as_object().contains(JS(info)))
|
||||||
|
{
|
||||||
|
output.info.rippledInfo = serverInfoRippled->at(JS(result)).as_object().at(JS(info)).as_object();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
output.info.validatedLedger.age = age < 0 ? 0 : age;
|
||||||
|
output.info.validatedLedger.hash = ripple::strHex(lgrInfo->hash);
|
||||||
|
output.info.validatedLedger.seq = lgrInfo->seq;
|
||||||
|
output.info.validatedLedger.fees = fees;
|
||||||
|
|
||||||
|
output.info.cache.size = backend_->cache().size();
|
||||||
|
output.info.cache.isFull = backend_->cache().isFull();
|
||||||
|
output.info.cache.latestLedgerSeq = backend_->cache().latestLedgerSequence();
|
||||||
|
output.info.cache.objectHitRate = backend_->cache().getObjectHitRate();
|
||||||
|
output.info.cache.successorHitRate = backend_->cache().getSuccessorHitRate();
|
||||||
|
|
||||||
|
return output;
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
friend void
|
||||||
|
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, Output const& output)
|
||||||
|
{
|
||||||
|
jv = {
|
||||||
|
{JS(info), output.info},
|
||||||
|
{JS(validated), output.validated},
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
friend void
|
||||||
|
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, InfoSection const& info)
|
||||||
|
{
|
||||||
|
jv = {
|
||||||
|
{JS(complete_ledgers), info.completeLedgers},
|
||||||
|
{JS(load_factor), info.loadFactor},
|
||||||
|
{"clio_version", info.clioVersion},
|
||||||
|
{JS(validated_ledger), info.validatedLedger},
|
||||||
|
{"cache", info.cache},
|
||||||
|
};
|
||||||
|
|
||||||
|
if (info.rippledInfo)
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
auto const& rippledInfo = info.rippledInfo.value();
|
||||||
|
jv.as_object()[JS(load_factor)] = rippledInfo.at(JS(load_factor));
|
||||||
|
jv.as_object()[JS(validation_quorum)] = rippledInfo.at(JS(validation_quorum));
|
||||||
|
jv.as_object()["rippled_version"] = rippledInfo.at(JS(build_version));
|
||||||
|
}
|
||||||
|
catch (std::exception const&)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (info.adminSection)
|
||||||
|
{
|
||||||
|
jv.as_object()["etl"] = info.adminSection->etl;
|
||||||
|
jv.as_object()[JS(counters)] = info.adminSection->counters;
|
||||||
|
jv.as_object()[JS(counters)].as_object()["subscriptions"] = info.adminSection->subscriptions;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
friend void
|
||||||
|
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, ValidatedLedgerSection const& validated)
|
||||||
|
{
|
||||||
|
jv = {
|
||||||
|
{JS(age), validated.age},
|
||||||
|
{JS(hash), validated.hash},
|
||||||
|
{JS(seq), validated.seq},
|
||||||
|
{JS(base_fee_xrp), validated.fees->base.decimalXRP()},
|
||||||
|
{JS(reserve_base_xrp), validated.fees->reserve.decimalXRP()},
|
||||||
|
{JS(reserve_inc_xrp), validated.fees->increment.decimalXRP()},
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
friend void
|
||||||
|
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, CacheSection const& cache)
|
||||||
|
{
|
||||||
|
jv = {
|
||||||
|
{"size", cache.size},
|
||||||
|
{"is_full", cache.isFull},
|
||||||
|
{"latest_ledger_seq", cache.latestLedgerSeq},
|
||||||
|
{"object_hit_rate", cache.objectHitRate},
|
||||||
|
{"successor_hit_rate", cache.successorHitRate},
|
||||||
|
};
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using ServerInfoHandler = BaseServerInfoHandler<SubscriptionManager, ETLLoadBalancer, ReportingETL, RPC::Counters>;
|
||||||
|
|
||||||
|
} // namespace RPCng
|
||||||
@@ -98,7 +98,7 @@ public:
|
|||||||
publish(std::shared_ptr<Message> const& message, Key const& key);
|
publish(std::shared_ptr<Message> const& message, Key const& key);
|
||||||
|
|
||||||
std::uint64_t
|
std::uint64_t
|
||||||
count()
|
count() const
|
||||||
{
|
{
|
||||||
return subCount_.load();
|
return subCount_.load();
|
||||||
}
|
}
|
||||||
@@ -330,7 +330,7 @@ public:
|
|||||||
cleanup(session_ptr session);
|
cleanup(session_ptr session);
|
||||||
|
|
||||||
boost::json::object
|
boost::json::object
|
||||||
report()
|
report() const
|
||||||
{
|
{
|
||||||
boost::json::object counts = {};
|
boost::json::object counts = {};
|
||||||
|
|
||||||
|
|||||||
391
unittests/rpc/handlers/ServerInfoTest.cpp
Normal file
391
unittests/rpc/handlers/ServerInfoTest.cpp
Normal file
@@ -0,0 +1,391 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of clio: https://github.com/XRPLF/clio
|
||||||
|
Copyright (c) 2023, 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 <rpc/common/AnyHandler.h>
|
||||||
|
#include <rpc/ngHandlers/ServerInfo.h>
|
||||||
|
#include <util/Fixtures.h>
|
||||||
|
#include <util/TestObject.h>
|
||||||
|
|
||||||
|
using namespace RPCng;
|
||||||
|
namespace json = boost::json;
|
||||||
|
using namespace testing;
|
||||||
|
|
||||||
|
using TestServerInfoHandler =
|
||||||
|
BaseServerInfoHandler<MockSubscriptionManager, MockETLLoadBalancer, MockReportingETL, MockCounters>;
|
||||||
|
|
||||||
|
constexpr static auto LEDGERHASH = "4BC50C9B0D8515D3EAAE1E74B29A95804346C491EE1A95BF25E4AAB854A6A652";
|
||||||
|
constexpr static auto CLIENTIP = "1.1.1.1";
|
||||||
|
|
||||||
|
class RPCServerInfoHandlerTest : public HandlerBaseTest,
|
||||||
|
public MockETLLoadBalancerTest,
|
||||||
|
public MockSubscriptionManagerTest,
|
||||||
|
public MockReportingETLTest,
|
||||||
|
public MockCountersTest
|
||||||
|
{
|
||||||
|
protected:
|
||||||
|
void
|
||||||
|
SetUp() override
|
||||||
|
{
|
||||||
|
HandlerBaseTest::SetUp();
|
||||||
|
MockETLLoadBalancerTest::SetUp();
|
||||||
|
MockSubscriptionManagerTest::SetUp();
|
||||||
|
MockReportingETLTest::SetUp();
|
||||||
|
MockCountersTest::SetUp();
|
||||||
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
TearDown() override
|
||||||
|
{
|
||||||
|
MockCountersTest::TearDown();
|
||||||
|
MockReportingETLTest::TearDown();
|
||||||
|
MockSubscriptionManagerTest::TearDown();
|
||||||
|
MockETLLoadBalancerTest::TearDown();
|
||||||
|
HandlerBaseTest::TearDown();
|
||||||
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
validateNormalOutput(RPCng::ReturnType const& output)
|
||||||
|
{
|
||||||
|
ASSERT_TRUE(output);
|
||||||
|
auto const& result = output.value().as_object();
|
||||||
|
EXPECT_TRUE(result.contains("info"));
|
||||||
|
|
||||||
|
auto const& info = result.at("info").as_object();
|
||||||
|
EXPECT_TRUE(info.contains("complete_ledgers"));
|
||||||
|
EXPECT_STREQ(info.at("complete_ledgers").as_string().c_str(), "10-30");
|
||||||
|
EXPECT_TRUE(info.contains("load_factor"));
|
||||||
|
EXPECT_TRUE(info.contains("clio_version"));
|
||||||
|
EXPECT_TRUE(info.contains("validated_ledger"));
|
||||||
|
|
||||||
|
auto const& validated = info.at("validated_ledger").as_object();
|
||||||
|
EXPECT_TRUE(validated.contains("age"));
|
||||||
|
EXPECT_EQ(validated.at("age").as_uint64(), 3u);
|
||||||
|
EXPECT_TRUE(validated.contains("hash"));
|
||||||
|
EXPECT_STREQ(validated.at("hash").as_string().c_str(), LEDGERHASH);
|
||||||
|
EXPECT_TRUE(validated.contains("seq"));
|
||||||
|
EXPECT_EQ(validated.at("seq").as_uint64(), 30u);
|
||||||
|
EXPECT_TRUE(validated.contains("base_fee_xrp"));
|
||||||
|
EXPECT_EQ(validated.at("base_fee_xrp").as_double(), 1e-06);
|
||||||
|
EXPECT_TRUE(validated.contains("reserve_base_xrp"));
|
||||||
|
EXPECT_EQ(validated.at("reserve_base_xrp").as_double(), 3e-06);
|
||||||
|
EXPECT_TRUE(validated.contains("reserve_inc_xrp"));
|
||||||
|
EXPECT_EQ(validated.at("reserve_inc_xrp").as_double(), 2e-06);
|
||||||
|
|
||||||
|
auto const& cache = info.at("cache").as_object();
|
||||||
|
EXPECT_TRUE(cache.contains("size"));
|
||||||
|
EXPECT_TRUE(cache.contains("is_full"));
|
||||||
|
EXPECT_TRUE(cache.contains("latest_ledger_seq"));
|
||||||
|
EXPECT_TRUE(cache.contains("object_hit_rate"));
|
||||||
|
EXPECT_TRUE(cache.contains("successor_hit_rate"));
|
||||||
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
validateAdminOutput(RPCng::ReturnType const& output)
|
||||||
|
{
|
||||||
|
auto const& result = output.value().as_object();
|
||||||
|
auto const& info = result.at("info").as_object();
|
||||||
|
EXPECT_TRUE(info.contains("etl"));
|
||||||
|
EXPECT_TRUE(info.contains("counters"));
|
||||||
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
validateRippledOutput(RPCng::ReturnType const& output)
|
||||||
|
{
|
||||||
|
auto const& result = output.value().as_object();
|
||||||
|
auto const& info = result.at("info").as_object();
|
||||||
|
EXPECT_TRUE(info.contains("load_factor"));
|
||||||
|
EXPECT_EQ(info.at("load_factor").as_int64(), 234);
|
||||||
|
EXPECT_TRUE(info.contains("validation_quorum"));
|
||||||
|
EXPECT_EQ(info.at("validation_quorum").as_int64(), 456);
|
||||||
|
EXPECT_TRUE(info.contains("rippled_version"));
|
||||||
|
EXPECT_STREQ(info.at("rippled_version").as_string().c_str(), "1234");
|
||||||
|
}
|
||||||
|
|
||||||
|
void
|
||||||
|
validateCacheOutput(RPCng::ReturnType const& output)
|
||||||
|
{
|
||||||
|
auto const& result = output.value().as_object();
|
||||||
|
auto const& info = result.at("info").as_object();
|
||||||
|
auto const& cache = info.at("cache").as_object();
|
||||||
|
EXPECT_EQ(cache.at("size").as_uint64(), 1u);
|
||||||
|
EXPECT_EQ(cache.at("is_full").as_bool(), false);
|
||||||
|
EXPECT_EQ(cache.at("latest_ledger_seq").as_uint64(), 30u);
|
||||||
|
EXPECT_EQ(cache.at("object_hit_rate").as_double(), 1.0);
|
||||||
|
EXPECT_EQ(cache.at("successor_hit_rate").as_double(), 1.0);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
TEST_F(RPCServerInfoHandlerTest, NoRangeErrorsOutWithNotReady)
|
||||||
|
{
|
||||||
|
auto const handler = AnyHandler{TestServerInfoHandler{
|
||||||
|
mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}};
|
||||||
|
|
||||||
|
runSpawn([&](auto& yield) {
|
||||||
|
auto const req = json::parse("{}");
|
||||||
|
auto const output = handler.process(req, Context{std::ref(yield)});
|
||||||
|
|
||||||
|
ASSERT_FALSE(output);
|
||||||
|
auto const err = RPC::makeError(output.error());
|
||||||
|
EXPECT_EQ(err.at("error").as_string(), "emptyDatabase");
|
||||||
|
EXPECT_EQ(err.at("error_message").as_string(), "The server has no data in the database");
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
TEST_F(RPCServerInfoHandlerTest, NoLedgerInfoErrorsOutWithInternal)
|
||||||
|
{
|
||||||
|
MockBackend* rawBackendPtr = static_cast<MockBackend*>(mockBackendPtr.get());
|
||||||
|
|
||||||
|
mockBackendPtr->updateRange(10); // min
|
||||||
|
mockBackendPtr->updateRange(30); // max
|
||||||
|
|
||||||
|
ON_CALL(*rawBackendPtr, fetchLedgerBySequence).WillByDefault(Return(std::nullopt));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, fetchLedgerBySequence).Times(1);
|
||||||
|
|
||||||
|
auto const handler = AnyHandler{TestServerInfoHandler{
|
||||||
|
mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}};
|
||||||
|
|
||||||
|
runSpawn([&](auto& yield) {
|
||||||
|
auto const req = json::parse("{}");
|
||||||
|
auto const output = handler.process(req, Context{std::ref(yield)});
|
||||||
|
|
||||||
|
ASSERT_FALSE(output);
|
||||||
|
auto const err = RPC::makeError(output.error());
|
||||||
|
EXPECT_EQ(err.at("error").as_string(), "internal");
|
||||||
|
EXPECT_EQ(err.at("error_message").as_string(), "Internal error.");
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
TEST_F(RPCServerInfoHandlerTest, NoFeesErrorsOutWithInternal)
|
||||||
|
{
|
||||||
|
MockBackend* rawBackendPtr = static_cast<MockBackend*>(mockBackendPtr.get());
|
||||||
|
|
||||||
|
mockBackendPtr->updateRange(10); // min
|
||||||
|
mockBackendPtr->updateRange(30); // max
|
||||||
|
|
||||||
|
auto const ledgerinfo = CreateLedgerInfo(LEDGERHASH, 30);
|
||||||
|
ON_CALL(*rawBackendPtr, fetchLedgerBySequence).WillByDefault(Return(ledgerinfo));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, fetchLedgerBySequence).Times(1);
|
||||||
|
ON_CALL(*rawBackendPtr, doFetchLedgerObject).WillByDefault(Return(std::nullopt));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, doFetchLedgerObject).Times(1);
|
||||||
|
|
||||||
|
auto const handler = AnyHandler{TestServerInfoHandler{
|
||||||
|
mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}};
|
||||||
|
|
||||||
|
runSpawn([&](auto& yield) {
|
||||||
|
auto const req = json::parse("{}");
|
||||||
|
auto const output = handler.process(req, Context{std::ref(yield)});
|
||||||
|
|
||||||
|
ASSERT_FALSE(output);
|
||||||
|
auto const err = RPC::makeError(output.error());
|
||||||
|
EXPECT_EQ(err.at("error").as_string(), "internal");
|
||||||
|
EXPECT_EQ(err.at("error_message").as_string(), "Internal error.");
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
TEST_F(RPCServerInfoHandlerTest, DefaultOutputIsPresent)
|
||||||
|
{
|
||||||
|
MockBackend* rawBackendPtr = static_cast<MockBackend*>(mockBackendPtr.get());
|
||||||
|
MockETLLoadBalancer* rawBalancerPtr = static_cast<MockETLLoadBalancer*>(mockLoadBalancerPtr.get());
|
||||||
|
|
||||||
|
mockBackendPtr->updateRange(10); // min
|
||||||
|
mockBackendPtr->updateRange(30); // max
|
||||||
|
|
||||||
|
auto const ledgerinfo = CreateLedgerInfo(LEDGERHASH, 30, 3); // 3 seconds old
|
||||||
|
ON_CALL(*rawBackendPtr, fetchLedgerBySequence).WillByDefault(Return(ledgerinfo));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, fetchLedgerBySequence).Times(1);
|
||||||
|
|
||||||
|
auto const feeBlob = CreateFeeSettingBlob(1, 2, 3, 4, 0);
|
||||||
|
ON_CALL(*rawBackendPtr, doFetchLedgerObject).WillByDefault(Return(feeBlob));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, doFetchLedgerObject).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawBalancerPtr, forwardToRippled).WillByDefault(Return(std::nullopt));
|
||||||
|
EXPECT_CALL(*rawBalancerPtr, forwardToRippled(testing::_, testing::Eq(CLIENTIP), testing::_)).Times(1);
|
||||||
|
|
||||||
|
auto const handler = AnyHandler{TestServerInfoHandler{
|
||||||
|
mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}};
|
||||||
|
|
||||||
|
runSpawn([&](auto& yield) {
|
||||||
|
auto const req = json::parse("{}");
|
||||||
|
auto const output = handler.process(req, Context{std::ref(yield), {}, false, CLIENTIP});
|
||||||
|
|
||||||
|
validateNormalOutput(output);
|
||||||
|
|
||||||
|
// no admin section present by default
|
||||||
|
auto const& result = output.value().as_object();
|
||||||
|
auto const& info = result.at("info").as_object();
|
||||||
|
EXPECT_FALSE(info.contains("etl"));
|
||||||
|
EXPECT_FALSE(info.contains("counters"));
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
TEST_F(RPCServerInfoHandlerTest, AdminSectionPresentWhenAdminFlagIsSet)
|
||||||
|
{
|
||||||
|
MockBackend* rawBackendPtr = static_cast<MockBackend*>(mockBackendPtr.get());
|
||||||
|
MockETLLoadBalancer* rawBalancerPtr = static_cast<MockETLLoadBalancer*>(mockLoadBalancerPtr.get());
|
||||||
|
MockCounters* rawCountersPtr = static_cast<MockCounters*>(mockCountersPtr.get());
|
||||||
|
MockSubscriptionManager* rawSubscriptionManagerPtr =
|
||||||
|
static_cast<MockSubscriptionManager*>(mockSubscriptionManagerPtr.get());
|
||||||
|
MockReportingETL* rawReportingETLPtr = static_cast<MockReportingETL*>(mockReportingETLPtr.get());
|
||||||
|
|
||||||
|
mockBackendPtr->updateRange(10); // min
|
||||||
|
mockBackendPtr->updateRange(30); // max
|
||||||
|
|
||||||
|
auto const empty = boost::json::object{};
|
||||||
|
auto const ledgerinfo = CreateLedgerInfo(LEDGERHASH, 30, 3); // 3 seconds old
|
||||||
|
ON_CALL(*rawBackendPtr, fetchLedgerBySequence).WillByDefault(Return(ledgerinfo));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, fetchLedgerBySequence).Times(1);
|
||||||
|
|
||||||
|
auto const feeBlob = CreateFeeSettingBlob(1, 2, 3, 4, 0);
|
||||||
|
ON_CALL(*rawBackendPtr, doFetchLedgerObject).WillByDefault(Return(feeBlob));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, doFetchLedgerObject).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawBalancerPtr, forwardToRippled).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawBalancerPtr, forwardToRippled).Times(1);
|
||||||
|
|
||||||
|
// admin calls
|
||||||
|
ON_CALL(*rawCountersPtr, report).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawCountersPtr, report).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawSubscriptionManagerPtr, report).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawSubscriptionManagerPtr, report).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawReportingETLPtr, getInfo).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawReportingETLPtr, getInfo).Times(1);
|
||||||
|
|
||||||
|
auto const handler = AnyHandler{TestServerInfoHandler{
|
||||||
|
mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}};
|
||||||
|
|
||||||
|
runSpawn([&](auto& yield) {
|
||||||
|
auto const req = json::parse("{}");
|
||||||
|
auto const output = handler.process(req, Context{std::ref(yield), {}, true});
|
||||||
|
|
||||||
|
validateNormalOutput(output);
|
||||||
|
validateAdminOutput(output);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
TEST_F(RPCServerInfoHandlerTest, RippledForwardedValuesPresent)
|
||||||
|
{
|
||||||
|
MockBackend* rawBackendPtr = static_cast<MockBackend*>(mockBackendPtr.get());
|
||||||
|
MockETLLoadBalancer* rawBalancerPtr = static_cast<MockETLLoadBalancer*>(mockLoadBalancerPtr.get());
|
||||||
|
MockCounters* rawCountersPtr = static_cast<MockCounters*>(mockCountersPtr.get());
|
||||||
|
MockSubscriptionManager* rawSubscriptionManagerPtr =
|
||||||
|
static_cast<MockSubscriptionManager*>(mockSubscriptionManagerPtr.get());
|
||||||
|
MockReportingETL* rawReportingETLPtr = static_cast<MockReportingETL*>(mockReportingETLPtr.get());
|
||||||
|
|
||||||
|
mockBackendPtr->updateRange(10); // min
|
||||||
|
mockBackendPtr->updateRange(30); // max
|
||||||
|
|
||||||
|
auto const empty = boost::json::object{};
|
||||||
|
auto const ledgerinfo = CreateLedgerInfo(LEDGERHASH, 30, 3); // 3 seconds old
|
||||||
|
ON_CALL(*rawBackendPtr, fetchLedgerBySequence).WillByDefault(Return(ledgerinfo));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, fetchLedgerBySequence).Times(1);
|
||||||
|
|
||||||
|
auto const feeBlob = CreateFeeSettingBlob(1, 2, 3, 4, 0);
|
||||||
|
ON_CALL(*rawBackendPtr, doFetchLedgerObject).WillByDefault(Return(feeBlob));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, doFetchLedgerObject).Times(1);
|
||||||
|
|
||||||
|
auto const rippledObj = boost::json::parse(R"({
|
||||||
|
"result": {
|
||||||
|
"info": {
|
||||||
|
"build_version": "1234",
|
||||||
|
"validation_quorum": 456,
|
||||||
|
"load_factor": 234
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})");
|
||||||
|
ON_CALL(*rawBalancerPtr, forwardToRippled).WillByDefault(Return(rippledObj.as_object()));
|
||||||
|
EXPECT_CALL(*rawBalancerPtr, forwardToRippled).Times(1);
|
||||||
|
|
||||||
|
// admin calls
|
||||||
|
ON_CALL(*rawCountersPtr, report).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawCountersPtr, report).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawSubscriptionManagerPtr, report).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawSubscriptionManagerPtr, report).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawReportingETLPtr, getInfo).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawReportingETLPtr, getInfo).Times(1);
|
||||||
|
|
||||||
|
auto const handler = AnyHandler{TestServerInfoHandler{
|
||||||
|
mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}};
|
||||||
|
|
||||||
|
runSpawn([&](auto& yield) {
|
||||||
|
auto const req = json::parse("{}");
|
||||||
|
auto const output = handler.process(req, Context{std::ref(yield), {}, true});
|
||||||
|
|
||||||
|
validateNormalOutput(output);
|
||||||
|
validateAdminOutput(output);
|
||||||
|
validateRippledOutput(output);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
TEST_F(RPCServerInfoHandlerTest, RippledForwardedValuesMissingNoExceptionThrown)
|
||||||
|
{
|
||||||
|
MockBackend* rawBackendPtr = static_cast<MockBackend*>(mockBackendPtr.get());
|
||||||
|
MockETLLoadBalancer* rawBalancerPtr = static_cast<MockETLLoadBalancer*>(mockLoadBalancerPtr.get());
|
||||||
|
MockCounters* rawCountersPtr = static_cast<MockCounters*>(mockCountersPtr.get());
|
||||||
|
MockSubscriptionManager* rawSubscriptionManagerPtr =
|
||||||
|
static_cast<MockSubscriptionManager*>(mockSubscriptionManagerPtr.get());
|
||||||
|
MockReportingETL* rawReportingETLPtr = static_cast<MockReportingETL*>(mockReportingETLPtr.get());
|
||||||
|
|
||||||
|
mockBackendPtr->updateRange(10); // min
|
||||||
|
mockBackendPtr->updateRange(30); // max
|
||||||
|
|
||||||
|
auto const empty = boost::json::object{};
|
||||||
|
auto const ledgerinfo = CreateLedgerInfo(LEDGERHASH, 30, 3); // 3 seconds old
|
||||||
|
ON_CALL(*rawBackendPtr, fetchLedgerBySequence).WillByDefault(Return(ledgerinfo));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, fetchLedgerBySequence).Times(1);
|
||||||
|
|
||||||
|
auto const feeBlob = CreateFeeSettingBlob(1, 2, 3, 4, 0);
|
||||||
|
ON_CALL(*rawBackendPtr, doFetchLedgerObject).WillByDefault(Return(feeBlob));
|
||||||
|
EXPECT_CALL(*rawBackendPtr, doFetchLedgerObject).Times(1);
|
||||||
|
|
||||||
|
auto const rippledObj = boost::json::parse(R"({
|
||||||
|
"result": {
|
||||||
|
"info": {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})");
|
||||||
|
ON_CALL(*rawBalancerPtr, forwardToRippled).WillByDefault(Return(rippledObj.as_object()));
|
||||||
|
EXPECT_CALL(*rawBalancerPtr, forwardToRippled).Times(1);
|
||||||
|
|
||||||
|
// admin calls
|
||||||
|
ON_CALL(*rawCountersPtr, report).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawCountersPtr, report).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawSubscriptionManagerPtr, report).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawSubscriptionManagerPtr, report).Times(1);
|
||||||
|
|
||||||
|
ON_CALL(*rawReportingETLPtr, getInfo).WillByDefault(Return(empty));
|
||||||
|
EXPECT_CALL(*rawReportingETLPtr, getInfo).Times(1);
|
||||||
|
|
||||||
|
auto const handler = AnyHandler{TestServerInfoHandler{
|
||||||
|
mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}};
|
||||||
|
|
||||||
|
runSpawn([&](auto& yield) {
|
||||||
|
auto const req = json::parse("{}");
|
||||||
|
auto const output = handler.process(req, Context{std::ref(yield), {}, true});
|
||||||
|
|
||||||
|
validateNormalOutput(output);
|
||||||
|
validateAdminOutput(output);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO: In the future we'd like to refactor to add mock and test for cache
|
||||||
@@ -19,16 +19,20 @@
|
|||||||
|
|
||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include "MockBackend.h"
|
||||||
|
#include "MockCounters.h"
|
||||||
|
#include "MockETLLoadBalancer.h"
|
||||||
|
#include "MockReportingETL.h"
|
||||||
|
#include "MockSubscriptionManager.h"
|
||||||
|
#include <log/Logger.h>
|
||||||
|
|
||||||
|
#include <boost/asio.hpp>
|
||||||
|
#include <gtest/gtest.h>
|
||||||
|
|
||||||
#include <ios>
|
#include <ios>
|
||||||
#include <mutex>
|
#include <mutex>
|
||||||
#include <thread>
|
#include <thread>
|
||||||
|
|
||||||
#include <boost/asio.hpp>
|
|
||||||
|
|
||||||
#include "MockBackend.h"
|
|
||||||
#include <gtest/gtest.h>
|
|
||||||
#include <log/Logger.h>
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @brief Fixture with LogService support.
|
* @brief Fixture with LogService support.
|
||||||
*/
|
*/
|
||||||
@@ -170,7 +174,7 @@ protected:
|
|||||||
};
|
};
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @brief Fixture with an mock backend
|
* @brief Fixture with a mock backend
|
||||||
*/
|
*/
|
||||||
struct MockBackendTest : virtual public NoLoggerFixture
|
struct MockBackendTest : virtual public NoLoggerFixture
|
||||||
{
|
{
|
||||||
@@ -191,11 +195,95 @@ protected:
|
|||||||
std::shared_ptr<BackendInterface> mockBackendPtr;
|
std::shared_ptr<BackendInterface> mockBackendPtr;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @brief Fixture with a mock subscription manager
|
||||||
|
*/
|
||||||
|
struct MockSubscriptionManagerTest : virtual public NoLoggerFixture
|
||||||
|
{
|
||||||
|
void
|
||||||
|
SetUp() override
|
||||||
|
{
|
||||||
|
NoLoggerFixture::SetUp();
|
||||||
|
mockSubscriptionManagerPtr = std::make_shared<MockSubscriptionManager>();
|
||||||
|
}
|
||||||
|
void
|
||||||
|
TearDown() override
|
||||||
|
{
|
||||||
|
mockSubscriptionManagerPtr.reset();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
std::shared_ptr<MockSubscriptionManager> mockSubscriptionManagerPtr;
|
||||||
|
};
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @brief Fixture with a mock etl balancer
|
||||||
|
*/
|
||||||
|
struct MockETLLoadBalancerTest : virtual public NoLoggerFixture
|
||||||
|
{
|
||||||
|
void
|
||||||
|
SetUp() override
|
||||||
|
{
|
||||||
|
NoLoggerFixture::SetUp();
|
||||||
|
mockLoadBalancerPtr = std::make_shared<MockETLLoadBalancer>();
|
||||||
|
}
|
||||||
|
void
|
||||||
|
TearDown() override
|
||||||
|
{
|
||||||
|
mockLoadBalancerPtr.reset();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
std::shared_ptr<MockETLLoadBalancer> mockLoadBalancerPtr;
|
||||||
|
};
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @brief Fixture with a mock subscription manager
|
||||||
|
*/
|
||||||
|
struct MockReportingETLTest : virtual public NoLoggerFixture
|
||||||
|
{
|
||||||
|
void
|
||||||
|
SetUp() override
|
||||||
|
{
|
||||||
|
NoLoggerFixture::SetUp();
|
||||||
|
mockReportingETLPtr = std::make_shared<MockReportingETL>();
|
||||||
|
}
|
||||||
|
void
|
||||||
|
TearDown() override
|
||||||
|
{
|
||||||
|
mockReportingETLPtr.reset();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
std::shared_ptr<MockReportingETL> mockReportingETLPtr;
|
||||||
|
};
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @brief Fixture with mock counters
|
||||||
|
*/
|
||||||
|
struct MockCountersTest : virtual public NoLoggerFixture
|
||||||
|
{
|
||||||
|
void
|
||||||
|
SetUp() override
|
||||||
|
{
|
||||||
|
NoLoggerFixture::SetUp();
|
||||||
|
mockCountersPtr = std::make_shared<MockCounters>();
|
||||||
|
}
|
||||||
|
void
|
||||||
|
TearDown() override
|
||||||
|
{
|
||||||
|
mockCountersPtr.reset();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected:
|
||||||
|
std::shared_ptr<MockCounters> mockCountersPtr;
|
||||||
|
};
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @brief Fixture with an mock backend and an embedded boost::asio context
|
* @brief Fixture with an mock backend and an embedded boost::asio context
|
||||||
* Handler unittest base class
|
* Handler unittest base class
|
||||||
*/
|
*/
|
||||||
class HandlerBaseTest : public MockBackendTest, public SyncAsioContextTest
|
struct HandlerBaseTest : public MockBackendTest, public SyncAsioContextTest
|
||||||
{
|
{
|
||||||
void
|
void
|
||||||
SetUp() override
|
SetUp() override
|
||||||
|
|||||||
@@ -33,134 +33,118 @@ public:
|
|||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::optional<ripple::LedgerInfo>,
|
std::optional<ripple::LedgerInfo>,
|
||||||
fetchLedgerBySequence,
|
fetchLedgerBySequence,
|
||||||
(std::uint32_t const sequence, boost::asio::yield_context& yield),
|
(std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::optional<ripple::LedgerInfo>,
|
std::optional<ripple::LedgerInfo>,
|
||||||
fetchLedgerByHash,
|
fetchLedgerByHash,
|
||||||
(ripple::uint256 const& hash, boost::asio::yield_context& yield),
|
(ripple::uint256 const&, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::optional<std::uint32_t>,
|
std::optional<std::uint32_t>,
|
||||||
fetchLatestLedgerSequence,
|
fetchLatestLedgerSequence,
|
||||||
(boost::asio::yield_context & yield),
|
(boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::optional<TransactionAndMetadata>,
|
std::optional<TransactionAndMetadata>,
|
||||||
fetchTransaction,
|
fetchTransaction,
|
||||||
(ripple::uint256 const& hash, boost::asio::yield_context& yield),
|
(ripple::uint256 const&, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::vector<TransactionAndMetadata>,
|
std::vector<TransactionAndMetadata>,
|
||||||
fetchTransactions,
|
fetchTransactions,
|
||||||
(std::vector<ripple::uint256> const& hashes, boost::asio::yield_context& yield),
|
(std::vector<ripple::uint256> const&, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
TransactionsAndCursor,
|
TransactionsAndCursor,
|
||||||
fetchAccountTransactions,
|
fetchAccountTransactions,
|
||||||
(ripple::AccountID const& account,
|
(ripple::AccountID const&,
|
||||||
std::uint32_t const limit,
|
std::uint32_t const,
|
||||||
bool forward,
|
bool,
|
||||||
std::optional<TransactionsCursor> const& cursor,
|
std::optional<TransactionsCursor> const&,
|
||||||
boost::asio::yield_context& yield),
|
boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::vector<TransactionAndMetadata>,
|
std::vector<TransactionAndMetadata>,
|
||||||
fetchAllTransactionsInLedger,
|
fetchAllTransactionsInLedger,
|
||||||
(std::uint32_t const ledgerSequence, boost::asio::yield_context& yield),
|
(std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::vector<ripple::uint256>,
|
std::vector<ripple::uint256>,
|
||||||
fetchAllTransactionHashesInLedger,
|
fetchAllTransactionHashesInLedger,
|
||||||
(std::uint32_t const ledgerSequence, boost::asio::yield_context& yield),
|
(std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::optional<NFT>,
|
std::optional<NFT>,
|
||||||
fetchNFT,
|
fetchNFT,
|
||||||
(ripple::uint256 const& tokenID, std::uint32_t const ledgerSequence, boost::asio::yield_context& yieldd),
|
(ripple::uint256 const&, std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
TransactionsAndCursor,
|
TransactionsAndCursor,
|
||||||
fetchNFTTransactions,
|
fetchNFTTransactions,
|
||||||
(ripple::uint256 const& tokenID,
|
(ripple::uint256 const&,
|
||||||
std::uint32_t const limit,
|
std::uint32_t const,
|
||||||
bool const forward,
|
bool const,
|
||||||
std::optional<TransactionsCursor> const& cursorIn,
|
std::optional<TransactionsCursor> const&,
|
||||||
boost::asio::yield_context& yield),
|
boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::vector<Blob>,
|
std::vector<Blob>,
|
||||||
doFetchLedgerObjects,
|
doFetchLedgerObjects,
|
||||||
(std::vector<ripple::uint256> const& key, std::uint32_t const sequence, boost::asio::yield_context& yield),
|
(std::vector<ripple::uint256> const&, std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::optional<Blob>,
|
std::optional<Blob>,
|
||||||
doFetchLedgerObject,
|
doFetchLedgerObject,
|
||||||
(ripple::uint256 const& key, std::uint32_t const sequence, boost::asio::yield_context& yield),
|
(ripple::uint256 const&, std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::vector<LedgerObject>,
|
std::vector<LedgerObject>,
|
||||||
fetchLedgerDiff,
|
fetchLedgerDiff,
|
||||||
(std::uint32_t const ledgerSequence, boost::asio::yield_context& yield),
|
(std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
std::optional<ripple::uint256>,
|
std::optional<ripple::uint256>,
|
||||||
doFetchSuccessorKey,
|
doFetchSuccessorKey,
|
||||||
(ripple::uint256 key, std::uint32_t const ledgerSequence, boost::asio::yield_context& yield),
|
(ripple::uint256, std::uint32_t const, boost::asio::yield_context&),
|
||||||
(const, override));
|
(const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(std::optional<LedgerRange>, hardFetchLedgerRange, (boost::asio::yield_context&), (const, override));
|
||||||
std::optional<LedgerRange>,
|
|
||||||
hardFetchLedgerRange,
|
|
||||||
(boost::asio::yield_context & yield),
|
|
||||||
(const, override));
|
|
||||||
|
|
||||||
MOCK_METHOD(void, writeLedger, (ripple::LedgerInfo const& ledgerInfo, std::string&& ledgerHeader), (override));
|
MOCK_METHOD(void, writeLedger, (ripple::LedgerInfo const&, std::string&&), (override));
|
||||||
|
|
||||||
MOCK_METHOD(void, writeLedgerObject, (std::string && key, std::uint32_t const seq, std::string&& blob), (override));
|
MOCK_METHOD(void, writeLedgerObject, (std::string&&, std::uint32_t const, std::string&&), (override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(
|
||||||
void,
|
void,
|
||||||
writeTransaction,
|
writeTransaction,
|
||||||
(std::string && hash,
|
(std::string&&, std::uint32_t const, std::uint32_t const, std::string&&, std::string&&),
|
||||||
std::uint32_t const seq,
|
|
||||||
std::uint32_t const date,
|
|
||||||
std::string&& transaction,
|
|
||||||
std::string&& metadata),
|
|
||||||
(override));
|
(override));
|
||||||
|
|
||||||
MOCK_METHOD(void, writeNFTs, (std::vector<NFTsData> && blob), (override));
|
MOCK_METHOD(void, writeNFTs, (std::vector<NFTsData> &&), (override));
|
||||||
|
|
||||||
MOCK_METHOD(void, writeAccountTransactions, (std::vector<AccountTransactionsData> && blob), (override));
|
MOCK_METHOD(void, writeAccountTransactions, (std::vector<AccountTransactionsData> &&), (override));
|
||||||
|
|
||||||
MOCK_METHOD(void, writeNFTTransactions, (std::vector<NFTTransactionsData> && blob), (override));
|
MOCK_METHOD(void, writeNFTTransactions, (std::vector<NFTTransactionsData> &&), (override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(void, writeSuccessor, (std::string && key, std::uint32_t const, std::string&&), (override));
|
||||||
void,
|
|
||||||
writeSuccessor,
|
|
||||||
(std::string && key, std::uint32_t const seq, std::string&& successor),
|
|
||||||
(override));
|
|
||||||
|
|
||||||
MOCK_METHOD(void, startWrites, (), (const, override));
|
MOCK_METHOD(void, startWrites, (), (const, override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(bool, doOnlineDelete, (std::uint32_t, boost::asio::yield_context&), (const, override));
|
||||||
bool,
|
|
||||||
doOnlineDelete,
|
|
||||||
(std::uint32_t numLedgersToKeep, boost::asio::yield_context& yield),
|
|
||||||
(const, override));
|
|
||||||
|
|
||||||
MOCK_METHOD(bool, isTooBusy, (), (const, override));
|
MOCK_METHOD(bool, isTooBusy, (), (const, override));
|
||||||
|
|
||||||
@@ -168,11 +152,7 @@ public:
|
|||||||
|
|
||||||
MOCK_METHOD(void, close, (), (override));
|
MOCK_METHOD(void, close, (), (override));
|
||||||
|
|
||||||
MOCK_METHOD(
|
MOCK_METHOD(void, doWriteLedgerObject, (std::string&&, std::uint32_t const, std::string&&), (override));
|
||||||
void,
|
|
||||||
doWriteLedgerObject,
|
|
||||||
(std::string && key, std::uint32_t const seq, std::string&& blob),
|
|
||||||
(override));
|
|
||||||
|
|
||||||
MOCK_METHOD(bool, doFinishWrites, (), (override));
|
MOCK_METHOD(bool, doFinishWrites, (), (override));
|
||||||
};
|
};
|
||||||
|
|||||||
41
unittests/util/MockCounters.h
Normal file
41
unittests/util/MockCounters.h
Normal file
@@ -0,0 +1,41 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of clio: https://github.com/XRPLF/clio
|
||||||
|
Copyright (c) 2023, 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 <boost/json.hpp>
|
||||||
|
#include <gmock/gmock.h>
|
||||||
|
|
||||||
|
#include <chrono>
|
||||||
|
|
||||||
|
class MockCounters
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
MockCounters()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
MOCK_METHOD(void, rpcErrored, (std::string const&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, rpcComplete, (std::string const&, std::chrono::microseconds const&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, rpcForwarded, (std::string const&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(boost::json::object, report, (), (const));
|
||||||
|
};
|
||||||
53
unittests/util/MockETLLoadBalancer.h
Normal file
53
unittests/util/MockETLLoadBalancer.h
Normal file
@@ -0,0 +1,53 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of clio: https://github.com/XRPLF/clio
|
||||||
|
Copyright (c) 2023, 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/ETLSource.h>
|
||||||
|
|
||||||
|
#include <boost/asio/spawn.hpp>
|
||||||
|
#include <boost/json.hpp>
|
||||||
|
#include <gmock/gmock.h>
|
||||||
|
|
||||||
|
#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h"
|
||||||
|
#include <grpcpp/grpcpp.h>
|
||||||
|
|
||||||
|
#include <optional>
|
||||||
|
|
||||||
|
class MockETLLoadBalancer
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
MockETLLoadBalancer()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
MOCK_METHOD(void, loadInitialLedger, (std::uint32_t, bool), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(std::optional<org::xrpl::rpc::v1::GetLedgerResponse>, fetchLedger, (uint32_t, bool, bool), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(bool, shouldPropagateTxnStream, (ETLSource*), (const));
|
||||||
|
|
||||||
|
MOCK_METHOD(boost::json::value, toJson, (), (const));
|
||||||
|
|
||||||
|
MOCK_METHOD(
|
||||||
|
std::optional<boost::json::object>,
|
||||||
|
forwardToRippled,
|
||||||
|
(boost::json::object const&, std::string const&, boost::asio::yield_context&),
|
||||||
|
(const));
|
||||||
|
};
|
||||||
41
unittests/util/MockReportingETL.h
Normal file
41
unittests/util/MockReportingETL.h
Normal file
@@ -0,0 +1,41 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of clio: https://github.com/XRPLF/clio
|
||||||
|
Copyright (c) 2023, 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 <boost/json.hpp>
|
||||||
|
#include <gmock/gmock.h>
|
||||||
|
|
||||||
|
#include <chrono>
|
||||||
|
|
||||||
|
class MockReportingETL
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
MockReportingETL()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
MOCK_METHOD(boost::json::object, getInfo, (), (const));
|
||||||
|
|
||||||
|
MOCK_METHOD(std::chrono::time_point<std::chrono::system_clock>, getLastPublish, (), (const));
|
||||||
|
|
||||||
|
MOCK_METHOD(std::uint32_t, lastPublishAgeSeconds, (), (const));
|
||||||
|
|
||||||
|
MOCK_METHOD(std::uint32_t, lastCloseAgeSeconds, (), (const));
|
||||||
|
};
|
||||||
98
unittests/util/MockSubscriptionManager.h
Normal file
98
unittests/util/MockSubscriptionManager.h
Normal file
@@ -0,0 +1,98 @@
|
|||||||
|
//------------------------------------------------------------------------------
|
||||||
|
/*
|
||||||
|
This file is part of clio: https://github.com/XRPLF/clio
|
||||||
|
Copyright (c) 2023, 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 <ripple/ledger/ReadView.h>
|
||||||
|
#include <webserver/WsBase.h>
|
||||||
|
|
||||||
|
#include <boost/asio/spawn.hpp>
|
||||||
|
#include <boost/json.hpp>
|
||||||
|
#include <gmock/gmock.h>
|
||||||
|
|
||||||
|
#include <optional>
|
||||||
|
|
||||||
|
class MockSubscriptionManager
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
using session_ptr = std::shared_ptr<WsBase>;
|
||||||
|
MockSubscriptionManager()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
MOCK_METHOD(boost::json::object, subLedger, (boost::asio::yield_context&, session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(
|
||||||
|
void,
|
||||||
|
pubLedger,
|
||||||
|
(ripple::LedgerInfo const&, ripple::Fees const&, std::string const&, std::uint32_t),
|
||||||
|
());
|
||||||
|
|
||||||
|
MOCK_METHOD(
|
||||||
|
void,
|
||||||
|
pubBookChanges,
|
||||||
|
(ripple::LedgerInfo const&, std::vector<Backend::TransactionAndMetadata> const&),
|
||||||
|
());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubLedger, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subTransactions, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubTransactions, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, pubTransaction, (Backend::TransactionAndMetadata const&, ripple::LedgerInfo const&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subAccount, (ripple::AccountID const&, session_ptr&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubAccount, (ripple::AccountID const&, session_ptr&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subBook, (ripple::Book const&, session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubBook, (ripple::Book const&, session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subBookChanges, (std::shared_ptr<WsBase>), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubBookChanges, (std::shared_ptr<WsBase>), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subManifest, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubManifest, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subValidation, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubValidation, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, forwardProposedTransaction, (boost::json::object const&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, forwardManifest, (boost::json::object const&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, forwardValidation, (boost::json::object const&), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subProposedAccount, (ripple::AccountID const&, session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubProposedAccount, (ripple::AccountID const&, session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, subProposedTransactions, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, unsubProposedTransactions, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(void, cleanup, (session_ptr), ());
|
||||||
|
|
||||||
|
MOCK_METHOD(boost::json::object, report, (), (const));
|
||||||
|
};
|
||||||
@@ -18,10 +18,13 @@
|
|||||||
//==============================================================================
|
//==============================================================================
|
||||||
|
|
||||||
#include "TestObject.h"
|
#include "TestObject.h"
|
||||||
|
#include <backend/DBHelpers.h>
|
||||||
|
|
||||||
#include <ripple/protocol/STArray.h>
|
#include <ripple/protocol/STArray.h>
|
||||||
#include <ripple/protocol/TER.h>
|
#include <ripple/protocol/TER.h>
|
||||||
|
|
||||||
|
#include <chrono>
|
||||||
|
|
||||||
ripple::AccountID
|
ripple::AccountID
|
||||||
GetAccountIDWithString(std::string_view id)
|
GetAccountIDWithString(std::string_view id)
|
||||||
{
|
{
|
||||||
@@ -29,11 +32,21 @@ GetAccountIDWithString(std::string_view id)
|
|||||||
}
|
}
|
||||||
|
|
||||||
ripple::LedgerInfo
|
ripple::LedgerInfo
|
||||||
CreateLedgerInfo(std::string_view ledgerHash, ripple::LedgerIndex seq)
|
CreateLedgerInfo(std::string_view ledgerHash, ripple::LedgerIndex seq, std::optional<uint32_t> age)
|
||||||
{
|
{
|
||||||
|
using namespace std::chrono;
|
||||||
|
|
||||||
auto ledgerinfo = ripple::LedgerInfo();
|
auto ledgerinfo = ripple::LedgerInfo();
|
||||||
ledgerinfo.hash = ripple::uint256{ledgerHash};
|
ledgerinfo.hash = ripple::uint256{ledgerHash};
|
||||||
ledgerinfo.seq = seq;
|
ledgerinfo.seq = seq;
|
||||||
|
|
||||||
|
if (age)
|
||||||
|
{
|
||||||
|
auto const now = duration_cast<seconds>(system_clock::now().time_since_epoch());
|
||||||
|
auto const closeTime = (now - seconds{age.value()}).count() - rippleEpochStart;
|
||||||
|
ledgerinfo.closeTime = ripple::NetClock::time_point{seconds{closeTime}};
|
||||||
|
}
|
||||||
|
|
||||||
return ledgerinfo;
|
return ledgerinfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -23,6 +23,7 @@
|
|||||||
|
|
||||||
#include <ripple/ledger/ReadView.h>
|
#include <ripple/ledger/ReadView.h>
|
||||||
|
|
||||||
|
#include <optional>
|
||||||
#include <string_view>
|
#include <string_view>
|
||||||
|
|
||||||
/*
|
/*
|
||||||
@@ -35,7 +36,7 @@ GetAccountIDWithString(std::string_view id);
|
|||||||
* Create a simple ledgerInfo object with only hash and seq
|
* Create a simple ledgerInfo object with only hash and seq
|
||||||
*/
|
*/
|
||||||
[[nodiscard]] ripple::LedgerInfo
|
[[nodiscard]] ripple::LedgerInfo
|
||||||
CreateLedgerInfo(std::string_view ledgerHash, ripple::LedgerIndex seq);
|
CreateLedgerInfo(std::string_view ledgerHash, ripple::LedgerIndex seq, std::optional<uint32_t> age = std::nullopt);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Create a FeeSetting ledger object
|
* Create a FeeSetting ledger object
|
||||||
|
|||||||
Reference in New Issue
Block a user