mirror of
https://github.com/XRPLF/clio.git
synced 2025-11-04 11:55:51 +00:00
@@ -97,13 +97,14 @@ target_sources (clio PRIVATE
|
||||
src/data/cassandra/Handle.cpp
|
||||
src/data/cassandra/SettingsProvider.cpp
|
||||
## ETL
|
||||
src/etl/Source.cpp
|
||||
src/etl/ProbingSource.cpp
|
||||
src/etl/NFTHelpers.cpp
|
||||
src/etl/ETLService.cpp
|
||||
src/etl/ETLState.cpp
|
||||
src/etl/LoadBalancer.cpp
|
||||
src/etl/impl/ForwardCache.cpp
|
||||
src/etl/Source.cpp
|
||||
src/etl/impl/ForwardingSource.cpp
|
||||
src/etl/impl/GrpcSource.cpp
|
||||
src/etl/impl/SubscriptionSource.cpp
|
||||
## Feed
|
||||
src/feed/SubscriptionManager.cpp
|
||||
src/feed/impl/TransactionFeed.cpp
|
||||
@@ -164,6 +165,7 @@ target_sources (clio PRIVATE
|
||||
src/util/prometheus/OStream.cpp
|
||||
src/util/prometheus/Prometheus.cpp
|
||||
src/util/Random.cpp
|
||||
src/util/Retry.cpp
|
||||
src/util/requests/RequestBuilder.cpp
|
||||
src/util/requests/Types.cpp
|
||||
src/util/requests/WsConnection.cpp
|
||||
@@ -212,6 +214,7 @@ if (tests)
|
||||
unittests/util/prometheus/MetricBuilderTests.cpp
|
||||
unittests/util/prometheus/MetricsFamilyTests.cpp
|
||||
unittests/util/prometheus/OStreamTests.cpp
|
||||
unittests/util/RetryTests.cpp
|
||||
## Async framework
|
||||
unittests/util/async/AnyExecutionContextTests.cpp
|
||||
unittests/util/async/AnyStrandTests.cpp
|
||||
@@ -223,13 +226,18 @@ if (tests)
|
||||
unittests/util/requests/SslContextTests.cpp
|
||||
unittests/util/requests/WsConnectionTests.cpp
|
||||
# ETL
|
||||
unittests/etl/AmendmentBlockHandlerTests.cpp
|
||||
unittests/etl/CacheLoaderTests.cpp
|
||||
unittests/etl/ETLStateTests.cpp
|
||||
unittests/etl/ExtractionDataPipeTests.cpp
|
||||
unittests/etl/ExtractorTests.cpp
|
||||
unittests/etl/TransformerTests.cpp
|
||||
unittests/etl/CacheLoaderTests.cpp
|
||||
unittests/etl/AmendmentBlockHandlerTests.cpp
|
||||
unittests/etl/ForwardingSourceTests.cpp
|
||||
unittests/etl/GrpcSourceTests.cpp
|
||||
unittests/etl/SourceTests.cpp
|
||||
unittests/etl/SubscriptionSourceTests.cpp
|
||||
unittests/etl/SubscriptionSourceDependenciesTests.cpp
|
||||
unittests/etl/LedgerPublisherTests.cpp
|
||||
unittests/etl/ETLStateTests.cpp
|
||||
unittests/etl/TransformerTests.cpp
|
||||
# RPC
|
||||
unittests/rpc/ErrorTests.cpp
|
||||
unittests/rpc/BaseTests.cpp
|
||||
|
||||
@@ -123,9 +123,6 @@ concept SomeRetryPolicy = requires(T a, boost::asio::io_context ioc, CassandraEr
|
||||
{
|
||||
a.retry([]() {})
|
||||
} -> std::same_as<void>;
|
||||
{
|
||||
a.calculateDelay(attempt)
|
||||
} -> std::same_as<std::chrono::milliseconds>;
|
||||
};
|
||||
|
||||
} // namespace data::cassandra
|
||||
|
||||
@@ -19,16 +19,16 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/cassandra/Handle.hpp"
|
||||
#include "data/cassandra/Error.hpp"
|
||||
#include "data/cassandra/Types.hpp"
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/Retry.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
|
||||
#include <algorithm>
|
||||
#include <chrono>
|
||||
#include <cmath>
|
||||
|
||||
namespace data::cassandra::impl {
|
||||
|
||||
@@ -38,14 +38,18 @@ namespace data::cassandra::impl {
|
||||
class ExponentialBackoffRetryPolicy {
|
||||
util::Logger log_{"Backend"};
|
||||
|
||||
boost::asio::steady_timer timer_;
|
||||
uint32_t attempt_ = 0u;
|
||||
util::Retry retry_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create a new retry policy instance with the io_context provided
|
||||
*/
|
||||
ExponentialBackoffRetryPolicy(boost::asio::io_context& ioc) : timer_{boost::asio::make_strand(ioc)}
|
||||
ExponentialBackoffRetryPolicy(boost::asio::io_context& ioc)
|
||||
: retry_(util::makeRetryExponentialBackoff(
|
||||
std::chrono::milliseconds(1),
|
||||
std::chrono::seconds(1),
|
||||
boost::asio::make_strand(ioc)
|
||||
))
|
||||
{
|
||||
}
|
||||
|
||||
@@ -57,9 +61,9 @@ public:
|
||||
[[nodiscard]] bool
|
||||
shouldRetry([[maybe_unused]] CassandraError err)
|
||||
{
|
||||
auto const delay = calculateDelay(attempt_);
|
||||
LOG(log_.error()) << "Cassandra write error: " << err << ", current retries " << attempt_ << ", retrying in "
|
||||
<< delay.count() << " milliseconds";
|
||||
auto const delayMs = std::chrono::duration_cast<std::chrono::milliseconds>(retry_.delayValue()).count();
|
||||
LOG(log_.error()) << "Cassandra write error: " << err << ", current retries " << retry_.attemptNumber()
|
||||
<< ", retrying in " << delayMs << " milliseconds";
|
||||
|
||||
return true; // keep retrying forever
|
||||
}
|
||||
@@ -73,20 +77,7 @@ public:
|
||||
void
|
||||
retry(Fn&& fn)
|
||||
{
|
||||
timer_.expires_after(calculateDelay(attempt_++));
|
||||
timer_.async_wait([fn = std::forward<Fn>(fn)]([[maybe_unused]] auto const& err) {
|
||||
// todo: deal with cancellation (thru err)
|
||||
fn();
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Calculates the wait time before attempting another retry
|
||||
*/
|
||||
static std::chrono::milliseconds
|
||||
calculateDelay(uint32_t attempt)
|
||||
{
|
||||
return std::chrono::milliseconds{lround(std::pow(2, std::min(10u, attempt)))};
|
||||
retry_.retry(std::forward<Fn>(fn));
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@@ -24,7 +24,6 @@
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/LoadBalancer.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/impl/AmendmentBlock.hpp"
|
||||
#include "etl/impl/CacheLoader.hpp"
|
||||
|
||||
@@ -23,9 +23,7 @@
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etl/ETLService.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/ProbingSource.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Random.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
@@ -52,22 +50,6 @@ using namespace util;
|
||||
|
||||
namespace etl {
|
||||
|
||||
std::unique_ptr<Source>
|
||||
LoadBalancer::make_Source(
|
||||
Config const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManager> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgers> validatedLedgers,
|
||||
LoadBalancer& balancer
|
||||
)
|
||||
{
|
||||
auto src = std::make_unique<ProbingSource>(config, ioc, backend, subscriptions, validatedLedgers, balancer);
|
||||
src->run();
|
||||
|
||||
return src;
|
||||
}
|
||||
|
||||
std::shared_ptr<LoadBalancer>
|
||||
LoadBalancer::make_LoadBalancer(
|
||||
Config const& config,
|
||||
@@ -98,7 +80,7 @@ LoadBalancer::LoadBalancer(
|
||||
auto const allowNoEtl = config.valueOr("allow_no_etl", false);
|
||||
|
||||
auto const checkOnETLFailure = [this, allowNoEtl](std::string const& log) {
|
||||
LOG(log_.error()) << log;
|
||||
LOG(log_.warn()) << log;
|
||||
|
||||
if (!allowNoEtl) {
|
||||
LOG(log_.error()) << "Set allow_no_etl as true in config to allow clio run without valid ETL sources.";
|
||||
@@ -107,15 +89,26 @@ LoadBalancer::LoadBalancer(
|
||||
};
|
||||
|
||||
for (auto const& entry : config.array("etl_sources")) {
|
||||
std::unique_ptr<Source> source = make_Source(entry, ioc, backend, subscriptions, validatedLedgers, *this);
|
||||
auto source = make_Source(
|
||||
entry,
|
||||
ioc,
|
||||
backend,
|
||||
subscriptions,
|
||||
validatedLedgers,
|
||||
[this]() {
|
||||
if (not hasForwardingSource_)
|
||||
chooseForwardingSource();
|
||||
},
|
||||
[this]() { chooseForwardingSource(); }
|
||||
);
|
||||
|
||||
// checking etl node validity
|
||||
auto const stateOpt = ETLState::fetchETLStateFromSource(*source);
|
||||
auto const stateOpt = ETLState::fetchETLStateFromSource(source);
|
||||
|
||||
if (!stateOpt) {
|
||||
checkOnETLFailure(fmt::format(
|
||||
"Failed to fetch ETL state from source = {} Please check the configuration and network",
|
||||
source->toString()
|
||||
source.toString()
|
||||
));
|
||||
} else if (etlState_ && etlState_->networkID && stateOpt->networkID && etlState_->networkID != stateOpt->networkID) {
|
||||
checkOnETLFailure(fmt::format(
|
||||
@@ -128,11 +121,17 @@ LoadBalancer::LoadBalancer(
|
||||
}
|
||||
|
||||
sources_.push_back(std::move(source));
|
||||
LOG(log_.info()) << "Added etl source - " << sources_.back()->toString();
|
||||
LOG(log_.info()) << "Added etl source - " << sources_.back().toString();
|
||||
}
|
||||
|
||||
if (sources_.empty())
|
||||
checkOnETLFailure("No ETL sources configured. Please check the configuration");
|
||||
|
||||
// This is made separate from source creation to prevent UB in case one of the sources will call
|
||||
// chooseForwardingSource while we are still filling the sources_ vector
|
||||
for (auto& source : sources_) {
|
||||
source.run();
|
||||
}
|
||||
}
|
||||
|
||||
LoadBalancer::~LoadBalancer()
|
||||
@@ -146,11 +145,11 @@ LoadBalancer::loadInitialLedger(uint32_t sequence, bool cacheOnly)
|
||||
std::vector<std::string> response;
|
||||
auto const success = execute(
|
||||
[this, &response, &sequence, cacheOnly](auto& source) {
|
||||
auto [data, res] = source->loadInitialLedger(sequence, downloadRanges_, cacheOnly);
|
||||
auto [data, res] = source.loadInitialLedger(sequence, downloadRanges_, cacheOnly);
|
||||
|
||||
if (!res) {
|
||||
LOG(log_.error()) << "Failed to download initial ledger."
|
||||
<< " Sequence = " << sequence << " source = " << source->toString();
|
||||
<< " Sequence = " << sequence << " source = " << source.toString();
|
||||
} else {
|
||||
response = std::move(data);
|
||||
}
|
||||
@@ -168,17 +167,17 @@ LoadBalancer::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObje
|
||||
GetLedgerResponseType response;
|
||||
bool const success = execute(
|
||||
[&response, ledgerSequence, getObjects, getObjectNeighbors, log = log_](auto& source) {
|
||||
auto [status, data] = source->fetchLedger(ledgerSequence, getObjects, getObjectNeighbors);
|
||||
auto [status, data] = source.fetchLedger(ledgerSequence, getObjects, getObjectNeighbors);
|
||||
response = std::move(data);
|
||||
if (status.ok() && response.validated()) {
|
||||
LOG(log.info()) << "Successfully fetched ledger = " << ledgerSequence
|
||||
<< " from source = " << source->toString();
|
||||
<< " from source = " << source.toString();
|
||||
return true;
|
||||
}
|
||||
|
||||
LOG(log.warn()) << "Could not fetch ledger " << ledgerSequence << ", Reply: " << response.DebugString()
|
||||
<< ", error_code: " << status.error_code() << ", error_msg: " << status.error_message()
|
||||
<< ", source = " << source->toString();
|
||||
<< ", source = " << source.toString();
|
||||
return false;
|
||||
},
|
||||
ledgerSequence
|
||||
@@ -203,7 +202,7 @@ LoadBalancer::forwardToRippled(
|
||||
auto numAttempts = 0u;
|
||||
|
||||
while (numAttempts < sources_.size()) {
|
||||
if (auto res = sources_[sourceIdx]->forwardToRippled(request, clientIp, yield))
|
||||
if (auto res = sources_[sourceIdx].forwardToRippled(request, clientIp, yield))
|
||||
return res;
|
||||
|
||||
sourceIdx = (sourceIdx + 1) % sources_.size();
|
||||
@@ -213,27 +212,12 @@ LoadBalancer::forwardToRippled(
|
||||
return {};
|
||||
}
|
||||
|
||||
bool
|
||||
LoadBalancer::shouldPropagateTxnStream(Source* in) const
|
||||
{
|
||||
for (auto& src : sources_) {
|
||||
ASSERT(src != nullptr, "Source is nullptr");
|
||||
|
||||
// We pick the first Source encountered that is connected
|
||||
if (src->isConnected())
|
||||
return *src == *in;
|
||||
}
|
||||
|
||||
// If no sources connected, then this stream has not been forwarded
|
||||
return true;
|
||||
}
|
||||
|
||||
boost::json::value
|
||||
LoadBalancer::toJson() const
|
||||
{
|
||||
boost::json::array ret;
|
||||
for (auto& src : sources_)
|
||||
ret.push_back(src->toJson());
|
||||
ret.push_back(src.toJson());
|
||||
|
||||
return ret;
|
||||
}
|
||||
@@ -252,23 +236,23 @@ LoadBalancer::execute(Func f, uint32_t ledgerSequence)
|
||||
auto& source = sources_[sourceIdx];
|
||||
|
||||
LOG(log_.debug()) << "Attempting to execute func. ledger sequence = " << ledgerSequence
|
||||
<< " - source = " << source->toString();
|
||||
<< " - source = " << source.toString();
|
||||
// Originally, it was (source->hasLedger(ledgerSequence) || true)
|
||||
/* Sometimes rippled has ledger but doesn't actually know. However,
|
||||
but this does NOT happen in the normal case and is safe to remove
|
||||
This || true is only needed when loading full history standalone */
|
||||
if (source->hasLedger(ledgerSequence)) {
|
||||
if (source.hasLedger(ledgerSequence)) {
|
||||
bool const res = f(source);
|
||||
if (res) {
|
||||
LOG(log_.debug()) << "Successfully executed func at source = " << source->toString()
|
||||
LOG(log_.debug()) << "Successfully executed func at source = " << source.toString()
|
||||
<< " - ledger sequence = " << ledgerSequence;
|
||||
break;
|
||||
}
|
||||
|
||||
LOG(log_.warn()) << "Failed to execute func at source = " << source->toString()
|
||||
LOG(log_.warn()) << "Failed to execute func at source = " << source.toString()
|
||||
<< " - ledger sequence = " << ledgerSequence;
|
||||
} else {
|
||||
LOG(log_.warn()) << "Ledger not present at source = " << source->toString()
|
||||
LOG(log_.warn()) << "Ledger not present at source = " << source.toString()
|
||||
<< " - ledger sequence = " << ledgerSequence;
|
||||
}
|
||||
sourceIdx = (sourceIdx + 1) % sources_.size();
|
||||
@@ -293,4 +277,17 @@ LoadBalancer::getETLState() noexcept
|
||||
return etlState_;
|
||||
}
|
||||
|
||||
void
|
||||
LoadBalancer::chooseForwardingSource()
|
||||
{
|
||||
hasForwardingSource_ = false;
|
||||
for (auto& source : sources_) {
|
||||
if (source.isConnected()) {
|
||||
source.setForwarding(true);
|
||||
hasForwardingSource_ = true;
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etl
|
||||
|
||||
@@ -22,6 +22,7 @@
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "feed/SubscriptionManager.hpp"
|
||||
#include "util/config/Config.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
@@ -36,6 +37,7 @@
|
||||
#include <org/xrpl/rpc/v1/ledger.pb.h>
|
||||
#include <ripple/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
@@ -44,7 +46,6 @@
|
||||
#include <vector>
|
||||
|
||||
namespace etl {
|
||||
class Source;
|
||||
class ProbingSource;
|
||||
} // namespace etl
|
||||
|
||||
@@ -71,10 +72,11 @@ private:
|
||||
static constexpr std::uint32_t DEFAULT_DOWNLOAD_RANGES = 16;
|
||||
|
||||
util::Logger log_{"ETL"};
|
||||
std::vector<std::unique_ptr<Source>> sources_;
|
||||
std::vector<Source> sources_;
|
||||
std::optional<ETLState> etlState_;
|
||||
std::uint32_t downloadRanges_ =
|
||||
DEFAULT_DOWNLOAD_RANGES; /*< The number of markers to use when downloading intial ledger */
|
||||
std::atomic_bool hasForwardingSource_{false};
|
||||
|
||||
public:
|
||||
/**
|
||||
@@ -138,19 +140,6 @@ public:
|
||||
OptionalGetLedgerResponseType
|
||||
fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors);
|
||||
|
||||
/**
|
||||
* @brief Determine whether messages received on the transactions_proposed stream should be forwarded to subscribing
|
||||
* clients.
|
||||
*
|
||||
* The server subscribes to transactions_proposed on multiple Sources, yet only forwards messages from one source at
|
||||
* any given time (to avoid sending duplicate messages to clients).
|
||||
*
|
||||
* @param in Source in question
|
||||
* @return true if messages should be forwarded
|
||||
*/
|
||||
bool
|
||||
shouldPropagateTxnStream(Source* in) const;
|
||||
|
||||
/**
|
||||
* @return JSON representation of the state of this load balancer.
|
||||
*/
|
||||
@@ -180,26 +169,6 @@ public:
|
||||
getETLState() noexcept;
|
||||
|
||||
private:
|
||||
/**
|
||||
* @brief A factory function for the ETL source.
|
||||
*
|
||||
* @param config The configuration to use
|
||||
* @param ioc The io_context to run on
|
||||
* @param backend BackendInterface implementation
|
||||
* @param subscriptions Subscription manager
|
||||
* @param validatedLedgers The network validated ledgers datastructure
|
||||
* @param balancer The load balancer
|
||||
*/
|
||||
static std::unique_ptr<Source>
|
||||
make_Source(
|
||||
util::Config const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManager> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgers> validatedLedgers,
|
||||
LoadBalancer& balancer
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Execute a function on a randomly selected source.
|
||||
*
|
||||
@@ -215,5 +184,12 @@ private:
|
||||
template <class Func>
|
||||
bool
|
||||
execute(Func f, uint32_t ledgerSequence);
|
||||
|
||||
/**
|
||||
* @brief Choose a new source to forward requests
|
||||
*/
|
||||
void
|
||||
chooseForwardingSource();
|
||||
};
|
||||
|
||||
} // namespace etl
|
||||
|
||||
@@ -1,232 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2022, 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/ProbingSource.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etl/LoadBalancer.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "feed/SubscriptionManager.hpp"
|
||||
#include "util/config/Config.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/ssl/context.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/uuid/nil_generator.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <grpcpp/support/status.h>
|
||||
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl {
|
||||
|
||||
ProbingSource::ProbingSource(
|
||||
util::Config const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManager> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgers> nwvl,
|
||||
LoadBalancer& balancer,
|
||||
boost::asio::ssl::context sslCtx
|
||||
)
|
||||
: sslCtx_{std::move(sslCtx)}
|
||||
, sslSrc_{make_shared<
|
||||
SslSource>(config, ioc, std::ref(sslCtx_), backend, subscriptions, nwvl, balancer, make_SSLHooks())}
|
||||
, plainSrc_{make_shared<PlainSource>(config, ioc, backend, subscriptions, nwvl, balancer, make_PlainHooks())}
|
||||
{
|
||||
}
|
||||
|
||||
void
|
||||
ProbingSource::run()
|
||||
{
|
||||
sslSrc_->run();
|
||||
plainSrc_->run();
|
||||
}
|
||||
|
||||
void
|
||||
ProbingSource::pause()
|
||||
{
|
||||
sslSrc_->pause();
|
||||
plainSrc_->pause();
|
||||
}
|
||||
|
||||
void
|
||||
ProbingSource::resume()
|
||||
{
|
||||
sslSrc_->resume();
|
||||
plainSrc_->resume();
|
||||
}
|
||||
|
||||
bool
|
||||
ProbingSource::isConnected() const
|
||||
{
|
||||
return currentSrc_ && currentSrc_->isConnected();
|
||||
}
|
||||
|
||||
bool
|
||||
ProbingSource::hasLedger(uint32_t sequence) const
|
||||
{
|
||||
if (!currentSrc_)
|
||||
return false;
|
||||
return currentSrc_->hasLedger(sequence);
|
||||
}
|
||||
|
||||
boost::json::object
|
||||
ProbingSource::toJson() const
|
||||
{
|
||||
if (!currentSrc_) {
|
||||
boost::json::object sourcesJson = {
|
||||
{"ws", plainSrc_->toJson()},
|
||||
{"wss", sslSrc_->toJson()},
|
||||
};
|
||||
|
||||
return {
|
||||
{"probing", sourcesJson},
|
||||
};
|
||||
}
|
||||
return currentSrc_->toJson();
|
||||
}
|
||||
|
||||
std::string
|
||||
ProbingSource::toString() const
|
||||
{
|
||||
if (!currentSrc_)
|
||||
return "{probing... ws: " + plainSrc_->toString() + ", wss: " + sslSrc_->toString() + "}";
|
||||
return currentSrc_->toString();
|
||||
}
|
||||
|
||||
boost::uuids::uuid
|
||||
ProbingSource::token() const
|
||||
{
|
||||
if (!currentSrc_)
|
||||
return boost::uuids::nil_uuid();
|
||||
return currentSrc_->token();
|
||||
}
|
||||
|
||||
std::pair<std::vector<std::string>, bool>
|
||||
ProbingSource::loadInitialLedger(std::uint32_t sequence, std::uint32_t numMarkers, bool cacheOnly)
|
||||
{
|
||||
if (!currentSrc_)
|
||||
return {{}, false};
|
||||
return currentSrc_->loadInitialLedger(sequence, numMarkers, cacheOnly);
|
||||
}
|
||||
|
||||
std::pair<grpc::Status, ProbingSource::GetLedgerResponseType>
|
||||
ProbingSource::fetchLedger(uint32_t sequence, bool getObjects, bool getObjectNeighbors)
|
||||
{
|
||||
if (!currentSrc_)
|
||||
return {};
|
||||
return currentSrc_->fetchLedger(sequence, getObjects, getObjectNeighbors);
|
||||
}
|
||||
|
||||
std::optional<boost::json::object>
|
||||
ProbingSource::forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& clientIp,
|
||||
boost::asio::yield_context yield
|
||||
) const
|
||||
{
|
||||
if (!currentSrc_) // Source may connect to rippled before the connection built to check the validity
|
||||
{
|
||||
if (auto res = plainSrc_->forwardToRippled(request, clientIp, yield))
|
||||
return res;
|
||||
|
||||
return sslSrc_->forwardToRippled(request, clientIp, yield);
|
||||
}
|
||||
return currentSrc_->forwardToRippled(request, clientIp, yield);
|
||||
}
|
||||
|
||||
std::optional<boost::json::object>
|
||||
ProbingSource::requestFromRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& clientIp,
|
||||
boost::asio::yield_context yield
|
||||
) const
|
||||
{
|
||||
if (!currentSrc_)
|
||||
return {};
|
||||
return currentSrc_->requestFromRippled(request, clientIp, yield);
|
||||
}
|
||||
|
||||
SourceHooks
|
||||
ProbingSource::make_SSLHooks() noexcept
|
||||
{
|
||||
return {// onConnected
|
||||
[this](auto ec) {
|
||||
std::lock_guard const lck(mtx_);
|
||||
if (currentSrc_)
|
||||
return SourceHooks::Action::STOP;
|
||||
|
||||
if (!ec) {
|
||||
plainSrc_->pause();
|
||||
currentSrc_ = sslSrc_;
|
||||
LOG(log_.info()) << "Selected WSS as the main source: " << currentSrc_->toString();
|
||||
}
|
||||
return SourceHooks::Action::PROCEED;
|
||||
},
|
||||
// onDisconnected
|
||||
[this](auto /* ec */) {
|
||||
std::lock_guard const lck(mtx_);
|
||||
if (currentSrc_) {
|
||||
currentSrc_ = nullptr;
|
||||
plainSrc_->resume();
|
||||
}
|
||||
return SourceHooks::Action::STOP;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
SourceHooks
|
||||
ProbingSource::make_PlainHooks() noexcept
|
||||
{
|
||||
return {// onConnected
|
||||
[this](auto ec) {
|
||||
std::lock_guard const lck(mtx_);
|
||||
if (currentSrc_)
|
||||
return SourceHooks::Action::STOP;
|
||||
|
||||
if (!ec) {
|
||||
sslSrc_->pause();
|
||||
currentSrc_ = plainSrc_;
|
||||
LOG(log_.info()) << "Selected Plain WS as the main source: " << currentSrc_->toString();
|
||||
}
|
||||
return SourceHooks::Action::PROCEED;
|
||||
},
|
||||
// onDisconnected
|
||||
[this](auto /* ec */) {
|
||||
std::lock_guard const lck(mtx_);
|
||||
if (currentSrc_) {
|
||||
currentSrc_ = nullptr;
|
||||
sslSrc_->resume();
|
||||
}
|
||||
return SourceHooks::Action::STOP;
|
||||
}
|
||||
};
|
||||
};
|
||||
} // namespace etl
|
||||
@@ -1,147 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2022, 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 "data/BackendInterface.hpp"
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etl/LoadBalancer.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "util/config/Config.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/ssl/context.hpp>
|
||||
#include <boost/beast/core.hpp>
|
||||
#include <boost/beast/core/string.hpp>
|
||||
#include <boost/beast/ssl.hpp>
|
||||
#include <boost/beast/websocket.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief This Source implementation attempts to connect over both secure websocket and plain websocket.
|
||||
*
|
||||
* First to connect pauses the other and the probing is considered done at this point.
|
||||
* If however the connected source loses connection the probing is kickstarted again.
|
||||
*/
|
||||
class ProbingSource : public Source {
|
||||
public:
|
||||
// TODO: inject when unit tests will be written for ProbingSource
|
||||
using GetLedgerResponseType = org::xrpl::rpc::v1::GetLedgerResponse;
|
||||
|
||||
private:
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
std::mutex mtx_;
|
||||
boost::asio::ssl::context sslCtx_;
|
||||
std::shared_ptr<Source> sslSrc_;
|
||||
std::shared_ptr<Source> plainSrc_;
|
||||
std::shared_ptr<Source> currentSrc_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of the probing source.
|
||||
*
|
||||
* @param config The configuration to use
|
||||
* @param ioc io context to run on
|
||||
* @param backend BackendInterface implementation
|
||||
* @param subscriptions Subscription manager
|
||||
* @param nwvl The network validated ledgers datastructure
|
||||
* @param balancer Load balancer to use
|
||||
* @param sslCtx The SSL context to use; defaults to tlsv12
|
||||
*/
|
||||
ProbingSource(
|
||||
util::Config const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManager> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgers> nwvl,
|
||||
LoadBalancer& balancer,
|
||||
boost::asio::ssl::context sslCtx = boost::asio::ssl::context{boost::asio::ssl::context::tlsv12}
|
||||
);
|
||||
|
||||
~ProbingSource() override = default;
|
||||
|
||||
void
|
||||
run() override;
|
||||
|
||||
void
|
||||
pause() override;
|
||||
|
||||
void
|
||||
resume() override;
|
||||
|
||||
bool
|
||||
isConnected() const override;
|
||||
|
||||
bool
|
||||
hasLedger(uint32_t sequence) const override;
|
||||
|
||||
boost::json::object
|
||||
toJson() const override;
|
||||
|
||||
std::string
|
||||
toString() const override;
|
||||
|
||||
std::pair<std::vector<std::string>, bool>
|
||||
loadInitialLedger(std::uint32_t sequence, std::uint32_t numMarkers, bool cacheOnly = false) override;
|
||||
|
||||
std::pair<grpc::Status, GetLedgerResponseType>
|
||||
fetchLedger(uint32_t sequence, bool getObjects = true, bool getObjectNeighbors = false) override;
|
||||
|
||||
std::optional<boost::json::object>
|
||||
forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& clientIp,
|
||||
boost::asio::yield_context yield
|
||||
) const override;
|
||||
|
||||
boost::uuids::uuid
|
||||
token() const override;
|
||||
|
||||
private:
|
||||
std::optional<boost::json::object>
|
||||
requestFromRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& clientIp,
|
||||
boost::asio::yield_context yield
|
||||
) const override;
|
||||
|
||||
SourceHooks
|
||||
make_SSLHooks() noexcept;
|
||||
|
||||
SourceHooks
|
||||
make_PlainHooks() noexcept;
|
||||
};
|
||||
} // namespace etl
|
||||
@@ -1,7 +1,7 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2022, the clio developers.
|
||||
Copyright (c) 2024, 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
|
||||
@@ -19,162 +19,51 @@
|
||||
|
||||
#include "etl/Source.hpp"
|
||||
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "feed/SubscriptionManager.hpp"
|
||||
#include "util/config/Config.hpp"
|
||||
|
||||
#include <boost/asio/ip/tcp.hpp>
|
||||
#include <boost/asio/post.hpp>
|
||||
#include <boost/asio/ssl/stream_base.hpp>
|
||||
#include <boost/beast/core/error.hpp>
|
||||
#include <boost/beast/core/role.hpp>
|
||||
#include <boost/beast/core/stream_traits.hpp>
|
||||
#include <boost/beast/http/field.hpp>
|
||||
#include <boost/beast/websocket/rfc6455.hpp>
|
||||
#include <boost/beast/websocket/stream_base.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
|
||||
namespace etl {
|
||||
|
||||
static boost::beast::websocket::stream_base::timeout
|
||||
make_TimeoutOption()
|
||||
{
|
||||
return boost::beast::websocket::stream_base::timeout::suggested(boost::beast::role_type::client);
|
||||
}
|
||||
template class SourceImpl<>;
|
||||
|
||||
void
|
||||
PlainSource::close(bool startAgain)
|
||||
{
|
||||
timer_.cancel();
|
||||
boost::asio::post(strand_, [this, startAgain]() {
|
||||
if (closing_)
|
||||
return;
|
||||
|
||||
if (derived().ws().is_open()) {
|
||||
// onStop() also calls close(). If the async_close is called twice,
|
||||
// an assertion fails. Using closing_ makes sure async_close is only
|
||||
// called once
|
||||
closing_ = true;
|
||||
derived().ws().async_close(boost::beast::websocket::close_code::normal, [this, startAgain](auto ec) {
|
||||
if (ec) {
|
||||
LOG(log_.error()) << "async_close: error code = " << ec << " - " << toString();
|
||||
}
|
||||
closing_ = false;
|
||||
if (startAgain) {
|
||||
ws_ = std::make_unique<StreamType>(strand_);
|
||||
run();
|
||||
}
|
||||
});
|
||||
} else if (startAgain) {
|
||||
ws_ = std::make_unique<StreamType>(strand_);
|
||||
run();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void
|
||||
SslSource::close(bool startAgain)
|
||||
{
|
||||
timer_.cancel();
|
||||
boost::asio::post(strand_, [this, startAgain]() {
|
||||
if (closing_)
|
||||
return;
|
||||
|
||||
if (derived().ws().is_open()) {
|
||||
// onStop() also calls close(). If the async_close is called twice, an assertion fails. Using closing_
|
||||
// makes sure async_close is only called once
|
||||
closing_ = true;
|
||||
derived().ws().async_close(boost::beast::websocket::close_code::normal, [this, startAgain](auto ec) {
|
||||
if (ec) {
|
||||
LOG(log_.error()) << "async_close: error code = " << ec << " - " << toString();
|
||||
}
|
||||
closing_ = false;
|
||||
if (startAgain) {
|
||||
ws_ = std::make_unique<StreamType>(strand_, *sslCtx_);
|
||||
run();
|
||||
}
|
||||
});
|
||||
} else if (startAgain) {
|
||||
ws_ = std::make_unique<StreamType>(strand_, *sslCtx_);
|
||||
run();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void
|
||||
PlainSource::onConnect(
|
||||
boost::beast::error_code ec,
|
||||
boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint
|
||||
Source
|
||||
make_Source(
|
||||
util::Config const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManager> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgers> validatedLedgers,
|
||||
Source::OnDisconnectHook onDisconnect,
|
||||
Source::OnConnectHook onConnect
|
||||
)
|
||||
{
|
||||
if (ec) {
|
||||
// start over
|
||||
reconnect(ec);
|
||||
} else {
|
||||
connected_ = true;
|
||||
numFailures_ = 0;
|
||||
auto const ip = config.valueOr<std::string>("ip", {});
|
||||
auto const wsPort = config.valueOr<std::string>("ws_port", {});
|
||||
auto const grpcPort = config.valueOr<std::string>("grpc_port", {});
|
||||
|
||||
// Websocket stream has it's own timeout system
|
||||
boost::beast::get_lowest_layer(derived().ws()).expires_never();
|
||||
impl::GrpcSource grpcSource{ip, grpcPort, std::move(backend)};
|
||||
auto subscriptionSource = std::make_unique<impl::SubscriptionSource>(
|
||||
ioc,
|
||||
ip,
|
||||
wsPort,
|
||||
std::move(validatedLedgers),
|
||||
std::move(subscriptions),
|
||||
std::move(onConnect),
|
||||
std::move(onDisconnect)
|
||||
);
|
||||
impl::ForwardingSource forwardingSource{ip, wsPort};
|
||||
|
||||
derived().ws().set_option(make_TimeoutOption());
|
||||
derived().ws().set_option(
|
||||
boost::beast::websocket::stream_base::decorator([](boost::beast::websocket::request_type& req) {
|
||||
req.set(boost::beast::http::field::user_agent, "clio-client");
|
||||
req.set("X-User", "clio-client");
|
||||
})
|
||||
);
|
||||
|
||||
// Update the host_ string. This will provide the value of the
|
||||
// Host HTTP header during the WebSocket handshake.
|
||||
// See https://tools.ietf.org/html/rfc7230#section-5.4
|
||||
auto host = ip_ + ':' + std::to_string(endpoint.port());
|
||||
derived().ws().async_handshake(host, "/", [this](auto ec) { onHandshake(ec); });
|
||||
}
|
||||
return Source{
|
||||
ip, wsPort, grpcPort, std::move(grpcSource), std::move(subscriptionSource), std::move(forwardingSource)
|
||||
};
|
||||
}
|
||||
|
||||
void
|
||||
SslSource::onConnect(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint)
|
||||
{
|
||||
if (ec) {
|
||||
// start over
|
||||
reconnect(ec);
|
||||
} else {
|
||||
connected_ = true;
|
||||
numFailures_ = 0;
|
||||
|
||||
// Websocket stream has it's own timeout system
|
||||
boost::beast::get_lowest_layer(derived().ws()).expires_never();
|
||||
|
||||
derived().ws().set_option(make_TimeoutOption());
|
||||
derived().ws().set_option(
|
||||
boost::beast::websocket::stream_base::decorator([](boost::beast::websocket::request_type& req) {
|
||||
req.set(boost::beast::http::field::user_agent, "clio-client");
|
||||
req.set("X-User", "clio-client");
|
||||
})
|
||||
);
|
||||
|
||||
// Update the host_ string. This will provide the value of the
|
||||
// Host HTTP header during the WebSocket handshake.
|
||||
// See https://tools.ietf.org/html/rfc7230#section-5.4
|
||||
auto host = ip_ + ':' + std::to_string(endpoint.port());
|
||||
ws().next_layer().async_handshake(boost::asio::ssl::stream_base::client, [this, endpoint](auto ec) {
|
||||
onSslHandshake(ec, endpoint);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
SslSource::onSslHandshake(
|
||||
boost::beast::error_code ec,
|
||||
boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint
|
||||
)
|
||||
{
|
||||
if (ec) {
|
||||
reconnect(ec);
|
||||
} else {
|
||||
auto host = ip_ + ':' + std::to_string(endpoint.port());
|
||||
ws().async_handshake(host, "/", [this](auto ec) { onHandshake(ec); });
|
||||
}
|
||||
}
|
||||
} // namespace etl
|
||||
|
||||
1020
src/etl/Source.hpp
1020
src/etl/Source.hpp
File diff suppressed because it is too large
Load Diff
@@ -21,6 +21,7 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etl/NFTHelpers.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
@@ -33,6 +34,7 @@
|
||||
#include <ripple/basics/strHex.h>
|
||||
#include <ripple/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
@@ -145,7 +147,7 @@ public:
|
||||
continue;
|
||||
}
|
||||
cacheUpdates.push_back(
|
||||
{*ripple::uint256::fromVoidChecked(obj.key()), {obj.mutable_data()->begin(), obj.mutable_data()->end()}}
|
||||
{*ripple::uint256::fromVoidChecked(obj.key()), {obj.data().begin(), obj.data().end()}}
|
||||
);
|
||||
if (!cacheOnly) {
|
||||
if (!lastKey_.empty())
|
||||
@@ -193,4 +195,21 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
inline std::vector<AsyncCallData>
|
||||
makeAsyncCallData(uint32_t const sequence, uint32_t const numMarkers)
|
||||
{
|
||||
auto const markers = getMarkers(numMarkers);
|
||||
|
||||
std::vector<AsyncCallData> result;
|
||||
result.reserve(markers.size());
|
||||
|
||||
for (size_t i = 0; i + 1 < markers.size(); ++i) {
|
||||
result.emplace_back(sequence, markers[i], markers[i + 1]);
|
||||
}
|
||||
if (not markers.empty()) {
|
||||
result.emplace_back(sequence, markers.back(), std::nullopt);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
} // namespace etl::impl
|
||||
|
||||
@@ -1,95 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2022, 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/impl/ForwardCache.hpp"
|
||||
|
||||
#include "etl/Source.hpp"
|
||||
#include "rpc/RPCHelpers.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/value_to.hpp>
|
||||
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <optional>
|
||||
#include <shared_mutex>
|
||||
#include <string>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
void
|
||||
ForwardCache::freshen()
|
||||
{
|
||||
LOG(log_.trace()) << "Freshening ForwardCache";
|
||||
|
||||
auto numOutstanding = std::make_shared<std::atomic_uint>(latestForwarded_.size());
|
||||
|
||||
for (auto const& cacheEntry : latestForwarded_) {
|
||||
boost::asio::spawn(
|
||||
strand_,
|
||||
[this, numOutstanding, command = cacheEntry.first](boost::asio::yield_context yield) {
|
||||
boost::json::object const request = {{"command", command}};
|
||||
auto resp = source_.requestFromRippled(request, std::nullopt, yield);
|
||||
|
||||
if (!resp || resp->contains("error"))
|
||||
resp = {};
|
||||
|
||||
{
|
||||
std::scoped_lock const lk(mtx_);
|
||||
latestForwarded_[command] = resp;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
ForwardCache::clear()
|
||||
{
|
||||
std::scoped_lock const lk(mtx_);
|
||||
for (auto& cacheEntry : latestForwarded_)
|
||||
latestForwarded_[cacheEntry.first] = {};
|
||||
}
|
||||
|
||||
std::optional<boost::json::object>
|
||||
ForwardCache::get(boost::json::object const& request) const
|
||||
{
|
||||
std::optional<std::string> command = {};
|
||||
if (request.contains("command") && !request.contains("method") && request.at("command").is_string()) {
|
||||
command = boost::json::value_to<std::string>(request.at("command"));
|
||||
} else if (request.contains("method") && !request.contains("command") && request.at("method").is_string()) {
|
||||
command = boost::json::value_to<std::string>(request.at("method"));
|
||||
}
|
||||
|
||||
if (!command)
|
||||
return {};
|
||||
if (rpc::specifiesCurrentOrClosedLedger(request))
|
||||
return {};
|
||||
|
||||
std::shared_lock const lk(mtx_);
|
||||
if (!latestForwarded_.contains(*command))
|
||||
return {};
|
||||
|
||||
return {latestForwarded_.at(*command)};
|
||||
}
|
||||
|
||||
} // namespace etl::impl
|
||||
@@ -1,85 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2022, 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/config/Config.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <boost/json.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
|
||||
#include <cstdint>
|
||||
#include <optional>
|
||||
#include <shared_mutex>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace etl {
|
||||
class Source;
|
||||
} // namespace etl
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
/**
|
||||
* @brief Cache for rippled responses
|
||||
*/
|
||||
class ForwardCache {
|
||||
using ResponseType = std::optional<boost::json::object>;
|
||||
static constexpr std::uint32_t DEFAULT_DURATION = 10;
|
||||
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
mutable std::shared_mutex mtx_;
|
||||
std::unordered_map<std::string, ResponseType> latestForwarded_;
|
||||
boost::asio::strand<boost::asio::io_context::executor_type> strand_;
|
||||
etl::Source const& source_;
|
||||
std::uint32_t duration_ = DEFAULT_DURATION;
|
||||
|
||||
void
|
||||
clear();
|
||||
|
||||
public:
|
||||
ForwardCache(util::Config const& config, boost::asio::io_context& ioc, Source const& source)
|
||||
: strand_(boost::asio::make_strand(ioc)), source_(source)
|
||||
{
|
||||
if (config.contains("cache")) {
|
||||
auto commands = config.arrayOrThrow("cache", "Source cache must be array");
|
||||
|
||||
if (config.contains("cache_duration"))
|
||||
duration_ = config.valueOrThrow<uint32_t>("cache_duration", "Source cache_duration must be a number");
|
||||
|
||||
for (auto const& command : commands) {
|
||||
auto key = command.valueOrThrow<std::string>("Source forward command must be array of strings");
|
||||
latestForwarded_[key] = {};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
freshen();
|
||||
|
||||
std::optional<boost::json::object>
|
||||
get(boost::json::object const& request) const;
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
98
src/etl/impl/ForwardingSource.cpp
Normal file
98
src/etl/impl/ForwardingSource.cpp
Normal file
@@ -0,0 +1,98 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/ForwardingSource.hpp"
|
||||
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/beast/http/field.hpp>
|
||||
#include <boost/beast/version.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <boost/json/serialize.hpp>
|
||||
#include <fmt/core.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <exception>
|
||||
#include <optional>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
ForwardingSource::ForwardingSource(
|
||||
std::string ip,
|
||||
std::string wsPort,
|
||||
std::chrono::steady_clock::duration connectionTimeout
|
||||
)
|
||||
: log_(fmt::format("ForwardingSource[{}:{}]", ip, wsPort)), connectionBuilder_(std::move(ip), std::move(wsPort))
|
||||
{
|
||||
connectionBuilder_.setConnectionTimeout(connectionTimeout)
|
||||
.addHeader(
|
||||
{boost::beast::http::field::user_agent, fmt::format("{} websocket-client-coro", BOOST_BEAST_VERSION_STRING)}
|
||||
);
|
||||
}
|
||||
|
||||
std::optional<boost::json::object>
|
||||
ForwardingSource::forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& forwardToRippledClientIp,
|
||||
boost::asio::yield_context yield
|
||||
) const
|
||||
{
|
||||
auto connectionBuilder = connectionBuilder_;
|
||||
if (forwardToRippledClientIp) {
|
||||
connectionBuilder.addHeader(
|
||||
{boost::beast::http::field::forwarded, fmt::format("for={}", *forwardToRippledClientIp)}
|
||||
);
|
||||
}
|
||||
auto expectedConnection = connectionBuilder.connect(yield);
|
||||
if (not expectedConnection) {
|
||||
return std::nullopt;
|
||||
}
|
||||
auto& connection = expectedConnection.value();
|
||||
|
||||
auto writeError = connection->write(boost::json::serialize(request), yield);
|
||||
if (writeError) {
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
auto response = connection->read(yield);
|
||||
if (not response) {
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
boost::json::value parsedResponse;
|
||||
try {
|
||||
parsedResponse = boost::json::parse(*response);
|
||||
if (not parsedResponse.is_object())
|
||||
throw std::runtime_error("response is not an object");
|
||||
} catch (std::exception const& e) {
|
||||
LOG(log_.error()) << "Error parsing response from rippled: " << e.what() << ". Response: " << *response;
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
auto responseObject = parsedResponse.as_object();
|
||||
responseObject["forwarded"] = true;
|
||||
return responseObject;
|
||||
}
|
||||
|
||||
} // namespace etl::impl
|
||||
62
src/etl/impl/ForwardingSource.hpp
Normal file
62
src/etl/impl/ForwardingSource.hpp
Normal file
@@ -0,0 +1,62 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/log/Logger.hpp"
|
||||
#include "util/requests/WsConnection.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
class ForwardingSource {
|
||||
util::Logger log_;
|
||||
util::requests::WsConnectionBuilder connectionBuilder_;
|
||||
static constexpr std::chrono::seconds CONNECTION_TIMEOUT{3};
|
||||
|
||||
public:
|
||||
ForwardingSource(
|
||||
std::string ip_,
|
||||
std::string wsPort_,
|
||||
std::chrono::steady_clock::duration connectionTimeout = CONNECTION_TIMEOUT
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Forward a request to rippled.
|
||||
*
|
||||
* @param request The request to forward
|
||||
* @param clientIp IP of the client forwarding this request if known
|
||||
* @param yield The coroutine context
|
||||
* @return Response wrapped in an optional on success; nullopt otherwise
|
||||
*/
|
||||
std::optional<boost::json::object>
|
||||
forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& forwardToRippledClientIp,
|
||||
boost::asio::yield_context yield
|
||||
) const;
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
151
src/etl/impl/GrpcSource.cpp
Normal file
151
src/etl/impl/GrpcSource.cpp
Normal file
@@ -0,0 +1,151 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/GrpcSource.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/impl/AsyncData.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/ip/address.hpp>
|
||||
#include <boost/asio/ip/tcp.hpp>
|
||||
#include <fmt/core.h>
|
||||
#include <grpcpp/client_context.h>
|
||||
#include <grpcpp/security/credentials.h>
|
||||
#include <grpcpp/support/channel_arguments.h>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <exception>
|
||||
#include <memory>
|
||||
#include <sstream>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
GrpcSource::GrpcSource(std::string const& ip, std::string const& grpcPort, std::shared_ptr<BackendInterface> backend)
|
||||
: log_(fmt::format("ETL_Grpc[{}:{}]", ip, grpcPort)), backend_(std::move(backend))
|
||||
{
|
||||
try {
|
||||
boost::asio::ip::tcp::endpoint const endpoint{boost::asio::ip::make_address(ip), std::stoi(grpcPort)};
|
||||
std::stringstream ss;
|
||||
ss << endpoint;
|
||||
grpc::ChannelArguments chArgs;
|
||||
chArgs.SetMaxReceiveMessageSize(-1);
|
||||
stub_ = org::xrpl::rpc::v1::XRPLedgerAPIService::NewStub(
|
||||
grpc::CreateCustomChannel(ss.str(), grpc::InsecureChannelCredentials(), chArgs)
|
||||
);
|
||||
LOG(log_.debug()) << "Made stub for remote.";
|
||||
} catch (std::exception const& e) {
|
||||
LOG(log_.warn()) << "Exception while creating stub: " << e.what() << ".";
|
||||
}
|
||||
}
|
||||
|
||||
std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>
|
||||
GrpcSource::fetchLedger(uint32_t sequence, bool getObjects, bool getObjectNeighbors)
|
||||
{
|
||||
org::xrpl::rpc::v1::GetLedgerResponse response;
|
||||
if (!stub_)
|
||||
return {{grpc::StatusCode::INTERNAL, "No Stub"}, response};
|
||||
|
||||
// Ledger header with txns and metadata
|
||||
org::xrpl::rpc::v1::GetLedgerRequest request;
|
||||
grpc::ClientContext context;
|
||||
|
||||
request.mutable_ledger()->set_sequence(sequence);
|
||||
request.set_transactions(true);
|
||||
request.set_expand(true);
|
||||
request.set_get_objects(getObjects);
|
||||
request.set_get_object_neighbors(getObjectNeighbors);
|
||||
request.set_user("ETL");
|
||||
|
||||
grpc::Status const status = stub_->GetLedger(&context, request, &response);
|
||||
|
||||
if (status.ok() && !response.is_unlimited()) {
|
||||
log_.warn() << "is_unlimited is false. Make sure secure_gateway is set correctly on the ETL source. Status = "
|
||||
<< status.error_message();
|
||||
}
|
||||
|
||||
return {status, std::move(response)};
|
||||
}
|
||||
|
||||
std::pair<std::vector<std::string>, bool>
|
||||
GrpcSource::loadInitialLedger(uint32_t const sequence, uint32_t const numMarkers, bool const cacheOnly)
|
||||
{
|
||||
if (!stub_)
|
||||
return {{}, false};
|
||||
|
||||
std::vector<etl::impl::AsyncCallData> calls = impl::makeAsyncCallData(sequence, numMarkers);
|
||||
|
||||
LOG(log_.debug()) << "Starting data download for ledger " << sequence << ".";
|
||||
|
||||
grpc::CompletionQueue cq;
|
||||
for (auto& c : calls)
|
||||
c.call(stub_, cq);
|
||||
|
||||
void* tag = nullptr;
|
||||
bool ok = false;
|
||||
size_t numFinished = 0;
|
||||
bool abort = false;
|
||||
size_t const incr = 500000;
|
||||
size_t progress = incr;
|
||||
std::vector<std::string> edgeKeys;
|
||||
|
||||
while (numFinished < calls.size() && cq.Next(&tag, &ok)) {
|
||||
ASSERT(tag != nullptr, "Tag can't be null.");
|
||||
auto ptr = static_cast<etl::impl::AsyncCallData*>(tag);
|
||||
|
||||
if (!ok) {
|
||||
LOG(log_.error()) << "loadInitialLedger - ok is false";
|
||||
return {{}, false}; // handle cancelled
|
||||
}
|
||||
|
||||
LOG(log_.trace()) << "Marker prefix = " << ptr->getMarkerPrefix();
|
||||
|
||||
auto result = ptr->process(stub_, cq, *backend_, abort, cacheOnly);
|
||||
if (result != etl::impl::AsyncCallData::CallStatus::MORE) {
|
||||
++numFinished;
|
||||
LOG(log_.debug()) << "Finished a marker. "
|
||||
<< "Current number of finished = " << numFinished;
|
||||
|
||||
if (auto lastKey = ptr->getLastKey(); !lastKey.empty())
|
||||
edgeKeys.push_back(std::move(lastKey));
|
||||
}
|
||||
|
||||
if (result == etl::impl::AsyncCallData::CallStatus::ERRORED)
|
||||
abort = true;
|
||||
|
||||
if (backend_->cache().size() > progress) {
|
||||
LOG(log_.info()) << "Downloaded " << backend_->cache().size() << " records from rippled";
|
||||
progress += incr;
|
||||
}
|
||||
}
|
||||
|
||||
LOG(log_.info()) << "Finished loadInitialLedger. cache size = " << backend_->cache().size() << ", abort = " << abort
|
||||
<< ".";
|
||||
return {std::move(edgeKeys), !abort};
|
||||
}
|
||||
|
||||
} // namespace etl::impl
|
||||
70
src/etl/impl/GrpcSource.hpp
Normal file
70
src/etl/impl/GrpcSource.hpp
Normal file
@@ -0,0 +1,70 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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 "data/BackendInterface.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <ripple/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
namespace etl::impl {
|
||||
|
||||
class GrpcSource {
|
||||
util::Logger log_;
|
||||
std::unique_ptr<org::xrpl::rpc::v1::XRPLedgerAPIService::Stub> stub_;
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
|
||||
public:
|
||||
GrpcSource(std::string const& ip, std::string const& grpcPort, std::shared_ptr<BackendInterface> backend);
|
||||
|
||||
/**
|
||||
* @brief Fetch data for a specific ledger.
|
||||
*
|
||||
* This function will continuously try to fetch data for the specified ledger until the fetch succeeds, the ledger
|
||||
* is found in the database, or the server is shutting down.
|
||||
*
|
||||
* @param sequence Sequence of the ledger to fetch
|
||||
* @param getObjects Whether to get the account state diff between this ledger and the prior one; defaults to true
|
||||
* @param getObjectNeighbors Whether to request object neighbors; defaults to false
|
||||
* @return A std::pair of the response status and the response itself
|
||||
*/
|
||||
std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>
|
||||
fetchLedger(uint32_t sequence, bool getObjects = true, bool getObjectNeighbors = false);
|
||||
|
||||
/**
|
||||
* @brief Download a ledger in full.
|
||||
*
|
||||
* @param sequence Sequence of the ledger to download
|
||||
* @param numMarkers Number of markers to generate for async calls
|
||||
* @param cacheOnly Only insert into cache, not the DB; defaults to false
|
||||
* @return A std::pair of the data and a bool indicating whether the download was successful
|
||||
*/
|
||||
std::pair<std::vector<std::string>, bool>
|
||||
loadInitialLedger(uint32_t sequence, uint32_t numMarkers, bool cacheOnly = false);
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
320
src/etl/impl/SubscriptionSource.cpp
Normal file
320
src/etl/impl/SubscriptionSource.cpp
Normal file
@@ -0,0 +1,320 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/SubscriptionSource.hpp"
|
||||
|
||||
#include "rpc/JS.hpp"
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/Retry.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include <boost/algorithm/string/classification.hpp>
|
||||
#include <boost/algorithm/string/split.hpp>
|
||||
#include <boost/asio/error.hpp>
|
||||
#include <boost/asio/executor_work_guard.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/post.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/ssl/error.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <boost/asio/use_future.hpp>
|
||||
#include <boost/beast/http/field.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <boost/json/serialize.hpp>
|
||||
#include <boost/json/value_to.hpp>
|
||||
#include <boost/lexical_cast.hpp>
|
||||
#include <fmt/core.h>
|
||||
#include <openssl/err.h>
|
||||
#include <ripple/protocol/jss.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <exception>
|
||||
#include <future>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
SubscriptionSource::~SubscriptionSource()
|
||||
{
|
||||
stop();
|
||||
retry_.cancel();
|
||||
|
||||
if (runFuture_.valid())
|
||||
runFuture_.wait();
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::run()
|
||||
{
|
||||
subscribe();
|
||||
}
|
||||
|
||||
bool
|
||||
SubscriptionSource::hasLedger(uint32_t sequence) const
|
||||
{
|
||||
auto validatedLedgersData = validatedLedgersData_.lock();
|
||||
for (auto& pair : validatedLedgersData->validatedLedgers) {
|
||||
if (sequence >= pair.first && sequence <= pair.second) {
|
||||
return true;
|
||||
}
|
||||
if (sequence < pair.first) {
|
||||
// validatedLedgers_ is a sorted list of disjoint ranges
|
||||
// if the sequence comes before this range, the sequence will
|
||||
// come before all subsequent ranges
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool
|
||||
SubscriptionSource::isConnected() const
|
||||
{
|
||||
return isConnected_;
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::setForwarding(bool isForwarding)
|
||||
{
|
||||
isForwarding_ = isForwarding;
|
||||
}
|
||||
|
||||
std::chrono::steady_clock::time_point
|
||||
SubscriptionSource::lastMessageTime() const
|
||||
{
|
||||
return lastMessageTime_.lock().get();
|
||||
}
|
||||
|
||||
std::string const&
|
||||
SubscriptionSource::validatedRange() const
|
||||
{
|
||||
return validatedLedgersData_.lock()->validatedLedgersRaw;
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::stop()
|
||||
{
|
||||
stop_ = true;
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::subscribe()
|
||||
{
|
||||
runFuture_ = boost::asio::spawn(
|
||||
strand_,
|
||||
[this, _ = boost::asio::make_work_guard(strand_)](boost::asio::yield_context yield) {
|
||||
auto connection = wsConnectionBuilder_.connect(yield);
|
||||
if (not connection) {
|
||||
handleError(connection.error(), yield);
|
||||
return;
|
||||
}
|
||||
|
||||
wsConnection_ = std::move(connection).value();
|
||||
isConnected_ = true;
|
||||
onConnect_();
|
||||
|
||||
auto const& subscribeCommand = getSubscribeCommandJson();
|
||||
auto const writeErrorOpt = wsConnection_->write(subscribeCommand, yield);
|
||||
if (writeErrorOpt) {
|
||||
handleError(writeErrorOpt.value(), yield);
|
||||
return;
|
||||
}
|
||||
|
||||
retry_.reset();
|
||||
|
||||
while (!stop_) {
|
||||
auto const message = wsConnection_->read(yield);
|
||||
if (not message) {
|
||||
handleError(message.error(), yield);
|
||||
return;
|
||||
}
|
||||
|
||||
auto const handleErrorOpt = handleMessage(message.value());
|
||||
if (handleErrorOpt) {
|
||||
handleError(handleErrorOpt.value(), yield);
|
||||
return;
|
||||
}
|
||||
}
|
||||
// Close the connection
|
||||
handleError(
|
||||
util::requests::RequestError{"Subscription source stopped", boost::asio::error::operation_aborted},
|
||||
yield
|
||||
);
|
||||
},
|
||||
boost::asio::use_future
|
||||
);
|
||||
}
|
||||
|
||||
std::optional<util::requests::RequestError>
|
||||
SubscriptionSource::handleMessage(std::string const& message)
|
||||
{
|
||||
setLastMessageTime();
|
||||
|
||||
try {
|
||||
auto const raw = boost::json::parse(message);
|
||||
auto const object = raw.as_object();
|
||||
uint32_t ledgerIndex = 0;
|
||||
|
||||
static constexpr char const* const JS_LedgerClosed = "ledgerClosed";
|
||||
static constexpr char const* const JS_ValidationReceived = "validationReceived";
|
||||
static constexpr char const* const JS_ManifestReceived = "manifestReceived";
|
||||
|
||||
if (object.contains(JS(result))) {
|
||||
auto const& result = object.at(JS(result)).as_object();
|
||||
if (result.contains(JS(ledger_index)))
|
||||
ledgerIndex = result.at(JS(ledger_index)).as_int64();
|
||||
|
||||
if (result.contains(JS(validated_ledgers))) {
|
||||
auto validatedLedgers = boost::json::value_to<std::string>(result.at(JS(validated_ledgers)));
|
||||
setValidatedRange(std::move(validatedLedgers));
|
||||
}
|
||||
LOG(log_.info()) << "Received a message on ledger subscription stream. Message : " << object;
|
||||
|
||||
} else if (object.contains(JS(type)) && object.at(JS(type)) == JS_LedgerClosed) {
|
||||
LOG(log_.info()) << "Received a message on ledger subscription stream. Message : " << object;
|
||||
if (object.contains(JS(ledger_index))) {
|
||||
ledgerIndex = object.at(JS(ledger_index)).as_int64();
|
||||
}
|
||||
if (object.contains(JS(validated_ledgers))) {
|
||||
auto validatedLedgers = boost::json::value_to<std::string>(object.at(JS(validated_ledgers)));
|
||||
setValidatedRange(std::move(validatedLedgers));
|
||||
}
|
||||
|
||||
} else {
|
||||
if (isForwarding_) {
|
||||
if (object.contains(JS(transaction))) {
|
||||
dependencies_.forwardProposedTransaction(object);
|
||||
} else if (object.contains(JS(type)) && object.at(JS(type)) == JS_ValidationReceived) {
|
||||
dependencies_.forwardValidation(object);
|
||||
} else if (object.contains(JS(type)) && object.at(JS(type)) == JS_ManifestReceived) {
|
||||
dependencies_.forwardManifest(object);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (ledgerIndex != 0) {
|
||||
LOG(log_.trace()) << "Pushing ledger sequence = " << ledgerIndex;
|
||||
dependencies_.pushValidatedLedger(ledgerIndex);
|
||||
}
|
||||
|
||||
return std::nullopt;
|
||||
} catch (std::exception const& e) {
|
||||
LOG(log_.error()) << "Exception in handleMessage : " << e.what();
|
||||
return util::requests::RequestError{fmt::format("Error handling message: {}", e.what())};
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::handleError(util::requests::RequestError const& error, boost::asio::yield_context yield)
|
||||
{
|
||||
isConnected_ = false;
|
||||
if (not stop_) {
|
||||
onDisconnect_();
|
||||
isForwarding_ = false;
|
||||
}
|
||||
|
||||
if (wsConnection_ != nullptr) {
|
||||
auto const error = wsConnection_->close(yield);
|
||||
if (error) {
|
||||
LOG(log_.error()) << "Error closing websocket connection: " << error->message();
|
||||
}
|
||||
wsConnection_.reset();
|
||||
}
|
||||
|
||||
logError(error);
|
||||
if (not stop_) {
|
||||
retry_.retry([this] { subscribe(); });
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::logError(util::requests::RequestError const& error) const
|
||||
{
|
||||
auto const& errorCodeOpt = error.errorCode();
|
||||
|
||||
if (not errorCodeOpt or
|
||||
(errorCodeOpt.value() != boost::asio::error::operation_aborted &&
|
||||
errorCodeOpt.value() != boost::asio::error::connection_refused)) {
|
||||
LOG(log_.error()) << error.message();
|
||||
} else {
|
||||
LOG(log_.warn()) << error.message();
|
||||
}
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::setLastMessageTime()
|
||||
{
|
||||
lastMessageTime_.lock().get() = std::chrono::steady_clock::now();
|
||||
}
|
||||
|
||||
void
|
||||
SubscriptionSource::setValidatedRange(std::string range)
|
||||
{
|
||||
std::vector<std::string> ranges;
|
||||
boost::split(ranges, range, [](char const c) { return c == ','; });
|
||||
|
||||
std::vector<std::pair<uint32_t, uint32_t>> pairs;
|
||||
pairs.reserve(ranges.size());
|
||||
for (auto& pair : ranges) {
|
||||
std::vector<std::string> minAndMax;
|
||||
|
||||
boost::split(minAndMax, pair, boost::is_any_of("-"));
|
||||
|
||||
if (minAndMax.size() == 1) {
|
||||
uint32_t const sequence = std::stoll(minAndMax[0]);
|
||||
pairs.emplace_back(sequence, sequence);
|
||||
} else {
|
||||
if (minAndMax.size() != 2) {
|
||||
throw std::runtime_error(fmt::format(
|
||||
"Error parsing range: {}.Min and max should be of size 2. Got size = {}", range, minAndMax.size()
|
||||
));
|
||||
}
|
||||
uint32_t const min = std::stoll(minAndMax[0]);
|
||||
uint32_t const max = std::stoll(minAndMax[1]);
|
||||
pairs.emplace_back(min, max);
|
||||
}
|
||||
}
|
||||
std::sort(pairs.begin(), pairs.end(), [](auto left, auto right) { return left.first < right.first; });
|
||||
|
||||
auto dataLock = validatedLedgersData_.lock();
|
||||
dataLock->validatedLedgers = std::move(pairs);
|
||||
dataLock->validatedLedgersRaw = std::move(range);
|
||||
}
|
||||
|
||||
std::string const&
|
||||
SubscriptionSource::getSubscribeCommandJson()
|
||||
{
|
||||
static boost::json::object const jsonValue{
|
||||
{"command", "subscribe"},
|
||||
{"streams", {"ledger", "manifests", "validations", "transactions_proposed"}},
|
||||
};
|
||||
static std::string const jsonString = boost::json::serialize(jsonValue);
|
||||
return jsonString;
|
||||
}
|
||||
|
||||
} // namespace etl::impl
|
||||
213
src/etl/impl/SubscriptionSource.hpp
Normal file
213
src/etl/impl/SubscriptionSource.hpp
Normal file
@@ -0,0 +1,213 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/SubscriptionSourceDependencies.hpp"
|
||||
#include "util/Mutex.hpp"
|
||||
#include "util/Retry.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
#include "util/requests/WsConnection.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <boost/beast/http/field.hpp>
|
||||
#include <fmt/core.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <future>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
/**
|
||||
* @brief This class is used to subscribe to a source of ledger data and forward it to the subscription manager.
|
||||
*/
|
||||
class SubscriptionSource {
|
||||
public:
|
||||
using OnConnectHook = std::function<void()>;
|
||||
using OnDisconnectHook = std::function<void()>;
|
||||
|
||||
private:
|
||||
util::Logger log_;
|
||||
util::requests::WsConnectionBuilder wsConnectionBuilder_;
|
||||
util::requests::WsConnectionPtr wsConnection_;
|
||||
|
||||
struct ValidatedLedgersData {
|
||||
std::vector<std::pair<uint32_t, uint32_t>> validatedLedgers;
|
||||
std::string validatedLedgersRaw{"N/A"};
|
||||
};
|
||||
util::Mutex<ValidatedLedgersData> validatedLedgersData_;
|
||||
|
||||
SubscriptionSourceDependencies dependencies_;
|
||||
|
||||
boost::asio::strand<boost::asio::io_context::executor_type> strand_;
|
||||
|
||||
util::Retry retry_;
|
||||
|
||||
OnConnectHook onConnect_;
|
||||
OnDisconnectHook onDisconnect_;
|
||||
|
||||
std::atomic_bool isConnected_{false};
|
||||
std::atomic_bool stop_{false};
|
||||
std::atomic_bool isForwarding_{false};
|
||||
|
||||
util::Mutex<std::chrono::steady_clock::time_point> lastMessageTime_;
|
||||
|
||||
std::future<void> runFuture_;
|
||||
|
||||
static constexpr std::chrono::seconds CONNECTION_TIMEOUT{30};
|
||||
static constexpr std::chrono::seconds RETRY_MAX_DELAY{30};
|
||||
static constexpr std::chrono::seconds RETRY_DELAY{1};
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Construct a new Subscription Source object
|
||||
*
|
||||
* @tparam NetworkValidatedLedgersType The type of the network validated ledgers object
|
||||
* @tparam SubscriptionManagerType The type of the subscription manager object
|
||||
* @param ioContext The io_context to use
|
||||
* @param ip The ip address of the source
|
||||
* @param wsPort The port of the source
|
||||
* @param validatedLedgers The network validated ledgers object
|
||||
* @param subscriptions The subscription manager object
|
||||
* @param onDisconnect The onDisconnect hook. Called when the connection is lost
|
||||
* @param connectionTimeout The connection timeout. Defaults to 30 seconds
|
||||
* @param retryDelay The retry delay. Defaults to 1 second
|
||||
*/
|
||||
template <typename NetworkValidatedLedgersType, typename SubscriptionManagerType>
|
||||
SubscriptionSource(
|
||||
boost::asio::io_context& ioContext,
|
||||
std::string const& ip,
|
||||
std::string const& wsPort,
|
||||
std::shared_ptr<NetworkValidatedLedgersType> validatedLedgers,
|
||||
std::shared_ptr<SubscriptionManagerType> subscriptions,
|
||||
OnConnectHook onConnect,
|
||||
OnDisconnectHook onDisconnect,
|
||||
std::chrono::steady_clock::duration const connectionTimeout = CONNECTION_TIMEOUT,
|
||||
std::chrono::steady_clock::duration const retryDelay = RETRY_DELAY
|
||||
)
|
||||
: log_(fmt::format("GrpcSource[{}:{}]", ip, wsPort))
|
||||
, wsConnectionBuilder_(ip, wsPort)
|
||||
, dependencies_(std::move(validatedLedgers), std::move(subscriptions))
|
||||
, strand_(boost::asio::make_strand(ioContext))
|
||||
, retry_(util::makeRetryExponentialBackoff(retryDelay, RETRY_MAX_DELAY, strand_))
|
||||
, onConnect_(std::move(onConnect))
|
||||
, onDisconnect_(std::move(onDisconnect))
|
||||
{
|
||||
wsConnectionBuilder_.addHeader({boost::beast::http::field::user_agent, "clio-client"})
|
||||
.addHeader({"X-User", "clio-client"})
|
||||
.setConnectionTimeout(connectionTimeout);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Destroy the Subscription Source object
|
||||
*
|
||||
* @note This will block to wait for all the async operations to complete. io_context must be still running
|
||||
*/
|
||||
~SubscriptionSource();
|
||||
|
||||
/**
|
||||
* @brief Run the source
|
||||
*/
|
||||
void
|
||||
run();
|
||||
|
||||
/**
|
||||
* @brief Check if the source has a ledger
|
||||
*
|
||||
* @param sequence The sequence of the ledger
|
||||
* @return true if the source has the ledger, false otherwise
|
||||
*/
|
||||
bool
|
||||
hasLedger(uint32_t sequence) const;
|
||||
|
||||
/**
|
||||
* @brief Check if the source is connected
|
||||
*
|
||||
* @return true if the source is connected, false otherwise
|
||||
*/
|
||||
bool
|
||||
isConnected() const;
|
||||
|
||||
/**
|
||||
* @brief Set source forwarding
|
||||
*
|
||||
* @note If forwarding is true the source will forward messages to the subscription manager. Forwarding is being
|
||||
* reset on disconnect.
|
||||
* @param isForwarding The new forwarding state
|
||||
*/
|
||||
void
|
||||
setForwarding(bool isForwarding);
|
||||
|
||||
/**
|
||||
* @brief Get the last message time (even if the last message had an error)
|
||||
*
|
||||
* @return The last message time
|
||||
*/
|
||||
std::chrono::steady_clock::time_point
|
||||
lastMessageTime() const;
|
||||
|
||||
/**
|
||||
* @brief Get the last received raw string of the validated ledgers
|
||||
*
|
||||
* @return The validated ledgers string
|
||||
*/
|
||||
std::string const&
|
||||
validatedRange() const;
|
||||
|
||||
/**
|
||||
* @brief Stop the source. The source will complete already scheduled operations but will not schedule new ones
|
||||
*/
|
||||
void
|
||||
stop();
|
||||
|
||||
private:
|
||||
void
|
||||
subscribe();
|
||||
|
||||
std::optional<util::requests::RequestError>
|
||||
handleMessage(std::string const& message);
|
||||
|
||||
void
|
||||
handleError(util::requests::RequestError const& error, boost::asio::yield_context yield);
|
||||
|
||||
void
|
||||
logError(util::requests::RequestError const& error) const;
|
||||
|
||||
void
|
||||
setLastMessageTime();
|
||||
|
||||
void
|
||||
setValidatedRange(std::string range);
|
||||
|
||||
static std::string const&
|
||||
getSubscribeCommandJson();
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
118
src/etl/impl/SubscriptionSourceDependencies.hpp
Normal file
118
src/etl/impl/SubscriptionSourceDependencies.hpp
Normal file
@@ -0,0 +1,118 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/object.hpp>
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
class SubscriptionSourceDependencies {
|
||||
struct Concept;
|
||||
std::unique_ptr<Concept> pImpl_;
|
||||
|
||||
public:
|
||||
template <typename NetworkValidatedLedgersType, typename SubscriptionManagerType>
|
||||
SubscriptionSourceDependencies(
|
||||
std::shared_ptr<NetworkValidatedLedgersType> networkValidatedLedgers,
|
||||
std::shared_ptr<SubscriptionManagerType> subscriptions
|
||||
)
|
||||
: pImpl_{std::make_unique<Model<NetworkValidatedLedgersType, SubscriptionManagerType>>(
|
||||
std::move(networkValidatedLedgers),
|
||||
std::move(subscriptions)
|
||||
)}
|
||||
{
|
||||
}
|
||||
|
||||
void
|
||||
forwardProposedTransaction(boost::json::object const& receivedTxJson)
|
||||
{
|
||||
pImpl_->forwardProposedTransaction(receivedTxJson);
|
||||
}
|
||||
|
||||
void
|
||||
forwardValidation(boost::json::object const& validationJson) const
|
||||
{
|
||||
pImpl_->forwardValidation(validationJson);
|
||||
}
|
||||
void
|
||||
forwardManifest(boost::json::object const& manifestJson) const
|
||||
{
|
||||
pImpl_->forwardManifest(manifestJson);
|
||||
}
|
||||
void
|
||||
pushValidatedLedger(uint32_t const idx)
|
||||
{
|
||||
pImpl_->pushValidatedLedger(idx);
|
||||
}
|
||||
|
||||
private:
|
||||
struct Concept {
|
||||
virtual ~Concept() = default;
|
||||
|
||||
virtual void
|
||||
forwardProposedTransaction(boost::json::object const& receivedTxJson) = 0;
|
||||
virtual void
|
||||
forwardValidation(boost::json::object const& validationJson) const = 0;
|
||||
virtual void
|
||||
forwardManifest(boost::json::object const& manifestJson) const = 0;
|
||||
virtual void
|
||||
pushValidatedLedger(uint32_t idx) = 0;
|
||||
};
|
||||
|
||||
template <typename SomeNetworkValidatedLedgersType, typename SomeSubscriptionManagerType>
|
||||
class Model : public Concept {
|
||||
std::shared_ptr<SomeNetworkValidatedLedgersType> networkValidatedLedgers_;
|
||||
std::shared_ptr<SomeSubscriptionManagerType> subscriptions_;
|
||||
|
||||
public:
|
||||
Model(
|
||||
std::shared_ptr<SomeNetworkValidatedLedgersType> networkValidatedLedgers,
|
||||
std::shared_ptr<SomeSubscriptionManagerType> subscriptions
|
||||
)
|
||||
: networkValidatedLedgers_{std::move(networkValidatedLedgers)}, subscriptions_{std::move(subscriptions)}
|
||||
{
|
||||
}
|
||||
void
|
||||
forwardProposedTransaction(boost::json::object const& receivedTxJson) override
|
||||
{
|
||||
subscriptions_->forwardProposedTransaction(receivedTxJson);
|
||||
}
|
||||
void
|
||||
forwardValidation(boost::json::object const& validationJson) const override
|
||||
{
|
||||
subscriptions_->forwardValidation(validationJson);
|
||||
}
|
||||
void
|
||||
forwardManifest(boost::json::object const& manifestJson) const override
|
||||
{
|
||||
subscriptions_->forwardManifest(manifestJson);
|
||||
}
|
||||
void
|
||||
pushValidatedLedger(uint32_t idx) override
|
||||
{
|
||||
networkValidatedLedgers_->push(idx);
|
||||
}
|
||||
};
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
@@ -160,6 +160,8 @@ start(io_context& ioc, std::uint32_t numThreads)
|
||||
v.emplace_back([&ioc] { ioc.run(); });
|
||||
|
||||
ioc.run();
|
||||
for (auto& t : v)
|
||||
t.join();
|
||||
}
|
||||
|
||||
int
|
||||
|
||||
122
src/util/Mutex.hpp
Normal file
122
src/util/Mutex.hpp
Normal file
@@ -0,0 +1,122 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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 <mutex>
|
||||
#include <type_traits>
|
||||
|
||||
namespace util {
|
||||
|
||||
template <typename ProtectedDataType>
|
||||
class Mutex;
|
||||
|
||||
/**
|
||||
* @brief A lock on a mutex that provides access to the protected data.
|
||||
*
|
||||
* @tparam ProtectedDataType data type to hold
|
||||
*/
|
||||
template <typename ProtectedDataType>
|
||||
class Lock {
|
||||
std::scoped_lock<std::mutex> lock_;
|
||||
ProtectedDataType& data_;
|
||||
|
||||
public:
|
||||
ProtectedDataType const&
|
||||
operator*() const
|
||||
{
|
||||
return data_;
|
||||
}
|
||||
|
||||
ProtectedDataType&
|
||||
operator*()
|
||||
{
|
||||
return data_;
|
||||
}
|
||||
|
||||
ProtectedDataType const&
|
||||
get() const
|
||||
{
|
||||
return data_;
|
||||
}
|
||||
|
||||
ProtectedDataType&
|
||||
get()
|
||||
{
|
||||
return data_;
|
||||
}
|
||||
|
||||
ProtectedDataType const*
|
||||
operator->() const
|
||||
{
|
||||
return &data_;
|
||||
}
|
||||
|
||||
ProtectedDataType*
|
||||
operator->()
|
||||
{
|
||||
return &data_;
|
||||
}
|
||||
|
||||
private:
|
||||
friend class Mutex<std::remove_const_t<ProtectedDataType>>;
|
||||
|
||||
explicit Lock(std::mutex& mutex, ProtectedDataType& data) : lock_(mutex), data_(data)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief A container for data that is protected by a mutex. Inspired by Mutex in Rust.
|
||||
*
|
||||
* @tparam ProtectedDataType data type to hold
|
||||
*/
|
||||
template <typename ProtectedDataType>
|
||||
class Mutex {
|
||||
mutable std::mutex mutex_;
|
||||
ProtectedDataType data_;
|
||||
|
||||
public:
|
||||
Mutex() = default;
|
||||
|
||||
explicit Mutex(ProtectedDataType data) : data_(std::move(data))
|
||||
{
|
||||
}
|
||||
|
||||
template <typename... Args>
|
||||
static Mutex
|
||||
make(Args&&... args)
|
||||
{
|
||||
return Mutex{ProtectedDataType{std::forward<Args>(args)...}};
|
||||
}
|
||||
|
||||
Lock<ProtectedDataType const>
|
||||
lock() const
|
||||
{
|
||||
return Lock<ProtectedDataType const>{mutex_, data_};
|
||||
}
|
||||
|
||||
Lock<ProtectedDataType>
|
||||
lock()
|
||||
{
|
||||
return Lock<ProtectedDataType>{mutex_, data_};
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace util
|
||||
115
src/util/Retry.cpp
Normal file
115
src/util/Retry.cpp
Normal file
@@ -0,0 +1,115 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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 "util/Retry.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
|
||||
#include <algorithm>
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace util {
|
||||
|
||||
RetryStrategy::RetryStrategy(std::chrono::steady_clock::duration delay) : initialDelay_(delay), delay_(delay)
|
||||
{
|
||||
}
|
||||
|
||||
std::chrono::steady_clock::duration
|
||||
RetryStrategy::getDelay() const
|
||||
{
|
||||
return delay_;
|
||||
}
|
||||
|
||||
void
|
||||
RetryStrategy::increaseDelay()
|
||||
{
|
||||
delay_ = nextDelay();
|
||||
}
|
||||
|
||||
void
|
||||
RetryStrategy::reset()
|
||||
{
|
||||
delay_ = initialDelay_;
|
||||
}
|
||||
|
||||
Retry::Retry(RetryStrategyPtr strategy, boost::asio::strand<boost::asio::io_context::executor_type> strand)
|
||||
: strategy_(std::move(strategy)), timer_(strand.get_inner_executor())
|
||||
{
|
||||
}
|
||||
|
||||
Retry::~Retry()
|
||||
{
|
||||
cancel();
|
||||
}
|
||||
|
||||
void
|
||||
Retry::cancel()
|
||||
{
|
||||
timer_.cancel();
|
||||
}
|
||||
|
||||
size_t
|
||||
Retry::attemptNumber() const
|
||||
{
|
||||
return attemptNumber_;
|
||||
}
|
||||
|
||||
std::chrono::steady_clock::duration
|
||||
Retry::delayValue() const
|
||||
{
|
||||
return strategy_->getDelay();
|
||||
}
|
||||
|
||||
void
|
||||
Retry::reset()
|
||||
{
|
||||
attemptNumber_ = 0;
|
||||
strategy_->reset();
|
||||
}
|
||||
|
||||
ExponentialBackoffStrategy::ExponentialBackoffStrategy(
|
||||
std::chrono::steady_clock::duration delay,
|
||||
std::chrono::steady_clock::duration maxDelay
|
||||
)
|
||||
: RetryStrategy(delay), maxDelay_(maxDelay)
|
||||
{
|
||||
}
|
||||
|
||||
std::chrono::steady_clock::duration
|
||||
ExponentialBackoffStrategy::nextDelay() const
|
||||
{
|
||||
auto const next = getDelay() * 2;
|
||||
return std::min(next, maxDelay_);
|
||||
}
|
||||
|
||||
Retry
|
||||
makeRetryExponentialBackoff(
|
||||
std::chrono::steady_clock::duration delay,
|
||||
std::chrono::steady_clock::duration maxDelay,
|
||||
boost::asio::strand<boost::asio::io_context::executor_type> strand
|
||||
)
|
||||
{
|
||||
return Retry(std::make_unique<ExponentialBackoffStrategy>(delay, maxDelay), std::move(strand));
|
||||
}
|
||||
|
||||
} // namespace util
|
||||
171
src/util/Retry.hpp
Normal file
171
src/util/Retry.hpp
Normal file
@@ -0,0 +1,171 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/asio/error.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/steady_timer.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <memory>
|
||||
|
||||
namespace util {
|
||||
|
||||
/**
|
||||
* @brief Interface for retry strategies
|
||||
*/
|
||||
class RetryStrategy {
|
||||
std::chrono::steady_clock::duration initialDelay_;
|
||||
std::chrono::steady_clock::duration delay_;
|
||||
|
||||
public:
|
||||
RetryStrategy(std::chrono::steady_clock::duration delay);
|
||||
virtual ~RetryStrategy() = default;
|
||||
|
||||
/**
|
||||
* @brief Get the current delay value
|
||||
*
|
||||
* @return std::chrono::steady_clock::duration
|
||||
*/
|
||||
std::chrono::steady_clock::duration
|
||||
getDelay() const;
|
||||
|
||||
/**
|
||||
* @brief Increase the delay value
|
||||
*/
|
||||
void
|
||||
increaseDelay();
|
||||
|
||||
/**
|
||||
* @brief Reset the delay value
|
||||
*/
|
||||
void
|
||||
reset();
|
||||
|
||||
protected:
|
||||
/**
|
||||
* @brief Compute the next delay value
|
||||
*
|
||||
* @return std::chrono::steady_clock::duration
|
||||
*/
|
||||
virtual std::chrono::steady_clock::duration
|
||||
nextDelay() const = 0;
|
||||
};
|
||||
using RetryStrategyPtr = std::unique_ptr<RetryStrategy>;
|
||||
|
||||
/**
|
||||
* @brief A retry mechanism
|
||||
*/
|
||||
class Retry {
|
||||
RetryStrategyPtr strategy_;
|
||||
boost::asio::steady_timer timer_;
|
||||
size_t attemptNumber_ = 0;
|
||||
|
||||
public:
|
||||
Retry(RetryStrategyPtr strategy, boost::asio::strand<boost::asio::io_context::executor_type> strand);
|
||||
~Retry();
|
||||
|
||||
/**
|
||||
* @brief Schedule a retry
|
||||
*
|
||||
* @tparam Fn The type of the callable to execute
|
||||
* @param func The callable to execute
|
||||
*/
|
||||
template <typename Fn>
|
||||
void
|
||||
retry(Fn&& func)
|
||||
{
|
||||
timer_.expires_after(strategy_->getDelay());
|
||||
strategy_->increaseDelay();
|
||||
timer_.async_wait([this, func = std::forward<Fn>(func)](boost::system::error_code const& ec) {
|
||||
if (ec == boost::asio::error::operation_aborted) {
|
||||
return;
|
||||
}
|
||||
++attemptNumber_;
|
||||
func();
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Cancel scheduled retry if any
|
||||
*/
|
||||
void
|
||||
cancel();
|
||||
|
||||
/**
|
||||
* @brief Get the current attempt number
|
||||
*
|
||||
* @return size_t
|
||||
*/
|
||||
size_t
|
||||
attemptNumber() const;
|
||||
|
||||
/**
|
||||
* @brief Get the current delay value
|
||||
*
|
||||
* @return std::chrono::steady_clock::duration
|
||||
*/
|
||||
std::chrono::steady_clock::duration
|
||||
delayValue() const;
|
||||
|
||||
/**
|
||||
* @brief Reset the delay value and attempt number
|
||||
*/
|
||||
void
|
||||
reset();
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Create a retry mechanism with exponential backoff strategy
|
||||
*
|
||||
* @param delay The initial delay value
|
||||
* @param maxDelay The maximum delay value
|
||||
* @param strand The strand to use for async operations
|
||||
* @return Retry
|
||||
*/
|
||||
class ExponentialBackoffStrategy : public RetryStrategy {
|
||||
std::chrono::steady_clock::duration maxDelay_;
|
||||
|
||||
public:
|
||||
ExponentialBackoffStrategy(std::chrono::steady_clock::duration delay, std::chrono::steady_clock::duration maxDelay);
|
||||
|
||||
private:
|
||||
std::chrono::steady_clock::duration
|
||||
nextDelay() const override;
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Create a retry mechanism with exponential backoff strategy
|
||||
*
|
||||
* @param delay The initial delay value
|
||||
* @param maxDelay The maximum delay value
|
||||
* @param strand The strand to use for async operations
|
||||
* @return Retry
|
||||
*/
|
||||
Retry
|
||||
makeRetryExponentialBackoff(
|
||||
std::chrono::steady_clock::duration delay,
|
||||
std::chrono::steady_clock::duration maxDelay,
|
||||
boost::asio::strand<boost::asio::io_context::executor_type> strand
|
||||
);
|
||||
|
||||
} // namespace util
|
||||
@@ -28,7 +28,9 @@
|
||||
#include <boost/filesystem/path.hpp>
|
||||
#include <boost/json/conversion.hpp>
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/log/attributes/attribute_value_set.hpp>
|
||||
#include <boost/log/core/core.hpp>
|
||||
#include <boost/log/expressions/filter.hpp>
|
||||
#include <boost/log/expressions/predicates/channel_severity_filter.hpp>
|
||||
#include <boost/log/keywords/auto_flush.hpp>
|
||||
#include <boost/log/keywords/file_name.hpp>
|
||||
@@ -55,11 +57,14 @@
|
||||
#include <ostream>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
#include <utility>
|
||||
|
||||
namespace util {
|
||||
|
||||
Logger LogService::general_log_ = Logger{"General"};
|
||||
Logger LogService::alert_log_ = Logger{"Alert"};
|
||||
boost::log::filter LogService::filter_{};
|
||||
|
||||
std::ostream&
|
||||
operator<<(std::ostream& stream, Severity sev)
|
||||
@@ -156,9 +161,7 @@ LogService::init(util::Config const& config)
|
||||
"Performance",
|
||||
};
|
||||
|
||||
auto core = boost::log::core::get();
|
||||
auto min_severity = boost::log::expressions::channel_severity_filter(log_channel, log_severity);
|
||||
|
||||
std::unordered_map<std::string, Severity> min_severity;
|
||||
for (auto const& channel : channels)
|
||||
min_severity[channel] = defaultSeverity;
|
||||
min_severity["Alert"] = Severity::WRN; // Channel for alerts, always warning severity
|
||||
@@ -171,7 +174,19 @@ LogService::init(util::Config const& config)
|
||||
min_severity[name] = cfg.valueOr<Severity>("log_level", defaultSeverity);
|
||||
}
|
||||
|
||||
core->set_filter(min_severity);
|
||||
auto log_filter = [min_severity = std::move(min_severity),
|
||||
defaultSeverity](boost::log::attribute_value_set const& attributes) -> bool {
|
||||
auto const channel = attributes[log_channel];
|
||||
auto const severity = attributes[log_severity];
|
||||
if (!channel || !severity)
|
||||
return false;
|
||||
if (auto const it = min_severity.find(channel.get()); it != min_severity.end())
|
||||
return severity.get() >= it->second;
|
||||
return severity.get() >= defaultSeverity;
|
||||
};
|
||||
|
||||
filter_ = boost::log::filter{std::move(log_filter)};
|
||||
boost::log::core::get()->set_filter(filter_);
|
||||
LOG(LogService::info()) << "Default log level = " << defaultSeverity;
|
||||
}
|
||||
|
||||
|
||||
@@ -28,6 +28,7 @@
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/log/core/core.hpp>
|
||||
#include <boost/log/core/record.hpp>
|
||||
#include <boost/log/expressions/filter.hpp>
|
||||
#include <boost/log/expressions/keyword.hpp>
|
||||
#include <boost/log/expressions/predicates/channel_severity_filter.hpp>
|
||||
#include <boost/log/keywords/channel.hpp>
|
||||
@@ -226,6 +227,7 @@ public:
|
||||
class LogService {
|
||||
static Logger general_log_; /*< Global logger for General channel */
|
||||
static Logger alert_log_; /*< Global logger for Alerts channel */
|
||||
static boost::log::filter filter_;
|
||||
|
||||
public:
|
||||
LogService() = delete;
|
||||
|
||||
@@ -20,6 +20,7 @@
|
||||
#include "util/requests/RequestBuilder.hpp"
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
#include "util/requests/impl/StreamData.hpp"
|
||||
|
||||
@@ -43,6 +44,7 @@
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
#include <variant>
|
||||
#include <vector>
|
||||
|
||||
namespace util::requests {
|
||||
@@ -61,7 +63,7 @@ RequestBuilder::RequestBuilder(std::string host, std::string port) : host_(std::
|
||||
RequestBuilder&
|
||||
RequestBuilder::addHeader(HttpHeader const& header)
|
||||
{
|
||||
request_.set(header.name, header.value);
|
||||
std::visit([&](auto const& name) { request_.set(name, header.value); }, header.name);
|
||||
return *this;
|
||||
}
|
||||
|
||||
@@ -95,11 +97,16 @@ RequestBuilder::setTarget(std::string_view target)
|
||||
return *this;
|
||||
}
|
||||
|
||||
RequestBuilder&
|
||||
RequestBuilder::setSslEnabled(bool const enabled)
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::getSsl(boost::asio::yield_context yield)
|
||||
{
|
||||
sslEnabled_ = enabled;
|
||||
return *this;
|
||||
return doSslRequest(yield, http::verb::get);
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::getPlain(boost::asio::yield_context yield)
|
||||
{
|
||||
return doPlainRequest(yield, http::verb::get);
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
@@ -108,6 +115,18 @@ RequestBuilder::get(asio::yield_context yield)
|
||||
return doRequest(yield, http::verb::get);
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::postSsl(boost::asio::yield_context yield)
|
||||
{
|
||||
return doSslRequest(yield, http::verb::post);
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::postPlain(boost::asio::yield_context yield)
|
||||
{
|
||||
return doPlainRequest(yield, http::verb::post);
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::post(asio::yield_context yield)
|
||||
{
|
||||
@@ -115,28 +134,41 @@ RequestBuilder::post(asio::yield_context yield)
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::doRequest(asio::yield_context yield, beast::http::verb method)
|
||||
RequestBuilder::doSslRequest(asio::yield_context yield, beast::http::verb method)
|
||||
{
|
||||
if (sslEnabled_) {
|
||||
auto streamData = impl::SslTcpStreamData::create(yield);
|
||||
if (not streamData.has_value())
|
||||
return Unexpected{std::move(streamData).error()};
|
||||
auto streamData = impl::SslTcpStreamData::create(yield);
|
||||
if (not streamData.has_value())
|
||||
return Unexpected{std::move(streamData).error()};
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wold-style-cast"
|
||||
if (!SSL_set_tlsext_host_name(streamData->stream.native_handle(), host_.c_str())) {
|
||||
if (!SSL_set_tlsext_host_name(streamData->stream.native_handle(), host_.c_str())) {
|
||||
#pragma GCC diagnostic pop
|
||||
beast::error_code errorCode;
|
||||
errorCode.assign(static_cast<int>(::ERR_get_error()), asio::error::get_ssl_category());
|
||||
return Unexpected{RequestError{"SSL setup failed", errorCode}};
|
||||
}
|
||||
return doRequestImpl(std::move(streamData).value(), yield, method);
|
||||
beast::error_code errorCode;
|
||||
errorCode.assign(static_cast<int>(::ERR_get_error()), asio::error::get_ssl_category());
|
||||
return Unexpected{RequestError{"SSL setup failed", errorCode}};
|
||||
}
|
||||
return doRequestImpl(std::move(streamData).value(), yield, method);
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::doPlainRequest(asio::yield_context yield, beast::http::verb method)
|
||||
{
|
||||
auto streamData = impl::TcpStreamData{yield};
|
||||
return doRequestImpl(std::move(streamData), yield, method);
|
||||
}
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::doRequest(asio::yield_context yield, beast::http::verb method)
|
||||
{
|
||||
auto result = doSslRequest(yield, method);
|
||||
if (result.has_value())
|
||||
return result;
|
||||
|
||||
LOG(log_.debug()) << "SSL request failed: " << result.error().message() << ". Falling back to plain request.";
|
||||
return doPlainRequest(yield, method);
|
||||
}
|
||||
|
||||
template <typename StreamDataType>
|
||||
Expected<std::string, RequestError>
|
||||
RequestBuilder::doRequestImpl(StreamDataType&& streamData, asio::yield_context yield, http::verb const method)
|
||||
@@ -178,7 +210,7 @@ RequestBuilder::doRequestImpl(StreamDataType&& streamData, asio::yield_context y
|
||||
return Unexpected{RequestError{"Read error", errorCode}};
|
||||
|
||||
if (response.result() != http::status::ok)
|
||||
return Unexpected{RequestError{"Response status not OK"}};
|
||||
return Unexpected{RequestError{"Response status is not OK"}};
|
||||
|
||||
beast::get_lowest_layer(stream).socket().shutdown(tcp::socket::shutdown_both, errorCode);
|
||||
|
||||
|
||||
@@ -20,6 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
@@ -42,11 +43,11 @@ namespace util::requests {
|
||||
* @brief Builder for HTTP requests
|
||||
*/
|
||||
class RequestBuilder {
|
||||
util::Logger log_{"RequestBuilder"};
|
||||
std::string host_;
|
||||
std::string port_;
|
||||
std::chrono::milliseconds timeout_{DEFAULT_TIMEOUT};
|
||||
boost::beast::http::request<boost::beast::http::string_body> request_;
|
||||
bool sslEnabled_{false};
|
||||
|
||||
public:
|
||||
/**
|
||||
@@ -107,18 +108,32 @@ public:
|
||||
setTarget(std::string_view target);
|
||||
|
||||
/**
|
||||
* @brief Set SSL enabled or disabled
|
||||
* @brief Perform a GET request with SSL asynchronously
|
||||
*
|
||||
* @note Default is false
|
||||
* @note It is not thread-safe to call get() and post() of the same RequestBuilder from multiple threads. But it is
|
||||
* fine to call only get() or only post() of the same RequestBuilder from multiple threads.
|
||||
*
|
||||
* @param ssl boolean value to set
|
||||
* @return reference to itself
|
||||
* @param yield yield context
|
||||
* @return expected response or error
|
||||
*/
|
||||
RequestBuilder&
|
||||
setSslEnabled(bool enabled);
|
||||
Expected<std::string, RequestError>
|
||||
getSsl(boost::asio::yield_context yield);
|
||||
|
||||
/**
|
||||
* @brief Perform a GET request asynchronously
|
||||
* @brief Perform a GET request without SSL asynchronously
|
||||
*
|
||||
* @note It is not thread-safe to call get() and post() of the same RequestBuilder from multiple threads. But it is
|
||||
* fine to call only get() or only post() of the same RequestBuilder from multiple threads.
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return expected response or error
|
||||
*/
|
||||
Expected<std::string, RequestError>
|
||||
getPlain(boost::asio::yield_context yield);
|
||||
|
||||
/**
|
||||
* @brief Perform a GET request asynchronously. The SSL will be used first, if it fails, the plain connection will
|
||||
* be used.
|
||||
*
|
||||
* @note It is not thread-safe to call get() and post() of the same RequestBuilder from multiple threads. But it is
|
||||
* fine to call only get() or only post() of the same RequestBuilder from multiple threads.
|
||||
@@ -130,7 +145,32 @@ public:
|
||||
get(boost::asio::yield_context yield);
|
||||
|
||||
/**
|
||||
* @brief Perform a POST request asynchronously
|
||||
* @brief Perform a POST request with SSL asynchronously
|
||||
*
|
||||
* @note It is not thread-safe to call get() and post() of the same RequestBuilder from multiple threads. But it is
|
||||
* fine to call only get() or only post() of the same RequestBuilder from multiple threads.
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return expected response or error
|
||||
*/
|
||||
Expected<std::string, RequestError>
|
||||
postSsl(boost::asio::yield_context yield);
|
||||
|
||||
/**
|
||||
* @brief Perform a POST request without SSL asynchronously
|
||||
*
|
||||
* @note It is not thread-safe to call get() and post() of the same RequestBuilder from multiple threads. But it is
|
||||
* fine to call only get() or only post() of the same RequestBuilder from multiple threads.
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return expected response or error
|
||||
*/
|
||||
Expected<std::string, RequestError>
|
||||
postPlain(boost::asio::yield_context yield);
|
||||
|
||||
/**
|
||||
* @brief Perform a POST request asynchronously. The SSL will be used first, if it fails, the plain connection will
|
||||
* be used.
|
||||
*
|
||||
* @note It is not thread-safe to call get() and post() of the same RequestBuilder from multiple threads. But it is
|
||||
* fine to call only get() or only post() of the same RequestBuilder from multiple threads.
|
||||
@@ -144,6 +184,12 @@ public:
|
||||
static constexpr std::chrono::milliseconds DEFAULT_TIMEOUT{30000};
|
||||
|
||||
private:
|
||||
Expected<std::string, RequestError>
|
||||
doSslRequest(boost::asio::yield_context yield, boost::beast::http::verb method);
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
doPlainRequest(boost::asio::yield_context yield, boost::beast::http::verb method);
|
||||
|
||||
Expected<std::string, RequestError>
|
||||
doRequest(boost::asio::yield_context yield, boost::beast::http::verb method);
|
||||
|
||||
|
||||
@@ -19,26 +19,50 @@
|
||||
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include "util/requests/impl/SslContext.hpp"
|
||||
|
||||
#include <boost/beast/core/error.hpp>
|
||||
#include <boost/beast/http/field.hpp>
|
||||
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
|
||||
namespace util::requests {
|
||||
|
||||
RequestError::RequestError(std::string message) : message(std::move(message))
|
||||
RequestError::RequestError(std::string message) : message_(std::move(message))
|
||||
{
|
||||
}
|
||||
|
||||
RequestError::RequestError(std::string msg, boost::beast::error_code const& ec) : message(std::move(msg))
|
||||
RequestError::RequestError(std::string message, boost::beast::error_code errorCode)
|
||||
: message_(std::move(message)), errorCode_(errorCode)
|
||||
{
|
||||
message.append(": ");
|
||||
message.append(ec.message());
|
||||
message_.append(": ");
|
||||
if (auto const sslError = impl::sslErrorToString(errorCode); sslError.has_value()) {
|
||||
message_.append(sslError.value());
|
||||
} else {
|
||||
message_.append(errorCode.message());
|
||||
}
|
||||
}
|
||||
|
||||
std::string const&
|
||||
RequestError::message() const
|
||||
{
|
||||
return message_;
|
||||
}
|
||||
|
||||
std::optional<boost::beast::error_code> const&
|
||||
RequestError::errorCode() const
|
||||
{
|
||||
return errorCode_;
|
||||
}
|
||||
|
||||
HttpHeader::HttpHeader(boost::beast::http::field name, std::string value) : name(name), value(std::move(value))
|
||||
{
|
||||
}
|
||||
|
||||
HttpHeader::HttpHeader(std::string name, std::string value) : name(std::move(name)), value(std::move(value))
|
||||
{
|
||||
}
|
||||
|
||||
} // namespace util::requests
|
||||
|
||||
@@ -22,14 +22,20 @@
|
||||
#include <boost/beast/core/error.hpp>
|
||||
#include <boost/beast/http/field.hpp>
|
||||
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <variant>
|
||||
|
||||
namespace util::requests {
|
||||
|
||||
/**
|
||||
* @brief Error type for HTTP requests
|
||||
*/
|
||||
struct RequestError {
|
||||
class RequestError {
|
||||
std::string message_;
|
||||
std::optional<boost::beast::error_code> errorCode_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Construct a new Request Error object
|
||||
*
|
||||
@@ -43,9 +49,23 @@ struct RequestError {
|
||||
* @param message error message
|
||||
* @param ec error code from boost::beast
|
||||
*/
|
||||
RequestError(std::string msg, boost::beast::error_code const& ec);
|
||||
RequestError(std::string message, boost::beast::error_code errorCode);
|
||||
|
||||
std::string message;
|
||||
/**
|
||||
* @brief Get the error message
|
||||
*
|
||||
* @return std::string
|
||||
*/
|
||||
std::string const&
|
||||
message() const;
|
||||
|
||||
/**
|
||||
* @brief Get the error code, if any
|
||||
*
|
||||
* @return std::optional<boost::beast::error_code>
|
||||
*/
|
||||
std::optional<boost::beast::error_code> const&
|
||||
errorCode() const;
|
||||
};
|
||||
|
||||
/**
|
||||
@@ -53,8 +73,9 @@ struct RequestError {
|
||||
*/
|
||||
struct HttpHeader {
|
||||
HttpHeader(boost::beast::http::field name, std::string value);
|
||||
HttpHeader(std::string name, std::string value);
|
||||
|
||||
boost::beast::http::field name;
|
||||
std::variant<boost::beast::http::field, std::string> name;
|
||||
std::string value;
|
||||
};
|
||||
|
||||
|
||||
@@ -20,6 +20,7 @@
|
||||
#include "util/requests/WsConnection.hpp"
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
#include "util/requests/impl/StreamData.hpp"
|
||||
#include "util/requests/impl/WsConnectionImpl.hpp"
|
||||
@@ -78,39 +79,53 @@ WsConnectionBuilder::setTarget(std::string target)
|
||||
}
|
||||
|
||||
WsConnectionBuilder&
|
||||
WsConnectionBuilder::setConnectionTimeout(std::chrono::milliseconds timeout)
|
||||
WsConnectionBuilder::setConnectionTimeout(std::chrono::steady_clock::duration timeout)
|
||||
{
|
||||
timeout_ = timeout;
|
||||
connectionTimeout_ = timeout;
|
||||
return *this;
|
||||
}
|
||||
|
||||
WsConnectionBuilder&
|
||||
WsConnectionBuilder::setSslEnabled(bool sslEnabled)
|
||||
WsConnectionBuilder::setWsHandshakeTimeout(std::chrono::steady_clock::duration timeout)
|
||||
{
|
||||
sslEnabled_ = sslEnabled;
|
||||
wsHandshakeTimeout_ = timeout;
|
||||
return *this;
|
||||
}
|
||||
|
||||
Expected<WsConnectionPtr, RequestError>
|
||||
WsConnectionBuilder::sslConnect(asio::yield_context yield) const
|
||||
{
|
||||
auto streamData = impl::SslWsStreamData::create(yield);
|
||||
if (not streamData.has_value())
|
||||
return Unexpected{std::move(streamData).error()};
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wold-style-cast"
|
||||
if (!SSL_set_tlsext_host_name(streamData->stream.next_layer().native_handle(), host_.c_str())) {
|
||||
#pragma GCC diagnostic pop
|
||||
beast::error_code errorCode;
|
||||
errorCode.assign(static_cast<int>(::ERR_get_error()), beast::net::error::get_ssl_category());
|
||||
return Unexpected{RequestError{"SSL setup failed", errorCode}};
|
||||
}
|
||||
return connectImpl(std::move(streamData).value(), yield);
|
||||
}
|
||||
|
||||
Expected<WsConnectionPtr, RequestError>
|
||||
WsConnectionBuilder::plainConnect(asio::yield_context yield) const
|
||||
{
|
||||
return connectImpl(impl::WsStreamData{yield}, yield);
|
||||
}
|
||||
|
||||
Expected<WsConnectionPtr, RequestError>
|
||||
WsConnectionBuilder::connect(asio::yield_context yield) const
|
||||
{
|
||||
if (sslEnabled_) {
|
||||
auto streamData = impl::SslWsStreamData::create(yield);
|
||||
if (not streamData.has_value())
|
||||
return Unexpected{std::move(streamData).error()};
|
||||
auto sslConnection = sslConnect(yield);
|
||||
if (sslConnection.has_value())
|
||||
return sslConnection;
|
||||
LOG(log_.debug()) << "SSL connection failed with error: " << sslConnection.error().message()
|
||||
<< ". Falling back to plain connection.";
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wold-style-cast"
|
||||
if (!SSL_set_tlsext_host_name(streamData->stream.next_layer().native_handle(), host_.c_str())) {
|
||||
#pragma GCC diagnostic pop
|
||||
beast::error_code errorCode;
|
||||
errorCode.assign(static_cast<int>(::ERR_get_error()), beast::net::error::get_ssl_category());
|
||||
return Unexpected{RequestError{"SSL setup failed", errorCode}};
|
||||
}
|
||||
return connectImpl(std::move(streamData).value(), yield);
|
||||
}
|
||||
|
||||
return connectImpl(impl::WsStreamData{yield}, yield);
|
||||
return plainConnect(yield);
|
||||
}
|
||||
|
||||
template <typename StreamDataType>
|
||||
@@ -127,13 +142,13 @@ WsConnectionBuilder::connectImpl(StreamDataType&& streamData, asio::yield_contex
|
||||
|
||||
auto& ws = streamData.stream;
|
||||
|
||||
beast::get_lowest_layer(ws).expires_after(timeout_);
|
||||
beast::get_lowest_layer(ws).expires_after(connectionTimeout_);
|
||||
auto endpoint = beast::get_lowest_layer(ws).async_connect(results, yield[errorCode]);
|
||||
if (errorCode)
|
||||
return Unexpected{RequestError{"Connect error", errorCode}};
|
||||
|
||||
if constexpr (StreamDataType::sslEnabled) {
|
||||
beast::get_lowest_layer(ws).expires_after(timeout_);
|
||||
beast::get_lowest_layer(ws).expires_after(connectionTimeout_);
|
||||
ws.next_layer().async_handshake(asio::ssl::stream_base::client, yield[errorCode]);
|
||||
if (errorCode)
|
||||
return Unexpected{RequestError{"SSL handshake error", errorCode}};
|
||||
@@ -142,10 +157,12 @@ WsConnectionBuilder::connectImpl(StreamDataType&& streamData, asio::yield_contex
|
||||
// Turn off the timeout on the tcp_stream, because the websocket stream has its own timeout system
|
||||
beast::get_lowest_layer(ws).expires_never();
|
||||
|
||||
ws.set_option(websocket::stream_base::timeout::suggested(beast::role_type::client));
|
||||
auto wsTimeout = websocket::stream_base::timeout::suggested(beast::role_type::client);
|
||||
wsTimeout.handshake_timeout = wsHandshakeTimeout_;
|
||||
ws.set_option(wsTimeout);
|
||||
ws.set_option(websocket::stream_base::decorator([this](websocket::request_type& req) {
|
||||
for (auto const& header : headers_)
|
||||
req.set(header.name, header.value);
|
||||
std::visit([&](auto const& name) { req.set(name, header.value); }, header.name);
|
||||
}));
|
||||
|
||||
std::string const host = fmt::format("{}:{}", host_, endpoint.port());
|
||||
|
||||
@@ -20,8 +20,12 @@
|
||||
#pragma once
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include <boost/asio/bind_cancellation_slot.hpp>
|
||||
#include <boost/asio/cancellation_signal.hpp>
|
||||
#include <boost/asio/cancellation_type.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/beast/core/tcp_stream.hpp>
|
||||
@@ -31,22 +35,49 @@
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace util::requests {
|
||||
|
||||
/**
|
||||
* @brief Interface for WebSocket connections. It is used to hide SSL and plain connections behind the same interface.
|
||||
*
|
||||
* @note WsConnection must not be destroyed while there are pending asynchronous operations on it.
|
||||
*/
|
||||
class WsConnection {
|
||||
public:
|
||||
virtual ~WsConnection() = default;
|
||||
|
||||
/**
|
||||
* @brief Read a message from the WebSocket
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return Expected<std::string, RequestError> message or error
|
||||
*/
|
||||
virtual Expected<std::string, RequestError>
|
||||
read(boost::asio::yield_context yield) = 0;
|
||||
|
||||
/**
|
||||
* @brief Write a message to the WebSocket
|
||||
*
|
||||
* @param message message to write
|
||||
* @param yield yield context
|
||||
* @return std::optional<RequestError> error if any
|
||||
*/
|
||||
virtual std::optional<RequestError>
|
||||
write(std::string const& message, boost::asio::yield_context yield) = 0;
|
||||
|
||||
/**
|
||||
* @brief Close the WebSocket
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return std::optional<RequestError> error if any
|
||||
*/
|
||||
virtual std::optional<RequestError>
|
||||
close(boost::asio::yield_context yield) = 0;
|
||||
close(boost::asio::yield_context yield, std::chrono::steady_clock::duration timeout = DEFAULT_TIMEOUT) = 0;
|
||||
|
||||
static constexpr std::chrono::seconds DEFAULT_TIMEOUT{5};
|
||||
};
|
||||
using WsConnectionPtr = std::unique_ptr<WsConnection>;
|
||||
|
||||
@@ -54,12 +85,13 @@ using WsConnectionPtr = std::unique_ptr<WsConnection>;
|
||||
* @brief Builder for WebSocket connections
|
||||
*/
|
||||
class WsConnectionBuilder {
|
||||
util::Logger log_{"WsConnectionBuilder"};
|
||||
std::string host_;
|
||||
std::string port_;
|
||||
std::vector<HttpHeader> headers_;
|
||||
std::chrono::milliseconds timeout_{DEFAULT_TIMEOUT};
|
||||
std::chrono::steady_clock::duration connectionTimeout_{DEFAULT_TIMEOUT};
|
||||
std::chrono::steady_clock::duration wsHandshakeTimeout_{DEFAULT_TIMEOUT};
|
||||
std::string target_{"/"};
|
||||
bool sslEnabled_{false};
|
||||
|
||||
public:
|
||||
WsConnectionBuilder(std::string host, std::string port);
|
||||
@@ -92,27 +124,43 @@ public:
|
||||
setTarget(std::string target);
|
||||
|
||||
/**
|
||||
* @brief Set the timeout for connection establishing operations
|
||||
* @brief Set the timeout for connection establishing operations. Default is 5 seconds
|
||||
*
|
||||
* @param timeout timeout to set
|
||||
* @return RequestBuilder& this
|
||||
*/
|
||||
WsConnectionBuilder&
|
||||
setConnectionTimeout(std::chrono::milliseconds timeout);
|
||||
setConnectionTimeout(std::chrono::steady_clock::duration timeout);
|
||||
|
||||
/**
|
||||
* @brief Set whether SSL is enabled
|
||||
* @brief Set the timeout for WebSocket handshake. Default is 5 seconds
|
||||
*
|
||||
* @note Default is false
|
||||
*
|
||||
* @param enabled whether SSL is enabled
|
||||
* @param timeout timeout to set
|
||||
* @return RequestBuilder& this
|
||||
*/
|
||||
WsConnectionBuilder&
|
||||
setSslEnabled(bool enabled);
|
||||
setWsHandshakeTimeout(std::chrono::steady_clock::duration timeout);
|
||||
|
||||
/**
|
||||
* @brief Connect to the host asynchronously
|
||||
* @brief Connect to the host using SSL asynchronously
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return Expected<WsConnection, RequestError> WebSocket connection or error
|
||||
*/
|
||||
Expected<WsConnectionPtr, RequestError>
|
||||
sslConnect(boost::asio::yield_context yield) const;
|
||||
|
||||
/**
|
||||
* @brief Connect to the host without SSL asynchronously
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return Expected<WsConnection, RequestError> WebSocket connection or error
|
||||
*/
|
||||
Expected<WsConnectionPtr, RequestError>
|
||||
plainConnect(boost::asio::yield_context yield) const;
|
||||
|
||||
/**
|
||||
* @brief Connect to the host trying SSL first then plain if SSL fails
|
||||
*
|
||||
* @param yield yield context
|
||||
* @return Expected<WsConnection, RequestError> WebSocket connection or error
|
||||
@@ -120,7 +168,7 @@ public:
|
||||
Expected<WsConnectionPtr, RequestError>
|
||||
connect(boost::asio::yield_context yield) const;
|
||||
|
||||
static constexpr std::chrono::milliseconds DEFAULT_TIMEOUT{5000};
|
||||
static constexpr std::chrono::seconds DEFAULT_TIMEOUT{5};
|
||||
|
||||
private:
|
||||
template <class StreamDataType>
|
||||
|
||||
@@ -20,16 +20,24 @@
|
||||
#include "util/requests/impl/SslContext.hpp"
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include <boost/asio/buffer.hpp>
|
||||
#include <boost/asio/ssl/context.hpp>
|
||||
#include <boost/asio/ssl/error.hpp>
|
||||
#include <boost/asio/ssl/verify_mode.hpp>
|
||||
#include <boost/beast/core/error.hpp>
|
||||
#include <boost/lexical_cast.hpp>
|
||||
#include <fmt/core.h>
|
||||
#include <openssl/err.h>
|
||||
|
||||
#include <array>
|
||||
#include <cstddef>
|
||||
#include <filesystem>
|
||||
#include <fstream>
|
||||
#include <ios>
|
||||
#include <optional>
|
||||
#include <sstream>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
@@ -87,4 +95,24 @@ makeSslContext()
|
||||
return context;
|
||||
}
|
||||
|
||||
std::optional<std::string>
|
||||
sslErrorToString(boost::beast::error_code const& error)
|
||||
{
|
||||
if (error.category() != boost::asio::error::get_ssl_category())
|
||||
return std::nullopt;
|
||||
|
||||
std::string errorString = fmt::format(
|
||||
"({},{}) ",
|
||||
boost::lexical_cast<std::string>(ERR_GET_LIB(error.value())),
|
||||
boost::lexical_cast<std::string>(ERR_GET_REASON(error.value()))
|
||||
);
|
||||
|
||||
static constexpr size_t BUFFER_SIZE = 128;
|
||||
char buf[BUFFER_SIZE];
|
||||
::ERR_error_string_n(error.value(), buf, sizeof(buf));
|
||||
errorString += buf;
|
||||
|
||||
return errorString;
|
||||
}
|
||||
|
||||
} // namespace util::requests::impl
|
||||
|
||||
@@ -23,10 +23,17 @@
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include <boost/asio/ssl/context.hpp>
|
||||
#include <boost/beast/core/error.hpp>
|
||||
|
||||
#include <optional>
|
||||
#include <string>
|
||||
|
||||
namespace util::requests::impl {
|
||||
|
||||
Expected<boost::asio::ssl::context, RequestError>
|
||||
makeSslContext();
|
||||
|
||||
std::optional<std::string>
|
||||
sslErrorToString(boost::beast::error_code const& error);
|
||||
|
||||
} // namespace util::requests::impl
|
||||
|
||||
@@ -32,7 +32,9 @@
|
||||
#include <boost/beast/ssl/ssl_stream.hpp>
|
||||
#include <boost/beast/websocket/rfc6455.hpp>
|
||||
#include <boost/beast/websocket/stream.hpp>
|
||||
#include <boost/beast/websocket/stream_base.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
@@ -75,8 +77,15 @@ public:
|
||||
}
|
||||
|
||||
std::optional<RequestError>
|
||||
close(boost::asio::yield_context yield) override
|
||||
close(boost::asio::yield_context yield, std::chrono::steady_clock::duration const timeout = DEFAULT_TIMEOUT)
|
||||
override
|
||||
{
|
||||
// Set the timeout for closing the connection
|
||||
boost::beast::websocket::stream_base::timeout wsTimeout{};
|
||||
ws_.get_option(wsTimeout);
|
||||
wsTimeout.handshake_timeout = timeout;
|
||||
ws_.set_option(wsTimeout);
|
||||
|
||||
boost::beast::error_code errorCode;
|
||||
ws_.async_close(boost::beast::websocket::close_code::normal, yield[errorCode]);
|
||||
if (errorCode)
|
||||
|
||||
@@ -23,20 +23,19 @@
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <cassandra.h>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <optional>
|
||||
|
||||
using namespace data::cassandra;
|
||||
using namespace data::cassandra::impl;
|
||||
using namespace testing;
|
||||
|
||||
class BackendCassandraRetryPolicyTest : public SyncAsioContextTest {};
|
||||
struct BackendCassandraRetryPolicyTest : SyncAsioContextTest {
|
||||
ExponentialBackoffRetryPolicy retryPolicy{ctx};
|
||||
};
|
||||
|
||||
TEST_F(BackendCassandraRetryPolicyTest, ShouldRetryAlwaysTrue)
|
||||
{
|
||||
auto retryPolicy = ExponentialBackoffRetryPolicy{ctx};
|
||||
EXPECT_TRUE(retryPolicy.shouldRetry(CassandraError{"timeout", CASS_ERROR_LIB_REQUEST_TIMED_OUT}));
|
||||
EXPECT_TRUE(retryPolicy.shouldRetry(CassandraError{"invalid data", CASS_ERROR_LIB_INVALID_DATA}));
|
||||
EXPECT_TRUE(retryPolicy.shouldRetry(CassandraError{"invalid query", CASS_ERROR_SERVER_INVALID_QUERY}));
|
||||
@@ -48,37 +47,30 @@ TEST_F(BackendCassandraRetryPolicyTest, ShouldRetryAlwaysTrue)
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(BackendCassandraRetryPolicyTest, CheckComputedBackoffDelayIsCorrect)
|
||||
{
|
||||
auto retryPolicy = ExponentialBackoffRetryPolicy{ctx};
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(0).count(), 1);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(1).count(), 2);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(2).count(), 4);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(3).count(), 8);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(4).count(), 16);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(5).count(), 32);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(6).count(), 64);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(7).count(), 128);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(8).count(), 256);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(9).count(), 512);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(10).count(), 1024);
|
||||
EXPECT_EQ(retryPolicy.calculateDelay(11).count(),
|
||||
1024); // 10 is max, same after that
|
||||
}
|
||||
|
||||
TEST_F(BackendCassandraRetryPolicyTest, RetryCorrectlyExecuted)
|
||||
{
|
||||
auto callCount = std::atomic_int{0};
|
||||
auto work = std::optional<boost::asio::io_context::work>{ctx};
|
||||
auto retryPolicy = ExponentialBackoffRetryPolicy{ctx};
|
||||
StrictMock<MockFunction<void()>> callback;
|
||||
EXPECT_CALL(callback, Call()).Times(3);
|
||||
|
||||
retryPolicy.retry([&callCount]() { ++callCount; });
|
||||
retryPolicy.retry([&callCount]() { ++callCount; });
|
||||
retryPolicy.retry([&callCount, &work]() {
|
||||
++callCount;
|
||||
work.reset();
|
||||
});
|
||||
|
||||
ctx.run();
|
||||
ASSERT_EQ(callCount, 3);
|
||||
for (auto i = 0; i < 3; ++i) {
|
||||
retryPolicy.retry([&callback]() { callback.Call(); });
|
||||
runContext();
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(BackendCassandraRetryPolicyTest, MutlipleRetryCancelPreviousCalls)
|
||||
{
|
||||
StrictMock<MockFunction<void()>> callback;
|
||||
EXPECT_CALL(callback, Call());
|
||||
|
||||
for (auto i = 0; i < 3; ++i)
|
||||
retryPolicy.retry([&callback]() { callback.Call(); });
|
||||
|
||||
runContext();
|
||||
}
|
||||
|
||||
TEST_F(BackendCassandraRetryPolicyTest, CallbackIsNotCalledIfContextDies)
|
||||
{
|
||||
StrictMock<MockFunction<void()>> callback;
|
||||
retryPolicy.retry([&callback]() { callback.Call(); });
|
||||
}
|
||||
|
||||
@@ -52,7 +52,6 @@ public:
|
||||
void
|
||||
SetUp() override
|
||||
{
|
||||
NoLoggerFixture::SetUp();
|
||||
state_.isStopping = false;
|
||||
state_.writeConflict = false;
|
||||
state_.isReadOnly = false;
|
||||
@@ -63,7 +62,6 @@ public:
|
||||
TearDown() override
|
||||
{
|
||||
extractor_.reset();
|
||||
NoLoggerFixture::TearDown();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
120
unittests/etl/ForwardingSourceTests.cpp
Normal file
120
unittests/etl/ForwardingSourceTests.cpp
Normal file
@@ -0,0 +1,120 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/ForwardingSource.hpp"
|
||||
#include "util/Fixtures.hpp"
|
||||
#include "util/TestWsServer.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/parse.hpp>
|
||||
#include <boost/json/serialize.hpp>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
|
||||
using namespace etl::impl;
|
||||
|
||||
struct ForwardingSourceTests : SyncAsioContextTest {
|
||||
TestWsServer server_{ctx, "0.0.0.0", 11114};
|
||||
ForwardingSource forwardingSource{"127.0.0.1", "11114", std::chrono::milliseconds{1}};
|
||||
};
|
||||
|
||||
TEST_F(ForwardingSourceTests, ConnectionFailed)
|
||||
{
|
||||
runSpawn([&](boost::asio::yield_context yield) {
|
||||
auto result = forwardingSource.forwardToRippled({}, {}, yield);
|
||||
EXPECT_FALSE(result);
|
||||
});
|
||||
}
|
||||
|
||||
struct ForwardingSourceOperationsTests : ForwardingSourceTests {
|
||||
std::string const message_ = R"({"data":"some_data"})";
|
||||
boost::json::object const reply_ = {{"reply", "some_reply"}};
|
||||
|
||||
TestWsConnection
|
||||
serverConnection(boost::asio::yield_context yield)
|
||||
{
|
||||
// First connection attempt is SSL handshake so it will fail
|
||||
auto failedConnection = server_.acceptConnection(yield);
|
||||
[&]() { ASSERT_FALSE(failedConnection); }();
|
||||
|
||||
auto connection = server_.acceptConnection(yield);
|
||||
[&]() { ASSERT_TRUE(connection); }();
|
||||
return std::move(connection).value();
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(ForwardingSourceOperationsTests, ReadFailed)
|
||||
{
|
||||
boost::asio::spawn(ctx, [&](boost::asio::yield_context yield) {
|
||||
auto connection = serverConnection(yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
runSpawn([&](boost::asio::yield_context yield) {
|
||||
auto result = forwardingSource.forwardToRippled(boost::json::parse(message_).as_object(), {}, yield);
|
||||
EXPECT_FALSE(result);
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(ForwardingSourceOperationsTests, ParseFailed)
|
||||
{
|
||||
boost::asio::spawn(ctx, [&](boost::asio::yield_context yield) {
|
||||
auto connection = serverConnection(yield);
|
||||
|
||||
auto receivedMessage = connection.receive(yield);
|
||||
[&]() { ASSERT_TRUE(receivedMessage); }();
|
||||
EXPECT_EQ(*receivedMessage, message_);
|
||||
|
||||
auto sendError = connection.send("invalid_json", yield);
|
||||
[&]() { ASSERT_FALSE(sendError) << *sendError; }();
|
||||
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
runSpawn([&](boost::asio::yield_context yield) {
|
||||
auto result = forwardingSource.forwardToRippled(boost::json::parse(message_).as_object(), {}, yield);
|
||||
EXPECT_FALSE(result);
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(ForwardingSourceOperationsTests, Success)
|
||||
{
|
||||
boost::asio::spawn(ctx, [&](boost::asio::yield_context yield) {
|
||||
auto connection = serverConnection(yield);
|
||||
|
||||
auto receivedMessage = connection.receive(yield);
|
||||
[&]() { ASSERT_TRUE(receivedMessage); }();
|
||||
EXPECT_EQ(*receivedMessage, message_);
|
||||
|
||||
auto sendError = connection.send(boost::json::serialize(reply_), yield);
|
||||
[&]() { ASSERT_FALSE(sendError) << *sendError; }();
|
||||
});
|
||||
|
||||
runSpawn([&](boost::asio::yield_context yield) {
|
||||
auto result = forwardingSource.forwardToRippled(boost::json::parse(message_).as_object(), "some_ip", yield);
|
||||
[&]() { ASSERT_TRUE(result); }();
|
||||
auto expectedReply = reply_;
|
||||
expectedReply["forwarded"] = true;
|
||||
EXPECT_EQ(*result, expectedReply) << *result;
|
||||
});
|
||||
}
|
||||
150
unittests/etl/GrpcSourceTests.cpp
Normal file
150
unittests/etl/GrpcSourceTests.cpp
Normal file
@@ -0,0 +1,150 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/GrpcSource.hpp"
|
||||
#include "util/Fixtures.hpp"
|
||||
#include "util/MockBackend.hpp"
|
||||
#include "util/MockXrpLedgerAPIService.hpp"
|
||||
#include "util/TestObject.hpp"
|
||||
#include "util/config/Config.hpp"
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
#include <grpcpp/server_context.h>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <gtest/gtest.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger_data.pb.h>
|
||||
#include <ripple/basics/base_uint.h>
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
using namespace etl::impl;
|
||||
|
||||
struct GrpcSourceTests : NoLoggerFixture, unittests::util::WithMockXrpLedgerAPIService {
|
||||
GrpcSourceTests()
|
||||
: WithMockXrpLedgerAPIService("localhost:50051")
|
||||
, mockBackend_(std::make_shared<testing::StrictMock<MockBackend>>(util::Config{}))
|
||||
, grpcSource_("127.0.0.1", "50051", mockBackend_)
|
||||
{
|
||||
}
|
||||
|
||||
std::shared_ptr<testing::StrictMock<MockBackend>> mockBackend_;
|
||||
testing::StrictMock<GrpcSource> grpcSource_;
|
||||
};
|
||||
|
||||
TEST_F(GrpcSourceTests, fetchLedger)
|
||||
{
|
||||
uint32_t const sequence = 123;
|
||||
bool const getObjects = true;
|
||||
bool const getObjectNeighbors = false;
|
||||
|
||||
EXPECT_CALL(mockXrpLedgerAPIService, GetLedger)
|
||||
.WillOnce([&](grpc::ServerContext* /*context*/,
|
||||
org::xrpl::rpc::v1::GetLedgerRequest const* request,
|
||||
org::xrpl::rpc::v1::GetLedgerResponse* response) {
|
||||
EXPECT_EQ(request->ledger().sequence(), sequence);
|
||||
EXPECT_TRUE(request->transactions());
|
||||
EXPECT_TRUE(request->expand());
|
||||
EXPECT_EQ(request->get_objects(), getObjects);
|
||||
EXPECT_EQ(request->get_object_neighbors(), getObjectNeighbors);
|
||||
EXPECT_EQ(request->user(), "ETL");
|
||||
response->set_validated(true);
|
||||
response->set_is_unlimited(false);
|
||||
response->set_object_neighbors_included(false);
|
||||
return grpc::Status{};
|
||||
});
|
||||
auto const [status, response] = grpcSource_.fetchLedger(sequence, getObjects, getObjectNeighbors);
|
||||
ASSERT_TRUE(status.ok());
|
||||
EXPECT_TRUE(response.validated());
|
||||
EXPECT_FALSE(response.is_unlimited());
|
||||
EXPECT_FALSE(response.object_neighbors_included());
|
||||
}
|
||||
|
||||
TEST_F(GrpcSourceTests, fetchLedgerNoStub)
|
||||
{
|
||||
testing::StrictMock<GrpcSource> wrongGrpcSource{"wrong", "wrong", mockBackend_};
|
||||
auto const [status, _response] = wrongGrpcSource.fetchLedger(0, false, false);
|
||||
EXPECT_EQ(status.error_code(), grpc::StatusCode::INTERNAL);
|
||||
}
|
||||
|
||||
TEST_F(GrpcSourceTests, loadInitialLedgerNoStub)
|
||||
{
|
||||
testing::StrictMock<GrpcSource> wrongGrpcSource{"wrong", "wrong", mockBackend_};
|
||||
auto const [data, success] = wrongGrpcSource.loadInitialLedger(0, 0, false);
|
||||
EXPECT_TRUE(data.empty());
|
||||
EXPECT_FALSE(success);
|
||||
}
|
||||
|
||||
struct GrpcSourceLoadInitialLedgerTests : GrpcSourceTests {
|
||||
uint32_t const sequence_ = 123;
|
||||
uint32_t const numMarkers_ = 4;
|
||||
bool const cacheOnly_ = false;
|
||||
};
|
||||
|
||||
TEST_F(GrpcSourceLoadInitialLedgerTests, GetLedgerDataFailed)
|
||||
{
|
||||
EXPECT_CALL(mockXrpLedgerAPIService, GetLedgerData)
|
||||
.Times(numMarkers_)
|
||||
.WillRepeatedly([&](grpc::ServerContext* /*context*/,
|
||||
org::xrpl::rpc::v1::GetLedgerDataRequest const* request,
|
||||
org::xrpl::rpc::v1::GetLedgerDataResponse* /*response*/) {
|
||||
EXPECT_EQ(request->ledger().sequence(), sequence_);
|
||||
EXPECT_EQ(request->user(), "ETL");
|
||||
return grpc::Status{grpc::StatusCode::NOT_FOUND, "Not found"};
|
||||
});
|
||||
|
||||
auto const [data, success] = grpcSource_.loadInitialLedger(sequence_, numMarkers_, cacheOnly_);
|
||||
EXPECT_TRUE(data.empty());
|
||||
EXPECT_FALSE(success);
|
||||
}
|
||||
|
||||
TEST_F(GrpcSourceLoadInitialLedgerTests, worksFine)
|
||||
{
|
||||
auto const key = ripple::uint256{4};
|
||||
std::string const keyStr{reinterpret_cast<char const*>(key.data()), ripple::uint256::size()};
|
||||
auto const object = CreateTicketLedgerObject("rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn", sequence_);
|
||||
auto const objectData = object.getSerializer().peekData();
|
||||
|
||||
EXPECT_CALL(mockXrpLedgerAPIService, GetLedgerData)
|
||||
.Times(numMarkers_)
|
||||
.WillRepeatedly([&](grpc::ServerContext* /*context*/,
|
||||
org::xrpl::rpc::v1::GetLedgerDataRequest const* request,
|
||||
org::xrpl::rpc::v1::GetLedgerDataResponse* response) {
|
||||
EXPECT_EQ(request->ledger().sequence(), sequence_);
|
||||
EXPECT_EQ(request->user(), "ETL");
|
||||
|
||||
response->set_is_unlimited(true);
|
||||
auto newObject = response->mutable_ledger_objects()->add_objects();
|
||||
newObject->set_key(key.data(), ripple::uint256::size());
|
||||
newObject->set_data(objectData.data(), objectData.size());
|
||||
|
||||
return grpc::Status{};
|
||||
});
|
||||
|
||||
EXPECT_CALL(*mockBackend_, writeNFTs).Times(numMarkers_);
|
||||
EXPECT_CALL(*mockBackend_, writeLedgerObject).Times(numMarkers_);
|
||||
|
||||
auto const [data, success] = grpcSource_.loadInitialLedger(sequence_, numMarkers_, cacheOnly_);
|
||||
|
||||
EXPECT_TRUE(success);
|
||||
EXPECT_EQ(data, std::vector<std::string>(4, keyStr));
|
||||
}
|
||||
186
unittests/etl/SourceTests.cpp
Normal file
186
unittests/etl/SourceTests.cpp
Normal file
@@ -0,0 +1,186 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/Source.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/value_to.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <gtest/gtest.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
using namespace etl;
|
||||
|
||||
using testing::Return;
|
||||
using testing::StrictMock;
|
||||
|
||||
struct GrpcSourceMock {
|
||||
using FetchLedgerReturnType = std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>;
|
||||
MOCK_METHOD(FetchLedgerReturnType, fetchLedger, (uint32_t, bool, bool));
|
||||
|
||||
using LoadLedgerReturnType = std::pair<std::vector<std::string>, bool>;
|
||||
MOCK_METHOD(LoadLedgerReturnType, loadInitialLedger, (uint32_t, uint32_t, bool));
|
||||
};
|
||||
|
||||
struct SubscriptionSourceMock {
|
||||
MOCK_METHOD(void, run, ());
|
||||
MOCK_METHOD(bool, hasLedger, (uint32_t), (const));
|
||||
MOCK_METHOD(bool, isConnected, (), (const));
|
||||
MOCK_METHOD(void, setForwarding, (bool));
|
||||
MOCK_METHOD(std::chrono::steady_clock::time_point, lastMessageTime, (), (const));
|
||||
MOCK_METHOD(std::string, validatedRange, (), (const));
|
||||
MOCK_METHOD(void, stop, ());
|
||||
};
|
||||
|
||||
struct ForwardingSourceMock {
|
||||
MOCK_METHOD(void, constructor, (std::string const&, std::string const&, std::chrono::steady_clock::duration));
|
||||
|
||||
using ForwardToRippledReturnType = std::optional<boost::json::object>;
|
||||
using ClientIpOpt = std::optional<std::string>;
|
||||
MOCK_METHOD(
|
||||
ForwardToRippledReturnType,
|
||||
forwardToRippled,
|
||||
(boost::json::object const&, ClientIpOpt const&, boost::asio::yield_context)
|
||||
);
|
||||
};
|
||||
|
||||
struct SourceTest : public ::testing::Test {
|
||||
boost::asio::io_context ioc_;
|
||||
|
||||
StrictMock<GrpcSourceMock> grpcSourceMock_;
|
||||
std::shared_ptr<StrictMock<SubscriptionSourceMock>> subscriptionSourceMock_ =
|
||||
std::make_shared<StrictMock<SubscriptionSourceMock>>();
|
||||
StrictMock<ForwardingSourceMock> forwardingSourceMock_;
|
||||
|
||||
SourceImpl<
|
||||
StrictMock<GrpcSourceMock>&,
|
||||
std::shared_ptr<StrictMock<SubscriptionSourceMock>>,
|
||||
StrictMock<ForwardingSourceMock>&>
|
||||
source_{
|
||||
"some_ip",
|
||||
"some_ws_port",
|
||||
"some_grpc_port",
|
||||
grpcSourceMock_,
|
||||
subscriptionSourceMock_,
|
||||
forwardingSourceMock_
|
||||
};
|
||||
};
|
||||
|
||||
TEST_F(SourceTest, run)
|
||||
{
|
||||
EXPECT_CALL(*subscriptionSourceMock_, run());
|
||||
source_.run();
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, isConnected)
|
||||
{
|
||||
EXPECT_CALL(*subscriptionSourceMock_, isConnected()).WillOnce(testing::Return(true));
|
||||
EXPECT_TRUE(source_.isConnected());
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, setForwarding)
|
||||
{
|
||||
EXPECT_CALL(*subscriptionSourceMock_, setForwarding(true));
|
||||
source_.setForwarding(true);
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, toJson)
|
||||
{
|
||||
EXPECT_CALL(*subscriptionSourceMock_, validatedRange()).WillOnce(Return(std::string("some_validated_range")));
|
||||
EXPECT_CALL(*subscriptionSourceMock_, isConnected()).WillOnce(Return(true));
|
||||
auto const lastMessageTime = std::chrono::steady_clock::now();
|
||||
EXPECT_CALL(*subscriptionSourceMock_, lastMessageTime()).WillOnce(Return(lastMessageTime));
|
||||
|
||||
auto const json = source_.toJson();
|
||||
|
||||
EXPECT_EQ(boost::json::value_to<std::string>(json.at("validated_range")), "some_validated_range");
|
||||
EXPECT_EQ(boost::json::value_to<std::string>(json.at("is_connected")), "1");
|
||||
EXPECT_EQ(boost::json::value_to<std::string>(json.at("ip")), "some_ip");
|
||||
EXPECT_EQ(boost::json::value_to<std::string>(json.at("ws_port")), "some_ws_port");
|
||||
EXPECT_EQ(boost::json::value_to<std::string>(json.at("grpc_port")), "some_grpc_port");
|
||||
auto lastMessageAgeStr = boost::json::value_to<std::string>(json.at("last_msg_age_seconds"));
|
||||
EXPECT_GE(std::stoi(lastMessageAgeStr), 0);
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, toString)
|
||||
{
|
||||
EXPECT_CALL(*subscriptionSourceMock_, validatedRange()).WillOnce(Return(std::string("some_validated_range")));
|
||||
|
||||
auto const str = source_.toString();
|
||||
EXPECT_EQ(
|
||||
str,
|
||||
"{validated range: some_validated_range, ip: some_ip, web socket port: some_ws_port, grpc port: some_grpc_port}"
|
||||
);
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, hasLedger)
|
||||
{
|
||||
uint32_t const ledgerSeq = 123;
|
||||
EXPECT_CALL(*subscriptionSourceMock_, hasLedger(ledgerSeq)).WillOnce(Return(true));
|
||||
EXPECT_TRUE(source_.hasLedger(ledgerSeq));
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, fetchLedger)
|
||||
{
|
||||
uint32_t const ledgerSeq = 123;
|
||||
|
||||
EXPECT_CALL(grpcSourceMock_, fetchLedger(ledgerSeq, true, false));
|
||||
auto const [actualStatus, actualResponse] = source_.fetchLedger(ledgerSeq);
|
||||
|
||||
EXPECT_EQ(actualStatus.error_code(), grpc::StatusCode::OK);
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, loadInitialLedger)
|
||||
{
|
||||
uint32_t const ledgerSeq = 123;
|
||||
uint32_t const numMarkers = 3;
|
||||
|
||||
EXPECT_CALL(grpcSourceMock_, loadInitialLedger(ledgerSeq, numMarkers, false))
|
||||
.WillOnce(Return(std::make_pair(std::vector<std::string>{}, true)));
|
||||
auto const [actualLedgers, actualSuccess] = source_.loadInitialLedger(ledgerSeq, numMarkers);
|
||||
|
||||
EXPECT_TRUE(actualLedgers.empty());
|
||||
EXPECT_TRUE(actualSuccess);
|
||||
}
|
||||
|
||||
TEST_F(SourceTest, forwardToRippled)
|
||||
{
|
||||
boost::json::object const request = {{"some_key", "some_value"}};
|
||||
std::optional<std::string> const clientIp = "some_client_ip";
|
||||
|
||||
EXPECT_CALL(forwardingSourceMock_, forwardToRippled(request, clientIp, testing::_)).WillOnce(Return(request));
|
||||
|
||||
boost::asio::io_context ioContext;
|
||||
boost::asio::spawn(ioContext, [&](boost::asio::yield_context yield) {
|
||||
auto const response = source_.forwardToRippled(request, clientIp, yield);
|
||||
EXPECT_EQ(response, request);
|
||||
});
|
||||
ioContext.run();
|
||||
}
|
||||
70
unittests/etl/SubscriptionSourceDependenciesTests.cpp
Normal file
70
unittests/etl/SubscriptionSourceDependenciesTests.cpp
Normal file
@@ -0,0 +1,70 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/SubscriptionSourceDependencies.hpp"
|
||||
#include "util/MockNetworkValidatedLedgers.hpp"
|
||||
#include "util/MockSubscriptionManager.hpp"
|
||||
|
||||
#include <boost/json/object.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
using namespace etl::impl;
|
||||
using testing::StrictMock;
|
||||
|
||||
struct SubscriptionSourceDependenciesTest : testing::Test {
|
||||
std::shared_ptr<StrictMock<MockNetworkValidatedLedgers>> networkValidatedLedgers_ =
|
||||
std::make_shared<StrictMock<MockNetworkValidatedLedgers>>();
|
||||
|
||||
std::shared_ptr<StrictMock<MockSubscriptionManager>> subscriptionManager_ =
|
||||
std::make_shared<StrictMock<MockSubscriptionManager>>();
|
||||
|
||||
SubscriptionSourceDependencies dependencies_{networkValidatedLedgers_, subscriptionManager_};
|
||||
};
|
||||
|
||||
TEST_F(SubscriptionSourceDependenciesTest, ForwardProposedTransaction)
|
||||
{
|
||||
boost::json::object const txJson = {{"tx", "json"}};
|
||||
EXPECT_CALL(*subscriptionManager_, forwardProposedTransaction(txJson));
|
||||
dependencies_.forwardProposedTransaction(txJson);
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceDependenciesTest, ForwardValidation)
|
||||
{
|
||||
boost::json::object const validationJson = {{"validation", "json"}};
|
||||
EXPECT_CALL(*subscriptionManager_, forwardValidation(validationJson));
|
||||
dependencies_.forwardValidation(validationJson);
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceDependenciesTest, ForwardManifest)
|
||||
{
|
||||
boost::json::object const manifestJson = {{"manifest", "json"}};
|
||||
EXPECT_CALL(*subscriptionManager_, forwardManifest(manifestJson));
|
||||
dependencies_.forwardManifest(manifestJson);
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceDependenciesTest, PushValidatedLedger)
|
||||
{
|
||||
uint32_t const idx = 42;
|
||||
EXPECT_CALL(*networkValidatedLedgers_, push(idx));
|
||||
dependencies_.pushValidatedLedger(idx);
|
||||
}
|
||||
493
unittests/etl/SubscriptionSourceTests.cpp
Normal file
493
unittests/etl/SubscriptionSourceTests.cpp
Normal file
@@ -0,0 +1,493 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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/impl/SubscriptionSource.hpp"
|
||||
#include "util/Fixtures.hpp"
|
||||
#include "util/MockNetworkValidatedLedgers.hpp"
|
||||
#include "util/MockSubscriptionManager.hpp"
|
||||
#include "util/TestWsServer.hpp"
|
||||
|
||||
#include <boost/asio/executor_work_guard.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/serialize.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <utility>
|
||||
|
||||
using namespace etl::impl;
|
||||
using testing::MockFunction;
|
||||
using testing::StrictMock;
|
||||
|
||||
struct SubscriptionSourceConnectionTests : public NoLoggerFixture {
|
||||
SubscriptionSourceConnectionTests()
|
||||
{
|
||||
subscriptionSource_->run();
|
||||
}
|
||||
|
||||
boost::asio::io_context ioContext_;
|
||||
|
||||
TestWsServer wsServer_{ioContext_, "0.0.0.0", 11113};
|
||||
|
||||
template <typename T>
|
||||
using StrictMockPtr = std::shared_ptr<StrictMock<T>>;
|
||||
|
||||
StrictMockPtr<MockNetworkValidatedLedgers> networkValidatedLedgers_ =
|
||||
std::make_shared<StrictMock<MockNetworkValidatedLedgers>>();
|
||||
StrictMockPtr<MockSubscriptionManager> subscriptionManager_ =
|
||||
std::make_shared<StrictMock<MockSubscriptionManager>>();
|
||||
|
||||
StrictMock<MockFunction<void()>> onConnectHook_;
|
||||
StrictMock<MockFunction<void()>> onDisconnectHook_;
|
||||
|
||||
std::unique_ptr<SubscriptionSource> subscriptionSource_ = std::make_unique<SubscriptionSource>(
|
||||
ioContext_,
|
||||
"127.0.0.1",
|
||||
"11113",
|
||||
networkValidatedLedgers_,
|
||||
subscriptionManager_,
|
||||
onConnectHook_.AsStdFunction(),
|
||||
onDisconnectHook_.AsStdFunction(),
|
||||
std::chrono::milliseconds(1),
|
||||
std::chrono::milliseconds(1)
|
||||
);
|
||||
|
||||
[[maybe_unused]] TestWsConnection
|
||||
serverConnection(boost::asio::yield_context yield)
|
||||
{
|
||||
// The first one is an SSL attempt
|
||||
auto failedConnection = wsServer_.acceptConnection(yield);
|
||||
[&]() { ASSERT_FALSE(failedConnection); }();
|
||||
|
||||
auto connection = wsServer_.acceptConnection(yield);
|
||||
[&]() { ASSERT_TRUE(connection) << connection.error().message(); }();
|
||||
|
||||
auto message = connection->receive(yield);
|
||||
[&]() {
|
||||
ASSERT_TRUE(message);
|
||||
EXPECT_EQ(
|
||||
message.value(),
|
||||
R"({"command":"subscribe","streams":["ledger","manifests","validations","transactions_proposed"]})"
|
||||
);
|
||||
}();
|
||||
return std::move(connection).value();
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(SubscriptionSourceConnectionTests, ConnectionFailed)
|
||||
{
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceConnectionTests, ConnectionFailed_Retry_ConnectionFailed)
|
||||
{
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceConnectionTests, ReadError)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = serverConnection(yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceConnectionTests, ReadError_Reconnect)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
for (int i = 0; i < 2; ++i) {
|
||||
auto connection = serverConnection(yield);
|
||||
connection.close(yield);
|
||||
}
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call()).Times(2);
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceConnectionTests, IsConnected)
|
||||
{
|
||||
EXPECT_FALSE(subscriptionSource_->isConnected());
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = serverConnection(yield);
|
||||
EXPECT_TRUE(subscriptionSource_->isConnected());
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
struct SubscriptionSourceReadTests : public SubscriptionSourceConnectionTests {
|
||||
[[maybe_unused]] TestWsConnection
|
||||
connectAndSendMessage(std::string const message, boost::asio::yield_context yield)
|
||||
{
|
||||
auto connection = serverConnection(yield);
|
||||
auto error = connection.send(message, yield);
|
||||
[&]() { ASSERT_FALSE(error) << *error; }();
|
||||
return connection;
|
||||
}
|
||||
};
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotWrongMessage_Reconnect)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage("something", yield);
|
||||
// We have to schedule receiving to receive close frame and boost will handle it automatically
|
||||
connection.receive(yield);
|
||||
serverConnection(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call()).Times(2);
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotResult)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"result":{})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotResultWithLedgerIndex)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"result":{"ledger_index":123}})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
EXPECT_CALL(*networkValidatedLedgers_, push(123));
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotResultWithLedgerIndexAsString_Reconnect)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"result":{"ledger_index":"123"}})", yield);
|
||||
// We have to schedule receiving to receive close frame and boost will handle it automatically
|
||||
connection.receive(yield);
|
||||
serverConnection(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call()).Times(2);
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotResultWithValidatedLedgersAsNumber_Reconnect)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"result":{"validated_ledgers":123}})", yield);
|
||||
// We have to schedule receiving to receive close frame and boost will handle it automatically
|
||||
connection.receive(yield);
|
||||
serverConnection(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call()).Times(2);
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotResultWithValidatedLedgers)
|
||||
{
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(123));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(124));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(455));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(456));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(457));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(32));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(31));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(789));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(790));
|
||||
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"result":{"validated_ledgers":"123-456,789,32"}})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(123));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(124));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(455));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(456));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(457));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(32));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(31));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(789));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(790));
|
||||
|
||||
EXPECT_EQ(subscriptionSource_->validatedRange(), "123-456,789,32");
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotResultWithValidatedLedgersWrongValue_Reconnect)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"result":{"validated_ledgers":"123-456-789,32"}})", yield);
|
||||
// We have to schedule receiving to receive close frame and boost will handle it automatically
|
||||
connection.receive(yield);
|
||||
serverConnection(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call()).Times(2);
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotResultWithLedgerIndexAndValidatedLedgers)
|
||||
{
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(1));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(1));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(2));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(3));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(4));
|
||||
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"result":{"ledger_index":123,"validated_ledgers":"1-3"}})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
EXPECT_CALL(*networkValidatedLedgers_, push(123));
|
||||
ioContext_.run();
|
||||
|
||||
EXPECT_EQ(subscriptionSource_->validatedRange(), "1-3");
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(0));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(1));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(2));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(3));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(4));
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotLedgerClosedWithLedgerIndex)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"type":"ledgerClosed","ledger_index":123})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
EXPECT_CALL(*networkValidatedLedgers_, push(123));
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotLedgerClosedWithLedgerIndexAsString_Reconnect)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"type":"ledgerClosed","ledger_index":"123"}})", yield);
|
||||
// We have to schedule receiving to receive close frame and boost will handle it automatically
|
||||
connection.receive(yield);
|
||||
serverConnection(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call()).Times(2);
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GorLedgerClosedWithValidatedLedgersAsNumber_Reconnect)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"type":"ledgerClosed","validated_ledgers":123})", yield);
|
||||
// We have to schedule receiving to receive close frame and boost will handle it automatically
|
||||
connection.receive(yield);
|
||||
serverConnection(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call()).Times(2);
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([]() {}).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotLedgerClosedWithValidatedLedgers)
|
||||
{
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(0));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(1));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(2));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(3));
|
||||
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"type":"ledgerClosed","validated_ledgers":"1-2"})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(0));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(1));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(2));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(3));
|
||||
EXPECT_EQ(subscriptionSource_->validatedRange(), "1-2");
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotLedgerClosedWithLedgerIndexAndValidatedLedgers)
|
||||
{
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(0));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(1));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(2));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(3));
|
||||
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection =
|
||||
connectAndSendMessage(R"({"type":"ledgerClosed","ledger_index":123,"validated_ledgers":"1-2"})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
EXPECT_CALL(*networkValidatedLedgers_, push(123));
|
||||
ioContext_.run();
|
||||
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(0));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(1));
|
||||
EXPECT_TRUE(subscriptionSource_->hasLedger(2));
|
||||
EXPECT_FALSE(subscriptionSource_->hasLedger(3));
|
||||
EXPECT_EQ(subscriptionSource_->validatedRange(), "1-2");
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotTransactionIsForwardingFalse)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"transaction":"some_transaction_data"})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotTransactionIsForwardingTrue)
|
||||
{
|
||||
subscriptionSource_->setForwarding(true);
|
||||
boost::json::object const message = {{"transaction", "some_transaction_data"}};
|
||||
|
||||
boost::asio::spawn(ioContext_, [&message, this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(boost::json::serialize(message), yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
EXPECT_CALL(*subscriptionManager_, forwardProposedTransaction(message));
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotValidationReceivedIsForwardingFalse)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"type":"validationReceived"})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotValidationReceivedIsForwardingTrue)
|
||||
{
|
||||
subscriptionSource_->setForwarding(true);
|
||||
boost::json::object const message = {{"type", "validationReceived"}};
|
||||
|
||||
boost::asio::spawn(ioContext_, [&message, this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(boost::json::serialize(message), yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
EXPECT_CALL(*subscriptionManager_, forwardValidation(message));
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotManiefstReceivedIsForwardingFalse)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(R"({"type":"manifestReceived"})", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, GotManifestReceivedIsForwardingTrue)
|
||||
{
|
||||
subscriptionSource_->setForwarding(true);
|
||||
boost::json::object const message = {{"type", "manifestReceived"}};
|
||||
|
||||
boost::asio::spawn(ioContext_, [&message, this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage(boost::json::serialize(message), yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
EXPECT_CALL(*subscriptionManager_, forwardManifest(message));
|
||||
ioContext_.run();
|
||||
}
|
||||
|
||||
TEST_F(SubscriptionSourceReadTests, LastMessageTime)
|
||||
{
|
||||
boost::asio::spawn(ioContext_, [this](boost::asio::yield_context yield) {
|
||||
auto connection = connectAndSendMessage("some_message", yield);
|
||||
connection.close(yield);
|
||||
});
|
||||
|
||||
EXPECT_CALL(onConnectHook_, Call());
|
||||
EXPECT_CALL(onDisconnectHook_, Call()).WillOnce([this]() { subscriptionSource_->stop(); });
|
||||
ioContext_.run();
|
||||
|
||||
auto const actualLastTimeMessage = subscriptionSource_->lastMessageTime();
|
||||
auto const now = std::chrono::steady_clock::now();
|
||||
auto const diff = std::chrono::duration_cast<std::chrono::milliseconds>(now - actualLastTimeMessage);
|
||||
EXPECT_LT(diff, std::chrono::milliseconds(100));
|
||||
}
|
||||
@@ -70,10 +70,9 @@ class LoggerFixture : virtual public ::testing::Test {
|
||||
FakeBuffer buffer_;
|
||||
std::ostream stream_ = std::ostream{&buffer_};
|
||||
|
||||
protected:
|
||||
public:
|
||||
// Simulates the `util::Logger::init(config)` call
|
||||
void
|
||||
SetUp() override
|
||||
LoggerFixture()
|
||||
{
|
||||
static std::once_flag once_;
|
||||
std::call_once(once_, [] {
|
||||
@@ -94,6 +93,7 @@ protected:
|
||||
core->set_logging_enabled(true);
|
||||
}
|
||||
|
||||
protected:
|
||||
void
|
||||
checkEqual(std::string expected)
|
||||
{
|
||||
@@ -113,12 +113,9 @@ protected:
|
||||
*
|
||||
* This is meant to be used as a base for other fixtures.
|
||||
*/
|
||||
class NoLoggerFixture : virtual public LoggerFixture {
|
||||
protected:
|
||||
void
|
||||
SetUp() override
|
||||
struct NoLoggerFixture : virtual LoggerFixture {
|
||||
NoLoggerFixture()
|
||||
{
|
||||
LoggerFixture::SetUp();
|
||||
boost::log::core::get()->set_logging_enabled(false);
|
||||
}
|
||||
};
|
||||
@@ -174,14 +171,20 @@ struct SyncAsioContextTest : virtual public NoLoggerFixture {
|
||||
{
|
||||
using namespace boost::asio;
|
||||
|
||||
auto called = false;
|
||||
testing::MockFunction<void()> call;
|
||||
spawn(ctx, [&, _ = make_work_guard(ctx)](yield_context yield) {
|
||||
f(yield);
|
||||
called = true;
|
||||
call.Call();
|
||||
});
|
||||
|
||||
EXPECT_CALL(call, Call());
|
||||
runContext();
|
||||
}
|
||||
|
||||
void
|
||||
runContext()
|
||||
{
|
||||
ctx.run();
|
||||
ASSERT_TRUE(called);
|
||||
ctx.reset();
|
||||
}
|
||||
|
||||
@@ -194,16 +197,9 @@ struct MockBackendTestBase : virtual public NoLoggerFixture {
|
||||
void
|
||||
SetUp() override
|
||||
{
|
||||
NoLoggerFixture::SetUp();
|
||||
backend.reset();
|
||||
}
|
||||
|
||||
void
|
||||
TearDown() override
|
||||
{
|
||||
NoLoggerFixture::TearDown();
|
||||
}
|
||||
|
||||
class BackendProxy {
|
||||
std::shared_ptr<BackendInterface> backend;
|
||||
|
||||
@@ -276,7 +272,6 @@ struct MockSubscriptionManagerTest : virtual public NoLoggerFixture {
|
||||
void
|
||||
SetUp() override
|
||||
{
|
||||
NoLoggerFixture::SetUp();
|
||||
mockSubscriptionManagerPtr = std::make_shared<MockSubscriptionManager>();
|
||||
}
|
||||
void
|
||||
@@ -296,7 +291,6 @@ struct MockLoadBalancerTest : virtual public NoLoggerFixture {
|
||||
void
|
||||
SetUp() override
|
||||
{
|
||||
NoLoggerFixture::SetUp();
|
||||
mockLoadBalancerPtr = std::make_shared<MockLoadBalancer>();
|
||||
}
|
||||
void
|
||||
@@ -316,7 +310,6 @@ struct MockETLServiceTest : virtual public NoLoggerFixture {
|
||||
void
|
||||
SetUp() override
|
||||
{
|
||||
NoLoggerFixture::SetUp();
|
||||
mockETLServicePtr = std::make_shared<MockETLService>();
|
||||
}
|
||||
void
|
||||
@@ -336,7 +329,6 @@ struct MockCountersTest : virtual public NoLoggerFixture {
|
||||
void
|
||||
SetUp() override
|
||||
{
|
||||
NoLoggerFixture::SetUp();
|
||||
mockCountersPtr = std::make_shared<MockCounters>();
|
||||
}
|
||||
void
|
||||
|
||||
@@ -19,7 +19,6 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/Source.hpp"
|
||||
#include "util/FakeFetchResponse.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
@@ -37,7 +36,6 @@ struct MockLoadBalancer {
|
||||
|
||||
MOCK_METHOD(void, loadInitialLedger, (std::uint32_t, bool), ());
|
||||
MOCK_METHOD(std::optional<FakeFetchResponse>, fetchLedger, (uint32_t, bool, bool), ());
|
||||
MOCK_METHOD(bool, shouldPropagateTxnStream, (etl::Source*), (const));
|
||||
MOCK_METHOD(boost::json::value, toJson, (), (const));
|
||||
MOCK_METHOD(
|
||||
std::optional<boost::json::object>,
|
||||
|
||||
@@ -18,8 +18,6 @@
|
||||
//==============================================================================
|
||||
#pragma once
|
||||
|
||||
#include "etl/Source.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/uuid/uuid.hpp>
|
||||
@@ -33,33 +31,28 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
class MockSource : public etl::Source {
|
||||
class MockSource {
|
||||
public:
|
||||
MOCK_METHOD(bool, isConnected, (), (const, override));
|
||||
MOCK_METHOD(boost::json::object, toJson, (), (const, override));
|
||||
MOCK_METHOD(void, run, (), (override));
|
||||
MOCK_METHOD(void, pause, (), (override));
|
||||
MOCK_METHOD(void, resume, (), (override));
|
||||
MOCK_METHOD(std::string, toString, (), (const, override));
|
||||
MOCK_METHOD(bool, hasLedger, (uint32_t), (const, override));
|
||||
MOCK_METHOD(
|
||||
(std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>),
|
||||
fetchLedger,
|
||||
(uint32_t, bool, bool),
|
||||
(override)
|
||||
);
|
||||
MOCK_METHOD((std::pair<std::vector<std::string>, bool>), loadInitialLedger, (uint32_t, uint32_t, bool), (override));
|
||||
MOCK_METHOD(bool, isConnected, (), (const));
|
||||
MOCK_METHOD(boost::json::object, toJson, (), (const));
|
||||
MOCK_METHOD(void, run, ());
|
||||
MOCK_METHOD(void, pause, ());
|
||||
MOCK_METHOD(void, resume, ());
|
||||
MOCK_METHOD(std::string, toString, (), (const));
|
||||
MOCK_METHOD(bool, hasLedger, (uint32_t), (const));
|
||||
MOCK_METHOD((std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>), fetchLedger, (uint32_t, bool, bool));
|
||||
MOCK_METHOD((std::pair<std::vector<std::string>, bool>), loadInitialLedger, (uint32_t, uint32_t, bool));
|
||||
MOCK_METHOD(
|
||||
std::optional<boost::json::object>,
|
||||
forwardToRippled,
|
||||
(boost::json::object const&, std::optional<std::string> const&, boost::asio::yield_context),
|
||||
(const, override)
|
||||
(const)
|
||||
);
|
||||
MOCK_METHOD(
|
||||
std::optional<boost::json::object>,
|
||||
requestFromRippled,
|
||||
(boost::json::object const&, std::optional<std::string> const&, boost::asio::yield_context),
|
||||
(const, override)
|
||||
(const)
|
||||
);
|
||||
MOCK_METHOD(boost::uuids::uuid, token, (), (const, override));
|
||||
MOCK_METHOD(boost::uuids::uuid, token, (), (const));
|
||||
};
|
||||
|
||||
@@ -27,6 +27,7 @@
|
||||
#include <boost/json/object.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <ripple/protocol/AccountID.h>
|
||||
#include <ripple/protocol/Book.h>
|
||||
#include <ripple/protocol/Fees.h>
|
||||
#include <ripple/protocol/LedgerHeader.h>
|
||||
|
||||
|
||||
104
unittests/util/MockXrpLedgerAPIService.hpp
Normal file
104
unittests/util/MockXrpLedgerAPIService.hpp
Normal file
@@ -0,0 +1,104 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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 <gmock/gmock.h>
|
||||
#include <grpcpp/security/server_credentials.h>
|
||||
#include <grpcpp/server.h>
|
||||
#include <grpcpp/server_builder.h>
|
||||
#include <grpcpp/server_context.h>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <gtest/gtest.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger_data.pb.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger_diff.pb.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger_entry.pb.h>
|
||||
#include <ripple/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
|
||||
namespace unittests::util {
|
||||
|
||||
struct MockXrpLedgerAPIService final : public org::xrpl::rpc::v1::XRPLedgerAPIService::Service {
|
||||
~MockXrpLedgerAPIService() override = default;
|
||||
|
||||
MOCK_METHOD(
|
||||
grpc::Status,
|
||||
GetLedger,
|
||||
(grpc::ServerContext * context,
|
||||
org::xrpl::rpc::v1::GetLedgerRequest const* request,
|
||||
org::xrpl::rpc::v1::GetLedgerResponse* response),
|
||||
(override)
|
||||
);
|
||||
|
||||
MOCK_METHOD(
|
||||
grpc::Status,
|
||||
GetLedgerEntry,
|
||||
(grpc::ServerContext * context,
|
||||
org::xrpl::rpc::v1::GetLedgerEntryRequest const* request,
|
||||
org::xrpl::rpc::v1::GetLedgerEntryResponse* response),
|
||||
(override)
|
||||
);
|
||||
|
||||
MOCK_METHOD(
|
||||
grpc::Status,
|
||||
GetLedgerData,
|
||||
(grpc::ServerContext * context,
|
||||
org::xrpl::rpc::v1::GetLedgerDataRequest const* request,
|
||||
org::xrpl::rpc::v1::GetLedgerDataResponse* response),
|
||||
(override)
|
||||
);
|
||||
|
||||
MOCK_METHOD(
|
||||
grpc::Status,
|
||||
GetLedgerDiff,
|
||||
(grpc::ServerContext * context,
|
||||
org::xrpl::rpc::v1::GetLedgerDiffRequest const* request,
|
||||
org::xrpl::rpc::v1::GetLedgerDiffResponse* response),
|
||||
(override)
|
||||
);
|
||||
};
|
||||
|
||||
struct WithMockXrpLedgerAPIService : virtual ::testing::Test {
|
||||
WithMockXrpLedgerAPIService(std::string serverAddress)
|
||||
{
|
||||
grpc::ServerBuilder builder;
|
||||
builder.AddListeningPort(serverAddress, grpc::InsecureServerCredentials());
|
||||
builder.RegisterService(&mockXrpLedgerAPIService);
|
||||
server_ = builder.BuildAndStart();
|
||||
serverThread_ = std::thread([this] { server_->Wait(); });
|
||||
}
|
||||
|
||||
~WithMockXrpLedgerAPIService() override
|
||||
{
|
||||
server_->Shutdown();
|
||||
serverThread_.join();
|
||||
}
|
||||
|
||||
MockXrpLedgerAPIService mockXrpLedgerAPIService;
|
||||
|
||||
private:
|
||||
std::unique_ptr<grpc::Server> server_;
|
||||
std::thread serverThread_;
|
||||
};
|
||||
|
||||
} // namespace unittests::util
|
||||
107
unittests/util/RetryTests.cpp
Normal file
107
unittests/util/RetryTests.cpp
Normal file
@@ -0,0 +1,107 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2024, 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 "util/Fixtures.hpp"
|
||||
#include "util/Retry.hpp"
|
||||
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <gmock/gmock.h>
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
|
||||
using namespace util;
|
||||
|
||||
struct RetryTests : virtual ::testing::Test {
|
||||
std::chrono::milliseconds const delay_{1};
|
||||
std::chrono::milliseconds const maxDelay_{5};
|
||||
};
|
||||
|
||||
TEST_F(RetryTests, ExponentialBackoffStrategy)
|
||||
{
|
||||
ExponentialBackoffStrategy strategy{delay_, maxDelay_};
|
||||
|
||||
EXPECT_EQ(strategy.getDelay(), delay_);
|
||||
|
||||
strategy.increaseDelay();
|
||||
EXPECT_EQ(strategy.getDelay(), delay_ * 2);
|
||||
|
||||
strategy.increaseDelay();
|
||||
EXPECT_LT(strategy.getDelay(), maxDelay_);
|
||||
|
||||
for (size_t i = 0; i < 10; ++i) {
|
||||
strategy.increaseDelay();
|
||||
EXPECT_EQ(strategy.getDelay(), maxDelay_);
|
||||
EXPECT_EQ(strategy.getDelay(), maxDelay_);
|
||||
}
|
||||
|
||||
strategy.reset();
|
||||
EXPECT_EQ(strategy.getDelay(), delay_);
|
||||
}
|
||||
|
||||
struct RetryWithExponentialBackoffStrategyTests : SyncAsioContextTest, RetryTests {
|
||||
RetryWithExponentialBackoffStrategyTests()
|
||||
{
|
||||
EXPECT_EQ(retry_.attemptNumber(), 0);
|
||||
EXPECT_EQ(retry_.delayValue(), delay_);
|
||||
}
|
||||
|
||||
Retry retry_ = makeRetryExponentialBackoff(delay_, maxDelay_, boost::asio::make_strand(ctx));
|
||||
testing::MockFunction<void()> mockCallback_;
|
||||
};
|
||||
|
||||
TEST_F(RetryWithExponentialBackoffStrategyTests, Retry)
|
||||
{
|
||||
retry_.retry(mockCallback_.AsStdFunction());
|
||||
|
||||
EXPECT_EQ(retry_.attemptNumber(), 0);
|
||||
|
||||
EXPECT_CALL(mockCallback_, Call());
|
||||
runContext();
|
||||
|
||||
EXPECT_EQ(retry_.attemptNumber(), 1);
|
||||
EXPECT_EQ(retry_.delayValue(), delay_ * 2);
|
||||
}
|
||||
|
||||
TEST_F(RetryWithExponentialBackoffStrategyTests, Cancel)
|
||||
{
|
||||
retry_.retry(mockCallback_.AsStdFunction());
|
||||
retry_.cancel();
|
||||
runContext();
|
||||
EXPECT_EQ(retry_.attemptNumber(), 0);
|
||||
|
||||
retry_.cancel();
|
||||
EXPECT_EQ(retry_.attemptNumber(), 0);
|
||||
}
|
||||
|
||||
TEST_F(RetryWithExponentialBackoffStrategyTests, Reset)
|
||||
{
|
||||
retry_.retry(mockCallback_.AsStdFunction());
|
||||
|
||||
EXPECT_CALL(mockCallback_, Call());
|
||||
runContext();
|
||||
|
||||
EXPECT_EQ(retry_.attemptNumber(), 1);
|
||||
EXPECT_EQ(retry_.delayValue(), delay_ * 2);
|
||||
|
||||
retry_.reset();
|
||||
EXPECT_EQ(retry_.attemptNumber(), 0);
|
||||
EXPECT_EQ(retry_.delayValue(), delay_);
|
||||
}
|
||||
@@ -47,7 +47,12 @@ using tcp = boost::asio::ip::tcp;
|
||||
namespace {
|
||||
|
||||
void
|
||||
doSession(beast::tcp_stream stream, TestHttpServer::RequestHandler requestHandler, asio::yield_context yield)
|
||||
doSession(
|
||||
beast::tcp_stream stream,
|
||||
TestHttpServer::RequestHandler requestHandler,
|
||||
asio::yield_context yield,
|
||||
bool const allowToFail
|
||||
)
|
||||
{
|
||||
beast::error_code errorCode;
|
||||
|
||||
@@ -64,6 +69,9 @@ doSession(beast::tcp_stream stream, TestHttpServer::RequestHandler requestHandle
|
||||
if (errorCode == http::error::end_of_stream)
|
||||
return;
|
||||
|
||||
if (allowToFail and errorCode)
|
||||
return;
|
||||
|
||||
ASSERT_FALSE(errorCode) << errorCode.message();
|
||||
|
||||
auto response = requestHandler(req);
|
||||
@@ -78,6 +86,9 @@ doSession(beast::tcp_stream stream, TestHttpServer::RequestHandler requestHandle
|
||||
// Send the response
|
||||
beast::async_write(stream, std::move(messageGenerator), yield[errorCode]);
|
||||
|
||||
if (allowToFail and errorCode)
|
||||
return;
|
||||
|
||||
ASSERT_FALSE(errorCode) << errorCode.message();
|
||||
|
||||
if (!keep_alive) {
|
||||
@@ -104,18 +115,21 @@ TestHttpServer::TestHttpServer(boost::asio::io_context& context, std::string hos
|
||||
}
|
||||
|
||||
void
|
||||
TestHttpServer::handleRequest(TestHttpServer::RequestHandler handler)
|
||||
TestHttpServer::handleRequest(TestHttpServer::RequestHandler handler, bool const allowToFail)
|
||||
{
|
||||
boost::asio::spawn(
|
||||
acceptor_.get_executor(),
|
||||
[this, handler = std::move(handler)](asio::yield_context yield) mutable {
|
||||
[this, allowToFail, handler = std::move(handler)](asio::yield_context yield) mutable {
|
||||
boost::beast::error_code errorCode;
|
||||
tcp::socket socket(this->acceptor_.get_executor());
|
||||
acceptor_.async_accept(socket, yield[errorCode]);
|
||||
|
||||
if (allowToFail and errorCode)
|
||||
return;
|
||||
|
||||
[&]() { ASSERT_FALSE(errorCode) << errorCode.message(); }();
|
||||
|
||||
doSession(beast::tcp_stream{std::move(socket)}, std::move(handler), yield);
|
||||
doSession(beast::tcp_stream{std::move(socket)}, std::move(handler), yield, allowToFail);
|
||||
},
|
||||
boost::asio::detached
|
||||
);
|
||||
|
||||
@@ -51,9 +51,10 @@ public:
|
||||
* @note This method schedules to process only one request
|
||||
*
|
||||
* @param handler RequestHandler to use for incoming request
|
||||
* @param allowToFail if true, the server will not throw an exception if the request fails
|
||||
*/
|
||||
void
|
||||
handleRequest(RequestHandler handler);
|
||||
handleRequest(RequestHandler handler, bool allowToFail = false);
|
||||
|
||||
private:
|
||||
boost::asio::ip::tcp::acceptor acceptor_;
|
||||
|
||||
@@ -19,6 +19,9 @@
|
||||
|
||||
#include "util/TestWsServer.hpp"
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include <boost/asio/buffer.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/ip/address.hpp>
|
||||
@@ -29,6 +32,8 @@
|
||||
#include <boost/beast/core/error.hpp>
|
||||
#include <boost/beast/core/flat_buffer.hpp>
|
||||
#include <boost/beast/core/role.hpp>
|
||||
#include <boost/beast/core/tcp_stream.hpp>
|
||||
#include <boost/beast/version.hpp>
|
||||
#include <boost/beast/websocket/error.hpp>
|
||||
#include <boost/beast/websocket/rfc6455.hpp>
|
||||
#include <boost/beast/websocket/stream_base.hpp>
|
||||
@@ -39,22 +44,16 @@
|
||||
#include <utility>
|
||||
|
||||
namespace asio = boost::asio;
|
||||
namespace beast = boost::beast;
|
||||
namespace websocket = boost::beast::websocket;
|
||||
|
||||
TestWsConnection::TestWsConnection(asio::ip::tcp::socket&& socket, boost::asio::yield_context yield)
|
||||
: ws_(std::move(socket))
|
||||
TestWsConnection::TestWsConnection(websocket::stream<boost::beast::tcp_stream> wsStream) : ws_(std::move(wsStream))
|
||||
{
|
||||
ws_.set_option(websocket::stream_base::timeout::suggested(beast::role_type::server));
|
||||
beast::error_code errorCode;
|
||||
ws_.async_accept(yield[errorCode]);
|
||||
[&]() { ASSERT_FALSE(errorCode) << errorCode.message(); }();
|
||||
}
|
||||
|
||||
std::optional<std::string>
|
||||
TestWsConnection::send(std::string const& message, boost::asio::yield_context yield)
|
||||
{
|
||||
beast::error_code errorCode;
|
||||
boost::beast::error_code errorCode;
|
||||
ws_.async_write(asio::buffer(message), yield[errorCode]);
|
||||
if (errorCode)
|
||||
return errorCode.message();
|
||||
@@ -64,21 +63,21 @@ TestWsConnection::send(std::string const& message, boost::asio::yield_context yi
|
||||
std::optional<std::string>
|
||||
TestWsConnection::receive(boost::asio::yield_context yield)
|
||||
{
|
||||
beast::error_code errorCode;
|
||||
beast::flat_buffer buffer;
|
||||
boost::beast::error_code errorCode;
|
||||
boost::beast::flat_buffer buffer;
|
||||
|
||||
ws_.async_read(buffer, yield[errorCode]);
|
||||
if (errorCode == websocket::error::closed)
|
||||
return std::nullopt;
|
||||
|
||||
[&]() { ASSERT_FALSE(errorCode) << errorCode.message(); }();
|
||||
return beast::buffers_to_string(buffer.data());
|
||||
return boost::beast::buffers_to_string(buffer.data());
|
||||
}
|
||||
|
||||
std::optional<std::string>
|
||||
TestWsConnection::close(boost::asio::yield_context yield)
|
||||
{
|
||||
beast::error_code errorCode;
|
||||
boost::beast::error_code errorCode;
|
||||
ws_.async_close(websocket::close_code::normal, yield[errorCode]);
|
||||
if (errorCode)
|
||||
return errorCode.message();
|
||||
@@ -93,23 +92,39 @@ TestWsServer::TestWsServer(asio::io_context& context, std::string const& host, i
|
||||
acceptor_.bind(endpoint);
|
||||
}
|
||||
|
||||
TestWsConnection
|
||||
util::Expected<TestWsConnection, util::requests::RequestError>
|
||||
TestWsServer::acceptConnection(asio::yield_context yield)
|
||||
{
|
||||
acceptor_.listen(asio::socket_base::max_listen_connections);
|
||||
beast::error_code errorCode;
|
||||
|
||||
boost::beast::error_code errorCode;
|
||||
asio::ip::tcp::socket socket(acceptor_.get_executor());
|
||||
acceptor_.async_accept(socket, yield[errorCode]);
|
||||
[&]() { ASSERT_FALSE(errorCode) << errorCode.message(); }();
|
||||
return TestWsConnection(std::move(socket), yield);
|
||||
if (errorCode)
|
||||
return util::Unexpected{util::requests::RequestError{"Accept error", errorCode}};
|
||||
|
||||
boost::beast::websocket::stream<boost::beast::tcp_stream> ws(std::move(socket));
|
||||
ws.set_option(websocket::stream_base::timeout::suggested(boost::beast::role_type::server));
|
||||
ws.async_accept(yield[errorCode]);
|
||||
if (errorCode)
|
||||
return util::Unexpected{util::requests::RequestError{"Handshake error", errorCode}};
|
||||
|
||||
return TestWsConnection(std::move(ws));
|
||||
}
|
||||
|
||||
void
|
||||
TestWsServer::acceptConnectionAndDropIt(asio::yield_context yield)
|
||||
{
|
||||
acceptConnectionWithoutHandshake(yield);
|
||||
}
|
||||
|
||||
boost::asio::ip::tcp::socket
|
||||
TestWsServer::acceptConnectionWithoutHandshake(boost::asio::yield_context yield)
|
||||
{
|
||||
acceptor_.listen(asio::socket_base::max_listen_connections);
|
||||
beast::error_code errorCode;
|
||||
boost::beast::error_code errorCode;
|
||||
asio::ip::tcp::socket socket(acceptor_.get_executor());
|
||||
acceptor_.async_accept(socket, yield[errorCode]);
|
||||
[&]() { ASSERT_FALSE(errorCode) << errorCode.message(); }();
|
||||
return socket;
|
||||
}
|
||||
|
||||
@@ -19,6 +19,9 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/ip/tcp.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
@@ -36,7 +39,7 @@ public:
|
||||
using SendCallback = std::function<void()>;
|
||||
using ReceiveCallback = std::function<void(std::string)>;
|
||||
|
||||
TestWsConnection(boost::asio::ip::tcp::socket&& socket, boost::asio::yield_context yield);
|
||||
TestWsConnection(boost::beast::websocket::stream<boost::beast::tcp_stream> wsStream);
|
||||
|
||||
// returns error message if error occurs
|
||||
std::optional<std::string>
|
||||
@@ -56,9 +59,12 @@ class TestWsServer {
|
||||
public:
|
||||
TestWsServer(boost::asio::io_context& context, std::string const& host, int port);
|
||||
|
||||
TestWsConnection
|
||||
util::Expected<TestWsConnection, util::requests::RequestError>
|
||||
acceptConnection(boost::asio::yield_context yield);
|
||||
|
||||
void
|
||||
acceptConnectionAndDropIt(boost::asio::yield_context yield);
|
||||
|
||||
boost::asio::ip::tcp::socket
|
||||
acceptConnectionWithoutHandshake(boost::asio::yield_context yield);
|
||||
};
|
||||
|
||||
@@ -35,6 +35,7 @@
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <variant>
|
||||
#include <vector>
|
||||
|
||||
using namespace util::requests;
|
||||
@@ -48,11 +49,13 @@ struct RequestBuilderTestBundle {
|
||||
std::string target;
|
||||
};
|
||||
|
||||
struct RequestBuilderTest : SyncAsioContextTest, testing::WithParamInterface<RequestBuilderTestBundle> {
|
||||
struct RequestBuilderTestBase : SyncAsioContextTest {
|
||||
TestHttpServer server{ctx, "0.0.0.0", 11111};
|
||||
RequestBuilder builder{"localhost", "11111"};
|
||||
};
|
||||
|
||||
struct RequestBuilderTest : RequestBuilderTestBase, testing::WithParamInterface<RequestBuilderTestBundle> {};
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(
|
||||
RequestBuilderTest,
|
||||
RequestBuilderTest,
|
||||
@@ -61,7 +64,9 @@ INSTANTIATE_TEST_CASE_P(
|
||||
RequestBuilderTestBundle{
|
||||
"GetWithHeaders",
|
||||
http::verb::get,
|
||||
{{http::field::accept, "text/html"}, {http::field::authorization, "password"}},
|
||||
{{http::field::accept, "text/html"},
|
||||
{http::field::authorization, "password"},
|
||||
{"Custom_header", "some_value"}},
|
||||
"/"
|
||||
},
|
||||
RequestBuilderTestBundle{"GetWithTarget", http::verb::get, {}, "/test"},
|
||||
@@ -69,7 +74,9 @@ INSTANTIATE_TEST_CASE_P(
|
||||
RequestBuilderTestBundle{
|
||||
"PostWithHeaders",
|
||||
http::verb::post,
|
||||
{{http::field::accept, "text/html"}, {http::field::authorization, "password"}},
|
||||
{{http::field::accept, "text/html"},
|
||||
{http::field::authorization, "password"},
|
||||
{"Custom_header", "some_value"}},
|
||||
"/"
|
||||
},
|
||||
RequestBuilderTestBundle{"PostWithTarget", http::verb::post, {}, "/test"}
|
||||
@@ -86,8 +93,18 @@ TEST_P(RequestBuilderTest, SimpleRequest)
|
||||
server.handleRequest(
|
||||
[&replyBody](http::request<http::string_body> request) -> std::optional<http::response<http::string_body>> {
|
||||
[&]() {
|
||||
ASSERT_TRUE(request.target() == GetParam().target);
|
||||
ASSERT_TRUE(request.method() == GetParam().method);
|
||||
EXPECT_TRUE(request.target() == GetParam().target);
|
||||
EXPECT_TRUE(request.method() == GetParam().method);
|
||||
for (auto const& header : GetParam().headers) {
|
||||
std::visit(
|
||||
[&](auto const& name) {
|
||||
auto it = request.find(name);
|
||||
ASSERT_NE(it, request.end());
|
||||
EXPECT_EQ(it->value(), header.value);
|
||||
},
|
||||
header.name
|
||||
);
|
||||
}
|
||||
}();
|
||||
return http::response<http::string_body>{http::status::ok, 11, replyBody};
|
||||
}
|
||||
@@ -97,14 +114,14 @@ TEST_P(RequestBuilderTest, SimpleRequest)
|
||||
auto const response = [&]() -> util::Expected<std::string, RequestError> {
|
||||
switch (GetParam().method) {
|
||||
case http::verb::get:
|
||||
return builder.get(yield);
|
||||
return builder.getPlain(yield);
|
||||
case http::verb::post:
|
||||
return builder.post(yield);
|
||||
return builder.postPlain(yield);
|
||||
default:
|
||||
return util::Unexpected{RequestError{"Invalid HTTP verb"}};
|
||||
}
|
||||
}();
|
||||
ASSERT_TRUE(response) << response.error().message;
|
||||
ASSERT_TRUE(response) << response.error().message();
|
||||
EXPECT_EQ(response.value(), replyBody);
|
||||
});
|
||||
}
|
||||
@@ -123,7 +140,7 @@ TEST_F(RequestBuilderTest, Timeout)
|
||||
}
|
||||
);
|
||||
runSpawn([this](asio::yield_context yield) {
|
||||
auto response = builder.get(yield);
|
||||
auto response = builder.getPlain(yield);
|
||||
EXPECT_FALSE(response);
|
||||
});
|
||||
}
|
||||
@@ -147,8 +164,8 @@ TEST_F(RequestBuilderTest, RequestWithBody)
|
||||
);
|
||||
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto const response = builder.get(yield);
|
||||
ASSERT_TRUE(response) << response.error().message;
|
||||
auto const response = builder.getPlain(yield);
|
||||
ASSERT_TRUE(response) << response.error().message();
|
||||
EXPECT_EQ(response.value(), replyBody) << response.value();
|
||||
});
|
||||
}
|
||||
@@ -157,9 +174,9 @@ TEST_F(RequestBuilderTest, ResolveError)
|
||||
{
|
||||
builder = RequestBuilder{"wrong_host", "11111"};
|
||||
runSpawn([this](asio::yield_context yield) {
|
||||
auto const response = builder.get(yield);
|
||||
auto const response = builder.getPlain(yield);
|
||||
ASSERT_FALSE(response);
|
||||
EXPECT_TRUE(response.error().message.starts_with("Resolve error")) << response.error().message;
|
||||
EXPECT_TRUE(response.error().message().starts_with("Resolve error")) << response.error().message();
|
||||
});
|
||||
}
|
||||
|
||||
@@ -168,21 +185,75 @@ TEST_F(RequestBuilderTest, ConnectionError)
|
||||
builder = RequestBuilder{"localhost", "11112"};
|
||||
builder.setTimeout(std::chrono::milliseconds{1});
|
||||
runSpawn([this](asio::yield_context yield) {
|
||||
auto const response = builder.get(yield);
|
||||
auto const response = builder.getPlain(yield);
|
||||
ASSERT_FALSE(response);
|
||||
EXPECT_TRUE(response.error().message.starts_with("Connection error")) << response.error().message;
|
||||
EXPECT_TRUE(response.error().message().starts_with("Connection error")) << response.error().message();
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(RequestBuilderTest, WritingError)
|
||||
TEST_F(RequestBuilderTest, ResponseStatusIsNotOk)
|
||||
{
|
||||
server.handleRequest([](auto&&) -> std::optional<http::response<http::string_body>> {
|
||||
return http::response<http::string_body>{http::status::not_found, 11, "Not found"};
|
||||
});
|
||||
|
||||
runSpawn([this](asio::yield_context yield) {
|
||||
auto const response = builder.getPlain(yield);
|
||||
ASSERT_FALSE(response);
|
||||
EXPECT_TRUE(response.error().message().starts_with("Response status is not OK")) << response.error().message();
|
||||
});
|
||||
}
|
||||
|
||||
struct RequestBuilderSslTestBundle {
|
||||
std::string testName;
|
||||
boost::beast::http::verb method;
|
||||
};
|
||||
|
||||
struct RequestBuilderSslTest : RequestBuilderTestBase, testing::WithParamInterface<RequestBuilderSslTestBundle> {};
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(
|
||||
RequestBuilderSslTest,
|
||||
RequestBuilderSslTest,
|
||||
testing::Values(
|
||||
RequestBuilderSslTestBundle{"Get", http::verb::get},
|
||||
RequestBuilderSslTestBundle{"Post", http::verb::post}
|
||||
),
|
||||
[](auto const& info) { return info.param.testName; }
|
||||
);
|
||||
|
||||
TEST_P(RequestBuilderSslTest, TrySslUsePlain)
|
||||
{
|
||||
// First try will be SSL, but the server can't handle SSL requests
|
||||
server.handleRequest(
|
||||
[](http::request<http::string_body> request) -> std::optional<http::response<http::string_body>> {
|
||||
[](auto&&) -> std::optional<http::response<http::string_body>> {
|
||||
[]() { FAIL(); }();
|
||||
return std::nullopt;
|
||||
},
|
||||
true
|
||||
);
|
||||
|
||||
server.handleRequest(
|
||||
[&](http::request<http::string_body> request) -> std::optional<http::response<http::string_body>> {
|
||||
[&]() {
|
||||
EXPECT_EQ(request.target(), "/");
|
||||
EXPECT_EQ(request.method(), http::verb::get);
|
||||
EXPECT_EQ(request.method(), GetParam().method);
|
||||
}();
|
||||
return std::nullopt;
|
||||
return http::response<http::string_body>{http::status::ok, 11, "Hello, world!"};
|
||||
}
|
||||
);
|
||||
|
||||
runSpawn([this](asio::yield_context yield) {
|
||||
auto const response = [&]() -> util::Expected<std::string, RequestError> {
|
||||
switch (GetParam().method) {
|
||||
case http::verb::get:
|
||||
return builder.get(yield);
|
||||
case http::verb::post:
|
||||
return builder.post(yield);
|
||||
default:
|
||||
return util::Unexpected{RequestError{"Invalid HTTP verb"}};
|
||||
}
|
||||
}();
|
||||
ASSERT_TRUE(response) << response.error().message();
|
||||
EXPECT_EQ(response.value(), "Hello, world!");
|
||||
});
|
||||
}
|
||||
|
||||
@@ -17,19 +17,28 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "util/Expected.hpp"
|
||||
#include "util/Fixtures.hpp"
|
||||
#include "util/TestWsServer.hpp"
|
||||
#include "util/requests/Types.hpp"
|
||||
#include "util/requests/WsConnection.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/post.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <boost/asio/use_future.hpp>
|
||||
#include <boost/beast/http/field.hpp>
|
||||
#include <gtest/gtest.h>
|
||||
#include <openssl/ssl.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <iostream>
|
||||
#include <optional>
|
||||
#include <ostream>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <vector>
|
||||
|
||||
using namespace util::requests;
|
||||
@@ -39,6 +48,14 @@ namespace http = boost::beast::http;
|
||||
struct WsConnectionTestsBase : SyncAsioContextTest {
|
||||
WsConnectionBuilder builder{"localhost", "11112"};
|
||||
TestWsServer server{ctx, "0.0.0.0", 11112};
|
||||
|
||||
template <typename T, typename E>
|
||||
T
|
||||
unwrap(util::Expected<T, E> expected)
|
||||
{
|
||||
[&]() { ASSERT_TRUE(expected.has_value()) << expected.error().message(); }();
|
||||
return std::move(expected).value();
|
||||
}
|
||||
};
|
||||
|
||||
struct WsConnectionTestBundle {
|
||||
@@ -65,7 +82,9 @@ INSTANTIATE_TEST_CASE_P(
|
||||
WsConnectionTestBundle{"singleHeader", {{http::field::accept, "text/html"}}, std::nullopt},
|
||||
WsConnectionTestBundle{
|
||||
"multiple headers",
|
||||
{{http::field::accept, "text/html"}, {http::field::authorization, "password"}},
|
||||
{{http::field::accept, "text/html"},
|
||||
{http::field::authorization, "password"},
|
||||
{"Custom_header", "some_value"}},
|
||||
std::nullopt
|
||||
},
|
||||
WsConnectionTestBundle{"target", {}, "/target"}
|
||||
@@ -74,17 +93,13 @@ INSTANTIATE_TEST_CASE_P(
|
||||
|
||||
TEST_P(WsConnectionTests, SendAndReceive)
|
||||
{
|
||||
auto target = GetParam().target;
|
||||
if (target) {
|
||||
if (auto const target = GetParam().target; target) {
|
||||
builder.setTarget(*target);
|
||||
}
|
||||
|
||||
for (auto const& header : GetParam().headers) {
|
||||
builder.addHeader(header);
|
||||
}
|
||||
builder.addHeaders(GetParam().headers);
|
||||
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) {
|
||||
auto serverConnection = server.acceptConnection(yield);
|
||||
auto serverConnection = unwrap(server.acceptConnection(yield));
|
||||
|
||||
for (size_t i = 0; i < clientMessages.size(); ++i) {
|
||||
auto message = serverConnection.receive(yield);
|
||||
@@ -96,29 +111,58 @@ TEST_P(WsConnectionTests, SendAndReceive)
|
||||
});
|
||||
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto maybeConnection = builder.connect(yield);
|
||||
ASSERT_TRUE(maybeConnection.has_value()) << maybeConnection.error().message;
|
||||
auto maybeConnection = builder.plainConnect(yield);
|
||||
ASSERT_TRUE(maybeConnection.has_value()) << maybeConnection.error().message();
|
||||
auto& connection = *maybeConnection;
|
||||
|
||||
for (size_t i = 0; i < serverMessages.size(); ++i) {
|
||||
auto error = connection->write(clientMessages.at(i), yield);
|
||||
ASSERT_FALSE(error) << error->message;
|
||||
ASSERT_FALSE(error) << error->message();
|
||||
|
||||
auto message = connection->read(yield);
|
||||
ASSERT_TRUE(message.has_value()) << message.error().message;
|
||||
ASSERT_TRUE(message.has_value()) << message.error().message();
|
||||
EXPECT_EQ(serverMessages.at(i), message.value());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(WsConnectionTests, TrySslUsePlain)
|
||||
{
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) {
|
||||
// Client attempts to establish SSL connection first which will fail
|
||||
auto failedConnection = server.acceptConnection(yield);
|
||||
EXPECT_FALSE(failedConnection.has_value());
|
||||
|
||||
auto serverConnection = unwrap(server.acceptConnection(yield));
|
||||
auto message = serverConnection.receive(yield);
|
||||
EXPECT_EQ(message, "hello");
|
||||
|
||||
auto error = serverConnection.send("goodbye", yield);
|
||||
EXPECT_FALSE(error) << *error;
|
||||
});
|
||||
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto maybeConnection = builder.connect(yield);
|
||||
ASSERT_TRUE(maybeConnection.has_value()) << maybeConnection.error().message();
|
||||
auto& connection = *maybeConnection;
|
||||
|
||||
auto error = connection->write("hello", yield);
|
||||
ASSERT_FALSE(error) << error->message();
|
||||
|
||||
auto message = connection->read(yield);
|
||||
ASSERT_TRUE(message.has_value()) << message.error().message();
|
||||
EXPECT_EQ(message.value(), "goodbye");
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(WsConnectionTests, Timeout)
|
||||
{
|
||||
builder.setConnectionTimeout(std::chrono::milliseconds{1});
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto connection = builder.connect(yield);
|
||||
auto connection = builder.plainConnect(yield);
|
||||
ASSERT_FALSE(connection.has_value());
|
||||
|
||||
EXPECT_TRUE(connection.error().message.starts_with("Connect error"));
|
||||
EXPECT_TRUE(connection.error().message().starts_with("Connect error"));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -126,9 +170,9 @@ TEST_F(WsConnectionTests, ResolveError)
|
||||
{
|
||||
builder = WsConnectionBuilder{"wrong_host", "11112"};
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto connection = builder.connect(yield);
|
||||
auto connection = builder.plainConnect(yield);
|
||||
ASSERT_FALSE(connection.has_value());
|
||||
EXPECT_TRUE(connection.error().message.starts_with("Resolve error")) << connection.error().message;
|
||||
EXPECT_TRUE(connection.error().message().starts_with("Resolve error")) << connection.error().message();
|
||||
});
|
||||
}
|
||||
|
||||
@@ -137,27 +181,55 @@ TEST_F(WsConnectionTests, WsHandshakeError)
|
||||
builder.setConnectionTimeout(std::chrono::milliseconds{1});
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) { server.acceptConnectionAndDropIt(yield); });
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto connection = builder.connect(yield);
|
||||
auto connection = builder.plainConnect(yield);
|
||||
ASSERT_FALSE(connection.has_value());
|
||||
EXPECT_TRUE(connection.error().message.starts_with("Handshake error")) << connection.error().message;
|
||||
EXPECT_TRUE(connection.error().message().starts_with("Handshake error")) << connection.error().message();
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(WsConnectionTests, WsHandshakeTimeout)
|
||||
{
|
||||
builder.setWsHandshakeTimeout(std::chrono::milliseconds{1});
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) {
|
||||
auto socket = server.acceptConnectionWithoutHandshake(yield);
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds{10});
|
||||
});
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto connection = builder.plainConnect(yield);
|
||||
ASSERT_FALSE(connection.has_value());
|
||||
EXPECT_TRUE(connection.error().message().starts_with("Handshake error")) << connection.error().message();
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(WsConnectionTests, CloseConnection)
|
||||
{
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) {
|
||||
auto serverConnection = server.acceptConnection(yield);
|
||||
auto serverConnection = unwrap(server.acceptConnection(yield));
|
||||
|
||||
auto message = serverConnection.receive(yield);
|
||||
EXPECT_EQ(std::nullopt, message);
|
||||
});
|
||||
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto connection = builder.connect(yield);
|
||||
ASSERT_TRUE(connection.has_value()) << connection.error().message;
|
||||
auto connection = unwrap(builder.plainConnect(yield));
|
||||
|
||||
auto error = connection->operator*().close(yield);
|
||||
EXPECT_FALSE(error.has_value()) << error->message;
|
||||
auto error = connection->close(yield);
|
||||
EXPECT_FALSE(error.has_value()) << error->message();
|
||||
});
|
||||
}
|
||||
|
||||
TEST_F(WsConnectionTests, CloseConnectionTimeout)
|
||||
{
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) {
|
||||
auto serverConnection = unwrap(server.acceptConnection(yield));
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds{10});
|
||||
});
|
||||
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto connection = unwrap(builder.plainConnect(yield));
|
||||
|
||||
auto error = connection->close(yield, std::chrono::milliseconds{1});
|
||||
EXPECT_TRUE(error.has_value());
|
||||
});
|
||||
}
|
||||
|
||||
@@ -165,7 +237,7 @@ TEST_F(WsConnectionTests, MultipleConnections)
|
||||
{
|
||||
for (size_t i = 0; i < 2; ++i) {
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) {
|
||||
auto serverConnection = server.acceptConnection(yield);
|
||||
auto serverConnection = unwrap(server.acceptConnection(yield));
|
||||
auto message = serverConnection.receive(yield);
|
||||
|
||||
ASSERT_TRUE(message.has_value());
|
||||
@@ -173,11 +245,11 @@ TEST_F(WsConnectionTests, MultipleConnections)
|
||||
});
|
||||
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto connection = builder.connect(yield);
|
||||
ASSERT_TRUE(connection.has_value()) << connection.error().message;
|
||||
auto connection = builder.plainConnect(yield);
|
||||
ASSERT_TRUE(connection.has_value()) << connection.error().message();
|
||||
|
||||
auto error = connection->operator*().write("hello", yield);
|
||||
ASSERT_FALSE(error) << error->message;
|
||||
ASSERT_FALSE(error) << error->message();
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -201,22 +273,22 @@ INSTANTIATE_TEST_SUITE_P(
|
||||
}
|
||||
);
|
||||
|
||||
TEST_P(WsConnectionErrorTests, WriteError)
|
||||
TEST_P(WsConnectionErrorTests, ReadWriteError)
|
||||
{
|
||||
asio::spawn(ctx, [&](asio::yield_context yield) {
|
||||
auto serverConnection = server.acceptConnection(yield);
|
||||
auto serverConnection = unwrap(server.acceptConnection(yield));
|
||||
|
||||
auto error = serverConnection.close(yield);
|
||||
EXPECT_FALSE(error.has_value()) << *error;
|
||||
});
|
||||
|
||||
runSpawn([&](asio::yield_context yield) {
|
||||
auto maybeConnection = builder.connect(yield);
|
||||
ASSERT_TRUE(maybeConnection.has_value()) << maybeConnection.error().message;
|
||||
auto maybeConnection = builder.plainConnect(yield);
|
||||
ASSERT_TRUE(maybeConnection.has_value()) << maybeConnection.error().message();
|
||||
auto& connection = *maybeConnection;
|
||||
|
||||
auto error = connection->close(yield);
|
||||
EXPECT_FALSE(error.has_value()) << error->message;
|
||||
EXPECT_FALSE(error.has_value()) << error->message();
|
||||
|
||||
switch (GetParam()) {
|
||||
case WsConnectionErrorTestsBundle::Read: {
|
||||
|
||||
@@ -166,12 +166,6 @@ protected:
|
||||
runner.emplace([this] { ctx.run(); });
|
||||
}
|
||||
|
||||
void
|
||||
SetUp() override
|
||||
{
|
||||
NoLoggerFixture::SetUp();
|
||||
}
|
||||
|
||||
// this ctx is for dos timer
|
||||
boost::asio::io_context ctxSync;
|
||||
Config cfg{boost::json::parse(JSONData)};
|
||||
|
||||
Reference in New Issue
Block a user