ctid for tx (#907)

Fix #898 and #917
This commit is contained in:
cyan317
2023-10-11 09:47:05 +01:00
committed by GitHub
parent 5e6682ddc7
commit 8d783ecd6a
24 changed files with 1038 additions and 219 deletions

View File

@@ -90,6 +90,7 @@ target_sources (clio PRIVATE
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
## Feed
@@ -133,7 +134,6 @@ target_sources (clio PRIVATE
src/rpc/handlers/NoRippleCheck.cpp
src/rpc/handlers/Random.cpp
src/rpc/handlers/TransactionEntry.cpp
src/rpc/handlers/Tx.cpp
## Util
src/util/config/Config.cpp
src/util/log/Logger.cpp
@@ -170,6 +170,7 @@ if (tests)
unittests/etl/CacheLoaderTests.cpp
unittests/etl/AmendmentBlockHandlerTests.cpp
unittests/etl/LedgerPublisherTests.cpp
unittests/etl/ETLStateTests.cpp
# RPC
unittests/rpc/ErrorTests.cpp
unittests/rpc/BaseTests.cpp

View File

@@ -205,6 +205,15 @@ public:
return result;
}
/**
* @brief Get the etl nodes' state
*/
etl::ETLState
getETLState() const noexcept
{
return loadBalancer_->getETLState();
}
private:
/**
* @brief Run the ETL pipeline.

59
src/etl/ETLState.cpp Normal file
View File

@@ -0,0 +1,59 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <data/BackendInterface.h>
#include <etl/ETLState.h>
#include <etl/Source.h>
#include <rpc/JS.h>
namespace etl {
ETLState
ETLState::fetchETLStateFromSource(Source const& source) noexcept
{
auto const serverInfoRippled = data::synchronous([&source](auto yield) {
return source.forwardToRippled({{"command", "server_info"}}, std::nullopt, yield);
});
if (serverInfoRippled)
return boost::json::value_to<ETLState>(boost::json::value(*serverInfoRippled));
return ETLState{};
}
ETLState
tag_invoke(boost::json::value_to_tag<ETLState>, boost::json::value const& jv)
{
ETLState state;
auto const& jsonObject = jv.as_object();
if (!jsonObject.contains(JS(error)))
{
if (jsonObject.contains(JS(result)) && jsonObject.at(JS(result)).as_object().contains(JS(info)))
{
auto const rippledInfo = jsonObject.at(JS(result)).as_object().at(JS(info)).as_object();
if (rippledInfo.contains(JS(network_id)))
state.networkID.emplace(boost::json::value_to<int64_t>(rippledInfo.at(JS(network_id))));
}
}
return state;
}
} // namespace etl

48
src/etl/ETLState.h Normal file
View File

@@ -0,0 +1,48 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <boost/json.hpp>
#include <cstdint>
#include <optional>
namespace etl {
class Source;
/**
* @brief This class is responsible for fetching and storing the state of the ETL information, such as the network id
*/
struct ETLState
{
std::optional<uint32_t> networkID;
/**
* @brief Fetch the ETL state from the rippled server
*/
static ETLState
fetchETLStateFromSource(Source const& source) noexcept;
};
ETLState
tag_invoke(boost::json::value_to_tag<ETLState>, boost::json::value const& jv);
} // namespace etl

View File

@@ -87,9 +87,33 @@ LoadBalancer::LoadBalancer(
{
std::unique_ptr<Source> source = make_Source(entry, ioc, backend, subscriptions, validatedLedgers, *this);
// checking etl node validity
auto const state = ETLState::fetchETLStateFromSource(*source);
if (!state.networkID)
{
LOG(log_.error()) << "Failed to fetch ETL state from source = " << source->toString()
<< " Please check the configuration and network";
throw std::logic_error("ETL node not available");
}
if (etlState_ && etlState_->networkID != state.networkID)
{
LOG(log_.error()) << "ETL sources must be on the same network. "
<< "Source network id = " << *(state.networkID)
<< " does not match others network id = " << *(etlState_->networkID);
throw std::logic_error("ETL nodes are not in the same network");
}
etlState_ = state;
sources_.push_back(std::move(source));
LOG(log_.info()) << "Added etl source - " << sources_.back()->toString();
}
if (sources_.empty())
{
LOG(log_.error()) << "No ETL sources configured. Please check the configuration";
throw std::logic_error("No ETL sources configured");
}
}
LoadBalancer::~LoadBalancer()
@@ -152,7 +176,7 @@ LoadBalancer::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObje
std::optional<boost::json::object>
LoadBalancer::forwardToRippled(
boost::json::object const& request,
std::string const& clientIp,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const
{
srand(static_cast<unsigned>(time(0)));
@@ -245,4 +269,12 @@ LoadBalancer::execute(Func f, uint32_t ledgerSequence)
}
return true;
}
ETLState
LoadBalancer::getETLState() const noexcept
{
assert(etlState_); // etlState_ is set in the constructor
return *etlState_;
}
} // namespace etl

View File

