mirror of
https://github.com/XRPLF/clio.git
synced 2025-11-28 07:35:52 +00:00
feat: New ETL by default (#2752)
This commit is contained in:
@@ -54,7 +54,7 @@ format:
|
||||
_help_max_pargs_hwrap:
|
||||
- If a positional argument group contains more than this many
|
||||
- arguments, then force it to a vertical layout.
|
||||
max_pargs_hwrap: 6
|
||||
max_pargs_hwrap: 5
|
||||
_help_max_rows_cmdline:
|
||||
- If a cmdline positional group consumes more than this many
|
||||
- lines without nesting, then invalidate the layout (and nest)
|
||||
|
||||
@@ -2,7 +2,6 @@ add_subdirectory(util)
|
||||
add_subdirectory(data)
|
||||
add_subdirectory(cluster)
|
||||
add_subdirectory(etl)
|
||||
add_subdirectory(etlng)
|
||||
add_subdirectory(feed)
|
||||
add_subdirectory(rpc)
|
||||
add_subdirectory(web)
|
||||
|
||||
@@ -5,10 +5,9 @@ target_link_libraries(
|
||||
clio_app
|
||||
PUBLIC clio_cluster
|
||||
clio_etl
|
||||
clio_etlng
|
||||
clio_feed
|
||||
clio_web
|
||||
clio_rpc
|
||||
clio_migration
|
||||
clio_rpc
|
||||
clio_web
|
||||
PRIVATE Boost::program_options
|
||||
)
|
||||
|
||||
@@ -27,9 +27,8 @@
|
||||
#include "data/LedgerCache.hpp"
|
||||
#include "etl/ETLService.hpp"
|
||||
#include "etl/LoadBalancer.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgers.hpp"
|
||||
#include "etlng/LoadBalancer.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManager.hpp"
|
||||
#include "migration/MigrationInspectorFactory.hpp"
|
||||
#include "rpc/Counters.hpp"
|
||||
@@ -103,7 +102,7 @@ ClioApplication::run(bool const useNgWebServer)
|
||||
// This is not the only io context in the application.
|
||||
boost::asio::io_context ioc{threads};
|
||||
|
||||
// Similarly we need a context to run ETLng on
|
||||
// Similarly we need a context to run ETL on
|
||||
// In the future we can remove the raw ioc and use ctx instead
|
||||
util::async::CoroExecutionContext ctx{threads};
|
||||
|
||||
@@ -142,19 +141,12 @@ ClioApplication::run(bool const useNgWebServer)
|
||||
// ETL uses the balancer to extract data.
|
||||
// The server uses the balancer to forward RPCs to a rippled node.
|
||||
// The balancer itself publishes to streams (transactions_proposed and accounts_proposed)
|
||||
auto balancer = [&] -> std::shared_ptr<etlng::LoadBalancerInterface> {
|
||||
if (config_.get<bool>("__ng_etl")) {
|
||||
return etlng::LoadBalancer::makeLoadBalancer(
|
||||
config_, ioc, backend, subscriptions, std::make_unique<util::MTRandomGenerator>(), ledgers
|
||||
);
|
||||
}
|
||||
|
||||
return etl::LoadBalancer::makeLoadBalancer(
|
||||
config_, ioc, backend, subscriptions, std::make_unique<util::MTRandomGenerator>(), ledgers
|
||||
);
|
||||
}();
|
||||
auto balancer = etl::LoadBalancer::makeLoadBalancer(
|
||||
config_, ioc, backend, subscriptions, std::make_unique<util::MTRandomGenerator>(), ledgers
|
||||
);
|
||||
|
||||
// ETL is responsible for writing and publishing to streams. In read-only mode, ETL only publishes
|
||||
// TODO: don't use ioc (Publisher uses it)
|
||||
auto etl = etl::ETLService::makeETLService(config_, ioc, ctx, backend, subscriptions, balancer, ledgers);
|
||||
|
||||
auto workQueue = rpc::WorkQueue::makeWorkQueue(config_);
|
||||
|
||||
@@ -20,8 +20,8 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "util/CoroutineGroup.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
@@ -78,8 +78,8 @@ public:
|
||||
static std::function<void(boost::asio::yield_context)>
|
||||
makeOnStopCallback(
|
||||
ServerType& server,
|
||||
etlng::LoadBalancerInterface& balancer,
|
||||
etlng::ETLServiceInterface& etl,
|
||||
etl::LoadBalancerInterface& balancer,
|
||||
etl::ETLServiceInterface& etl,
|
||||
feed::SubscriptionManagerInterface& subscriptions,
|
||||
data::BackendInterface& backend,
|
||||
boost::asio::io_context& ioc
|
||||
|
||||
@@ -20,7 +20,7 @@
|
||||
#include "data/LedgerCache.hpp"
|
||||
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
@@ -89,7 +89,7 @@ LedgerCache::update(std::vector<LedgerObject> const& objs, uint32_t seq, bool is
|
||||
}
|
||||
|
||||
void
|
||||
LedgerCache::update(std::vector<etlng::model::Object> const& objs, uint32_t seq)
|
||||
LedgerCache::update(std::vector<etl::model::Object> const& objs, uint32_t seq)
|
||||
{
|
||||
if (disabled_)
|
||||
return;
|
||||
|
||||
@@ -21,7 +21,7 @@
|
||||
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/prometheus/Bool.hpp"
|
||||
#include "util/prometheus/Counter.hpp"
|
||||
#include "util/prometheus/Label.hpp"
|
||||
@@ -98,7 +98,7 @@ public:
|
||||
update(std::vector<LedgerObject> const& objs, uint32_t seq, bool isBackground) override;
|
||||
|
||||
void
|
||||
update(std::vector<etlng::model::Object> const& objs, uint32_t seq) override;
|
||||
update(std::vector<etl::model::Object> const& objs, uint32_t seq) override;
|
||||
|
||||
std::optional<Blob>
|
||||
get(ripple::uint256 const& key, uint32_t seq) const override;
|
||||
|
||||
@@ -20,7 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
#include <xrpl/basics/hardened_hash.h>
|
||||
@@ -63,7 +63,7 @@ public:
|
||||
* @param seq The sequence to update cache for
|
||||
*/
|
||||
virtual void
|
||||
update(std::vector<etlng::model::Object> const& objs, uint32_t seq) = 0;
|
||||
update(std::vector<etl::model::Object> const& objs, uint32_t seq) = 0;
|
||||
|
||||
/**
|
||||
* @brief Fetch a cached object by its key and sequence number.
|
||||
|
||||
@@ -19,7 +19,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief The interface of a handler for amendment blocking
|
||||
@@ -40,4 +40,4 @@ struct AmendmentBlockHandlerInterface {
|
||||
stop() = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -7,14 +7,24 @@ target_sources(
|
||||
ETLService.cpp
|
||||
ETLState.cpp
|
||||
LoadBalancer.cpp
|
||||
MPTHelpers.cpp
|
||||
NetworkValidatedLedgers.cpp
|
||||
NFTHelpers.cpp
|
||||
Source.cpp
|
||||
MPTHelpers.cpp
|
||||
impl/AmendmentBlockHandler.cpp
|
||||
impl/AsyncGrpcCall.cpp
|
||||
impl/Extraction.cpp
|
||||
impl/ForwardingSource.cpp
|
||||
impl/GrpcSource.cpp
|
||||
impl/Loading.cpp
|
||||
impl/Monitor.cpp
|
||||
impl/SubscriptionSource.cpp
|
||||
impl/TaskManager.cpp
|
||||
impl/ext/Cache.cpp
|
||||
impl/ext/Core.cpp
|
||||
impl/ext/MPT.cpp
|
||||
impl/ext/NFT.cpp
|
||||
impl/ext/Successor.cpp
|
||||
)
|
||||
|
||||
target_link_libraries(clio_etl PUBLIC clio_data)
|
||||
|
||||
@@ -21,12 +21,12 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "etl/CacheLoaderInterface.hpp"
|
||||
#include "etl/CacheLoaderSettings.hpp"
|
||||
#include "etl/impl/CacheLoader.hpp"
|
||||
#include "etl/impl/CursorFromAccountProvider.hpp"
|
||||
#include "etl/impl/CursorFromDiffProvider.hpp"
|
||||
#include "etl/impl/CursorFromFixDiffNumProvider.hpp"
|
||||
#include "etlng/CacheLoaderInterface.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/async/context/BasicExecutionContext.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
@@ -48,7 +48,7 @@ namespace etl {
|
||||
* @tparam ExecutionContextType The type of the execution context to use
|
||||
*/
|
||||
template <typename ExecutionContextType = util::async::CoroExecutionContext>
|
||||
class CacheLoader : public etlng::CacheLoaderInterface {
|
||||
class CacheLoader : public CacheLoaderInterface {
|
||||
using CacheLoaderType = impl::CacheLoaderImpl<data::LedgerCacheInterface>;
|
||||
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
@@ -21,7 +21,7 @@
|
||||
|
||||
#include <cstdint>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief An interface for the Cache Loader
|
||||
@@ -50,4 +50,4 @@ struct CacheLoaderInterface {
|
||||
wait() noexcept = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -20,12 +20,12 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief An interface for the Cache Updater
|
||||
@@ -63,4 +63,4 @@ struct CacheUpdaterInterface {
|
||||
setFull() = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -1,7 +1,7 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2023, the clio developers.
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
@@ -20,371 +20,385 @@
|
||||
#include "etl/ETLService.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/CacheLoader.hpp"
|
||||
#include "etl/CorruptionDetector.hpp"
|
||||
#include "etl/LoadBalancer.hpp"
|
||||
#include "etl/CacheLoaderInterface.hpp"
|
||||
#include "etl/CacheUpdaterInterface.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/ExtractorInterface.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LedgerPublisherInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/MonitorProviderInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/TaskManagerProviderInterface.hpp"
|
||||
#include "etl/impl/AmendmentBlockHandler.hpp"
|
||||
#include "etl/impl/ExtractionDataPipe.hpp"
|
||||
#include "etl/impl/Extractor.hpp"
|
||||
#include "etl/impl/CacheUpdater.hpp"
|
||||
#include "etl/impl/Extraction.hpp"
|
||||
#include "etl/impl/LedgerFetcher.hpp"
|
||||
#include "etl/impl/LedgerLoader.hpp"
|
||||
#include "etl/impl/LedgerPublisher.hpp"
|
||||
#include "etl/impl/Transformer.hpp"
|
||||
#include "etlng/ETLService.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etlng/LoadBalancer.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/impl/LedgerPublisher.hpp"
|
||||
#include "etlng/impl/MonitorProvider.hpp"
|
||||
#include "etlng/impl/TaskManagerProvider.hpp"
|
||||
#include "etlng/impl/ext/Cache.hpp"
|
||||
#include "etlng/impl/ext/Core.hpp"
|
||||
#include "etlng/impl/ext/MPT.hpp"
|
||||
#include "etlng/impl/ext/NFT.hpp"
|
||||
#include "etlng/impl/ext/Successor.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "etl/impl/Loading.hpp"
|
||||
#include "etl/impl/MonitorProvider.hpp"
|
||||
#include "etl/impl/Registry.hpp"
|
||||
#include "etl/impl/Scheduling.hpp"
|
||||
#include "etl/impl/TaskManager.hpp"
|
||||
#include "etl/impl/TaskManagerProvider.hpp"
|
||||
#include "etl/impl/ext/Cache.hpp"
|
||||
#include "etl/impl/ext/Core.hpp"
|
||||
#include "etl/impl/ext/MPT.hpp"
|
||||
#include "etl/impl/ext/NFT.hpp"
|
||||
#include "etl/impl/ext/Successor.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Constants.hpp"
|
||||
#include "util/Profiler.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <xrpl/beast/core/CurrentThreadName.h>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/signals2/connection.hpp>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <stdexcept>
|
||||
#include <thread>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl {
|
||||
|
||||
std::shared_ptr<etlng::ETLServiceInterface>
|
||||
std::shared_ptr<ETLServiceInterface>
|
||||
ETLService::makeETLService(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> ledgers
|
||||
)
|
||||
{
|
||||
std::shared_ptr<etlng::ETLServiceInterface> ret;
|
||||
std::shared_ptr<ETLServiceInterface> ret;
|
||||
|
||||
if (config.get<bool>("__ng_etl")) {
|
||||
ASSERT(
|
||||
std::dynamic_pointer_cast<etlng::LoadBalancer>(balancer), "LoadBalancer type must be etlng::LoadBalancer"
|
||||
);
|
||||
auto state = std::make_shared<SystemState>();
|
||||
state->isStrictReadonly = config.get<bool>("read_only");
|
||||
|
||||
auto state = std::make_shared<etl::SystemState>();
|
||||
state->isStrictReadonly = config.get<bool>("read_only");
|
||||
auto fetcher = std::make_shared<impl::LedgerFetcher>(backend, balancer);
|
||||
auto extractor = std::make_shared<impl::Extractor>(fetcher);
|
||||
auto publisher = std::make_shared<impl::LedgerPublisher>(ioc, backend, subscriptions, *state);
|
||||
auto cacheLoader = std::make_shared<CacheLoader<>>(config, backend, backend->cache());
|
||||
auto cacheUpdater = std::make_shared<impl::CacheUpdater>(backend->cache());
|
||||
auto amendmentBlockHandler = std::make_shared<impl::AmendmentBlockHandler>(ctx, *state);
|
||||
auto monitorProvider = std::make_shared<impl::MonitorProvider>();
|
||||
|
||||
auto fetcher = std::make_shared<etl::impl::LedgerFetcher>(backend, balancer);
|
||||
auto extractor = std::make_shared<etlng::impl::Extractor>(fetcher);
|
||||
auto publisher = std::make_shared<etlng::impl::LedgerPublisher>(ioc, backend, subscriptions, *state);
|
||||
auto cacheLoader = std::make_shared<etl::CacheLoader<>>(config, backend, backend->cache());
|
||||
auto cacheUpdater = std::make_shared<etlng::impl::CacheUpdater>(backend->cache());
|
||||
auto amendmentBlockHandler = std::make_shared<etlng::impl::AmendmentBlockHandler>(ctx, *state);
|
||||
auto monitorProvider = std::make_shared<etlng::impl::MonitorProvider>();
|
||||
backend->setCorruptionDetector(CorruptionDetector{*state, backend->cache()});
|
||||
|
||||
backend->setCorruptionDetector(CorruptionDetector{*state, backend->cache()});
|
||||
auto loader = std::make_shared<impl::Loader>(
|
||||
backend,
|
||||
impl::makeRegistry(
|
||||
*state,
|
||||
impl::CacheExt{cacheUpdater},
|
||||
impl::CoreExt{backend},
|
||||
impl::SuccessorExt{backend, backend->cache()},
|
||||
impl::NFTExt{backend},
|
||||
impl::MPTExt{backend}
|
||||
),
|
||||
amendmentBlockHandler,
|
||||
state
|
||||
);
|
||||
|
||||
auto loader = std::make_shared<etlng::impl::Loader>(
|
||||
backend,
|
||||
etlng::impl::makeRegistry(
|
||||
*state,
|
||||
etlng::impl::CacheExt{cacheUpdater},
|
||||
etlng::impl::CoreExt{backend},
|
||||
etlng::impl::SuccessorExt{backend, backend->cache()},
|
||||
etlng::impl::NFTExt{backend},
|
||||
etlng::impl::MPTExt{backend}
|
||||
),
|
||||
amendmentBlockHandler,
|
||||
state
|
||||
);
|
||||
auto taskManagerProvider = std::make_shared<impl::TaskManagerProvider>(*ledgers, extractor, loader);
|
||||
|
||||
auto taskManagerProvider = std::make_shared<etlng::impl::TaskManagerProvider>(*ledgers, extractor, loader);
|
||||
|
||||
ret = std::make_shared<etlng::ETLService>(
|
||||
ctx,
|
||||
config,
|
||||
backend,
|
||||
balancer,
|
||||
ledgers,
|
||||
publisher,
|
||||
cacheLoader,
|
||||
cacheUpdater,
|
||||
extractor,
|
||||
loader, // loader itself
|
||||
loader, // initial load observer
|
||||
taskManagerProvider,
|
||||
monitorProvider,
|
||||
state
|
||||
);
|
||||
} else {
|
||||
ASSERT(std::dynamic_pointer_cast<etl::LoadBalancer>(balancer), "LoadBalancer type must be etl::LoadBalancer");
|
||||
ret = std::make_shared<etl::ETLService>(config, ioc, backend, subscriptions, balancer, ledgers);
|
||||
}
|
||||
ret = std::make_shared<ETLService>(
|
||||
ctx,
|
||||
config,
|
||||
backend,
|
||||
balancer,
|
||||
ledgers,
|
||||
publisher,
|
||||
cacheLoader,
|
||||
cacheUpdater,
|
||||
extractor,
|
||||
loader, // loader itself
|
||||
loader, // initial load observer
|
||||
taskManagerProvider,
|
||||
monitorProvider,
|
||||
state
|
||||
);
|
||||
|
||||
// inject networkID into subscriptions, as transaction feed require it to inject CTID in response
|
||||
if (auto const state = ret->getETLState(); state)
|
||||
subscriptions->setNetworkID(state->networkID);
|
||||
if (auto const etlState = ret->getETLState(); etlState)
|
||||
subscriptions->setNetworkID(etlState->networkID);
|
||||
|
||||
ret->run();
|
||||
return ret;
|
||||
}
|
||||
|
||||
// Database must be populated when this starts
|
||||
std::optional<uint32_t>
|
||||
ETLService::runETLPipeline(uint32_t startSequence, uint32_t numExtractors)
|
||||
ETLService::ETLService(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::reference_wrapper<util::config::ClioConfigDefinition const> config,
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::shared_ptr<LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> ledgers,
|
||||
std::shared_ptr<LedgerPublisherInterface> publisher,
|
||||
std::shared_ptr<CacheLoaderInterface> cacheLoader,
|
||||
std::shared_ptr<CacheUpdaterInterface> cacheUpdater,
|
||||
std::shared_ptr<ExtractorInterface> extractor,
|
||||
std::shared_ptr<LoaderInterface> loader,
|
||||
std::shared_ptr<InitialLoadObserverInterface> initialLoadObserver,
|
||||
std::shared_ptr<TaskManagerProviderInterface> taskManagerProvider,
|
||||
std::shared_ptr<MonitorProviderInterface> monitorProvider,
|
||||
std::shared_ptr<SystemState> state
|
||||
)
|
||||
: ctx_(std::move(ctx))
|
||||
, config_(config)
|
||||
, backend_(std::move(backend))
|
||||
, balancer_(std::move(balancer))
|
||||
, ledgers_(std::move(ledgers))
|
||||
, publisher_(std::move(publisher))
|
||||
, cacheLoader_(std::move(cacheLoader))
|
||||
, cacheUpdater_(std::move(cacheUpdater))
|
||||
, extractor_(std::move(extractor))
|
||||
, loader_(std::move(loader))
|
||||
, initialLoadObserver_(std::move(initialLoadObserver))
|
||||
, taskManagerProvider_(std::move(taskManagerProvider))
|
||||
, monitorProvider_(std::move(monitorProvider))
|
||||
, state_(std::move(state))
|
||||
, startSequence_(config.get().maybeValue<uint32_t>("start_sequence"))
|
||||
, finishSequence_(config.get().maybeValue<uint32_t>("finish_sequence"))
|
||||
{
|
||||
if (finishSequence_ && startSequence > *finishSequence_)
|
||||
return {};
|
||||
ASSERT(not state_->isWriting, "ETL should never start in writer mode");
|
||||
|
||||
LOG(log_.debug()) << "Wait for cache containing seq " << startSequence - 1
|
||||
<< " current cache last seq =" << backend_->cache().latestLedgerSequence();
|
||||
backend_->cache().waitUntilCacheContainsSeq(startSequence - 1);
|
||||
if (startSequence_.has_value())
|
||||
LOG(log_.info()) << "Start sequence: " << *startSequence_;
|
||||
|
||||
LOG(log_.debug()) << "Starting etl pipeline";
|
||||
state_.isWriting = true;
|
||||
if (finishSequence_.has_value())
|
||||
LOG(log_.info()) << "Finish sequence: " << *finishSequence_;
|
||||
|
||||
auto const rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
ASSERT(rng.has_value(), "Parent ledger range can't be null");
|
||||
ASSERT(
|
||||
rng->maxSequence >= startSequence - 1,
|
||||
"Got not parent ledger. rnd->maxSequence = {}, startSequence = {}",
|
||||
rng->maxSequence,
|
||||
startSequence
|
||||
);
|
||||
|
||||
auto const begin = std::chrono::system_clock::now();
|
||||
auto extractors = std::vector<std::unique_ptr<ExtractorType>>{};
|
||||
auto pipe = DataPipeType{numExtractors, startSequence};
|
||||
|
||||
for (auto i = 0u; i < numExtractors; ++i) {
|
||||
extractors.push_back(
|
||||
std::make_unique<ExtractorType>(
|
||||
pipe, networkValidatedLedgers_, ledgerFetcher_, startSequence + i, finishSequence_, state_
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
auto transformer =
|
||||
TransformerType{pipe, backend_, ledgerLoader_, ledgerPublisher_, amendmentBlockHandler_, startSequence, state_};
|
||||
transformer.waitTillFinished(); // suspend current thread until exit condition is met
|
||||
pipe.cleanup(); // TODO: this should probably happen automatically using destructor
|
||||
|
||||
// wait for all of the extractors to stop
|
||||
for (auto& t : extractors)
|
||||
t->waitTillFinished();
|
||||
|
||||
auto const end = std::chrono::system_clock::now();
|
||||
auto const lastPublishedSeq = ledgerPublisher_.getLastPublishedSequence();
|
||||
static constexpr auto kNANOSECONDS_PER_SECOND = 1'000'000'000.0;
|
||||
LOG(log_.debug()) << "Extracted and wrote " << lastPublishedSeq.value_or(startSequence) - startSequence << " in "
|
||||
<< ((end - begin).count()) / kNANOSECONDS_PER_SECOND;
|
||||
|
||||
state_.isWriting = false;
|
||||
|
||||
LOG(log_.debug()) << "Stopping etl pipeline";
|
||||
return lastPublishedSeq;
|
||||
LOG(log_.info()) << "Starting in " << (state_->isStrictReadonly ? "STRICT READONLY MODE" : "WRITE MODE");
|
||||
}
|
||||
|
||||
// Main loop of ETL.
|
||||
// The software begins monitoring the ledgers that are validated by the network.
|
||||
// The member networkValidatedLedgers_ keeps track of the sequences of ledgers validated by the network.
|
||||
// Whenever a ledger is validated by the network, the software looks for that ledger in the database. Once the ledger is
|
||||
// found in the database, the software publishes that ledger to the ledgers stream. If a network validated ledger is not
|
||||
// found in the database after a certain amount of time, then the software attempts to take over responsibility of the
|
||||
// ETL process, where it writes new ledgers to the database. The software will relinquish control of the ETL process if
|
||||
// it detects that another process has taken over ETL.
|
||||
void
|
||||
ETLService::monitor()
|
||||
ETLService::~ETLService()
|
||||
{
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
if (!rng) {
|
||||
LOG(log_.info()) << "Database is empty. Will download a ledger from the network.";
|
||||
std::optional<ripple::LedgerHeader> ledger;
|
||||
|
||||
try {
|
||||
if (startSequence_) {
|
||||
LOG(log_.info()) << "ledger sequence specified in config. "
|
||||
<< "Will begin ETL process starting with ledger " << *startSequence_;
|
||||
ledger = ledgerLoader_.loadInitialLedger(*startSequence_);
|
||||
} else {
|
||||
LOG(log_.info()) << "Waiting for next ledger to be validated by network...";
|
||||
std::optional<uint32_t> mostRecentValidated = networkValidatedLedgers_->getMostRecent();
|
||||
|
||||
if (mostRecentValidated) {
|
||||
LOG(log_.info()) << "Ledger " << *mostRecentValidated << " has been validated. Downloading...";
|
||||
ledger = ledgerLoader_.loadInitialLedger(*mostRecentValidated);
|
||||
} else {
|
||||
LOG(log_.info()) << "The wait for the next validated ledger has been aborted. "
|
||||
"Exiting monitor loop";
|
||||
return;
|
||||
}
|
||||
}
|
||||
} catch (std::runtime_error const& e) {
|
||||
LOG(log_.fatal()) << "Failed to load initial ledger: " << e.what();
|
||||
amendmentBlockHandler_.notifyAmendmentBlocked();
|
||||
return;
|
||||
}
|
||||
|
||||
if (ledger) {
|
||||
rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
} else {
|
||||
LOG(log_.error()) << "Failed to load initial ledger. Exiting monitor loop";
|
||||
return;
|
||||
}
|
||||
} else {
|
||||
if (startSequence_)
|
||||
LOG(log_.warn()) << "start sequence specified but db is already populated";
|
||||
|
||||
LOG(log_.info()) << "Database already populated. Picking up from the tip of history";
|
||||
cacheLoader_.load(rng->maxSequence);
|
||||
}
|
||||
|
||||
ASSERT(rng.has_value(), "Ledger range can't be null");
|
||||
uint32_t nextSequence = rng->maxSequence + 1;
|
||||
|
||||
LOG(log_.debug()) << "Database is populated. Starting monitor loop. sequence = " << nextSequence;
|
||||
|
||||
while (not isStopping()) {
|
||||
nextSequence = publishNextSequence(nextSequence);
|
||||
}
|
||||
}
|
||||
|
||||
uint32_t
|
||||
ETLService::publishNextSequence(uint32_t nextSequence)
|
||||
{
|
||||
if (auto rng = backend_->hardFetchLedgerRangeNoThrow(); rng && rng->maxSequence >= nextSequence) {
|
||||
ledgerPublisher_.publish(nextSequence, {});
|
||||
++nextSequence;
|
||||
} else if (networkValidatedLedgers_->waitUntilValidatedByNetwork(nextSequence, util::kMILLISECONDS_PER_SECOND)) {
|
||||
LOG(log_.info()) << "Ledger with sequence = " << nextSequence << " has been validated by the network. "
|
||||
<< "Attempting to find in database and publish";
|
||||
|
||||
// Attempt to take over responsibility of ETL writer after 10 failed
|
||||
// attempts to publish the ledger. publishLedger() fails if the
|
||||
// ledger that has been validated by the network is not found in the
|
||||
// database after the specified number of attempts. publishLedger()
|
||||
// waits one second between each attempt to read the ledger from the
|
||||
// database
|
||||
constexpr size_t kTIMEOUT_SECONDS = 10;
|
||||
bool const success = ledgerPublisher_.publish(nextSequence, kTIMEOUT_SECONDS);
|
||||
|
||||
if (!success) {
|
||||
LOG(log_.warn()) << "Failed to publish ledger with sequence = " << nextSequence << " . Beginning ETL";
|
||||
|
||||
// returns the most recent sequence published. empty optional if no sequence was published
|
||||
std::optional<uint32_t> lastPublished = runETLPipeline(nextSequence, extractorThreads_);
|
||||
LOG(log_.info()) << "Aborting ETL. Falling back to publishing";
|
||||
|
||||
// if no ledger was published, don't increment nextSequence
|
||||
if (lastPublished)
|
||||
nextSequence = *lastPublished + 1;
|
||||
} else {
|
||||
++nextSequence;
|
||||
}
|
||||
}
|
||||
return nextSequence;
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::monitorReadOnly()
|
||||
{
|
||||
LOG(log_.debug()) << "Starting reporting in strict read only mode";
|
||||
|
||||
auto const latestSequenceOpt = [this]() -> std::optional<uint32_t> {
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
|
||||
if (!rng) {
|
||||
if (auto net = networkValidatedLedgers_->getMostRecent()) {
|
||||
return net;
|
||||
}
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
return rng->maxSequence;
|
||||
}();
|
||||
|
||||
if (!latestSequenceOpt.has_value()) {
|
||||
return;
|
||||
}
|
||||
|
||||
uint32_t latestSequence = *latestSequenceOpt;
|
||||
|
||||
cacheLoader_.load(latestSequence);
|
||||
latestSequence++;
|
||||
|
||||
while (not isStopping()) {
|
||||
if (auto rng = backend_->hardFetchLedgerRangeNoThrow(); rng && rng->maxSequence >= latestSequence) {
|
||||
ledgerPublisher_.publish(latestSequence, {});
|
||||
latestSequence = latestSequence + 1;
|
||||
} else {
|
||||
// if we can't, wait until it's validated by the network, or 1 second passes, whichever occurs
|
||||
// first. Even if we don't hear from rippled, if ledgers are being written to the db, we publish
|
||||
// them.
|
||||
networkValidatedLedgers_->waitUntilValidatedByNetwork(latestSequence, util::kMILLISECONDS_PER_SECOND);
|
||||
}
|
||||
}
|
||||
stop();
|
||||
LOG(log_.debug()) << "Destroying ETL";
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::run()
|
||||
{
|
||||
LOG(log_.info()) << "Starting reporting etl";
|
||||
state_.isStopping = false;
|
||||
LOG(log_.info()) << "Running ETL...";
|
||||
|
||||
doWork();
|
||||
mainLoop_.emplace(ctx_.execute([this] {
|
||||
auto const rng = loadInitialLedgerIfNeeded();
|
||||
|
||||
LOG(log_.info()) << "Waiting for next ledger to be validated by network...";
|
||||
std::optional<uint32_t> const mostRecentValidated = ledgers_->getMostRecent();
|
||||
|
||||
if (not mostRecentValidated) {
|
||||
LOG(log_.info()) << "The wait for the next validated ledger has been aborted. "
|
||||
"Exiting monitor loop";
|
||||
return;
|
||||
}
|
||||
|
||||
if (not rng.has_value()) {
|
||||
LOG(log_.warn()) << "Initial ledger download got cancelled - stopping ETL service";
|
||||
return;
|
||||
}
|
||||
|
||||
auto const nextSequence = rng->maxSequence + 1;
|
||||
|
||||
LOG(log_.debug()) << "Database is populated. Starting monitor loop. sequence = " << nextSequence;
|
||||
startMonitor(nextSequence);
|
||||
|
||||
// If we are a writer as the result of loading the initial ledger - start loading
|
||||
if (state_->isWriting)
|
||||
startLoading(nextSequence);
|
||||
}));
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::doWork()
|
||||
ETLService::stop()
|
||||
{
|
||||
worker_ = std::thread([this]() {
|
||||
beast::setCurrentThreadName("ETLService worker");
|
||||
LOG(log_.info()) << "Stop called";
|
||||
|
||||
if (state_.isStrictReadonly) {
|
||||
monitorReadOnly();
|
||||
} else {
|
||||
monitor();
|
||||
if (mainLoop_)
|
||||
mainLoop_->wait();
|
||||
if (taskMan_)
|
||||
taskMan_->stop();
|
||||
if (monitor_)
|
||||
monitor_->stop();
|
||||
}
|
||||
|
||||
boost::json::object
|
||||
ETLService::getInfo() const
|
||||
{
|
||||
boost::json::object result;
|
||||
|
||||
result["etl_sources"] = balancer_->toJson();
|
||||
result["is_writer"] = static_cast<int>(state_->isWriting);
|
||||
result["read_only"] = static_cast<int>(state_->isStrictReadonly);
|
||||
auto last = publisher_->getLastPublish();
|
||||
if (last.time_since_epoch().count() != 0)
|
||||
result["last_publish_age_seconds"] = std::to_string(publisher_->lastPublishAgeSeconds());
|
||||
return result;
|
||||
}
|
||||
|
||||
bool
|
||||
ETLService::isAmendmentBlocked() const
|
||||
{
|
||||
return state_->isAmendmentBlocked;
|
||||
}
|
||||
|
||||
bool
|
||||
ETLService::isCorruptionDetected() const
|
||||
{
|
||||
return state_->isCorruptionDetected;
|
||||
}
|
||||
|
||||
std::optional<ETLState>
|
||||
ETLService::getETLState() const
|
||||
{
|
||||
return balancer_->getETLState();
|
||||
}
|
||||
|
||||
std::uint32_t
|
||||
ETLService::lastCloseAgeSeconds() const
|
||||
{
|
||||
return publisher_->lastCloseAgeSeconds();
|
||||
}
|
||||
|
||||
std::optional<data::LedgerRange>
|
||||
ETLService::loadInitialLedgerIfNeeded()
|
||||
{
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
if (not rng.has_value()) {
|
||||
ASSERT(
|
||||
not state_->isStrictReadonly,
|
||||
"Database is empty but this node is in strict readonly mode. Can't write initial ledger."
|
||||
);
|
||||
|
||||
LOG(log_.info()) << "Database is empty. Will download a ledger from the network.";
|
||||
state_->isWriting = true; // immediately become writer as the db is empty
|
||||
|
||||
auto const getMostRecent = [this]() {
|
||||
LOG(log_.info()) << "Waiting for next ledger to be validated by network...";
|
||||
return ledgers_->getMostRecent();
|
||||
};
|
||||
|
||||
if (auto const maybeSeq = startSequence_.or_else(getMostRecent); maybeSeq.has_value()) {
|
||||
auto const seq = *maybeSeq;
|
||||
LOG(log_.info()) << "Starting from sequence " << seq
|
||||
<< ". Initial ledger download and extraction can take a while...";
|
||||
|
||||
auto [ledger, timeDiff] = ::util::timed<std::chrono::duration<double>>([this, seq]() {
|
||||
return extractor_->extractLedgerOnly(seq).and_then(
|
||||
[this, seq](auto&& data) -> std::optional<ripple::LedgerHeader> {
|
||||
// TODO: loadInitialLedger in balancer should be called fetchEdgeKeys or similar
|
||||
auto res = balancer_->loadInitialLedger(seq, *initialLoadObserver_);
|
||||
if (not res.has_value() and res.error() == InitialLedgerLoadError::Cancelled) {
|
||||
LOG(log_.debug()) << "Initial ledger load got cancelled";
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
ASSERT(res.has_value(), "Initial ledger retry logic failed");
|
||||
data.edgeKeys = std::move(res).value();
|
||||
|
||||
return loader_->loadInitialLedger(data);
|
||||
}
|
||||
);
|
||||
});
|
||||
|
||||
if (not ledger.has_value()) {
|
||||
LOG(log_.error()) << "Failed to load initial ledger. Exiting monitor loop";
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
LOG(log_.debug()) << "Time to download and store ledger = " << timeDiff;
|
||||
LOG(log_.info()) << "Finished loadInitialLedger. cache size = " << backend_->cache().size();
|
||||
|
||||
return backend_->hardFetchLedgerRangeNoThrow();
|
||||
}
|
||||
});
|
||||
|
||||
LOG(log_.info()) << "The wait for the next validated ledger has been aborted. "
|
||||
"Exiting monitor loop";
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
LOG(log_.info()) << "Database already populated. Picking up from the tip of history";
|
||||
cacheLoader_->load(rng->maxSequence);
|
||||
|
||||
return rng;
|
||||
}
|
||||
|
||||
ETLService::ETLService(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> ledgers
|
||||
)
|
||||
: backend_(backend)
|
||||
, loadBalancer_(balancer)
|
||||
, networkValidatedLedgers_(std::move(ledgers))
|
||||
, cacheLoader_(config, backend, backend->cache())
|
||||
, ledgerFetcher_(backend, balancer)
|
||||
, ledgerLoader_(backend, balancer, ledgerFetcher_, state_)
|
||||
, ledgerPublisher_(ioc, backend, backend->cache(), subscriptions, state_)
|
||||
, amendmentBlockHandler_(ioc, state_)
|
||||
void
|
||||
ETLService::startMonitor(uint32_t seq)
|
||||
{
|
||||
startSequence_ = config.maybeValue<uint32_t>("start_sequence");
|
||||
finishSequence_ = config.maybeValue<uint32_t>("finish_sequence");
|
||||
state_.isStrictReadonly = config.get<bool>("read_only");
|
||||
extractorThreads_ = config.get<uint32_t>("extractor_threads");
|
||||
monitor_ = monitorProvider_->make(ctx_, backend_, ledgers_, seq);
|
||||
|
||||
// This should probably be done in the backend factory but we don't have state available until here
|
||||
backend_->setCorruptionDetector(CorruptionDetector{state_, backend->cache()});
|
||||
monitorNewSeqSubscription_ = monitor_->subscribeToNewSequence([this](uint32_t seq) {
|
||||
LOG(log_.info()) << "ETLService (via Monitor) got new seq from db: " << seq;
|
||||
|
||||
if (state_->writeConflict) {
|
||||
LOG(log_.info()) << "Got a write conflict; Giving up writer seat immediately";
|
||||
giveUpWriter();
|
||||
}
|
||||
|
||||
if (not state_->isWriting) {
|
||||
auto const diff = data::synchronousAndRetryOnTimeout([this, seq](auto yield) {
|
||||
return backend_->fetchLedgerDiff(seq, yield);
|
||||
});
|
||||
|
||||
cacheUpdater_->update(seq, diff);
|
||||
backend_->updateRange(seq);
|
||||
}
|
||||
|
||||
publisher_->publish(seq, {});
|
||||
});
|
||||
|
||||
monitorDbStalledSubscription_ = monitor_->subscribeToDbStalled([this]() {
|
||||
LOG(log_.warn()) << "ETLService received DbStalled signal from Monitor";
|
||||
if (not state_->isStrictReadonly and not state_->isWriting)
|
||||
attemptTakeoverWriter();
|
||||
});
|
||||
|
||||
monitor_->run();
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::startLoading(uint32_t seq)
|
||||
{
|
||||
ASSERT(not state_->isStrictReadonly, "This should only happen on writer nodes");
|
||||
taskMan_ = taskManagerProvider_->make(ctx_, *monitor_, seq, finishSequence_);
|
||||
|
||||
// FIXME: this legacy name "extractor_threads" is no longer accurate (we have coroutines now)
|
||||
taskMan_->run(config_.get().get<std::size_t>("extractor_threads"));
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::attemptTakeoverWriter()
|
||||
{
|
||||
ASSERT(not state_->isStrictReadonly, "This should only happen on writer nodes");
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
ASSERT(rng.has_value(), "Ledger range can't be null");
|
||||
|
||||
state_->isWriting = true; // switch to writer
|
||||
LOG(log_.info()) << "Taking over the ETL writer seat";
|
||||
startLoading(rng->maxSequence + 1);
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::giveUpWriter()
|
||||
{
|
||||
ASSERT(not state_->isStrictReadonly, "This should only happen on writer nodes");
|
||||
state_->isWriting = false;
|
||||
state_->writeConflict = false;
|
||||
taskMan_ = nullptr;
|
||||
}
|
||||
|
||||
} // namespace etl
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2023, the clio developers.
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
@@ -20,57 +20,64 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/CacheLoader.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/CacheLoaderInterface.hpp"
|
||||
#include "etl/CacheUpdaterInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/ExtractorInterface.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LedgerPublisherInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/MonitorProviderInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/TaskManagerInterface.hpp"
|
||||
#include "etl/TaskManagerProviderInterface.hpp"
|
||||
#include "etl/impl/AmendmentBlockHandler.hpp"
|
||||
#include "etl/impl/ExtractionDataPipe.hpp"
|
||||
#include "etl/impl/Extractor.hpp"
|
||||
#include "etl/impl/CacheUpdater.hpp"
|
||||
#include "etl/impl/Extraction.hpp"
|
||||
#include "etl/impl/LedgerFetcher.hpp"
|
||||
#include "etl/impl/LedgerLoader.hpp"
|
||||
#include "etl/impl/LedgerPublisher.hpp"
|
||||
#include "etl/impl/Transformer.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/impl/LedgerPublisher.hpp"
|
||||
#include "etlng/impl/TaskManagerProvider.hpp"
|
||||
#include "etl/impl/Loading.hpp"
|
||||
#include "etl/impl/Registry.hpp"
|
||||
#include "etl/impl/Scheduling.hpp"
|
||||
#include "etl/impl/TaskManager.hpp"
|
||||
#include "etl/impl/ext/Cache.hpp"
|
||||
#include "etl/impl/ext/Core.hpp"
|
||||
#include "etl/impl/ext/NFT.hpp"
|
||||
#include "etl/impl/ext/Successor.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/async/AnyOperation.hpp"
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <grpcpp/grpcpp.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
#include <boost/signals2/connection.hpp>
|
||||
#include <fmt/format.h>
|
||||
#include <xrpl/basics/Blob.h>
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
#include <xrpl/basics/strHex.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/ledger.pb.h>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
#include <xrpl/protocol/STTx.h>
|
||||
#include <xrpl/protocol/TxFormats.h>
|
||||
#include <xrpl/protocol/TxMeta.h>
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
|
||||
struct AccountTransactionsData;
|
||||
struct NFTTransactionsData;
|
||||
struct NFTsData;
|
||||
|
||||
/**
|
||||
* @brief This namespace contains everything to do with the ETL and ETL sources.
|
||||
*/
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief A tag class to help identify ETLService in templated code.
|
||||
*/
|
||||
struct ETLServiceTag {
|
||||
virtual ~ETLServiceTag() = default;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept SomeETLService = std::derived_from<T, ETLServiceTag>;
|
||||
|
||||
/**
|
||||
* @brief This class is responsible for continuously extracting data from a p2p node, and writing that data to the
|
||||
* databases.
|
||||
@@ -84,64 +91,36 @@ concept SomeETLService = std::derived_from<T, ETLServiceTag>;
|
||||
* the others will fall back to monitoring/publishing. In this sense, this class dynamically transitions from monitoring
|
||||
* to writing and from writing to monitoring, based on the activity of other processes running on different machines.
|
||||
*/
|
||||
class ETLService : public etlng::ETLServiceInterface, ETLServiceTag {
|
||||
// TODO: make these template parameters in ETLService
|
||||
using DataPipeType = etl::impl::ExtractionDataPipe<org::xrpl::rpc::v1::GetLedgerResponse>;
|
||||
using CacheLoaderType = etl::CacheLoader<>;
|
||||
using LedgerFetcherType = etl::impl::LedgerFetcher;
|
||||
using ExtractorType = etl::impl::Extractor<DataPipeType, LedgerFetcherType>;
|
||||
using LedgerLoaderType = etl::impl::LedgerLoader<LedgerFetcherType>;
|
||||
using LedgerPublisherType = etl::impl::LedgerPublisher;
|
||||
using AmendmentBlockHandlerType = etl::impl::AmendmentBlockHandler;
|
||||
using TransformerType =
|
||||
etl::impl::Transformer<DataPipeType, LedgerLoaderType, LedgerPublisherType, AmendmentBlockHandlerType>;
|
||||
|
||||
class ETLService : public ETLServiceInterface {
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
util::async::AnyExecutionContext ctx_;
|
||||
std::reference_wrapper<util::config::ClioConfigDefinition const> config_;
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> loadBalancer_;
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> networkValidatedLedgers_;
|
||||
std::shared_ptr<LoadBalancerInterface> balancer_;
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> ledgers_;
|
||||
std::shared_ptr<LedgerPublisherInterface> publisher_;
|
||||
std::shared_ptr<CacheLoaderInterface> cacheLoader_;
|
||||
std::shared_ptr<CacheUpdaterInterface> cacheUpdater_;
|
||||
std::shared_ptr<ExtractorInterface> extractor_;
|
||||
std::shared_ptr<LoaderInterface> loader_;
|
||||
std::shared_ptr<InitialLoadObserverInterface> initialLoadObserver_;
|
||||
std::shared_ptr<TaskManagerProviderInterface> taskManagerProvider_;
|
||||
std::shared_ptr<MonitorProviderInterface> monitorProvider_;
|
||||
std::shared_ptr<SystemState> state_;
|
||||
|
||||
std::uint32_t extractorThreads_ = 1;
|
||||
std::thread worker_;
|
||||
|
||||
CacheLoaderType cacheLoader_;
|
||||
LedgerFetcherType ledgerFetcher_;
|
||||
LedgerLoaderType ledgerLoader_;
|
||||
LedgerPublisherType ledgerPublisher_;
|
||||
AmendmentBlockHandlerType amendmentBlockHandler_;
|
||||
|
||||
SystemState state_;
|
||||
|
||||
size_t numMarkers_ = 2;
|
||||
std::optional<uint32_t> startSequence_;
|
||||
std::optional<uint32_t> finishSequence_;
|
||||
|
||||
std::unique_ptr<MonitorInterface> monitor_;
|
||||
std::unique_ptr<TaskManagerInterface> taskMan_;
|
||||
|
||||
boost::signals2::scoped_connection monitorNewSeqSubscription_;
|
||||
boost::signals2::scoped_connection monitorDbStalledSubscription_;
|
||||
|
||||
std::optional<util::async::AnyOperation<void>> mainLoop_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of ETLService.
|
||||
*
|
||||
* @param config The configuration to use
|
||||
* @param ioc io context to run on
|
||||
* @param backend BackendInterface implementation
|
||||
* @param subscriptions Subscription manager
|
||||
* @param balancer Load balancer to use
|
||||
* @param ledgers The network validated ledgers datastructure
|
||||
*/
|
||||
ETLService(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> ledgers
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Move constructor is deleted because ETL service shares its fields by reference
|
||||
*/
|
||||
ETLService(ETLService&&) = delete;
|
||||
|
||||
/**
|
||||
* @brief A factory function to spawn new ETLService instances.
|
||||
*
|
||||
@@ -156,182 +135,90 @@ public:
|
||||
* @param ledgers The network validated ledgers datastructure
|
||||
* @return A shared pointer to a new instance of ETLService
|
||||
*/
|
||||
static std::shared_ptr<etlng::ETLServiceInterface>
|
||||
static std::shared_ptr<ETLServiceInterface>
|
||||
makeETLService(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
boost::asio::io_context& ioc, // TODO: remove (LedgerPublisher needs to be changed)
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> ledgers
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Stops components and joins worker thread.
|
||||
*/
|
||||
~ETLService() override
|
||||
{
|
||||
if (not state_.isStopping)
|
||||
stop();
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Stop the ETL service.
|
||||
* @note This method blocks until the ETL service has stopped.
|
||||
*/
|
||||
void
|
||||
stop() override
|
||||
{
|
||||
LOG(log_.info()) << "Stop called";
|
||||
|
||||
state_.isStopping = true;
|
||||
cacheLoader_.stop();
|
||||
|
||||
if (worker_.joinable())
|
||||
worker_.join();
|
||||
|
||||
LOG(log_.debug()) << "Joined ETLService worker thread";
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get time passed since last ledger close, in seconds.
|
||||
* @brief Create an instance of ETLService.
|
||||
*
|
||||
* @return Time passed since last ledger close
|
||||
* @param ctx The execution context for asynchronous operations
|
||||
* @param config The Clio configuration definition
|
||||
* @param backend Interface to the backend database
|
||||
* @param balancer Load balancer for distributing work
|
||||
* @param ledgers Interface for accessing network validated ledgers
|
||||
* @param publisher Interface for publishing ledger data
|
||||
* @param cacheLoader Interface for loading cache data
|
||||
* @param cacheUpdater Interface for updating cache data
|
||||
* @param extractor The extractor to use
|
||||
* @param loader Interface for loading data
|
||||
* @param initialLoadObserver The observer for initial data loading
|
||||
* @param taskManagerProvider The provider of the task manager instance
|
||||
* @param monitorProvider The provider of the monitor instance
|
||||
* @param state System state tracking object
|
||||
*/
|
||||
std::uint32_t
|
||||
lastCloseAgeSeconds() const override
|
||||
{
|
||||
return ledgerPublisher_.lastCloseAgeSeconds();
|
||||
}
|
||||
ETLService(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::reference_wrapper<util::config::ClioConfigDefinition const> config,
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::shared_ptr<LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> ledgers,
|
||||
std::shared_ptr<LedgerPublisherInterface> publisher,
|
||||
std::shared_ptr<CacheLoaderInterface> cacheLoader,
|
||||
std::shared_ptr<CacheUpdaterInterface> cacheUpdater,
|
||||
std::shared_ptr<ExtractorInterface> extractor,
|
||||
std::shared_ptr<LoaderInterface> loader,
|
||||
std::shared_ptr<InitialLoadObserverInterface> initialLoadObserver,
|
||||
std::shared_ptr<TaskManagerProviderInterface> taskManagerProvider,
|
||||
std::shared_ptr<MonitorProviderInterface> monitorProvider,
|
||||
std::shared_ptr<SystemState> state
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Check for the amendment blocked state.
|
||||
*
|
||||
* @return true if currently amendment blocked; false otherwise
|
||||
*/
|
||||
bool
|
||||
isAmendmentBlocked() const override
|
||||
{
|
||||
return state_.isAmendmentBlocked;
|
||||
}
|
||||
~ETLService() override;
|
||||
|
||||
/**
|
||||
* @brief Check whether Clio detected DB corruptions.
|
||||
*
|
||||
* @return true if corruption of DB was detected and cache was stopped.
|
||||
*/
|
||||
bool
|
||||
isCorruptionDetected() const override
|
||||
{
|
||||
return state_.isCorruptionDetected;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get state of ETL as a JSON object
|
||||
*
|
||||
* @return The state of ETL as a JSON object
|
||||
*/
|
||||
boost::json::object
|
||||
getInfo() const override
|
||||
{
|
||||
boost::json::object result;
|
||||
|
||||
result["etl_sources"] = loadBalancer_->toJson();
|
||||
result["is_writer"] = static_cast<int>(state_.isWriting);
|
||||
result["read_only"] = static_cast<int>(state_.isStrictReadonly);
|
||||
auto last = ledgerPublisher_.getLastPublish();
|
||||
if (last.time_since_epoch().count() != 0)
|
||||
result["last_publish_age_seconds"] = std::to_string(ledgerPublisher_.lastPublishAgeSeconds());
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get the etl nodes' state
|
||||
* @return The etl nodes' state, nullopt if etl nodes are not connected
|
||||
*/
|
||||
std::optional<etl::ETLState>
|
||||
getETLState() const noexcept override
|
||||
{
|
||||
return loadBalancer_->getETLState();
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Start all components to run ETL service.
|
||||
*/
|
||||
void
|
||||
run() override;
|
||||
|
||||
private:
|
||||
/**
|
||||
* @brief Run the ETL pipeline.
|
||||
*
|
||||
* Extracts ledgers and writes them to the database, until a write conflict occurs (or the server shuts down).
|
||||
* @note database must already be populated when this function is called
|
||||
*
|
||||
* @param startSequence the first ledger to extract
|
||||
* @param numExtractors number of extractors to use
|
||||
* @return The last ledger written to the database, if any
|
||||
*/
|
||||
std::optional<uint32_t>
|
||||
runETLPipeline(uint32_t startSequence, uint32_t numExtractors);
|
||||
|
||||
/**
|
||||
* @brief Monitor the network for newly validated ledgers.
|
||||
*
|
||||
* Also monitor the database to see if any process is writing those ledgers.
|
||||
* This function is called when the application starts, and will only return when the application is shutting down.
|
||||
* If the software detects the database is empty, this function will call loadInitialLedger(). If the software
|
||||
* detects ledgers are not being written, this function calls runETLPipeline(). Otherwise, this function publishes
|
||||
* ledgers as they are written to the database.
|
||||
*/
|
||||
void
|
||||
monitor();
|
||||
stop() override;
|
||||
|
||||
/**
|
||||
* @brief Monitor the network for newly validated ledgers and publish them to the ledgers stream
|
||||
*
|
||||
* @param nextSequence the ledger sequence to publish
|
||||
* @return The next ledger sequence to publish
|
||||
*/
|
||||
uint32_t
|
||||
publishNextSequence(uint32_t nextSequence);
|
||||
boost::json::object
|
||||
getInfo() const override;
|
||||
|
||||
/**
|
||||
* @brief Monitor the database for newly written ledgers.
|
||||
*
|
||||
* Similar to the monitor(), except this function will never call runETLPipeline() or loadInitialLedger().
|
||||
* This function only publishes ledgers as they are written to the database.
|
||||
*/
|
||||
void
|
||||
monitorReadOnly();
|
||||
|
||||
/**
|
||||
* @return true if stopping; false otherwise
|
||||
*/
|
||||
bool
|
||||
isStopping() const
|
||||
{
|
||||
return state_.isStopping;
|
||||
}
|
||||
isAmendmentBlocked() const override;
|
||||
|
||||
bool
|
||||
isCorruptionDetected() const override;
|
||||
|
||||
std::optional<ETLState>
|
||||
getETLState() const override;
|
||||
|
||||
/**
|
||||
* @brief Get the number of markers to use during the initial ledger download.
|
||||
*
|
||||
* This is equivalent to the degree of parallelism during the initial ledger download.
|
||||
*
|
||||
* @return The number of markers
|
||||
*/
|
||||
std::uint32_t
|
||||
getNumMarkers() const
|
||||
{
|
||||
return numMarkers_;
|
||||
}
|
||||
lastCloseAgeSeconds() const override;
|
||||
|
||||
private:
|
||||
std::optional<data::LedgerRange>
|
||||
loadInitialLedgerIfNeeded();
|
||||
|
||||
/**
|
||||
* @brief Spawn the worker thread and start monitoring.
|
||||
*/
|
||||
void
|
||||
doWork();
|
||||
startMonitor(uint32_t seq);
|
||||
|
||||
void
|
||||
startLoading(uint32_t seq);
|
||||
|
||||
void
|
||||
attemptTakeoverWriter();
|
||||
|
||||
void
|
||||
giveUpWriter();
|
||||
};
|
||||
|
||||
} // namespace etl
|
||||
|
||||
@@ -26,7 +26,7 @@
|
||||
#include <cstdint>
|
||||
#include <optional>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief This is a base class for any ETL service implementations.
|
||||
@@ -77,7 +77,7 @@ struct ETLServiceInterface {
|
||||
* @brief Get the etl nodes' state
|
||||
* @return The etl nodes' state, nullopt if etl nodes are not connected
|
||||
*/
|
||||
[[nodiscard]] virtual std::optional<etl::ETLState>
|
||||
[[nodiscard]] virtual std::optional<ETLState>
|
||||
getETLState() const = 0;
|
||||
|
||||
/**
|
||||
@@ -89,4 +89,4 @@ struct ETLServiceInterface {
|
||||
lastCloseAgeSeconds() const = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -19,12 +19,12 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
#include <optional>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief An interface for the Extractor
|
||||
@@ -51,4 +51,4 @@ struct ExtractorInterface {
|
||||
extractLedgerOnly(uint32_t seq) = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -19,7 +19,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
|
||||
@@ -28,7 +28,7 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief The interface for observing the initial ledger load
|
||||
@@ -51,4 +51,4 @@ struct InitialLoadObserverInterface {
|
||||
) = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -23,7 +23,7 @@
|
||||
#include <cstdint>
|
||||
#include <optional>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief The interface of a scheduler for the extraction process
|
||||
@@ -71,4 +71,4 @@ struct LedgerPublisherInterface {
|
||||
lastPublishAgeSeconds() const = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -21,9 +21,10 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
@@ -64,7 +65,7 @@ using util::prometheus::Labels;
|
||||
|
||||
namespace etl {
|
||||
|
||||
std::shared_ptr<etlng::LoadBalancerInterface>
|
||||
std::shared_ptr<LoadBalancerInterface>
|
||||
LoadBalancer::makeLoadBalancer(
|
||||
ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
@@ -158,7 +159,6 @@ LoadBalancer::LoadBalancer(
|
||||
auto source = sourceFactory(
|
||||
*it,
|
||||
ioc,
|
||||
backend,
|
||||
subscriptions,
|
||||
validatedLedgers,
|
||||
forwardingTimeout,
|
||||
@@ -212,26 +212,32 @@ LoadBalancer::LoadBalancer(
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<std::string>
|
||||
LoadBalancer::loadInitialLedger(uint32_t sequence, std::chrono::steady_clock::duration retryAfter)
|
||||
InitialLedgerLoadResult
|
||||
LoadBalancer::loadInitialLedger(
|
||||
uint32_t sequence,
|
||||
InitialLoadObserverInterface& loadObserver,
|
||||
std::chrono::steady_clock::duration retryAfter
|
||||
)
|
||||
{
|
||||
std::vector<std::string> response;
|
||||
execute(
|
||||
[this, &response, &sequence](auto& source) {
|
||||
auto [data, res] = source->loadInitialLedger(sequence, downloadRanges_);
|
||||
InitialLedgerLoadResult response;
|
||||
|
||||
if (!res) {
|
||||
execute(
|
||||
[this, &response, &sequence, &loadObserver](auto& source) {
|
||||
auto res = source->loadInitialLedger(sequence, downloadRanges_, loadObserver);
|
||||
|
||||
if (not res.has_value() and res.error() == InitialLedgerLoadError::Errored) {
|
||||
LOG(log_.error()) << "Failed to download initial ledger."
|
||||
<< " Sequence = " << sequence << " source = " << source->toString();
|
||||
} else {
|
||||
response = std::move(data);
|
||||
return false; // should retry on error
|
||||
}
|
||||
|
||||
return res;
|
||||
response = std::move(res); // cancelled or data received
|
||||
return true;
|
||||
},
|
||||
sequence,
|
||||
retryAfter
|
||||
);
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
|
||||
@@ -21,13 +21,12 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Mutex.hpp"
|
||||
#include "util/Random.hpp"
|
||||
#include "util/ResponseExpirationCache.hpp"
|
||||
@@ -54,7 +53,6 @@
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl {
|
||||
@@ -76,7 +74,7 @@ concept SomeLoadBalancer = std::derived_from<T, LoadBalancerTag>;
|
||||
* which ledgers have been validated by the network, and the range of ledgers each etl source has). This class also
|
||||
* allows requests for ledger data to be load balanced across all possible ETL sources.
|
||||
*/
|
||||
class LoadBalancer : public etlng::LoadBalancerInterface, LoadBalancerTag {
|
||||
class LoadBalancer : public LoadBalancerInterface, LoadBalancerTag {
|
||||
public:
|
||||
using RawLedgerObjectType = org::xrpl::rpc::v1::RawLedgerObject;
|
||||
using GetLedgerResponseType = org::xrpl::rpc::v1::GetLedgerResponse;
|
||||
@@ -149,7 +147,7 @@ public:
|
||||
* @param backend BackendInterface implementation
|
||||
* @param subscriptions Subscription manager
|
||||
* @param randomGenerator A random generator to use for selecting sources
|
||||
* @param validatedLedgers The network validated ledgers data structure
|
||||
* @param validatedLedgers The network validated ledgers datastructure
|
||||
* @param sourceFactory A factory function to create a source
|
||||
* @return A shared pointer to a new instance of LoadBalancer
|
||||
*/
|
||||
@@ -164,20 +162,6 @@ public:
|
||||
SourceFactory sourceFactory = makeSource
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Load the initial ledger, writing data to the queue.
|
||||
* @note This function will retry indefinitely until the ledger is downloaded.
|
||||
*
|
||||
* @param sequence Sequence of ledger to download
|
||||
* @param retryAfter Time to wait between retries (2 seconds by default)
|
||||
* @return A std::vector<std::string> The ledger data
|
||||
*/
|
||||
std::vector<std::string>
|
||||
loadInitialLedger(
|
||||
uint32_t sequence,
|
||||
std::chrono::steady_clock::duration retryAfter = std::chrono::seconds{2}
|
||||
) override;
|
||||
|
||||
/**
|
||||
* @brief Load the initial ledger, writing data to the queue.
|
||||
* @note This function will retry indefinitely until the ledger is downloaded or the download is cancelled.
|
||||
@@ -187,16 +171,12 @@ public:
|
||||
* @param retryAfter Time to wait between retries (2 seconds by default)
|
||||
* @return A std::expected with ledger edge keys on success, or InitialLedgerLoadError on failure
|
||||
*/
|
||||
etlng::InitialLedgerLoadResult
|
||||
InitialLedgerLoadResult
|
||||
loadInitialLedger(
|
||||
[[maybe_unused]] uint32_t sequence,
|
||||
[[maybe_unused]] etlng::InitialLoadObserverInterface& observer,
|
||||
[[maybe_unused]] std::chrono::steady_clock::duration retryAfter
|
||||
) override
|
||||
{
|
||||
ASSERT(false, "Not available for old ETL");
|
||||
std::unreachable();
|
||||
}
|
||||
uint32_t sequence,
|
||||
InitialLoadObserverInterface& observer,
|
||||
std::chrono::steady_clock::duration retryAfter
|
||||
) override;
|
||||
|
||||
/**
|
||||
* @brief Fetch data for a specific ledger.
|
||||
|
||||
@@ -21,7 +21,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
@@ -37,7 +37,7 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief Represents possible errors for initial ledger load
|
||||
@@ -76,21 +76,10 @@ public:
|
||||
[[nodiscard]] virtual InitialLedgerLoadResult
|
||||
loadInitialLedger(
|
||||
uint32_t sequence,
|
||||
etlng::InitialLoadObserverInterface& loader,
|
||||
InitialLoadObserverInterface& loader,
|
||||
std::chrono::steady_clock::duration retryAfter = std::chrono::seconds{2}
|
||||
) = 0;
|
||||
|
||||
/**
|
||||
* @brief Load the initial ledger, writing data to the queue.
|
||||
* @note This function will retry indefinitely until the ledger is downloaded.
|
||||
*
|
||||
* @param sequence Sequence of ledger to download
|
||||
* @param retryAfter Time to wait between retries (2 seconds by default)
|
||||
* @return A std::vector<std::string> The ledger data
|
||||
*/
|
||||
[[nodiscard]] virtual std::vector<std::string>
|
||||
loadInitialLedger(uint32_t sequence, std::chrono::steady_clock::duration retryAfter = std::chrono::seconds{2}) = 0;
|
||||
|
||||
/**
|
||||
* @brief Fetch data for a specific ledger.
|
||||
*
|
||||
@@ -141,7 +130,7 @@ public:
|
||||
* @brief Return state of ETL nodes.
|
||||
* @return ETL state, nullopt if etl nodes not available
|
||||
*/
|
||||
[[nodiscard]] virtual std::optional<etl::ETLState>
|
||||
[[nodiscard]] virtual std::optional<ETLState>
|
||||
getETLState() noexcept = 0;
|
||||
|
||||
/**
|
||||
@@ -154,4 +143,4 @@ public:
|
||||
stop(boost::asio::yield_context yield) = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -19,14 +19,14 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
|
||||
#include <expected>
|
||||
#include <optional>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief Enumeration of possible errors that can occur during loading operations
|
||||
@@ -59,4 +59,4 @@ struct LoaderInterface {
|
||||
loadInitialLedger(model::LedgerData const& data) = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -38,7 +38,7 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::model {
|
||||
namespace etl::model {
|
||||
|
||||
/**
|
||||
* @brief A specification for the Registry.
|
||||
@@ -190,4 +190,4 @@ struct Task {
|
||||
uint32_t seq;
|
||||
};
|
||||
|
||||
} // namespace etlng::model
|
||||
} // namespace etl::model
|
||||
@@ -26,7 +26,7 @@
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief An interface for the monitor service
|
||||
@@ -88,4 +88,4 @@ public:
|
||||
stop() = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -20,15 +20,15 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief An interface for providing Monitor instances
|
||||
@@ -55,10 +55,10 @@ struct MonitorProviderInterface {
|
||||
make(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
uint32_t startSequence,
|
||||
std::chrono::steady_clock::duration dbStalledReportDelay = kDEFAULT_DB_STALLED_REPORT_DELAY
|
||||
) = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -6,7 +6,7 @@ A single Clio node has one or more ETL sources specified in the config file. Cli
|
||||
|
||||
Upon receiving a message on the stream, Clio fetches the data associated with the newly validated ledger from one of the ETL sources. The fetch is performed via a gRPC request called `GetLedger`. This request returns the ledger header, transactions and metadata blobs, and every ledger object added/modified/deleted as part of this ledger. The ETL subsystem then writes all of this data to the databases, and moves on to the next ledger.
|
||||
|
||||
If the database is not empty, clio will first come up in a "soft" read-only mode. In read-only mode, the server does not perform ETL and simply publishes new ledgers as they are written to the database. If the database is not updated within a certain time period (currently hard coded at 20 seconds), clio will begin the ETL process and start writing to the database. The database will report an error when trying to write a record with a key that already exists. ETL uses this error to determine that another process is writing to the database, and subsequently falls back to a soft read-only mode. clio can also operate in strict read-only mode, in which case they will never write to the database.
|
||||
If the database is not empty, clio will first come up in a "soft" read-only mode. In read-only mode, the server does not perform ETL and simply publishes new ledgers as they are written to the database. If the database is not updated within a certain time period (currently hard coded at 10 seconds), clio will begin the ETL process and start writing to the database. The database will report an error when trying to write a record with a key that already exists. ETL uses this error to determine that another process is writing to the database, and subsequently falls back to a soft read-only mode. clio can also operate in strict read-only mode, in which case they will never write to the database.
|
||||
|
||||
## Ledger cache
|
||||
|
||||
|
||||
@@ -19,13 +19,13 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief The interface for a registry that can dispatch transactions and objects to extensions.
|
||||
@@ -36,15 +36,15 @@ namespace etlng {
|
||||
* The registry itself consists of Extensions.
|
||||
* Each extension must define at least one valid hook:
|
||||
* - for ongoing ETL dispatch:
|
||||
* - void onLedgerData(etlng::model::LedgerData const&)
|
||||
* - void onTransaction(uint32_t, etlng::model::Transaction const&)
|
||||
* - void onObject(uint32_t, etlng::model::Object const&)
|
||||
* - void onLedgerData(etl::model::LedgerData const&)
|
||||
* - void onTransaction(uint32_t, etl::model::Transaction const&)
|
||||
* - void onObject(uint32_t, etl::model::Object const&)
|
||||
* - for initial ledger load
|
||||
* - void onInitialData(etlng::model::LedgerData const&)
|
||||
* - void onInitialTransaction(uint32_t, etlng::model::Transaction const&)
|
||||
* - void onInitialData(etl::model::LedgerData const&)
|
||||
* - void onInitialTransaction(uint32_t, etl::model::Transaction const&)
|
||||
* - for initial objects (called for each downloaded batch)
|
||||
* - void onInitialObjects(uint32_t, std::vector<etlng::model::Object> const&, std::string)
|
||||
* - void onInitialObject(uint32_t, etlng::model::Object const&)
|
||||
* - void onInitialObjects(uint32_t, std::vector<etl::model::Object> const&, std::string)
|
||||
* - void onInitialObject(uint32_t, etl::model::Object const&)
|
||||
*
|
||||
* When the registry dispatches (initial)data or objects, each of the above hooks will be called in order on each
|
||||
* registered extension.
|
||||
@@ -57,7 +57,7 @@ namespace etlng {
|
||||
* The specification is setup like so:
|
||||
* @code{.cpp}
|
||||
* struct Ext {
|
||||
* using spec = etlng::model::Spec<
|
||||
* using spec = etl::model::Spec<
|
||||
* ripple::TxType::ttNFTOKEN_BURN,
|
||||
* ripple::TxType::ttNFTOKEN_ACCEPT_OFFER,
|
||||
* ripple::TxType::ttNFTOKEN_CREATE_OFFER,
|
||||
@@ -65,7 +65,7 @@ namespace etlng {
|
||||
* ripple::TxType::ttNFTOKEN_MINT>;
|
||||
*
|
||||
* static void
|
||||
* onInitialTransaction(uint32_t, etlng::model::Transaction const&);
|
||||
* onInitialTransaction(uint32_t, etl::model::Transaction const&);
|
||||
* };
|
||||
* @endcode
|
||||
*/
|
||||
@@ -105,4 +105,4 @@ struct RegistryInterface {
|
||||
dispatch(model::LedgerData const& data) = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -19,11 +19,11 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
|
||||
#include <optional>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief The interface of a scheduler for the extraction process
|
||||
@@ -39,4 +39,4 @@ struct SchedulerInterface {
|
||||
next() = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -19,7 +19,6 @@
|
||||
|
||||
#include "etl/Source.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/impl/ForwardingSource.hpp"
|
||||
#include "etl/impl/GrpcSource.hpp"
|
||||
@@ -41,7 +40,6 @@ SourcePtr
|
||||
makeSource(
|
||||
util::config::ObjectView const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
@@ -55,7 +53,7 @@ makeSource(
|
||||
auto const grpcPort = config.get<std::string>("grpc_port");
|
||||
|
||||
impl::ForwardingSource forwardingSource{ip, wsPort, forwardingTimeout};
|
||||
impl::GrpcSource grpcSource{ip, grpcPort, std::move(backend)};
|
||||
impl::GrpcSource grpcSource{ip, grpcPort};
|
||||
auto subscriptionSource = std::make_unique<impl::SubscriptionSource>(
|
||||
ioc,
|
||||
ip,
|
||||
|
||||
@@ -19,7 +19,8 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
@@ -47,7 +48,6 @@ namespace etl {
|
||||
|
||||
/**
|
||||
* @brief Provides an implementation of a ETL source
|
||||
*
|
||||
*/
|
||||
class SourceBase {
|
||||
public:
|
||||
@@ -77,7 +77,7 @@ public:
|
||||
*
|
||||
* @return true if source is connected; false otherwise
|
||||
*/
|
||||
virtual bool
|
||||
[[nodiscard]] virtual bool
|
||||
isConnected() const = 0;
|
||||
|
||||
/**
|
||||
@@ -93,11 +93,11 @@ public:
|
||||
*
|
||||
* @return JSON representation of the source
|
||||
*/
|
||||
virtual boost::json::object
|
||||
[[nodiscard]] virtual boost::json::object
|
||||
toJson() const = 0;
|
||||
|
||||
/** @return String representation of the source (for debug) */
|
||||
virtual std::string
|
||||
[[nodiscard]] virtual std::string
|
||||
toString() const = 0;
|
||||
|
||||
/**
|
||||
@@ -106,7 +106,7 @@ public:
|
||||
* @param sequence The ledger sequence to check
|
||||
* @return true if ledger is in the range of this source; false otherwise
|
||||
*/
|
||||
virtual bool
|
||||
[[nodiscard]] virtual bool
|
||||
hasLedger(uint32_t sequence) const = 0;
|
||||
|
||||
/**
|
||||
@@ -120,7 +120,7 @@ public:
|
||||
* @param getObjectNeighbors Whether to request object neighbors; defaults to false
|
||||
* @return A std::pair of the response status and the response itself
|
||||
*/
|
||||
virtual std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>
|
||||
[[nodiscard]] virtual std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>
|
||||
fetchLedger(uint32_t sequence, bool getObjects = true, bool getObjectNeighbors = false) = 0;
|
||||
|
||||
/**
|
||||
@@ -128,10 +128,11 @@ public:
|
||||
*
|
||||
* @param sequence Sequence of the ledger to download
|
||||
* @param numMarkers Number of markers to generate for async calls
|
||||
* @param loader InitialLoadObserverInterface implementation
|
||||
* @return A std::pair of the data and a bool indicating whether the download was successful
|
||||
*/
|
||||
virtual std::pair<std::vector<std::string>, bool>
|
||||
loadInitialLedger(uint32_t sequence, std::uint32_t numMarkers) = 0;
|
||||
virtual InitialLedgerLoadResult
|
||||
loadInitialLedger(uint32_t sequence, std::uint32_t numMarkers, InitialLoadObserverInterface& loader) = 0;
|
||||
|
||||
/**
|
||||
* @brief Forward a request to rippled.
|
||||
@@ -142,7 +143,7 @@ public:
|
||||
* @param yield The coroutine context
|
||||
* @return Response on success or error on failure
|
||||
*/
|
||||
virtual std::expected<boost::json::object, rpc::ClioError>
|
||||
[[nodiscard]] virtual std::expected<boost::json::object, rpc::ClioError>
|
||||
forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& forwardToRippledClientIp,
|
||||
@@ -156,7 +157,6 @@ using SourcePtr = std::unique_ptr<SourceBase>;
|
||||
using SourceFactory = std::function<SourcePtr(
|
||||
util::config::ObjectView const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
@@ -170,7 +170,6 @@ using SourceFactory = std::function<SourcePtr(
|
||||
*
|
||||
* @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 data structure
|
||||
* @param forwardingTimeout The timeout for forwarding to rippled
|
||||
@@ -180,11 +179,10 @@ using SourceFactory = std::function<SourcePtr(
|
||||
* as forwarding.
|
||||
* @return The created source
|
||||
*/
|
||||
SourcePtr
|
||||
[[nodiscard]] SourcePtr
|
||||
makeSource(
|
||||
util::config::ObjectView const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
|
||||
@@ -21,7 +21,7 @@
|
||||
|
||||
#include <cstddef>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief An interface for the Task Manager
|
||||
@@ -43,4 +43,4 @@ struct TaskManagerInterface {
|
||||
stop() = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -19,8 +19,8 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "etlng/TaskManagerInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/TaskManagerInterface.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
|
||||
#include <cstddef>
|
||||
@@ -29,7 +29,7 @@
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
|
||||
namespace etlng {
|
||||
namespace etl {
|
||||
|
||||
/**
|
||||
* @brief An interface for providing the Task Manager
|
||||
@@ -55,4 +55,4 @@ struct TaskManagerProviderInterface {
|
||||
) = 0;
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
} // namespace etl
|
||||
@@ -20,12 +20,12 @@
|
||||
#include "etl/impl/AmendmentBlockHandler.hpp"
|
||||
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
#include <utility>
|
||||
|
||||
namespace etl::impl {
|
||||
@@ -37,20 +37,35 @@ AmendmentBlockHandler::ActionType const AmendmentBlockHandler::kDEFAULT_AMENDMEN
|
||||
};
|
||||
|
||||
AmendmentBlockHandler::AmendmentBlockHandler(
|
||||
boost::asio::io_context& ioc,
|
||||
util::async::AnyExecutionContext ctx,
|
||||
SystemState& state,
|
||||
std::chrono::steady_clock::duration interval,
|
||||
ActionType action
|
||||
)
|
||||
: state_{std::ref(state)}, repeat_{ioc}, interval_{interval}, action_{std::move(action)}
|
||||
: state_{std::ref(state)}, interval_{interval}, ctx_{std::move(ctx)}, action_{std::move(action)}
|
||||
{
|
||||
}
|
||||
|
||||
AmendmentBlockHandler::~AmendmentBlockHandler()
|
||||
{
|
||||
stop();
|
||||
}
|
||||
|
||||
void
|
||||
AmendmentBlockHandler::notifyAmendmentBlocked()
|
||||
{
|
||||
state_.get().isAmendmentBlocked = true;
|
||||
repeat_.start(interval_, action_);
|
||||
if (not operation_.has_value())
|
||||
operation_.emplace(ctx_.executeRepeatedly(interval_, action_));
|
||||
}
|
||||
|
||||
void
|
||||
AmendmentBlockHandler::stop()
|
||||
{
|
||||
if (operation_.has_value()) {
|
||||
operation_->abort();
|
||||
operation_.reset();
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etl::impl
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2023, 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,8 +19,10 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/AmendmentBlockHandlerInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "util/Repeat.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/async/AnyOperation.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/post.hpp>
|
||||
@@ -28,17 +30,19 @@
|
||||
|
||||
#include <chrono>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
class AmendmentBlockHandler {
|
||||
class AmendmentBlockHandler : public AmendmentBlockHandlerInterface {
|
||||
public:
|
||||
using ActionType = std::function<void()>;
|
||||
|
||||
private:
|
||||
std::reference_wrapper<SystemState> state_;
|
||||
util::Repeat repeat_;
|
||||
std::chrono::steady_clock::duration interval_;
|
||||
util::async::AnyExecutionContext ctx_;
|
||||
std::optional<util::async::AnyOperation<void>> operation_;
|
||||
|
||||
ActionType action_;
|
||||
|
||||
@@ -46,14 +50,19 @@ public:
|
||||
static ActionType const kDEFAULT_AMENDMENT_BLOCK_ACTION;
|
||||
|
||||
AmendmentBlockHandler(
|
||||
boost::asio::io_context& ioc,
|
||||
util::async::AnyExecutionContext ctx,
|
||||
SystemState& state,
|
||||
std::chrono::steady_clock::duration interval = std::chrono::seconds{1},
|
||||
ActionType action = kDEFAULT_AMENDMENT_BLOCK_ACTION
|
||||
);
|
||||
|
||||
~AmendmentBlockHandler() override;
|
||||
|
||||
void
|
||||
notifyAmendmentBlocked();
|
||||
stop() override;
|
||||
|
||||
void
|
||||
notifyAmendmentBlocked() override;
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
|
||||
@@ -1,221 +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.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etl/MPTHelpers.hpp"
|
||||
#include "etl/NFTHelpers.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <grpcpp/client_context.h>
|
||||
#include <grpcpp/grpcpp.h>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger_data.pb.h>
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
#include <xrpl/basics/strHex.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
class AsyncCallData {
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
std::unique_ptr<org::xrpl::rpc::v1::GetLedgerDataResponse> cur_;
|
||||
std::unique_ptr<org::xrpl::rpc::v1::GetLedgerDataResponse> next_;
|
||||
|
||||
org::xrpl::rpc::v1::GetLedgerDataRequest request_;
|
||||
std::unique_ptr<grpc::ClientContext> context_;
|
||||
|
||||
grpc::Status status_;
|
||||
unsigned char nextPrefix_;
|
||||
|
||||
std::string lastKey_;
|
||||
|
||||
public:
|
||||
AsyncCallData(uint32_t seq, ripple::uint256 const& marker, std::optional<ripple::uint256> const& nextMarker)
|
||||
{
|
||||
request_.mutable_ledger()->set_sequence(seq);
|
||||
if (marker.isNonZero()) {
|
||||
request_.set_marker(marker.data(), ripple::uint256::size());
|
||||
}
|
||||
request_.set_user("ETL");
|
||||
nextPrefix_ = 0x00;
|
||||
if (nextMarker)
|
||||
nextPrefix_ = nextMarker->data()[0];
|
||||
|
||||
unsigned char const prefix = marker.data()[0];
|
||||
|
||||
LOG(log_.debug()) << "Setting up AsyncCallData. marker = " << ripple::strHex(marker)
|
||||
<< " . prefix = " << ripple::strHex(std::string(1, prefix))
|
||||
<< " . nextPrefix_ = " << ripple::strHex(std::string(1, nextPrefix_));
|
||||
|
||||
ASSERT(
|
||||
nextPrefix_ > prefix || nextPrefix_ == 0x00,
|
||||
"Next prefix must be greater than current prefix. Got: nextPrefix_ = {}, prefix = {}",
|
||||
nextPrefix_,
|
||||
prefix
|
||||
);
|
||||
|
||||
cur_ = std::make_unique<org::xrpl::rpc::v1::GetLedgerDataResponse>();
|
||||
next_ = std::make_unique<org::xrpl::rpc::v1::GetLedgerDataResponse>();
|
||||
context_ = std::make_unique<grpc::ClientContext>();
|
||||
}
|
||||
|
||||
enum class CallStatus { MORE, DONE, ERRORED };
|
||||
|
||||
CallStatus
|
||||
process(
|
||||
std::unique_ptr<org::xrpl::rpc::v1::XRPLedgerAPIService::Stub>& stub,
|
||||
grpc::CompletionQueue& cq,
|
||||
BackendInterface& backend,
|
||||
bool abort,
|
||||
bool cacheOnly = false
|
||||
)
|
||||
{
|
||||
LOG(log_.trace()) << "Processing response. "
|
||||
<< "Marker prefix = " << getMarkerPrefix();
|
||||
if (abort) {
|
||||
LOG(log_.error()) << "AsyncCallData aborted";
|
||||
return CallStatus::ERRORED;
|
||||
}
|
||||
if (!status_.ok()) {
|
||||
LOG(log_.error()) << "AsyncCallData status_ not ok: code = " << status_.error_code()
|
||||
<< " message = " << status_.error_message();
|
||||
return CallStatus::ERRORED;
|
||||
}
|
||||
if (!next_->is_unlimited()) {
|
||||
LOG(log_.warn()) << "AsyncCallData is_unlimited is false. "
|
||||
<< "Make sure secure_gateway is set correctly at the ETL source";
|
||||
}
|
||||
|
||||
std::swap(cur_, next_);
|
||||
|
||||
bool more = true;
|
||||
|
||||
// if no marker returned, we are done
|
||||
if (cur_->marker().empty())
|
||||
more = false;
|
||||
|
||||
// if returned marker is greater than our end, we are done
|
||||
unsigned char const prefix = cur_->marker()[0];
|
||||
if (nextPrefix_ != 0x00 && prefix >= nextPrefix_)
|
||||
more = false;
|
||||
|
||||
// if we are not done, make the next async call
|
||||
if (more) {
|
||||
request_.set_marker(cur_->marker());
|
||||
call(stub, cq);
|
||||
}
|
||||
|
||||
auto const numObjects = cur_->ledger_objects().objects_size();
|
||||
LOG(log_.debug()) << "Writing " << numObjects << " objects";
|
||||
|
||||
std::vector<data::LedgerObject> cacheUpdates;
|
||||
cacheUpdates.reserve(numObjects);
|
||||
|
||||
for (int i = 0; i < numObjects; ++i) {
|
||||
auto& obj = *(cur_->mutable_ledger_objects()->mutable_objects(i));
|
||||
if (!more && nextPrefix_ != 0x00) {
|
||||
if (static_cast<unsigned char>(obj.key()[0]) >= nextPrefix_)
|
||||
continue;
|
||||
}
|
||||
cacheUpdates.push_back(
|
||||
{*ripple::uint256::fromVoidChecked(obj.key()), {obj.data().begin(), obj.data().end()}}
|
||||
);
|
||||
if (!cacheOnly) {
|
||||
if (!lastKey_.empty())
|
||||
backend.writeSuccessor(std::move(lastKey_), request_.ledger().sequence(), std::string{obj.key()});
|
||||
lastKey_ = obj.key();
|
||||
backend.writeNFTs(getNFTDataFromObj(request_.ledger().sequence(), obj.key(), obj.data()));
|
||||
|
||||
auto const maybeMPTHolder = getMPTHolderFromObj(obj.key(), obj.data());
|
||||
if (maybeMPTHolder)
|
||||
backend.writeMPTHolders({*maybeMPTHolder});
|
||||
|
||||
backend.writeLedgerObject(
|
||||
std::move(*obj.mutable_key()), request_.ledger().sequence(), std::move(*obj.mutable_data())
|
||||
);
|
||||
}
|
||||
}
|
||||
backend.cache().update(cacheUpdates, request_.ledger().sequence(), cacheOnly);
|
||||
LOG(log_.debug()) << "Wrote " << numObjects << " objects. Got more: " << (more ? "YES" : "NO");
|
||||
|
||||
return more ? CallStatus::MORE : CallStatus::DONE;
|
||||
}
|
||||
|
||||
void
|
||||
call(std::unique_ptr<org::xrpl::rpc::v1::XRPLedgerAPIService::Stub>& stub, grpc::CompletionQueue& cq)
|
||||
{
|
||||
context_ = std::make_unique<grpc::ClientContext>();
|
||||
|
||||
std::unique_ptr<grpc::ClientAsyncResponseReader<org::xrpl::rpc::v1::GetLedgerDataResponse>> rpc(
|
||||
stub->PrepareAsyncGetLedgerData(context_.get(), request_, &cq)
|
||||
);
|
||||
|
||||
rpc->StartCall();
|
||||
|
||||
rpc->Finish(next_.get(), &status_, this);
|
||||
}
|
||||
|
||||
std::string
|
||||
getMarkerPrefix()
|
||||
{
|
||||
if (next_->marker().empty()) {
|
||||
return "";
|
||||
}
|
||||
return ripple::strHex(std::string{next_->marker().data()[0]});
|
||||
}
|
||||
|
||||
std::string
|
||||
getLastKey()
|
||||
{
|
||||
return lastKey_;
|
||||
}
|
||||
};
|
||||
|
||||
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
|
||||
@@ -17,12 +17,12 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/AsyncGrpcCall.hpp"
|
||||
#include "etl/impl/AsyncGrpcCall.hpp"
|
||||
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/impl/Extraction.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/impl/Extraction.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
@@ -41,7 +41,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
AsyncGrpcCall::AsyncGrpcCall(
|
||||
uint32_t seq,
|
||||
@@ -78,7 +78,7 @@ AsyncGrpcCall::CallStatus
|
||||
AsyncGrpcCall::process(
|
||||
std::unique_ptr<AsyncGrpcCall::StubType>& stub,
|
||||
grpc::CompletionQueue& cq,
|
||||
etlng::InitialLoadObserverInterface& loader,
|
||||
InitialLoadObserverInterface& loader,
|
||||
bool abort
|
||||
)
|
||||
{
|
||||
@@ -121,7 +121,7 @@ AsyncGrpcCall::process(
|
||||
}
|
||||
|
||||
auto const numObjects = cur_->ledger_objects().objects_size();
|
||||
std::vector<etlng::model::Object> data;
|
||||
std::vector<model::Object> data;
|
||||
data.reserve(numObjects);
|
||||
|
||||
for (int i = 0; i < numObjects; ++i) {
|
||||
@@ -132,7 +132,7 @@ AsyncGrpcCall::process(
|
||||
}
|
||||
|
||||
lastKey_ = obj.key(); // this will end up the last key we actually touched eventually
|
||||
data.push_back(etlng::impl::extractObj(std::move(obj)));
|
||||
data.push_back(impl::extractObj(std::move(obj)));
|
||||
}
|
||||
|
||||
if (not data.empty())
|
||||
@@ -171,7 +171,7 @@ AsyncGrpcCall::getLastKey()
|
||||
std::vector<AsyncGrpcCall>
|
||||
AsyncGrpcCall::makeAsyncCalls(uint32_t const sequence, uint32_t const numMarkers)
|
||||
{
|
||||
auto const markers = etl::getMarkers(numMarkers);
|
||||
auto const markers = getMarkers(numMarkers);
|
||||
|
||||
std::vector<AsyncGrpcCall> result;
|
||||
result.reserve(markers.size());
|
||||
@@ -185,4 +185,4 @@ AsyncGrpcCall::makeAsyncCalls(uint32_t const sequence, uint32_t const numMarkers
|
||||
return result;
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,7 +19,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <grpcpp/client_context.h>
|
||||
@@ -34,7 +34,7 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class AsyncGrpcCall {
|
||||
public:
|
||||
@@ -68,7 +68,7 @@ public:
|
||||
process(
|
||||
std::unique_ptr<StubType>& stub,
|
||||
grpc::CompletionQueue& cq,
|
||||
etlng::InitialLoadObserverInterface& loader,
|
||||
InitialLoadObserverInterface& loader,
|
||||
bool abort
|
||||
);
|
||||
|
||||
@@ -82,4 +82,4 @@ public:
|
||||
getLastKey();
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -53,7 +53,7 @@ class CacheLoaderImpl {
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::reference_wrapper<CacheType> cache_;
|
||||
|
||||
etl::ThreadSafeQueue<CursorPair> queue_;
|
||||
ThreadSafeQueue<CursorPair> queue_;
|
||||
std::atomic_int16_t remaining_;
|
||||
|
||||
std::chrono::steady_clock::time_point startTime_ = std::chrono::steady_clock::now();
|
||||
|
||||
@@ -21,15 +21,15 @@
|
||||
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/CacheUpdaterInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/CacheUpdaterInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class CacheUpdater : public CacheUpdaterInterface {
|
||||
std::reference_wrapper<data::LedgerCacheInterface> cache_;
|
||||
@@ -66,4 +66,4 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -17,13 +17,13 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/Extraction.hpp"
|
||||
#include "etl/impl/Extraction.hpp"
|
||||
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/LedgerFetcherInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/impl/LedgerFetcher.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/LedgerUtils.hpp"
|
||||
#include "util/Profiler.hpp"
|
||||
@@ -47,7 +47,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
model::Object::ModType
|
||||
extractModType(PBModType type)
|
||||
@@ -216,4 +216,4 @@ Extractor::extractLedgerOnly(uint32_t seq)
|
||||
return batch;
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,10 +19,10 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/ExtractorInterface.hpp"
|
||||
#include "etl/LedgerFetcherInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/impl/LedgerFetcher.hpp"
|
||||
#include "etlng/ExtractorInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <google/protobuf/repeated_ptr_field.h>
|
||||
@@ -44,7 +44,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
using PBObjType = org::xrpl::rpc::v1::RawLedgerObject;
|
||||
using PBModType = PBObjType::ModificationType;
|
||||
@@ -77,7 +77,7 @@ maybeExtractSuccessors(PBLedgerResponseType const& data);
|
||||
|
||||
// fetches the data in gRPC and transforms to local representation
|
||||
class Extractor : public ExtractorInterface {
|
||||
std::shared_ptr<etl::LedgerFetcherInterface> fetcher_;
|
||||
std::shared_ptr<LedgerFetcherInterface> fetcher_;
|
||||
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
@@ -86,7 +86,7 @@ private:
|
||||
unpack();
|
||||
|
||||
public:
|
||||
Extractor(std::shared_ptr<etl::LedgerFetcherInterface> fetcher) : fetcher_(std::move(fetcher))
|
||||
Extractor(std::shared_ptr<LedgerFetcherInterface> fetcher) : fetcher_(std::move(fetcher))
|
||||
{
|
||||
}
|
||||
|
||||
@@ -104,4 +104,4 @@ public:
|
||||
extractLedgerOnly(uint32_t seq) override;
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -1,136 +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.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/ETLHelpers.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <vector>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
/**
|
||||
* @brief A collection of thread safe async queues used by Extractor and Transformer to communicate
|
||||
*/
|
||||
template <typename RawDataType>
|
||||
class ExtractionDataPipe {
|
||||
public:
|
||||
using DataType = std::optional<RawDataType>;
|
||||
using QueueType = ThreadSafeQueue<DataType>; // TODO: probably should use boost::lockfree::queue instead?
|
||||
|
||||
static constexpr auto kTOTAL_MAX_IN_QUEUE = 1000u;
|
||||
|
||||
private:
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
uint32_t stride_;
|
||||
uint32_t startSequence_;
|
||||
|
||||
std::vector<std::shared_ptr<QueueType>> queues_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create a new instance of the extraction data pipe
|
||||
*
|
||||
* @param stride
|
||||
* @param startSequence
|
||||
*/
|
||||
ExtractionDataPipe(uint32_t stride, uint32_t startSequence) : stride_{stride}, startSequence_{startSequence}
|
||||
{
|
||||
auto const maxQueueSize = kTOTAL_MAX_IN_QUEUE / stride;
|
||||
for (size_t i = 0; i < stride_; ++i)
|
||||
queues_.push_back(std::make_unique<QueueType>(maxQueueSize));
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Push new data package for the specified sequence.
|
||||
*
|
||||
* Note: Potentially blocks until the underlying queue can accommodate another entry.
|
||||
*
|
||||
* @param sequence The sequence for which to enqueue the data package
|
||||
* @param data The data to store
|
||||
*/
|
||||
void
|
||||
push(uint32_t sequence, DataType&& data)
|
||||
{
|
||||
getQueue(sequence)->push(std::move(data));
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get data package for the given sequence
|
||||
*
|
||||
* Note: Potentially blocks until data is available.
|
||||
*
|
||||
* @param sequence The sequence for which data is required
|
||||
* @return The data wrapped in an optional; nullopt means that there is no more data to expect
|
||||
*/
|
||||
DataType
|
||||
popNext(uint32_t sequence)
|
||||
{
|
||||
return getQueue(sequence)->pop();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Get the stride
|
||||
*/
|
||||
uint32_t
|
||||
getStride() const
|
||||
{
|
||||
return stride_;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Hint the Transformer that the queue is done sending data
|
||||
* @param sequence The sequence for which the extractor queue is to be hinted
|
||||
*/
|
||||
void
|
||||
finish(uint32_t sequence)
|
||||
{
|
||||
// empty optional hints the Transformer to shut down
|
||||
push(sequence, std::nullopt);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Unblocks internal queues
|
||||
*
|
||||
* Note: For now this must be called by the ETL when Transformer exits.
|
||||
*/
|
||||
void
|
||||
cleanup()
|
||||
{
|
||||
// TODO: this should not have to be called by hand. it should be done via RAII
|
||||
for (auto i = 0u; i < stride_; ++i)
|
||||
getQueue(i)->tryPop(); // pop from each queue that might be blocked on a push
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<QueueType>
|
||||
getQueue(uint32_t sequence)
|
||||
{
|
||||
LOG(log_.debug()) << "Grabbing extraction queue for " << sequence << "; start was " << startSequence_;
|
||||
return queues_[(sequence - startSequence_) % stride_];
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
@@ -1,147 +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.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Profiler.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <xrpl/beast/core/CurrentThreadName.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <thread>
|
||||
#include <utility>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
/**
|
||||
* @brief Extractor thread that is fetching GRPC data and enqueue it on the DataPipeType
|
||||
*/
|
||||
template <typename DataPipeType, typename LedgerFetcherType>
|
||||
class Extractor {
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
std::reference_wrapper<DataPipeType> pipe_;
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> networkValidatedLedgers_;
|
||||
std::reference_wrapper<LedgerFetcherType> ledgerFetcher_;
|
||||
uint32_t startSequence_;
|
||||
std::optional<uint32_t> finishSequence_;
|
||||
std::reference_wrapper<SystemState const> state_; // shared state for ETL
|
||||
|
||||
std::thread thread_;
|
||||
|
||||
public:
|
||||
Extractor(
|
||||
DataPipeType& pipe,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> networkValidatedLedgers,
|
||||
LedgerFetcherType& ledgerFetcher,
|
||||
uint32_t startSequence,
|
||||
std::optional<uint32_t> finishSequence,
|
||||
SystemState const& state
|
||||
)
|
||||
: pipe_(std::ref(pipe))
|
||||
, networkValidatedLedgers_{std::move(networkValidatedLedgers)}
|
||||
, ledgerFetcher_{std::ref(ledgerFetcher)}
|
||||
, startSequence_{startSequence}
|
||||
, finishSequence_{finishSequence}
|
||||
, state_{std::cref(state)}
|
||||
{
|
||||
thread_ = std::thread([this]() { process(); });
|
||||
}
|
||||
|
||||
~Extractor()
|
||||
{
|
||||
if (thread_.joinable())
|
||||
thread_.join();
|
||||
}
|
||||
|
||||
void
|
||||
waitTillFinished()
|
||||
{
|
||||
ASSERT(thread_.joinable(), "Extractor thread must be joinable");
|
||||
thread_.join();
|
||||
}
|
||||
|
||||
private:
|
||||
void
|
||||
process()
|
||||
{
|
||||
beast::setCurrentThreadName("ETLService extract");
|
||||
|
||||
double totalTime = 0.0;
|
||||
auto currentSequence = startSequence_;
|
||||
|
||||
while (!shouldFinish(currentSequence) &&
|
||||
networkValidatedLedgers_->waitUntilValidatedByNetwork(currentSequence)) {
|
||||
auto [fetchResponse, time] = ::util::timed<std::chrono::duration<double>>([this, currentSequence]() {
|
||||
return ledgerFetcher_.get().fetchDataAndDiff(currentSequence);
|
||||
});
|
||||
totalTime += time;
|
||||
|
||||
// if the fetch is unsuccessful, stop. fetchLedger only returns false if the server is shutting down, or
|
||||
// if the ledger was found in the database (which means another process already wrote the ledger that
|
||||
// this process was trying to extract; this is a form of a write conflict). Otherwise, fetchDataAndDiff
|
||||
// will keep trying to fetch the specified ledger until successful.
|
||||
if (!fetchResponse)
|
||||
break;
|
||||
|
||||
// TODO: extract this part into a strategy perhaps
|
||||
auto const tps = fetchResponse->transactions_list().transactions_size() / time;
|
||||
LOG(log_.info()) << "Extract phase time = " << time << "; Extract phase tps = " << tps
|
||||
<< "; Avg extract time = " << totalTime / (currentSequence - startSequence_ + 1)
|
||||
<< "; seq = " << currentSequence;
|
||||
|
||||
pipe_.get().push(currentSequence, std::move(fetchResponse));
|
||||
currentSequence += pipe_.get().getStride();
|
||||
}
|
||||
|
||||
pipe_.get().finish(startSequence_);
|
||||
}
|
||||
|
||||
[[nodiscard]] bool
|
||||
isStopping() const
|
||||
{
|
||||
return state_.get().isStopping;
|
||||
}
|
||||
|
||||
[[nodiscard]] bool
|
||||
hasWriteConflict() const
|
||||
{
|
||||
return state_.get().writeConflict;
|
||||
}
|
||||
|
||||
[[nodiscard]] bool
|
||||
shouldFinish(uint32_t seq) const
|
||||
{
|
||||
// Stopping conditions:
|
||||
// - if there is a write conflict in the load thread, the ETL mechanism should stop.
|
||||
// - if the entire server is shutting down - this can be detected in a variety of ways.
|
||||
// - when the given sequence is past the finishSequence in case one is specified
|
||||
return hasWriteConflict() || isStopping() || (finishSequence_ && seq > *finishSequence_);
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
@@ -19,13 +19,14 @@
|
||||
|
||||
#include "etl/impl/GrpcSource.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/impl/AsyncData.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/impl/AsyncGrpcCall.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "web/Resolver.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/ip/tcp.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <fmt/format.h>
|
||||
#include <grpc/grpc.h>
|
||||
#include <grpcpp/client_context.h>
|
||||
@@ -35,33 +36,41 @@
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <exception>
|
||||
#include <expected>
|
||||
#include <memory>
|
||||
#include <sstream>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace {
|
||||
|
||||
std::string
|
||||
resolve(std::string const& ip, std::string const& port)
|
||||
{
|
||||
web::Resolver resolver;
|
||||
|
||||
if (auto const results = resolver.resolve(ip, port); not results.empty())
|
||||
return results.at(0);
|
||||
|
||||
throw std::runtime_error("Failed to resolve " + ip + ":" + port);
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
GrpcSource::GrpcSource(std::string const& ip, std::string const& grpcPort, std::shared_ptr<BackendInterface> backend)
|
||||
: log_(fmt::format("GrpcSource[{}:{}]", ip, grpcPort)), backend_(std::move(backend))
|
||||
GrpcSource::GrpcSource(std::string const& ip, std::string const& grpcPort, std::chrono::system_clock::duration deadline)
|
||||
: log_(fmt::format("ETL_Grpc[{}:{}]", ip, grpcPort))
|
||||
, initialLoadShouldStop_(std::make_unique<std::atomic_bool>(false))
|
||||
, deadline_{deadline}
|
||||
{
|
||||
try {
|
||||
boost::asio::io_context ctx;
|
||||
boost::asio::ip::tcp::resolver resolver{ctx};
|
||||
|
||||
auto const resolverResult = resolver.resolve(ip, grpcPort);
|
||||
if (resolverResult.empty())
|
||||
throw std::runtime_error("Failed to resolve " + ip + ":" + grpcPort);
|
||||
|
||||
std::stringstream ss;
|
||||
ss << resolverResult.begin()->endpoint();
|
||||
|
||||
grpc::ChannelArguments chArgs;
|
||||
chArgs.SetMaxReceiveMessageSize(-1);
|
||||
chArgs.SetInt(GRPC_ARG_KEEPALIVE_TIME_MS, kKEEPALIVE_PING_INTERVAL_MS);
|
||||
@@ -70,7 +79,7 @@ GrpcSource::GrpcSource(std::string const& ip, std::string const& grpcPort, std::
|
||||
chArgs.SetInt(GRPC_ARG_HTTP2_MAX_PINGS_WITHOUT_DATA, kMAX_PINGS_WITHOUT_DATA);
|
||||
|
||||
stub_ = org::xrpl::rpc::v1::XRPLedgerAPIService::NewStub(
|
||||
grpc::CreateCustomChannel(ss.str(), grpc::InsecureChannelCredentials(), chArgs)
|
||||
grpc::CreateCustomChannel(resolve(ip, grpcPort), grpc::InsecureChannelCredentials(), chArgs)
|
||||
);
|
||||
|
||||
LOG(log_.debug()) << "Made stub for remote.";
|
||||
@@ -89,7 +98,7 @@ GrpcSource::fetchLedger(uint32_t sequence, bool getObjects, bool getObjectNeighb
|
||||
org::xrpl::rpc::v1::GetLedgerRequest request;
|
||||
grpc::ClientContext context;
|
||||
|
||||
context.set_deadline(std::chrono::system_clock::now() + kDEADLINE); // Prevent indefinite blocking
|
||||
context.set_deadline(std::chrono::system_clock::now() + deadline_); // Prevent indefinite blocking
|
||||
|
||||
request.mutable_ledger()->set_sequence(sequence);
|
||||
request.set_transactions(true);
|
||||
@@ -100,7 +109,7 @@ GrpcSource::fetchLedger(uint32_t sequence, bool getObjects, bool getObjectNeighb
|
||||
|
||||
grpc::Status const status = stub_->GetLedger(&context, request, &response);
|
||||
|
||||
if (status.ok() && !response.is_unlimited()) {
|
||||
if (status.ok() and not response.is_unlimited()) {
|
||||
log_.warn() << "is_unlimited is false. Make sure secure_gateway is set correctly on the ETL source. Status = "
|
||||
<< status.error_message();
|
||||
}
|
||||
@@ -108,41 +117,46 @@ GrpcSource::fetchLedger(uint32_t sequence, bool getObjects, bool getObjectNeighb
|
||||
return {status, std::move(response)};
|
||||
}
|
||||
|
||||
std::pair<std::vector<std::string>, bool>
|
||||
GrpcSource::loadInitialLedger(uint32_t const sequence, uint32_t const numMarkers)
|
||||
InitialLedgerLoadResult
|
||||
GrpcSource::loadInitialLedger(
|
||||
uint32_t const sequence,
|
||||
uint32_t const numMarkers,
|
||||
InitialLoadObserverInterface& observer
|
||||
)
|
||||
{
|
||||
if (!stub_)
|
||||
return {{}, false};
|
||||
if (*initialLoadShouldStop_)
|
||||
return std::unexpected{InitialLedgerLoadError::Cancelled};
|
||||
|
||||
std::vector<etl::impl::AsyncCallData> calls = impl::makeAsyncCallData(sequence, numMarkers);
|
||||
if (!stub_)
|
||||
return std::unexpected{InitialLedgerLoadError::Errored};
|
||||
|
||||
std::vector<AsyncGrpcCall> calls = AsyncGrpcCall::makeAsyncCalls(sequence, numMarkers);
|
||||
|
||||
LOG(log_.debug()) << "Starting data download for ledger " << sequence << ".";
|
||||
|
||||
grpc::CompletionQueue cq;
|
||||
for (auto& c : calls)
|
||||
c.call(stub_, cq);
|
||||
grpc::CompletionQueue queue;
|
||||
for (auto& call : calls)
|
||||
call.call(stub_, queue);
|
||||
|
||||
std::vector<std::string> edgeKeys;
|
||||
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;
|
||||
size_t numFinished = 0;
|
||||
|
||||
while (numFinished < calls.size() && cq.Next(&tag, &ok)) {
|
||||
while (numFinished < calls.size() && queue.Next(&tag, &ok)) {
|
||||
ASSERT(tag != nullptr, "Tag can't be null.");
|
||||
auto ptr = static_cast<etl::impl::AsyncCallData*>(tag);
|
||||
auto ptr = static_cast<AsyncGrpcCall*>(tag);
|
||||
|
||||
if (!ok) {
|
||||
LOG(log_.error()) << "loadInitialLedger - ok is false";
|
||||
return {{}, false}; // handle cancelled
|
||||
if (not ok or *initialLoadShouldStop_) {
|
||||
LOG(log_.error()) << "loadInitialLedger cancelled";
|
||||
return std::unexpected{InitialLedgerLoadError::Cancelled};
|
||||
}
|
||||
|
||||
LOG(log_.trace()) << "Marker prefix = " << ptr->getMarkerPrefix();
|
||||
|
||||
auto result = ptr->process(stub_, cq, *backend_, abort);
|
||||
if (result != etl::impl::AsyncCallData::CallStatus::MORE) {
|
||||
auto result = ptr->process(stub_, queue, observer, abort);
|
||||
if (result != AsyncGrpcCall::CallStatus::More) {
|
||||
++numFinished;
|
||||
LOG(log_.debug()) << "Finished a marker. Current number of finished = " << numFinished;
|
||||
|
||||
@@ -150,18 +164,20 @@ GrpcSource::loadInitialLedger(uint32_t const sequence, uint32_t const numMarkers
|
||||
edgeKeys.push_back(std::move(lastKey));
|
||||
}
|
||||
|
||||
if (result == etl::impl::AsyncCallData::CallStatus::ERRORED)
|
||||
if (result == AsyncGrpcCall::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};
|
||||
if (abort)
|
||||
return std::unexpected{InitialLedgerLoadError::Errored};
|
||||
|
||||
return edgeKeys;
|
||||
}
|
||||
|
||||
void
|
||||
GrpcSource::stop(boost::asio::yield_context)
|
||||
{
|
||||
initialLoadShouldStop_->store(true);
|
||||
}
|
||||
|
||||
} // namespace etl::impl
|
||||
|
||||
@@ -19,25 +19,29 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#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_;
|
||||
std::unique_ptr<std::atomic_bool> initialLoadShouldStop_;
|
||||
std::chrono::system_clock::duration deadline_;
|
||||
|
||||
static constexpr auto kKEEPALIVE_PING_INTERVAL_MS = 10000;
|
||||
static constexpr auto kKEEPALIVE_TIMEOUT_MS = 5000;
|
||||
@@ -46,7 +50,11 @@ class GrpcSource {
|
||||
static constexpr auto kDEADLINE = std::chrono::seconds(30);
|
||||
|
||||
public:
|
||||
GrpcSource(std::string const& ip, std::string const& grpcPort, std::shared_ptr<BackendInterface> backend);
|
||||
GrpcSource(
|
||||
std::string const& ip,
|
||||
std::string const& grpcPort,
|
||||
std::chrono::system_clock::duration deadline = kDEADLINE
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Fetch data for a specific ledger.
|
||||
@@ -67,10 +75,19 @@ public:
|
||||
*
|
||||
* @param sequence Sequence of the ledger to download
|
||||
* @param numMarkers Number of markers to generate for async calls
|
||||
* @return A std::pair of the data and a bool indicating whether the download was successful
|
||||
* @param observer InitialLoadObserverInterface implementation
|
||||
* @return Downloaded data or an indication of error or cancellation
|
||||
*/
|
||||
std::pair<std::vector<std::string>, bool>
|
||||
loadInitialLedger(uint32_t sequence, uint32_t numMarkers);
|
||||
InitialLedgerLoadResult
|
||||
loadInitialLedger(uint32_t sequence, uint32_t numMarkers, InitialLoadObserverInterface& observer);
|
||||
|
||||
/**
|
||||
* @brief Stop any ongoing operations
|
||||
* @note This is used to cancel any ongoing initial ledger downloads
|
||||
* @param yield The coroutine context
|
||||
*/
|
||||
void
|
||||
stop(boost::asio::yield_context yield);
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
|
||||
@@ -21,7 +21,7 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/LedgerFetcherInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <grpcpp/grpcpp.h>
|
||||
@@ -41,13 +41,13 @@ private:
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> loadBalancer_;
|
||||
std::shared_ptr<LoadBalancerInterface> loadBalancer_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of the fetcher
|
||||
*/
|
||||
LedgerFetcher(std::shared_ptr<BackendInterface> backend, std::shared_ptr<etlng::LoadBalancerInterface> balancer)
|
||||
LedgerFetcher(std::shared_ptr<BackendInterface> backend, std::shared_ptr<LoadBalancerInterface> balancer)
|
||||
: backend_(std::move(backend)), loadBalancer_(std::move(balancer))
|
||||
{
|
||||
}
|
||||
|
||||
@@ -1,276 +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.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/MPTHelpers.hpp"
|
||||
#include "etl/NFTHelpers.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/impl/LedgerFetcher.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/LedgerUtils.hpp"
|
||||
#include "util/Profiler.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
#include <xrpl/basics/strHex.h>
|
||||
#include <xrpl/beast/core/CurrentThreadName.h>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
#include <xrpl/protocol/STTx.h>
|
||||
#include <xrpl/protocol/Serializer.h>
|
||||
#include <xrpl/protocol/TxMeta.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
/**
|
||||
* @brief Account transactions, NFT transactions and NFT data bundled togeher.
|
||||
*/
|
||||
struct FormattedTransactionsData {
|
||||
std::vector<AccountTransactionsData> accountTxData;
|
||||
std::vector<NFTTransactionsData> nfTokenTxData;
|
||||
std::vector<NFTsData> nfTokensData;
|
||||
std::vector<MPTHolderData> mptHoldersData;
|
||||
std::vector<NFTsData> nfTokenURIChanges;
|
||||
};
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
/**
|
||||
* @brief Loads ledger data into the DB
|
||||
*/
|
||||
template <typename LedgerFetcherType>
|
||||
class LedgerLoader {
|
||||
public:
|
||||
using GetLedgerResponseType = etlng::LoadBalancerInterface::GetLedgerResponseType;
|
||||
using OptionalGetLedgerResponseType = etlng::LoadBalancerInterface::OptionalGetLedgerResponseType;
|
||||
using RawLedgerObjectType = etlng::LoadBalancerInterface::RawLedgerObjectType;
|
||||
|
||||
private:
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> loadBalancer_;
|
||||
std::reference_wrapper<LedgerFetcherType> fetcher_;
|
||||
std::reference_wrapper<SystemState const> state_; // shared state for ETL
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of the loader
|
||||
*/
|
||||
LedgerLoader(
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> balancer,
|
||||
LedgerFetcherType& fetcher,
|
||||
SystemState const& state
|
||||
)
|
||||
: backend_{std::move(backend)}
|
||||
, loadBalancer_{std::move(balancer)}
|
||||
, fetcher_{std::ref(fetcher)}
|
||||
, state_{std::cref(state)}
|
||||
{
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Insert extracted transaction into the ledger
|
||||
*
|
||||
* Insert all of the extracted transactions into the ledger, returning transactions related to accounts,
|
||||
* transactions related to NFTs, and NFTs themselves for later processing.
|
||||
*
|
||||
* @param ledger ledger to insert transactions into
|
||||
* @param data data extracted from an ETL source
|
||||
* @return The necessary info to write the account_transactions/account_tx and nft_token_transactions tables
|
||||
*/
|
||||
FormattedTransactionsData
|
||||
insertTransactions(ripple::LedgerHeader const& ledger, GetLedgerResponseType& data)
|
||||
{
|
||||
FormattedTransactionsData result;
|
||||
|
||||
std::vector<NFTsData> nfTokenURIChanges;
|
||||
for (auto& txn : *(data.mutable_transactions_list()->mutable_transactions())) {
|
||||
std::string* raw = txn.mutable_transaction_blob();
|
||||
|
||||
ripple::SerialIter it{raw->data(), raw->size()};
|
||||
ripple::STTx const sttx{it};
|
||||
|
||||
LOG(log_.trace()) << "Inserting transaction = " << sttx.getTransactionID();
|
||||
|
||||
ripple::TxMeta const txMeta{sttx.getTransactionID(), ledger.seq, txn.metadata_blob()};
|
||||
|
||||
auto const [nftTxs, maybeNFT] = getNFTDataFromTx(txMeta, sttx);
|
||||
result.nfTokenTxData.insert(result.nfTokenTxData.end(), nftTxs.begin(), nftTxs.end());
|
||||
|
||||
// We need to unique the URI changes separately, in case the URI changes are discarded
|
||||
if (maybeNFT) {
|
||||
if (maybeNFT->onlyUriChanged) {
|
||||
nfTokenURIChanges.push_back(*maybeNFT);
|
||||
} else {
|
||||
result.nfTokensData.push_back(*maybeNFT);
|
||||
}
|
||||
}
|
||||
|
||||
auto const maybeMPTHolder = getMPTHolderFromTx(txMeta, sttx);
|
||||
if (maybeMPTHolder)
|
||||
result.mptHoldersData.push_back(*maybeMPTHolder);
|
||||
|
||||
result.accountTxData.emplace_back(txMeta, sttx.getTransactionID());
|
||||
static constexpr std::size_t kEY_SIZE = 32;
|
||||
std::string keyStr{reinterpret_cast<char const*>(sttx.getTransactionID().data()), kEY_SIZE};
|
||||
backend_->writeTransaction(
|
||||
std::move(keyStr),
|
||||
ledger.seq,
|
||||
ledger.closeTime.time_since_epoch().count(),
|
||||
std::move(*raw),
|
||||
std::move(*txn.mutable_metadata_blob())
|
||||
);
|
||||
}
|
||||
|
||||
result.nfTokensData = getUniqueNFTsDatas(result.nfTokensData);
|
||||
nfTokenURIChanges = getUniqueNFTsDatas(nfTokenURIChanges);
|
||||
|
||||
// Put uri change at the end to ensure the uri not overwritten
|
||||
result.nfTokensData.insert(result.nfTokensData.end(), nfTokenURIChanges.begin(), nfTokenURIChanges.end());
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Download a ledger with specified sequence in full
|
||||
*
|
||||
* Note: This takes several minutes or longer.
|
||||
*
|
||||
* @param sequence the sequence of the ledger to download
|
||||
* @return The ledger downloaded, with a full transaction and account state map
|
||||
*/
|
||||
std::optional<ripple::LedgerHeader>
|
||||
loadInitialLedger(uint32_t sequence)
|
||||
{
|
||||
// check that database is actually empty
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
if (rng) {
|
||||
ASSERT(false, "Database is not empty");
|
||||
return {};
|
||||
}
|
||||
|
||||
// Fetch the ledger from the network. This function will not return until either the fetch is successful, or the
|
||||
// server is being shutdown. This only fetches the ledger header and the transactions+metadata
|
||||
OptionalGetLedgerResponseType ledgerData{fetcher_.get().fetchData(sequence)};
|
||||
if (!ledgerData)
|
||||
return {};
|
||||
|
||||
ripple::LedgerHeader lgrInfo = ::util::deserializeHeader(ripple::makeSlice(ledgerData->ledger_header()));
|
||||
|
||||
LOG(log_.debug()) << "Deserialized ledger header. " << ::util::toString(lgrInfo);
|
||||
|
||||
auto timeDiff = ::util::timed<std::chrono::duration<double>>([this, sequence, &lgrInfo, &ledgerData]() {
|
||||
backend_->startWrites();
|
||||
|
||||
LOG(log_.debug()) << "Started writes";
|
||||
|
||||
backend_->writeLedger(lgrInfo, std::move(*ledgerData->mutable_ledger_header()));
|
||||
|
||||
LOG(log_.debug()) << "Wrote ledger";
|
||||
FormattedTransactionsData insertTxResult = insertTransactions(lgrInfo, *ledgerData);
|
||||
LOG(log_.debug()) << "Inserted txns";
|
||||
|
||||
// download the full account state map. This function downloads full
|
||||
// ledger data and pushes the downloaded data into the writeQueue.
|
||||
// asyncWriter consumes from the queue and inserts the data into the
|
||||
// Ledger object. Once the below call returns, all data has been pushed
|
||||
// into the queue
|
||||
auto edgeKeys = loadBalancer_->loadInitialLedger(sequence);
|
||||
|
||||
size_t numWrites = 0;
|
||||
backend_->cache().setFull();
|
||||
|
||||
auto seconds =
|
||||
::util::timed<std::chrono::seconds>([this, keys = std::move(edgeKeys), sequence, &numWrites]() mutable {
|
||||
for (auto& key : keys) {
|
||||
LOG(log_.debug()) << "Writing edge key = " << ripple::strHex(key);
|
||||
auto succ = backend_->cache().getSuccessor(*ripple::uint256::fromVoidChecked(key), sequence);
|
||||
if (succ)
|
||||
backend_->writeSuccessor(std::move(key), sequence, uint256ToString(succ->key));
|
||||
}
|
||||
|
||||
ripple::uint256 prev = data::kFIRST_KEY;
|
||||
while (auto cur = backend_->cache().getSuccessor(prev, sequence)) {
|
||||
ASSERT(cur.has_value(), "Successor for key {} must exist", ripple::strHex(prev));
|
||||
if (prev == data::kFIRST_KEY)
|
||||
backend_->writeSuccessor(uint256ToString(prev), sequence, uint256ToString(cur->key));
|
||||
|
||||
if (isBookDir(cur->key, cur->blob)) {
|
||||
auto base = getBookBase(cur->key);
|
||||
// make sure the base is not an actual object
|
||||
if (!backend_->cache().get(base, sequence)) {
|
||||
auto succ = backend_->cache().getSuccessor(base, sequence);
|
||||
ASSERT(succ.has_value(), "Book base {} must have a successor", ripple::strHex(base));
|
||||
if (succ->key == cur->key) {
|
||||
LOG(log_.debug()) << "Writing book successor = " << ripple::strHex(base) << " - "
|
||||
<< ripple::strHex(cur->key);
|
||||
|
||||
backend_->writeSuccessor(
|
||||
uint256ToString(base), sequence, uint256ToString(cur->key)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
++numWrites;
|
||||
}
|
||||
|
||||
prev = cur->key;
|
||||
static constexpr std::size_t kLOG_STRIDE = 100000;
|
||||
if (numWrites % kLOG_STRIDE == 0 && numWrites != 0)
|
||||
LOG(log_.info()) << "Wrote " << numWrites << " book successors";
|
||||
}
|
||||
|
||||
backend_->writeSuccessor(uint256ToString(prev), sequence, uint256ToString(data::kLAST_KEY));
|
||||
++numWrites;
|
||||
});
|
||||
|
||||
LOG(log_.info()) << "Looping through cache and submitting all writes took " << seconds
|
||||
<< " seconds. numWrites = " << std::to_string(numWrites);
|
||||
|
||||
LOG(log_.debug()) << "Loaded initial ledger";
|
||||
|
||||
if (not state_.get().isStopping) {
|
||||
backend_->writeAccountTransactions(std::move(insertTxResult.accountTxData));
|
||||
backend_->writeNFTs(insertTxResult.nfTokensData);
|
||||
backend_->writeNFTTransactions(insertTxResult.nfTokenTxData);
|
||||
backend_->writeMPTHolders(insertTxResult.mptHoldersData);
|
||||
}
|
||||
|
||||
backend_->finishWrites(sequence);
|
||||
});
|
||||
|
||||
LOG(log_.debug()) << "Time to download and store ledger = " << timeDiff;
|
||||
return lgrInfo;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
@@ -1,7 +1,7 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2023, the clio developers.
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
@@ -21,12 +21,12 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/LedgerPublisherInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etlng/LedgerPublisherInterface.hpp"
|
||||
#include "etl/impl/Loading.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Mutex.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/prometheus/Counter.hpp"
|
||||
#include "util/prometheus/Prometheus.hpp"
|
||||
@@ -34,6 +34,7 @@
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/post.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <fmt/format.h>
|
||||
#include <xrpl/basics/chrono.h>
|
||||
#include <xrpl/protocol/Fees.h>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
@@ -68,18 +69,16 @@ namespace etl::impl {
|
||||
* includes reading all of the transactions from the database) is done from the application wide asio io_service, and a
|
||||
* strand is used to ensure ledgers are published in order.
|
||||
*/
|
||||
class LedgerPublisher : public etlng::LedgerPublisherInterface {
|
||||
class LedgerPublisher : public LedgerPublisherInterface {
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
boost::asio::strand<boost::asio::io_context::executor_type> publishStrand_;
|
||||
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::reference_wrapper<data::LedgerCacheInterface> cache_;
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions_;
|
||||
std::reference_wrapper<SystemState const> state_; // shared state for ETL
|
||||
|
||||
std::chrono::time_point<ripple::NetClock> lastCloseTime_;
|
||||
mutable std::shared_mutex closeTimeMtx_;
|
||||
util::Mutex<std::chrono::time_point<ripple::NetClock>, std::shared_mutex> lastCloseTime_;
|
||||
|
||||
std::reference_wrapper<util::prometheus::CounterInt> lastPublishSeconds_ = PrometheusService::counterInt(
|
||||
"etl_last_publish_seconds",
|
||||
@@ -87,23 +86,20 @@ class LedgerPublisher : public etlng::LedgerPublisherInterface {
|
||||
"Seconds since epoch of the last published ledger"
|
||||
);
|
||||
|
||||
std::optional<uint32_t> lastPublishedSequence_;
|
||||
mutable std::shared_mutex lastPublishedSeqMtx_;
|
||||
util::Mutex<std::optional<uint32_t>, std::shared_mutex> lastPublishedSequence_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of the publisher
|
||||
*/
|
||||
LedgerPublisher(
|
||||
boost::asio::io_context& ioc,
|
||||
boost::asio::io_context& ioc, // TODO: replace with AsyncContext shared with ETLService
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
data::LedgerCacheInterface& cache,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
SystemState const& state
|
||||
)
|
||||
: publishStrand_{boost::asio::make_strand(ioc)}
|
||||
, backend_{std::move(backend)}
|
||||
, cache_{cache}
|
||||
, subscriptions_{std::move(subscriptions)}
|
||||
, state_{std::cref(state)}
|
||||
{
|
||||
@@ -168,25 +164,10 @@ public:
|
||||
boost::asio::post(publishStrand_, [this, lgrInfo = lgrInfo]() {
|
||||
LOG(log_.info()) << "Publishing ledger " << std::to_string(lgrInfo.seq);
|
||||
|
||||
if (!state_.get().isWriting) {
|
||||
LOG(log_.info()) << "Updating ledger range for read node.";
|
||||
|
||||
if (!cache_.get().isDisabled()) {
|
||||
std::vector<data::LedgerObject> const diff = data::synchronousAndRetryOnTimeout([&](auto yield) {
|
||||
return backend_->fetchLedgerDiff(lgrInfo.seq, yield);
|
||||
});
|
||||
|
||||
cache_.get().update(diff, lgrInfo.seq);
|
||||
}
|
||||
|
||||
backend_->updateRange(lgrInfo.seq);
|
||||
}
|
||||
|
||||
setLastClose(lgrInfo.closeTime);
|
||||
auto age = lastCloseAgeSeconds();
|
||||
|
||||
// if the ledger closed over MAX_LEDGER_AGE_SECONDS ago, assume we are still catching up and don't publish
|
||||
// TODO: this probably should be a strategy
|
||||
static constexpr std::uint32_t kMAX_LEDGER_AGE_SECONDS = 600;
|
||||
if (age < kMAX_LEDGER_AGE_SECONDS) {
|
||||
std::optional<ripple::Fees> fees = data::synchronousAndRetryOnTimeout([&](auto yield) {
|
||||
@@ -194,17 +175,14 @@ public:
|
||||
});
|
||||
ASSERT(fees.has_value(), "Fees must exist for ledger {}", lgrInfo.seq);
|
||||
|
||||
std::vector<data::TransactionAndMetadata> transactions =
|
||||
data::synchronousAndRetryOnTimeout([&](auto yield) {
|
||||
return backend_->fetchAllTransactionsInLedger(lgrInfo.seq, yield);
|
||||
});
|
||||
auto transactions = data::synchronousAndRetryOnTimeout([&](auto yield) {
|
||||
return backend_->fetchAllTransactionsInLedger(lgrInfo.seq, yield);
|
||||
});
|
||||
|
||||
auto const ledgerRange = backend_->fetchLedgerRange();
|
||||
ASSERT(ledgerRange.has_value(), "Ledger range must exist");
|
||||
|
||||
std::string const range =
|
||||
std::to_string(ledgerRange->minSequence) + "-" + std::to_string(ledgerRange->maxSequence);
|
||||
|
||||
auto const range = fmt::format("{}-{}", ledgerRange->minSequence, ledgerRange->maxSequence);
|
||||
subscriptions_->pubLedger(lgrInfo, *fees, range, transactions.size());
|
||||
|
||||
// order with transaction index
|
||||
@@ -217,15 +195,15 @@ public:
|
||||
object2.getFieldU32(ripple::sfTransactionIndex);
|
||||
});
|
||||
|
||||
for (auto& txAndMeta : transactions)
|
||||
for (auto const& txAndMeta : transactions)
|
||||
subscriptions_->pubTransaction(txAndMeta, lgrInfo);
|
||||
|
||||
subscriptions_->pubBookChanges(lgrInfo, transactions);
|
||||
|
||||
setLastPublishTime();
|
||||
LOG(log_.info()) << "Published ledger " << std::to_string(lgrInfo.seq);
|
||||
LOG(log_.info()) << "Published ledger " << lgrInfo.seq;
|
||||
} else {
|
||||
LOG(log_.info()) << "Skipping publishing ledger " << std::to_string(lgrInfo.seq);
|
||||
LOG(log_.info()) << "Skipping publishing ledger " << lgrInfo.seq;
|
||||
}
|
||||
});
|
||||
|
||||
@@ -260,32 +238,30 @@ public:
|
||||
std::uint32_t
|
||||
lastCloseAgeSeconds() const override
|
||||
{
|
||||
std::shared_lock const lck(closeTimeMtx_);
|
||||
auto closeTime = lastCloseTime_.lock()->time_since_epoch().count();
|
||||
auto now = std::chrono::duration_cast<std::chrono::seconds>(std::chrono::system_clock::now().time_since_epoch())
|
||||
.count();
|
||||
auto closeTime = lastCloseTime_.time_since_epoch().count();
|
||||
if (now < (kRIPPLE_EPOCH_START + closeTime))
|
||||
return 0;
|
||||
return now - (kRIPPLE_EPOCH_START + closeTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get the sequence of the last scheduled ledger to publish, Be aware that the ledger may not have been
|
||||
* @brief Get the sequence of the last schueduled ledger to publish, Be aware that the ledger may not have been
|
||||
* published to network
|
||||
*/
|
||||
std::optional<uint32_t>
|
||||
getLastPublishedSequence() const
|
||||
{
|
||||
std::scoped_lock const lck(lastPublishedSeqMtx_);
|
||||
return lastPublishedSequence_;
|
||||
return *lastPublishedSequence_.lock();
|
||||
}
|
||||
|
||||
private:
|
||||
void
|
||||
setLastClose(std::chrono::time_point<ripple::NetClock> lastCloseTime)
|
||||
{
|
||||
std::scoped_lock const lck(closeTimeMtx_);
|
||||
lastCloseTime_ = lastCloseTime;
|
||||
auto closeTime = lastCloseTime_.lock<std::scoped_lock>();
|
||||
*closeTime = lastCloseTime;
|
||||
}
|
||||
|
||||
void
|
||||
@@ -299,8 +275,8 @@ private:
|
||||
void
|
||||
setLastPublishedSequence(std::optional<uint32_t> lastPublishedSequence)
|
||||
{
|
||||
std::scoped_lock const lck(lastPublishedSeqMtx_);
|
||||
lastPublishedSequence_ = lastPublishedSequence;
|
||||
auto lastPublishSeq = lastPublishedSequence_.lock();
|
||||
*lastPublishSeq = lastPublishedSequence;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
@@ -17,15 +17,14 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/Loading.hpp"
|
||||
#include "etl/impl/Loading.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/AmendmentBlockHandlerInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/RegistryInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/impl/LedgerLoader.hpp"
|
||||
#include "etlng/AmendmentBlockHandlerInterface.hpp"
|
||||
#include "etlng/LoaderInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/RegistryInterface.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Constants.hpp"
|
||||
#include "util/LedgerUtils.hpp"
|
||||
@@ -44,13 +43,13 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
Loader::Loader(
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<RegistryInterface> registry,
|
||||
std::shared_ptr<AmendmentBlockHandlerInterface> amendmentBlockHandler,
|
||||
std::shared_ptr<etl::SystemState> state
|
||||
std::shared_ptr<SystemState> state
|
||||
)
|
||||
: backend_(std::move(backend))
|
||||
, registry_(std::move(registry))
|
||||
@@ -154,4 +153,4 @@ Loader::loadInitialLedger(model::LedgerData const& data)
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -20,13 +20,12 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/AmendmentBlockHandlerInterface.hpp"
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/RegistryInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/impl/LedgerLoader.hpp"
|
||||
#include "etlng/AmendmentBlockHandlerInterface.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoaderInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/RegistryInterface.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <org/xrpl/rpc/v1/ledger.pb.h>
|
||||
@@ -46,13 +45,13 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class Loader : public LoaderInterface, public InitialLoadObserverInterface {
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<RegistryInterface> registry_;
|
||||
std::shared_ptr<AmendmentBlockHandlerInterface> amendmentBlockHandler_;
|
||||
std::shared_ptr<etl::SystemState> state_;
|
||||
std::shared_ptr<SystemState> state_;
|
||||
|
||||
std::size_t initialLoadWrittenObjects_{0u};
|
||||
std::size_t initialLoadWrites_{0u};
|
||||
@@ -67,7 +66,7 @@ public:
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<RegistryInterface> registry,
|
||||
std::shared_ptr<AmendmentBlockHandlerInterface> amendmentBlockHandler,
|
||||
std::shared_ptr<etl::SystemState> state
|
||||
std::shared_ptr<SystemState> state
|
||||
);
|
||||
|
||||
Loader(Loader const&) = delete;
|
||||
@@ -91,4 +90,4 @@ public:
|
||||
loadInitialLedger(model::LedgerData const& data) override;
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -17,7 +17,7 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/Monitor.hpp"
|
||||
#include "etl/impl/Monitor.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
@@ -36,11 +36,11 @@
|
||||
#include <optional>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
Monitor::Monitor(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
uint32_t startSequence,
|
||||
std::chrono::steady_clock::duration dbStalledReportDelay
|
||||
)
|
||||
@@ -165,4 +165,4 @@ Monitor::doWork()
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -20,8 +20,8 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "util/Mutex.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/async/AnyOperation.hpp"
|
||||
@@ -38,12 +38,12 @@
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class Monitor : public MonitorInterface {
|
||||
util::async::AnyStrand strand_;
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers_;
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers_;
|
||||
|
||||
std::atomic_uint32_t nextSequence_;
|
||||
std::optional<util::async::AnyOperation<void>> repeatedTask_;
|
||||
@@ -66,7 +66,7 @@ public:
|
||||
Monitor(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
uint32_t startSequence,
|
||||
std::chrono::steady_clock::duration dbStalledReportDelay
|
||||
);
|
||||
@@ -98,4 +98,4 @@ private:
|
||||
doWork();
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -20,10 +20,10 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/MonitorProviderInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "etlng/MonitorProviderInterface.hpp"
|
||||
#include "etlng/impl/Monitor.hpp"
|
||||
#include "etl/impl/Monitor.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
|
||||
#include <chrono>
|
||||
@@ -31,7 +31,7 @@
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class MonitorProvider : public MonitorProviderInterface {
|
||||
public:
|
||||
@@ -39,7 +39,7 @@ public:
|
||||
make(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::shared_ptr<NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
uint32_t startSequence,
|
||||
std::chrono::steady_clock::duration dbStalledReportDelay
|
||||
) override
|
||||
@@ -50,4 +50,4 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,9 +19,9 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/RegistryInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/RegistryInterface.hpp"
|
||||
|
||||
#include <xrpl/protocol/TxFormats.h>
|
||||
|
||||
@@ -34,43 +34,41 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
template <typename T>
|
||||
concept HasLedgerDataHook = requires(T p) {
|
||||
{ p.onLedgerData(std::declval<etlng::model::LedgerData>()) } -> std::same_as<void>;
|
||||
{ p.onLedgerData(std::declval<model::LedgerData>()) } -> std::same_as<void>;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept HasInitialDataHook = requires(T p) {
|
||||
{ p.onInitialData(std::declval<etlng::model::LedgerData>()) } -> std::same_as<void>;
|
||||
{ p.onInitialData(std::declval<model::LedgerData>()) } -> std::same_as<void>;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept HasTransactionHook = requires(T p) {
|
||||
{ p.onTransaction(uint32_t{}, std::declval<etlng::model::Transaction>()) } -> std::same_as<void>;
|
||||
{ p.onTransaction(uint32_t{}, std::declval<model::Transaction>()) } -> std::same_as<void>;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept HasObjectHook = requires(T p) {
|
||||
{ p.onObject(uint32_t{}, std::declval<etlng::model::Object>()) } -> std::same_as<void>;
|
||||
{ p.onObject(uint32_t{}, std::declval<model::Object>()) } -> std::same_as<void>;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept HasInitialTransactionHook = requires(T p) {
|
||||
{ p.onInitialTransaction(uint32_t{}, std::declval<etlng::model::Transaction>()) } -> std::same_as<void>;
|
||||
{ p.onInitialTransaction(uint32_t{}, std::declval<model::Transaction>()) } -> std::same_as<void>;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept HasInitialObjectsHook = requires(T p) {
|
||||
{
|
||||
p.onInitialObjects(uint32_t{}, std::declval<std::vector<etlng::model::Object>>(), std::string{})
|
||||
} -> std::same_as<void>;
|
||||
{ p.onInitialObjects(uint32_t{}, std::declval<std::vector<model::Object>>(), std::string{}) } -> std::same_as<void>;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept HasInitialObjectHook = requires(T p) {
|
||||
{ p.onInitialObject(uint32_t{}, std::declval<etlng::model::Object>()) } -> std::same_as<void>;
|
||||
{ p.onInitialObject(uint32_t{}, std::declval<model::Object>()) } -> std::same_as<void>;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
@@ -91,7 +89,7 @@ concept SomeExtension = NoTwoOfKind<T> and ContainsValidHook<T>;
|
||||
|
||||
template <SomeExtension... Ps>
|
||||
class Registry : public RegistryInterface {
|
||||
std::reference_wrapper<etl::SystemState const> state_;
|
||||
std::reference_wrapper<SystemState const> state_;
|
||||
std::tuple<Ps...> store_;
|
||||
|
||||
static_assert(
|
||||
@@ -105,7 +103,7 @@ class Registry : public RegistryInterface {
|
||||
);
|
||||
|
||||
public:
|
||||
explicit constexpr Registry(etl::SystemState const& state, SomeExtension auto&&... exts)
|
||||
explicit constexpr Registry(SystemState const& state, SomeExtension auto&&... exts)
|
||||
requires(std::is_same_v<std::decay_t<decltype(exts)>, std::decay_t<Ps>> and ...)
|
||||
: state_{state}, store_(std::forward<Ps>(exts)...)
|
||||
{
|
||||
@@ -228,9 +226,9 @@ private:
|
||||
};
|
||||
|
||||
static auto
|
||||
makeRegistry(etl::SystemState const& state, auto&&... exts)
|
||||
makeRegistry(SystemState const& state, auto&&... exts)
|
||||
{
|
||||
return std::make_unique<Registry<std::decay_t<decltype(exts)>...>>(state, std::forward<decltype(exts)>(exts)...);
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,9 +19,9 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/SchedulerInterface.hpp"
|
||||
#include "etl/SchedulerInterface.hpp"
|
||||
|
||||
#include <sys/types.h>
|
||||
|
||||
@@ -35,13 +35,13 @@
|
||||
#include <type_traits>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
template <typename T>
|
||||
concept SomeScheduler = std::is_base_of_v<SchedulerInterface, std::decay_t<T>>;
|
||||
|
||||
class ForwardScheduler : public SchedulerInterface {
|
||||
std::reference_wrapper<etl::NetworkValidatedLedgersInterface> ledgers_;
|
||||
std::reference_wrapper<NetworkValidatedLedgersInterface> ledgers_;
|
||||
|
||||
uint32_t startSeq_;
|
||||
std::optional<uint32_t> maxSeq_;
|
||||
@@ -54,7 +54,7 @@ public:
|
||||
}
|
||||
|
||||
ForwardScheduler(
|
||||
std::reference_wrapper<etl::NetworkValidatedLedgersInterface> ledgers,
|
||||
std::reference_wrapper<NetworkValidatedLedgersInterface> ledgers,
|
||||
uint32_t startSeq,
|
||||
std::optional<uint32_t> maxSeq = std::nullopt
|
||||
)
|
||||
@@ -148,4 +148,4 @@ makeScheduler(SomeScheduler auto&&... schedulers)
|
||||
);
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,6 +19,8 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/InitialLoadObserverInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "etl/Source.hpp"
|
||||
#include "etl/impl/ForwardingSource.hpp"
|
||||
#include "etl/impl/GrpcSource.hpp"
|
||||
@@ -51,8 +53,8 @@ namespace etl::impl {
|
||||
*/
|
||||
template <
|
||||
typename GrpcSourceType = GrpcSource,
|
||||
typename SubscriptionSourceTypePtr = std::unique_ptr<SubscriptionSource>,
|
||||
typename ForwardingSourceType = ForwardingSource>
|
||||
typename SubscriptionSourceTypePtr = std::unique_ptr<impl::SubscriptionSource>,
|
||||
typename ForwardingSourceType = impl::ForwardingSource>
|
||||
class SourceImpl : public SourceBase {
|
||||
std::string ip_;
|
||||
std::string wsPort_;
|
||||
@@ -106,6 +108,7 @@ public:
|
||||
stop(boost::asio::yield_context yield) final
|
||||
{
|
||||
subscriptionSource_->stop(yield);
|
||||
grpcSource_.stop(yield);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -198,12 +201,13 @@ public:
|
||||
*
|
||||
* @param sequence Sequence of the ledger to download
|
||||
* @param numMarkers Number of markers to generate for async calls
|
||||
* @param loader InitialLoadObserverInterface implementation
|
||||
* @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, std::uint32_t numMarkers) final
|
||||
InitialLedgerLoadResult
|
||||
loadInitialLedger(uint32_t sequence, std::uint32_t numMarkers, InitialLoadObserverInterface& loader) final
|
||||
{
|
||||
return grpcSource_.loadInitialLedger(sequence, numMarkers);
|
||||
return grpcSource_.loadInitialLedger(sequence, numMarkers, loader);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -17,15 +17,15 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/TaskManager.hpp"
|
||||
#include "etl/impl/TaskManager.hpp"
|
||||
|
||||
#include "etlng/ExtractorInterface.hpp"
|
||||
#include "etlng/LoaderInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "etlng/SchedulerInterface.hpp"
|
||||
#include "etlng/impl/Monitor.hpp"
|
||||
#include "etlng/impl/TaskQueue.hpp"
|
||||
#include "etl/ExtractorInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/SchedulerInterface.hpp"
|
||||
#include "etl/impl/Monitor.hpp"
|
||||
#include "etl/impl/TaskQueue.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Constants.hpp"
|
||||
#include "util/LedgerUtils.hpp"
|
||||
@@ -44,7 +44,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
TaskManager::TaskManager(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
@@ -190,4 +190,4 @@ TaskManager::stop()
|
||||
wait();
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,13 +19,13 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/ExtractorInterface.hpp"
|
||||
#include "etlng/LoaderInterface.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "etlng/SchedulerInterface.hpp"
|
||||
#include "etlng/TaskManagerInterface.hpp"
|
||||
#include "etlng/impl/Monitor.hpp"
|
||||
#include "etlng/impl/TaskQueue.hpp"
|
||||
#include "etl/ExtractorInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/SchedulerInterface.hpp"
|
||||
#include "etl/TaskManagerInterface.hpp"
|
||||
#include "etl/impl/Monitor.hpp"
|
||||
#include "etl/impl/TaskQueue.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/async/AnyOperation.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
@@ -39,7 +39,7 @@
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class TaskManager : public TaskManagerInterface {
|
||||
static constexpr auto kQUEUE_SIZE_LIMIT = 2048uz;
|
||||
@@ -95,4 +95,4 @@ private:
|
||||
spawnLoader(TaskQueue& queue);
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,14 +19,14 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etl/ExtractorInterface.hpp"
|
||||
#include "etl/LoaderInterface.hpp"
|
||||
#include "etl/MonitorInterface.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/ExtractorInterface.hpp"
|
||||
#include "etlng/LoaderInterface.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "etlng/TaskManagerInterface.hpp"
|
||||
#include "etlng/TaskManagerProviderInterface.hpp"
|
||||
#include "etlng/impl/Scheduling.hpp"
|
||||
#include "etlng/impl/TaskManager.hpp"
|
||||
#include "etl/TaskManagerInterface.hpp"
|
||||
#include "etl/TaskManagerProviderInterface.hpp"
|
||||
#include "etl/impl/Scheduling.hpp"
|
||||
#include "etl/impl/TaskManager.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
@@ -35,13 +35,13 @@
|
||||
#include <optional>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
/**
|
||||
* @brief Implementation of the TaskManagerProvider interface
|
||||
*/
|
||||
class TaskManagerProvider : public TaskManagerProviderInterface {
|
||||
std::reference_wrapper<etl::NetworkValidatedLedgersInterface> ledgers_;
|
||||
std::reference_wrapper<NetworkValidatedLedgersInterface> ledgers_;
|
||||
std::shared_ptr<ExtractorInterface> extractor_;
|
||||
std::shared_ptr<LoaderInterface> loader_;
|
||||
|
||||
@@ -54,7 +54,7 @@ public:
|
||||
* @param loader The loader
|
||||
*/
|
||||
TaskManagerProvider(
|
||||
std::reference_wrapper<etl::NetworkValidatedLedgersInterface> ledgers,
|
||||
std::reference_wrapper<NetworkValidatedLedgersInterface> ledgers,
|
||||
std::shared_ptr<ExtractorInterface> extractor,
|
||||
std::shared_ptr<LoaderInterface> loader
|
||||
)
|
||||
@@ -79,4 +79,4 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,7 +19,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Mutex.hpp"
|
||||
|
||||
@@ -33,7 +33,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
struct ReverseOrderComparator {
|
||||
[[nodiscard]] bool
|
||||
@@ -167,4 +167,4 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -1,424 +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.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/impl/AmendmentBlockHandler.hpp"
|
||||
#include "etl/impl/LedgerLoader.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/LedgerUtils.hpp"
|
||||
#include "util/Profiler.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <grpcpp/grpcpp.h>
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
#include <xrpl/basics/strHex.h>
|
||||
#include <xrpl/beast/core/CurrentThreadName.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <set>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etl::impl {
|
||||
|
||||
/*
|
||||
* TODO:
|
||||
*
|
||||
* 1) loading of data into db should not really be part of transform right?
|
||||
* 2) can we just prepare the data and give it to the loader afterwards?
|
||||
* 3) how to deal with cache update that is needed to write successors if neighbours not included?
|
||||
*/
|
||||
|
||||
/**
|
||||
* @brief Transformer thread that prepares new ledger out of raw data from GRPC.
|
||||
*/
|
||||
template <
|
||||
typename DataPipeType,
|
||||
typename LedgerLoaderType,
|
||||
typename LedgerPublisherType,
|
||||
typename AmendmentBlockHandlerType>
|
||||
class Transformer {
|
||||
using GetLedgerResponseType = typename LedgerLoaderType::GetLedgerResponseType;
|
||||
using RawLedgerObjectType = typename LedgerLoaderType::RawLedgerObjectType;
|
||||
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
std::reference_wrapper<DataPipeType> pipe_;
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::reference_wrapper<LedgerLoaderType> loader_;
|
||||
std::reference_wrapper<LedgerPublisherType> publisher_;
|
||||
std::reference_wrapper<AmendmentBlockHandlerType> amendmentBlockHandler_;
|
||||
|
||||
uint32_t startSequence_;
|
||||
std::reference_wrapper<SystemState> state_; // shared state for ETL
|
||||
|
||||
std::thread thread_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of the transformer.
|
||||
*
|
||||
* This spawns a new thread that reads from the data pipe and writes ledgers to the DB using LedgerLoader and
|
||||
* LedgerPublisher.
|
||||
*/
|
||||
Transformer(
|
||||
DataPipeType& pipe,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
LedgerLoaderType& loader,
|
||||
LedgerPublisherType& publisher,
|
||||
AmendmentBlockHandlerType& amendmentBlockHandler,
|
||||
uint32_t startSequence,
|
||||
SystemState& state
|
||||
)
|
||||
: pipe_{std::ref(pipe)}
|
||||
, backend_{std::move(backend)}
|
||||
, loader_{std::ref(loader)}
|
||||
, publisher_{std::ref(publisher)}
|
||||
, amendmentBlockHandler_{std::ref(amendmentBlockHandler)}
|
||||
, startSequence_{startSequence}
|
||||
, state_{std::ref(state)}
|
||||
{
|
||||
thread_ = std::thread([this]() { process(); });
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Joins the transformer thread.
|
||||
*/
|
||||
~Transformer()
|
||||
{
|
||||
if (thread_.joinable())
|
||||
thread_.join();
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Block calling thread until transformer thread exits.
|
||||
*/
|
||||
void
|
||||
waitTillFinished()
|
||||
{
|
||||
ASSERT(thread_.joinable(), "Transformer thread must be joinable");
|
||||
thread_.join();
|
||||
}
|
||||
|
||||
private:
|
||||
void
|
||||
process()
|
||||
{
|
||||
beast::setCurrentThreadName("ETLService transform");
|
||||
uint32_t currentSequence = startSequence_;
|
||||
|
||||
while (not hasWriteConflict()) {
|
||||
auto fetchResponse = pipe_.get().popNext(currentSequence);
|
||||
++currentSequence;
|
||||
|
||||
// if fetchResponse is an empty optional, the extractor thread has stopped and the transformer should
|
||||
// stop as well
|
||||
if (!fetchResponse)
|
||||
break;
|
||||
|
||||
if (isStopping())
|
||||
continue;
|
||||
|
||||
auto const start = std::chrono::system_clock::now();
|
||||
auto [lgrInfo, success] = buildNextLedger(*fetchResponse);
|
||||
|
||||
if (success) {
|
||||
auto const numTxns = fetchResponse->transactions_list().transactions_size();
|
||||
auto const numObjects = fetchResponse->ledger_objects().objects_size();
|
||||
auto const end = std::chrono::system_clock::now();
|
||||
auto const duration = ((end - start).count()) / 1000000000.0;
|
||||
|
||||
LOG(log_.info()) << "Load phase of ETL. Successfully wrote ledger! Ledger info: "
|
||||
<< util::toString(lgrInfo) << ". txn count = " << numTxns
|
||||
<< ". object count = " << numObjects << ". load time = " << duration
|
||||
<< ". load txns per second = " << numTxns / duration
|
||||
<< ". load objs per second = " << numObjects / duration;
|
||||
|
||||
// success is false if the ledger was already written
|
||||
publisher_.get().publish(lgrInfo);
|
||||
} else {
|
||||
LOG(log_.error()) << "Error writing ledger. " << util::toString(lgrInfo);
|
||||
}
|
||||
|
||||
setWriteConflict(not success);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Build the next ledger using the previous ledger and the extracted data.
|
||||
* @note rawData should be data that corresponds to the ledger immediately following the previous seq.
|
||||
*
|
||||
* @param rawData Data extracted from an ETL source
|
||||
* @return The newly built ledger and data to write to the database
|
||||
*/
|
||||
std::pair<ripple::LedgerHeader, bool>
|
||||
buildNextLedger(GetLedgerResponseType& rawData)
|
||||
{
|
||||
LOG(log_.debug()) << "Beginning ledger update";
|
||||
ripple::LedgerHeader lgrInfo = ::util::deserializeHeader(ripple::makeSlice(rawData.ledger_header()));
|
||||
|
||||
LOG(log_.debug()) << "Deserialized ledger header. " << ::util::toString(lgrInfo);
|
||||
backend_->startWrites();
|
||||
backend_->writeLedger(lgrInfo, std::move(*rawData.mutable_ledger_header()));
|
||||
|
||||
writeSuccessors(lgrInfo, rawData);
|
||||
std::optional<FormattedTransactionsData> insertTxResultOp;
|
||||
try {
|
||||
updateCache(lgrInfo, rawData);
|
||||
|
||||
LOG(log_.debug()) << "Inserted/modified/deleted all objects. Number of objects = "
|
||||
<< rawData.ledger_objects().objects_size();
|
||||
|
||||
insertTxResultOp.emplace(loader_.get().insertTransactions(lgrInfo, rawData));
|
||||
} catch (std::runtime_error const& e) {
|
||||
LOG(log_.fatal()) << "Failed to build next ledger: " << e.what();
|
||||
|
||||
amendmentBlockHandler_.get().notifyAmendmentBlocked();
|
||||
return {ripple::LedgerHeader{}, false};
|
||||
}
|
||||
|
||||
LOG(log_.debug()) << "Inserted all transactions. Number of transactions = "
|
||||
<< rawData.transactions_list().transactions_size();
|
||||
|
||||
backend_->writeAccountTransactions(std::move(insertTxResultOp->accountTxData));
|
||||
backend_->writeNFTs(insertTxResultOp->nfTokensData);
|
||||
backend_->writeNFTTransactions(insertTxResultOp->nfTokenTxData);
|
||||
backend_->writeMPTHolders(insertTxResultOp->mptHoldersData);
|
||||
|
||||
auto [success, duration] =
|
||||
::util::timed<std::chrono::duration<double>>([&]() { return backend_->finishWrites(lgrInfo.seq); });
|
||||
|
||||
LOG(log_.debug()) << "Finished writes. Total time: " << std::to_string(duration);
|
||||
LOG(log_.debug()) << "Finished ledger update: " << ::util::toString(lgrInfo);
|
||||
|
||||
return {lgrInfo, success};
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Update cache from new ledger data.
|
||||
*
|
||||
* @param lgrInfo Ledger info
|
||||
* @param rawData Ledger data from GRPC
|
||||
*/
|
||||
void
|
||||
updateCache(ripple::LedgerHeader const& lgrInfo, GetLedgerResponseType& rawData)
|
||||
{
|
||||
std::vector<data::LedgerObject> cacheUpdates;
|
||||
cacheUpdates.reserve(rawData.ledger_objects().objects_size());
|
||||
|
||||
// TODO change these to unordered_set
|
||||
std::set<ripple::uint256> bookSuccessorsToCalculate;
|
||||
std::set<ripple::uint256> modified;
|
||||
|
||||
for (auto& obj : *(rawData.mutable_ledger_objects()->mutable_objects())) {
|
||||
auto key = ripple::uint256::fromVoidChecked(obj.key());
|
||||
ASSERT(key.has_value(), "Failed to deserialize key from void");
|
||||
|
||||
cacheUpdates.push_back({*key, {obj.mutable_data()->begin(), obj.mutable_data()->end()}});
|
||||
LOG(log_.debug()) << "key = " << ripple::strHex(*key) << " - mod type = " << obj.mod_type();
|
||||
|
||||
if (obj.mod_type() != RawLedgerObjectType::MODIFIED && !rawData.object_neighbors_included()) {
|
||||
LOG(log_.debug()) << "object neighbors not included. using cache";
|
||||
|
||||
if (!backend_->cache().isFull() || backend_->cache().latestLedgerSequence() != lgrInfo.seq - 1)
|
||||
throw std::logic_error("Cache is not full, but object neighbors were not included");
|
||||
|
||||
auto const blob = obj.mutable_data();
|
||||
auto checkBookBase = false;
|
||||
auto const isDeleted = (blob->size() == 0);
|
||||
|
||||
if (isDeleted) {
|
||||
auto const old = backend_->cache().get(*key, lgrInfo.seq - 1);
|
||||
ASSERT(old.has_value(), "Deleted object {} must be in cache", ripple::strHex(*key));
|
||||
checkBookBase = isBookDir(*key, *old);
|
||||
} else {
|
||||
checkBookBase = isBookDir(*key, *blob);
|
||||
}
|
||||
|
||||
if (checkBookBase) {
|
||||
LOG(log_.debug()) << "Is book dir. Key = " << ripple::strHex(*key);
|
||||
|
||||
auto const bookBase = getBookBase(*key);
|
||||
auto const oldFirstDir = backend_->cache().getSuccessor(bookBase, lgrInfo.seq - 1);
|
||||
ASSERT(
|
||||
oldFirstDir.has_value(),
|
||||
"Book base must have a successor for lgrInfo.seq - 1 = {}",
|
||||
lgrInfo.seq - 1
|
||||
);
|
||||
|
||||
// We deleted the first directory, or we added a directory prior to the old first
|
||||
// directory
|
||||
if ((isDeleted && key == oldFirstDir->key) || (!isDeleted && key < oldFirstDir->key)) {
|
||||
LOG(log_.debug())
|
||||
<< "Need to recalculate book base successor. base = " << ripple::strHex(bookBase)
|
||||
<< " - key = " << ripple::strHex(*key) << " - isDeleted = " << isDeleted
|
||||
<< " - seq = " << lgrInfo.seq;
|
||||
bookSuccessorsToCalculate.insert(bookBase);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (obj.mod_type() == RawLedgerObjectType::MODIFIED)
|
||||
modified.insert(*key);
|
||||
|
||||
backend_->writeLedgerObject(std::move(*obj.mutable_key()), lgrInfo.seq, std::move(*obj.mutable_data()));
|
||||
}
|
||||
|
||||
backend_->cache().update(cacheUpdates, lgrInfo.seq);
|
||||
|
||||
// rippled didn't send successor information, so use our cache
|
||||
if (!rawData.object_neighbors_included()) {
|
||||
LOG(log_.debug()) << "object neighbors not included. using cache";
|
||||
if (!backend_->cache().isFull() || backend_->cache().latestLedgerSequence() != lgrInfo.seq)
|
||||
throw std::logic_error("Cache is not full, but object neighbors were not included");
|
||||
|
||||
for (auto const& obj : cacheUpdates) {
|
||||
if (modified.contains(obj.key))
|
||||
continue;
|
||||
|
||||
auto lb = backend_->cache().getPredecessor(obj.key, lgrInfo.seq);
|
||||
if (!lb)
|
||||
lb = {.key = data::kFIRST_KEY, .blob = {}};
|
||||
|
||||
auto ub = backend_->cache().getSuccessor(obj.key, lgrInfo.seq);
|
||||
if (!ub)
|
||||
ub = {.key = data::kLAST_KEY, .blob = {}};
|
||||
|
||||
if (obj.blob.empty()) {
|
||||
LOG(log_.debug()) << "writing successor for deleted object " << ripple::strHex(obj.key) << " - "
|
||||
<< ripple::strHex(lb->key) << " - " << ripple::strHex(ub->key);
|
||||
|
||||
backend_->writeSuccessor(uint256ToString(lb->key), lgrInfo.seq, uint256ToString(ub->key));
|
||||
} else {
|
||||
backend_->writeSuccessor(uint256ToString(lb->key), lgrInfo.seq, uint256ToString(obj.key));
|
||||
backend_->writeSuccessor(uint256ToString(obj.key), lgrInfo.seq, uint256ToString(ub->key));
|
||||
|
||||
LOG(log_.debug()) << "writing successor for new object " << ripple::strHex(lb->key) << " - "
|
||||
<< ripple::strHex(obj.key) << " - " << ripple::strHex(ub->key);
|
||||
}
|
||||
}
|
||||
|
||||
for (auto const& base : bookSuccessorsToCalculate) {
|
||||
auto succ = backend_->cache().getSuccessor(base, lgrInfo.seq);
|
||||
if (succ) {
|
||||
backend_->writeSuccessor(uint256ToString(base), lgrInfo.seq, uint256ToString(succ->key));
|
||||
|
||||
LOG(log_.debug()) << "Updating book successor " << ripple::strHex(base) << " - "
|
||||
<< ripple::strHex(succ->key);
|
||||
} else {
|
||||
backend_->writeSuccessor(uint256ToString(base), lgrInfo.seq, uint256ToString(data::kLAST_KEY));
|
||||
|
||||
LOG(log_.debug()) << "Updating book successor " << ripple::strHex(base) << " - "
|
||||
<< ripple::strHex(data::kLAST_KEY);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Write successors info into DB.
|
||||
*
|
||||
* @param lgrInfo Ledger info
|
||||
* @param rawData Ledger data from GRPC
|
||||
*/
|
||||
void
|
||||
writeSuccessors(ripple::LedgerHeader const& lgrInfo, GetLedgerResponseType& rawData)
|
||||
{
|
||||
// Write successor info, if included from rippled
|
||||
if (rawData.object_neighbors_included()) {
|
||||
LOG(log_.debug()) << "object neighbors included";
|
||||
|
||||
for (auto& obj : *(rawData.mutable_book_successors())) {
|
||||
auto firstBook = std::move(*obj.mutable_first_book());
|
||||
if (!firstBook.size())
|
||||
firstBook = uint256ToString(data::kLAST_KEY);
|
||||
LOG(log_.debug()) << "writing book successor " << ripple::strHex(obj.book_base()) << " - "
|
||||
<< ripple::strHex(firstBook);
|
||||
|
||||
backend_->writeSuccessor(std::move(*obj.mutable_book_base()), lgrInfo.seq, std::move(firstBook));
|
||||
}
|
||||
|
||||
for (auto& obj : *(rawData.mutable_ledger_objects()->mutable_objects())) {
|
||||
if (obj.mod_type() != RawLedgerObjectType::MODIFIED) {
|
||||
std::string* predPtr = obj.mutable_predecessor();
|
||||
if (predPtr->empty())
|
||||
*predPtr = uint256ToString(data::kFIRST_KEY);
|
||||
std::string* succPtr = obj.mutable_successor();
|
||||
if (succPtr->empty())
|
||||
*succPtr = uint256ToString(data::kLAST_KEY);
|
||||
|
||||
if (obj.mod_type() == RawLedgerObjectType::DELETED) {
|
||||
LOG(log_.debug()) << "Modifying successors for deleted object " << ripple::strHex(obj.key())
|
||||
<< " - " << ripple::strHex(*predPtr) << " - " << ripple::strHex(*succPtr);
|
||||
|
||||
backend_->writeSuccessor(std::move(*predPtr), lgrInfo.seq, std::move(*succPtr));
|
||||
} else {
|
||||
LOG(log_.debug()) << "adding successor for new object " << ripple::strHex(obj.key()) << " - "
|
||||
<< ripple::strHex(*predPtr) << " - " << ripple::strHex(*succPtr);
|
||||
|
||||
backend_->writeSuccessor(std::move(*predPtr), lgrInfo.seq, std::string{obj.key()});
|
||||
backend_->writeSuccessor(std::string{obj.key()}, lgrInfo.seq, std::move(*succPtr));
|
||||
}
|
||||
} else
|
||||
LOG(log_.debug()) << "object modified " << ripple::strHex(obj.key());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** @return true if the transformer is stopping; false otherwise */
|
||||
bool
|
||||
isStopping() const
|
||||
{
|
||||
return state_.get().isStopping;
|
||||
}
|
||||
|
||||
/** @return true if there was a write conflict; false otherwise */
|
||||
bool
|
||||
hasWriteConflict() const
|
||||
{
|
||||
return state_.get().writeConflict;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Sets the write conflict flag.
|
||||
*
|
||||
* @param conflict The value to set
|
||||
*/
|
||||
void
|
||||
setWriteConflict(bool conflict)
|
||||
{
|
||||
state_.get().writeConflict = conflict;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etl::impl
|
||||
@@ -17,10 +17,10 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/ext/Cache.hpp"
|
||||
#include "etl/impl/ext/Cache.hpp"
|
||||
|
||||
#include "etlng/CacheUpdaterInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/CacheUpdaterInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
@@ -29,7 +29,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
CacheExt::CacheExt(std::shared_ptr<CacheUpdaterInterface> cacheUpdater) : cacheUpdater_(std::move(cacheUpdater))
|
||||
{
|
||||
@@ -57,4 +57,4 @@ CacheExt::onInitialObjects(uint32_t seq, std::vector<model::Object> const& objs,
|
||||
cacheUpdater_->update(seq, objs);
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -19,9 +19,9 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/CacheUpdaterInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/impl/CacheUpdater.hpp"
|
||||
#include "etl/CacheUpdaterInterface.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/impl/CacheUpdater.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
@@ -29,7 +29,7 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class CacheExt {
|
||||
std::shared_ptr<CacheUpdaterInterface> cacheUpdater_;
|
||||
@@ -56,4 +56,4 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -17,17 +17,17 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/ext/Core.hpp"
|
||||
#include "etl/impl/ext/Core.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
CoreExt::CoreExt(std::shared_ptr<BackendInterface> backend) : backend_(std::move(backend))
|
||||
{
|
||||
@@ -80,4 +80,4 @@ CoreExt::insertTransactions(model::LedgerData const& data)
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -20,7 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
@@ -28,7 +28,7 @@
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class CoreExt {
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
@@ -55,4 +55,4 @@ private:
|
||||
insertTransactions(model::LedgerData const& data);
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -17,12 +17,12 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/ext/MPT.hpp"
|
||||
#include "etl/impl/ext/MPT.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "etl/MPTHelpers.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <xrpl/basics/strHex.h>
|
||||
@@ -33,7 +33,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
MPTExt::MPTExt(std::shared_ptr<BackendInterface> backend) : backend_(std::move(backend))
|
||||
{
|
||||
@@ -50,7 +50,7 @@ void
|
||||
MPTExt::onInitialObject(uint32_t, model::Object const& obj)
|
||||
{
|
||||
LOG(log_.trace()) << "got initial object with key: " << ripple::strHex(obj.key);
|
||||
if (auto const mptHolder = etl::getMPTHolderFromObj(obj.keyRaw, obj.dataRaw); mptHolder.has_value())
|
||||
if (auto const mptHolder = getMPTHolderFromObj(obj.keyRaw, obj.dataRaw); mptHolder.has_value())
|
||||
backend_->writeMPTHolders({*mptHolder});
|
||||
}
|
||||
|
||||
@@ -67,7 +67,7 @@ MPTExt::writeMPTHoldersFromTransactions(model::LedgerData const& data)
|
||||
std::vector<MPTHolderData> holders;
|
||||
|
||||
for (auto const& tx : data.transactions) {
|
||||
if (auto const mptHolder = etl::getMPTHolderFromTx(tx.meta, tx.sttx); mptHolder.has_value())
|
||||
if (auto const mptHolder = getMPTHolderFromTx(tx.meta, tx.sttx); mptHolder.has_value())
|
||||
holders.push_back(*mptHolder);
|
||||
}
|
||||
|
||||
@@ -75,4 +75,4 @@ MPTExt::writeMPTHoldersFromTransactions(model::LedgerData const& data)
|
||||
backend_->writeMPTHolders(holders);
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -20,7 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <xrpl/basics/strHex.h>
|
||||
@@ -31,7 +31,7 @@
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class MPTExt {
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
@@ -54,4 +54,4 @@ private:
|
||||
writeMPTHoldersFromTransactions(model::LedgerData const& data);
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -17,12 +17,12 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/ext/NFT.hpp"
|
||||
#include "etl/impl/ext/NFT.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/NFTHelpers.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
@@ -30,7 +30,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
NFTExt::NFTExt(std::shared_ptr<BackendInterface> backend) : backend_(std::move(backend))
|
||||
{
|
||||
@@ -47,7 +47,7 @@ void
|
||||
NFTExt::onInitialObject(uint32_t seq, model::Object const& obj)
|
||||
{
|
||||
LOG(log_.trace()) << "got initial object with key = " << obj.key;
|
||||
backend_->writeNFTs(etl::getNFTDataFromObj(seq, obj.keyRaw, obj.dataRaw));
|
||||
backend_->writeNFTs(getNFTDataFromObj(seq, obj.keyRaw, obj.dataRaw));
|
||||
}
|
||||
|
||||
void
|
||||
@@ -64,15 +64,15 @@ NFTExt::writeNFTs(model::LedgerData const& data)
|
||||
std::vector<NFTTransactionsData> nftTxs;
|
||||
|
||||
for (auto const& tx : data.transactions) {
|
||||
auto const [txs, maybeNFT] = etl::getNFTDataFromTx(tx.meta, tx.sttx);
|
||||
auto const [txs, maybeNFT] = getNFTDataFromTx(tx.meta, tx.sttx);
|
||||
nftTxs.insert(nftTxs.end(), txs.begin(), txs.end());
|
||||
if (maybeNFT)
|
||||
nfts.push_back(*maybeNFT);
|
||||
}
|
||||
|
||||
// This is uniqued so that we only write latest modification (as in previous implementation)
|
||||
backend_->writeNFTs(etl::getUniqueNFTsDatas(nfts));
|
||||
backend_->writeNFTs(getUniqueNFTsDatas(nfts));
|
||||
backend_->writeNFTTransactions(nftTxs);
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -20,13 +20,13 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class NFTExt {
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
@@ -49,4 +49,4 @@ private:
|
||||
writeNFTs(model::LedgerData const& data);
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -17,13 +17,13 @@
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/impl/ext/Successor.hpp"
|
||||
#include "etl/impl/ext/Successor.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
@@ -39,7 +39,7 @@
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
SuccessorExt::SuccessorExt(std::shared_ptr<BackendInterface> backend, data::LedgerCacheInterface& cache)
|
||||
: backend_(std::move(backend)), cache_(cache)
|
||||
@@ -219,4 +219,4 @@ SuccessorExt::writeEdgeKeys(std::uint32_t seq, auto const& edgeKeys) const
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -22,7 +22,7 @@
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
@@ -35,7 +35,7 @@
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
namespace etl::impl {
|
||||
|
||||
class SuccessorExt {
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
@@ -79,4 +79,4 @@ private:
|
||||
writeEdgeKeys(std::uint32_t seq, auto const& edgeKeys) const;
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
} // namespace etl::impl
|
||||
@@ -1,23 +0,0 @@
|
||||
add_library(clio_etlng)
|
||||
|
||||
target_sources(
|
||||
clio_etlng
|
||||
PRIVATE ETLService.cpp
|
||||
LoadBalancer.cpp
|
||||
Source.cpp
|
||||
impl/AmendmentBlockHandler.cpp
|
||||
impl/AsyncGrpcCall.cpp
|
||||
impl/Extraction.cpp
|
||||
impl/GrpcSource.cpp
|
||||
impl/ForwardingSource.cpp
|
||||
impl/Loading.cpp
|
||||
impl/Monitor.cpp
|
||||
impl/TaskManager.cpp
|
||||
impl/ext/Cache.cpp
|
||||
impl/ext/Core.cpp
|
||||
impl/ext/MPT.cpp
|
||||
impl/ext/NFT.cpp
|
||||
impl/ext/Successor.cpp
|
||||
)
|
||||
|
||||
target_link_libraries(clio_etlng PUBLIC clio_data)
|
||||
@@ -1,334 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#include "etlng/ETLService.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/LedgerCacheInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/impl/AmendmentBlockHandler.hpp"
|
||||
#include "etl/impl/LedgerFetcher.hpp"
|
||||
#include "etlng/CacheLoaderInterface.hpp"
|
||||
#include "etlng/CacheUpdaterInterface.hpp"
|
||||
#include "etlng/ExtractorInterface.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LedgerPublisherInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/LoaderInterface.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "etlng/MonitorProviderInterface.hpp"
|
||||
#include "etlng/TaskManagerProviderInterface.hpp"
|
||||
#include "etlng/impl/AmendmentBlockHandler.hpp"
|
||||
#include "etlng/impl/CacheUpdater.hpp"
|
||||
#include "etlng/impl/Extraction.hpp"
|
||||
#include "etlng/impl/LedgerPublisher.hpp"
|
||||
#include "etlng/impl/Loading.hpp"
|
||||
#include "etlng/impl/Registry.hpp"
|
||||
#include "etlng/impl/Scheduling.hpp"
|
||||
#include "etlng/impl/TaskManager.hpp"
|
||||
#include "etlng/impl/ext/Cache.hpp"
|
||||
#include "etlng/impl/ext/Core.hpp"
|
||||
#include "etlng/impl/ext/NFT.hpp"
|
||||
#include "etlng/impl/ext/Successor.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Profiler.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/signals2/connection.hpp>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng {
|
||||
|
||||
ETLService::ETLService(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::reference_wrapper<util::config::ClioConfigDefinition const> config,
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::shared_ptr<LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> ledgers,
|
||||
std::shared_ptr<LedgerPublisherInterface> publisher,
|
||||
std::shared_ptr<CacheLoaderInterface> cacheLoader,
|
||||
std::shared_ptr<CacheUpdaterInterface> cacheUpdater,
|
||||
std::shared_ptr<ExtractorInterface> extractor,
|
||||
std::shared_ptr<LoaderInterface> loader,
|
||||
std::shared_ptr<InitialLoadObserverInterface> initialLoadObserver,
|
||||
std::shared_ptr<etlng::TaskManagerProviderInterface> taskManagerProvider,
|
||||
std::shared_ptr<etlng::MonitorProviderInterface> monitorProvider,
|
||||
std::shared_ptr<etl::SystemState> state
|
||||
)
|
||||
: ctx_(std::move(ctx))
|
||||
, config_(config)
|
||||
, backend_(std::move(backend))
|
||||
, balancer_(std::move(balancer))
|
||||
, ledgers_(std::move(ledgers))
|
||||
, publisher_(std::move(publisher))
|
||||
, cacheLoader_(std::move(cacheLoader))
|
||||
, cacheUpdater_(std::move(cacheUpdater))
|
||||
, extractor_(std::move(extractor))
|
||||
, loader_(std::move(loader))
|
||||
, initialLoadObserver_(std::move(initialLoadObserver))
|
||||
, taskManagerProvider_(std::move(taskManagerProvider))
|
||||
, monitorProvider_(std::move(monitorProvider))
|
||||
, state_(std::move(state))
|
||||
, startSequence_(config.get().maybeValue<uint32_t>("start_sequence"))
|
||||
, finishSequence_(config.get().maybeValue<uint32_t>("finish_sequence"))
|
||||
{
|
||||
ASSERT(not state_->isWriting, "ETL should never start in writer mode");
|
||||
|
||||
if (startSequence_.has_value())
|
||||
LOG(log_.info()) << "Start sequence: " << *startSequence_;
|
||||
|
||||
if (finishSequence_.has_value())
|
||||
LOG(log_.info()) << "Finish sequence: " << *finishSequence_;
|
||||
|
||||
LOG(log_.info()) << "Starting in " << (state_->isStrictReadonly ? "STRICT READONLY MODE" : "WRITE MODE");
|
||||
}
|
||||
|
||||
ETLService::~ETLService()
|
||||
{
|
||||
stop();
|
||||
LOG(log_.debug()) << "Destroying ETLng";
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::run()
|
||||
{
|
||||
LOG(log_.info()) << "Running ETLng...";
|
||||
|
||||
mainLoop_.emplace(ctx_.execute([this] {
|
||||
auto const rng = loadInitialLedgerIfNeeded();
|
||||
|
||||
LOG(log_.info()) << "Waiting for next ledger to be validated by network...";
|
||||
std::optional<uint32_t> const mostRecentValidated = ledgers_->getMostRecent();
|
||||
|
||||
if (not mostRecentValidated) {
|
||||
LOG(log_.info()) << "The wait for the next validated ledger has been aborted. "
|
||||
"Exiting monitor loop";
|
||||
return;
|
||||
}
|
||||
|
||||
if (not rng.has_value()) {
|
||||
LOG(log_.warn()) << "Initial ledger download got cancelled - stopping ETL service";
|
||||
return;
|
||||
}
|
||||
|
||||
auto const nextSequence = rng->maxSequence + 1;
|
||||
|
||||
LOG(log_.debug()) << "Database is populated. Starting monitor loop. sequence = " << nextSequence;
|
||||
startMonitor(nextSequence);
|
||||
|
||||
// If we are a writer as the result of loading the initial ledger - start loading
|
||||
if (state_->isWriting)
|
||||
startLoading(nextSequence);
|
||||
}));
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::stop()
|
||||
{
|
||||
LOG(log_.info()) << "Stop called";
|
||||
|
||||
if (mainLoop_)
|
||||
mainLoop_->wait();
|
||||
if (taskMan_)
|
||||
taskMan_->stop();
|
||||
if (monitor_)
|
||||
monitor_->stop();
|
||||
}
|
||||
|
||||
boost::json::object
|
||||
ETLService::getInfo() const
|
||||
{
|
||||
boost::json::object result;
|
||||
|
||||
result["etl_sources"] = balancer_->toJson();
|
||||
result["is_writer"] = static_cast<int>(state_->isWriting);
|
||||
result["read_only"] = static_cast<int>(state_->isStrictReadonly);
|
||||
auto last = publisher_->getLastPublish();
|
||||
if (last.time_since_epoch().count() != 0)
|
||||
result["last_publish_age_seconds"] = std::to_string(publisher_->lastPublishAgeSeconds());
|
||||
return result;
|
||||
}
|
||||
|
||||
bool
|
||||
ETLService::isAmendmentBlocked() const
|
||||
{
|
||||
return state_->isAmendmentBlocked;
|
||||
}
|
||||
|
||||
bool
|
||||
ETLService::isCorruptionDetected() const
|
||||
{
|
||||
return state_->isCorruptionDetected;
|
||||
}
|
||||
|
||||
std::optional<etl::ETLState>
|
||||
ETLService::getETLState() const
|
||||
{
|
||||
return balancer_->getETLState();
|
||||
}
|
||||
|
||||
std::uint32_t
|
||||
ETLService::lastCloseAgeSeconds() const
|
||||
{
|
||||
return publisher_->lastCloseAgeSeconds();
|
||||
}
|
||||
|
||||
std::optional<data::LedgerRange>
|
||||
ETLService::loadInitialLedgerIfNeeded()
|
||||
{
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
if (not rng.has_value()) {
|
||||
ASSERT(
|
||||
not state_->isStrictReadonly,
|
||||
"Database is empty but this node is in strict readonly mode. Can't write initial ledger."
|
||||
);
|
||||
|
||||
LOG(log_.info()) << "Database is empty. Will download a ledger from the network.";
|
||||
state_->isWriting = true; // immediately become writer as the db is empty
|
||||
|
||||
auto const getMostRecent = [this]() {
|
||||
LOG(log_.info()) << "Waiting for next ledger to be validated by network...";
|
||||
return ledgers_->getMostRecent();
|
||||
};
|
||||
|
||||
if (auto const maybeSeq = startSequence_.or_else(getMostRecent); maybeSeq.has_value()) {
|
||||
auto const seq = *maybeSeq;
|
||||
LOG(log_.info()) << "Starting from sequence " << seq
|
||||
<< ". Initial ledger download and extraction can take a while...";
|
||||
|
||||
auto [ledger, timeDiff] = ::util::timed<std::chrono::duration<double>>([this, seq]() {
|
||||
return extractor_->extractLedgerOnly(seq).and_then(
|
||||
[this, seq](auto&& data) -> std::optional<ripple::LedgerHeader> {
|
||||
// TODO: loadInitialLedger in balancer should be called fetchEdgeKeys or similar
|
||||
auto res = balancer_->loadInitialLedger(seq, *initialLoadObserver_);
|
||||
if (not res.has_value() and res.error() == InitialLedgerLoadError::Cancelled) {
|
||||
LOG(log_.debug()) << "Initial ledger load got cancelled";
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
ASSERT(res.has_value(), "Initial ledger retry logic failed");
|
||||
data.edgeKeys = std::move(res).value();
|
||||
|
||||
return loader_->loadInitialLedger(data);
|
||||
}
|
||||
);
|
||||
});
|
||||
|
||||
if (not ledger.has_value()) {
|
||||
LOG(log_.error()) << "Failed to load initial ledger. Exiting monitor loop";
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
LOG(log_.debug()) << "Time to download and store ledger = " << timeDiff;
|
||||
LOG(log_.info()) << "Finished loadInitialLedger. cache size = " << backend_->cache().size();
|
||||
|
||||
return backend_->hardFetchLedgerRangeNoThrow();
|
||||
}
|
||||
|
||||
LOG(log_.info()) << "The wait for the next validated ledger has been aborted. "
|
||||
"Exiting monitor loop";
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
LOG(log_.info()) << "Database already populated. Picking up from the tip of history";
|
||||
cacheLoader_->load(rng->maxSequence);
|
||||
|
||||
return rng;
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::startMonitor(uint32_t seq)
|
||||
{
|
||||
monitor_ = monitorProvider_->make(ctx_, backend_, ledgers_, seq);
|
||||
|
||||
monitorNewSeqSubscription_ = monitor_->subscribeToNewSequence([this](uint32_t seq) {
|
||||
LOG(log_.info()) << "ETLService (via Monitor) got new seq from db: " << seq;
|
||||
|
||||
if (state_->writeConflict) {
|
||||
LOG(log_.info()) << "Got a write conflict; Giving up writer seat immediately";
|
||||
giveUpWriter();
|
||||
}
|
||||
|
||||
if (not state_->isWriting) {
|
||||
auto const diff = data::synchronousAndRetryOnTimeout([this, seq](auto yield) {
|
||||
return backend_->fetchLedgerDiff(seq, yield);
|
||||
});
|
||||
|
||||
cacheUpdater_->update(seq, diff);
|
||||
backend_->updateRange(seq);
|
||||
}
|
||||
|
||||
publisher_->publish(seq, {});
|
||||
});
|
||||
|
||||
monitorDbStalledSubscription_ = monitor_->subscribeToDbStalled([this]() {
|
||||
LOG(log_.warn()) << "ETLService received DbStalled signal from Monitor";
|
||||
if (not state_->isStrictReadonly and not state_->isWriting)
|
||||
attemptTakeoverWriter();
|
||||
});
|
||||
|
||||
monitor_->run();
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::startLoading(uint32_t seq)
|
||||
{
|
||||
ASSERT(not state_->isStrictReadonly, "This should only happen on writer nodes");
|
||||
taskMan_ = taskManagerProvider_->make(ctx_, *monitor_, seq, finishSequence_);
|
||||
|
||||
// FIXME: this legacy name "extractor_threads" is no longer accurate (we have coroutines now)
|
||||
taskMan_->run(config_.get().get<std::size_t>("extractor_threads"));
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::attemptTakeoverWriter()
|
||||
{
|
||||
ASSERT(not state_->isStrictReadonly, "This should only happen on writer nodes");
|
||||
auto rng = backend_->hardFetchLedgerRangeNoThrow();
|
||||
ASSERT(rng.has_value(), "Ledger range can't be null");
|
||||
|
||||
state_->isWriting = true; // switch to writer
|
||||
LOG(log_.info()) << "Taking over the ETL writer seat";
|
||||
startLoading(rng->maxSequence + 1);
|
||||
}
|
||||
|
||||
void
|
||||
ETLService::giveUpWriter()
|
||||
{
|
||||
ASSERT(not state_->isStrictReadonly, "This should only happen on writer nodes");
|
||||
state_->isWriting = false;
|
||||
state_->writeConflict = false;
|
||||
taskMan_ = nullptr;
|
||||
}
|
||||
|
||||
} // namespace etlng
|
||||
@@ -1,199 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etl/impl/AmendmentBlockHandler.hpp"
|
||||
#include "etl/impl/LedgerFetcher.hpp"
|
||||
#include "etlng/CacheLoaderInterface.hpp"
|
||||
#include "etlng/CacheUpdaterInterface.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etlng/ExtractorInterface.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LedgerPublisherInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/LoaderInterface.hpp"
|
||||
#include "etlng/MonitorInterface.hpp"
|
||||
#include "etlng/MonitorProviderInterface.hpp"
|
||||
#include "etlng/TaskManagerInterface.hpp"
|
||||
#include "etlng/TaskManagerProviderInterface.hpp"
|
||||
#include "etlng/impl/AmendmentBlockHandler.hpp"
|
||||
#include "etlng/impl/CacheUpdater.hpp"
|
||||
#include "etlng/impl/Extraction.hpp"
|
||||
#include "etlng/impl/LedgerPublisher.hpp"
|
||||
#include "etlng/impl/Loading.hpp"
|
||||
#include "etlng/impl/Registry.hpp"
|
||||
#include "etlng/impl/Scheduling.hpp"
|
||||
#include "etlng/impl/TaskManager.hpp"
|
||||
#include "etlng/impl/ext/Cache.hpp"
|
||||
#include "etlng/impl/ext/Core.hpp"
|
||||
#include "etlng/impl/ext/NFT.hpp"
|
||||
#include "etlng/impl/ext/Successor.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/async/AnyOperation.hpp"
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/signals2/connection.hpp>
|
||||
#include <fmt/format.h>
|
||||
#include <xrpl/basics/Blob.h>
|
||||
#include <xrpl/basics/base_uint.h>
|
||||
#include <xrpl/basics/strHex.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/ledger.pb.h>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
#include <xrpl/protocol/STTx.h>
|
||||
#include <xrpl/protocol/TxFormats.h>
|
||||
#include <xrpl/protocol/TxMeta.h>
|
||||
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
|
||||
namespace etlng {
|
||||
|
||||
/**
|
||||
* @brief This class is responsible for continuously extracting data from a p2p node, and writing that data to the
|
||||
* databases.
|
||||
*
|
||||
* Usually, multiple different processes share access to the same network accessible databases, in which case only one
|
||||
* such process is performing ETL and writing to the database. The other processes simply monitor the database for new
|
||||
* ledgers, and publish those ledgers to the various subscription streams. If a monitoring process determines that the
|
||||
* ETL writer has failed (no new ledgers written for some time), the process will attempt to become the ETL writer.
|
||||
*
|
||||
* If there are multiple monitoring processes that try to become the ETL writer at the same time, one will win out, and
|
||||
* the others will fall back to monitoring/publishing. In this sense, this class dynamically transitions from monitoring
|
||||
* to writing and from writing to monitoring, based on the activity of other processes running on different machines.
|
||||
*/
|
||||
class ETLService : public ETLServiceInterface {
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
util::async::AnyExecutionContext ctx_;
|
||||
std::reference_wrapper<util::config::ClioConfigDefinition const> config_;
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<LoadBalancerInterface> balancer_;
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> ledgers_;
|
||||
std::shared_ptr<LedgerPublisherInterface> publisher_;
|
||||
std::shared_ptr<CacheLoaderInterface> cacheLoader_;
|
||||
std::shared_ptr<CacheUpdaterInterface> cacheUpdater_;
|
||||
std::shared_ptr<ExtractorInterface> extractor_;
|
||||
std::shared_ptr<LoaderInterface> loader_;
|
||||
std::shared_ptr<InitialLoadObserverInterface> initialLoadObserver_;
|
||||
std::shared_ptr<etlng::TaskManagerProviderInterface> taskManagerProvider_;
|
||||
std::shared_ptr<etlng::MonitorProviderInterface> monitorProvider_;
|
||||
std::shared_ptr<etl::SystemState> state_;
|
||||
|
||||
std::optional<uint32_t> startSequence_;
|
||||
std::optional<uint32_t> finishSequence_;
|
||||
|
||||
std::unique_ptr<MonitorInterface> monitor_;
|
||||
std::unique_ptr<TaskManagerInterface> taskMan_;
|
||||
|
||||
boost::signals2::scoped_connection monitorNewSeqSubscription_;
|
||||
boost::signals2::scoped_connection monitorDbStalledSubscription_;
|
||||
|
||||
std::optional<util::async::AnyOperation<void>> mainLoop_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of ETLService.
|
||||
*
|
||||
* @param ctx The execution context for asynchronous operations
|
||||
* @param config The Clio configuration definition
|
||||
* @param backend Interface to the backend database
|
||||
* @param balancer Load balancer for distributing work
|
||||
* @param ledgers Interface for accessing network validated ledgers
|
||||
* @param publisher Interface for publishing ledger data
|
||||
* @param cacheLoader Interface for loading cache data
|
||||
* @param cacheUpdater Interface for updating cache data
|
||||
* @param extractor The extractor to use
|
||||
* @param loader Interface for loading data
|
||||
* @param initialLoadObserver The observer for initial data loading
|
||||
* @param taskManagerProvider The provider of the task manager instance
|
||||
* @param monitorProvider The provider of the monitor instance
|
||||
* @param state System state tracking object
|
||||
*/
|
||||
ETLService(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
std::reference_wrapper<util::config::ClioConfigDefinition const> config,
|
||||
std::shared_ptr<data::BackendInterface> backend,
|
||||
std::shared_ptr<LoadBalancerInterface> balancer,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> ledgers,
|
||||
std::shared_ptr<LedgerPublisherInterface> publisher,
|
||||
std::shared_ptr<CacheLoaderInterface> cacheLoader,
|
||||
std::shared_ptr<CacheUpdaterInterface> cacheUpdater,
|
||||
std::shared_ptr<ExtractorInterface> extractor,
|
||||
std::shared_ptr<LoaderInterface> loader,
|
||||
std::shared_ptr<InitialLoadObserverInterface> initialLoadObserver,
|
||||
std::shared_ptr<etlng::TaskManagerProviderInterface> taskManagerProvider,
|
||||
std::shared_ptr<etlng::MonitorProviderInterface> monitorProvider,
|
||||
std::shared_ptr<etl::SystemState> state
|
||||
);
|
||||
|
||||
~ETLService() override;
|
||||
|
||||
void
|
||||
run() override;
|
||||
|
||||
void
|
||||
stop() override;
|
||||
|
||||
boost::json::object
|
||||
getInfo() const override;
|
||||
|
||||
bool
|
||||
isAmendmentBlocked() const override;
|
||||
|
||||
bool
|
||||
isCorruptionDetected() const override;
|
||||
|
||||
std::optional<etl::ETLState>
|
||||
getETLState() const override;
|
||||
|
||||
std::uint32_t
|
||||
lastCloseAgeSeconds() const override;
|
||||
|
||||
private:
|
||||
std::optional<data::LedgerRange>
|
||||
loadInitialLedgerIfNeeded();
|
||||
|
||||
void
|
||||
startMonitor(uint32_t seq);
|
||||
|
||||
void
|
||||
startLoading(uint32_t seq);
|
||||
|
||||
void
|
||||
attemptTakeoverWriter();
|
||||
|
||||
void
|
||||
giveUpWriter();
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
@@ -1,416 +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 "etlng/LoadBalancer.hpp"
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etl/ETLState.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/Source.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/CoroutineGroup.hpp"
|
||||
#include "util/Profiler.hpp"
|
||||
#include "util/Random.hpp"
|
||||
#include "util/ResponseExpirationCache.hpp"
|
||||
#include "util/config/ArrayView.hpp"
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/config/ObjectView.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/prometheus/Label.hpp"
|
||||
#include "util/prometheus/Prometheus.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/array.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/value.hpp>
|
||||
#include <boost/json/value_to.hpp>
|
||||
#include <fmt/format.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <expected>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
using namespace util::config;
|
||||
using util::prometheus::Labels;
|
||||
|
||||
namespace etlng {
|
||||
|
||||
std::shared_ptr<LoadBalancerInterface>
|
||||
LoadBalancer::makeLoadBalancer(
|
||||
ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::unique_ptr<util::RandomGeneratorInterface> randomGenerator,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
SourceFactory sourceFactory
|
||||
)
|
||||
{
|
||||
return std::make_shared<LoadBalancer>(
|
||||
config,
|
||||
ioc,
|
||||
std::move(backend),
|
||||
std::move(subscriptions),
|
||||
std::move(randomGenerator),
|
||||
std::move(validatedLedgers),
|
||||
std::move(sourceFactory)
|
||||
);
|
||||
}
|
||||
|
||||
LoadBalancer::LoadBalancer(
|
||||
ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::unique_ptr<util::RandomGeneratorInterface> randomGenerator,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
SourceFactory sourceFactory
|
||||
)
|
||||
: randomGenerator_(std::move(randomGenerator))
|
||||
, forwardingCounters_{
|
||||
.successDuration = PrometheusService::counterInt(
|
||||
"forwarding_duration_milliseconds_counter",
|
||||
Labels({util::prometheus::Label{"status", "success"}}),
|
||||
"The duration of processing successful forwarded requests"
|
||||
),
|
||||
.failDuration = PrometheusService::counterInt(
|
||||
"forwarding_duration_milliseconds_counter",
|
||||
Labels({util::prometheus::Label{"status", "fail"}}),
|
||||
"The duration of processing failed forwarded requests"
|
||||
),
|
||||
.retries = PrometheusService::counterInt(
|
||||
"forwarding_retries_counter",
|
||||
Labels(),
|
||||
"The number of retries before a forwarded request was successful. Initial attempt excluded"
|
||||
),
|
||||
.cacheHit = PrometheusService::counterInt(
|
||||
"forwarding_cache_hit_counter",
|
||||
Labels(),
|
||||
"The number of requests that we served from the cache"
|
||||
),
|
||||
.cacheMiss = PrometheusService::counterInt(
|
||||
"forwarding_cache_miss_counter",
|
||||
Labels(),
|
||||
"The number of requests that were not served from the cache"
|
||||
)
|
||||
}
|
||||
{
|
||||
auto const forwardingCacheTimeout = config.get<float>("forwarding.cache_timeout");
|
||||
if (forwardingCacheTimeout > 0.f) {
|
||||
forwardingCache_ = util::ResponseExpirationCache{
|
||||
util::config::ClioConfigDefinition::toMilliseconds(forwardingCacheTimeout),
|
||||
{"server_info", "server_state", "server_definitions", "fee", "ledger_closed"}
|
||||
};
|
||||
}
|
||||
|
||||
auto const numMarkers = config.getValueView("num_markers");
|
||||
if (numMarkers.hasValue()) {
|
||||
auto const value = numMarkers.asIntType<uint32_t>();
|
||||
downloadRanges_ = value;
|
||||
} else if (backend->fetchLedgerRange()) {
|
||||
downloadRanges_ = 4;
|
||||
}
|
||||
|
||||
auto const allowNoEtl = config.get<bool>("allow_no_etl");
|
||||
|
||||
auto const checkOnETLFailure = [this, allowNoEtl](std::string const& 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.";
|
||||
throw std::logic_error("ETL configuration error.");
|
||||
}
|
||||
};
|
||||
|
||||
auto const forwardingTimeout =
|
||||
ClioConfigDefinition::toMilliseconds(config.get<float>("forwarding.request_timeout"));
|
||||
auto const etlArray = config.getArray("etl_sources");
|
||||
for (auto it = etlArray.begin<ObjectView>(); it != etlArray.end<ObjectView>(); ++it) {
|
||||
auto source = sourceFactory(
|
||||
*it,
|
||||
ioc,
|
||||
subscriptions,
|
||||
validatedLedgers,
|
||||
forwardingTimeout,
|
||||
[this]() {
|
||||
if (not hasForwardingSource_.lock().get())
|
||||
chooseForwardingSource();
|
||||
},
|
||||
[this](bool wasForwarding) {
|
||||
if (wasForwarding)
|
||||
chooseForwardingSource();
|
||||
},
|
||||
[this]() {
|
||||
if (forwardingCache_.has_value())
|
||||
forwardingCache_->invalidate();
|
||||
}
|
||||
);
|
||||
|
||||
// checking etl node validity
|
||||
auto const stateOpt = etl::ETLState::fetchETLStateFromSource(*source);
|
||||
|
||||
if (!stateOpt) {
|
||||
LOG(log_.warn()) << "Failed to fetch ETL state from source = " << source->toString()
|
||||
<< " Please check the configuration and network";
|
||||
} else if (etlState_ && etlState_->networkID != stateOpt->networkID) {
|
||||
checkOnETLFailure(
|
||||
fmt::format(
|
||||
"ETL sources must be on the same network. Source network id = {} does not match others network id "
|
||||
"= {}",
|
||||
stateOpt->networkID,
|
||||
etlState_->networkID
|
||||
)
|
||||
);
|
||||
} else {
|
||||
etlState_ = stateOpt;
|
||||
}
|
||||
|
||||
sources_.push_back(std::move(source));
|
||||
LOG(log_.info()) << "Added etl source - " << sources_.back()->toString();
|
||||
}
|
||||
|
||||
if (!etlState_)
|
||||
checkOnETLFailure("Failed to fetch ETL state from any source. Please check the configuration and network");
|
||||
|
||||
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 const& source : sources_) {
|
||||
source->run();
|
||||
}
|
||||
}
|
||||
|
||||
InitialLedgerLoadResult
|
||||
LoadBalancer::loadInitialLedger(
|
||||
uint32_t sequence,
|
||||
etlng::InitialLoadObserverInterface& loadObserver,
|
||||
std::chrono::steady_clock::duration retryAfter
|
||||
)
|
||||
{
|
||||
InitialLedgerLoadResult response;
|
||||
|
||||
execute(
|
||||
[this, &response, &sequence, &loadObserver](auto& source) {
|
||||
auto res = source->loadInitialLedger(sequence, downloadRanges_, loadObserver);
|
||||
|
||||
if (not res.has_value() and res.error() == InitialLedgerLoadError::Errored) {
|
||||
LOG(log_.error()) << "Failed to download initial ledger."
|
||||
<< " Sequence = " << sequence << " source = " << source->toString();
|
||||
return false; // should retry on error
|
||||
}
|
||||
|
||||
response = std::move(res); // cancelled or data received
|
||||
return true;
|
||||
},
|
||||
sequence,
|
||||
retryAfter
|
||||
);
|
||||
|
||||
return response;
|
||||
}
|
||||
|
||||
LoadBalancer::OptionalGetLedgerResponseType
|
||||
LoadBalancer::fetchLedger(
|
||||
uint32_t ledgerSequence,
|
||||
bool getObjects,
|
||||
bool getObjectNeighbors,
|
||||
std::chrono::steady_clock::duration retryAfter
|
||||
)
|
||||
{
|
||||
GetLedgerResponseType response;
|
||||
execute(
|
||||
[&response, ledgerSequence, getObjects, getObjectNeighbors, log = log_](auto& source) {
|
||||
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();
|
||||
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();
|
||||
return false;
|
||||
},
|
||||
ledgerSequence,
|
||||
retryAfter
|
||||
);
|
||||
return response;
|
||||
}
|
||||
|
||||
std::expected<boost::json::object, rpc::CombinedError>
|
||||
LoadBalancer::forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& clientIp,
|
||||
bool isAdmin,
|
||||
boost::asio::yield_context yield
|
||||
)
|
||||
{
|
||||
if (not request.contains("command"))
|
||||
return std::unexpected{rpc::ClioError::RpcCommandIsMissing};
|
||||
|
||||
auto const cmd = boost::json::value_to<std::string>(request.at("command"));
|
||||
if (forwardingCache_) {
|
||||
if (auto cachedResponse = forwardingCache_->get(cmd); cachedResponse) {
|
||||
forwardingCounters_.cacheHit.get() += 1;
|
||||
return std::move(cachedResponse).value();
|
||||
}
|
||||
}
|
||||
forwardingCounters_.cacheMiss.get() += 1;
|
||||
|
||||
ASSERT(not sources_.empty(), "ETL sources must be configured to forward requests.");
|
||||
std::size_t sourceIdx = randomGenerator_->uniform(0ul, sources_.size() - 1);
|
||||
|
||||
auto numAttempts = 0u;
|
||||
|
||||
auto xUserValue = isAdmin ? kADMIN_FORWARDING_X_USER_VALUE : kUSER_FORWARDING_X_USER_VALUE;
|
||||
|
||||
std::optional<boost::json::object> response;
|
||||
rpc::ClioError error = rpc::ClioError::EtlConnectionError;
|
||||
while (numAttempts < sources_.size()) {
|
||||
auto [res, duration] =
|
||||
util::timed([&]() { return sources_[sourceIdx]->forwardToRippled(request, clientIp, xUserValue, yield); });
|
||||
if (res) {
|
||||
forwardingCounters_.successDuration.get() += duration;
|
||||
response = std::move(res).value();
|
||||
break;
|
||||
}
|
||||
forwardingCounters_.failDuration.get() += duration;
|
||||
++forwardingCounters_.retries.get();
|
||||
error = std::max(error, res.error()); // Choose the best result between all sources
|
||||
|
||||
sourceIdx = (sourceIdx + 1) % sources_.size();
|
||||
++numAttempts;
|
||||
}
|
||||
|
||||
if (response) {
|
||||
if (forwardingCache_ and not response->contains("error"))
|
||||
forwardingCache_->put(cmd, *response);
|
||||
return std::move(response).value();
|
||||
}
|
||||
|
||||
return std::unexpected{error};
|
||||
}
|
||||
|
||||
boost::json::value
|
||||
LoadBalancer::toJson() const
|
||||
{
|
||||
boost::json::array ret;
|
||||
for (auto& src : sources_)
|
||||
ret.push_back(src->toJson());
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename Func>
|
||||
void
|
||||
LoadBalancer::execute(Func f, uint32_t ledgerSequence, std::chrono::steady_clock::duration retryAfter)
|
||||
{
|
||||
ASSERT(not sources_.empty(), "ETL sources must be configured to execute functions.");
|
||||
size_t sourceIdx = randomGenerator_->uniform(0ul, sources_.size() - 1);
|
||||
|
||||
size_t numAttempts = 0;
|
||||
|
||||
while (true) {
|
||||
auto& source = sources_[sourceIdx];
|
||||
|
||||
LOG(log_.debug()) << "Attempting to execute func. ledger sequence = " << ledgerSequence
|
||||
<< " - 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)) {
|
||||
bool const res = f(source);
|
||||
if (res) {
|
||||
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()
|
||||
<< " - ledger sequence = " << ledgerSequence;
|
||||
} else {
|
||||
LOG(log_.warn()) << "Ledger not present at source = " << source->toString()
|
||||
<< " - ledger sequence = " << ledgerSequence;
|
||||
}
|
||||
sourceIdx = (sourceIdx + 1) % sources_.size();
|
||||
numAttempts++;
|
||||
if (numAttempts % sources_.size() == 0) {
|
||||
LOG(log_.info()) << "Ledger sequence " << ledgerSequence
|
||||
<< " is not yet available from any configured sources. Sleeping and trying again";
|
||||
std::this_thread::sleep_for(retryAfter);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::optional<etl::ETLState>
|
||||
LoadBalancer::getETLState() noexcept
|
||||
{
|
||||
if (!etlState_) {
|
||||
// retry ETLState fetch
|
||||
etlState_ = etl::ETLState::fetchETLStateFromSource(*this);
|
||||
}
|
||||
return etlState_;
|
||||
}
|
||||
|
||||
void
|
||||
LoadBalancer::stop(boost::asio::yield_context yield)
|
||||
{
|
||||
util::CoroutineGroup group{yield};
|
||||
std::ranges::for_each(sources_, [&group, yield](auto& source) {
|
||||
group.spawn(yield, [&source](boost::asio::yield_context innerYield) { source->stop(innerYield); });
|
||||
});
|
||||
group.asyncWait(yield);
|
||||
}
|
||||
|
||||
void
|
||||
LoadBalancer::chooseForwardingSource()
|
||||
{
|
||||
LOG(log_.info()) << "Choosing a new source to forward subscriptions";
|
||||
auto hasForwardingSourceLock = hasForwardingSource_.lock();
|
||||
hasForwardingSourceLock.get() = false;
|
||||
for (auto& source : sources_) {
|
||||
if (not hasForwardingSourceLock.get() and source->isConnected()) {
|
||||
source->setForwarding(true);
|
||||
hasForwardingSourceLock.get() = true;
|
||||
} else {
|
||||
source->setForwarding(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etlng
|
||||
@@ -1,287 +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/ETLState.hpp"
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/Source.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Mutex.hpp"
|
||||
#include "util/Random.hpp"
|
||||
#include "util/ResponseExpirationCache.hpp"
|
||||
#include "util/config/ConfigDefinition.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/prometheus/Counter.hpp"
|
||||
|
||||
#include <boost/asio.hpp>
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <boost/json/value.hpp>
|
||||
#include <grpcpp/grpcpp.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <org/xrpl/rpc/v1/ledger.pb.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <concepts>
|
||||
#include <cstdint>
|
||||
#include <expected>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng {
|
||||
|
||||
/**
|
||||
* @brief A tag class to help identify LoadBalancer in templated code.
|
||||
*/
|
||||
struct LoadBalancerTag {
|
||||
virtual ~LoadBalancerTag() = default;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
concept SomeLoadBalancer = std::derived_from<T, LoadBalancerTag>;
|
||||
|
||||
/**
|
||||
* @brief This class is used to manage connections to transaction processing processes.
|
||||
*
|
||||
* This class spawns a listener for each etl source, which listens to messages on the ledgers stream (to keep track of
|
||||
* which ledgers have been validated by the network, and the range of ledgers each etl source has). This class also
|
||||
* allows requests for ledger data to be load balanced across all possible ETL sources.
|
||||
*/
|
||||
class LoadBalancer : public etlng::LoadBalancerInterface, LoadBalancerTag {
|
||||
public:
|
||||
using RawLedgerObjectType = org::xrpl::rpc::v1::RawLedgerObject;
|
||||
using GetLedgerResponseType = org::xrpl::rpc::v1::GetLedgerResponse;
|
||||
using OptionalGetLedgerResponseType = std::optional<GetLedgerResponseType>;
|
||||
|
||||
private:
|
||||
static constexpr std::uint32_t kDEFAULT_DOWNLOAD_RANGES = 16;
|
||||
|
||||
util::Logger log_{"ETL"};
|
||||
// Forwarding cache must be destroyed after sources because sources have a callback to invalidate cache
|
||||
std::optional<util::ResponseExpirationCache> forwardingCache_;
|
||||
std::optional<std::string> forwardingXUserValue_;
|
||||
|
||||
std::unique_ptr<util::RandomGeneratorInterface> randomGenerator_;
|
||||
|
||||
std::vector<SourcePtr> sources_;
|
||||
std::optional<etl::ETLState> etlState_;
|
||||
std::uint32_t downloadRanges_ =
|
||||
kDEFAULT_DOWNLOAD_RANGES; /*< The number of markers to use when downloading initial ledger */
|
||||
|
||||
struct ForwardingCounters {
|
||||
std::reference_wrapper<util::prometheus::CounterInt> successDuration;
|
||||
std::reference_wrapper<util::prometheus::CounterInt> failDuration;
|
||||
std::reference_wrapper<util::prometheus::CounterInt> retries;
|
||||
std::reference_wrapper<util::prometheus::CounterInt> cacheHit;
|
||||
std::reference_wrapper<util::prometheus::CounterInt> cacheMiss;
|
||||
} forwardingCounters_;
|
||||
|
||||
// Using mutext instead of atomic_bool because choosing a new source to
|
||||
// forward messages should be done with a mutual exclusion otherwise there will be a race condition
|
||||
util::Mutex<bool> hasForwardingSource_{false};
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Value for the X-User header when forwarding admin requests
|
||||
*/
|
||||
static constexpr std::string_view kADMIN_FORWARDING_X_USER_VALUE = "clio_admin";
|
||||
|
||||
/**
|
||||
* @brief Value for the X-User header when forwarding user requests
|
||||
*/
|
||||
static constexpr std::string_view kUSER_FORWARDING_X_USER_VALUE = "clio_user";
|
||||
|
||||
/**
|
||||
* @brief Create an instance of the load balancer.
|
||||
*
|
||||
* @param config The configuration to use
|
||||
* @param ioc The io_context to run on
|
||||
* @param backend BackendInterface implementation
|
||||
* @param subscriptions Subscription manager
|
||||
* @param randomGenerator A random generator to use for selecting sources
|
||||
* @param validatedLedgers The network validated ledgers datastructure
|
||||
* @param sourceFactory A factory function to create a source
|
||||
*/
|
||||
LoadBalancer(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::unique_ptr<util::RandomGeneratorInterface> randomGenerator,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
SourceFactory sourceFactory = makeSource
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief A factory function for the load balancer.
|
||||
*
|
||||
* @param config The configuration to use
|
||||
* @param ioc The io_context to run on
|
||||
* @param backend BackendInterface implementation
|
||||
* @param subscriptions Subscription manager
|
||||
* @param randomGenerator A random generator to use for selecting sources
|
||||
* @param validatedLedgers The network validated ledgers datastructure
|
||||
* @param sourceFactory A factory function to create a source
|
||||
* @return A shared pointer to a new instance of LoadBalancer
|
||||
*/
|
||||
static std::shared_ptr<LoadBalancerInterface>
|
||||
makeLoadBalancer(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::unique_ptr<util::RandomGeneratorInterface> randomGenerator,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
SourceFactory sourceFactory = makeSource
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Load the initial ledger, writing data to the queue.
|
||||
* @note This function will retry indefinitely until the ledger is downloaded.
|
||||
*
|
||||
* @param sequence Sequence of ledger to download
|
||||
* @param retryAfter Time to wait between retries (2 seconds by default)
|
||||
* @return A std::vector<std::string> The ledger data
|
||||
*/
|
||||
std::vector<std::string>
|
||||
loadInitialLedger(
|
||||
[[maybe_unused]] uint32_t sequence,
|
||||
[[maybe_unused]] std::chrono::steady_clock::duration retryAfter
|
||||
) override
|
||||
{
|
||||
ASSERT(false, "Not available for new ETL");
|
||||
std::unreachable();
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief Load the initial ledger, writing data to the queue.
|
||||
* @note This function will retry indefinitely until the ledger is downloaded or the download is cancelled.
|
||||
*
|
||||
* @param sequence Sequence of ledger to download
|
||||
* @param observer The observer to notify of progress
|
||||
* @param retryAfter Time to wait between retries (2 seconds by default)
|
||||
* @return A std::expected with ledger edge keys on success, or InitialLedgerLoadError on failure
|
||||
*/
|
||||
InitialLedgerLoadResult
|
||||
loadInitialLedger(
|
||||
uint32_t sequence,
|
||||
etlng::InitialLoadObserverInterface& observer,
|
||||
std::chrono::steady_clock::duration retryAfter
|
||||
) override;
|
||||
|
||||
/**
|
||||
* @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 ledgerSequence Sequence of the ledger to fetch
|
||||
* @param getObjects Whether to get the account state diff between this ledger and the prior one
|
||||
* @param getObjectNeighbors Whether to request object neighbors
|
||||
* @param retryAfter Time to wait between retries (2 seconds by default)
|
||||
* @return The extracted data, if extraction was successful. If the ledger was found
|
||||
* in the database or the server is shutting down, the optional will be empty
|
||||
*/
|
||||
OptionalGetLedgerResponseType
|
||||
fetchLedger(
|
||||
uint32_t ledgerSequence,
|
||||
bool getObjects,
|
||||
bool getObjectNeighbors,
|
||||
std::chrono::steady_clock::duration retryAfter = std::chrono::seconds{2}
|
||||
) override;
|
||||
|
||||
/**
|
||||
* @brief Represent the state of this load balancer as a JSON object
|
||||
*
|
||||
* @return JSON representation of the state of this load balancer.
|
||||
*/
|
||||
boost::json::value
|
||||
toJson() const override;
|
||||
|
||||
/**
|
||||
* @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, if known
|
||||
* @param isAdmin Whether the request is from an admin
|
||||
* @param yield The coroutine context
|
||||
* @return Response received from rippled node as JSON object on success or error on failure
|
||||
*/
|
||||
std::expected<boost::json::object, rpc::CombinedError>
|
||||
forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& clientIp,
|
||||
bool isAdmin,
|
||||
boost::asio::yield_context yield
|
||||
) override;
|
||||
|
||||
/**
|
||||
* @brief Return state of ETL nodes.
|
||||
* @return ETL state, nullopt if etl nodes not available
|
||||
*/
|
||||
std::optional<etl::ETLState>
|
||||
getETLState() noexcept override;
|
||||
|
||||
/**
|
||||
* @brief Stop the load balancer. This will stop all subscription sources.
|
||||
* @note This function will asynchronously wait for all sources to stop.
|
||||
*
|
||||
* @param yield The coroutine context
|
||||
*/
|
||||
void
|
||||
stop(boost::asio::yield_context yield) override;
|
||||
|
||||
private:
|
||||
/**
|
||||
* @brief Execute a function on a randomly selected source.
|
||||
*
|
||||
* @note f is a function that takes an Source as an argument and returns a bool.
|
||||
* Attempt to execute f for one randomly chosen Source that has the specified ledger. If f returns false, another
|
||||
* randomly chosen Source is used. The process repeats until f returns true.
|
||||
*
|
||||
* @param f Function to execute. This function takes the ETL source as an argument, and returns a bool
|
||||
* @param ledgerSequence f is executed for each Source that has this ledger
|
||||
* @param retryAfter Time to wait between retries (2 seconds by default)
|
||||
* server is shutting down
|
||||
*/
|
||||
template <typename Func>
|
||||
void
|
||||
execute(Func f, uint32_t ledgerSequence, std::chrono::steady_clock::duration retryAfter = std::chrono::seconds{2});
|
||||
|
||||
/**
|
||||
* @brief Choose a new source to forward requests
|
||||
*/
|
||||
void
|
||||
chooseForwardingSource();
|
||||
};
|
||||
|
||||
} // namespace etlng
|
||||
@@ -1,73 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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 "etlng/Source.hpp"
|
||||
|
||||
#include "etl/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etl/impl/SubscriptionSource.hpp"
|
||||
#include "etlng/impl/ForwardingSource.hpp"
|
||||
#include "etlng/impl/GrpcSource.hpp"
|
||||
#include "etlng/impl/SourceImpl.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "util/config/ObjectView.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng {
|
||||
|
||||
SourcePtr
|
||||
makeSource(
|
||||
util::config::ObjectView const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
SourceBase::OnConnectHook onConnect,
|
||||
SourceBase::OnDisconnectHook onDisconnect,
|
||||
SourceBase::OnLedgerClosedHook onLedgerClosed
|
||||
)
|
||||
{
|
||||
auto const ip = config.get<std::string>("ip");
|
||||
auto const wsPort = config.get<std::string>("ws_port");
|
||||
auto const grpcPort = config.get<std::string>("grpc_port");
|
||||
|
||||
etlng::impl::ForwardingSource forwardingSource{ip, wsPort, forwardingTimeout};
|
||||
impl::GrpcSource grpcSource{ip, grpcPort};
|
||||
auto subscriptionSource = std::make_unique<etl::impl::SubscriptionSource>(
|
||||
ioc,
|
||||
ip,
|
||||
wsPort,
|
||||
std::move(validatedLedgers),
|
||||
std::move(subscriptions),
|
||||
std::move(onConnect),
|
||||
std::move(onDisconnect),
|
||||
std::move(onLedgerClosed)
|
||||
);
|
||||
|
||||
return std::make_unique<impl::SourceImpl<>>(
|
||||
ip, wsPort, grpcPort, std::move(grpcSource), std::move(subscriptionSource), std::move(forwardingSource)
|
||||
);
|
||||
}
|
||||
|
||||
} // namespace etlng
|
||||
@@ -1,194 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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/NetworkValidatedLedgersInterface.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "util/config/ObjectView.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/spawn.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 <chrono>
|
||||
#include <cstdint>
|
||||
#include <expected>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng {
|
||||
|
||||
/**
|
||||
* @brief Provides an implementation of a ETL source
|
||||
*/
|
||||
class SourceBase {
|
||||
public:
|
||||
using OnConnectHook = std::function<void()>;
|
||||
using OnDisconnectHook = std::function<void(bool)>;
|
||||
using OnLedgerClosedHook = std::function<void()>;
|
||||
|
||||
virtual ~SourceBase() = default;
|
||||
|
||||
/**
|
||||
* @brief Run subscriptions loop of the source
|
||||
*/
|
||||
virtual void
|
||||
run() = 0;
|
||||
|
||||
/**
|
||||
* @brief Stop Source.
|
||||
* @note This method will asynchronously wait for source to be stopped.
|
||||
*
|
||||
* @param yield The coroutine context.
|
||||
*/
|
||||
virtual void
|
||||
stop(boost::asio::yield_context yield) = 0;
|
||||
|
||||
/**
|
||||
* @brief Check if source is connected
|
||||
*
|
||||
* @return true if source is connected; false otherwise
|
||||
*/
|
||||
[[nodiscard]] virtual bool
|
||||
isConnected() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Set the forwarding state of the source.
|
||||
*
|
||||
* @param isForwarding Whether to forward or not
|
||||
*/
|
||||
virtual void
|
||||
setForwarding(bool isForwarding) = 0;
|
||||
|
||||
/**
|
||||
* @brief Represent the source as a JSON object
|
||||
*
|
||||
* @return JSON representation of the source
|
||||
*/
|
||||
[[nodiscard]] virtual boost::json::object
|
||||
toJson() const = 0;
|
||||
|
||||
/** @return String representation of the source (for debug) */
|
||||
[[nodiscard]] virtual std::string
|
||||
toString() const = 0;
|
||||
|
||||
/**
|
||||
* @brief Check if ledger is known by this source.
|
||||
*
|
||||
* @param sequence The ledger sequence to check
|
||||
* @return true if ledger is in the range of this source; false otherwise
|
||||
*/
|
||||
[[nodiscard]] virtual bool
|
||||
hasLedger(uint32_t sequence) const = 0;
|
||||
|
||||
/**
|
||||
* @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
|
||||
*/
|
||||
[[nodiscard]] virtual std::pair<grpc::Status, org::xrpl::rpc::v1::GetLedgerResponse>
|
||||
fetchLedger(uint32_t sequence, bool getObjects = true, bool getObjectNeighbors = false) = 0;
|
||||
|
||||
/**
|
||||
* @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 loader InitialLoadObserverInterface implementation
|
||||
* @return A std::pair of the data and a bool indicating whether the download was successful
|
||||
*/
|
||||
virtual InitialLedgerLoadResult
|
||||
loadInitialLedger(uint32_t sequence, std::uint32_t numMarkers, etlng::InitialLoadObserverInterface& loader) = 0;
|
||||
|
||||
/**
|
||||
* @brief Forward a request to rippled.
|
||||
*
|
||||
* @param request The request to forward
|
||||
* @param forwardToRippledClientIp IP of the client forwarding this request if known
|
||||
* @param xUserValue Value of the X-User header
|
||||
* @param yield The coroutine context
|
||||
* @return Response on success or error on failure
|
||||
*/
|
||||
[[nodiscard]] virtual std::expected<boost::json::object, rpc::ClioError>
|
||||
forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& forwardToRippledClientIp,
|
||||
std::string_view xUserValue,
|
||||
boost::asio::yield_context yield
|
||||
) const = 0;
|
||||
};
|
||||
|
||||
using SourcePtr = std::unique_ptr<SourceBase>;
|
||||
|
||||
using SourceFactory = std::function<SourcePtr(
|
||||
util::config::ObjectView const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
SourceBase::OnConnectHook onConnect,
|
||||
SourceBase::OnDisconnectHook onDisconnect,
|
||||
SourceBase::OnLedgerClosedHook onLedgerClosed
|
||||
)>;
|
||||
|
||||
/**
|
||||
* @brief Create a source
|
||||
*
|
||||
* @param config The configuration to use
|
||||
* @param ioc The io_context to run on
|
||||
* @param subscriptions Subscription manager
|
||||
* @param validatedLedgers The network validated ledgers data structure
|
||||
* @param forwardingTimeout The timeout for forwarding to rippled
|
||||
* @param onConnect The hook to call on connect
|
||||
* @param onDisconnect The hook to call on disconnect
|
||||
* @param onLedgerClosed The hook to call on ledger closed. This is called when a ledger is closed and the source is set
|
||||
* as forwarding.
|
||||
* @return The created source
|
||||
*/
|
||||
[[nodiscard]] SourcePtr
|
||||
makeSource(
|
||||
util::config::ObjectView const& config,
|
||||
boost::asio::io_context& ioc,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
std::shared_ptr<etl::NetworkValidatedLedgersInterface> validatedLedgers,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
SourceBase::OnConnectHook onConnect,
|
||||
SourceBase::OnDisconnectHook onDisconnect,
|
||||
SourceBase::OnLedgerClosedHook onLedgerClosed
|
||||
);
|
||||
|
||||
} // namespace etlng
|
||||
@@ -1,71 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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 "etlng/impl/AmendmentBlockHandler.hpp"
|
||||
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <chrono>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
AmendmentBlockHandler::ActionType const AmendmentBlockHandler::kDEFAULT_AMENDMENT_BLOCK_ACTION = []() {
|
||||
static util::Logger const log{"ETL"}; // NOLINT(readability-identifier-naming)
|
||||
LOG(log.fatal()) << "Can't process new ledgers: The current ETL source is not compatible with the version of "
|
||||
<< "the libxrpl Clio is currently using. Please upgrade Clio to a newer version.";
|
||||
};
|
||||
|
||||
AmendmentBlockHandler::AmendmentBlockHandler(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
etl::SystemState& state,
|
||||
std::chrono::steady_clock::duration interval,
|
||||
ActionType action
|
||||
)
|
||||
: state_{std::ref(state)}, interval_{interval}, ctx_{std::move(ctx)}, action_{std::move(action)}
|
||||
{
|
||||
}
|
||||
|
||||
AmendmentBlockHandler::~AmendmentBlockHandler()
|
||||
{
|
||||
stop();
|
||||
}
|
||||
|
||||
void
|
||||
AmendmentBlockHandler::notifyAmendmentBlocked()
|
||||
{
|
||||
state_.get().isAmendmentBlocked = true;
|
||||
if (not operation_.has_value())
|
||||
operation_.emplace(ctx_.executeRepeatedly(interval_, action_));
|
||||
}
|
||||
|
||||
void
|
||||
AmendmentBlockHandler::stop()
|
||||
{
|
||||
if (operation_.has_value()) {
|
||||
operation_->abort();
|
||||
operation_.reset();
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -1,68 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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/SystemState.hpp"
|
||||
#include "etlng/AmendmentBlockHandlerInterface.hpp"
|
||||
#include "util/async/AnyExecutionContext.hpp"
|
||||
#include "util/async/AnyOperation.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/post.hpp>
|
||||
#include <boost/asio/steady_timer.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <functional>
|
||||
#include <optional>
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
class AmendmentBlockHandler : public AmendmentBlockHandlerInterface {
|
||||
public:
|
||||
using ActionType = std::function<void()>;
|
||||
|
||||
private:
|
||||
std::reference_wrapper<etl::SystemState> state_;
|
||||
std::chrono::steady_clock::duration interval_;
|
||||
util::async::AnyExecutionContext ctx_;
|
||||
std::optional<util::async::AnyOperation<void>> operation_;
|
||||
|
||||
ActionType action_;
|
||||
|
||||
public:
|
||||
static ActionType const kDEFAULT_AMENDMENT_BLOCK_ACTION;
|
||||
|
||||
AmendmentBlockHandler(
|
||||
util::async::AnyExecutionContext ctx,
|
||||
etl::SystemState& state,
|
||||
std::chrono::steady_clock::duration interval = std::chrono::seconds{1},
|
||||
ActionType action = kDEFAULT_AMENDMENT_BLOCK_ACTION
|
||||
);
|
||||
|
||||
~AmendmentBlockHandler() override;
|
||||
|
||||
void
|
||||
stop() override;
|
||||
|
||||
void
|
||||
notifyAmendmentBlocked() override;
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -1,116 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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 "etlng/impl/ForwardingSource.hpp"
|
||||
|
||||
#include "rpc/Errors.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/format.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <exception>
|
||||
#include <optional>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
ForwardingSource::ForwardingSource(
|
||||
std::string ip,
|
||||
std::string wsPort,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
std::chrono::steady_clock::duration connTimeout
|
||||
)
|
||||
: log_(fmt::format("ForwardingSource[{}:{}]", ip, wsPort))
|
||||
, connectionBuilder_(std::move(ip), std::move(wsPort))
|
||||
, forwardingTimeout_{forwardingTimeout}
|
||||
{
|
||||
connectionBuilder_.setConnectionTimeout(connTimeout)
|
||||
.addHeader(
|
||||
{boost::beast::http::field::user_agent, fmt::format("{} websocket-client-coro", BOOST_BEAST_VERSION_STRING)}
|
||||
);
|
||||
}
|
||||
|
||||
std::expected<boost::json::object, rpc::ClioError>
|
||||
ForwardingSource::forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& forwardToRippledClientIp,
|
||||
std::string_view xUserValue,
|
||||
boost::asio::yield_context yield
|
||||
) const
|
||||
{
|
||||
auto connectionBuilder = connectionBuilder_;
|
||||
if (forwardToRippledClientIp) {
|
||||
connectionBuilder.addHeader(
|
||||
{boost::beast::http::field::forwarded, fmt::format("for={}", *forwardToRippledClientIp)}
|
||||
);
|
||||
}
|
||||
|
||||
connectionBuilder.addHeader({"X-User", std::string{xUserValue}});
|
||||
|
||||
auto expectedConnection = connectionBuilder.connect(yield);
|
||||
if (not expectedConnection) {
|
||||
LOG(log_.debug()) << "Couldn't connect to rippled to forward request.";
|
||||
return std::unexpected{rpc::ClioError::EtlConnectionError};
|
||||
}
|
||||
auto& connection = expectedConnection.value();
|
||||
|
||||
auto writeError = connection->write(boost::json::serialize(request), yield, forwardingTimeout_);
|
||||
if (writeError) {
|
||||
LOG(log_.debug()) << "Error sending request to rippled to forward request.";
|
||||
return std::unexpected{rpc::ClioError::EtlRequestError};
|
||||
}
|
||||
|
||||
auto response = connection->read(yield, forwardingTimeout_);
|
||||
if (not response) {
|
||||
if (auto errorCode = response.error().errorCode();
|
||||
errorCode.has_value() and errorCode->value() == boost::system::errc::timed_out) {
|
||||
LOG(log_.debug()) << "Request to rippled timed out";
|
||||
return std::unexpected{rpc::ClioError::EtlRequestTimeout};
|
||||
}
|
||||
LOG(log_.debug()) << "Error sending request to rippled to forward request.";
|
||||
return std::unexpected{rpc::ClioError::EtlRequestError};
|
||||
}
|
||||
|
||||
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_.debug()) << "Error parsing response from rippled: " << e.what() << ". Response: " << *response;
|
||||
return std::unexpected{rpc::ClioError::EtlInvalidResponse};
|
||||
}
|
||||
|
||||
auto responseObject = parsedResponse.as_object();
|
||||
responseObject["forwarded"] = true;
|
||||
|
||||
return responseObject;
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -1,70 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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 "rpc/Errors.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/requests/WsConnection.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
|
||||
#include <chrono>
|
||||
#include <expected>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
class ForwardingSource {
|
||||
util::Logger log_;
|
||||
util::requests::WsConnectionBuilder connectionBuilder_;
|
||||
std::chrono::steady_clock::duration forwardingTimeout_;
|
||||
|
||||
static constexpr std::chrono::seconds kCONNECTION_TIMEOUT{3};
|
||||
|
||||
public:
|
||||
ForwardingSource(
|
||||
std::string ip,
|
||||
std::string wsPort,
|
||||
std::chrono::steady_clock::duration forwardingTimeout,
|
||||
std::chrono::steady_clock::duration connTimeout = ForwardingSource::kCONNECTION_TIMEOUT
|
||||
);
|
||||
|
||||
/**
|
||||
* @brief Forward a request to rippled.
|
||||
*
|
||||
* @param request The request to forward
|
||||
* @param forwardToRippledClientIp IP of the client forwarding this request if known
|
||||
* @param xUserValue Optional value for X-User header
|
||||
* @param yield The coroutine context
|
||||
* @return Response on success or error on failure
|
||||
*/
|
||||
std::expected<boost::json::object, rpc::ClioError>
|
||||
forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& forwardToRippledClientIp,
|
||||
std::string_view xUserValue,
|
||||
boost::asio::yield_context yield
|
||||
) const;
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -1,183 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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 "etlng/impl/GrpcSource.hpp"
|
||||
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/impl/AsyncGrpcCall.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "web/Resolver.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <fmt/format.h>
|
||||
#include <grpc/grpc.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 <atomic>
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <exception>
|
||||
#include <expected>
|
||||
#include <memory>
|
||||
#include <stdexcept>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace {
|
||||
|
||||
std::string
|
||||
resolve(std::string const& ip, std::string const& port)
|
||||
{
|
||||
web::Resolver resolver;
|
||||
|
||||
if (auto const results = resolver.resolve(ip, port); not results.empty())
|
||||
return results.at(0);
|
||||
|
||||
throw std::runtime_error("Failed to resolve " + ip + ":" + port);
|
||||
}
|
||||
|
||||
} // namespace
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
GrpcSource::GrpcSource(std::string const& ip, std::string const& grpcPort, std::chrono::system_clock::duration deadline)
|
||||
: log_(fmt::format("ETL_Grpc[{}:{}]", ip, grpcPort))
|
||||
, initialLoadShouldStop_(std::make_unique<std::atomic_bool>(false))
|
||||
, deadline_{deadline}
|
||||
{
|
||||
try {
|
||||
grpc::ChannelArguments chArgs;
|
||||
chArgs.SetMaxReceiveMessageSize(-1);
|
||||
chArgs.SetInt(GRPC_ARG_KEEPALIVE_TIME_MS, kKEEPALIVE_PING_INTERVAL_MS);
|
||||
chArgs.SetInt(GRPC_ARG_KEEPALIVE_TIMEOUT_MS, kKEEPALIVE_TIMEOUT_MS);
|
||||
chArgs.SetInt(GRPC_ARG_KEEPALIVE_PERMIT_WITHOUT_CALLS, static_cast<int>(kKEEPALIVE_PERMIT_WITHOUT_CALLS));
|
||||
chArgs.SetInt(GRPC_ARG_HTTP2_MAX_PINGS_WITHOUT_DATA, kMAX_PINGS_WITHOUT_DATA);
|
||||
|
||||
stub_ = org::xrpl::rpc::v1::XRPLedgerAPIService::NewStub(
|
||||
grpc::CreateCustomChannel(resolve(ip, grpcPort), 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};
|
||||
|
||||
org::xrpl::rpc::v1::GetLedgerRequest request;
|
||||
grpc::ClientContext context;
|
||||
|
||||
context.set_deadline(std::chrono::system_clock::now() + deadline_); // Prevent indefinite blocking
|
||||
|
||||
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() and not 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)};
|
||||
}
|
||||
|
||||
InitialLedgerLoadResult
|
||||
GrpcSource::loadInitialLedger(
|
||||
uint32_t const sequence,
|
||||
uint32_t const numMarkers,
|
||||
etlng::InitialLoadObserverInterface& observer
|
||||
)
|
||||
{
|
||||
if (*initialLoadShouldStop_)
|
||||
return std::unexpected{InitialLedgerLoadError::Cancelled};
|
||||
|
||||
if (!stub_)
|
||||
return std::unexpected{InitialLedgerLoadError::Errored};
|
||||
|
||||
std::vector<AsyncGrpcCall> calls = AsyncGrpcCall::makeAsyncCalls(sequence, numMarkers);
|
||||
|
||||
LOG(log_.debug()) << "Starting data download for ledger " << sequence << ".";
|
||||
|
||||
grpc::CompletionQueue queue;
|
||||
for (auto& call : calls)
|
||||
call.call(stub_, queue);
|
||||
|
||||
std::vector<std::string> edgeKeys;
|
||||
void* tag = nullptr;
|
||||
bool ok = false;
|
||||
bool abort = false;
|
||||
size_t numFinished = 0;
|
||||
|
||||
while (numFinished < calls.size() && queue.Next(&tag, &ok)) {
|
||||
ASSERT(tag != nullptr, "Tag can't be null.");
|
||||
auto ptr = static_cast<AsyncGrpcCall*>(tag);
|
||||
|
||||
if (not ok or *initialLoadShouldStop_) {
|
||||
LOG(log_.error()) << "loadInitialLedger cancelled";
|
||||
return std::unexpected{InitialLedgerLoadError::Cancelled};
|
||||
}
|
||||
|
||||
LOG(log_.trace()) << "Marker prefix = " << ptr->getMarkerPrefix();
|
||||
|
||||
auto result = ptr->process(stub_, queue, observer, abort);
|
||||
if (result != AsyncGrpcCall::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 == AsyncGrpcCall::CallStatus::Errored)
|
||||
abort = true;
|
||||
}
|
||||
|
||||
if (abort)
|
||||
return std::unexpected{InitialLedgerLoadError::Errored};
|
||||
|
||||
return edgeKeys;
|
||||
}
|
||||
|
||||
void
|
||||
GrpcSource::stop(boost::asio::yield_context)
|
||||
{
|
||||
initialLoadShouldStop_->store(true);
|
||||
}
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -1,93 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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 "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
#include <xrpl/proto/org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
#include <utility>
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
class GrpcSource {
|
||||
util::Logger log_;
|
||||
std::unique_ptr<org::xrpl::rpc::v1::XRPLedgerAPIService::Stub> stub_;
|
||||
std::unique_ptr<std::atomic_bool> initialLoadShouldStop_;
|
||||
std::chrono::system_clock::duration deadline_;
|
||||
|
||||
static constexpr auto kKEEPALIVE_PING_INTERVAL_MS = 10000;
|
||||
static constexpr auto kKEEPALIVE_TIMEOUT_MS = 5000;
|
||||
static constexpr auto kKEEPALIVE_PERMIT_WITHOUT_CALLS = true; // Allow keepalive pings when no calls
|
||||
static constexpr auto kMAX_PINGS_WITHOUT_DATA = 0; // No limit
|
||||
static constexpr auto kDEADLINE = std::chrono::seconds(30);
|
||||
|
||||
public:
|
||||
GrpcSource(
|
||||
std::string const& ip,
|
||||
std::string const& grpcPort,
|
||||
std::chrono::system_clock::duration deadline = kDEADLINE
|
||||
);
|
||||
|
||||
/**
|
||||
* @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 observer InitialLoadObserverInterface implementation
|
||||
* @return Downloaded data or an indication of error or cancellation
|
||||
*/
|
||||
InitialLedgerLoadResult
|
||||
loadInitialLedger(uint32_t sequence, uint32_t numMarkers, etlng::InitialLoadObserverInterface& observer);
|
||||
|
||||
/**
|
||||
* @brief Stop any ongoing operations
|
||||
* @note This is used to cancel any ongoing initial ledger downloads
|
||||
* @param yield The coroutine context
|
||||
*/
|
||||
void
|
||||
stop(boost::asio::yield_context yield);
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -1,283 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
This file is part of clio: https://github.com/XRPLF/clio
|
||||
Copyright (c) 2025, the clio developers.
|
||||
|
||||
Permission to use, copy, modify, and distribute this software for any
|
||||
purpose with or without fee is hereby granted, provided that the above
|
||||
copyright notice and this permission notice appear in all copies.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
*/
|
||||
//==============================================================================
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "etl/SystemState.hpp"
|
||||
#include "etlng/LedgerPublisherInterface.hpp"
|
||||
#include "etlng/impl/Loading.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "util/Assert.hpp"
|
||||
#include "util/Mutex.hpp"
|
||||
#include "util/log/Logger.hpp"
|
||||
#include "util/prometheus/Counter.hpp"
|
||||
#include "util/prometheus/Prometheus.hpp"
|
||||
|
||||
#include <boost/asio/io_context.hpp>
|
||||
#include <boost/asio/post.hpp>
|
||||
#include <boost/asio/strand.hpp>
|
||||
#include <fmt/format.h>
|
||||
#include <xrpl/basics/chrono.h>
|
||||
#include <xrpl/protocol/Fees.h>
|
||||
#include <xrpl/protocol/LedgerHeader.h>
|
||||
#include <xrpl/protocol/SField.h>
|
||||
#include <xrpl/protocol/STObject.h>
|
||||
#include <xrpl/protocol/Serializer.h>
|
||||
|
||||
#include <algorithm>
|
||||
#include <chrono>
|
||||
#include <cstddef>
|
||||
#include <cstdint>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <optional>
|
||||
#include <shared_mutex>
|
||||
#include <string>
|
||||
#include <thread>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
/**
|
||||
* @brief Publishes ledgers in a synchronized fashion.
|
||||
*
|
||||
* If ETL is started far behind the network, ledgers will be written and published very rapidly. Monitoring processes
|
||||
* will publish ledgers as they are written. However, to publish a ledger, the monitoring process needs to read all of
|
||||
* the transactions for that ledger from the database. Reading the transactions from the database requires network
|
||||
* calls, which can be slow. It is imperative however that the monitoring processes keep up with the writer, else the
|
||||
* monitoring processes will not be able to detect if the writer failed. Therefore, publishing each ledger (which
|
||||
* includes reading all of the transactions from the database) is done from the application wide asio io_service, and a
|
||||
* strand is used to ensure ledgers are published in order.
|
||||
*/
|
||||
class LedgerPublisher : public etlng::LedgerPublisherInterface {
|
||||
util::Logger log_{"ETL"};
|
||||
|
||||
boost::asio::strand<boost::asio::io_context::executor_type> publishStrand_;
|
||||
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions_;
|
||||
std::reference_wrapper<etl::SystemState const> state_; // shared state for ETL
|
||||
|
||||
util::Mutex<std::chrono::time_point<ripple::NetClock>, std::shared_mutex> lastCloseTime_;
|
||||
|
||||
std::reference_wrapper<util::prometheus::CounterInt> lastPublishSeconds_ = PrometheusService::counterInt(
|
||||
"etl_last_publish_seconds",
|
||||
{},
|
||||
"Seconds since epoch of the last published ledger"
|
||||
);
|
||||
|
||||
util::Mutex<std::optional<uint32_t>, std::shared_mutex> lastPublishedSequence_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Create an instance of the publisher
|
||||
*/
|
||||
LedgerPublisher(
|
||||
boost::asio::io_context& ioc, // TODO: replace with AsyncContext shared with ETLServiceNg
|
||||
std::shared_ptr<BackendInterface> backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions,
|
||||
etl::SystemState const& state
|
||||
)
|
||||
: publishStrand_{boost::asio::make_strand(ioc)}
|
||||
, backend_{std::move(backend)}
|
||||
, subscriptions_{std::move(subscriptions)}
|
||||
, state_{std::cref(state)}
|
||||
{
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Attempt to read the specified ledger from the database, and then publish that ledger to the ledgers
|
||||
* stream.
|
||||
*
|
||||
* @param ledgerSequence the sequence of the ledger to publish
|
||||
* @param maxAttempts the number of times to attempt to read the ledger from the database
|
||||
* @param attemptsDelay the delay between attempts to read the ledger from the database
|
||||
* @return Whether the ledger was found in the database and published
|
||||
*/
|
||||
bool
|
||||
publish(
|
||||
uint32_t ledgerSequence,
|
||||
std::optional<uint32_t> maxAttempts,
|
||||
std::chrono::steady_clock::duration attemptsDelay = std::chrono::seconds{1}
|
||||
) override
|
||||
{
|
||||
LOG(log_.info()) << "Attempting to publish ledger = " << ledgerSequence;
|
||||
size_t numAttempts = 0;
|
||||
while (not state_.get().isStopping) {
|
||||
auto range = backend_->hardFetchLedgerRangeNoThrow();
|
||||
|
||||
if (!range || range->maxSequence < ledgerSequence) {
|
||||
++numAttempts;
|
||||
LOG(log_.debug()) << "Trying to publish. Could not find ledger with sequence = " << ledgerSequence;
|
||||
|
||||
// We try maxAttempts times to publish the ledger, waiting one second in between each attempt.
|
||||
if (maxAttempts && numAttempts >= maxAttempts) {
|
||||
LOG(log_.debug()) << "Failed to publish ledger after " << numAttempts << " attempts.";
|
||||
return false;
|
||||
}
|
||||
std::this_thread::sleep_for(attemptsDelay);
|
||||
continue;
|
||||
}
|
||||
|
||||
auto lgr = data::synchronousAndRetryOnTimeout([&](auto yield) {
|
||||
return backend_->fetchLedgerBySequence(ledgerSequence, yield);
|
||||
});
|
||||
|
||||
ASSERT(lgr.has_value(), "Ledger must exist in database. Ledger sequence = {}", ledgerSequence);
|
||||
publish(*lgr);
|
||||
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Publish the passed ledger asynchronously.
|
||||
*
|
||||
* All ledgers are published thru publishStrand_ which ensures that all publishes are performed in a serial fashion.
|
||||
*
|
||||
* @param lgrInfo the ledger to publish
|
||||
*/
|
||||
void
|
||||
publish(ripple::LedgerHeader const& lgrInfo)
|
||||
{
|
||||
boost::asio::post(publishStrand_, [this, lgrInfo = lgrInfo]() {
|
||||
LOG(log_.info()) << "Publishing ledger " << std::to_string(lgrInfo.seq);
|
||||
|
||||
setLastClose(lgrInfo.closeTime);
|
||||
auto age = lastCloseAgeSeconds();
|
||||
|
||||
// if the ledger closed over MAX_LEDGER_AGE_SECONDS ago, assume we are still catching up and don't publish
|
||||
static constexpr std::uint32_t kMAX_LEDGER_AGE_SECONDS = 600;
|
||||
if (age < kMAX_LEDGER_AGE_SECONDS) {
|
||||
std::optional<ripple::Fees> fees = data::synchronousAndRetryOnTimeout([&](auto yield) {
|
||||
return backend_->fetchFees(lgrInfo.seq, yield);
|
||||
});
|
||||
ASSERT(fees.has_value(), "Fees must exist for ledger {}", lgrInfo.seq);
|
||||
|
||||
auto transactions = data::synchronousAndRetryOnTimeout([&](auto yield) {
|
||||
return backend_->fetchAllTransactionsInLedger(lgrInfo.seq, yield);
|
||||
});
|
||||
|
||||
auto const ledgerRange = backend_->fetchLedgerRange();
|
||||
ASSERT(ledgerRange.has_value(), "Ledger range must exist");
|
||||
|
||||
auto const range = fmt::format("{}-{}", ledgerRange->minSequence, ledgerRange->maxSequence);
|
||||
subscriptions_->pubLedger(lgrInfo, *fees, range, transactions.size());
|
||||
|
||||
// order with transaction index
|
||||
std::ranges::sort(transactions, [](auto const& t1, auto const& t2) {
|
||||
ripple::SerialIter iter1{t1.metadata.data(), t1.metadata.size()};
|
||||
ripple::STObject const object1(iter1, ripple::sfMetadata);
|
||||
ripple::SerialIter iter2{t2.metadata.data(), t2.metadata.size()};
|
||||
ripple::STObject const object2(iter2, ripple::sfMetadata);
|
||||
return object1.getFieldU32(ripple::sfTransactionIndex) <
|
||||
object2.getFieldU32(ripple::sfTransactionIndex);
|
||||
});
|
||||
|
||||
for (auto const& txAndMeta : transactions)
|
||||
subscriptions_->pubTransaction(txAndMeta, lgrInfo);
|
||||
|
||||
subscriptions_->pubBookChanges(lgrInfo, transactions);
|
||||
|
||||
setLastPublishTime();
|
||||
LOG(log_.info()) << "Published ledger " << lgrInfo.seq;
|
||||
} else {
|
||||
LOG(log_.info()) << "Skipping publishing ledger " << lgrInfo.seq;
|
||||
}
|
||||
});
|
||||
|
||||
// we track latest publish-requested seq, not necessarily already published
|
||||
setLastPublishedSequence(lgrInfo.seq);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get time passed since last publish, in seconds
|
||||
*/
|
||||
std::uint32_t
|
||||
lastPublishAgeSeconds() const override
|
||||
{
|
||||
return std::chrono::duration_cast<std::chrono::seconds>(std::chrono::system_clock::now() - getLastPublish())
|
||||
.count();
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get last publish time as a time point
|
||||
*/
|
||||
std::chrono::time_point<std::chrono::system_clock>
|
||||
getLastPublish() const override
|
||||
{
|
||||
return std::chrono::time_point<std::chrono::system_clock>{
|
||||
std::chrono::seconds{lastPublishSeconds_.get().value()}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get time passed since last ledger close, in seconds
|
||||
*/
|
||||
std::uint32_t
|
||||
lastCloseAgeSeconds() const override
|
||||
{
|
||||
auto closeTime = lastCloseTime_.lock()->time_since_epoch().count();
|
||||
auto now = std::chrono::duration_cast<std::chrono::seconds>(std::chrono::system_clock::now().time_since_epoch())
|
||||
.count();
|
||||
if (now < (kRIPPLE_EPOCH_START + closeTime))
|
||||
return 0;
|
||||
return now - (kRIPPLE_EPOCH_START + closeTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Get the sequence of the last schueduled ledger to publish, Be aware that the ledger may not have been
|
||||
* published to network
|
||||
*/
|
||||
std::optional<uint32_t>
|
||||
getLastPublishedSequence() const
|
||||
{
|
||||
return *lastPublishedSequence_.lock();
|
||||
}
|
||||
|
||||
private:
|
||||
void
|
||||
setLastClose(std::chrono::time_point<ripple::NetClock> lastCloseTime)
|
||||
{
|
||||
auto closeTime = lastCloseTime_.lock<std::scoped_lock>();
|
||||
*closeTime = lastCloseTime;
|
||||
}
|
||||
|
||||
void
|
||||
setLastPublishTime()
|
||||
{
|
||||
using namespace std::chrono;
|
||||
auto const nowSeconds = duration_cast<seconds>(system_clock::now().time_since_epoch()).count();
|
||||
lastPublishSeconds_.get().set(nowSeconds);
|
||||
}
|
||||
|
||||
void
|
||||
setLastPublishedSequence(std::optional<uint32_t> lastPublishedSequence)
|
||||
{
|
||||
auto lastPublishSeq = lastPublishedSequence_.lock();
|
||||
*lastPublishSeq = lastPublishedSequence;
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -1,234 +0,0 @@
|
||||
//------------------------------------------------------------------------------
|
||||
/*
|
||||
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/SubscriptionSource.hpp"
|
||||
#include "etlng/InitialLoadObserverInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etlng/Source.hpp"
|
||||
#include "etlng/impl/ForwardingSource.hpp"
|
||||
#include "etlng/impl/GrpcSource.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
|
||||
#include <boost/asio/spawn.hpp>
|
||||
#include <boost/json/object.hpp>
|
||||
#include <grpcpp/support/status.h>
|
||||
#include <org/xrpl/rpc/v1/get_ledger.pb.h>
|
||||
|
||||
#include <chrono>
|
||||
#include <cstdint>
|
||||
#include <expected>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <utility>
|
||||
#include <vector>
|
||||
|
||||
namespace etlng::impl {
|
||||
|
||||
/**
|
||||
* @brief Provides an implementation of a ETL source
|
||||
*
|
||||
* @tparam GrpcSourceType The type of the gRPC source
|
||||
* @tparam SubscriptionSourceTypePtr The type of the subscription source
|
||||
* @tparam ForwardingSourceType The type of the forwarding source
|
||||
*/
|
||||
template <
|
||||
typename GrpcSourceType = GrpcSource,
|
||||
typename SubscriptionSourceTypePtr = std::unique_ptr<etl::impl::SubscriptionSource>,
|
||||
typename ForwardingSourceType = etlng::impl::ForwardingSource>
|
||||
class SourceImpl : public SourceBase {
|
||||
std::string ip_;
|
||||
std::string wsPort_;
|
||||
std::string grpcPort_;
|
||||
|
||||
GrpcSourceType grpcSource_;
|
||||
SubscriptionSourceTypePtr subscriptionSource_;
|
||||
ForwardingSourceType forwardingSource_;
|
||||
|
||||
public:
|
||||
/**
|
||||
* @brief Construct a new SourceImpl object
|
||||
*
|
||||
* @param ip The IP of the source
|
||||
* @param wsPort The web socket port of the source
|
||||
* @param grpcPort The gRPC port of the source
|
||||
* @param grpcSource The gRPC source
|
||||
* @param subscriptionSource The subscription source
|
||||
* @param forwardingSource The forwarding source
|
||||
*/
|
||||
template <typename SomeGrpcSourceType, typename SomeForwardingSourceType>
|
||||
requires std::is_same_v<GrpcSourceType, SomeGrpcSourceType> and
|
||||
std::is_same_v<ForwardingSourceType, SomeForwardingSourceType>
|
||||
SourceImpl(
|
||||
std::string ip,
|
||||
std::string wsPort,
|
||||
std::string grpcPort,
|
||||
SomeGrpcSourceType&& grpcSource,
|
||||
SubscriptionSourceTypePtr subscriptionSource,
|
||||
SomeForwardingSourceType&& forwardingSource
|
||||
)
|
||||
: ip_(std::move(ip))
|
||||
, wsPort_(std::move(wsPort))
|
||||
, grpcPort_(std::move(grpcPort))
|
||||
, grpcSource_(std::forward<SomeGrpcSourceType>(grpcSource))
|
||||
, subscriptionSource_(std::move(subscriptionSource))
|
||||
, forwardingSource_(std::forward<SomeForwardingSourceType>(forwardingSource))
|
||||
{
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Run subscriptions loop of the source
|
||||
*/
|
||||
void
|
||||
run() final
|
||||
{
|
||||
subscriptionSource_->run();
|
||||
}
|
||||
|
||||
void
|
||||
stop(boost::asio::yield_context yield) final
|
||||
{
|
||||
subscriptionSource_->stop(yield);
|
||||
grpcSource_.stop(yield);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Check if source is connected
|
||||
*
|
||||
* @return true if source is connected; false otherwise
|
||||
*/
|
||||
bool
|
||||
isConnected() const final
|
||||
{
|
||||
return subscriptionSource_->isConnected();
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Set the forwarding state of the source.
|
||||
*
|
||||
* @param isForwarding Whether to forward or not
|
||||
*/
|
||||
void
|
||||
setForwarding(bool isForwarding) final
|
||||
{
|
||||
subscriptionSource_->setForwarding(isForwarding);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Represent the source as a JSON object
|
||||
*
|
||||
* @return JSON representation of the source
|
||||
*/
|
||||
boost::json::object
|
||||
toJson() const final
|
||||
{
|
||||
boost::json::object res;
|
||||
|
||||
res["validated_range"] = subscriptionSource_->validatedRange();
|
||||
res["is_connected"] = std::to_string(static_cast<int>(subscriptionSource_->isConnected()));
|
||||
res["ip"] = ip_;
|
||||
res["ws_port"] = wsPort_;
|
||||
res["grpc_port"] = grpcPort_;
|
||||
|
||||
auto last = subscriptionSource_->lastMessageTime();
|
||||
if (last.time_since_epoch().count() != 0) {
|
||||
res["last_msg_age_seconds"] = std::to_string(
|
||||
std::chrono::duration_cast<std::chrono::seconds>(std::chrono::steady_clock::now() - last).count()
|
||||
);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
/** @return String representation of the source (for debug) */
|
||||
std::string
|
||||
toString() const final
|
||||
{
|
||||
return "{validated range: " + subscriptionSource_->validatedRange() + ", ip: " + ip_ +
|
||||
", web socket port: " + wsPort_ + ", grpc port: " + grpcPort_ + "}";
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Check if ledger is known by this source.
|
||||
*
|
||||
* @param sequence The ledger sequence to check
|
||||
* @return true if ledger is in the range of this source; false otherwise
|
||||
*/
|
||||
bool
|
||||
hasLedger(uint32_t sequence) const final
|
||||
{
|
||||
return subscriptionSource_->hasLedger(sequence);
|
||||
}
|
||||
|
||||
/**
|
||||
* @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) final
|
||||
{
|
||||
return grpcSource_.fetchLedger(sequence, getObjects, getObjectNeighbors);
|
||||
}
|
||||
|
||||
/**
|
||||
* @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 loader InitialLoadObserverInterface implementation
|
||||
* @return A std::pair of the data and a bool indicating whether the download was successful
|
||||
*/
|
||||
InitialLedgerLoadResult
|
||||
loadInitialLedger(uint32_t sequence, std::uint32_t numMarkers, etlng::InitialLoadObserverInterface& loader) final
|
||||
{
|
||||
return grpcSource_.loadInitialLedger(sequence, numMarkers, loader);
|
||||
}
|
||||
|
||||
/**
|
||||
* @brief Forward a request to rippled.
|
||||
*
|
||||
* @param request The request to forward
|
||||
* @param forwardToRippledClientIp IP of the client forwarding this request if known
|
||||
* @param xUserValue Optional value of the X-User header
|
||||
* @param yield The coroutine context
|
||||
* @return Response or ClioError
|
||||
*/
|
||||
std::expected<boost::json::object, rpc::ClioError>
|
||||
forwardToRippled(
|
||||
boost::json::object const& request,
|
||||
std::optional<std::string> const& forwardToRippledClientIp,
|
||||
std::string_view xUserValue,
|
||||
boost::asio::yield_context yield
|
||||
) const final
|
||||
{
|
||||
return forwardingSource_.forwardToRippled(request, forwardToRippledClientIp, xUserValue, yield);
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace etlng::impl
|
||||
@@ -20,7 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "rpc/RPCHelpers.hpp"
|
||||
#include "rpc/WorkQueue.hpp"
|
||||
@@ -88,7 +88,7 @@ public:
|
||||
RPCEngine(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<BackendInterface> const& backend,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etl::LoadBalancerInterface> const& balancer,
|
||||
web::dosguard::DOSGuardInterface const& dosGuard,
|
||||
WorkQueue& workQueue,
|
||||
CountersType& counters,
|
||||
@@ -130,7 +130,7 @@ public:
|
||||
makeRPCEngine(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<BackendInterface> const& backend,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etl::LoadBalancerInterface> const& balancer,
|
||||
web::dosguard::DOSGuardInterface const& dosGuard,
|
||||
WorkQueue& workQueue,
|
||||
CountersType& counters,
|
||||
|
||||
@@ -19,7 +19,7 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "rpc/RPCCenter.hpp"
|
||||
#include "rpc/RPCHelpers.hpp"
|
||||
@@ -41,13 +41,13 @@ template <typename CountersType, typename HandlerProviderType>
|
||||
class ForwardingProxy {
|
||||
util::Logger log_{"RPC"};
|
||||
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> balancer_;
|
||||
std::shared_ptr<etl::LoadBalancerInterface> balancer_;
|
||||
std::reference_wrapper<CountersType> counters_;
|
||||
std::shared_ptr<HandlerProviderType const> handlerProvider_;
|
||||
|
||||
public:
|
||||
ForwardingProxy(
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etl::LoadBalancerInterface> const& balancer,
|
||||
CountersType& counters,
|
||||
std::shared_ptr<HandlerProviderType const> const& handlerProvider
|
||||
)
|
||||
|
||||
@@ -21,8 +21,8 @@
|
||||
|
||||
#include "data/AmendmentCenterInterface.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Counters.hpp"
|
||||
#include "rpc/common/AnyHandler.hpp"
|
||||
@@ -77,8 +77,8 @@ ProductionHandlerProvider::ProductionHandlerProvider(
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<BackendInterface> const& backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> const& subscriptionManager,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const& etl,
|
||||
std::shared_ptr<etl::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const& etl,
|
||||
std::shared_ptr<data::AmendmentCenterInterface const> const& amendmentCenter,
|
||||
Counters const& counters
|
||||
)
|
||||
|
||||
@@ -21,8 +21,8 @@
|
||||
|
||||
#include "data/AmendmentCenterInterface.hpp"
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/common/AnyHandler.hpp"
|
||||
#include "rpc/common/HandlerProvider.hpp"
|
||||
@@ -54,8 +54,8 @@ public:
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<BackendInterface> const& backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> const& subscriptionManager,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const& etl,
|
||||
std::shared_ptr<etl::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const& etl,
|
||||
std::shared_ptr<data::AmendmentCenterInterface const> const& amendmentCenter,
|
||||
Counters const& counters
|
||||
);
|
||||
|
||||
@@ -20,7 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "rpc/JS.hpp"
|
||||
#include "rpc/common/JsonBool.hpp"
|
||||
@@ -56,7 +56,7 @@ namespace rpc {
|
||||
class AccountTxHandler {
|
||||
util::Logger log_{"RPC"};
|
||||
std::shared_ptr<BackendInterface> sharedPtrBackend_;
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> etl_;
|
||||
std::shared_ptr<etl::ETLServiceInterface const> etl_;
|
||||
|
||||
public:
|
||||
static constexpr auto kLIMIT_MIN = 1;
|
||||
@@ -115,7 +115,7 @@ public:
|
||||
*/
|
||||
AccountTxHandler(
|
||||
std::shared_ptr<BackendInterface> const& sharedPtrBackend,
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const& etl
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const& etl
|
||||
)
|
||||
: sharedPtrBackend_(sharedPtrBackend), etl_{etl}
|
||||
{
|
||||
|
||||
@@ -21,8 +21,8 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etlng/LoadBalancerInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "etl/LoadBalancerInterface.hpp"
|
||||
#include "feed/SubscriptionManagerInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "rpc/JS.hpp"
|
||||
@@ -68,8 +68,8 @@ class BaseServerInfoHandler {
|
||||
|
||||
std::shared_ptr<BackendInterface> backend_;
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> subscriptions_;
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> balancer_;
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> etl_;
|
||||
std::shared_ptr<etl::LoadBalancerInterface> balancer_;
|
||||
std::shared_ptr<etl::ETLServiceInterface const> etl_;
|
||||
std::reference_wrapper<CountersType const> counters_;
|
||||
|
||||
public:
|
||||
@@ -154,8 +154,8 @@ public:
|
||||
BaseServerInfoHandler(
|
||||
std::shared_ptr<BackendInterface> const& backend,
|
||||
std::shared_ptr<feed::SubscriptionManagerInterface> const& subscriptions,
|
||||
std::shared_ptr<etlng::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const& etl,
|
||||
std::shared_ptr<etl::LoadBalancerInterface> const& balancer,
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const& etl,
|
||||
CountersType const& counters
|
||||
)
|
||||
: backend_(backend)
|
||||
|
||||
@@ -21,7 +21,7 @@
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "data/Types.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "rpc/JS.hpp"
|
||||
#include "rpc/RPCHelpers.hpp"
|
||||
@@ -60,7 +60,7 @@ namespace rpc {
|
||||
*/
|
||||
class TxHandler {
|
||||
std::shared_ptr<BackendInterface> sharedPtrBackend_;
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> etl_;
|
||||
std::shared_ptr<etl::ETLServiceInterface const> etl_;
|
||||
|
||||
public:
|
||||
/**
|
||||
@@ -103,7 +103,7 @@ public:
|
||||
*/
|
||||
TxHandler(
|
||||
std::shared_ptr<BackendInterface> const& sharedPtrBackend,
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const& etl
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const& etl
|
||||
)
|
||||
: sharedPtrBackend_(sharedPtrBackend), etl_(etl)
|
||||
{
|
||||
|
||||
@@ -289,7 +289,6 @@ getClioConfig()
|
||||
ConfigValue{ConfigType::String}.defaultValue("cassandra").withConstraint(gValidateProvider)},
|
||||
|
||||
{"allow_no_etl", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
|
||||
{"__ng_etl", ConfigValue{ConfigType::Boolean}.defaultValue(false)},
|
||||
{"etl_sources.[].ip", Array{ConfigValue{ConfigType::String}.optional().withConstraint(gValidateIp)}},
|
||||
{"etl_sources.[].ws_port", Array{ConfigValue{ConfigType::String}.optional().withConstraint(gValidatePort)}},
|
||||
{"etl_sources.[].grpc_port", Array{ConfigValue{ConfigType::String}.optional().withConstraint(gValidatePort)}},
|
||||
|
||||
@@ -20,7 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "rpc/Factories.hpp"
|
||||
#include "rpc/JS.hpp"
|
||||
@@ -64,7 +64,7 @@ template <typename RPCEngineType>
|
||||
class RPCServerHandler {
|
||||
std::shared_ptr<BackendInterface const> const backend_;
|
||||
std::shared_ptr<RPCEngineType> const rpcEngine_;
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const etl_;
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const etl_;
|
||||
util::TagDecoratorFactory const tagFactory_;
|
||||
rpc::impl::ProductionAPIVersionParser apiVersionParser_; // can be injected if needed
|
||||
std::reference_wrapper<web::dosguard::DOSGuardInterface> dosguard_;
|
||||
@@ -86,7 +86,7 @@ public:
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<BackendInterface const> const& backend,
|
||||
std::shared_ptr<RPCEngineType> const& rpcEngine,
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const& etl,
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const& etl,
|
||||
web::dosguard::DOSGuardInterface& dosguard
|
||||
)
|
||||
: backend_(backend)
|
||||
|
||||
@@ -20,7 +20,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "data/BackendInterface.hpp"
|
||||
#include "etlng/ETLServiceInterface.hpp"
|
||||
#include "etl/ETLServiceInterface.hpp"
|
||||
#include "rpc/Errors.hpp"
|
||||
#include "rpc/Factories.hpp"
|
||||
#include "rpc/JS.hpp"
|
||||
@@ -70,7 +70,7 @@ template <typename RPCEngineType>
|
||||
class RPCServerHandler {
|
||||
std::shared_ptr<BackendInterface const> const backend_;
|
||||
std::shared_ptr<RPCEngineType> const rpcEngine_;
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const etl_;
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const etl_;
|
||||
std::reference_wrapper<dosguard::DOSGuardInterface> dosguard_;
|
||||
util::TagDecoratorFactory const tagFactory_;
|
||||
rpc::impl::ProductionAPIVersionParser apiVersionParser_; // can be injected if needed
|
||||
@@ -92,7 +92,7 @@ public:
|
||||
util::config::ClioConfigDefinition const& config,
|
||||
std::shared_ptr<BackendInterface const> const& backend,
|
||||
std::shared_ptr<RPCEngineType> const& rpcEngine,
|
||||
std::shared_ptr<etlng::ETLServiceInterface const> const& etl,
|
||||
std::shared_ptr<etl::ETLServiceInterface const> const& etl,
|
||||
dosguard::DOSGuardInterface& dosguard
|
||||
)
|
||||
: backend_(backend)
|
||||
|
||||
@@ -20,8 +20,8 @@
|
||||
#include "util/BinaryTestObject.hpp"
|
||||
|
||||
#include "data/DBHelpers.hpp"
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/impl/Extraction.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/impl/Extraction.hpp"
|
||||
#include "util/StringUtils.hpp"
|
||||
#include "util/TestObject.hpp"
|
||||
|
||||
@@ -74,7 +74,7 @@ createTxAndMeta(std::string hashStr, std::string metaStr, std::string txnStr)
|
||||
return {ripple::STTx{it}, ripple::TxMeta{hash, kSEQ, metaBlob}};
|
||||
}
|
||||
|
||||
etlng::model::Transaction
|
||||
etl::model::Transaction
|
||||
createTransaction(ripple::TxType type, std::string hashStr, std::string metaStr, std::string txnStr)
|
||||
{
|
||||
auto const [sttx, meta] = createTxAndMeta(hashStr, metaStr, txnStr);
|
||||
@@ -89,8 +89,8 @@ createTransaction(ripple::TxType type, std::string hashStr, std::string metaStr,
|
||||
};
|
||||
}
|
||||
|
||||
etlng::model::Object
|
||||
createObject(etlng::model::Object::ModType modType, std::string key)
|
||||
etl::model::Object
|
||||
createObject(etl::model::Object::ModType modType, std::string key)
|
||||
{
|
||||
// random object taken from initial ledger load
|
||||
static constinit auto const kOBJ_PRED = "B00AA769C00726371689ED66A7CF57C2502F1BF4BDFF2ACADF67A2A7B5E8960A";
|
||||
@@ -104,16 +104,16 @@ createObject(etlng::model::Object::ModType modType, std::string key)
|
||||
return {
|
||||
.key = binaryStringToUint256(hexStringToBinaryString(key)),
|
||||
.keyRaw = hexStringToBinaryString(key),
|
||||
.data = modType == etlng::model::Object::ModType::Deleted ? ripple::Blob{} : *ripple::strUnHex(kOBJ_BLOB),
|
||||
.dataRaw = modType == etlng::model::Object::ModType::Deleted ? "" : hexStringToBinaryString(kOBJ_BLOB),
|
||||
.data = modType == etl::model::Object::ModType::Deleted ? ripple::Blob{} : *ripple::strUnHex(kOBJ_BLOB),
|
||||
.dataRaw = modType == etl::model::Object::ModType::Deleted ? "" : hexStringToBinaryString(kOBJ_BLOB),
|
||||
.successor = hexStringToBinaryString(kOBJ_SUCC),
|
||||
.predecessor = hexStringToBinaryString(kOBJ_PRED),
|
||||
.type = modType,
|
||||
};
|
||||
}
|
||||
|
||||
etlng::model::Object
|
||||
createObjectWithBookBase(etlng::model::Object::ModType modType, std::string key)
|
||||
etl::model::Object
|
||||
createObjectWithBookBase(etl::model::Object::ModType modType, std::string key)
|
||||
{
|
||||
// random object taken from initial ledger load
|
||||
static constinit auto const kOBJ_PRED = "B00AA769C00726371689ED66A7CF57C2502F1BF4BDFF2ACADF67A2A7B5E8960A";
|
||||
@@ -126,15 +126,15 @@ createObjectWithBookBase(etlng::model::Object::ModType modType, std::string key)
|
||||
return {
|
||||
.key = binaryStringToUint256(hexStringToBinaryString(key)),
|
||||
.keyRaw = hexStringToBinaryString(key),
|
||||
.data = modType == etlng::model::Object::ModType::Deleted ? ripple::Blob{} : *ripple::strUnHex(kOBJ_BLOB),
|
||||
.dataRaw = modType == etlng::model::Object::ModType::Deleted ? "" : hexStringToBinaryString(kOBJ_BLOB),
|
||||
.data = modType == etl::model::Object::ModType::Deleted ? ripple::Blob{} : *ripple::strUnHex(kOBJ_BLOB),
|
||||
.dataRaw = modType == etl::model::Object::ModType::Deleted ? "" : hexStringToBinaryString(kOBJ_BLOB),
|
||||
.successor = hexStringToBinaryString(kOBJ_SUCC),
|
||||
.predecessor = hexStringToBinaryString(kOBJ_PRED),
|
||||
.type = modType,
|
||||
};
|
||||
}
|
||||
|
||||
etlng::model::Object
|
||||
etl::model::Object
|
||||
createObjectWithTwoNFTs()
|
||||
{
|
||||
std::string const url1 = "abcd1";
|
||||
@@ -163,11 +163,11 @@ createObjectWithTwoNFTs()
|
||||
std::string(static_cast<char const*>(serializerNftPage.getDataPtr()), serializerNftPage.getDataLength()),
|
||||
.successor = "",
|
||||
.predecessor = "",
|
||||
.type = etlng::model::Object::ModType::Created,
|
||||
.type = etl::model::Object::ModType::Created,
|
||||
};
|
||||
}
|
||||
|
||||
etlng::model::Object
|
||||
etl::model::Object
|
||||
createObjectWithMPT()
|
||||
{
|
||||
constexpr auto kACCOUNT = "rM2AGCCCRb373FRuD8wHyUwUsh2dV4BW5Q";
|
||||
@@ -191,11 +191,11 @@ createObjectWithMPT()
|
||||
),
|
||||
.successor = "",
|
||||
.predecessor = "",
|
||||
.type = etlng::model::Object::ModType::Created,
|
||||
.type = etl::model::Object::ModType::Created,
|
||||
};
|
||||
}
|
||||
|
||||
etlng::model::BookSuccessor
|
||||
etl::model::BookSuccessor
|
||||
createSuccessor()
|
||||
{
|
||||
return {
|
||||
@@ -206,12 +206,12 @@ createSuccessor()
|
||||
};
|
||||
}
|
||||
|
||||
etlng::impl::PBLedgerResponseType
|
||||
etl::impl::PBLedgerResponseType
|
||||
createDataAndDiff()
|
||||
{
|
||||
auto const rawHeaderBlob = hexStringToBinaryString(kRAW_HEADER);
|
||||
|
||||
auto res = etlng::impl::PBLedgerResponseType();
|
||||
auto res = etl::impl::PBLedgerResponseType();
|
||||
res.set_ledger_header(rawHeaderBlob);
|
||||
res.set_objects_included(true);
|
||||
res.set_object_neighbors_included(true);
|
||||
@@ -252,12 +252,12 @@ createDataAndDiff()
|
||||
return res;
|
||||
}
|
||||
|
||||
etlng::impl::PBLedgerResponseType
|
||||
etl::impl::PBLedgerResponseType
|
||||
createData()
|
||||
{
|
||||
auto const rawHeaderBlob = hexStringToBinaryString(kRAW_HEADER);
|
||||
|
||||
auto res = etlng::impl::PBLedgerResponseType();
|
||||
auto res = etl::impl::PBLedgerResponseType();
|
||||
res.set_ledger_header(rawHeaderBlob);
|
||||
res.set_objects_included(false);
|
||||
res.set_object_neighbors_included(false);
|
||||
|
||||
@@ -19,8 +19,8 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/Models.hpp"
|
||||
#include "etlng/impl/Extraction.hpp"
|
||||
#include "etl/Models.hpp"
|
||||
#include "etl/impl/Extraction.hpp"
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include <xrpl/protocol/STTx.h>
|
||||
@@ -153,7 +153,7 @@ createTxAndMeta(
|
||||
std::string txnStr = kDEFAULT_TXN_HEX
|
||||
);
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::model::Transaction
|
||||
[[maybe_unused, nodiscard]] etl::model::Transaction
|
||||
createTransaction(
|
||||
ripple::TxType type,
|
||||
std::string hashStr = kDEFAULT_HASH,
|
||||
@@ -161,31 +161,31 @@ createTransaction(
|
||||
std::string txnStr = kDEFAULT_TXN_HEX
|
||||
);
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::model::Object
|
||||
[[maybe_unused, nodiscard]] etl::model::Object
|
||||
createObject(
|
||||
etlng::model::Object::ModType modType = etlng::model::Object::ModType::Created,
|
||||
etl::model::Object::ModType modType = etl::model::Object::ModType::Created,
|
||||
std::string key = kDEFAULT_OBJ_KEY
|
||||
);
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::model::Object
|
||||
[[maybe_unused, nodiscard]] etl::model::Object
|
||||
createObjectWithBookBase(
|
||||
etlng::model::Object::ModType modType = etlng::model::Object::ModType::Created,
|
||||
etl::model::Object::ModType modType = etl::model::Object::ModType::Created,
|
||||
std::string key = kDEFAULT_OBJ_KEY
|
||||
);
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::model::Object
|
||||
[[maybe_unused, nodiscard]] etl::model::Object
|
||||
createObjectWithTwoNFTs();
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::model::Object
|
||||
[[maybe_unused, nodiscard]] etl::model::Object
|
||||
createObjectWithMPT();
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::model::BookSuccessor
|
||||
[[maybe_unused, nodiscard]] etl::model::BookSuccessor
|
||||
createSuccessor();
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::impl::PBLedgerResponseType
|
||||
[[maybe_unused, nodiscard]] etl::impl::PBLedgerResponseType
|
||||
createDataAndDiff();
|
||||
|
||||
[[maybe_unused, nodiscard]] etlng::impl::PBLedgerResponseType
|
||||
[[maybe_unused, nodiscard]] etl::impl::PBLedgerResponseType
|
||||
createData();
|
||||
|
||||
} // namespace util
|
||||
|
||||
@@ -19,11 +19,11 @@
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "etlng/AmendmentBlockHandlerInterface.hpp"
|
||||
#include "etl/AmendmentBlockHandlerInterface.hpp"
|
||||
|
||||
#include <gmock/gmock.h>
|
||||
|
||||
struct MockAmendmentBlockHandler : etlng::AmendmentBlockHandlerInterface {
|
||||
struct MockAmendmentBlockHandler : etl::AmendmentBlockHandlerInterface {
|
||||
MOCK_METHOD(void, notifyAmendmentBlocked, (), (override));
|
||||
MOCK_METHOD(void, stop, (), (override));
|
||||
};
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user