diff --git a/CMakeLists.txt b/CMakeLists.txt index a678e67c..d0ef3318 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -41,7 +41,7 @@ target_sources(clio PRIVATE src/main/impl/Build.cpp ## Backend src/backend/BackendInterface.cpp - src/backend/SimpleCache.cpp + src/backend/LedgerCache.cpp ## NextGen Backend src/backend/cassandra/impl/Future.cpp src/backend/cassandra/impl/Cluster.cpp @@ -52,10 +52,12 @@ target_sources(clio PRIVATE src/backend/cassandra/Handle.cpp src/backend/cassandra/SettingsProvider.cpp ## ETL - src/etl/ETLSource.cpp - src/etl/ProbingETLSource.cpp + src/etl/Source.cpp + src/etl/ProbingSource.cpp src/etl/NFTHelpers.cpp - src/etl/ReportingETL.cpp + src/etl/ETLService.cpp + src/etl/LoadBalancer.cpp + src/etl/impl/ForwardCache.cpp ## Subscriptions src/subscriptions/SubscriptionManager.cpp ## RPC @@ -112,6 +114,8 @@ if(BUILD_TESTS) unittests/SubscriptionTest.cpp unittests/SubscriptionManagerTest.cpp unittests/util/TestObject.cpp + # ETL + unittests/etl/ExtractionDataPipeTest.cpp # RPC unittests/rpc/ErrorTests.cpp unittests/rpc/BaseTests.cpp diff --git a/cloud-example-config.json b/cloud-example-config.json index e985d764..0d638d8c 100644 --- a/cloud-example-config.json +++ b/cloud-example-config.json @@ -1,38 +1,35 @@ { - "database": - { - "type":"cassandra", - "cassandra": - { - "secure_connect_bundle":"[path/to/zip. ignore if using contact_points]", - "contact_points":"[ip. ignore if using secure_connect_bundle]", - "port":"[port. ignore if using_secure_connect_bundle]", - "keyspace":"clio", - "username":"[username, if any]", - "password":"[password, if any]", - "max_requests_outstanding":25000, - "threads":8 + "database": { + "type": "cassandra", + "cassandra": { + "secure_connect_bundle": "[path/to/zip. ignore if using contact_points]", + "contact_points": "[ip. ignore if using secure_connect_bundle]", + "port": "[port. ignore if using_secure_connect_bundle]", + "keyspace": "clio", + "username": "[username, if any]", + "password": "[password, if any]", + "max_requests_outstanding": 25000, + "threads": 8 } }, - "etl_sources": - [ + "etl_sources": [ { - "ip":"[rippled ip]", - "ws_port":"6006", - "grpc_port":"50051" + "ip": "[rippled ip]", + "ws_port": "6006", + "grpc_port": "50051" } ], - "dos_guard": - { - "whitelist":["127.0.0.1"] + "dos_guard": { + "whitelist": [ + "127.0.0.1" + ] }, - "server":{ - "ip":"0.0.0.0", - "port":8080 + "server": { + "ip": "0.0.0.0", + "port": 8080 }, - "log_level":"debug", - "log_file":"./clio.log", - "online_delete":0, - "extractor_threads":8, - "read_only":false + "log_level": "debug", + "log_file": "./clio.log", + "extractor_threads": 8, + "read_only": false } diff --git a/src/backend/BackendFactory.h b/src/backend/BackendFactory.h index 190a5ae2..a2981945 100644 --- a/src/backend/BackendFactory.h +++ b/src/backend/BackendFactory.h @@ -43,9 +43,7 @@ make_Backend(boost::asio::io_context& ioc, clio::Config const& config) if (boost::iequals(type, "cassandra") or boost::iequals(type, "cassandra-new")) { auto cfg = config.section("database." + type); - auto ttl = config.valueOr("online_delete", 0) * 4; - backend = - std::make_shared(Backend::Cassandra::SettingsProvider{cfg, ttl}); + backend = std::make_shared(Backend::Cassandra::SettingsProvider{cfg}); } if (!backend) diff --git a/src/backend/BackendInterface.h b/src/backend/BackendInterface.h index 3692d758..2d9e5d01 100644 --- a/src/backend/BackendInterface.h +++ b/src/backend/BackendInterface.h @@ -21,7 +21,7 @@ #include #include -#include +#include #include #include #include @@ -168,7 +168,7 @@ class BackendInterface protected: mutable std::shared_mutex rngMtx_; std::optional range; - SimpleCache cache_; + LedgerCache cache_; /** * @brief Public read methods @@ -182,23 +182,21 @@ public: BackendInterface() = default; virtual ~BackendInterface() = default; - /*! @brief LEDGER METHODS */ -public: /** * @brief Cache that holds states of the ledger - * - * const version holds the original cache state; the other tracks - * historical changes. - * - * @return SimpleCache const& + * @return Immutable cache */ - SimpleCache const& + LedgerCache const& cache() const { return cache_; } - SimpleCache& + /** + * @brief Cache that holds states of the ledger + * @return Mutable cache + */ + LedgerCache& cache() { return cache_; @@ -563,18 +561,6 @@ public: bool finishWrites(std::uint32_t const ledgerSequence); - /** - * @brief Selectively delets parts of the database. - * - * @param numLedgersToKeep Unsigned 32-bit integer on number of ledgers to - * keep. - * @param yield Currently executing coroutine. - * @return true - * @return false - */ - virtual bool - doOnlineDelete(std::uint32_t numLedgersToKeep, boost::asio::yield_context& yield) const = 0; - virtual bool isTooBusy() const = 0; diff --git a/src/backend/CassandraBackend.h b/src/backend/CassandraBackend.h index 068f99bd..13be6d8e 100644 --- a/src/backend/CassandraBackend.h +++ b/src/backend/CassandraBackend.h @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -231,7 +232,7 @@ public: { if (auto const maybeValue = result.template get>(); maybeValue) { - return deserializeHeader(ripple::makeSlice(*maybeValue)); + return util::deserializeHeader(ripple::makeSlice(*maybeValue)); } log_.error() << "Could not fetch ledger by sequence - no rows"; @@ -792,14 +793,6 @@ public: // probably was used in PG to start a transaction or smth. } - /*! Unused in this implementation */ - bool - doOnlineDelete(std::uint32_t const numLedgersToKeep, boost::asio::yield_context& yield) const override - { - log_.trace() << __func__ << " call"; - return true; - } - bool isTooBusy() const override { diff --git a/src/backend/DBHelpers.h b/src/backend/DBHelpers.h index d36c3712..ec87829e 100644 --- a/src/backend/DBHelpers.h +++ b/src/backend/DBHelpers.h @@ -30,8 +30,9 @@ #include -/// Struct used to keep track of what to write to -/// account_transactions/account_tx tables +/** + * @brief Struct used to keep track of what to write to account_transactions/account_tx tables + */ struct AccountTransactionsData { boost::container::flat_set accounts; @@ -50,8 +51,11 @@ struct AccountTransactionsData AccountTransactionsData() = default; }; -/// Represents a link from a tx to an NFT that was targeted/modified/created -/// by it. Gets written to nf_token_transactions table and the like. +/** + * @brief Represents a link from a tx to an NFT that was targeted/modified/created by it + * + * Gets written to nf_token_transactions table and the like. + */ struct NFTTransactionsData { ripple::uint256 tokenID; @@ -65,8 +69,11 @@ struct NFTTransactionsData } }; -/// Represents an NFT state at a particular ledger. Gets written to nf_tokens -/// table and the like. +/** + * @brief Represents an NFT state at a particular ledger. + * + * Gets written to nf_tokens table and the like. + */ struct NFTsData { ripple::uint256 tokenID; @@ -195,28 +202,6 @@ getBookBase(T const& key) return ret; } -inline ripple::LedgerInfo -deserializeHeader(ripple::Slice data) -{ - ripple::SerialIter sit(data.data(), data.size()); - - ripple::LedgerInfo info; - - info.seq = sit.get32(); - info.drops = sit.get64(); - info.parentHash = sit.get256(); - info.txHash = sit.get256(); - info.accountHash = sit.get256(); - info.parentCloseTime = ripple::NetClock::time_point{ripple::NetClock::duration{sit.get32()}}; - info.closeTime = ripple::NetClock::time_point{ripple::NetClock::duration{sit.get32()}}; - info.closeTimeResolution = ripple::NetClock::duration{sit.get8()}; - info.closeFlags = sit.get8(); - - info.hash = sit.get256(); - - return info; -} - inline std::string uint256ToString(ripple::uint256 const& uint) { diff --git a/src/backend/SimpleCache.cpp b/src/backend/LedgerCache.cpp similarity index 86% rename from src/backend/SimpleCache.cpp rename to src/backend/LedgerCache.cpp index 51f6fc1a..ab65ec0a 100644 --- a/src/backend/SimpleCache.cpp +++ b/src/backend/LedgerCache.cpp @@ -17,18 +17,19 @@ */ //============================================================================== -#include +#include + namespace Backend { uint32_t -SimpleCache::latestLedgerSequence() const +LedgerCache::latestLedgerSequence() const { std::shared_lock lck{mtx_}; return latestSeq_; } void -SimpleCache::update(std::vector const& objs, uint32_t seq, bool isBackground) +LedgerCache::update(std::vector const& objs, uint32_t seq, bool isBackground) { if (disabled_) return; @@ -64,7 +65,7 @@ SimpleCache::update(std::vector const& objs, uint32_t seq, bool is } std::optional -SimpleCache::getSuccessor(ripple::uint256 const& key, uint32_t seq) const +LedgerCache::getSuccessor(ripple::uint256 const& key, uint32_t seq) const { if (!full_) return {}; @@ -80,7 +81,7 @@ SimpleCache::getSuccessor(ripple::uint256 const& key, uint32_t seq) const } std::optional -SimpleCache::getPredecessor(ripple::uint256 const& key, uint32_t seq) const +LedgerCache::getPredecessor(ripple::uint256 const& key, uint32_t seq) const { if (!full_) return {}; @@ -93,8 +94,9 @@ SimpleCache::getPredecessor(ripple::uint256 const& key, uint32_t seq) const --e; return {{e->first, e->second.blob}}; } + std::optional -SimpleCache::get(ripple::uint256 const& key, uint32_t seq) const +LedgerCache::get(ripple::uint256 const& key, uint32_t seq) const { std::shared_lock lck{mtx_}; if (seq > latestSeq_) @@ -110,13 +112,13 @@ SimpleCache::get(ripple::uint256 const& key, uint32_t seq) const } void -SimpleCache::setDisabled() +LedgerCache::setDisabled() { disabled_ = true; } void -SimpleCache::setFull() +LedgerCache::setFull() { if (disabled_) return; @@ -127,28 +129,32 @@ SimpleCache::setFull() } bool -SimpleCache::isFull() const +LedgerCache::isFull() const { return full_; } + size_t -SimpleCache::size() const +LedgerCache::size() const { std::shared_lock lck{mtx_}; return map_.size(); } + float -SimpleCache::getObjectHitRate() const +LedgerCache::getObjectHitRate() const { if (!objectReqCounter_) return 1; return ((float)objectHitCounter_) / objectReqCounter_; } + float -SimpleCache::getSuccessorHitRate() const +LedgerCache::getSuccessorHitRate() const { if (!successorReqCounter_) return 1; return ((float)successorHitCounter_) / successorReqCounter_; } + } // namespace Backend diff --git a/src/backend/SimpleCache.h b/src/backend/LedgerCache.h similarity index 93% rename from src/backend/SimpleCache.h rename to src/backend/LedgerCache.h index bf0d6dc7..7da33e41 100644 --- a/src/backend/SimpleCache.h +++ b/src/backend/LedgerCache.h @@ -27,8 +27,10 @@ #include #include #include + namespace Backend { -class SimpleCache + +class LedgerCache { struct CacheEntry { @@ -39,22 +41,23 @@ class SimpleCache // counters for fetchLedgerObject(s) hit rate mutable std::atomic_uint32_t objectReqCounter_ = 0; mutable std::atomic_uint32_t objectHitCounter_ = 0; + // counters for fetchSuccessorKey hit rate mutable std::atomic_uint32_t successorReqCounter_ = 0; mutable std::atomic_uint32_t successorHitCounter_ = 0; std::map map_; + mutable std::shared_mutex mtx_; uint32_t latestSeq_ = 0; std::atomic_bool full_ = false; std::atomic_bool disabled_ = false; - // temporary set to prevent background thread from writing already deleted - // data. not used when cache is full + + // temporary set to prevent background thread from writing already deleted data. not used when cache is full std::unordered_set> deletes_; public: - // Update the cache with new ledger objects - // set isBackground to true when writing old data from a background thread + // Update the cache with new ledger objects set isBackground to true when writing old data from a background thread void update(std::vector const& blobs, uint32_t seq, bool isBackground = false); diff --git a/src/etl/ETLHelpers.h b/src/etl/ETLHelpers.h index 1ce6a609..49867146 100644 --- a/src/etl/ETLHelpers.h +++ b/src/etl/ETLHelpers.h @@ -26,20 +26,20 @@ #include #include -/// This datastructure is used to keep track of the sequence of the most recent -/// ledger validated by the network. There are two methods that will wait until -/// certain conditions are met. This datastructure is able to be "stopped". When -/// the datastructure is stopped, any threads currently waiting are unblocked. -/// Any later calls to methods of this datastructure will not wait. Once the -/// datastructure is stopped, the datastructure remains stopped for the rest of -/// its lifetime. +/** + * @brief This datastructure is used to keep track of the sequence of the most recent ledger validated by the network. + * + * There are two methods that will wait until certain conditions are met. This datastructure is able to be "stopped". + * When the datastructure is stopped, any threads currently waiting are unblocked. + * Any later calls to methods of this datastructure will not wait. Once the datastructure is stopped, the datastructure + * remains stopped for the rest of its lifetime. + */ class NetworkValidatedLedgers { // max sequence validated by network std::optional max_; mutable std::mutex m_; - std::condition_variable cv_; public: @@ -49,8 +49,11 @@ public: return std::make_shared(); } - /// Notify the datastructure that idx has been validated by the network - /// @param idx sequence validated by network + /** + * @brief Notify the datastructure that idx has been validated by the network + * + * @param idx sequence validated by network + */ void push(uint32_t idx) { @@ -60,10 +63,13 @@ public: cv_.notify_all(); } - /// Get most recently validated sequence. If no ledgers are known to have - /// been validated, this function waits until the next ledger is validated - /// @return sequence of most recently validated ledger. empty optional if - /// the datastructure has been stopped + /** + * @brief Get most recently validated sequence. + * + * If no ledgers are known to have been validated, this function waits until the next ledger is validated + * + * @return sequence of most recently validated ledger. empty optional if the datastructure has been stopped + */ std::optional getMostRecent() { @@ -72,10 +78,13 @@ public: return max_; } - /// Waits for the sequence to be validated by the network - /// @param sequence to wait for - /// @return true if sequence was validated, false otherwise - /// a return value of false means the datastructure has been stopped + /** + * @brief Waits for the sequence to be validated by the network + * + * @param sequence to wait for + * @return true if sequence was validated, false otherwise a return value of false means the datastructure has been + * stopped + */ bool waitUntilValidatedByNetwork(uint32_t sequence, std::optional maxWaitMs = {}) { @@ -89,10 +98,13 @@ public: } }; -/// Generic thread-safe queue with an optional maximum size -/// Note, we can't use a lockfree queue here, since we need the ability to wait -/// for an element to be added or removed from the queue. These waits are -/// blocking calls. +// TODO: does the note make sense? lockfree queues provide the same blocking behaviour just without mutex, don't they? +/** + * @brief Generic thread-safe queue with a max capacity + * + * @note (original note) We can't use a lockfree queue here, since we need the ability to wait for an element to be + * added or removed from the queue. These waits are blocking calls. + */ template class ThreadSafeQueue { @@ -100,75 +112,96 @@ class ThreadSafeQueue mutable std::mutex m_; std::condition_variable cv_; - std::optional maxSize_; + uint32_t maxSize_; public: - /// @param maxSize maximum size of the queue. Calls that would cause the - /// queue to exceed this size will block until free space is available + /** + * @brief Create an instance of the queue + * + * @param maxSize maximum size of the queue. Calls that would cause the queue to exceed this size will block until + * free space is available + */ ThreadSafeQueue(uint32_t maxSize) : maxSize_(maxSize) { } - /// Create a queue with no maximum size - ThreadSafeQueue() = default; - - /// @param elt element to push onto queue - /// if maxSize is set, this method will block until free space is available + /** + * @brief Push element onto the queue + * + * Note: This method will block until free space is available + * + * @param elt element to push onto queue + */ void push(T const& elt) { std::unique_lock lck(m_); - // if queue has a max size, wait until not full - if (maxSize_) - cv_.wait(lck, [this]() { return queue_.size() <= *maxSize_; }); + cv_.wait(lck, [this]() { return queue_.size() <= maxSize_; }); queue_.push(elt); cv_.notify_all(); } - /// @param elt element to push onto queue. elt is moved from - /// if maxSize is set, this method will block until free space is available + /** + * @brief Push element onto the queue + * + * Note: This method will block until free space is available + * + * @param elt element to push onto queue. elt is moved from + */ void push(T&& elt) { std::unique_lock lck(m_); - // if queue has a max size, wait until not full - if (maxSize_) - cv_.wait(lck, [this]() { return queue_.size() <= *maxSize_; }); + cv_.wait(lck, [this]() { return queue_.size() <= maxSize_; }); queue_.push(std::move(elt)); cv_.notify_all(); } - /// @return element popped from queue. Will block until queue is non-empty + /** + * @brief Pop element from the queue + * + * Note: Will block until queue is non-empty + * + * @return element popped from queue + */ T pop() { std::unique_lock lck(m_); cv_.wait(lck, [this]() { return !queue_.empty(); }); + T ret = std::move(queue_.front()); queue_.pop(); - // if queue has a max size, unblock any possible pushers - if (maxSize_) - cv_.notify_all(); + + cv_.notify_all(); return ret; } - /// @return element popped from queue. Will block until queue is non-empty + + /** + * @brief Attempt to pop an element + * + * @return element popped from queue or empty optional if queue was empty + */ std::optional tryPop() { std::scoped_lock lck(m_); if (queue_.empty()) return {}; + T ret = std::move(queue_.front()); queue_.pop(); - // if queue has a max size, unblock any possible pushers - if (maxSize_) - cv_.notify_all(); + + cv_.notify_all(); return ret; } }; -/// Parititions the uint256 keyspace into numMarkers partitions, each of equal -/// size. +/** + * @brief Parititions the uint256 keyspace into numMarkers partitions, each of equal size. + * + * @param numMarkers total markers to partition for + */ inline std::vector getMarkers(size_t numMarkers) { diff --git a/src/etl/ETLService.cpp b/src/etl/ETLService.cpp new file mode 100644 index 00000000..f96d2836 --- /dev/null +++ b/src/etl/ETLService.cpp @@ -0,0 +1,259 @@ +//------------------------------------------------------------------------------ +/* + 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 + +using namespace clio; + +// Database must be populated when this starts +std::optional +ETLService::runETLPipeline(uint32_t startSequence, int numExtractors) +{ + if (finishSequence_ && startSequence > *finishSequence_) + return {}; + + log_.debug() << "Starting etl pipeline"; + state_.isWriting = true; + + auto rng = backend_->hardFetchLedgerRangeNoThrow(); + if (!rng || rng->maxSequence < startSequence - 1) + { + assert(false); + throw std::runtime_error("runETLPipeline: parent ledger is null"); + } + + auto const begin = std::chrono::system_clock::now(); + auto extractors = std::vector>{}; + auto pipe = DataPipeType{numExtractors, startSequence}; + + for (auto i = 0u; i < numExtractors; ++i) + extractors.push_back(std::make_unique( + pipe, networkValidatedLedgers_, ledgerFetcher_, startSequence + i, finishSequence_, state_)); + + auto transformer = TransformerType{pipe, backend_, ledgerLoader_, ledgerPublisher_, 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(); + log_.debug() << "Extracted and wrote " << lastPublishedSeq.value_or(startSequence) - startSequence << " in " + << ((end - begin).count()) / 1000000000.0; + + state_.isWriting = false; + + log_.debug() << "Stopping etl pipeline"; + return lastPublishedSeq; +} + +// Main loop of ETL. +// The software begins monitoring the ledgers that are validated by the nework. +// 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() +{ + auto rng = backend_->hardFetchLedgerRangeNoThrow(); + if (!rng) + { + log_.info() << "Database is empty. Will download a ledger " + "from the network."; + std::optional ledger; + + if (startSequence_) + { + log_.info() << "ledger sequence specified in config. " + << "Will begin ETL process starting with ledger " << *startSequence_; + ledger = ledgerLoader_.loadInitialLedger(*startSequence_); + } + else + { + log_.info() << "Waiting for next ledger to be validated by network..."; + std::optional mostRecentValidated = networkValidatedLedgers_->getMostRecent(); + + if (mostRecentValidated) + { + log_.info() << "Ledger " << *mostRecentValidated << " has been validated. " + << "Downloading..."; + ledger = ledgerLoader_.loadInitialLedger(*mostRecentValidated); + } + else + { + log_.info() << "The wait for the next validated " + << "ledger has been aborted. " + << "Exiting monitor loop"; + return; + } + } + + if (ledger) + { + rng = backend_->hardFetchLedgerRangeNoThrow(); + } + else + { + log_.error() << "Failed to load initial ledger. Exiting monitor loop"; + return; + } + } + else + { + if (startSequence_) + log_.warn() << "start sequence specified but db is already populated"; + + log_.info() << "Database already populated. Picking up from the tip of history"; + cacheLoader_.load(rng->maxSequence); + } + + assert(rng); + uint32_t nextSequence = rng->maxSequence + 1; + + log_.debug() << "Database is populated. " + << "Starting monitor loop. sequence = " << nextSequence; + + while (true) + { + if (auto rng = backend_->hardFetchLedgerRangeNoThrow(); rng && rng->maxSequence >= nextSequence) + { + ledgerPublisher_.publish(nextSequence, {}); + ++nextSequence; + } + else if (networkValidatedLedgers_->waitUntilValidatedByNetwork(nextSequence, 1000)) + { + 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 timeoutSeconds = 10; + bool success = ledgerPublisher_.publish(nextSequence, timeoutSeconds); + + if (!success) + { + 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 lastPublished = runETLPipeline(nextSequence, extractorThreads_); + log_.info() << "Aborting ETL. Falling back to publishing"; + + // if no ledger was published, don't increment nextSequence + if (lastPublished) + nextSequence = *lastPublished + 1; + } + else + { + ++nextSequence; + } + } + } +} + +void +ETLService::monitorReadOnly() +{ + log_.debug() << "Starting reporting in strict read only mode"; + + auto rng = backend_->hardFetchLedgerRangeNoThrow(); + uint32_t latestSequence; + + if (!rng) + { + if (auto net = networkValidatedLedgers_->getMostRecent()) + latestSequence = *net; + else + return; + } + else + { + latestSequence = rng->maxSequence; + } + + cacheLoader_.load(latestSequence); + latestSequence++; + + while (true) + { + 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, 1000); + } + } +} + +void +ETLService::run() +{ + log_.info() << "Starting reporting etl"; + state_.isStopping = false; + + doWork(); +} + +void +ETLService::doWork() +{ + worker_ = std::thread([this]() { + beast::setCurrentThreadName("rippled: ETLService worker"); + + if (state_.isReadOnly) + monitorReadOnly(); + else + monitor(); + }); +} + +ETLService::ETLService( + clio::Config const& config, + boost::asio::io_context& ioc, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr balancer, + std::shared_ptr ledgers) + : backend_(backend) + , loadBalancer_(balancer) + , networkValidatedLedgers_(ledgers) + , cacheLoader_(config, ioc, backend, backend->cache()) + , ledgerFetcher_(backend, balancer) + , ledgerLoader_(backend, balancer, ledgerFetcher_, state_) + , ledgerPublisher_(ioc, backend, subscriptions, state_) +{ + startSequence_ = config.maybeValue("start_sequence"); + finishSequence_ = config.maybeValue("finish_sequence"); + state_.isReadOnly = config.valueOr("read_only", state_.isReadOnly); + extractorThreads_ = config.valueOr("extractor_threads", extractorThreads_); + txnThreshold_ = config.valueOr("txn_threshold", txnThreshold_); +} diff --git a/src/etl/ETLService.h b/src/etl/ETLService.h new file mode 100644 index 00000000..11fc44b6 --- /dev/null +++ b/src/etl/ETLService.h @@ -0,0 +1,239 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" +#include + +#include + +struct AccountTransactionsData; +struct NFTTransactionsData; +struct NFTsData; +class SubscriptionManager; + +/** + * @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 +{ + // TODO: make these template parameters in ETLService + using SubscriptionManagerType = SubscriptionManager; + using LoadBalancerType = LoadBalancer; + using NetworkValidatedLedgersType = NetworkValidatedLedgers; + using DataPipeType = clio::detail::ExtractionDataPipe; + using CacheLoaderType = clio::detail::CacheLoader; + using LedgerFetcherType = clio::detail::LedgerFetcher; + using ExtractorType = clio::detail::Extractor; + using LedgerLoaderType = clio::detail::LedgerLoader; + using LedgerPublisherType = clio::detail::LedgerPublisher; + using TransformerType = clio::detail::Transformer; + + clio::Logger log_{"ETL"}; + + std::shared_ptr backend_; + std::shared_ptr loadBalancer_; + std::shared_ptr networkValidatedLedgers_; + + std::uint32_t extractorThreads_ = 1; + std::thread worker_; + + CacheLoaderType cacheLoader_; + LedgerFetcherType ledgerFetcher_; + LedgerLoaderType ledgerLoader_; + LedgerPublisherType ledgerPublisher_; + + SystemState state_; + + size_t numMarkers_ = 2; + std::optional startSequence_; + std::optional finishSequence_; + size_t txnThreshold_ = 0; + +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( + clio::Config const& config, + boost::asio::io_context& ioc, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr balancer, + std::shared_ptr ledgers); + + static std::shared_ptr + make_ETLService( + clio::Config const& config, + boost::asio::io_context& ioc, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr balancer, + std::shared_ptr ledgers) + { + auto etl = std::make_shared(config, ioc, backend, subscriptions, balancer, ledgers); + etl->run(); + + return etl; + } + + /** + * @brief Stops components and joins worker thread + */ + ~ETLService() + { + log_.info() << "onStop called"; + log_.debug() << "Stopping Reporting ETL"; + + state_.isStopping = true; + cacheLoader_.stop(); + + if (worker_.joinable()) + worker_.join(); + + log_.debug() << "Joined ETLService worker thread"; + } + + /** + * @brief Get time passed since last ledger close, in seconds + */ + std::uint32_t + lastCloseAgeSeconds() const + { + return ledgerPublisher_.lastCloseAgeSeconds(); + } + + /** + * @brief Get state of ETL as a JSON object + */ + boost::json::object + getInfo() const + { + boost::json::object result; + + result["etl_sources"] = loadBalancer_->toJson(); + result["is_writer"] = state_.isWriting.load(); + result["read_only"] = state_.isReadOnly; + auto last = ledgerPublisher_.getLastPublish(); + if (last.time_since_epoch().count() != 0) + result["last_publish_age_seconds"] = std::to_string(ledgerPublisher_.lastPublishAgeSeconds()); + return result; + } + +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 + * @return the last ledger written to the database, if any + */ + std::optional + runETLPipeline(uint32_t startSequence, int offset); + + /** + * @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(); + + /** + * @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() + { + return state_.isStopping; + } + + /** + * @brief Get the number of markers to use during the initial ledger download + * + * This is equivelent to the degree of parallelism during the initial ledger download. + * + * @return the number of markers + */ + std::uint32_t + getNumMarkers() + { + return numMarkers_; + } + + /** + * @brief Start all components to run ETL service + */ + void + run(); + + /** + * @brief Spawn the worker thread and start monitoring + */ + void + doWork(); +}; diff --git a/src/etl/LoadBalancer.cpp b/src/etl/LoadBalancer.cpp new file mode 100644 index 00000000..1819d000 --- /dev/null +++ b/src/etl/LoadBalancer.cpp @@ -0,0 +1,235 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include + +using namespace clio; + +std::unique_ptr +LoadBalancer::make_Source( + clio::Config const& config, + boost::asio::io_context& ioContext, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr networkValidatedLedgers, + LoadBalancer& balancer) +{ + auto src = + std::make_unique(config, ioContext, backend, subscriptions, networkValidatedLedgers, balancer); + + src->run(); + + return src; +} + +std::shared_ptr +LoadBalancer::make_LoadBalancer( + clio::Config const& config, + boost::asio::io_context& ioc, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr validatedLedgers) +{ + return std::make_shared(config, ioc, backend, subscriptions, validatedLedgers); +} + +LoadBalancer::LoadBalancer( + clio::Config const& config, + boost::asio::io_context& ioContext, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr nwvl) +{ + if (auto value = config.maybeValue("num_markers"); value) + downloadRanges_ = std::clamp(*value, 1u, 256u); + else if (backend->fetchLedgerRange()) + downloadRanges_ = 4; + + for (auto const& entry : config.array("etl_sources")) + { + std::unique_ptr source = make_Source(entry, ioContext, backend, subscriptions, nwvl, *this); + + sources_.push_back(std::move(source)); + log_.info() << "Added etl source - " << sources_.back()->toString(); + } +} + +std::pair, bool> +LoadBalancer::loadInitialLedger(uint32_t sequence, bool cacheOnly) +{ + std::vector response; + auto const success = execute( + [this, &response, &sequence, cacheOnly](auto& source) { + auto [data, res] = source->loadInitialLedger(sequence, downloadRanges_, cacheOnly); + + if (!res) + log_.error() << "Failed to download initial ledger." + << " Sequence = " << sequence << " source = " << source->toString(); + else + response = std::move(data); + + return res; + }, + sequence); + return {std::move(response), success}; +} + +LoadBalancer::DataType +LoadBalancer::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors) +{ + RawDataType response; + bool success = 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.info() << "Successfully fetched ledger = " << ledgerSequence + << " from source = " << source->toString(); + return true; + } + else + { + 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); + if (success) + return response; + else + return {}; +} + +std::optional +LoadBalancer::forwardToRippled( + boost::json::object const& request, + std::string const& clientIp, + boost::asio::yield_context& yield) const +{ + srand((unsigned)time(0)); + auto sourceIdx = rand() % sources_.size(); + auto numAttempts = 0; + + while (numAttempts < sources_.size()) + { + if (auto res = sources_[sourceIdx]->forwardToRippled(request, clientIp, yield)) + return res; + + sourceIdx = (sourceIdx + 1) % sources_.size(); + ++numAttempts; + } + + return {}; +} + +bool +LoadBalancer::shouldPropagateTxnStream(Source* in) const +{ + for (auto& src : sources_) + { + assert(src); + + // We pick the first Source encountered that is connected + if (src->isConnected()) + return *src == *in; + } + + // If no sources connected, then this stream has not been forwarded + return true; +} + +boost::json::value +LoadBalancer::toJson() const +{ + boost::json::array ret; + for (auto& src : sources_) + ret.push_back(src->toJson()); + + return ret; +} + +template +bool +LoadBalancer::execute(Func f, uint32_t ledgerSequence) +{ + srand((unsigned)time(0)); + auto sourceIdx = rand() % sources_.size(); + auto numAttempts = 0; + + while (true) + { + auto& source = sources_[sourceIdx]; + + 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 res = f(source); + if (res) + { + log_.debug() << "Successfully executed func at source = " << source->toString() + << " - ledger sequence = " << ledgerSequence; + break; + } + else + { + log_.warn() << "Failed to execute func at source = " << source->toString() + << " - ledger sequence = " << ledgerSequence; + } + } + else + { + log_.warn() << "Ledger not present at source = " << source->toString() + << " - ledger sequence = " << ledgerSequence; + } + sourceIdx = (sourceIdx + 1) % sources_.size(); + numAttempts++; + if (numAttempts % sources_.size() == 0) + { + log_.info() << "Ledger sequence " << ledgerSequence << " is not yet available from any configured sources. " + << "Sleeping and trying again"; + std::this_thread::sleep_for(std::chrono::seconds(2)); + } + } + return true; +} diff --git a/src/etl/LoadBalancer.h b/src/etl/LoadBalancer.h new file mode 100644 index 00000000..00c578ab --- /dev/null +++ b/src/etl/LoadBalancer.h @@ -0,0 +1,160 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include +#include +#include + +#include +#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" +#include + +class Source; +class ProbingSource; +class SubscriptionManager; + +/** + * @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: + using RawDataType = org::xrpl::rpc::v1::GetLedgerResponse; + using DataType = std::optional; + +private: + clio::Logger log_{"ETL"}; + std::vector> sources_; + std::uint32_t downloadRanges_ = 16; + +public: + /** + * @brief Create an instance of the load balancer + * + * @param config The configuration to use + * @param ioContext io context to run on + * @param backend BackendInterface implementation + * @param subscriptions Subscription manager + * @param nwvl The network validated ledgers datastructure + */ + LoadBalancer( + clio::Config const& config, + boost::asio::io_context& ioContext, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr nwvl); + + static std::shared_ptr + make_LoadBalancer( + clio::Config const& config, + boost::asio::io_context& ioc, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr validatedLedgers); + + static std::unique_ptr + make_Source( + clio::Config const& config, + boost::asio::io_context& ioContext, + std::shared_ptr backend, + std::shared_ptr subscriptions, + std::shared_ptr networkValidatedLedgers, + LoadBalancer& balancer); + + ~LoadBalancer() + { + sources_.clear(); + } + + /** + * @brief Load the initial ledger, writing data to the queue + * + * @param sequence sequence of ledger to download + */ + std::pair, bool> + loadInitialLedger(uint32_t sequence, bool cacheOnly = false); + + /** + * @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 ledger to fetch data for + * @param getObjects if true, fetch diff between specified ledger and previous + * @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 + */ + DataType + fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors); + + /** + * @brief Determine whether messages received on the transactions_proposed stream should be forwarded to subscribing + * clients. + * + * The server subscribes to transactions_proposed on multiple Sources, yet only forwards messages from one source at + * any given time (to avoid sending duplicate messages to clients). + * + * @param in Source in question + * @return true if messages should be forwarded + */ + bool + shouldPropagateTxnStream(Source* in) const; + + /** + * @return JSON representation of the state of this load balancer + */ + boost::json::value + toJson() const; + + /** + * @brief Forward a JSON RPC request to a randomly selected rippled node + * + * @param request JSON-RPC request + * @return response received from rippled node + */ + std::optional + forwardToRippled(boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context& yield) + const; + +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 + * @return true if f was eventually executed successfully. false if the ledger was found in the database or the + * server is shutting down + */ + template + bool + execute(Func f, uint32_t ledgerSequence); +}; diff --git a/src/etl/NFTHelpers.h b/src/etl/NFTHelpers.h index 597ff272..92314b82 100644 --- a/src/etl/NFTHelpers.h +++ b/src/etl/NFTHelpers.h @@ -24,10 +24,14 @@ #include #include -// Pulling from tx via ReportingETL +/** + * @brief Pull NFT data from TX via ETLService + */ std::pair, std::optional> getNFTDataFromTx(ripple::TxMeta const& txMeta, ripple::STTx const& sttx); -// Pulling from ledger object via loadInitialLedger +/** + * @brief Pull NFT data from ledger object via loadInitialLedger + */ std::vector getNFTDataFromObj(std::uint32_t const seq, std::string const& key, std::string const& blob); diff --git a/src/etl/ProbingETLSource.cpp b/src/etl/ProbingSource.cpp similarity index 76% rename from src/etl/ProbingETLSource.cpp rename to src/etl/ProbingSource.cpp index ec08f36c..1317ef66 100644 --- a/src/etl/ProbingETLSource.cpp +++ b/src/etl/ProbingSource.cpp @@ -17,55 +17,55 @@ */ //============================================================================== -#include +#include #include using namespace clio; -ProbingETLSource::ProbingETLSource( +ProbingSource::ProbingSource( clio::Config const& config, boost::asio::io_context& ioc, std::shared_ptr backend, std::shared_ptr subscriptions, std::shared_ptr nwvl, - ETLLoadBalancer& balancer, + LoadBalancer& balancer, boost::asio::ssl::context sslCtx) : sslCtx_{std::move(sslCtx)} , sslSrc_{make_shared< - SslETLSource>(config, ioc, std::ref(sslCtx_), backend, subscriptions, nwvl, balancer, make_SSLHooks())} - , plainSrc_{make_shared(config, ioc, backend, subscriptions, nwvl, balancer, make_PlainHooks())} + SslSource>(config, ioc, std::ref(sslCtx_), backend, subscriptions, nwvl, balancer, make_SSLHooks())} + , plainSrc_{make_shared(config, ioc, backend, subscriptions, nwvl, balancer, make_PlainHooks())} { } void -ProbingETLSource::run() +ProbingSource::run() { sslSrc_->run(); plainSrc_->run(); } void -ProbingETLSource::pause() +ProbingSource::pause() { sslSrc_->pause(); plainSrc_->pause(); } void -ProbingETLSource::resume() +ProbingSource::resume() { sslSrc_->resume(); plainSrc_->resume(); } bool -ProbingETLSource::isConnected() const +ProbingSource::isConnected() const { return currentSrc_ && currentSrc_->isConnected(); } bool -ProbingETLSource::hasLedger(uint32_t sequence) const +ProbingSource::hasLedger(uint32_t sequence) const { if (!currentSrc_) return false; @@ -73,7 +73,7 @@ ProbingETLSource::hasLedger(uint32_t sequence) const } boost::json::object -ProbingETLSource::toJson() const +ProbingSource::toJson() const { if (!currentSrc_) { @@ -90,7 +90,7 @@ ProbingETLSource::toJson() const } std::string -ProbingETLSource::toString() const +ProbingSource::toString() const { if (!currentSrc_) return "{probing... ws: " + plainSrc_->toString() + ", wss: " + sslSrc_->toString() + "}"; @@ -98,23 +98,23 @@ ProbingETLSource::toString() const } boost::uuids::uuid -ProbingETLSource::token() const +ProbingSource::token() const { if (!currentSrc_) return boost::uuids::nil_uuid(); return currentSrc_->token(); } -bool -ProbingETLSource::loadInitialLedger(std::uint32_t ledgerSequence, std::uint32_t numMarkers, bool cacheOnly) +std::pair, bool> +ProbingSource::loadInitialLedger(std::uint32_t ledgerSequence, std::uint32_t numMarkers, bool cacheOnly) { if (!currentSrc_) - return false; + return {{}, false}; return currentSrc_->loadInitialLedger(ledgerSequence, numMarkers, cacheOnly); } std::pair -ProbingETLSource::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors) +ProbingSource::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors) { if (!currentSrc_) return {}; @@ -122,7 +122,7 @@ ProbingETLSource::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool get } std::optional -ProbingETLSource::forwardToRippled( +ProbingSource::forwardToRippled( boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context& yield) const @@ -133,7 +133,7 @@ ProbingETLSource::forwardToRippled( } std::optional -ProbingETLSource::requestFromRippled( +ProbingSource::requestFromRippled( boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context& yield) const @@ -143,14 +143,14 @@ ProbingETLSource::requestFromRippled( return currentSrc_->requestFromRippled(request, clientIp, yield); } -ETLSourceHooks -ProbingETLSource::make_SSLHooks() noexcept +SourceHooks +ProbingSource::make_SSLHooks() noexcept { return {// onConnected [this](auto ec) { std::lock_guard lck(mtx_); if (currentSrc_) - return ETLSourceHooks::Action::STOP; + return SourceHooks::Action::STOP; if (!ec) { @@ -158,7 +158,7 @@ ProbingETLSource::make_SSLHooks() noexcept currentSrc_ = sslSrc_; log_.info() << "Selected WSS as the main source: " << currentSrc_->toString(); } - return ETLSourceHooks::Action::PROCEED; + return SourceHooks::Action::PROCEED; }, // onDisconnected [this](auto ec) { @@ -168,18 +168,18 @@ ProbingETLSource::make_SSLHooks() noexcept currentSrc_ = nullptr; plainSrc_->resume(); } - return ETLSourceHooks::Action::STOP; + return SourceHooks::Action::STOP; }}; } -ETLSourceHooks -ProbingETLSource::make_PlainHooks() noexcept +SourceHooks +ProbingSource::make_PlainHooks() noexcept { return {// onConnected [this](auto ec) { std::lock_guard lck(mtx_); if (currentSrc_) - return ETLSourceHooks::Action::STOP; + return SourceHooks::Action::STOP; if (!ec) { @@ -187,7 +187,7 @@ ProbingETLSource::make_PlainHooks() noexcept currentSrc_ = plainSrc_; log_.info() << "Selected Plain WS as the main source: " << currentSrc_->toString(); } - return ETLSourceHooks::Action::PROCEED; + return SourceHooks::Action::PROCEED; }, // onDisconnected [this](auto ec) { @@ -197,6 +197,6 @@ ProbingETLSource::make_PlainHooks() noexcept currentSrc_ = nullptr; sslSrc_->resume(); } - return ETLSourceHooks::Action::STOP; + return SourceHooks::Action::STOP; }}; } diff --git a/src/etl/ProbingETLSource.h b/src/etl/ProbingSource.h similarity index 72% rename from src/etl/ProbingETLSource.h rename to src/etl/ProbingSource.h index 89cc2854..d7faed05 100644 --- a/src/etl/ProbingETLSource.h +++ b/src/etl/ProbingSource.h @@ -19,6 +19,10 @@ #pragma once +#include +#include +#include + #include #include #include @@ -27,35 +31,44 @@ #include -#include -#include -#include - -/// This ETLSource implementation attempts to connect over both secure websocket -/// and plain websocket. First to connect pauses the other and the probing is -/// considered done at this point. If however the connected source loses -/// connection the probing is kickstarted again. -class ProbingETLSource : public ETLSource +/** + * @brief This Source implementation attempts to connect over both secure websocket and plain websocket. + * + * First to connect pauses the other and the probing is considered done at this point. + * If however the connected source loses connection the probing is kickstarted again. + */ +class ProbingSource : public Source { clio::Logger log_{"ETL"}; std::mutex mtx_; boost::asio::ssl::context sslCtx_; - std::shared_ptr sslSrc_; - std::shared_ptr plainSrc_; - std::shared_ptr currentSrc_; + std::shared_ptr sslSrc_; + std::shared_ptr plainSrc_; + std::shared_ptr currentSrc_; public: - ProbingETLSource( + /** + * @brief Create an instance of the probing source + * + * @param config The configuration to use + * @param ioc io context to run on + * @param backend BackendInterface implementation + * @param subscriptions Subscription manager + * @param nwvl The network validated ledgers datastructure + * @param balancer Load balancer to use + * @param sslCtx The SSL context to use; defaults to tlsv12 + */ + ProbingSource( clio::Config const& config, boost::asio::io_context& ioc, std::shared_ptr backend, std::shared_ptr subscriptions, std::shared_ptr nwvl, - ETLLoadBalancer& balancer, + LoadBalancer& balancer, boost::asio::ssl::context sslCtx = boost::asio::ssl::context{boost::asio::ssl::context::tlsv12}); - ~ProbingETLSource() = default; + ~ProbingSource() = default; void run() override; @@ -78,7 +91,7 @@ public: std::string toString() const override; - bool + std::pair, bool> loadInitialLedger(std::uint32_t ledgerSequence, std::uint32_t numMarkers, bool cacheOnly = false) override; std::pair @@ -98,9 +111,9 @@ private: std::string const& clientIp, boost::asio::yield_context& yield) const override; - ETLSourceHooks + SourceHooks make_SSLHooks() noexcept; - ETLSourceHooks + SourceHooks make_PlainHooks() noexcept; }; diff --git a/src/etl/ReportingETL.cpp b/src/etl/ReportingETL.cpp deleted file mode 100644 index d7194f4e..00000000 --- a/src/etl/ReportingETL.cpp +++ /dev/null @@ -1,1141 +0,0 @@ -//------------------------------------------------------------------------------ -/* - This file is part of clio: https://github.com/XRPLF/clio - Copyright (c) 2022, the clio developers. - - Permission to use, copy, modify, and distribute this software for any - purpose with or without fee is hereby granted, provided that the above - copyright notice and this permission notice appear in all copies. - - THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES - WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF - MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR - ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES - WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN - ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF - OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. -*/ -//============================================================================== - -#include -#include - -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -using namespace clio; - -namespace clio::detail { -/// Convenience function for printing out basic ledger info -std::string -toString(ripple::LedgerInfo const& info) -{ - std::stringstream ss; - ss << "LedgerInfo { Sequence : " << info.seq << " Hash : " << strHex(info.hash) - << " TxHash : " << strHex(info.txHash) << " AccountHash : " << strHex(info.accountHash) - << " ParentHash : " << strHex(info.parentHash) << " }"; - return ss.str(); -} -} // namespace clio::detail - -FormattedTransactionsData -ReportingETL::insertTransactions(ripple::LedgerInfo const& ledger, org::xrpl::rpc::v1::GetLedgerResponse& data) -{ - FormattedTransactionsData result; - - 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 sttx{it}; - - log_.trace() << "Inserting transaction = " << sttx.getTransactionID(); - - ripple::TxMeta 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()); - if (maybeNFT) - result.nfTokensData.push_back(*maybeNFT); - - auto journal = ripple::debugLog(); - result.accountTxData.emplace_back(txMeta, sttx.getTransactionID(), journal); - std::string keyStr{(const char*)sttx.getTransactionID().data(), 32}; - backend_->writeTransaction( - std::move(keyStr), - ledger.seq, - ledger.closeTime.time_since_epoch().count(), - std::move(*raw), - std::move(*txn.mutable_metadata_blob())); - } - - // Remove all but the last NFTsData for each id. unique removes all - // but the first of a group, so we want to reverse sort by transaction - // index - std::sort(result.nfTokensData.begin(), result.nfTokensData.end(), [](NFTsData const& a, NFTsData const& b) { - return a.tokenID > b.tokenID && a.transactionIndex > b.transactionIndex; - }); - // Now we can unique the NFTs by tokenID. - auto last = - std::unique(result.nfTokensData.begin(), result.nfTokensData.end(), [](NFTsData const& a, NFTsData const& b) { - return a.tokenID == b.tokenID; - }); - result.nfTokensData.erase(last, result.nfTokensData.end()); - - return result; -} - -std::optional -ReportingETL::loadInitialLedger(uint32_t startingSequence) -{ - // check that database is actually empty - auto rng = backend_->hardFetchLedgerRangeNoThrow(); - if (rng) - { - log_.fatal() << "Database is not empty"; - assert(false); - 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 - std::optional ledgerData{fetchLedgerData(startingSequence)}; - if (!ledgerData) - return {}; - - ripple::LedgerInfo lgrInfo = deserializeHeader(ripple::makeSlice(ledgerData->ledger_header())); - - log_.debug() << "Deserialized ledger header. " << detail::toString(lgrInfo); - - auto timeDiff = util::timed>([&]() { - backend_->startWrites(); - - log_.debug() << "Started writes"; - - backend_->writeLedger(lgrInfo, std::move(*ledgerData->mutable_ledger_header())); - - log_.debug() << "Wrote ledger"; - FormattedTransactionsData insertTxResult = insertTransactions(lgrInfo, *ledgerData); - 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 - loadBalancer_->loadInitialLedger(startingSequence); - - log_.debug() << "Loaded initial ledger"; - - if (!stopping_) - { - backend_->writeAccountTransactions(std::move(insertTxResult.accountTxData)); - backend_->writeNFTs(std::move(insertTxResult.nfTokensData)); - backend_->writeNFTTransactions(std::move(insertTxResult.nfTokenTxData)); - } - backend_->finishWrites(startingSequence); - }); - log_.debug() << "Time to download and store ledger = " << timeDiff; - return lgrInfo; -} - -void -ReportingETL::publishLedger(ripple::LedgerInfo const& lgrInfo) -{ - log_.info() << "Publishing ledger " << std::to_string(lgrInfo.seq); - - if (!writing_) - { - log_.info() << "Updating cache"; - - std::vector diff = Backend::synchronousAndRetryOnTimeout( - [&](auto yield) { return backend_->fetchLedgerDiff(lgrInfo.seq, yield); }); - - backend_->cache().update(diff, lgrInfo.seq); - backend_->updateRange(lgrInfo.seq); - } - - setLastClose(lgrInfo.closeTime); - auto age = lastCloseAgeSeconds(); - // if the ledger closed over 10 minutes ago, assume we are still - // catching up and don't publish - if (age < 600) - { - std::optional fees = - Backend::synchronousAndRetryOnTimeout([&](auto yield) { return backend_->fetchFees(lgrInfo.seq, yield); }); - - std::vector transactions = Backend::synchronousAndRetryOnTimeout( - [&](auto yield) { return backend_->fetchAllTransactionsInLedger(lgrInfo.seq, yield); }); - - auto ledgerRange = backend_->fetchLedgerRange(); - assert(ledgerRange); - assert(fees); - - std::string range = std::to_string(ledgerRange->minSequence) + "-" + std::to_string(ledgerRange->maxSequence); - - subscriptions_->pubLedger(lgrInfo, *fees, range, transactions.size()); - - for (auto& txAndMeta : transactions) - subscriptions_->pubTransaction(txAndMeta, lgrInfo); - - subscriptions_->pubBookChanges(lgrInfo, transactions); - - log_.info() << "Published ledger " << std::to_string(lgrInfo.seq); - } - else - log_.info() << "Skipping publishing ledger " << std::to_string(lgrInfo.seq); - setLastPublish(); -} - -bool -ReportingETL::publishLedger(uint32_t ledgerSequence, std::optional maxAttempts) -{ - log_.info() << "Attempting to publish ledger = " << ledgerSequence; - size_t numAttempts = 0; - while (!stopping_) - { - auto range = backend_->hardFetchLedgerRangeNoThrow(); - - if (!range || range->maxSequence < ledgerSequence) - { - 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_.debug() << "Failed to publish ledger after " << numAttempts << " attempts."; - return false; - } - std::this_thread::sleep_for(std::chrono::seconds(1)); - ++numAttempts; - continue; - } - else - { - auto lgr = Backend::synchronousAndRetryOnTimeout( - [&](auto yield) { return backend_->fetchLedgerBySequence(ledgerSequence, yield); }); - - assert(lgr); - publishLedger(*lgr); - - return true; - } - } - return false; -} - -std::optional -ReportingETL::fetchLedgerData(uint32_t seq) -{ - log_.debug() << "Attempting to fetch ledger with sequence = " << seq; - - std::optional response = loadBalancer_->fetchLedger(seq, false, false); - if (response) - log_.trace() << "GetLedger reply = " << response->DebugString(); - return response; -} - -std::optional -ReportingETL::fetchLedgerDataAndDiff(uint32_t seq) -{ - log_.debug() << "Attempting to fetch ledger with sequence = " << seq; - - std::optional response = loadBalancer_->fetchLedger( - seq, true, !backend_->cache().isFull() || backend_->cache().latestLedgerSequence() >= seq); - if (response) - log_.trace() << "GetLedger reply = " << response->DebugString(); - return response; -} - -std::pair -ReportingETL::buildNextLedger(org::xrpl::rpc::v1::GetLedgerResponse& rawData) -{ - log_.debug() << "Beginning ledger update"; - ripple::LedgerInfo lgrInfo = deserializeHeader(ripple::makeSlice(rawData.ledger_header())); - - log_.debug() << "Deserialized ledger header. " << detail::toString(lgrInfo); - backend_->startWrites(); - log_.debug() << "started writes"; - - backend_->writeLedger(lgrInfo, std::move(*rawData.mutable_ledger_header())); - log_.debug() << "wrote ledger header"; - - // Write successor info, if included from rippled - if (rawData.object_neighbors_included()) - { - 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(Backend::lastKey); - 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() != org::xrpl::rpc::v1::RawLedgerObject::MODIFIED) - { - std::string* predPtr = obj.mutable_predecessor(); - if (!predPtr->size()) - *predPtr = uint256ToString(Backend::firstKey); - std::string* succPtr = obj.mutable_successor(); - if (!succPtr->size()) - *succPtr = uint256ToString(Backend::lastKey); - - if (obj.mod_type() == org::xrpl::rpc::v1::RawLedgerObject::DELETED) - { - 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_.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_.debug() << "object modified " << ripple::strHex(obj.key()); - } - } - std::vector cacheUpdates; - cacheUpdates.reserve(rawData.ledger_objects().objects_size()); - // TODO change these to unordered_set - std::set bookSuccessorsToCalculate; - std::set modified; - for (auto& obj : *(rawData.mutable_ledger_objects()->mutable_objects())) - { - auto key = ripple::uint256::fromVoidChecked(obj.key()); - assert(key); - cacheUpdates.push_back({*key, {obj.mutable_data()->begin(), obj.mutable_data()->end()}}); - log_.debug() << "key = " << ripple::strHex(*key) << " - mod type = " << obj.mod_type(); - - if (obj.mod_type() != org::xrpl::rpc::v1::RawLedgerObject::MODIFIED && !rawData.object_neighbors_included()) - { - log_.debug() << "object neighbors not included. using cache"; - if (!backend_->cache().isFull() || backend_->cache().latestLedgerSequence() != lgrInfo.seq - 1) - throw std::runtime_error( - "Cache is not full, but object neighbors were not " - "included"); - auto blob = obj.mutable_data(); - bool checkBookBase = false; - bool isDeleted = (blob->size() == 0); - if (isDeleted) - { - auto old = backend_->cache().get(*key, lgrInfo.seq - 1); - assert(old); - checkBookBase = isBookDir(*key, *old); - } - else - checkBookBase = isBookDir(*key, *blob); - if (checkBookBase) - { - log_.debug() << "Is book dir. key = " << ripple::strHex(*key); - auto bookBase = getBookBase(*key); - auto oldFirstDir = backend_->cache().getSuccessor(bookBase, lgrInfo.seq - 1); - assert(oldFirstDir); - // 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_.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() == org::xrpl::rpc::v1::RawLedgerObject::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_.debug() << "object neighbors not included. using cache"; - if (!backend_->cache().isFull() || backend_->cache().latestLedgerSequence() != lgrInfo.seq) - throw std::runtime_error( - "Cache is not full, but object neighbors were not " - "included"); - for (auto const& obj : cacheUpdates) - { - if (modified.count(obj.key)) - continue; - auto lb = backend_->cache().getPredecessor(obj.key, lgrInfo.seq); - if (!lb) - lb = {Backend::firstKey, {}}; - auto ub = backend_->cache().getSuccessor(obj.key, lgrInfo.seq); - if (!ub) - ub = {Backend::lastKey, {}}; - if (obj.blob.size() == 0) - { - 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_.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_.debug() << "Updating book successor " << ripple::strHex(base) << " - " - << ripple::strHex(succ->key); - } - else - { - backend_->writeSuccessor(uint256ToString(base), lgrInfo.seq, uint256ToString(Backend::lastKey)); - - log_.debug() << "Updating book successor " << ripple::strHex(base) << " - " - << ripple::strHex(Backend::lastKey); - } - } - } - - log_.debug() << "Inserted/modified/deleted all objects. Number of objects = " - << rawData.ledger_objects().objects_size(); - FormattedTransactionsData insertTxResult = insertTransactions(lgrInfo, rawData); - log_.debug() << "Inserted all transactions. Number of transactions = " - << rawData.transactions_list().transactions_size(); - backend_->writeAccountTransactions(std::move(insertTxResult.accountTxData)); - backend_->writeNFTs(std::move(insertTxResult.nfTokensData)); - backend_->writeNFTTransactions(std::move(insertTxResult.nfTokenTxData)); - log_.debug() << "wrote account_tx"; - - auto [success, duration] = - util::timed>([&]() { return backend_->finishWrites(lgrInfo.seq); }); - - log_.debug() << "Finished writes. took " << std::to_string(duration); - log_.debug() << "Finished ledger update. " << detail::toString(lgrInfo); - - return {lgrInfo, success}; -} - -// Database must be populated when this starts -std::optional -ReportingETL::runETLPipeline(uint32_t startSequence, int numExtractors) -{ - if (finishSequence_ && startSequence > *finishSequence_) - return {}; - - /* - * Behold, mortals! This function spawns three separate threads, which talk - * to each other via 2 different thread safe queues and 1 atomic variable. - * All threads and queues are function local. This function returns when all - * - * of the threads exit. There are two termination conditions: the first is - * if the load thread encounters a write conflict. In this case, the load - * thread sets writeConflict, an atomic bool, to true, which signals the - * other threads to stop. The second termination condition is when the - * entire server is shutting down, which is detected in one of three ways: - * 1. isStopping() returns true if the server is shutting down - * 2. networkValidatedLedgers_.waitUntilValidatedByNetwork returns - * false, signaling the wait was aborted. - * 3. fetchLedgerDataAndDiff returns an empty optional, signaling the fetch - * was aborted. - * In all cases, the extract thread detects this condition, - * and pushes an empty optional onto the transform queue. The transform - * thread, upon popping an empty optional, pushes an empty optional onto the - * load queue, and then returns. The load thread, upon popping an empty - * optional, returns. - */ - - log_.debug() << "Starting etl pipeline"; - writing_ = true; - - auto rng = backend_->hardFetchLedgerRangeNoThrow(); - if (!rng || rng->maxSequence < startSequence - 1) - { - assert(false); - throw std::runtime_error("runETLPipeline: parent ledger is null"); - } - std::atomic minSequence = rng->minSequence; - - std::atomic_bool writeConflict = false; - std::optional lastPublishedSequence; - uint32_t maxQueueSize = 1000 / numExtractors; - auto begin = std::chrono::system_clock::now(); - using QueueType = ThreadSafeQueue>; - std::vector> queues; - - auto getNext = [&queues, &startSequence, &numExtractors](uint32_t sequence) -> std::shared_ptr { - return queues[(sequence - startSequence) % numExtractors]; - }; - std::vector extractors; - for (size_t i = 0; i < numExtractors; ++i) - { - auto transformQueue = std::make_shared(maxQueueSize); - queues.push_back(transformQueue); - - extractors.emplace_back([this, &startSequence, &writeConflict, transformQueue, i, numExtractors]() { - beast::setCurrentThreadName("rippled: ReportingETL extract"); - uint32_t currentSequence = startSequence + i; - - double totalTime = 0; - - // there are two stopping conditions here. - // First, if there is a write conflict in the load thread, the - // ETL mechanism should stop. The other stopping condition is if - // the entire server is shutting down. This can be detected in a - // variety of ways. See the comment at the top of the function - while ((!finishSequence_ || currentSequence <= *finishSequence_) && - networkValidatedLedgers_->waitUntilValidatedByNetwork(currentSequence) && !writeConflict && - !isStopping()) - { - auto [fetchResponse, time] = util::timed>( - [&]() { return fetchLedgerDataAndDiff(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, fetchLedgerDataAndDiff will keep trying to - // fetch the specified ledger until successful - if (!fetchResponse) - { - break; - } - auto tps = fetchResponse->transactions_list().transactions_size() / time; - - log_.info() << "Extract phase time = " << time << " . Extract phase tps = " << tps - << " . Avg extract time = " << totalTime / (currentSequence - startSequence + 1) - << " . thread num = " << i << " . seq = " << currentSequence; - - transformQueue->push(std::move(fetchResponse)); - currentSequence += numExtractors; - if (finishSequence_ && currentSequence > *finishSequence_) - break; - } - // empty optional tells the transformer to shut down - transformQueue->push({}); - }); - } - - std::thread transformer{[this, &minSequence, &writeConflict, &startSequence, &getNext, &lastPublishedSequence]() { - beast::setCurrentThreadName("rippled: ReportingETL transform"); - uint32_t currentSequence = startSequence; - - while (!writeConflict) - { - std::optional fetchResponse{getNext(currentSequence)->pop()}; - ++currentSequence; - // if fetchResponse is an empty optional, the extracter thread - // has stopped and the transformer should stop as well - if (!fetchResponse) - { - break; - } - if (isStopping()) - continue; - - auto numTxns = fetchResponse->transactions_list().transactions_size(); - auto numObjects = fetchResponse->ledger_objects().objects_size(); - auto start = std::chrono::system_clock::now(); - auto [lgrInfo, success] = buildNextLedger(*fetchResponse); - auto end = std::chrono::system_clock::now(); - - auto duration = ((end - start).count()) / 1000000000.0; - if (success) - log_.info() << "Load phase of etl : " - << "Successfully wrote ledger! Ledger info: " << detail::toString(lgrInfo) - << ". txn count = " << numTxns << ". object count = " << numObjects - << ". load time = " << duration << ". load txns per second = " << numTxns / duration - << ". load objs per second = " << numObjects / duration; - else - log_.error() << "Error writing ledger. " << detail::toString(lgrInfo); - // success is false if the ledger was already written - if (success) - { - boost::asio::post(publishStrand_, [this, lgrInfo = lgrInfo]() { publishLedger(lgrInfo); }); - - lastPublishedSequence = lgrInfo.seq; - } - writeConflict = !success; - // TODO move online delete logic to an admin RPC call - if (onlineDeleteInterval_ && !deleting_ && lgrInfo.seq - minSequence > *onlineDeleteInterval_) - { - deleting_ = true; - ioContext_.post([this, &minSequence]() { - log_.info() << "Running online delete"; - - Backend::synchronous([&](boost::asio::yield_context& yield) { - backend_->doOnlineDelete(*onlineDeleteInterval_, yield); - }); - - log_.info() << "Finished online delete"; - auto rng = backend_->fetchLedgerRange(); - minSequence = rng->minSequence; - deleting_ = false; - }); - } - } - }}; - - transformer.join(); - for (size_t i = 0; i < numExtractors; ++i) - { - // pop from each queue that might be blocked on a push - getNext(i)->tryPop(); - } - // wait for all of the extractors to stop - for (auto& t : extractors) - t.join(); - auto end = std::chrono::system_clock::now(); - log_.debug() << "Extracted and wrote " << *lastPublishedSequence - startSequence << " in " - << ((end - begin).count()) / 1000000000.0; - writing_ = false; - - log_.debug() << "Stopping etl pipeline"; - - return lastPublishedSequence; -} - -// main loop. The software begins monitoring the ledgers that are validated -// by the nework. 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 -ReportingETL::monitor() -{ - auto rng = backend_->hardFetchLedgerRangeNoThrow(); - if (!rng) - { - log_.info() << "Database is empty. Will download a ledger " - "from the network."; - std::optional ledger; - if (startSequence_) - { - log_.info() << "ledger sequence specified in config. " - << "Will begin ETL process starting with ledger " << *startSequence_; - ledger = loadInitialLedger(*startSequence_); - } - else - { - log_.info() << "Waiting for next ledger to be validated by network..."; - std::optional mostRecentValidated = networkValidatedLedgers_->getMostRecent(); - if (mostRecentValidated) - { - log_.info() << "Ledger " << *mostRecentValidated << " has been validated. " - << "Downloading..."; - ledger = loadInitialLedger(*mostRecentValidated); - } - else - { - log_.info() << "The wait for the next validated " - << "ledger has been aborted. " - << "Exiting monitor loop"; - return; - } - } - if (ledger) - rng = backend_->hardFetchLedgerRangeNoThrow(); - else - { - log_.error() << "Failed to load initial ledger. Exiting monitor loop"; - return; - } - } - else - { - if (startSequence_) - { - log_.warn() << "start sequence specified but db is already populated"; - } - log_.info() << "Database already populated. Picking up from the tip of history"; - loadCache(rng->maxSequence); - } - assert(rng); - uint32_t nextSequence = rng->maxSequence + 1; - - log_.debug() << "Database is populated. " - << "Starting monitor loop. sequence = " << nextSequence; - while (true) - { - if (auto rng = backend_->hardFetchLedgerRangeNoThrow(); rng && rng->maxSequence >= nextSequence) - { - publishLedger(nextSequence, {}); - ++nextSequence; - } - else if (networkValidatedLedgers_->waitUntilValidatedByNetwork(nextSequence, 1000)) - { - 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 timeoutSeconds = 10; - bool success = publishLedger(nextSequence, timeoutSeconds); - if (!success) - { - log_.warn() << "Failed to publish ledger with sequence = " << nextSequence << " . Beginning ETL"; - // doContinousETLPipelined returns the most recent sequence - // published empty optional if no sequence was published - std::optional lastPublished = runETLPipeline(nextSequence, extractorThreads_); - log_.info() << "Aborting ETL. Falling back to publishing"; - // if no ledger was published, don't increment nextSequence - if (lastPublished) - nextSequence = *lastPublished + 1; - } - else - ++nextSequence; - } - } -} -bool -ReportingETL::loadCacheFromClioPeer( - uint32_t ledgerIndex, - std::string const& ip, - std::string const& port, - boost::asio::yield_context& yield) -{ - log_.info() << "Loading cache from peer. ip = " << ip << " . port = " << port; - namespace beast = boost::beast; // from - namespace http = beast::http; // from - namespace websocket = beast::websocket; // from - namespace net = boost::asio; // from - using tcp = boost::asio::ip::tcp; // from - try - { - boost::beast::error_code ec; - // These objects perform our I/O - tcp::resolver resolver{ioContext_}; - - log_.trace() << "Creating websocket"; - auto ws = std::make_unique>(ioContext_); - - // Look up the domain name - auto const results = resolver.async_resolve(ip, port, yield[ec]); - if (ec) - return {}; - - log_.trace() << "Connecting websocket"; - // Make the connection on the IP address we get from a lookup - ws->next_layer().async_connect(results, yield[ec]); - if (ec) - return false; - - log_.trace() << "Performing websocket handshake"; - // Perform the websocket handshake - ws->async_handshake(ip, "/", yield[ec]); - if (ec) - return false; - - std::optional marker; - - log_.trace() << "Sending request"; - auto getRequest = [&](auto marker) { - boost::json::object request = { - {"command", "ledger_data"}, - {"ledger_index", ledgerIndex}, - {"binary", true}, - {"out_of_order", true}, - {"limit", 2048}}; - - if (marker) - request["marker"] = *marker; - return request; - }; - - bool started = false; - size_t numAttempts = 0; - do - { - // Send the message - ws->async_write(net::buffer(boost::json::serialize(getRequest(marker))), yield[ec]); - if (ec) - { - log_.error() << "error writing = " << ec.message(); - return false; - } - - beast::flat_buffer buffer; - ws->async_read(buffer, yield[ec]); - if (ec) - { - log_.error() << "error reading = " << ec.message(); - return false; - } - - auto raw = beast::buffers_to_string(buffer.data()); - auto parsed = boost::json::parse(raw); - - if (!parsed.is_object()) - { - log_.error() << "Error parsing response: " << raw; - return false; - } - log_.trace() << "Successfully parsed response " << parsed; - - if (auto const& response = parsed.as_object(); response.contains("error")) - { - log_.error() << "Response contains error: " << response; - auto const& err = response.at("error"); - if (err.is_string() && err.as_string() == "lgrNotFound") - { - ++numAttempts; - if (numAttempts >= 5) - { - log_.error() << " ledger not found at peer after 5 attempts. " - "peer = " - << ip << " ledger = " << ledgerIndex - << ". Check your config and the health of the peer"; - return false; - } - log_.warn() << "Ledger not found. ledger = " << ledgerIndex << ". Sleeping and trying again"; - std::this_thread::sleep_for(std::chrono::seconds(1)); - continue; - } - return false; - } - started = true; - auto const& response = parsed.as_object()["result"].as_object(); - - if (!response.contains("cache_full") || !response.at("cache_full").as_bool()) - { - log_.error() << "cache not full for clio node. ip = " << ip; - return false; - } - if (response.contains("marker")) - marker = response.at("marker"); - else - marker = {}; - - auto const& state = response.at("state").as_array(); - - std::vector objects; - objects.reserve(state.size()); - for (auto const& ledgerObject : state) - { - auto const& obj = ledgerObject.as_object(); - - Backend::LedgerObject stateObject = {}; - - if (!stateObject.key.parseHex(obj.at("index").as_string().c_str())) - { - log_.error() << "failed to parse object id"; - return false; - } - boost::algorithm::unhex(obj.at("data").as_string().c_str(), std::back_inserter(stateObject.blob)); - objects.push_back(std::move(stateObject)); - } - backend_->cache().update(objects, ledgerIndex, true); - - if (marker) - log_.debug() << "At marker " << *marker; - } while (marker || !started); - - log_.info() << "Finished downloading ledger from clio node. ip = " << ip; - - backend_->cache().setFull(); - return true; - } - catch (std::exception const& e) - { - log_.error() << "Encountered exception : " << e.what() << " - ip = " << ip; - return false; - } -} - -void -ReportingETL::loadCache(uint32_t seq) -{ - if (cacheLoadStyle_ == CacheLoadStyle::NOT_AT_ALL) - { - backend_->cache().setDisabled(); - log_.warn() << "Cache is disabled. Not loading"; - return; - } - // sanity check to make sure we are not calling this multiple times - static std::atomic_bool loading = false; - if (loading) - { - assert(false); - return; - } - loading = true; - if (backend_->cache().isFull()) - { - assert(false); - return; - } - - if (clioPeers.size() > 0) - { - boost::asio::spawn(ioContext_, [this, seq](boost::asio::yield_context yield) { - for (auto const& peer : clioPeers) - { - // returns true on success - if (loadCacheFromClioPeer(seq, peer.ip, std::to_string(peer.port), yield)) - return; - } - // if we couldn't successfully load from any peers, load from db - loadCacheFromDb(seq); - }); - return; - } - else - { - loadCacheFromDb(seq); - } - // If loading synchronously, poll cache until full - while (cacheLoadStyle_ == CacheLoadStyle::SYNC && !backend_->cache().isFull()) - { - log_.debug() << "Cache not full. Cache size = " << backend_->cache().size() << ". Sleeping ..."; - std::this_thread::sleep_for(std::chrono::seconds(10)); - log_.info() << "Cache is full. Cache size = " << backend_->cache().size(); - } -} - -void -ReportingETL::loadCacheFromDb(uint32_t seq) -{ - // sanity check to make sure we are not calling this multiple times - static std::atomic_bool loading = false; - if (loading) - { - assert(false); - return; - } - loading = true; - std::vector diff; - auto append = [](auto&& a, auto&& b) { a.insert(std::end(a), std::begin(b), std::end(b)); }; - - for (size_t i = 0; i < numCacheDiffs_; ++i) - { - append(diff, Backend::synchronousAndRetryOnTimeout([&](auto yield) { - return backend_->fetchLedgerDiff(seq - i, yield); - })); - } - - std::sort(diff.begin(), diff.end(), [](auto a, auto b) { - return a.key < b.key || (a.key == b.key && a.blob.size() < b.blob.size()); - }); - diff.erase(std::unique(diff.begin(), diff.end(), [](auto a, auto b) { return a.key == b.key; }), diff.end()); - std::vector> cursors; - cursors.push_back({}); - for (auto& obj : diff) - { - if (obj.blob.size()) - cursors.push_back({obj.key}); - } - cursors.push_back({}); - std::stringstream cursorStr; - for (auto& c : cursors) - { - if (c) - cursorStr << ripple::strHex(*c) << ", "; - } - log_.info() << "Loading cache. num cursors = " << cursors.size() - 1; - log_.trace() << "cursors = " << cursorStr.str(); - - cacheDownloader_ = std::thread{[this, seq, cursors]() { - auto startTime = std::chrono::system_clock::now(); - auto markers = std::make_shared(0); - auto numRemaining = std::make_shared(cursors.size() - 1); - for (size_t i = 0; i < cursors.size() - 1; ++i) - { - std::optional start = cursors[i]; - std::optional end = cursors[i + 1]; - markers->wait(numCacheMarkers_); - ++(*markers); - boost::asio::spawn( - ioContext_, - [this, seq, start, end, numRemaining, startTime, markers](boost::asio::yield_context yield) { - std::optional cursor = start; - std::string cursorStr = - cursor.has_value() ? ripple::strHex(cursor.value()) : ripple::strHex(Backend::firstKey); - log_.debug() << "Starting a cursor: " << cursorStr << " markers = " << *markers; - - while (!stopping_) - { - auto res = Backend::retryOnTimeout([this, seq, &cursor, &yield]() { - return backend_->fetchLedgerPage(cursor, seq, cachePageFetchSize_, false, yield); - }); - backend_->cache().update(res.objects, seq, true); - if (!res.cursor || (end && *(res.cursor) > *end)) - break; - log_.trace() << "Loading cache. cache size = " << backend_->cache().size() - << " - cursor = " << ripple::strHex(res.cursor.value()) << " start = " << cursorStr - << " markers = " << *markers; - - cursor = std::move(res.cursor); - } - --(*markers); - markers->notify_one(); - if (--(*numRemaining) == 0) - { - auto endTime = std::chrono::system_clock::now(); - auto duration = std::chrono::duration_cast(endTime - startTime); - log_.info() << "Finished loading cache. cache size = " << backend_->cache().size() << ". Took " - << duration.count() << " seconds"; - backend_->cache().setFull(); - } - else - { - log_.info() << "Finished a cursor. num remaining = " << *numRemaining - << " start = " << cursorStr << " markers = " << *markers; - } - }); - } - }}; -} - -void -ReportingETL::monitorReadOnly() -{ - log_.debug() << "Starting reporting in strict read only mode"; - auto rng = backend_->hardFetchLedgerRangeNoThrow(); - uint32_t latestSequence; - if (!rng) - if (auto net = networkValidatedLedgers_->getMostRecent()) - latestSequence = *net; - else - return; - else - latestSequence = rng->maxSequence; - loadCache(latestSequence); - latestSequence++; - while (true) - { - if (auto rng = backend_->hardFetchLedgerRangeNoThrow(); rng && rng->maxSequence >= latestSequence) - { - publishLedger(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, 1000); - } -} - -void -ReportingETL::doWork() -{ - worker_ = std::thread([this]() { - beast::setCurrentThreadName("rippled: ReportingETL worker"); - if (readOnly_) - monitorReadOnly(); - else - monitor(); - }); -} - -ReportingETL::ReportingETL( - clio::Config const& config, - boost::asio::io_context& ioc, - std::shared_ptr backend, - std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr ledgers) - : backend_(backend) - , subscriptions_(subscriptions) - , loadBalancer_(balancer) - , ioContext_(ioc) - , publishStrand_(ioc) - , networkValidatedLedgers_(ledgers) -{ - startSequence_ = config.maybeValue("start_sequence"); - finishSequence_ = config.maybeValue("finish_sequence"); - readOnly_ = config.valueOr("read_only", readOnly_); - - if (auto interval = config.maybeValue("online_delete"); interval) - { - auto const max = std::numeric_limits::max(); - if (*interval > max) - { - std::stringstream msg; - msg << "online_delete cannot be greater than " << std::to_string(max); - throw std::runtime_error(msg.str()); - } - if (*interval > 0) - onlineDeleteInterval_ = *interval; - } - - extractorThreads_ = config.valueOr("extractor_threads", extractorThreads_); - txnThreshold_ = config.valueOr("txn_threshold", txnThreshold_); - if (config.contains("cache")) - { - auto const cache = config.section("cache"); - if (auto entry = cache.maybeValue("load"); entry) - { - if (boost::iequals(*entry, "sync")) - cacheLoadStyle_ = CacheLoadStyle::SYNC; - if (boost::iequals(*entry, "async")) - cacheLoadStyle_ = CacheLoadStyle::ASYNC; - if (boost::iequals(*entry, "none") or boost::iequals(*entry, "no")) - cacheLoadStyle_ = CacheLoadStyle::NOT_AT_ALL; - } - - numCacheDiffs_ = cache.valueOr("num_diffs", numCacheDiffs_); - numCacheMarkers_ = cache.valueOr("num_markers", numCacheMarkers_); - cachePageFetchSize_ = cache.valueOr("page_fetch_size", cachePageFetchSize_); - - if (auto peers = cache.maybeArray("peers"); peers) - { - for (auto const& peer : *peers) - { - auto ip = peer.value("ip"); - auto port = peer.value("port"); - - // todo: use emplace_back when clang is ready - clioPeers.push_back({ip, port}); - } - unsigned seed = std::chrono::system_clock::now().time_since_epoch().count(); - - std::shuffle(clioPeers.begin(), clioPeers.end(), std::default_random_engine(seed)); - } - } -} diff --git a/src/etl/ReportingETL.h b/src/etl/ReportingETL.h deleted file mode 100644 index b4c0be79..00000000 --- a/src/etl/ReportingETL.h +++ /dev/null @@ -1,402 +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 -#include -#include -#include -#include -#include -#include -#include -#include - -#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" -#include - -#include -#include -#include - -#include - -struct AccountTransactionsData; -struct NFTTransactionsData; -struct NFTsData; -struct FormattedTransactionsData -{ - std::vector accountTxData; - std::vector nfTokenTxData; - std::vector nfTokensData; -}; -class SubscriptionManager; - -/** - * 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 ReportingETL -{ -private: - clio::Logger log_{"ETL"}; - - std::shared_ptr backend_; - std::shared_ptr subscriptions_; - std::shared_ptr loadBalancer_; - std::optional onlineDeleteInterval_; - std::uint32_t extractorThreads_ = 1; - - enum class CacheLoadStyle { ASYNC, SYNC, NOT_AT_ALL }; - - CacheLoadStyle cacheLoadStyle_ = CacheLoadStyle::ASYNC; - - // number of diffs to use to generate cursors to traverse the ledger in - // parallel during initial cache download - size_t numCacheDiffs_ = 32; - // number of markers to use at one time to traverse the ledger in parallel - // during initial cache download - size_t numCacheMarkers_ = 48; - // number of ledger objects to fetch concurrently per marker during cache - // download - size_t cachePageFetchSize_ = 512; - // thread responsible for syncing the cache on startup - std::thread cacheDownloader_; - - struct ClioPeer - { - std::string ip; - int port; - }; - - std::vector clioPeers; - - std::thread worker_; - boost::asio::io_context& ioContext_; - - /// Strand to ensure that ledgers are published in order. - /// 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 - boost::asio::io_context::strand publishStrand_; - - /// Mechanism for communicating with ETL sources. ETLLoadBalancer wraps an - /// arbitrary number of ETL sources and load balances ETL requests across - /// those sources. - - /// Mechanism for detecting when the network has validated a new ledger. - /// This class provides a way to wait for a specific ledger to be validated - std::shared_ptr networkValidatedLedgers_; - - /// Whether the software is stopping - std::atomic_bool stopping_ = false; - /// Whether the software is performing online delete - // TODO this needs to live in the database, so diff servers can coordinate - // deletion - std::atomic_bool deleting_ = false; - - /// This variable controls the number of GetLedgerData calls that will be - /// executed in parallel during the initial ledger download. GetLedgerData - /// allows clients to page through a ledger over many RPC calls. - /// GetLedgerData returns a marker that is used as an offset in a subsequent - /// call. If numMarkers_ is greater than 1, there will be multiple chains of - /// GetLedgerData calls iterating over different parts of the same ledger in - /// parallel. This can dramatically speed up the time to download the - /// initial ledger. However, a higher value for this member variable puts - /// more load on the ETL source. - size_t numMarkers_ = 2; - - /// Whether the process is in strict read-only mode. In strict read-only - /// mode, the process will never attempt to become the ETL writer, and will - /// only publish ledgers as they are written to the database. - bool readOnly_ = false; - - /// Whether the process is writing to the database. Used by server_info - std::atomic_bool writing_ = false; - - /// Ledger sequence to start ETL from. If this is empty, ETL will start from - /// the next ledger validated by the network. If this is set, and the - /// database is already populated, an error is thrown. - std::optional startSequence_; - std::optional finishSequence_; - - size_t txnThreshold_ = 0; - - /// The time that the most recently published ledger was published. Used by - /// server_info - std::chrono::time_point lastPublish_; - - mutable std::shared_mutex publishTimeMtx_; - - void - setLastPublish() - { - std::scoped_lock lck(publishTimeMtx_); - lastPublish_ = std::chrono::system_clock::now(); - } - - /// The time that the most recently published ledger was closed. - std::chrono::time_point lastCloseTime_; - - mutable std::shared_mutex closeTimeMtx_; - - void - setLastClose(std::chrono::time_point lastCloseTime) - { - std::scoped_lock lck(closeTimeMtx_); - lastCloseTime_ = lastCloseTime; - } - - /// Download a ledger with specified sequence in full, via GetLedgerData, - /// and write the data to the databases. 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 - loadInitialLedger(uint32_t sequence); - - /// Populates the cache by walking through the given ledger. Should only be - /// called once. The default behavior is to return immediately and populate - /// the cache in the background. This can be overridden via config - /// parameter, to populate synchronously, or not at all - void - loadCache(uint32_t seq); - - void - loadCacheFromDb(uint32_t seq); - - bool - loadCacheFromClioPeer( - uint32_t ledgerSequence, - std::string const& ip, - std::string const& port, - boost::asio::yield_context& yield); - - /// Run ETL. 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 - /// @return the last ledger written to the database, if any - std::optional - runETLPipeline(uint32_t startSequence, int offset); - - /// 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(); - - /// 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(); - - /// Extract data for a particular ledger from an ETL source. This function - /// continously tries to extract the specified ledger (using all available - /// ETL sources) until the extraction succeeds, or the server shuts down. - /// @param sequence sequence of the ledger to extract - /// @return ledger header and transaction+metadata blobs. Empty optional - /// if the server is shutting down - std::optional - fetchLedgerData(uint32_t sequence); - - /// Extract data for a particular ledger from an ETL source. This function - /// continously tries to extract the specified ledger (using all available - /// ETL sources) until the extraction succeeds, or the server shuts down. - /// @param sequence sequence of the ledger to extract - /// @return ledger header, transaction+metadata blobs, and all ledger - /// objects created, modified or deleted between this ledger and the parent. - /// Empty optional if the server is shutting down - std::optional - fetchLedgerDataAndDiff(uint32_t sequence); - - /// Insert all of the extracted transactions into the ledger, returning - /// transactions related to accounts, transactions related to NFTs, and - /// NFTs themselves for later processsing. - /// @param ledger ledger to insert transactions into - /// @param data data extracted from an ETL source - /// @return struct that contains the neccessary info to write to the - /// account_transactions/account_tx and nft_token_transactions tables - /// (mostly transaction hashes, corresponding nodestore hashes and affected - /// accounts) - FormattedTransactionsData - insertTransactions(ripple::LedgerInfo const& ledger, org::xrpl::rpc::v1::GetLedgerResponse& data); - - // TODO update this documentation - /// Build the next ledger using the previous ledger and the extracted data. - /// This function calls insertTransactions() - /// @note rawData should be data that corresponds to the ledger immediately - /// following parent - /// @param parent the previous ledger - /// @param rawData data extracted from an ETL source - /// @return the newly built ledger and data to write to the database - std::pair - buildNextLedger(org::xrpl::rpc::v1::GetLedgerResponse& rawData); - - /// 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. 1 attempt per second - /// @return whether the ledger was found in the database and published - bool - publishLedger(uint32_t ledgerSequence, std::optional maxAttempts); - - /// Publish the passed in ledger - /// @param ledger the ledger to publish - void - publishLedger(ripple::LedgerInfo const& lgrInfo); - - bool - isStopping() - { - return stopping_; - } - - /// Get the number of markers to use during the initial ledger download. - /// This is equivelent to the degree of parallelism during the initial - /// ledger download - /// @return the number of markers - std::uint32_t - getNumMarkers() - { - return numMarkers_; - } - - /// start all of the necessary components and begin ETL - void - run() - { - log_.info() << "Starting reporting etl"; - stopping_ = false; - - doWork(); - } - - void - doWork(); - -public: - ReportingETL( - clio::Config const& config, - boost::asio::io_context& ioc, - std::shared_ptr backend, - std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr ledgers); - - static std::shared_ptr - make_ReportingETL( - clio::Config const& config, - boost::asio::io_context& ioc, - std::shared_ptr backend, - std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr ledgers) - { - auto etl = std::make_shared(config, ioc, backend, subscriptions, balancer, ledgers); - - etl->run(); - - return etl; - } - - ~ReportingETL() - { - log_.info() << "onStop called"; - log_.debug() << "Stopping Reporting ETL"; - stopping_ = true; - - if (worker_.joinable()) - worker_.join(); - if (cacheDownloader_.joinable()) - cacheDownloader_.join(); - - log_.debug() << "Joined ReportingETL worker thread"; - } - - boost::json::object - getInfo() const - { - boost::json::object result; - - result["etl_sources"] = loadBalancer_->toJson(); - result["is_writer"] = writing_.load(); - result["read_only"] = readOnly_; - auto last = getLastPublish(); - if (last.time_since_epoch().count() != 0) - result["last_publish_age_seconds"] = std::to_string(lastPublishAgeSeconds()); - return result; - } - - std::chrono::time_point - getLastPublish() const - { - std::shared_lock lck(publishTimeMtx_); - return lastPublish_; - } - - std::uint32_t - lastPublishAgeSeconds() const - { - return std::chrono::duration_cast(std::chrono::system_clock::now() - getLastPublish()) - .count(); - } - - std::uint32_t - lastCloseAgeSeconds() const - { - std::shared_lock lck(closeTimeMtx_); - auto now = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()) - .count(); - auto closeTime = lastCloseTime_.time_since_epoch().count(); - if (now < (rippleEpochStart + closeTime)) - return 0; - return now - (rippleEpochStart + closeTime); - } -}; diff --git a/src/etl/ETLSource.cpp b/src/etl/Source.cpp similarity index 70% rename from src/etl/ETLSource.cpp rename to src/etl/Source.cpp index 7409d787..b1c04984 100644 --- a/src/etl/ETLSource.cpp +++ b/src/etl/Source.cpp @@ -17,81 +17,27 @@ */ //============================================================================== +#include +#include +#include +#include +#include +#include +#include +#include +#include + #include #include #include #include #include #include -#include - -#include -#include -#include -#include -#include -#include -#include -#include #include using namespace clio; -void -ForwardCache::freshen() -{ - log_.trace() << "Freshening ForwardCache"; - - auto numOutstanding = std::make_shared(latestForwarded_.size()); - - for (auto const& cacheEntry : latestForwarded_) - { - boost::asio::spawn( - strand_, [this, numOutstanding, command = cacheEntry.first](boost::asio::yield_context yield) { - boost::json::object request = {{"command", command}}; - auto resp = source_.requestFromRippled(request, {}, yield); - - if (!resp || resp->contains("error")) - resp = {}; - - { - std::scoped_lock lk(mtx_); - latestForwarded_[command] = resp; - } - }); - } -} - -void -ForwardCache::clear() -{ - std::scoped_lock lk(mtx_); - for (auto& cacheEntry : latestForwarded_) - latestForwarded_[cacheEntry.first] = {}; -} - -std::optional -ForwardCache::get(boost::json::object const& request) const -{ - std::optional command = {}; - if (request.contains("command") && !request.contains("method") && request.at("command").is_string()) - command = request.at("command").as_string().c_str(); - else if (request.contains("method") && !request.contains("command") && request.at("method").is_string()) - command = request.at("method").as_string().c_str(); - - if (!command) - return {}; - if (RPC::specifiesCurrentOrClosedLedger(request)) - return {}; - - std::shared_lock lk(mtx_); - if (!latestForwarded_.contains(*command)) - return {}; - - return {latestForwarded_.at(*command)}; -} - static boost::beast::websocket::stream_base::timeout make_TimeoutOption() { @@ -116,7 +62,7 @@ make_TimeoutOption() template void -ETLSourceImpl::reconnect(boost::beast::error_code ec) +SourceImpl::reconnect(boost::beast::error_code ec) { if (paused_) return; @@ -162,7 +108,7 @@ ETLSourceImpl::reconnect(boost::beast::error_code ec) } void -PlainETLSource::close(bool startAgain) +PlainSource::close(bool startAgain) { timer_.cancel(); ioc_.post([this, startAgain]() { @@ -202,7 +148,7 @@ PlainETLSource::close(bool startAgain) } void -SslETLSource::close(bool startAgain) +SslSource::close(bool startAgain) { timer_.cancel(); ioc_.post([this, startAgain]() { @@ -244,7 +190,7 @@ SslETLSource::close(bool startAgain) template void -ETLSourceImpl::onResolve(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type results) +SourceImpl::onResolve(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type results) { log_.trace() << "ec = " << ec << " - " << toString(); if (ec) @@ -262,7 +208,7 @@ ETLSourceImpl::onResolve(boost::beast::error_code ec, boost::asio::ip:: } void -PlainETLSource::onConnect( +PlainSource::onConnect( boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint) { @@ -300,9 +246,7 @@ PlainETLSource::onConnect( } void -SslETLSource::onConnect( - boost::beast::error_code ec, - boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint) +SslSource::onConnect(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint) { log_.trace() << "ec = " << ec << " - " << toString(); if (ec) @@ -339,7 +283,7 @@ SslETLSource::onConnect( } void -SslETLSource::onSslHandshake( +SslSource::onSslHandshake( boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint) { @@ -358,10 +302,10 @@ SslETLSource::onSslHandshake( template void -ETLSourceImpl::onHandshake(boost::beast::error_code ec) +SourceImpl::onHandshake(boost::beast::error_code ec) { log_.trace() << "ec = " << ec << " - " << toString(); - if (auto action = hooks_.onConnected(ec); action == ETLSourceHooks::Action::STOP) + if (auto action = hooks_.onConnected(ec); action == SourceHooks::Action::STOP) return; if (ec) @@ -391,7 +335,7 @@ ETLSourceImpl::onHandshake(boost::beast::error_code ec) template void -ETLSourceImpl::onWrite(boost::beast::error_code ec, size_t bytesWritten) +SourceImpl::onWrite(boost::beast::error_code ec, size_t bytesWritten) { log_.trace() << "ec = " << ec << " - " << toString(); if (ec) @@ -407,7 +351,7 @@ ETLSourceImpl::onWrite(boost::beast::error_code ec, size_t bytesWritten template void -ETLSourceImpl::onRead(boost::beast::error_code ec, size_t size) +SourceImpl::onRead(boost::beast::error_code ec, size_t size) { log_.trace() << "ec = " << ec << " - " << toString(); // if error or error reading message, start over @@ -428,7 +372,7 @@ ETLSourceImpl::onRead(boost::beast::error_code ec, size_t size) template bool -ETLSourceImpl::handleMessage() +SourceImpl::handleMessage() { log_.trace() << toString(); @@ -544,13 +488,12 @@ public: assert(nextPrefix_ > prefix || nextPrefix_ == 0x00); cur_ = std::make_unique(); - next_ = std::make_unique(); - context_ = std::make_unique(); } enum class CallStatus { MORE, DONE, ERRORED }; + CallStatus process( std::unique_ptr& stub, @@ -661,26 +604,25 @@ public: }; template -bool -ETLSourceImpl::loadInitialLedger(uint32_t sequence, uint32_t numMarkers, bool cacheOnly) +std::pair, bool> +SourceImpl::loadInitialLedger(uint32_t sequence, uint32_t numMarkers, bool cacheOnly) { if (!stub_) - return false; + return {{}, false}; grpc::CompletionQueue cq; - void* tag; - bool ok = false; - std::vector calls; auto markers = getMarkers(numMarkers); for (size_t i = 0; i < markers.size(); ++i) { std::optional nextMarker; + if (i + 1 < markers.size()) nextMarker = markers[i + 1]; + calls.emplace_back(sequence, markers[i], nextMarker); } @@ -694,35 +636,37 @@ ETLSourceImpl::loadInitialLedger(uint32_t sequence, uint32_t numMarkers size_t incr = 500000; size_t progress = incr; std::vector edgeKeys; + while (numFinished < calls.size() && cq.Next(&tag, &ok)) { assert(tag); - auto ptr = static_cast(tag); if (!ok) { log_.error() << "loadInitialLedger - ok is false"; - return false; - // handle cancelled + return {{}, false}; // handle cancelled } else { log_.trace() << "Marker prefix = " << ptr->getMarkerPrefix(); + auto result = ptr->process(stub_, cq, *backend_, abort, cacheOnly); if (result != AsyncCallData::CallStatus::MORE) { numFinished++; log_.debug() << "Finished a marker. " << "Current number of finished = " << numFinished; + std::string lastKey = ptr->getLastKey(); + if (lastKey.size()) edgeKeys.push_back(ptr->getLastKey()); } + if (result == AsyncCallData::CallStatus::ERRORED) - { abort = true; - } + if (backend_->cache().size() > progress) { log_.info() << "Downloaded " << backend_->cache().size() << " records from rippled"; @@ -730,67 +674,14 @@ ETLSourceImpl::loadInitialLedger(uint32_t sequence, uint32_t numMarkers } } } + log_.info() << "Finished loadInitialLedger. cache size = " << backend_->cache().size(); - size_t numWrites = 0; - if (!abort) - { - backend_->cache().setFull(); - if (!cacheOnly) - { - auto seconds = util::timed([&]() { - for (auto& key : edgeKeys) - { - 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 = Backend::firstKey; - while (auto cur = backend_->cache().getSuccessor(prev, sequence)) - { - assert(cur); - if (prev == Backend::firstKey) - { - 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(cur->key, sequence)) - { - auto succ = backend_->cache().getSuccessor(base, sequence); - assert(succ); - if (succ->key == cur->key) - { - log_.debug() << "Writing book successor = " << ripple::strHex(base) << " - " - << ripple::strHex(cur->key); - - backend_->writeSuccessor(uint256ToString(base), sequence, uint256ToString(cur->key)); - } - } - ++numWrites; - } - prev = std::move(cur->key); - if (numWrites % 100000 == 0 && numWrites != 0) - log_.info() << "Wrote " << numWrites << " book successors"; - } - - backend_->writeSuccessor(uint256ToString(prev), sequence, uint256ToString(Backend::lastKey)); - - ++numWrites; - }); - log_.info() << "Looping through cache and submitting all writes took " << seconds - << " seconds. numWrites = " << std::to_string(numWrites); - } - } - return !abort; + return {std::move(edgeKeys), !abort}; } template std::pair -ETLSourceImpl::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors) +SourceImpl::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors) { org::xrpl::rpc::v1::GetLedgerResponse response; if (!stub_) @@ -808,7 +699,7 @@ ETLSourceImpl::fetchLedger(uint32_t ledgerSequence, bool getObjects, bo grpc::Status status = stub_->GetLedger(&context, request, &response); if (status.ok() && !response.is_unlimited()) { - log_.warn() << "ETLSourceImpl::fetchLedger - is_unlimited is " + log_.warn() << "SourceImpl::fetchLedger - is_unlimited is " "false. Make sure secure_gateway is set " "correctly on the ETL source. source = " << toString() << " status = " << status.error_message(); @@ -816,112 +707,9 @@ ETLSourceImpl::fetchLedger(uint32_t ledgerSequence, bool getObjects, bo return {status, std::move(response)}; } -static std::unique_ptr -make_ETLSource( - clio::Config const& config, - boost::asio::io_context& ioContext, - std::shared_ptr backend, - std::shared_ptr subscriptions, - std::shared_ptr networkValidatedLedgers, - ETLLoadBalancer& balancer) -{ - auto src = std::make_unique( - config, ioContext, backend, subscriptions, networkValidatedLedgers, balancer); - - src->run(); - - return src; -} - -ETLLoadBalancer::ETLLoadBalancer( - clio::Config const& config, - boost::asio::io_context& ioContext, - std::shared_ptr backend, - std::shared_ptr subscriptions, - std::shared_ptr nwvl) -{ - if (auto value = config.maybeValue("num_markers"); value) - downloadRanges_ = std::clamp(*value, 1u, 256u); - else if (backend->fetchLedgerRange()) - downloadRanges_ = 4; - - for (auto const& entry : config.array("etl_sources")) - { - std::unique_ptr source = make_ETLSource(entry, ioContext, backend, subscriptions, nwvl, *this); - - sources_.push_back(std::move(source)); - log_.info() << "Added etl source - " << sources_.back()->toString(); - } -} - -void -ETLLoadBalancer::loadInitialLedger(uint32_t sequence, bool cacheOnly) -{ - execute( - [this, &sequence, cacheOnly](auto& source) { - bool res = source->loadInitialLedger(sequence, downloadRanges_, cacheOnly); - if (!res) - { - log_.error() << "Failed to download initial ledger." - << " Sequence = " << sequence << " source = " << source->toString(); - } - return res; - }, - sequence); -} - -std::optional -ETLLoadBalancer::fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors) -{ - org::xrpl::rpc::v1::GetLedgerResponse response; - bool success = 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.info() << "Successfully fetched ledger = " << ledgerSequence - << " from source = " << source->toString(); - return true; - } - else - { - 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); - if (success) - return response; - else - return {}; -} - -std::optional -ETLLoadBalancer::forwardToRippled( - boost::json::object const& request, - std::string const& clientIp, - boost::asio::yield_context& yield) const -{ - srand((unsigned)time(0)); - auto sourceIdx = rand() % sources_.size(); - auto numAttempts = 0; - while (numAttempts < sources_.size()) - { - if (auto res = sources_[sourceIdx]->forwardToRippled(request, clientIp, yield)) - return res; - - sourceIdx = (sourceIdx + 1) % sources_.size(); - ++numAttempts; - } - return {}; -} - template std::optional -ETLSourceImpl::forwardToRippled( +SourceImpl::forwardToRippled( boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context& yield) const @@ -937,7 +725,7 @@ ETLSourceImpl::forwardToRippled( template std::optional -ETLSourceImpl::requestFromRippled( +SourceImpl::requestFromRippled( boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context& yield) const @@ -1028,53 +816,3 @@ ETLSourceImpl::requestFromRippled( return {}; } } - -template -bool -ETLLoadBalancer::execute(Func f, uint32_t ledgerSequence) -{ - srand((unsigned)time(0)); - auto sourceIdx = rand() % sources_.size(); - auto numAttempts = 0; - - while (true) - { - auto& source = sources_[sourceIdx]; - - 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 res = f(source); - if (res) - { - log_.debug() << "Successfully executed func at source = " << source->toString() - << " - ledger sequence = " << ledgerSequence; - break; - } - else - { - log_.warn() << "Failed to execute func at source = " << source->toString() - << " - ledger sequence = " << ledgerSequence; - } - } - else - { - log_.warn() << "Ledger not present at source = " << source->toString() - << " - ledger sequence = " << ledgerSequence; - } - sourceIdx = (sourceIdx + 1) % sources_.size(); - numAttempts++; - if (numAttempts % sources_.size() == 0) - { - log_.info() << "Ledger sequence " << ledgerSequence << " is not yet available from any configured sources. " - << "Sleeping and trying again"; - std::this_thread::sleep_for(std::chrono::seconds(2)); - } - } - return true; -} diff --git a/src/etl/ETLSource.h b/src/etl/Source.h similarity index 57% rename from src/etl/ETLSource.h rename to src/etl/Source.h index 199ccb27..a0ea61a1 100644 --- a/src/etl/ETLSource.h +++ b/src/etl/Source.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -37,67 +38,21 @@ #include #include -class ETLLoadBalancer; -class ETLSource; -class ProbingETLSource; +class LoadBalancer; +class Source; +class ProbingSource; class SubscriptionManager; -/// This class manages a connection to a single ETL source. This is almost -/// always a rippled node, but really could be another reporting node. This -/// class subscribes to the ledgers and transactions_proposed streams of the -/// associated rippled node, and keeps track of which ledgers the rippled node -/// has. This class also has methods for extracting said ledgers. Lastly this -/// class forwards transactions received on the transactions_proposed streams to -/// any subscribers. -class ForwardCache -{ - using response_type = std::optional; +// TODO: we use Source so that we can store a vector of Sources +// but we also use CRTP for implementation of the common logic - this is a bit strange because CRTP as used here is +// supposed to be used instead of an abstract base. +// Maybe we should rework this a bit. At this point there is not too much use in the CRTP implementation - we can move +// things into the base class instead. - clio::Logger log_{"ETL"}; - mutable std::atomic_bool stopping_ = false; - mutable std::shared_mutex mtx_; - std::unordered_map latestForwarded_; - - boost::asio::io_context::strand strand_; - boost::asio::steady_timer timer_; - ETLSource const& source_; - std::uint32_t duration_ = 10; - - void - clear(); - -public: - ForwardCache(clio::Config const& config, boost::asio::io_context& ioc, ETLSource const& source) - : strand_(ioc), timer_(strand_), source_(source) - { - if (config.contains("cache")) - { - auto commands = config.arrayOrThrow("cache", "ETLSource cache must be array"); - - if (config.contains("cache_duration")) - duration_ = - config.valueOrThrow("cache_duration", "ETLSource cache_duration must be a number"); - - for (auto const& command : commands) - { - auto key = command.valueOrThrow("ETLSource forward command must be array of strings"); - latestForwarded_[key] = {}; - } - } - } - - // This is to be called every freshenDuration_ seconds. - // It will request information from this etlSource, and - // will populate the cache with the latest value. If the - // request fails, it will evict that value from the cache. - void - freshen(); - - std::optional - get(boost::json::object const& command) const; -}; - -class ETLSource +/** + * @brief Base class for all ETL sources + */ +class Source { public: virtual bool @@ -124,7 +79,7 @@ public: virtual std::pair fetchLedger(uint32_t ledgerSequence, bool getObjects = true, bool getObjectNeighbors = false) = 0; - virtual bool + virtual std::pair, bool> loadInitialLedger(uint32_t sequence, std::uint32_t numMarkers, bool cacheOnly = false) = 0; virtual std::optional @@ -134,12 +89,10 @@ public: virtual boost::uuids::uuid token() const = 0; - virtual ~ETLSource() - { - } + virtual ~Source() = default; bool - operator==(ETLSource const& other) const + operator==(Source const& other) const { return token() == other.token(); } @@ -148,8 +101,8 @@ protected: clio::Logger log_{"ETL"}; private: - friend ForwardCache; - friend ProbingETLSource; + friend clio::detail::ForwardCache; + friend ProbingSource; virtual std::optional requestFromRippled( @@ -158,7 +111,10 @@ private: boost::asio::yield_context& yield) const = 0; }; -struct ETLSourceHooks +/** + * @brief Hooks for source events such as connects and disconnects + */ +struct SourceHooks { enum class Action { STOP, PROCEED }; @@ -166,130 +122,68 @@ struct ETLSourceHooks std::function onDisconnected; }; +/** + * @brief Base implementation of shared source logic (using CRTP) + */ template -class ETLSourceImpl : public ETLSource +class SourceImpl : public Source { std::string wsPort_; - std::string grpcPort_; std::unique_ptr stub_; - boost::asio::ip::tcp::resolver resolver_; - boost::beast::flat_buffer readBuffer_; std::vector> validatedLedgers_; - std::string validatedLedgersRaw_{"N/A"}; - std::shared_ptr networkValidatedLedgers_; - // beast::Journal journal_; - mutable std::mutex mtx_; - std::atomic_bool connected_{false}; - // true if this ETL source is forwarding transactions received on the - // transactions_proposed stream. There are usually multiple ETL sources, - // so to avoid forwarding the same transaction multiple times, we only - // forward from one particular ETL source at a time. + // true if this ETL source is forwarding transactions received on the transactions_proposed stream. There are + // usually multiple ETL sources, so to avoid forwarding the same transaction multiple times, we only forward from + // one particular ETL source at a time. std::atomic_bool forwardingStream_{false}; - // The last time a message was received on the ledgers stream std::chrono::system_clock::time_point lastMsgTime_; mutable std::mutex lastMsgTimeMtx_; std::shared_ptr backend_; std::shared_ptr subscriptions_; - ETLLoadBalancer& balancer_; + LoadBalancer& balancer_; - ForwardCache forwardCache_; + clio::detail::ForwardCache forwardCache_; boost::uuids::uuid uuid_; - std::optional - requestFromRippled( - boost::json::object const& request, - std::string const& clientIp, - boost::asio::yield_context& yield) const override; - protected: - Derived& - derived() - { - return static_cast(*this); - } - std::string ip_; - size_t numFailures_ = 0; boost::asio::io_context& ioc_; - - // used for retrying connections boost::asio::steady_timer timer_; std::atomic_bool closing_{false}; - std::atomic_bool paused_{false}; - ETLSourceHooks hooks_; - - void - run() override - { - log_.trace() << toString(); - - auto const host = ip_; - auto const port = wsPort_; - - resolver_.async_resolve(host, port, [this](auto ec, auto results) { onResolve(ec, results); }); - } + SourceHooks hooks_; public: - ~ETLSourceImpl() - { - derived().close(false); - } - - bool - isConnected() const override - { - return connected_; - } - - boost::uuids::uuid - token() const override - { - return uuid_; - } - - std::chrono::system_clock::time_point - getLastMsgTime() const - { - std::lock_guard lck(lastMsgTimeMtx_); - return lastMsgTime_; - } - - void - setLastMsgTime() - { - std::lock_guard lck(lastMsgTimeMtx_); - lastMsgTime_ = std::chrono::system_clock::now(); - } - - /// Create ETL source without gRPC endpoint - /// Fetch ledger and load initial ledger will fail for this source - /// Primarly used in read-only mode, to monitor when ledgers are validated - ETLSourceImpl( + /** + * @brief Create ETL source without gRPC endpoint + * + * Fetch ledger and load initial ledger will fail for this source. + * Primarly used in read-only mode, to monitor when ledgers are validated. + */ + SourceImpl( clio::Config const& config, boost::asio::io_context& ioContext, std::shared_ptr backend, std::shared_ptr subscriptions, std::shared_ptr networkValidatedLedgers, - ETLLoadBalancer& balancer, - ETLSourceHooks hooks) + LoadBalancer& balancer, + SourceHooks hooks) : resolver_(boost::asio::make_strand(ioContext)) , networkValidatedLedgers_(networkValidatedLedgers) , backend_(backend) @@ -327,8 +221,47 @@ public: } } - /// @param sequence ledger sequence to check for - /// @return true if this source has the desired ledger + ~SourceImpl() + { + derived().close(false); + } + + bool + isConnected() const override + { + return connected_; + } + + boost::uuids::uuid + token() const override + { + return uuid_; + } + + std::chrono::system_clock::time_point + getLastMsgTime() const + { + std::lock_guard lck(lastMsgTimeMtx_); + return lastMsgTime_; + } + + void + setLastMsgTime() + { + std::lock_guard lck(lastMsgTimeMtx_); + lastMsgTime_ = std::chrono::system_clock::now(); + } + + std::optional + requestFromRippled( + boost::json::object const& request, + std::string const& clientIp, + boost::asio::yield_context& yield) const override; + + /** + * @param sequence ledger sequence to check for + * @return true if this source has the desired ledger + */ bool hasLedger(uint32_t sequence) const override { @@ -350,9 +283,11 @@ public: return false; } - /// process the validated range received on the ledgers stream. set the - /// appropriate member variable - /// @param range validated range received on ledgers stream + /** + * @brief Process the validated range received on the ledgers stream. set the appropriate member variable + * + * @param range validated range received on ledgers stream + */ void setValidatedRange(std::string const& range) { @@ -386,8 +321,10 @@ public: validatedLedgersRaw_ = range; } - /// @return the validated range of this source - /// @note this is only used by server_info + /** + * @return the validated range of this source + * @note this is only used by server_info + */ std::string getValidatedRange() const { @@ -395,14 +332,19 @@ public: return validatedLedgersRaw_; } - /// Fetch the specified ledger - /// @param ledgerSequence sequence of the ledger to fetch - /// @getObjects whether to get the account state diff between this ledger - /// and the prior one - /// @return the extracted data and the result status + /** + * @brief Fetch the specified ledger + * + * @param ledgerSequence sequence of the ledger to fetch @getObjects whether to get the account state diff between + * this ledger and the prior one + * @return the extracted data and the result status + */ std::pair fetchLedger(uint32_t ledgerSequence, bool getObjects = true, bool getObjectNeighbors = false) override; + /** + * @brief Produces a human-readable string with info about the source + */ std::string toString() const override { @@ -410,35 +352,49 @@ public: ", grpc port: " + grpcPort_ + "}"; } + /** + * @brief Produces stats for this source in a json object + * @return json object with stats + */ boost::json::object toJson() const override { boost::json::object res; + res["validated_range"] = getValidatedRange(); res["is_connected"] = std::to_string(isConnected()); res["ip"] = ip_; res["ws_port"] = wsPort_; res["grpc_port"] = grpcPort_; + auto last = getLastMsgTime(); if (last.time_since_epoch().count() != 0) res["last_msg_age_seconds"] = std::to_string( std::chrono::duration_cast(std::chrono::system_clock::now() - getLastMsgTime()) .count()); + return res; } - /// Download a ledger in full - /// @param ledgerSequence sequence of the ledger to download - /// @param writeQueue queue to push downloaded ledger objects - /// @return true if the download was successful - bool + /** + * @brief Download a ledger in full + * + * @param ledgerSequence sequence of the ledger to download + * @param writeQueue queue to push downloaded ledger objects + * @return true if the download was successful + */ + std::pair, bool> loadInitialLedger(std::uint32_t ledgerSequence, std::uint32_t numMarkers, bool cacheOnly = false) override; - /// Attempt to reconnect to the ETL source + /** + * @brief Attempt to reconnect to the ETL source + */ void reconnect(boost::beast::error_code ec); - /// Pause the source effectively stopping it from trying to reconnect + /** + * @brief Pause the source effectively stopping it from trying to reconnect + */ void pause() override { @@ -446,7 +402,9 @@ public: derived().close(false); } - /// Resume the source allowing it to reconnect again + /** + * @brief Resume the source allowing it to reconnect again + */ void resume() override { @@ -454,61 +412,100 @@ public: derived().close(true); } - /// Callback + /** + * @brief Callback for resolving the server host + */ void onResolve(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type results); - /// Callback + /** + * @brief Callback for connection to the server + */ virtual void onConnect(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint) = 0; - /// Callback + /** + * @brief Callback for handshake with the server + */ void onHandshake(boost::beast::error_code ec); - /// Callback + /** + * @brief Callback for writing data + */ void onWrite(boost::beast::error_code ec, size_t size); - /// Callback + /** + * @brief Callback for data available to read + */ void onRead(boost::beast::error_code ec, size_t size); - /// Handle the most recently received message - /// @return true if the message was handled successfully. false on error + /** + * @brief Handle the most recently received message + * @return true if the message was handled successfully. false on error + */ bool handleMessage(); + /** + * @brief Forward a request to rippled + * @return response wrapped in an optional on success; nullopt otherwise + */ std::optional forwardToRippled(boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context& yield) const override; + +protected: + Derived& + derived() + { + return static_cast(*this); + } + + void + run() override + { + log_.trace() << toString(); + + auto const host = ip_; + auto const port = wsPort_; + + resolver_.async_resolve(host, port, [this](auto ec, auto results) { onResolve(ec, results); }); + } }; -class PlainETLSource : public ETLSourceImpl +class PlainSource : public SourceImpl { std::unique_ptr> ws_; public: - PlainETLSource( + PlainSource( clio::Config const& config, boost::asio::io_context& ioc, std::shared_ptr backend, std::shared_ptr subscriptions, std::shared_ptr nwvl, - ETLLoadBalancer& balancer, - ETLSourceHooks hooks) - : ETLSourceImpl(config, ioc, backend, subscriptions, nwvl, balancer, std::move(hooks)) + LoadBalancer& balancer, + SourceHooks hooks) + : SourceImpl(config, ioc, backend, subscriptions, nwvl, balancer, std::move(hooks)) , ws_(std::make_unique>( boost::asio::make_strand(ioc))) { } + /** + * @brief Callback for connection to the server + */ void onConnect(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint) override; - /// Close the websocket - /// @param startAgain whether to reconnect + /** + * @brief Close the websocket + * @param startAgain whether to reconnect + */ void close(bool startAgain); @@ -519,23 +516,23 @@ public: } }; -class SslETLSource : public ETLSourceImpl +class SslSource : public SourceImpl { std::optional> sslCtx_; std::unique_ptr>> ws_; public: - SslETLSource( + SslSource( clio::Config const& config, boost::asio::io_context& ioc, std::optional> sslCtx, std::shared_ptr backend, std::shared_ptr subscriptions, std::shared_ptr nwvl, - ETLLoadBalancer& balancer, - ETLSourceHooks hooks) - : ETLSourceImpl(config, ioc, backend, subscriptions, nwvl, balancer, std::move(hooks)) + LoadBalancer& balancer, + SourceHooks hooks) + : SourceImpl(config, ioc, backend, subscriptions, nwvl, balancer, std::move(hooks)) , sslCtx_(sslCtx) , ws_(std::make_unique>>( boost::asio::make_strand(ioc_), @@ -543,15 +540,23 @@ public: { } + /** + * @brief Callback for connection to the server + */ void onConnect(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint) override; + /** + * @brief Callback for SSL handshake completion + */ void onSslHandshake(boost::beast::error_code ec, boost::asio::ip::tcp::resolver::results_type::endpoint_type endpoint); - /// Close the websocket - /// @param startAgain whether to reconnect + /** + * @brief Close the websocket + * @param startAgain whether to reconnect + */ void close(bool startAgain); @@ -561,115 +566,3 @@ public: return *ws_; } }; - -/// 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 ETLLoadBalancer -{ -private: - clio::Logger log_{"ETL"}; - std::vector> sources_; - std::uint32_t downloadRanges_ = 16; - -public: - ETLLoadBalancer( - clio::Config const& config, - boost::asio::io_context& ioContext, - std::shared_ptr backend, - std::shared_ptr subscriptions, - std::shared_ptr nwvl); - - static std::shared_ptr - make_ETLLoadBalancer( - clio::Config const& config, - boost::asio::io_context& ioc, - std::shared_ptr backend, - std::shared_ptr subscriptions, - std::shared_ptr validatedLedgers) - { - return std::make_shared(config, ioc, backend, subscriptions, validatedLedgers); - } - - ~ETLLoadBalancer() - { - sources_.clear(); - } - - /// Load the initial ledger, writing data to the queue - /// @param sequence sequence of ledger to download - void - loadInitialLedger(uint32_t sequence, bool cacheOnly = false); - - /// 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 ledger to fetch data for - /// @param getObjects if true, fetch diff between specified ledger and - /// previous - /// @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 - std::optional - fetchLedger(uint32_t ledgerSequence, bool getObjects, bool getObjectNeighbors); - - /// Determine whether messages received on the transactions_proposed stream - /// should be forwarded to subscribing clients. The server subscribes to - /// transactions_proposed on multiple ETLSources, yet only forwards messages - /// from one source at any given time (to avoid sending duplicate messages - /// to clients). - /// @param in ETLSource in question - /// @return true if messages should be forwarded - bool - shouldPropagateTxnStream(ETLSource* in) const - { - for (auto& src : sources_) - { - assert(src); - // We pick the first ETLSource encountered that is connected - if (src->isConnected()) - { - return *src == *in; - } - } - - // If no sources connected, then this stream has not been forwarded - return true; - } - - boost::json::value - toJson() const - { - boost::json::array ret; - for (auto& src : sources_) - { - ret.push_back(src->toJson()); - } - return ret; - } - - /// Forward a JSON RPC request to a randomly selected rippled node - /// @param request JSON-RPC request - /// @return response received from rippled node - std::optional - forwardToRippled(boost::json::object const& request, std::string const& clientIp, boost::asio::yield_context& yield) - const; - -private: - /// f is a function that takes an ETLSource as an argument and returns a - /// bool. Attempt to execute f for one randomly chosen ETLSource that has - /// the specified ledger. If f returns false, another randomly chosen - /// ETLSource 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 ETLSource that has this - /// ledger - /// @return true if f was eventually executed successfully. false if the - /// ledger was found in the database or the server is shutting down - template - bool - execute(Func f, uint32_t ledgerSequence); -}; diff --git a/src/etl/SystemState.h b/src/etl/SystemState.h new file mode 100644 index 00000000..f9ac6b22 --- /dev/null +++ b/src/etl/SystemState.h @@ -0,0 +1,50 @@ +//------------------------------------------------------------------------------ +/* + 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 + +struct SystemState +{ + /** + * @brief Whether the process is in strict read-only mode + * + * In strict read-only mode, the process will never attempt to become the ETL writer, and will only publish ledgers + * as they are written to the database. + */ + bool isReadOnly = false; + + /** + * @brief Whether the process is writing to the database. + * + * Used by server_info + */ + std::atomic_bool isWriting = false; + + /** + * @brief Whether the software is stopping + */ + std::atomic_bool isStopping = false; + + /** + * @brief Whether a write conflict was detected + */ + std::atomic_bool writeConflict = false; +}; diff --git a/src/etl/impl/CacheLoader.h b/src/etl/impl/CacheLoader.h new file mode 100644 index 00000000..f4730ec3 --- /dev/null +++ b/src/etl/impl/CacheLoader.h @@ -0,0 +1,434 @@ +//------------------------------------------------------------------------------ +/* + 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 + +#include +#include +#include +#include +#include +#include +#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" +#include + +#include +#include +#include + +namespace clio::detail { + +/** + * @brief Cache loading interface + */ +template +class CacheLoader +{ + enum class LoadStyle { ASYNC, SYNC, NOT_AT_ALL }; + + clio::Logger log_{"ETL"}; + + std::reference_wrapper ioContext_; + std::shared_ptr backend_; + std::reference_wrapper cache_; + LoadStyle cacheLoadStyle_ = LoadStyle::ASYNC; + + // number of diffs to use to generate cursors to traverse the ledger in parallel during initial cache download + size_t numCacheDiffs_ = 32; + + // number of markers to use at one time to traverse the ledger in parallel during initial cache download + size_t numCacheMarkers_ = 48; + + // number of ledger objects to fetch concurrently per marker during cache download + size_t cachePageFetchSize_ = 512; + + struct ClioPeer + { + std::string ip; + int port; + }; + + std::vector clioPeers_; + + std::thread thread_; + std::atomic_bool stopping_ = false; + +public: + CacheLoader( + clio::Config const& config, + boost::asio::io_context& ioc, + std::shared_ptr const& backend, + CacheType& ledgerCache) + : ioContext_{std::ref(ioc)}, backend_{backend}, cache_{ledgerCache} + { + if (config.contains("cache")) + { + auto const cache = config.section("cache"); + if (auto entry = cache.maybeValue("load"); entry) + { + if (boost::iequals(*entry, "sync")) + cacheLoadStyle_ = LoadStyle::SYNC; + if (boost::iequals(*entry, "async")) + cacheLoadStyle_ = LoadStyle::ASYNC; + if (boost::iequals(*entry, "none") or boost::iequals(*entry, "no")) + cacheLoadStyle_ = LoadStyle::NOT_AT_ALL; + } + + numCacheDiffs_ = cache.valueOr("num_diffs", numCacheDiffs_); + numCacheMarkers_ = cache.valueOr("num_markers", numCacheMarkers_); + cachePageFetchSize_ = cache.valueOr("page_fetch_size", cachePageFetchSize_); + + if (auto peers = cache.maybeArray("peers"); peers) + { + for (auto const& peer : *peers) + { + auto ip = peer.value("ip"); + auto port = peer.value("port"); + + // todo: use emplace_back when clang is ready + clioPeers_.push_back({ip, port}); + } + + unsigned seed = std::chrono::system_clock::now().time_since_epoch().count(); + std::shuffle(std::begin(clioPeers_), std::end(clioPeers_), std::default_random_engine(seed)); + } + } + } + + ~CacheLoader() + { + stop(); + if (thread_.joinable()) + thread_.join(); + } + + /** + * @brief Populates the cache by walking through the given ledger. + * + * Should only be called once. The default behavior is to return immediately and populate the cache in the + * background. This can be overridden via config parameter, to populate synchronously, or not at all. + */ + void + load(uint32_t seq) + { + if (cacheLoadStyle_ == LoadStyle::NOT_AT_ALL) + { + cache_.get().setDisabled(); + log_.warn() << "Cache is disabled. Not loading"; + return; + } + + if (cache_.get().isFull()) + { + assert(false); + return; + } + + if (clioPeers_.size() > 0) + { + boost::asio::spawn(ioContext_.get(), [this, seq](boost::asio::yield_context yield) { + for (auto const& peer : clioPeers_) + { + // returns true on success + if (loadCacheFromClioPeer(seq, peer.ip, std::to_string(peer.port), yield)) + return; + } + + // if we couldn't successfully load from any peers, load from db + loadCacheFromDb(seq); + }); + return; + } + else + { + loadCacheFromDb(seq); + } + + // If loading synchronously, poll cache until full + while (cacheLoadStyle_ == LoadStyle::SYNC && not cache_.get().isFull()) + { + log_.debug() << "Cache not full. Cache size = " << cache_.get().size() << ". Sleeping ..."; + std::this_thread::sleep_for(std::chrono::seconds(10)); + if (cache_.get().isFull()) + log_.info() << "Cache is full. Cache size = " << cache_.get().size(); + } + } + + void + stop() + { + stopping_ = true; + } + +private: + bool + loadCacheFromClioPeer( + uint32_t ledgerIndex, + std::string const& ip, + std::string const& port, + boost::asio::yield_context& yield) + { + log_.info() << "Loading cache from peer. ip = " << ip << " . port = " << port; + namespace beast = boost::beast; // from + namespace http = beast::http; // from + namespace websocket = beast::websocket; // from + namespace net = boost::asio; // from + using tcp = boost::asio::ip::tcp; // from + try + { + boost::beast::error_code ec; + // These objects perform our I/O + tcp::resolver resolver{ioContext_.get()}; + + log_.trace() << "Creating websocket"; + auto ws = std::make_unique>(ioContext_.get()); + + // Look up the domain name + auto const results = resolver.async_resolve(ip, port, yield[ec]); + if (ec) + return {}; + + log_.trace() << "Connecting websocket"; + // Make the connection on the IP address we get from a lookup + ws->next_layer().async_connect(results, yield[ec]); + if (ec) + return false; + + log_.trace() << "Performing websocket handshake"; + // Perform the websocket handshake + ws->async_handshake(ip, "/", yield[ec]); + if (ec) + return false; + + std::optional marker; + + log_.trace() << "Sending request"; + auto getRequest = [&](auto marker) { + boost::json::object request = { + {"command", "ledger_data"}, + {"ledger_index", ledgerIndex}, + {"binary", true}, + {"out_of_order", true}, + {"limit", 2048}}; + + if (marker) + request["marker"] = *marker; + return request; + }; + + bool started = false; + size_t numAttempts = 0; + do + { + // Send the message + ws->async_write(net::buffer(boost::json::serialize(getRequest(marker))), yield[ec]); + if (ec) + { + log_.error() << "error writing = " << ec.message(); + return false; + } + + beast::flat_buffer buffer; + ws->async_read(buffer, yield[ec]); + if (ec) + { + log_.error() << "error reading = " << ec.message(); + return false; + } + + auto raw = beast::buffers_to_string(buffer.data()); + auto parsed = boost::json::parse(raw); + + if (!parsed.is_object()) + { + log_.error() << "Error parsing response: " << raw; + return false; + } + log_.trace() << "Successfully parsed response " << parsed; + + if (auto const& response = parsed.as_object(); response.contains("error")) + { + log_.error() << "Response contains error: " << response; + auto const& err = response.at("error"); + if (err.is_string() && err.as_string() == "lgrNotFound") + { + ++numAttempts; + if (numAttempts >= 5) + { + log_.error() << " ledger not found at peer after 5 attempts. " + "peer = " + << ip << " ledger = " << ledgerIndex + << ". Check your config and the health of the peer"; + return false; + } + log_.warn() << "Ledger not found. ledger = " << ledgerIndex << ". Sleeping and trying again"; + std::this_thread::sleep_for(std::chrono::seconds(1)); + continue; + } + return false; + } + started = true; + auto const& response = parsed.as_object()["result"].as_object(); + + if (!response.contains("cache_full") || !response.at("cache_full").as_bool()) + { + log_.error() << "cache not full for clio node. ip = " << ip; + return false; + } + if (response.contains("marker")) + marker = response.at("marker"); + else + marker = {}; + + auto const& state = response.at("state").as_array(); + + std::vector objects; + objects.reserve(state.size()); + for (auto const& ledgerObject : state) + { + auto const& obj = ledgerObject.as_object(); + + Backend::LedgerObject stateObject = {}; + + if (!stateObject.key.parseHex(obj.at("index").as_string().c_str())) + { + log_.error() << "failed to parse object id"; + return false; + } + boost::algorithm::unhex(obj.at("data").as_string().c_str(), std::back_inserter(stateObject.blob)); + objects.push_back(std::move(stateObject)); + } + cache_.get().update(objects, ledgerIndex, true); + + if (marker) + log_.debug() << "At marker " << *marker; + } while (marker || !started); + + log_.info() << "Finished downloading ledger from clio node. ip = " << ip; + + cache_.get().setFull(); + return true; + } + catch (std::exception const& e) + { + log_.error() << "Encountered exception : " << e.what() << " - ip = " << ip; + return false; + } + } + + void + loadCacheFromDb(uint32_t seq) + { + std::vector diff; + std::vector> cursors; + + auto append = [](auto&& a, auto&& b) { a.insert(std::end(a), std::begin(b), std::end(b)); }; + + for (size_t i = 0; i < numCacheDiffs_; ++i) + { + append(diff, Backend::synchronousAndRetryOnTimeout([&](auto yield) { + return backend_->fetchLedgerDiff(seq - i, yield); + })); + } + + std::sort(diff.begin(), diff.end(), [](auto a, auto b) { + return a.key < b.key || (a.key == b.key && a.blob.size() < b.blob.size()); + }); + + diff.erase(std::unique(diff.begin(), diff.end(), [](auto a, auto b) { return a.key == b.key; }), diff.end()); + + cursors.push_back({}); + for (auto& obj : diff) + if (obj.blob.size()) + cursors.push_back({obj.key}); + cursors.push_back({}); + + std::stringstream cursorStr; + for (auto& c : cursors) + if (c) + cursorStr << ripple::strHex(*c) << ", "; + + log_.info() << "Loading cache. num cursors = " << cursors.size() - 1; + log_.trace() << "cursors = " << cursorStr.str(); + + thread_ = std::thread{[this, seq, cursors]() { + auto startTime = std::chrono::system_clock::now(); + auto markers = std::make_shared(0); + auto numRemaining = std::make_shared(cursors.size() - 1); + + for (size_t i = 0; i < cursors.size() - 1; ++i) + { + auto const start = cursors[i]; + auto const end = cursors[i + 1]; + + markers->wait(numCacheMarkers_); + ++(*markers); + + boost::asio::spawn( + ioContext_.get(), + [this, seq, start, end, numRemaining, startTime, markers](boost::asio::yield_context yield) { + std::optional cursor = start; + std::string cursorStr = + cursor.has_value() ? ripple::strHex(cursor.value()) : ripple::strHex(Backend::firstKey); + log_.debug() << "Starting a cursor: " << cursorStr << " markers = " << *markers; + + while (not stopping_) + { + auto res = Backend::retryOnTimeout([this, seq, &cursor, &yield]() { + return backend_->fetchLedgerPage(cursor, seq, cachePageFetchSize_, false, yield); + }); + + cache_.get().update(res.objects, seq, true); + + if (!res.cursor || (end && *(res.cursor) > *end)) + break; + + log_.trace() << "Loading cache. cache size = " << cache_.get().size() + << " - cursor = " << ripple::strHex(res.cursor.value()) + << " start = " << cursorStr << " markers = " << *markers; + + cursor = std::move(res.cursor); + } + + --(*markers); + markers->notify_one(); + + if (--(*numRemaining) == 0) + { + auto endTime = std::chrono::system_clock::now(); + auto duration = std::chrono::duration_cast(endTime - startTime); + + log_.info() << "Finished loading cache. cache size = " << cache_.get().size() << ". Took " + << duration.count() << " seconds"; + cache_.get().setFull(); + } + else + { + log_.info() << "Finished a cursor. num remaining = " << *numRemaining + << " start = " << cursorStr << " markers = " << *markers; + } + }); + } + }}; + } +}; + +} // namespace clio::detail diff --git a/src/etl/impl/ExtractionDataPipe.h b/src/etl/impl/ExtractionDataPipe.h new file mode 100644 index 00000000..a9f66a99 --- /dev/null +++ b/src/etl/impl/ExtractionDataPipe.h @@ -0,0 +1,134 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include + +#include +#include + +namespace clio::detail { + +/** + * @brief A collection of thread safe async queues used by Extractor and Transformer to communicate + */ +template +class ExtractionDataPipe +{ +public: + using DataType = std::optional; + using QueueType = ThreadSafeQueue; // TODO: probably should use boost::lockfree::queue instead? + + constexpr static auto TOTAL_MAX_IN_QUEUE = 1000u; + +private: + clio::Logger log_{"ETL"}; + + uint32_t stride_; + uint32_t startSequence_; + + std::vector> 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 = TOTAL_MAX_IN_QUEUE / stride; + for (size_t i = 0; i < stride_; ++i) + queues_.push_back(std::make_unique(maxQueueSize)); + } + + /** + * @brief Push new data package for the specified sequence. + * + * Note: Potentially blocks until the underlying queue can accomodate 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 + getQueue(uint32_t sequence) + { + log_.debug() << "Grabbing extraction queue for " << sequence << "; start was " << startSequence_; + return queues_[(sequence - startSequence_) % stride_]; + } +}; + +} // namespace clio::detail diff --git a/src/etl/impl/Extractor.h b/src/etl/impl/Extractor.h new file mode 100644 index 00000000..9bb31772 --- /dev/null +++ b/src/etl/impl/Extractor.h @@ -0,0 +1,131 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include + +#include +#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" +#include + +#include +#include +#include + +namespace clio::detail { + +/** + * @brief Extractor thread that is fetching GRPC data and enqueue it on the DataPipeType + */ +template +class Extractor +{ + clio::Logger log_{"ETL"}; + + std::reference_wrapper pipe_; + std::shared_ptr networkValidatedLedgers_; + std::reference_wrapper ledgerFetcher_; + uint32_t startSequence_; + std::optional finishSequence_; + std::reference_wrapper state_; // shared state for ETL + + std::thread thread_; + +public: + Extractor( + DataPipeType& pipe, + std::shared_ptr networkValidatedLedgers, + LedgerFetcherType& ledgerFetcher, + uint32_t startSequence, + std::optional finishSequence, + SystemState const& state) + : pipe_(std::ref(pipe)) + , networkValidatedLedgers_{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()); + thread_.join(); + } + +private: + void + process() + { + beast::setCurrentThreadName("ETLService extract"); + + double totalTime = 0.0; + auto currentSequence = startSequence_; + + // Two 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. + while ((not finishSequence_ || currentSequence <= *finishSequence_) && not state_.get().writeConflict && + not isStopping() && networkValidatedLedgers_->waitUntilValidatedByNetwork(currentSequence)) + { + auto [fetchResponse, time] = util::timed>( + [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, fetchLedgerDataAndDiff + // will keep trying to fetch the specified ledger until successful. + if (!fetchResponse) + break; + + auto const tps = fetchResponse->transactions_list().transactions_size() / time; + 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(); + + if (finishSequence_ && currentSequence > *finishSequence_) + break; + } + + pipe_.get().finish(startSequence_); + } + + bool + isStopping() const + { + return state_.get().isStopping; + } +}; + +} // namespace clio::detail diff --git a/src/etl/impl/ForwardCache.cpp b/src/etl/impl/ForwardCache.cpp new file mode 100644 index 00000000..a3d7729f --- /dev/null +++ b/src/etl/impl/ForwardCache.cpp @@ -0,0 +1,83 @@ +//------------------------------------------------------------------------------ +/* + This file is part of clio: https://github.com/XRPLF/clio + Copyright (c) 2022, the clio developers. + + Permission to use, copy, modify, and distribute this software for any + purpose with or without fee is hereby granted, provided that the above + copyright notice and this permission notice appear in all copies. + + THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES + WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF + MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR + ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES + WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN + ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF + OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. +*/ +//============================================================================== + +#include +#include +#include + +#include +#include + +namespace clio::detail { + +void +ForwardCache::freshen() +{ + log_.trace() << "Freshening ForwardCache"; + + auto numOutstanding = std::make_shared(latestForwarded_.size()); + + for (auto const& cacheEntry : latestForwarded_) + { + boost::asio::spawn( + strand_, [this, numOutstanding, command = cacheEntry.first](boost::asio::yield_context yield) { + boost::json::object request = {{"command", command}}; + auto resp = source_.requestFromRippled(request, {}, yield); + + if (!resp || resp->contains("error")) + resp = {}; + + { + std::scoped_lock lk(mtx_); + latestForwarded_[command] = resp; + } + }); + } +} + +void +ForwardCache::clear() +{ + std::scoped_lock lk(mtx_); + for (auto& cacheEntry : latestForwarded_) + latestForwarded_[cacheEntry.first] = {}; +} + +std::optional +ForwardCache::get(boost::json::object const& request) const +{ + std::optional command = {}; + if (request.contains("command") && !request.contains("method") && request.at("command").is_string()) + command = request.at("command").as_string().c_str(); + else if (request.contains("method") && !request.contains("command") && request.at("method").is_string()) + command = request.at("method").as_string().c_str(); + + if (!command) + return {}; + if (RPC::specifiesCurrentOrClosedLedger(request)) + return {}; + + std::shared_lock lk(mtx_); + if (!latestForwarded_.contains(*command)) + return {}; + + return {latestForwarded_.at(*command)}; +} + +} // namespace clio::detail diff --git a/src/etl/impl/ForwardCache.h b/src/etl/impl/ForwardCache.h new file mode 100644 index 00000000..2c4a6147 --- /dev/null +++ b/src/etl/impl/ForwardCache.h @@ -0,0 +1,82 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include +#include + +#include +#include + +#include +#include +#include + +class Source; + +namespace clio::detail { + +/** + * @brief Cache for rippled responses + */ +class ForwardCache +{ + using ResponseType = std::optional; + + clio::Logger log_{"ETL"}; + + mutable std::shared_mutex mtx_; + std::unordered_map latestForwarded_; + boost::asio::io_context::strand strand_; + Source const& source_; + std::uint32_t duration_ = 10; + + void + clear(); + +public: + ForwardCache(clio::Config const& config, boost::asio::io_context& ioc, Source const& source) + : strand_(ioc), source_(source) + { + if (config.contains("cache")) + { + auto commands = config.arrayOrThrow("cache", "Source cache must be array"); + + if (config.contains("cache_duration")) + duration_ = config.valueOrThrow("cache_duration", "Source cache_duration must be a number"); + + for (auto const& command : commands) + { + auto key = command.valueOrThrow("Source forward command must be array of strings"); + latestForwarded_[key] = {}; + } + } + } + + void + freshen(); + + std::optional + get(boost::json::object const& command) const; +}; + +} // namespace clio::detail diff --git a/src/etl/impl/LedgerFetcher.h b/src/etl/impl/LedgerFetcher.h new file mode 100644 index 00000000..ebfd5346 --- /dev/null +++ b/src/etl/impl/LedgerFetcher.h @@ -0,0 +1,102 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include + +#include +#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" +#include + +#include + +namespace clio::detail { + +/** + * @brief GRPC Ledger data fetcher + */ +template +class LedgerFetcher +{ +public: + using DataType = typename LoadBalancerType::DataType; + +private: + clio::Logger log_{"ETL"}; + + std::shared_ptr backend_; + std::shared_ptr loadBalancer_; + +public: + /** + * @brief Create an instance of the fetcher + */ + LedgerFetcher(std::shared_ptr backend, std::shared_ptr balancer) + : backend_(backend), loadBalancer_(balancer) + { + } + + /** + * @brief Extract data for a particular ledger from an ETL source + * + * This function continously tries to extract the specified ledger (using all available ETL sources) until the + * extraction succeeds, or the server shuts down. + * + * @param sequence sequence of the ledger to extract + * @return ledger header and transaction+metadata blobs; empty optional if the server is shutting down + */ + DataType + fetchData(uint32_t seq) + { + log_.debug() << "Attempting to fetch ledger with sequence = " << seq; + + auto response = loadBalancer_->fetchLedger(seq, false, false); + if (response) + log_.trace() << "GetLedger reply = " << response->DebugString(); + return response; + } + + /** + * @brief Extract diff data for a particular ledger from an ETL source. + * + * This function continously tries to extract the specified ledger (using all available ETL sources) until the + * extraction succeeds, or the server shuts down. + * + * @param sequence sequence of the ledger to extract + * @return ledger header, transaction+metadata blobs, and all ledger objects created, modified or deleted between + * this ledger and the parent; Empty optional if the server is shutting down + */ + DataType + fetchDataAndDiff(uint32_t seq) + { + log_.debug() << "Attempting to fetch ledger with sequence = " << seq; + + auto response = loadBalancer_->fetchLedger( + seq, true, !backend_->cache().isFull() || backend_->cache().latestLedgerSequence() >= seq); + if (response) + log_.trace() << "GetLedger reply = " << response->DebugString(); + + return response; + } +}; + +} // namespace clio::detail diff --git a/src/etl/impl/LedgerLoader.h b/src/etl/impl/LedgerLoader.h new file mode 100644 index 00000000..f784665b --- /dev/null +++ b/src/etl/impl/LedgerLoader.h @@ -0,0 +1,252 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include +#include +#include +#include +#include + +#include +#include +#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" +#include + +#include + +struct FormattedTransactionsData +{ + std::vector accountTxData; + std::vector nfTokenTxData; + std::vector nfTokensData; +}; + +namespace clio::detail { + +/** + * @brief Loads ledger data into the DB + */ +template +class LedgerLoader +{ + clio::Logger log_{"ETL"}; + + std::shared_ptr backend_; + std::shared_ptr loadBalancer_; + std::reference_wrapper fetcher_; + std::reference_wrapper state_; // shared state for ETL + +public: + /** + * @brief Create an instance of the loader + */ + LedgerLoader( + std::shared_ptr backend, + std::shared_ptr balancer, + LedgerFetcherType& fetcher, + SystemState const& state) + : backend_{backend}, loadBalancer_{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 processsing. + * + * @param ledger ledger to insert transactions into + * @param data data extracted from an ETL source + * @return struct that contains the neccessary info to write to the account_transactions/account_tx and + * nft_token_transactions tables (mostly transaction hashes, corresponding nodestore hashes and affected accounts) + */ + FormattedTransactionsData + insertTransactions(ripple::LedgerInfo const& ledger, org::xrpl::rpc::v1::GetLedgerResponse& data) + { + FormattedTransactionsData result; + + 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 sttx{it}; + + log_.trace() << "Inserting transaction = " << sttx.getTransactionID(); + + ripple::TxMeta 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()); + if (maybeNFT) + result.nfTokensData.push_back(*maybeNFT); + + auto journal = ripple::debugLog(); + result.accountTxData.emplace_back(txMeta, sttx.getTransactionID(), journal); + std::string keyStr{(const char*)sttx.getTransactionID().data(), 32}; + backend_->writeTransaction( + std::move(keyStr), + ledger.seq, + ledger.closeTime.time_since_epoch().count(), + std::move(*raw), + std::move(*txn.mutable_metadata_blob())); + } + + // Remove all but the last NFTsData for each id. unique removes all but the first of a group, so we want to + // reverse sort by transaction index + std::sort(result.nfTokensData.begin(), result.nfTokensData.end(), [](NFTsData const& a, NFTsData const& b) { + return a.tokenID > b.tokenID && a.transactionIndex > b.transactionIndex; + }); + + // Now we can unique the NFTs by tokenID. + auto last = std::unique( + result.nfTokensData.begin(), result.nfTokensData.end(), [](NFTsData const& a, NFTsData const& b) { + return a.tokenID == b.tokenID; + }); + result.nfTokensData.erase(last, result.nfTokensData.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 + loadInitialLedger(uint32_t sequence) + { + // check that database is actually empty + auto rng = backend_->hardFetchLedgerRangeNoThrow(); + if (rng) + { + log_.fatal() << "Database is not empty"; + assert(false); + 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 + std::optional ledgerData{fetcher_.get().fetchData(sequence)}; + if (!ledgerData) + return {}; + + ripple::LedgerInfo lgrInfo = util::deserializeHeader(ripple::makeSlice(ledgerData->ledger_header())); + + log_.debug() << "Deserialized ledger header. " << util::toString(lgrInfo); + + auto timeDiff = util::timed>([this, sequence, &lgrInfo, &ledgerData]() { + backend_->startWrites(); + + log_.debug() << "Started writes"; + + backend_->writeLedger(lgrInfo, std::move(*ledgerData->mutable_ledger_header())); + + log_.debug() << "Wrote ledger"; + FormattedTransactionsData insertTxResult = insertTransactions(lgrInfo, *ledgerData); + 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, success] = loadBalancer_->loadInitialLedger(sequence); + + if (success) + { + size_t numWrites = 0; + backend_->cache().setFull(); + + auto seconds = util::timed([this, edgeKeys = &edgeKeys, sequence, &numWrites]() { + for (auto& key : *edgeKeys) + { + 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 = Backend::firstKey; + while (auto cur = backend_->cache().getSuccessor(prev, sequence)) + { + assert(cur); + if (prev == Backend::firstKey) + 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(cur->key, sequence)) + { + auto succ = backend_->cache().getSuccessor(base, sequence); + assert(succ); + if (succ->key == cur->key) + { + log_.debug() << "Writing book successor = " << ripple::strHex(base) << " - " + << ripple::strHex(cur->key); + + backend_->writeSuccessor( + uint256ToString(base), sequence, uint256ToString(cur->key)); + } + } + + ++numWrites; + } + + prev = std::move(cur->key); + if (numWrites % 100000 == 0 && numWrites != 0) + log_.info() << "Wrote " << numWrites << " book successors"; + } + + backend_->writeSuccessor(uint256ToString(prev), sequence, uint256ToString(Backend::lastKey)); + ++numWrites; + }); + + log_.info() << "Looping through cache and submitting all writes took " << seconds + << " seconds. numWrites = " << std::to_string(numWrites); + } + + log_.debug() << "Loaded initial ledger"; + + if (not state_.get().isStopping) + { + backend_->writeAccountTransactions(std::move(insertTxResult.accountTxData)); + backend_->writeNFTs(std::move(insertTxResult.nfTokensData)); + backend_->writeNFTTransactions(std::move(insertTxResult.nfTokenTxData)); + } + + backend_->finishWrites(sequence); + }); + + log_.debug() << "Time to download and store ledger = " << timeDiff; + return lgrInfo; + } +}; + +} // namespace clio::detail diff --git a/src/etl/impl/LedgerPublisher.h b/src/etl/impl/LedgerPublisher.h new file mode 100644 index 00000000..97b4b7d5 --- /dev/null +++ b/src/etl/impl/LedgerPublisher.h @@ -0,0 +1,252 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include +#include +#include + +#include + +#include + +namespace clio::detail { + +/** + * @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. + */ +template +class LedgerPublisher +{ + clio::Logger log_{"ETL"}; + + boost::asio::io_context::strand publishStrand_; + + std::shared_ptr backend_; + std::shared_ptr subscriptions_; + std::reference_wrapper state_; // shared state for ETL + + std::chrono::time_point lastCloseTime_; + mutable std::shared_mutex closeTimeMtx_; + + std::chrono::time_point lastPublish_; + mutable std::shared_mutex publishTimeMtx_; + + std::optional lastPublishedSequence_; + mutable std::shared_mutex lastPublishedSeqMtx_; + +public: + /** + * @brief Create an instance of the publisher + */ + LedgerPublisher( + boost::asio::io_context& ioc, + std::shared_ptr backend, + std::shared_ptr subscriptions, + SystemState const& state) + : publishStrand_{ioc}, backend_{backend}, subscriptions_{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. 1 attempt per second + * @return whether the ledger was found in the database and published + */ + bool + publish(uint32_t ledgerSequence, std::optional maxAttempts) + { + 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) + { + 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_.debug() << "Failed to publish ledger after " << numAttempts << " attempts."; + return false; + } + std::this_thread::sleep_for(std::chrono::seconds(1)); + ++numAttempts; + continue; + } + else + { + auto lgr = Backend::synchronousAndRetryOnTimeout( + [&](auto yield) { return backend_->fetchLedgerBySequence(ledgerSequence, yield); }); + + assert(lgr); + publish(*lgr); + + return true; + } + } + return false; + } + + /** + * @brief Publish the passed in ledger + * + * 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::LedgerInfo const& lgrInfo) + { + boost::asio::post(publishStrand_, [this, lgrInfo = lgrInfo]() { + log_.info() << "Publishing ledger " << std::to_string(lgrInfo.seq); + + if (!state_.get().isWriting) + { + log_.info() << "Updating cache"; + + std::vector diff = Backend::synchronousAndRetryOnTimeout( + [&](auto yield) { return backend_->fetchLedgerDiff(lgrInfo.seq, yield); }); + + backend_->cache().update(diff, lgrInfo.seq); // todo: inject cache to update, don't use backend cache + backend_->updateRange(lgrInfo.seq); + } + + setLastClose(lgrInfo.closeTime); + auto age = lastCloseAgeSeconds(); + + // if the ledger closed over 10 minutes ago, assume we are still catching up and don't publish + // TODO: this probably should be a strategy + if (age < 600) + { + std::optional fees = Backend::synchronousAndRetryOnTimeout( + [&](auto yield) { return backend_->fetchFees(lgrInfo.seq, yield); }); + + std::vector transactions = Backend::synchronousAndRetryOnTimeout( + [&](auto yield) { return backend_->fetchAllTransactionsInLedger(lgrInfo.seq, yield); }); + + auto ledgerRange = backend_->fetchLedgerRange(); + assert(ledgerRange); + assert(fees); + + std::string range = + std::to_string(ledgerRange->minSequence) + "-" + std::to_string(ledgerRange->maxSequence); + + subscriptions_->pubLedger(lgrInfo, *fees, range, transactions.size()); + + for (auto& txAndMeta : transactions) + subscriptions_->pubTransaction(txAndMeta, lgrInfo); + + subscriptions_->pubBookChanges(lgrInfo, transactions); + + setLastPublishTime(); + log_.info() << "Published ledger " << std::to_string(lgrInfo.seq); + } + else + log_.info() << "Skipping publishing ledger " << std::to_string(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 + { + return std::chrono::duration_cast(std::chrono::system_clock::now() - getLastPublish()) + .count(); + } + + /** + * @brief Get last publish time as a time point + */ + std::chrono::time_point + getLastPublish() const + { + std::shared_lock lck(publishTimeMtx_); + return lastPublish_; + } + + /** + * @brief Get time passed since last ledger close, in seconds + */ + std::uint32_t + lastCloseAgeSeconds() const + { + std::shared_lock lck(closeTimeMtx_); + auto now = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()) + .count(); + auto closeTime = lastCloseTime_.time_since_epoch().count(); + if (now < (rippleEpochStart + closeTime)) + return 0; + return now - (rippleEpochStart + closeTime); + } + + std::optional + getLastPublishedSequence() const + { + std::scoped_lock lck(lastPublishedSeqMtx_); + return lastPublishedSequence_; + } + +private: + void + setLastClose(std::chrono::time_point lastCloseTime) + { + std::scoped_lock lck(closeTimeMtx_); + lastCloseTime_ = lastCloseTime; + } + + void + setLastPublishTime() + { + std::scoped_lock lck(publishTimeMtx_); + lastPublish_ = std::chrono::system_clock::now(); + } + + void + setLastPublishedSequence(std::optional lastPublishedSequence) + { + std::scoped_lock lck(lastPublishedSeqMtx_); + lastPublishedSequence_ = lastPublishedSequence; + } +}; + +} // namespace clio::detail diff --git a/src/etl/impl/Transformer.h b/src/etl/impl/Transformer.h new file mode 100644 index 00000000..7a4e65fe --- /dev/null +++ b/src/etl/impl/Transformer.h @@ -0,0 +1,393 @@ +//------------------------------------------------------------------------------ +/* + 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 +#include +#include +#include +#include + +#include +#include "org/xrpl/rpc/v1/xrp_ledger.grpc.pb.h" +#include + +#include +#include +#include + +namespace clio::detail { + +/* + * 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 +class Transformer +{ + clio::Logger log_{"ETL"}; + + std::reference_wrapper pipe_; + std::shared_ptr backend_; + std::reference_wrapper loader_; + std::reference_wrapper publisher_; + uint32_t startSequence_; + std::reference_wrapper 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 backend, + LedgerLoaderType& loader, + LedgerPublisherType& publisher, + uint32_t startSequence, + SystemState& state) + : pipe_(std::ref(pipe)) + , backend_{backend} + , loader_(std::ref(loader)) + , publisher_(std::ref(publisher)) + , 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()); + thread_.join(); + } + +private: + bool + isStopping() const + { + return state_.get().isStopping; + } + + void + process() + { + beast::setCurrentThreadName("ETLService transform"); + uint32_t currentSequence = startSequence_; + + while (not state_.get().writeConflict) + { + auto fetchResponse = pipe_.get().popNext(currentSequence); + ++currentSequence; + + // if fetchResponse is an empty optional, the extracter thread has stopped and the transformer should + // stop as well + if (!fetchResponse) + break; + + if (isStopping()) + continue; + + auto const numTxns = fetchResponse->transactions_list().transactions_size(); + auto const numObjects = fetchResponse->ledger_objects().objects_size(); + auto const start = std::chrono::system_clock::now(); + auto [lgrInfo, success] = buildNextLedger(*fetchResponse); + auto const end = std::chrono::system_clock::now(); + + auto duration = ((end - start).count()) / 1000000000.0; + if (success) + 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; + else + log_.error() << "Error writing ledger. " << util::toString(lgrInfo); + + // success is false if the ledger was already written + if (success) + publisher_.get().publish(lgrInfo); + + state_.get().writeConflict = !success; + } + } + + // TODO update this documentation + /** + * @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 + buildNextLedger(org::xrpl::rpc::v1::GetLedgerResponse& rawData) + { + log_.debug() << "Beginning ledger update"; + ripple::LedgerInfo lgrInfo = util::deserializeHeader(ripple::makeSlice(rawData.ledger_header())); + + log_.debug() << "Deserialized ledger header. " << util::toString(lgrInfo); + backend_->startWrites(); + log_.debug() << "started writes"; + + backend_->writeLedger(lgrInfo, std::move(*rawData.mutable_ledger_header())); + log_.debug() << "wrote ledger header"; + + writeSuccessors(lgrInfo, rawData); + updateCache(lgrInfo, rawData); + + log_.debug() << "Inserted/modified/deleted all objects. Number of objects = " + << rawData.ledger_objects().objects_size(); + + FormattedTransactionsData insertTxResult = loader_.get().insertTransactions(lgrInfo, rawData); + + log_.debug() << "Inserted all transactions. Number of transactions = " + << rawData.transactions_list().transactions_size(); + + backend_->writeAccountTransactions(std::move(insertTxResult.accountTxData)); + backend_->writeNFTs(std::move(insertTxResult.nfTokensData)); + backend_->writeNFTTransactions(std::move(insertTxResult.nfTokenTxData)); + + log_.debug() << "wrote account_tx"; + + auto [success, duration] = + util::timed>([&]() { return backend_->finishWrites(lgrInfo.seq); }); + + log_.debug() << "Finished writes. took " << std::to_string(duration); + 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::LedgerInfo const& lgrInfo, org::xrpl::rpc::v1::GetLedgerResponse& rawData) + { + std::vector cacheUpdates; + cacheUpdates.reserve(rawData.ledger_objects().objects_size()); + + // TODO change these to unordered_set + std::set bookSuccessorsToCalculate; + std::set modified; + + for (auto& obj : *(rawData.mutable_ledger_objects()->mutable_objects())) + { + auto key = ripple::uint256::fromVoidChecked(obj.key()); + assert(key); + + cacheUpdates.push_back({*key, {obj.mutable_data()->begin(), obj.mutable_data()->end()}}); + log_.debug() << "key = " << ripple::strHex(*key) << " - mod type = " << obj.mod_type(); + + if (obj.mod_type() != org::xrpl::rpc::v1::RawLedgerObject::MODIFIED && !rawData.object_neighbors_included()) + { + log_.debug() << "object neighbors not included. using cache"; + + if (!backend_->cache().isFull() || backend_->cache().latestLedgerSequence() != lgrInfo.seq - 1) + throw std::runtime_error("Cache is not full, but object neighbors were not included"); + + auto const blob = obj.mutable_data(); + bool checkBookBase = false; + bool const isDeleted = (blob->size() == 0); + + if (isDeleted) + { + auto old = backend_->cache().get(*key, lgrInfo.seq - 1); + assert(old); + checkBookBase = isBookDir(*key, *old); + } + else + { + checkBookBase = isBookDir(*key, *blob); + } + + if (checkBookBase) + { + log_.debug() << "Is book dir. key = " << ripple::strHex(*key); + + auto bookBase = getBookBase(*key); + auto oldFirstDir = backend_->cache().getSuccessor(bookBase, lgrInfo.seq - 1); + assert(oldFirstDir); + + // 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_.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() == org::xrpl::rpc::v1::RawLedgerObject::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_.debug() << "object neighbors not included. using cache"; + if (!backend_->cache().isFull() || backend_->cache().latestLedgerSequence() != lgrInfo.seq) + throw std::runtime_error("Cache is not full, but object neighbors were not included"); + + for (auto const& obj : cacheUpdates) + { + if (modified.count(obj.key)) + continue; + + auto lb = backend_->cache().getPredecessor(obj.key, lgrInfo.seq); + if (!lb) + lb = {Backend::firstKey, {}}; + + auto ub = backend_->cache().getSuccessor(obj.key, lgrInfo.seq); + if (!ub) + ub = {Backend::lastKey, {}}; + + if (obj.blob.size() == 0) + { + 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_.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_.debug() << "Updating book successor " << ripple::strHex(base) << " - " + << ripple::strHex(succ->key); + } + else + { + backend_->writeSuccessor(uint256ToString(base), lgrInfo.seq, uint256ToString(Backend::lastKey)); + + log_.debug() << "Updating book successor " << ripple::strHex(base) << " - " + << ripple::strHex(Backend::lastKey); + } + } + } + } + + /** + * @brief Write successors info into DB + * + * @param lgrInfo Ledger info + * @param rawData Ledger data from GRPC + */ + void + writeSuccessors(ripple::LedgerInfo const& lgrInfo, org::xrpl::rpc::v1::GetLedgerResponse& rawData) + { + // Write successor info, if included from rippled + if (rawData.object_neighbors_included()) + { + 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(Backend::lastKey); + 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() != org::xrpl::rpc::v1::RawLedgerObject::MODIFIED) + { + std::string* predPtr = obj.mutable_predecessor(); + if (!predPtr->size()) + *predPtr = uint256ToString(Backend::firstKey); + std::string* succPtr = obj.mutable_successor(); + if (!succPtr->size()) + *succPtr = uint256ToString(Backend::lastKey); + + if (obj.mod_type() == org::xrpl::rpc::v1::RawLedgerObject::DELETED) + { + 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_.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_.debug() << "object modified " << ripple::strHex(obj.key()); + } + } + } +}; + +} // namespace clio::detail diff --git a/src/main/main.cpp b/src/main/main.cpp index 8ce16d3a..e3e49aef 100644 --- a/src/main/main.cpp +++ b/src/main/main.cpp @@ -27,7 +27,7 @@ #include #include -#include +#include #include #include #include @@ -182,8 +182,8 @@ try } LogService::info() << "Number of io threads = " << threads; - // IO context to handle all incoming requests, as well as other things - // This is not the only io context in the application + // IO context to handle all incoming requests, as well as other things. + // This is not the only io context in the application. boost::asio::io_context ioc{threads}; // Rate limiter, to prevent abuse @@ -196,20 +196,17 @@ try // Manages clients subscribed to streams auto subscriptions = SubscriptionManager::make_SubscriptionManager(config, backend); - // Tracks which ledgers have been validated by the - // network + // Tracks which ledgers have been validated by the network auto ledgers = NetworkValidatedLedgers::make_ValidatedLedgers(); // Handles the connection to one or more rippled nodes. // 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 = ETLLoadBalancer::make_ETLLoadBalancer(config, ioc, backend, subscriptions, ledgers); + // The balancer itself publishes to streams (transactions_proposed and accounts_proposed) + auto balancer = LoadBalancer::make_LoadBalancer(config, ioc, backend, subscriptions, ledgers); - // ETL is responsible for writing and publishing to streams. In read-only - // mode, ETL only publishes - auto etl = ReportingETL::make_ReportingETL(config, ioc, backend, subscriptions, balancer, ledgers); + // ETL is responsible for writing and publishing to streams. In read-only mode, ETL only publishes + auto etl = ETLService::make_ETLService(config, ioc, backend, subscriptions, balancer, ledgers); auto workQueue = WorkQueue::make_WorkQueue(config); auto counters = RPC::Counters::make_Counters(workQueue); diff --git a/src/rpc/Factories.cpp b/src/rpc/Factories.cpp index 0ce6ec93..db96f817 100644 --- a/src/rpc/Factories.cpp +++ b/src/rpc/Factories.cpp @@ -17,7 +17,7 @@ */ //============================================================================== -#include +#include #include #include #include diff --git a/src/rpc/Factories.h b/src/rpc/Factories.h index a6c9f4d8..f8f71445 100644 --- a/src/rpc/Factories.h +++ b/src/rpc/Factories.h @@ -42,8 +42,8 @@ class WsBase; class SubscriptionManager; -class ETLLoadBalancer; -class ReportingETL; +class LoadBalancer; +class ETLService; namespace RPC { diff --git a/src/rpc/RPCEngine.h b/src/rpc/RPCEngine.h index d718443e..7df40c41 100644 --- a/src/rpc/RPCEngine.h +++ b/src/rpc/RPCEngine.h @@ -21,7 +21,7 @@ #include #include -#include +#include #include #include #include @@ -45,8 +45,8 @@ class WsBase; class SubscriptionManager; -class ETLLoadBalancer; -class ReportingETL; +class LoadBalancer; +class ETLService; namespace RPC { @@ -61,7 +61,7 @@ class RPCEngineBase std::shared_ptr backend_; std::shared_ptr subscriptions_; - std::shared_ptr balancer_; + std::shared_ptr balancer_; std::reference_wrapper dosGuard_; std::reference_wrapper workQueue_; std::reference_wrapper counters_; @@ -73,8 +73,8 @@ public: RPCEngineBase( std::shared_ptr const& backend, std::shared_ptr const& subscriptions, - std::shared_ptr const& balancer, - std::shared_ptr const& etl, + std::shared_ptr const& balancer, + std::shared_ptr const& etl, clio::DOSGuard const& dosGuard, WorkQueue& workQueue, Counters& counters, @@ -94,8 +94,8 @@ public: clio::Config const& config, std::shared_ptr const& backend, std::shared_ptr const& subscriptions, - std::shared_ptr const& balancer, - std::shared_ptr const& etl, + std::shared_ptr const& balancer, + std::shared_ptr const& etl, clio::DOSGuard const& dosGuard, WorkQueue& workQueue, Counters& counters, diff --git a/src/rpc/common/impl/HandlerProvider.cpp b/src/rpc/common/impl/HandlerProvider.cpp index 41b8823f..7b76451d 100644 --- a/src/rpc/common/impl/HandlerProvider.cpp +++ b/src/rpc/common/impl/HandlerProvider.cpp @@ -19,7 +19,7 @@ #include -#include +#include #include #include @@ -56,8 +56,8 @@ namespace RPC::detail { ProductionHandlerProvider::ProductionHandlerProvider( std::shared_ptr const& backend, std::shared_ptr const& subscriptionManager, - std::shared_ptr const& balancer, - std::shared_ptr const& etl, + std::shared_ptr const& balancer, + std::shared_ptr const& etl, Counters const& counters) : handlerMap_{ {"account_channels", {AccountChannelsHandler{backend}}}, diff --git a/src/rpc/common/impl/HandlerProvider.h b/src/rpc/common/impl/HandlerProvider.h index 8144af53..2097566e 100644 --- a/src/rpc/common/impl/HandlerProvider.h +++ b/src/rpc/common/impl/HandlerProvider.h @@ -29,8 +29,8 @@ #include class SubscriptionManager; -class ReportingETL; -class ETLLoadBalancer; +class ETLService; +class LoadBalancer; namespace RPC { class Counters; @@ -56,8 +56,8 @@ public: ProductionHandlerProvider( std::shared_ptr const& backend, std::shared_ptr const& subscriptionManager, - std::shared_ptr const& balancer, - std::shared_ptr const& etl, + std::shared_ptr const& balancer, + std::shared_ptr const& etl, Counters const& counters); bool diff --git a/src/rpc/handlers/ServerInfo.h b/src/rpc/handlers/ServerInfo.h index 755ce7be..1236c318 100644 --- a/src/rpc/handlers/ServerInfo.h +++ b/src/rpc/handlers/ServerInfo.h @@ -30,8 +30,8 @@ #include class SubscriptionManager; -class ReportingETL; -class ETLLoadBalancer; +class ETLService; +class LoadBalancer; namespace RPC { class Counters; @@ -39,17 +39,13 @@ class Counters; namespace RPC { -template < - typename SubscriptionManagerType, - typename ETLLoadBalancerType, - typename ReportingETLType, - typename CountersType> +template class BaseServerInfoHandler { std::shared_ptr backend_; std::shared_ptr subscriptions_; - std::shared_ptr balancer_; - std::shared_ptr etl_; + std::shared_ptr balancer_; + std::shared_ptr etl_; std::reference_wrapper counters_; public: @@ -101,8 +97,8 @@ public: BaseServerInfoHandler( std::shared_ptr const& backend, std::shared_ptr const& subscriptions, - std::shared_ptr const& balancer, - std::shared_ptr const& etl, + std::shared_ptr const& balancer, + std::shared_ptr const& etl, CountersType const& counters) : backend_(backend) , subscriptions_(subscriptions) @@ -245,6 +241,6 @@ private: * * For more details see: https://xrpl.org/server_info-clio.html */ -using ServerInfoHandler = BaseServerInfoHandler; +using ServerInfoHandler = BaseServerInfoHandler; } // namespace RPC diff --git a/src/util/LedgerUtils.h b/src/util/LedgerUtils.h new file mode 100644 index 00000000..1e23b810 --- /dev/null +++ b/src/util/LedgerUtils.h @@ -0,0 +1,61 @@ +//------------------------------------------------------------------------------ +/* + 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 + +#include +#include + +namespace util { + +inline ripple::LedgerInfo +deserializeHeader(ripple::Slice data) +{ + ripple::SerialIter sit(data.data(), data.size()); + + ripple::LedgerInfo info; + + info.seq = sit.get32(); + info.drops = sit.get64(); + info.parentHash = sit.get256(); + info.txHash = sit.get256(); + info.accountHash = sit.get256(); + info.parentCloseTime = ripple::NetClock::time_point{ripple::NetClock::duration{sit.get32()}}; + info.closeTime = ripple::NetClock::time_point{ripple::NetClock::duration{sit.get32()}}; + info.closeTimeResolution = ripple::NetClock::duration{sit.get8()}; + info.closeFlags = sit.get8(); + + info.hash = sit.get256(); + + return info; +} + +inline std::string +toString(ripple::LedgerInfo const& info) +{ + std::stringstream ss; + ss << "LedgerInfo { Sequence : " << info.seq << " Hash : " << ripple::strHex(info.hash) + << " TxHash : " << strHex(info.txHash) << " AccountHash : " << ripple::strHex(info.accountHash) + << " ParentHash : " << strHex(info.parentHash) << " }"; + return ss.str(); +} + +} // namespace util diff --git a/src/webserver/HttpBase.h b/src/webserver/HttpBase.h index 8bc9d7ee..8d3168bd 100644 --- a/src/webserver/HttpBase.h +++ b/src/webserver/HttpBase.h @@ -36,7 +36,7 @@ #include #include -#include +#include #include #include
#include @@ -112,8 +112,8 @@ class HttpBase : public util::Taggable std::shared_ptr backend_; std::shared_ptr rpcEngine_; std::shared_ptr subscriptions_; - std::shared_ptr balancer_; - std::shared_ptr etl_; + std::shared_ptr balancer_; + std::shared_ptr etl_; util::TagDecoratorFactory const& tagFactory_; clio::DOSGuard& dosGuard_; send_lambda lambda_; @@ -167,8 +167,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer buffer) @@ -344,8 +344,8 @@ handleRequest( std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, std::string const& ip, diff --git a/src/webserver/HttpSession.h b/src/webserver/HttpSession.h index 4bf2eef0..583739e2 100644 --- a/src/webserver/HttpSession.h +++ b/src/webserver/HttpSession.h @@ -40,8 +40,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer buffer) diff --git a/src/webserver/Listener.h b/src/webserver/Listener.h index 05f37efa..cc305dea 100644 --- a/src/webserver/Listener.h +++ b/src/webserver/Listener.h @@ -47,8 +47,8 @@ class Detector : public std::enable_shared_from_this backend_; std::shared_ptr rpcEngine_; std::shared_ptr subscriptions_; - std::shared_ptr balancer_; - std::shared_ptr etl_; + std::shared_ptr balancer_; + std::shared_ptr etl_; util::TagDecoratorFactory const& tagFactory_; clio::DOSGuard& dosGuard_; boost::beast::flat_buffer buffer_; @@ -61,8 +61,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard) : ioc_(ioc) @@ -150,8 +150,8 @@ make_WebsocketSession( std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard) { @@ -181,8 +181,8 @@ make_WebsocketSession( std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard) { @@ -214,8 +214,8 @@ class Listener : public std::enable_shared_from_this backend_; std::shared_ptr rpcEngine_; std::shared_ptr subscriptions_; - std::shared_ptr balancer_; - std::shared_ptr etl_; + std::shared_ptr balancer_; + std::shared_ptr etl_; util::TagDecoratorFactory tagFactory_; clio::DOSGuard& dosGuard_; @@ -227,8 +227,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory tagFactory, clio::DOSGuard& dosGuard) : ioc_(ioc) @@ -326,8 +326,8 @@ make_HttpServer( std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, clio::DOSGuard& dosGuard) { static clio::Logger log{"WebServer"}; diff --git a/src/webserver/PlainWsSession.h b/src/webserver/PlainWsSession.h index ecd8cd5c..9a1b070d 100644 --- a/src/webserver/PlainWsSession.h +++ b/src/webserver/PlainWsSession.h @@ -25,7 +25,7 @@ #include #include -#include +#include #include #include #include @@ -38,7 +38,7 @@ namespace ssl = boost::asio::ssl; namespace websocket = boost::beast::websocket; using tcp = boost::asio::ip::tcp; -class ReportingETL; +class ETLService; // Echoes back all received WebSocket messages class PlainWsSession : public WsSession @@ -54,8 +54,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer&& buffer) @@ -88,8 +88,8 @@ class WsUpgrader : public std::enable_shared_from_this std::shared_ptr backend_; std::shared_ptr rpcEngine_; std::shared_ptr subscriptions_; - std::shared_ptr balancer_; - std::shared_ptr etl_; + std::shared_ptr balancer_; + std::shared_ptr etl_; util::TagDecoratorFactory const& tagFactory_; clio::DOSGuard& dosGuard_; http::request req_; @@ -103,8 +103,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer&& b) @@ -129,8 +129,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer&& b, diff --git a/src/webserver/SslHttpSession.h b/src/webserver/SslHttpSession.h index 9e50127d..b0c983e9 100644 --- a/src/webserver/SslHttpSession.h +++ b/src/webserver/SslHttpSession.h @@ -41,8 +41,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer buffer) diff --git a/src/webserver/SslWsSession.h b/src/webserver/SslWsSession.h index 8f473939..ec9984da 100644 --- a/src/webserver/SslWsSession.h +++ b/src/webserver/SslWsSession.h @@ -25,7 +25,7 @@ #include #include -#include +#include #include @@ -35,7 +35,7 @@ namespace ssl = boost::asio::ssl; namespace websocket = boost::beast::websocket; using tcp = boost::asio::ip::tcp; -class ReportingETL; +class ETLService; class SslWsSession : public WsSession { @@ -50,8 +50,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer&& b) @@ -83,8 +83,8 @@ class SslWsUpgrader : public std::enable_shared_from_this std::shared_ptr backend_; std::shared_ptr rpcEngine_; std::shared_ptr subscriptions_; - std::shared_ptr balancer_; - std::shared_ptr etl_; + std::shared_ptr balancer_; + std::shared_ptr etl_; util::TagDecoratorFactory const& tagFactory_; clio::DOSGuard& dosGuard_; http::request req_; @@ -98,8 +98,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer&& b) @@ -124,8 +124,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer&& b, diff --git a/src/webserver/WsBase.h b/src/webserver/WsBase.h index a4972ea9..0e715de4 100644 --- a/src/webserver/WsBase.h +++ b/src/webserver/WsBase.h @@ -20,8 +20,8 @@ #pragma once #include -#include -#include +#include +#include #include #include #include @@ -102,7 +102,7 @@ public: }; class SubscriptionManager; -class ETLLoadBalancer; +class LoadBalancer; template class WsSession : public WsBase, public std::enable_shared_from_this> @@ -118,8 +118,8 @@ class WsSession : public WsBase, public std::enable_shared_from_this objects. If this were shared, there would be // a cyclical dependency that would block destruction std::weak_ptr subscriptions_; - std::shared_ptr balancer_; - std::shared_ptr etl_; + std::shared_ptr balancer_; + std::shared_ptr etl_; util::TagDecoratorFactory const& tagFactory_; clio::DOSGuard& dosGuard_; std::mutex mtx_; @@ -151,8 +151,8 @@ public: std::shared_ptr backend, std::shared_ptr rpcEngine, std::shared_ptr subscriptions, - std::shared_ptr balancer, - std::shared_ptr etl, + std::shared_ptr balancer, + std::shared_ptr etl, util::TagDecoratorFactory const& tagFactory, clio::DOSGuard& dosGuard, boost::beast::flat_buffer&& buffer) diff --git a/unittests/backend/cassandra/BackendTests.cpp b/unittests/backend/cassandra/BackendTests.cpp index 7bcffaec..e9995bfd 100644 --- a/unittests/backend/cassandra/BackendTests.cpp +++ b/unittests/backend/cassandra/BackendTests.cpp @@ -83,14 +83,10 @@ TEST_F(BackendCassandraTest, Basic) boost::asio::spawn(ctx, [this, &done, &work](boost::asio::yield_context yield) { std::string rawHeader = - "03C3141A01633CD656F91B4EBB5EB89B791BD34DBC8A04BB6F407C5335BC54" - "351E" - "DD733898497E809E04074D14D271E4832D7888754F9230800761563A292FA2" - "315A" - "6DB6FE30CC5909B285080FCD6773CC883F9FE0EE4D439340AC592AADB973ED" - "3CF5" - "3E2232B33EF57CECAC2816E3122816E31A0A00F8377CD95DFA484CFAE28265" - "6A58" + "03C3141A01633CD656F91B4EBB5EB89B791BD34DBC8A04BB6F407C5335BC54351E" + "DD733898497E809E04074D14D271E4832D7888754F9230800761563A292FA2315A" + "6DB6FE30CC5909B285080FCD6773CC883F9FE0EE4D439340AC592AADB973ED3CF5" + "3E2232B33EF57CECAC2816E3122816E31A0A00F8377CD95DFA484CFAE282656A58" "CE5AA29652EFFD80AC59CD91416E4E13DBBE"; auto hexStringToBinaryString = [](auto const& hex) { @@ -117,7 +113,7 @@ TEST_F(BackendCassandraTest, Basic) }; std::string rawHeaderBlob = hexStringToBinaryString(rawHeader); - ripple::LedgerInfo lgrInfo = deserializeHeader(ripple::makeSlice(rawHeaderBlob)); + ripple::LedgerInfo lgrInfo = util::deserializeHeader(ripple::makeSlice(rawHeaderBlob)); backend->writeLedger(lgrInfo, std::move(rawHeaderBlob)); backend->writeSuccessor(uint256ToString(Backend::firstKey), lgrInfo.seq, uint256ToString(Backend::lastKey)); @@ -230,21 +226,15 @@ TEST_F(BackendCassandraTest, Basic) "FE6B2DEBCF9183A426BC022005DAC06CD4517E86C2548A80996019F3AC60A0" "9EED153BF60C992930D68F09F981142252F328CF91263417762570D67220CC" "B33B1370"; - std::string hashHex = - "0A81FB3D6324C2DCF73131505C6E4DC67981D7FC39F5E9574CEC4B1F22D28B" - "F7"; + std::string hashHex = "0A81FB3D6324C2DCF73131505C6E4DC67981D7FC39F5E9574CEC4B1F22D28BF7"; // this account is not related to the above transaction and // metadata std::string accountHex = - "1100612200000000240480FDBC2503CE1A872D0000000555516931B2AD018E" - "FFBE" - "17C5C9DCCF872F36837C2C6136ACF80F2A24079CF81FD0624000000005FF0E" - "0781" + "1100612200000000240480FDBC2503CE1A872D0000000555516931B2AD018EFFBE" + "17C5C9DCCF872F36837C2C6136ACF80F2A24079CF81FD0624000000005FF0E0781" "142252F328CF91263417762570D67220CCB33B1370"; - std::string accountIndexHex = - "E0311EB450B6177F969B94DBDDA83E99B7A0576ACD9079573876F16C0C004F" - "06"; + std::string accountIndexHex = "E0311EB450B6177F969B94DBDDA83E99B7A0576ACD9079573876F16C0C004F06"; // An NFTokenMint tx std::string nftTxnHex = @@ -903,26 +893,18 @@ TEST_F(BackendCassandraTest, CacheIntegration) backend->cache().setFull(); std::string rawHeader = - "03C3141A01633CD656F91B4EBB5EB89B791BD34DBC8A04BB6F407C5335BC54" - "351E" - "DD733898497E809E04074D14D271E4832D7888754F9230800761563A292FA2" - "315A" - "6DB6FE30CC5909B285080FCD6773CC883F9FE0EE4D439340AC592AADB973ED" - "3CF5" - "3E2232B33EF57CECAC2816E3122816E31A0A00F8377CD95DFA484CFAE28265" - "6A58" + "03C3141A01633CD656F91B4EBB5EB89B791BD34DBC8A04BB6F407C5335BC54351E" + "DD733898497E809E04074D14D271E4832D7888754F9230800761563A292FA2315A" + "6DB6FE30CC5909B285080FCD6773CC883F9FE0EE4D439340AC592AADB973ED3CF5" + "3E2232B33EF57CECAC2816E3122816E31A0A00F8377CD95DFA484CFAE282656A58" "CE5AA29652EFFD80AC59CD91416E4E13DBBE"; // this account is not related to the above transaction and // metadata std::string accountHex = - "1100612200000000240480FDBC2503CE1A872D0000000555516931B2AD018E" - "FFBE" - "17C5C9DCCF872F36837C2C6136ACF80F2A24079CF81FD0624000000005FF0E" - "0781" + "1100612200000000240480FDBC2503CE1A872D0000000555516931B2AD018EFFBE" + "17C5C9DCCF872F36837C2C6136ACF80F2A24079CF81FD0624000000005FF0E0781" "142252F328CF91263417762570D67220CCB33B1370"; - std::string accountIndexHex = - "E0311EB450B6177F969B94DBDDA83E99B7A0576ACD9079573876F16C0C004F" - "06"; + std::string accountIndexHex = "E0311EB450B6177F969B94DBDDA83E99B7A0576ACD9079573876F16C0C004F06"; auto hexStringToBinaryString = [](auto const& hex) { auto blob = ripple::strUnHex(hex); @@ -950,7 +932,7 @@ TEST_F(BackendCassandraTest, CacheIntegration) std::string rawHeaderBlob = hexStringToBinaryString(rawHeader); std::string accountBlob = hexStringToBinaryString(accountHex); std::string accountIndexBlob = hexStringToBinaryString(accountIndexHex); - ripple::LedgerInfo lgrInfo = deserializeHeader(ripple::makeSlice(rawHeaderBlob)); + ripple::LedgerInfo lgrInfo = util::deserializeHeader(ripple::makeSlice(rawHeaderBlob)); backend->startWrites(); backend->writeLedger(lgrInfo, std::move(rawHeaderBlob)); diff --git a/unittests/etl/ExtractionDataPipeTest.cpp b/unittests/etl/ExtractionDataPipeTest.cpp new file mode 100644 index 00000000..5ab8100a --- /dev/null +++ b/unittests/etl/ExtractionDataPipeTest.cpp @@ -0,0 +1,81 @@ +//------------------------------------------------------------------------------ +/* + 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 + +#include + +#include + +constexpr static auto STRIDE = 4; +constexpr static auto START_SEQ = 1234; + +class ETLExtractionDataPipeTest : public NoLoggerFixture +{ +protected: + clio::detail::ExtractionDataPipe pipe_{STRIDE, START_SEQ}; +}; + +TEST_F(ETLExtractionDataPipeTest, StrideMatchesInput) +{ + EXPECT_EQ(pipe_.getStride(), STRIDE); +} + +TEST_F(ETLExtractionDataPipeTest, PushedDataCanBeRetrievedAndMatchesOriginal) +{ + for (std::size_t i = 0; i < 8; ++i) + pipe_.push(START_SEQ + i, START_SEQ + i); + + for (std::size_t i = 0; i < 8; ++i) + { + auto const data = pipe_.popNext(START_SEQ + i); + EXPECT_EQ(data.value(), START_SEQ + i); + } +} + +TEST_F(ETLExtractionDataPipeTest, CallingFinishPushesAnEmptyOptional) +{ + for (std::size_t i = 0; i < 4; ++i) + pipe_.finish(START_SEQ + i); + + for (std::size_t i = 0; i < 4; ++i) + { + auto const data = pipe_.popNext(START_SEQ + i); + EXPECT_FALSE(data.has_value()); + } +} + +TEST_F(ETLExtractionDataPipeTest, CallingCleanupUnblocksOtherThread) +{ + std::atomic_bool unblocked = false; + auto bgThread = std::thread([this, &unblocked] { + for (std::size_t i = 0; i < 252; ++i) + pipe_.push(START_SEQ, 1234); // 251st element will block this thread here + unblocked = true; + }); + + // emulate waiting for above thread to push and get blocked + std::this_thread::sleep_for(std::chrono::milliseconds{100}); + + EXPECT_FALSE(unblocked); + pipe_.cleanup(); + + bgThread.join(); + EXPECT_TRUE(unblocked); +} diff --git a/unittests/rpc/handlers/ServerInfoTest.cpp b/unittests/rpc/handlers/ServerInfoTest.cpp index a59a7ad4..2b451d8d 100644 --- a/unittests/rpc/handlers/ServerInfoTest.cpp +++ b/unittests/rpc/handlers/ServerInfoTest.cpp @@ -27,15 +27,15 @@ namespace json = boost::json; using namespace testing; using TestServerInfoHandler = - BaseServerInfoHandler; + BaseServerInfoHandler; constexpr static auto LEDGERHASH = "4BC50C9B0D8515D3EAAE1E74B29A95804346C491EE1A95BF25E4AAB854A6A652"; constexpr static auto CLIENTIP = "1.1.1.1"; class RPCServerInfoHandlerTest : public HandlerBaseTest, - public MockETLLoadBalancerTest, + public MockLoadBalancerTest, public MockSubscriptionManagerTest, - public MockReportingETLTest, + public MockETLServiceTest, public MockCountersTest { protected: @@ -43,9 +43,9 @@ protected: SetUp() override { HandlerBaseTest::SetUp(); - MockETLLoadBalancerTest::SetUp(); + MockLoadBalancerTest::SetUp(); MockSubscriptionManagerTest::SetUp(); - MockReportingETLTest::SetUp(); + MockETLServiceTest::SetUp(); MockCountersTest::SetUp(); } @@ -53,9 +53,9 @@ protected: TearDown() override { MockCountersTest::TearDown(); - MockReportingETLTest::TearDown(); + MockETLServiceTest::TearDown(); MockSubscriptionManagerTest::TearDown(); - MockETLLoadBalancerTest::TearDown(); + MockLoadBalancerTest::TearDown(); HandlerBaseTest::TearDown(); } @@ -134,7 +134,7 @@ protected: TEST_F(RPCServerInfoHandlerTest, NoRangeErrorsOutWithNotReady) { auto const handler = AnyHandler{TestServerInfoHandler{ - mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}}; + mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockETLServicePtr, *mockCountersPtr}}; runSpawn([&](auto& yield) { auto const req = json::parse("{}"); @@ -158,7 +158,7 @@ TEST_F(RPCServerInfoHandlerTest, NoLedgerInfoErrorsOutWithInternal) EXPECT_CALL(*rawBackendPtr, fetchLedgerBySequence).Times(1); auto const handler = AnyHandler{TestServerInfoHandler{ - mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}}; + mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockETLServicePtr, *mockCountersPtr}}; runSpawn([&](auto& yield) { auto const req = json::parse("{}"); @@ -185,7 +185,7 @@ TEST_F(RPCServerInfoHandlerTest, NoFeesErrorsOutWithInternal) EXPECT_CALL(*rawBackendPtr, doFetchLedgerObject).Times(1); auto const handler = AnyHandler{TestServerInfoHandler{ - mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}}; + mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockETLServicePtr, *mockCountersPtr}}; runSpawn([&](auto& yield) { auto const req = json::parse("{}"); @@ -201,7 +201,7 @@ TEST_F(RPCServerInfoHandlerTest, NoFeesErrorsOutWithInternal) TEST_F(RPCServerInfoHandlerTest, DefaultOutputIsPresent) { MockBackend* rawBackendPtr = static_cast(mockBackendPtr.get()); - MockETLLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); + MockLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); mockBackendPtr->updateRange(10); // min mockBackendPtr->updateRange(30); // max @@ -218,7 +218,7 @@ TEST_F(RPCServerInfoHandlerTest, DefaultOutputIsPresent) EXPECT_CALL(*rawBalancerPtr, forwardToRippled(testing::_, testing::Eq(CLIENTIP), testing::_)).Times(1); auto const handler = AnyHandler{TestServerInfoHandler{ - mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}}; + mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockETLServicePtr, *mockCountersPtr}}; runSpawn([&](auto& yield) { auto const req = json::parse("{}"); @@ -237,11 +237,11 @@ TEST_F(RPCServerInfoHandlerTest, DefaultOutputIsPresent) TEST_F(RPCServerInfoHandlerTest, AdminSectionPresentWhenAdminFlagIsSet) { MockBackend* rawBackendPtr = static_cast(mockBackendPtr.get()); - MockETLLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); + MockLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); MockCounters* rawCountersPtr = static_cast(mockCountersPtr.get()); MockSubscriptionManager* rawSubscriptionManagerPtr = static_cast(mockSubscriptionManagerPtr.get()); - MockReportingETL* rawReportingETLPtr = static_cast(mockReportingETLPtr.get()); + MockETLService* rawETLServicePtr = static_cast(mockETLServicePtr.get()); mockBackendPtr->updateRange(10); // min mockBackendPtr->updateRange(30); // max @@ -265,11 +265,11 @@ TEST_F(RPCServerInfoHandlerTest, AdminSectionPresentWhenAdminFlagIsSet) ON_CALL(*rawSubscriptionManagerPtr, report).WillByDefault(Return(empty)); EXPECT_CALL(*rawSubscriptionManagerPtr, report).Times(1); - ON_CALL(*rawReportingETLPtr, getInfo).WillByDefault(Return(empty)); - EXPECT_CALL(*rawReportingETLPtr, getInfo).Times(1); + ON_CALL(*rawETLServicePtr, getInfo).WillByDefault(Return(empty)); + EXPECT_CALL(*rawETLServicePtr, getInfo).Times(1); auto const handler = AnyHandler{TestServerInfoHandler{ - mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}}; + mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockETLServicePtr, *mockCountersPtr}}; runSpawn([&](auto& yield) { auto const req = json::parse("{}"); @@ -283,11 +283,11 @@ TEST_F(RPCServerInfoHandlerTest, AdminSectionPresentWhenAdminFlagIsSet) TEST_F(RPCServerInfoHandlerTest, RippledForwardedValuesPresent) { MockBackend* rawBackendPtr = static_cast(mockBackendPtr.get()); - MockETLLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); + MockLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); MockCounters* rawCountersPtr = static_cast(mockCountersPtr.get()); MockSubscriptionManager* rawSubscriptionManagerPtr = static_cast(mockSubscriptionManagerPtr.get()); - MockReportingETL* rawReportingETLPtr = static_cast(mockReportingETLPtr.get()); + MockETLService* rawETLServicePtr = static_cast(mockETLServicePtr.get()); mockBackendPtr->updateRange(10); // min mockBackendPtr->updateRange(30); // max @@ -320,11 +320,11 @@ TEST_F(RPCServerInfoHandlerTest, RippledForwardedValuesPresent) ON_CALL(*rawSubscriptionManagerPtr, report).WillByDefault(Return(empty)); EXPECT_CALL(*rawSubscriptionManagerPtr, report).Times(1); - ON_CALL(*rawReportingETLPtr, getInfo).WillByDefault(Return(empty)); - EXPECT_CALL(*rawReportingETLPtr, getInfo).Times(1); + ON_CALL(*rawETLServicePtr, getInfo).WillByDefault(Return(empty)); + EXPECT_CALL(*rawETLServicePtr, getInfo).Times(1); auto const handler = AnyHandler{TestServerInfoHandler{ - mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}}; + mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockETLServicePtr, *mockCountersPtr}}; runSpawn([&](auto& yield) { auto const req = json::parse("{}"); @@ -339,11 +339,11 @@ TEST_F(RPCServerInfoHandlerTest, RippledForwardedValuesPresent) TEST_F(RPCServerInfoHandlerTest, RippledForwardedValuesMissingNoExceptionThrown) { MockBackend* rawBackendPtr = static_cast(mockBackendPtr.get()); - MockETLLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); + MockLoadBalancer* rawBalancerPtr = static_cast(mockLoadBalancerPtr.get()); MockCounters* rawCountersPtr = static_cast(mockCountersPtr.get()); MockSubscriptionManager* rawSubscriptionManagerPtr = static_cast(mockSubscriptionManagerPtr.get()); - MockReportingETL* rawReportingETLPtr = static_cast(mockReportingETLPtr.get()); + MockETLService* rawETLServicePtr = static_cast(mockETLServicePtr.get()); mockBackendPtr->updateRange(10); // min mockBackendPtr->updateRange(30); // max @@ -373,11 +373,11 @@ TEST_F(RPCServerInfoHandlerTest, RippledForwardedValuesMissingNoExceptionThrown) ON_CALL(*rawSubscriptionManagerPtr, report).WillByDefault(Return(empty)); EXPECT_CALL(*rawSubscriptionManagerPtr, report).Times(1); - ON_CALL(*rawReportingETLPtr, getInfo).WillByDefault(Return(empty)); - EXPECT_CALL(*rawReportingETLPtr, getInfo).Times(1); + ON_CALL(*rawETLServicePtr, getInfo).WillByDefault(Return(empty)); + EXPECT_CALL(*rawETLServicePtr, getInfo).Times(1); auto const handler = AnyHandler{TestServerInfoHandler{ - mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockReportingETLPtr, *mockCountersPtr}}; + mockBackendPtr, mockSubscriptionManagerPtr, mockLoadBalancerPtr, mockETLServicePtr, *mockCountersPtr}}; runSpawn([&](auto& yield) { auto const req = json::parse("{}"); diff --git a/unittests/util/Fixtures.h b/unittests/util/Fixtures.h index f02d1c59..c5737e77 100644 --- a/unittests/util/Fixtures.h +++ b/unittests/util/Fixtures.h @@ -21,8 +21,8 @@ #include "MockBackend.h" #include "MockCounters.h" -#include "MockETLLoadBalancer.h" -#include "MockReportingETL.h" +#include "MockETLService.h" +#include "MockLoadBalancer.h" #include "MockSubscriptionManager.h" #include @@ -219,13 +219,13 @@ protected: /** * @brief Fixture with a mock etl balancer */ -struct MockETLLoadBalancerTest : virtual public NoLoggerFixture +struct MockLoadBalancerTest : virtual public NoLoggerFixture { void SetUp() override { NoLoggerFixture::SetUp(); - mockLoadBalancerPtr = std::make_shared(); + mockLoadBalancerPtr = std::make_shared(); } void TearDown() override @@ -234,28 +234,28 @@ struct MockETLLoadBalancerTest : virtual public NoLoggerFixture } protected: - std::shared_ptr mockLoadBalancerPtr; + std::shared_ptr mockLoadBalancerPtr; }; /** * @brief Fixture with a mock subscription manager */ -struct MockReportingETLTest : virtual public NoLoggerFixture +struct MockETLServiceTest : virtual public NoLoggerFixture { void SetUp() override { NoLoggerFixture::SetUp(); - mockReportingETLPtr = std::make_shared(); + mockETLServicePtr = std::make_shared(); } void TearDown() override { - mockReportingETLPtr.reset(); + mockETLServicePtr.reset(); } protected: - std::shared_ptr mockReportingETLPtr; + std::shared_ptr mockETLServicePtr; }; /** diff --git a/unittests/util/MockBackend.h b/unittests/util/MockBackend.h index 665c5767..976f7b99 100644 --- a/unittests/util/MockBackend.h +++ b/unittests/util/MockBackend.h @@ -144,8 +144,6 @@ public: MOCK_METHOD(void, startWrites, (), (const, override)); - MOCK_METHOD(bool, doOnlineDelete, (std::uint32_t, boost::asio::yield_context&), (const, override)); - MOCK_METHOD(bool, isTooBusy, (), (const, override)); MOCK_METHOD(void, doWriteLedgerObject, (std::string&&, std::uint32_t const, std::string&&), (override)); diff --git a/unittests/util/MockReportingETL.h b/unittests/util/MockETLService.h similarity index 96% rename from unittests/util/MockReportingETL.h rename to unittests/util/MockETLService.h index 0238bab3..50a08bac 100644 --- a/unittests/util/MockReportingETL.h +++ b/unittests/util/MockETLService.h @@ -24,10 +24,10 @@ #include -class MockReportingETL +class MockETLService { public: - MockReportingETL() + MockETLService() { } diff --git a/unittests/util/MockETLLoadBalancer.h b/unittests/util/MockLoadBalancer.h similarity index 91% rename from unittests/util/MockETLLoadBalancer.h rename to unittests/util/MockLoadBalancer.h index 79e2c9f5..c37b5e43 100644 --- a/unittests/util/MockETLLoadBalancer.h +++ b/unittests/util/MockLoadBalancer.h @@ -19,7 +19,7 @@ #pragma once -#include +#include #include #include @@ -30,10 +30,10 @@ #include -class MockETLLoadBalancer +class MockLoadBalancer { public: - MockETLLoadBalancer() + MockLoadBalancer() { } @@ -41,7 +41,7 @@ public: MOCK_METHOD(std::optional, fetchLedger, (uint32_t, bool, bool), ()); - MOCK_METHOD(bool, shouldPropagateTxnStream, (ETLSource*), (const)); + MOCK_METHOD(bool, shouldPropagateTxnStream, (Source*), (const)); MOCK_METHOD(boost::json::value, toJson, (), (const));