@@ -21,6 +21,7 @@
#include <data/BackendInterface.h>
#include <etl/ETLHelpers.h>
#include <etl/ETLState.h>
#include <feed/SubscriptionManager.h>
#include <util/config/Config.h>
#include <util/log/Logger.h>
@@ -59,6 +60,7 @@ private:
util::Logger log_{"ETL"};
std::vector<std::unique_ptr<Source>> sources_;
std::optional<ETLState> etlState_;
std::uint32_t downloadRanges_ =
DEFAULT_DOWNLOAD_RANGES; /*< The number of markers to use when downloading intial ledger */
@@ -164,13 +166,21 @@ public:
* @brief Forward a JSON RPC request to a randomly selected rippled node.
*
* @param request JSON-RPC request to forward
* @param clientIp The IP address of the peer
* @param clientIp The IP address of the peer, if known
* @param yield The coroutine context
* @return Response received from rippled node as JSON object on success; nullopt on failure
*/
std::optional<boost::json::object>
forwardToRippled(boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context yield)
const;
forwardToRippled(
boost::json::object const& request,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const;
/**
* @brief Return state of ETL nodes.
*/
ETLState
getETLState() const noexcept;
private:
/**

View File

@@ -123,18 +123,23 @@ ProbingSource::fetchLedger(uint32_t sequence, bool getObjects, bool getObjectNei
std::optional<boost::json::object>
ProbingSource::forwardToRippled(
boost::json::object const& request,
std::string const& clientIp,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const
{
if (!currentSrc_)
return {};
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::string const& clientIp,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const
{
if (!currentSrc_)

View File

@@ -105,8 +105,10 @@ public:
fetchLedger(uint32_t sequence, bool getObjects = true, bool getObjectNeighbors = false) override;
std::optional<boost::json::object>
forwardToRippled(boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context yield)
const override;
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;
@@ -115,7 +117,7 @@ private:
std::optional<boost::json::object>
requestFromRippled(
boost::json::object const& request,
std::string const& clientIp,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const override;
SourceHooks

View File

@@ -51,6 +51,7 @@ class SubscriptionManager;
// things into the base class instead.
namespace etl {
class ProbingSource;
/**
@@ -124,13 +125,15 @@ public:
* @brief Forward a request to rippled.
*
* @param request The request to forward
* @param clientIp IP of the client forwarding this request
* @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
*/
virtual std::optional<boost::json::object>
forwardToRippled(boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context yield)
const = 0;
forwardToRippled(
boost::json::object const& request,
std::optional<std::string> const& forwardToRippledclientIp,
boost::asio::yield_context yield) const = 0;
/**
* @return A token that uniquely identifies this source instance.
@@ -162,7 +165,7 @@ private:
virtual std::optional<boost::json::object>
requestFromRippled(
boost::json::object const& request,
std::string const& clientIp,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const = 0;
};
@@ -303,18 +306,12 @@ public:
std::optional<boost::json::object>
requestFromRippled(
boost::json::object const& request,
std::string const& clientIp,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const override
{
LOG(log_.trace()) << "Attempting to forward request to tx. "
<< "request = " << boost::json::serialize(request);
LOG(log_.trace()) << "Attempting to forward request to tx. Request = " << boost::json::serialize(request);
boost::json::object response;
if (!isConnected())
{
LOG(log_.error()) << "Attempted to proxy but failed to connect to tx";
return {};
}
namespace beast = boost::beast;
namespace http = boost::beast::http;
@@ -339,12 +336,15 @@ public:
if (ec)
return {};
// Set a decorator to change the User-Agent of the handshake and to tell rippled to charge the client IP for
// RPC resources. See "secure_gateway" in
// if client ip is know, change the User-Agent of the handshake and to tell rippled to charge the client IP
// for RPC resources. See "secure_gateway" in
// https://github.com/ripple/rippled/blob/develop/cfg/rippled-example.cfg
// TODO: user-agent can be clio-[version]
ws->set_option(websocket::stream_base::decorator([&clientIp](websocket::request_type& req) {
req.set(http::field::user_agent, std::string(BOOST_BEAST_VERSION_STRING) + " websocket-client-coro");
req.set(http::field::forwarded, "for=" + clientIp);
if (clientIp)
req.set(http::field::forwarded, "for=" + *clientIp);
}));
ws->async_handshake(ip_, "/", yield[ec]);
@@ -536,8 +536,10 @@ public:
}
std::optional<boost::json::object>
forwardToRippled(boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context yield)
const override
forwardToRippled(
boost::json::object const& request,
std::optional<std::string> const& clientIp,
boost::asio::yield_context yield) const override
{
if (auto resp = forwardCache_.get(request); resp)
{

View File

@@ -38,7 +38,7 @@ ForwardCache::freshen()
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, {}, yield);
auto resp = source_.requestFromRippled(request, std::nullopt, yield);
if (!resp || resp->contains("error"))
resp = {};

View File

@@ -93,8 +93,8 @@ getDeliveredAmount(
// then its absence indicates that the amount delivered is listed in the
// Amount field. DeliveredAmount went live January 24, 2014.
// 446000000 is in Feb 2014, well after DeliveredAmount went live
static constexpr std::uint32_t FIRST_LEDGER_WITH_DELIVERED_AMOUNT = 4594095;
static constexpr std::uint32_t DELIVERED_AMOUNT_LIVE_DATE = 446000000;
static std::uint32_t constexpr FIRST_LEDGER_WITH_DELIVERED_AMOUNT = 4594095;
static std::uint32_t constexpr DELIVERED_AMOUNT_LIVE_DATE = 446000000;
if (ledgerSequence >= FIRST_LEDGER_WITH_DELIVERED_AMOUNT || date > DELIVERED_AMOUNT_LIVE_DATE)
{
return txn->getFieldAmount(ripple::sfAmount);
@@ -198,7 +198,7 @@ toJson(ripple::STBase const& obj)
}
std::pair<boost::json::object, boost::json::object>
toExpandedJson(data::TransactionAndMetadata const& blobs, NFTokenjson nftEnabled)
toExpandedJson(data::TransactionAndMetadata const& blobs, NFTokenjson nftEnabled, std::optional<uint16_t> networkId)
{
auto [txn, meta] = deserializeTxPlusMeta(blobs, blobs.ledgerSequence);
auto txnJson = toJson(*txn);
@@ -218,9 +218,32 @@ toExpandedJson(data::TransactionAndMetadata const& blobs, NFTokenjson nftEnabled
}
}
if (networkId)
{
// networkId is available, insert ctid field to tx
if (auto const ctid = rpc::encodeCTID(meta->getLgrSeq(), meta->getIndex(), *networkId))
{
txnJson[JS(ctid)] = *ctid;
}
}
return {txnJson, metaJson};
}
std::optional<std::string>
encodeCTID(uint32_t ledgerSeq, uint16_t txnIndex, uint16_t networkId) noexcept
{
static uint32_t constexpr MAX_LEDGER_SEQ = 0x0FFF'FFFF;
if (ledgerSeq > MAX_LEDGER_SEQ)
return {};
static uint64_t constexpr CTID_PREFIX = 0xC000'0000;
uint64_t const ctidValue =
((CTID_PREFIX + static_cast<uint64_t>(ledgerSeq)) << 32) + (static_cast<uint64_t>(txnIndex) << 16) + networkId;
return {fmt::format("{:016X}", ctidValue)};
}
bool
insertDeliveredAmount(
boost::json::object& metaJson,
@@ -571,7 +594,7 @@ traverseOwnedNodes(
// Only reserve 2048 nodes when fetching all owned ledger objects. If there
// are more, then keys will allocate more memory, which is suboptimal, but
// should only occur occasionally.
static constexpr std::uint32_t MIN_NODES = 2048;
static std::uint32_t constexpr MIN_NODES = 2048;
keys.reserve(std::min(MIN_NODES, limit));
auto start = std::chrono::system_clock::now();
@@ -722,8 +745,8 @@ parseRippleLibSeed(boost::json::value const& value)
auto const result = ripple::decodeBase58Token(value.as_string().c_str(), ripple::TokenType::None);
static constexpr std::size_t SEED_SIZE = 18;
static constexpr std::array<std::uint8_t, 2> SEED_PREFIX = {0xE1, 0x4B};
static std::size_t constexpr SEED_SIZE = 18;
static std::array<std::uint8_t, 2> constexpr SEED_PREFIX = {0xE1, 0x4B};
if (result.size() == SEED_SIZE && static_cast<std::uint8_t>(result[0]) == SEED_PREFIX[0] &&
static_cast<std::uint8_t>(result[1]) == SEED_PREFIX[1])
return ripple::Seed(ripple::makeSlice(result.substr(2)));

View File

@@ -36,6 +36,7 @@
#include <ripple/protocol/STLedgerEntry.h>
#include <ripple/protocol/STTx.h>
#include <boost/regex.hpp>
#include <fmt/core.h>
namespace rpc {
@@ -62,8 +63,18 @@ deserializeTxPlusMeta(data::TransactionAndMetadata const& blobs);
std::pair<std::shared_ptr<ripple::STTx const>, std::shared_ptr<ripple::TxMeta const>>
deserializeTxPlusMeta(data::TransactionAndMetadata const& blobs, std::uint32_t seq);
/**
* @brief Convert a TransactionAndMetadata to two JSON objects.
*
* @param blobs The TransactionAndMetadata to convert.
* @param nftEnabled Whether to include NFT information in the JSON.
* @param networkId The network ID to use for ctid, not include ctid if nullopt.
*/
std::pair<boost::json::object, boost::json::object>
toExpandedJson(data::TransactionAndMetadata const& blobs, NFTokenjson nftEnabled = NFTokenjson::DISABLE);
toExpandedJson(
data::TransactionAndMetadata const& blobs,
NFTokenjson nftEnabled = NFTokenjson::DISABLE,
std::optional<uint16_t> networkId = std::nullopt);
bool
insertDeliveredAmount(
@@ -225,6 +236,47 @@ isAmendmentEnabled(
uint32_t seq,
ripple::uint256 amendmentId);
std::optional<std::string>
encodeCTID(uint32_t ledgerSeq, uint16_t txnIndex, uint16_t networkId) noexcept;
template <typename T>
inline std::optional<std::tuple<uint32_t, uint16_t, uint16_t>>
decodeCTID(T const ctid) noexcept
{
auto const getCTID64 = [](T const ctid) noexcept -> std::optional<uint64_t> {
if constexpr (std::is_convertible_v<T, std::string>)
{
std::string const ctidString(ctid);
static std::size_t constexpr CTID_STRING_LENGTH = 16;
if (ctidString.length() != CTID_STRING_LENGTH)
return {};
if (!boost::regex_match(ctidString, boost::regex("^[0-9A-F]+$")))
return {};
return std::stoull(ctidString, nullptr, 16);
}
if constexpr (std::is_same_v<T, uint64_t>)
return ctid;
return {};
};
auto const ctidValue = getCTID64(ctid).value_or(0);
static uint64_t constexpr CTID_PREFIX = 0xC000'0000'0000'0000ULL;
static uint64_t constexpr CTID_PREFIX_MASK = 0xF000'0000'0000'0000ULL;
if ((ctidValue & CTID_PREFIX_MASK) != CTID_PREFIX)
return {};
uint32_t const ledgerSeq = (ctidValue >> 32) & 0xFFFF'FFFUL;
uint16_t const txnIndex = (ctidValue >> 16) & 0xFFFFU;
uint16_t const networkId = ctidValue & 0xFFFFU;
return {{ledgerSeq, txnIndex, networkId}};
}
template <class T>
void
logDuration(web::Context const& ctx, T const& dur)
@@ -232,7 +284,7 @@ logDuration(web::Context const& ctx, T const& dur)
using boost::json::serialize;
static util::Logger const log{"RPC"};
static constexpr std::int64_t DURATION_ERROR_THRESHOLD_SECONDS = 10;
static std::int64_t constexpr DURATION_ERROR_THRESHOLD_SECONDS = 10;
auto const millis = std::chrono::duration_cast<std::chrono::milliseconds>(dur).count();
auto const seconds = std::chrono::duration_cast<std::chrono::seconds>(dur).count();

View File

@@ -88,7 +88,7 @@ ProductionHandlerProvider::ProductionHandlerProvider(
{"random", {RandomHandler{}}},
{"server_info", {ServerInfoHandler{backend, subscriptionManager, balancer, etl, counters}}},
{"transaction_entry", {TransactionEntryHandler{backend}}},
{"tx", {TxHandler{backend}}},
{"tx", {TxHandler{backend, etl}}},
{"subscribe", {SubscribeHandler{backend, subscriptionManager}}},
{"unsubscribe", {UnsubscribeHandler{backend, subscriptionManager}}},
{"version", {VersionHandler{config}}},

View File

@@ -1,123 +0,0 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <rpc/handlers/Tx.h>
namespace rpc {
TxHandler::Result
TxHandler::process(Input input, Context const& ctx) const
{
static auto constexpr maxLedgerRange = 1000u;
auto const rangeSupplied = input.minLedger && input.maxLedger;
if (rangeSupplied)
{
if (*input.minLedger > *input.maxLedger)
return Error{Status{RippledError::rpcINVALID_LGR_RANGE}};
if (*input.maxLedger - *input.minLedger > maxLedgerRange)
return Error{Status{RippledError::rpcEXCESSIVE_LGR_RANGE}};
}
auto output = TxHandler::Output{.apiVersion = ctx.apiVersion};
auto const dbResponse =
sharedPtrBackend_->fetchTransaction(ripple::uint256{std::string_view(input.transaction)}, ctx.yield);
if (!dbResponse)
{
if (rangeSupplied)
{
auto const range = sharedPtrBackend_->fetchLedgerRange();
auto const searchedAll = range->maxSequence >= *input.maxLedger && range->minSequence <= *input.minLedger;
boost::json::object extra;
extra["searched_all"] = searchedAll;
return Error{Status{RippledError::rpcTXN_NOT_FOUND, std::move(extra)}};
}
return Error{Status{RippledError::rpcTXN_NOT_FOUND}};
}
if (!input.binary)
{
auto const [txn, meta] = toExpandedJson(*dbResponse, NFTokenjson::ENABLE);
output.tx = txn;
output.meta = meta;
}
else
{
output.txStr = ripple::strHex(dbResponse->transaction);
output.metaStr = ripple::strHex(dbResponse->metadata);
output.hash = std::move(input.transaction);
}
output.date = dbResponse->date;
output.ledgerIndex = dbResponse->ledgerSequence;
return output;
}
void
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, TxHandler::Output const& output)
{
auto obj = boost::json::object{};
if (output.tx)
{
obj = *output.tx;
obj[JS(meta)] = *output.meta;
}
else
{
obj[JS(meta)] = *output.metaStr;
obj[JS(tx)] = *output.txStr;
obj[JS(hash)] = output.hash;
}
obj[JS(validated)] = output.validated;
obj[JS(date)] = output.date;
obj[JS(ledger_index)] = output.ledgerIndex;
if (output.apiVersion < 2u)
obj[JS(inLedger)] = output.ledgerIndex;
jv = std::move(obj);
}
TxHandler::Input
tag_invoke(boost::json::value_to_tag<TxHandler::Input>, boost::json::value const& jv)
{
auto input = TxHandler::Input{};
auto const& jsonObject = jv.as_object();
input.transaction = jv.at(JS(transaction)).as_string().c_str();
if (jsonObject.contains(JS(binary)))
input.binary = jv.at(JS(binary)).as_bool();
if (jsonObject.contains(JS(min_ledger)))
input.minLedger = jv.at(JS(min_ledger)).as_int64();
if (jsonObject.contains(JS(max_ledger)))
input.maxLedger = jv.at(JS(max_ledger)).as_int64();
return input;
}
} // namespace rpc

View File

@@ -20,20 +20,18 @@
#pragma once
#include <data/BackendInterface.h>
#include <etl/ETLService.h>
#include <rpc/RPCHelpers.h>
#include <rpc/common/Types.h>
#include <rpc/common/Validators.h>
namespace rpc {
/**
* @brief The tx method retrieves information on a single transaction, by its identifying hash.
*
* For more details see: https://xrpl.org/tx.html
*/
class TxHandler
template <typename ETLServiceType>
class BaseTxHandler
{
std::shared_ptr<BackendInterface> sharedPtrBackend_;
std::shared_ptr<ETLServiceType const> etl_;
public:
struct Output
@@ -45,13 +43,15 @@ public:
std::optional<boost::json::object> tx{};
std::optional<std::string> metaStr{};
std::optional<std::string> txStr{};
std::optional<std::string> ctid{}; // ctid when binary=true
uint32_t apiVersion = 0u;
bool validated = true;
};
struct Input
{
std::string transaction;
std::optional<std::string> transaction;
std::optional<std::string> ctid;
bool binary = false;
std::optional<uint32_t> minLedger;
std::optional<uint32_t> maxLedger;
@@ -59,7 +59,10 @@ public:
using Result = HandlerReturnType<Output>;
TxHandler(std::shared_ptr<BackendInterface> const& sharedPtrBackend) : sharedPtrBackend_(sharedPtrBackend)
BaseTxHandler(
std::shared_ptr<BackendInterface> const& sharedPtrBackend,
std::shared_ptr<ETLServiceType const> const& etl)
: sharedPtrBackend_(sharedPtrBackend), etl_(etl)
{
}
@@ -67,23 +70,191 @@ public:
spec([[maybe_unused]] uint32_t apiVersion)
{
static const RpcSpec rpcSpec = {
{JS(transaction), validation::Required{}, validation::Uint256HexStringValidator},
{JS(transaction), validation::Uint256HexStringValidator},
{JS(binary), validation::Type<bool>{}},
{JS(min_ledger), validation::Type<uint32_t>{}},
{JS(max_ledger), validation::Type<uint32_t>{}},
{JS(ctid), validation::Type<std::string>{}},
};
return rpcSpec;
}
Result
process(Input input, Context const& ctx) const;
process(Input input, Context const& ctx) const
{
if (input.ctid && input.transaction) // ambiguous identifier
return Error{Status{RippledError::rpcINVALID_PARAMS}};
if (!input.ctid && !input.transaction) // at least one identifier must be supplied
return Error{Status{RippledError::rpcINVALID_PARAMS}};
static auto constexpr maxLedgerRange = 1000u;
auto const rangeSupplied = input.minLedger && input.maxLedger;
if (rangeSupplied)
{
if (*input.minLedger > *input.maxLedger)
return Error{Status{RippledError::rpcINVALID_LGR_RANGE}};
if (*input.maxLedger - *input.minLedger > maxLedgerRange)
return Error{Status{RippledError::rpcEXCESSIVE_LGR_RANGE}};
}
auto const currentNetId = etl_->getETLState().networkID;
std::optional<data::TransactionAndMetadata> dbResponse;
if (input.ctid)
{
auto const ctid = rpc::decodeCTID(*input.ctid);
if (!ctid)
return Error{Status{RippledError::rpcINVALID_PARAMS}};
auto const [lgrSeq, txnIdx, netId] = *ctid;
// when current network id is available, let us check the network id from parameter
if (currentNetId && netId != *currentNetId)
{
return Error{Status{
RippledError::rpcWRONG_NETWORK,
fmt::format(
"Wrong network. You should submit this request to a node running on NetworkID: {}", netId)}};
}
dbResponse = fetchTxViaCtid(lgrSeq, txnIdx, ctx.yield);
}
else
{
dbResponse = sharedPtrBackend_->fetchTransaction(ripple::uint256{input.transaction->c_str()}, ctx.yield);
}
auto output = BaseTxHandler::Output{.apiVersion = ctx.apiVersion};
if (!dbResponse)
{
if (rangeSupplied && input.transaction) // ranges not for ctid
{
auto const range = sharedPtrBackend_->fetchLedgerRange();
auto const searchedAll =
range->maxSequence >= *input.maxLedger && range->minSequence <= *input.minLedger;
boost::json::object extra;
extra["searched_all"] = searchedAll;
return Error{Status{RippledError::rpcTXN_NOT_FOUND, std::move(extra)}};
}
return Error{Status{RippledError::rpcTXN_NOT_FOUND}};
}
auto const [txn, meta] = toExpandedJson(*dbResponse, NFTokenjson::ENABLE, currentNetId);
if (!input.binary)
{
output.tx = txn;
output.meta = meta;
}
else
{
output.txStr = ripple::strHex(dbResponse->transaction);
output.metaStr = ripple::strHex(dbResponse->metadata);
// input.transaction might be not available, get hash via tx object
if (txn.contains(JS(hash)))
output.hash = txn.at(JS(hash)).as_string();
// append ctid here to mimic rippled 1.12 behavior: return ctid even binary=true
// rippled will change it in the future, ctid should be part of tx json which not available in binary mode
auto const txnIdx = boost::json::value_to<uint64_t>(meta.at("TransactionIndex"));
if (txnIdx <= 0xFFFFU && dbResponse->ledgerSequence < 0x0FFF'FFFFUL && currentNetId &&
*currentNetId <= 0xFFFFU)
{
output.ctid = rpc::encodeCTID(
dbResponse->ledgerSequence, static_cast<uint16_t>(txnIdx), static_cast<uint16_t>(*currentNetId));
}
}
output.date = dbResponse->date;
output.ledgerIndex = dbResponse->ledgerSequence;
return output;
}
private:
std::optional<data::TransactionAndMetadata>
fetchTxViaCtid(uint32_t ledgerSeq, uint32_t txId, boost::asio::yield_context yield) const
{
auto const txs = sharedPtrBackend_->fetchAllTransactionsInLedger(ledgerSeq, yield);
for (auto const& tx : txs)
{
auto const [txn, meta] = deserializeTxPlusMeta(tx, ledgerSeq);
if (meta->getIndex() == txId)
return tx;
}
return std::nullopt;
}
friend void
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, Output const& output);
tag_invoke(boost::json::value_from_tag, boost::json::value& jv, Output const& output)
{
auto obj = boost::json::object{};
if (output.tx)
{
obj = *output.tx;
obj[JS(meta)] = *output.meta;
}
else
{
obj[JS(meta)] = *output.metaStr;
obj[JS(tx)] = *output.txStr;
obj[JS(hash)] = output.hash;
}
obj[JS(validated)] = output.validated;
obj[JS(date)] = output.date;
obj[JS(ledger_index)] = output.ledgerIndex;
if (output.apiVersion < 2u)
obj[JS(inLedger)] = output.ledgerIndex;
if (output.ctid)
obj[JS(ctid)] = *output.ctid;
jv = std::move(obj);
}
friend Input
tag_invoke(boost::json::value_to_tag<Input>, boost::json::value const& jv);
tag_invoke(boost::json::value_to_tag<Input>, boost::json::value const& jv)
{
auto input = BaseTxHandler::Input{};
auto const& jsonObject = jv.as_object();
if (jsonObject.contains(JS(transaction)))
input.transaction = jv.at(JS(transaction)).as_string().c_str();
if (jsonObject.contains(JS(ctid)))
input.ctid = jv.at(JS(ctid)).as_string().c_str();
if (jsonObject.contains(JS(binary)))
input.binary = jv.at(JS(binary)).as_bool();
if (jsonObject.contains(JS(min_ledger)))
input.minLedger = jv.at(JS(min_ledger)).as_int64();
if (jsonObject.contains(JS(max_ledger)))
input.maxLedger = jv.at(JS(max_ledger)).as_int64();
return input;
}
};
/**
* @brief The tx method retrieves information on a single transaction, by its identifying hash.
*
* For more details see: https://xrpl.org/tx.html
*/
using TxHandler = BaseTxHandler<etl::ETLService>;
} // namespace rpc

View File

@@ -0,0 +1,72 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#include <etl/ETLState.h>
#include <util/MockSource.h>
#include <boost/json.hpp>
#include <gtest/gtest.h>
#include <util/Fixtures.h>
namespace json = boost::json;
using namespace util;
using namespace testing;
struct ETLStateTest : public NoLoggerFixture
{
MockSource const source = MockSource{};
};
TEST_F(ETLStateTest, Error)
{
EXPECT_CALL(source, forwardToRippled).WillOnce(Return(std::nullopt));
auto const state = etl::ETLState::fetchETLStateFromSource(source);
EXPECT_FALSE(state.networkID.has_value());
}
TEST_F(ETLStateTest, NetworkIdValid)
{
auto const json = json::parse(
R"JSON({
"result": {
"info": {
"network_id": 12
}
}
})JSON");
EXPECT_CALL(source, forwardToRippled).WillOnce(Return(json.as_object()));
auto const state = etl::ETLState::fetchETLStateFromSource(source);
ASSERT_TRUE(state.networkID.has_value());
EXPECT_EQ(state.networkID.value(), 12);
}
TEST_F(ETLStateTest, NetworkIdInvalid)
{
auto const json = json::parse(
R"JSON({
"result": {
"info": {
"network_id2": 12
}
}
})JSON");
EXPECT_CALL(source, forwardToRippled).WillOnce(Return(json.as_object()));
auto const state = etl::ETLState::fetchETLStateFromSource(source);
EXPECT_FALSE(state.networkID.has_value());
}

View File

@@ -372,7 +372,8 @@ TEST_F(RPCForwardingProxyTest, ForwardCallsBalancerWithCorrectParams)
auto const forwarded = json::parse(R"({"test": true, "command": "submit"})");
ON_CALL(*rawBalancerPtr, forwardToRippled).WillByDefault(Return(std::make_optional<json::object>()));
EXPECT_CALL(*rawBalancerPtr, forwardToRippled(forwarded.as_object(), CLIENT_IP, _)).Times(1);
EXPECT_CALL(*rawBalancerPtr, forwardToRippled(forwarded.as_object(), std::make_optional<std::string>(CLIENT_IP), _))
.Times(1);
ON_CALL(*rawHandlerProviderPtr, contains).WillByDefault(Return(true));
EXPECT_CALL(*rawHandlerProviderPtr, contains(method)).Times(1);
@@ -402,7 +403,8 @@ TEST_F(RPCForwardingProxyTest, ForwardingFailYieldsErrorStatus)
auto const forwarded = json::parse(R"({"test": true, "command": "submit"})");
ON_CALL(*rawBalancerPtr, forwardToRippled).WillByDefault(Return(std::nullopt));
EXPECT_CALL(*rawBalancerPtr, forwardToRippled(forwarded.as_object(), CLIENT_IP, _)).Times(1);
EXPECT_CALL(*rawBalancerPtr, forwardToRippled(forwarded.as_object(), std::make_optional<std::string>(CLIENT_IP), _))
.Times(1);
ON_CALL(*rawHandlerProviderPtr, contains).WillByDefault(Return(true));
EXPECT_CALL(*rawHandlerProviderPtr, contains(method)).Times(1);

View File

@@ -307,3 +307,37 @@ TEST_F(RPCHelpersTest, TraverseOwnedNodesWithUnexistingIndexMarker)
});
ctx.run();
}
TEST_F(RPCHelpersTest, EncodeCTID)
{
auto const ctid = encodeCTID(0x1234, 0x67, 0x89);
ASSERT_TRUE(ctid);
EXPECT_EQ(*ctid, "C000123400670089");
EXPECT_FALSE(encodeCTID(0x1FFFFFFF, 0x67, 0x89));
}
TEST_F(RPCHelpersTest, DecodeCTIDString)
{
auto const ctid = decodeCTID("C000123400670089");
ASSERT_TRUE(ctid);
EXPECT_EQ(*ctid, std::make_tuple(0x1234, 0x67, 0x89));
EXPECT_FALSE(decodeCTID("F000123400670089"));
EXPECT_FALSE(decodeCTID("F0001234006700"));
EXPECT_FALSE(decodeCTID("F000123400*700"));
}
TEST_F(RPCHelpersTest, DecodeCTIDInt)
{
uint64_t ctidStr = 0xC000123400670089;
auto const ctid = decodeCTID(ctidStr);
ASSERT_TRUE(ctid);
EXPECT_EQ(*ctid, std::make_tuple(0x1234, 0x67, 0x89));
ctidStr = 0xF000123400670089;
EXPECT_FALSE(decodeCTID(ctidStr));
}
TEST_F(RPCHelpersTest, DecodeInvalidCTID)
{
EXPECT_FALSE(decodeCTID('c'));
EXPECT_FALSE(decodeCTID(true));
}

View File

@@ -35,7 +35,6 @@ constexpr static auto CLIENTIP = "1.1.1.1";
class RPCServerInfoHandlerTest : public HandlerBaseTest,
public MockLoadBalancerTest,
public MockSubscriptionManagerTest,
public MockETLServiceTest,
public MockCountersTest
{
protected:
@@ -45,7 +44,6 @@ protected:
HandlerBaseTest::SetUp();
MockLoadBalancerTest::SetUp();
MockSubscriptionManagerTest::SetUp();
MockETLServiceTest::SetUp();
MockCountersTest::SetUp();
rawBackendPtr = dynamic_cast<MockBackend*>(mockBackendPtr.get());
@@ -58,7 +56,6 @@ protected:
TearDown() override
{
MockCountersTest::TearDown();
MockETLServiceTest::TearDown();
MockSubscriptionManagerTest::TearDown();
MockLoadBalancerTest::TearDown();
HandlerBaseTest::TearDown();

View File

@@ -28,13 +28,16 @@ using namespace rpc;
namespace json = boost::json;
using namespace testing;
using TestTxHandler = BaseTxHandler<MockETLService>;
auto constexpr static TXNID = "05FB0EB4B899F056FA095537C5817163801F544BAFCEA39C995D76DB4D16F9DD";
auto constexpr static NFTID = "05FB0EB4B899F056FA095537C5817163801F544BAFCEA39C995D76DB4D16F9DF";
auto constexpr static NFTID2 = "05FB0EB4B899F056FA095537C5817163801F544BAFCEA39C995D76DB4D16F9DA";
auto constexpr static ACCOUNT = "rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn";
auto constexpr static ACCOUNT2 = "rLEsXccBGNR3UPuPu2hUXPjziKC3qKSBun";
auto constexpr static CURRENCY = "0158415500000000C1F76FF6ECB0BAC600000000";
constexpr static auto CTID = "C002807000010002"; // seq 163952 txindex 1 netid 2
constexpr static auto SEQ_FROM_CTID = 163952;
auto constexpr static DEFAULT_OUT = R"({
"Account": "rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn",
"Fee": "2",
@@ -79,7 +82,7 @@ class RPCTxTest : public HandlerBaseTest
TEST_F(RPCTxTest, ExcessiveLgrRange)
{
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -100,7 +103,7 @@ TEST_F(RPCTxTest, ExcessiveLgrRange)
TEST_F(RPCTxTest, InvalidLgrRange)
{
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -122,11 +125,15 @@ TEST_F(RPCTxTest, TxnNotFound)
{
auto const rawBackendPtr = dynamic_cast<MockBackend*>(mockBackendPtr.get());
ASSERT_NE(rawBackendPtr, nullptr);
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _))
.WillByDefault(Return(std::optional<TransactionAndMetadata>{}));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _))
.WillOnce(Return(std::optional<TransactionAndMetadata>{}));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -148,11 +155,15 @@ TEST_F(RPCTxTest, TxnNotFoundInGivenRangeSearchAllFalse)
ASSERT_NE(rawBackendPtr, nullptr);
mockBackendPtr->updateRange(10); // min
mockBackendPtr->updateRange(30); // max
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _))
.WillByDefault(Return(std::optional<TransactionAndMetadata>{}));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _))
.WillOnce(Return(std::optional<TransactionAndMetadata>{}));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -177,11 +188,15 @@ TEST_F(RPCTxTest, TxnNotFoundInGivenRangeSearchAllTrue)
ASSERT_NE(rawBackendPtr, nullptr);
mockBackendPtr->updateRange(1); // min
mockBackendPtr->updateRange(1000); // max
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _))
.WillByDefault(Return(std::optional<TransactionAndMetadata>{}));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _))
.WillOnce(Return(std::optional<TransactionAndMetadata>{}));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -200,6 +215,39 @@ TEST_F(RPCTxTest, TxnNotFoundInGivenRangeSearchAllTrue)
});
}
// when ledger range and ctid are provided, searched_all should not be present, because the seq is specified in ctid
TEST_F(RPCTxTest, CtidNotFoundSearchAllFalse)
{
auto const rawBackendPtr = dynamic_cast<MockBackend*>(mockBackendPtr.get());
ASSERT_NE(rawBackendPtr, nullptr);
mockBackendPtr->updateRange(1); // min
mockBackendPtr->updateRange(1000); // max
EXPECT_CALL(*rawBackendPtr, fetchAllTransactionsInLedger(SEQ_FROM_CTID, _))
.WillOnce(Return(std::vector<TransactionAndMetadata>{}));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{2}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"ctid": "{}",
"min_ledger": 1,
"max_ledger": 1000
}})",
CTID));
auto const output = handler.process(req, Context{yield});
ASSERT_FALSE(output);
auto const err = rpc::makeError(output.error());
EXPECT_EQ(err.at("error").as_string(), "txnNotFound");
EXPECT_EQ(err.at("error_message").as_string(), "Transaction not found.");
EXPECT_FALSE(err.contains("searched_all"));
});
}
TEST_F(RPCTxTest, DefaultParameter_API_v1)
{
auto const rawBackendPtr = dynamic_cast<MockBackend*>(mockBackendPtr.get());
@@ -212,11 +260,14 @@ TEST_F(RPCTxTest, DefaultParameter_API_v1)
tx.date = 123456;
tx.ledgerSequence = 100;
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillByDefault(Return(tx));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -244,11 +295,14 @@ TEST_F(RPCTxTest, DefaultParameter_API_v2)
tx.date = 123456;
tx.ledgerSequence = 100;
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillByDefault(Return(tx));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -267,7 +321,7 @@ TEST_F(RPCTxTest, ReturnBinary)
auto constexpr static OUT = R"({
"meta": "201C00000064F8E311006FE864D50AA87BEE5380000158415500000000C1F76FF6ECB0BAC6000000004B4E9C06F24296074F7BC48F92A97916C6DC5EA96540000000000000C8E1E1F1031000",
"tx": "120007240000006464400000000000012C65D5071AFD498D00000158415500000000C1F76FF6ECB0BAC600000000D31252CF902EF8DD8451243869B38667CBD89DF368400000000000000273047465737481144B4E9C06F24296074F7BC48F92A97916C6DC5EA9",
"hash": "05FB0EB4B899F056FA095537C5817163801F544BAFCEA39C995D76DB4D16F9DD",
"hash": "2E2FBAAFF767227FE4381C4BE9855986A6B9F96C62F6E443731AB36F7BBB8A08",
"date": 123456,
"ledger_index": 100,
"inLedger": 100,
@@ -281,10 +335,57 @@ TEST_F(RPCTxTest, ReturnBinary)
CreateCreateOfferTransactionObject(ACCOUNT, 2, 100, CURRENCY, ACCOUNT2, 200, 300).getSerializer().peekData();
tx.date = 123456;
tx.ledgerSequence = 100;
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillByDefault(Return(tx));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
"transaction": "{}",
"binary": true
}})",
TXNID));
auto const output = handler.process(req, Context{yield});
ASSERT_TRUE(output);
EXPECT_EQ(*output, json::parse(OUT));
});
}
// mimic 1.12 rippled, return ctid when binary is true. This will be changed on rippled.
TEST_F(RPCTxTest, ReturnBinaryWithCTID)
{
// Note: `inLedger` is API v1 only. See DefaultOutput_*
auto constexpr static OUT = R"({
"meta": "201C00000064F8E311006FE864D50AA87BEE5380000158415500000000C1F76FF6ECB0BAC6000000004B4E9C06F24296074F7BC48F92A97916C6DC5EA96540000000000000C8E1E1F1031000",
"tx": "120007240000006464400000000000012C65D5071AFD498D00000158415500000000C1F76FF6ECB0BAC600000000D31252CF902EF8DD8451243869B38667CBD89DF368400000000000000273047465737481144B4E9C06F24296074F7BC48F92A97916C6DC5EA9",
"hash": "2E2FBAAFF767227FE4381C4BE9855986A6B9F96C62F6E443731AB36F7BBB8A08",
"date": 123456,
"ledger_index": 100,
"inLedger": 100,
"ctid": "C000006400640002",
"validated": true
})";
auto const rawBackendPtr = dynamic_cast<MockBackend*>(mockBackendPtr.get());
ASSERT_NE(rawBackendPtr, nullptr);
TransactionAndMetadata tx;
tx.metadata = CreateMetaDataForCreateOffer(CURRENCY, ACCOUNT, 100, 200, 300).getSerializer().peekData();
tx.transaction =
CreateCreateOfferTransactionObject(ACCOUNT, 2, 100, CURRENCY, ACCOUNT2, 200, 300).getSerializer().peekData();
tx.date = 123456;
tx.ledgerSequence = 100;
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{.networkID = 2}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -363,10 +464,14 @@ TEST_F(RPCTxTest, MintNFT)
ASSERT_NE(rawBackendPtr, nullptr);
tx.date = 123456;
tx.ledgerSequence = 100;
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillByDefault(Return(tx));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -386,10 +491,14 @@ TEST_F(RPCTxTest, NFTAcceptOffer)
ASSERT_NE(rawBackendPtr, nullptr);
tx.date = 123456;
tx.ledgerSequence = 100;
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillByDefault(Return(tx));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -410,10 +519,14 @@ TEST_F(RPCTxTest, NFTCancelOffer)
ASSERT_NE(rawBackendPtr, nullptr);
tx.date = 123456;
tx.ledgerSequence = 100;
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillByDefault(Return(tx));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this, &ids](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -442,10 +555,14 @@ TEST_F(RPCTxTest, NFTCreateOffer)
ASSERT_NE(rawBackendPtr, nullptr);
tx.date = 123456;
tx.ledgerSequence = 100;
ON_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillByDefault(Return(tx));
EXPECT_CALL(*rawBackendPtr, fetchTransaction).Times(1);
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TxHandler{mockBackendPtr}};
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
@@ -457,3 +574,251 @@ TEST_F(RPCTxTest, NFTCreateOffer)
EXPECT_TRUE(output->at("meta").at("offer_id").as_string() == NFTID2);
});
}
TEST_F(RPCTxTest, CTIDAndTransactionBothProvided)
{
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
"transaction": "{}",
"ctid": "{}"
}})",
TXNID,
CTID));
auto const output = handler.process(req, Context{yield});
ASSERT_FALSE(output);
auto const err = rpc::makeError(output.error());
EXPECT_EQ(err.at("error").as_string(), "invalidParams");
EXPECT_EQ(err.at("error_message").as_string(), "Invalid parameters.");
});
}
TEST_F(RPCTxTest, CTIDAndTransactionBothNotProvided)
{
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(R"({ "command": "tx"})");
auto const output = handler.process(req, Context{yield});
ASSERT_FALSE(output);
auto const err = rpc::makeError(output.error());
EXPECT_EQ(err.at("error").as_string(), "invalidParams");
EXPECT_EQ(err.at("error_message").as_string(), "Invalid parameters.");
});
}
TEST_F(RPCTxTest, CTIDInvalidType)
{
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(R"({ "command": "tx", "ctid": 123})");
auto const output = handler.process(req, Context{yield});
ASSERT_FALSE(output);
auto const err = rpc::makeError(output.error());
EXPECT_EQ(err.at("error").as_string(), "invalidParams");
EXPECT_EQ(err.at("error_message").as_string(), "Invalid parameters.");
});
}
TEST_F(RPCTxTest, CTIDInvalidString)
{
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{.networkID = 5}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(R"({ "command": "tx", "ctid": "B002807000010002"})");
auto const output = handler.process(req, Context{yield});
ASSERT_FALSE(output);
auto const err = rpc::makeError(output.error());
EXPECT_EQ(err.at("error").as_string(), "invalidParams");
EXPECT_EQ(err.at("error_message").as_string(), "Invalid parameters.");
});
}
TEST_F(RPCTxTest, CTIDNotMatch)
{
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{.networkID = 5}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
"ctid": "{}"
}})",
CTID));
auto const output = handler.process(req, Context{yield});
ASSERT_FALSE(output);
auto const err = rpc::makeError(output.error());
EXPECT_EQ(err.at("error_code").as_uint64(), 4);
EXPECT_EQ(
err.at("error_message").as_string(),
"Wrong network. You should submit this request to a node running on NetworkID: 2");
});
}
TEST_F(RPCTxTest, ReturnCTIDForTxInput)
{
auto constexpr static OUT = R"({
"Account":"rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn",
"Fee":"2",
"Sequence":100,
"SigningPubKey":"74657374",
"TakerGets":
{
"currency":"0158415500000000C1F76FF6ECB0BAC600000000",
"issuer":"rLEsXccBGNR3UPuPu2hUXPjziKC3qKSBun",
"value":"200"
},
"ctid":"C000006400640002",
"TakerPays":"300",
"TransactionType":"OfferCreate",
"hash":"2E2FBAAFF767227FE4381C4BE9855986A6B9F96C62F6E443731AB36F7BBB8A08",
"meta":
{
"AffectedNodes":
[
{
"CreatedNode":
{
"LedgerEntryType":"Offer",
"NewFields":
{
"TakerGets":"200",
"TakerPays":
{
"currency":"0158415500000000C1F76FF6ECB0BAC600000000",
"issuer":"rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn",
"value":"300"
}
}
}
}
],
"TransactionIndex":100,
"TransactionResult":"tesSUCCESS"
},
"date":123456,
"ledger_index":100,
"inLedger":100,
"validated": true
})";
auto const rawBackendPtr = dynamic_cast<MockBackend*>(mockBackendPtr.get());
TransactionAndMetadata tx;
tx.metadata = CreateMetaDataForCreateOffer(CURRENCY, ACCOUNT, 100, 200, 300).getSerializer().peekData();
tx.transaction =
CreateCreateOfferTransactionObject(ACCOUNT, 2, 100, CURRENCY, ACCOUNT2, 200, 300).getSerializer().peekData();
tx.date = 123456;
tx.ledgerSequence = 100;
EXPECT_CALL(*rawBackendPtr, fetchTransaction(ripple::uint256{TXNID}, _)).WillOnce(Return(tx));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{.networkID = 2}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
"transaction": "{}"
}})",
TXNID));
auto const output = handler.process(req, Context{yield});
ASSERT_TRUE(output);
EXPECT_EQ(*output, json::parse(OUT));
});
}
TEST_F(RPCTxTest, ViaCTID)
{
auto const static OUT = fmt::format(
R"({{
"Account":"rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn",
"Fee":"2",
"Sequence":100,
"SigningPubKey":"74657374",
"TakerGets":
{{
"currency":"0158415500000000C1F76FF6ECB0BAC600000000",
"issuer":"rLEsXccBGNR3UPuPu2hUXPjziKC3qKSBun",
"value":"200"
}},
"ctid":"{}",
"TakerPays":"300",
"TransactionType":"OfferCreate",
"hash":"2E2FBAAFF767227FE4381C4BE9855986A6B9F96C62F6E443731AB36F7BBB8A08",
"meta":
{{
"AffectedNodes":
[
{{
"CreatedNode":
{{
"LedgerEntryType":"Offer",
"NewFields":
{{
"TakerGets":"200",
"TakerPays":
{{
"currency":"0158415500000000C1F76FF6ECB0BAC600000000",
"issuer":"rf1BiGeXwwQoi8Z2ueFYTEXSwuJYfV2Jpn",
"value":"300"
}}
}}
}}
}}
],
"TransactionIndex":1,
"TransactionResult":"tesSUCCESS"
}},
"date":123456,
"ledger_index":{},
"inLedger":{},
"validated": true
}})",
CTID,
SEQ_FROM_CTID,
SEQ_FROM_CTID);
auto const rawBackendPtr = dynamic_cast<MockBackend*>(mockBackendPtr.get());
TransactionAndMetadata tx1;
tx1.metadata = CreateMetaDataForCreateOffer(CURRENCY, ACCOUNT, 1, 200, 300).getSerializer().peekData();
tx1.transaction =
CreateCreateOfferTransactionObject(ACCOUNT, 2, 100, CURRENCY, ACCOUNT2, 200, 300).getSerializer().peekData();
tx1.date = 123456;
tx1.ledgerSequence = SEQ_FROM_CTID;
TransactionAndMetadata tx2;
tx2.transaction = CreatePaymentTransactionObject(ACCOUNT, ACCOUNT2, 2, 3, 300).getSerializer().peekData();
tx2.metadata = CreatePaymentTransactionMetaObject(ACCOUNT, ACCOUNT2, 110, 30).getSerializer().peekData();
tx2.ledgerSequence = SEQ_FROM_CTID;
EXPECT_CALL(*rawBackendPtr, fetchAllTransactionsInLedger(SEQ_FROM_CTID, _)).WillOnce(Return(std::vector{tx1, tx2}));
auto const rawETLPtr = dynamic_cast<MockETLService*>(mockETLServicePtr.get());
ASSERT_NE(rawETLPtr, nullptr);
EXPECT_CALL(*rawETLPtr, getETLState).WillOnce(Return(etl::ETLState{.networkID = 2}));
runSpawn([this](auto yield) {
auto const handler = AnyHandler{TestTxHandler{mockBackendPtr, mockETLServicePtr}};
auto const req = json::parse(fmt::format(
R"({{
"command": "tx",
"ctid": "{}"
}})",
CTID));
auto const output = handler.process(req, Context{yield});
ASSERT_TRUE(output);
EXPECT_EQ(*output, json::parse(OUT));
});
}

View File

@@ -289,7 +289,7 @@ protected:
* @brief Fixture with an mock backend and an embedded boost::asio context
* Handler unittest base class
*/
struct HandlerBaseTest : public MockBackendTest, public SyncAsioContextTest
struct HandlerBaseTest : public MockBackendTest, public SyncAsioContextTest, public MockETLServiceTest
{
protected:
void
@@ -297,10 +297,13 @@ protected:
{
MockBackendTest::SetUp();
SyncAsioContextTest::SetUp();
MockETLServiceTest::SetUp();
}
void
TearDown() override
{
MockETLServiceTest::TearDown();
SyncAsioContextTest::TearDown();
MockBackendTest::TearDown();
}

View File

@@ -19,6 +19,8 @@
#pragma once
#include <etl/ETLState.h>
#include <boost/json.hpp>
#include <gmock/gmock.h>
@@ -31,4 +33,5 @@ struct MockETLService
MOCK_METHOD(std::uint32_t, lastPublishAgeSeconds, (), (const));
MOCK_METHOD(std::uint32_t, lastCloseAgeSeconds, (), (const));
MOCK_METHOD(bool, isAmendmentBlocked, (), (const));
MOCK_METHOD(etl::ETLState, getETLState, (), (const));
};

View File

@@ -39,6 +39,6 @@ struct MockLoadBalancer
MOCK_METHOD(
std::optional<boost::json::object>,
forwardToRippled,
(boost::json::object const&, std::string const&, boost::asio::yield_context),
(boost::json::object const&, std::optional<std::string> const&, boost::asio::yield_context),
(const));
};

View File

@@ -0,0 +1,52 @@
//------------------------------------------------------------------------------
/*
This file is part of clio: https://github.com/XRPLF/clio
Copyright (c) 2023, the clio developers.
Permission to use, copy, modify, and distribute this software for any
purpose with or without fee is hereby granted, provided that the above
copyright notice and this permission notice appear in all copies.
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
*/
//==============================================================================
#pragma once
#include <etl/Source.h>
#include <gmock/gmock.h>
class MockSource : public etl::Source
{
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(
std::optional<boost::json::object>,
forwardToRippled,
(boost::json::object const&, std::optional<std::string> const&, boost::asio::yield_context),
(const, override));
MOCK_METHOD(
std::optional<boost::json::object>,
requestFromRippled,
(boost::json::object const&, std::optional<std::string> const&, boost::asio::yield_context),
(const, override));
MOCK_METHOD(boost::uuids::uuid, token, (), (const, override));
};