From 414cdf284f189b907b63e5a899715810d182aa5c Mon Sep 17 00:00:00 2001 From: Justin Berman Date: Tue, 7 May 2024 10:46:02 -0700 Subject: [PATCH] Async wallet scanner (#23) Implement async wallet scanner. Adds a new functional test for direct wallet2 -> live RPC daemon interactions. This sets up a framework to test pointing the Seraphis wallet lib to a live daemon. Tests sending and scanning: - a normal transfer - a sweep single (0 change) - to a single subaddress - to 3 subaddresses (i.e. scanning using additional pub keys) * scan machine: option to force reorg avoidance increment first pass - when pointing to a daemon that does not support returning empty blocks when the client requests too high of a height, we have to be careful in our scanner to always request blocks below the chain tip, in every request. - by forcing the reorg avoidance increment on first pass, we make sure clients will always include the reorg avoidance increment when requesting blocks from the daemon, so the client can expect the request for blocks should *always* return an ok height. * core tests: check conversion tool on all legacy enote version types Stil TODO: - check complete scanning on all enote types - hit every branch condition for all enote versions * conn pool mock: epee http client connection pool - Enables concurrent network requests using the epee http client. - Still TODO for production: 1) close_connections 2) require the pool respect max_connections * enote finding context: IN LegacyUnscannedChunk, OUT ChunkData - finds owned enotes by legacy view scanning a chunk of blocks * async: function to remove minimum element from token queue - Useful when we want to remove elements of the token queue in an order that is different than insertion order. * async scanner: scan via RPC, fetching & scanning parallel chunks *How it works* Assume the user's wallet must start scanning blocks from height 5000. 1. The scanner begins by launching 10 RPC requests in parallel to fetch chunks of blocks as follows: ``` request 0: { start_height: 5000, max_block_count: 20 } request 1: { start_height: 5020, max_block_count: 20 } ... request 9: { start_height: 5180, max_block_count: 20 } ``` 2. As soon as any single request completes, the wallet immediately parses that chunk. - This is all in parallel. For example, as soon as request 7 responds, the wallet immediately begins parsing that chunk in parallel to any other chunks it's already parsing. 3. If a chunk does not include a total of max_block_count blocks, and the chunk is not the tip of the chain, this means there was a "gap" in the chunk request. The scanner launches another parallel RPC request to fill in the gap. - This gap can occur because the server will **not** return a chunk of blocks greater than 100mb (or 20k txs) via the /getblocks.bin` RPC endpoint ([`FIND_BLOCKCHAIN_SUPPLEMENT_MAX_SIZE`](https://github.com/monero-project/monero/blob/053ba2cf07649cea8134f8a188685ab7a5365e5c/src/cryptonote_core/blockchain.cpp#L65)) - The gap is from `(req.start_height + res.blocks.size())` to `(req.start_height + req.max_block_count)`. 4. As soon as the scanner finishes parsing the chunk, it immediately submits another parallel RPC request. 5. In parallel, the scanner identifies a user's received (and spent) enotes in each chunk. - For example, as soon as request 7 responds and the wallet parses it, the wallet scans that chunk in parallel to any other chunks it's already scanning. 6. Once a single chunk is fully scanned locally, the scanner launches a parallel task to fetch and scan the next chunk. 7. Once the scanner reaches the tip of the chain (the terminal chunk), the scanner terminates. *Some technical highlights* - The wallet scanner is backwards compatible with existing daemons (though it would need to point to an updated daemon to realize the perf speed-up). - On error cases such as the daemon going offline, the same wallet errors that wallet2 uses (that the wallet API expects) are propagated up to the higher-level Seraphis lib. - The implementation uses an http client connection pool (reusing the epee http client) to support parallel network requests ([related](https://github.com/seraphis-migration/wallet3/issues/58)). - A developer using the scanner can "bring their own blocks/network implementation" to the scanner by providing a callback function of the following type as a param to the async scanner constructor: `std::function` --------- Co-authored-by: jeffro256 --- src/async/CMakeLists.txt | 1 + src/async/mutex.h | 96 ++ src/async/token_queue.h | 32 +- src/common/rpc_client.h | 110 +++ src/cryptonote_core/blockchain.cpp | 10 +- src/ringct/rctTypes.h | 1 + src/rpc/core_rpc_server.cpp | 8 +- src/rpc/core_rpc_server_commands_defs.h | 5 +- src/rpc/daemon_messages.h | 1 + src/seraphis_impl/CMakeLists.txt | 1 + .../enote_finding_context_legacy.cpp | 192 ++++ .../enote_finding_context_legacy.h | 127 +++ src/seraphis_main/enote_finding_context.h | 56 ++ src/seraphis_main/scan_machine.cpp | 25 +- src/seraphis_main/scan_machine_types.h | 2 + src/seraphis_mocks/CMakeLists.txt | 4 + src/seraphis_mocks/mock_http_client_pool.cpp | 85 ++ src/seraphis_mocks/mock_http_client_pool.h | 147 +++ src/seraphis_mocks/mock_ledger_context.cpp | 2 +- .../scan_context_async_mock.cpp | 903 ++++++++++++++++++ src/seraphis_mocks/scan_context_async_mock.h | 242 +++++ src/seraphis_mocks/seraphis_mocks.h | 1 + src/wallet/wallet2.cpp | 16 - src/wallet/wallet2.h | 11 - src/wallet/wallet_errors.h | 32 + tests/core_tests/CMakeLists.txt | 4 + tests/core_tests/chaingen_main.cpp | 7 + tests/core_tests/chaingen_tests_list.h | 1 + tests/core_tests/enote_scanning.cpp | 271 ++++++ tests/core_tests/enote_scanning.h | 138 +++ tests/functional_tests/CMakeLists.txt | 13 +- .../functional_tests/functional_tests_rpc.py | 11 +- tests/functional_tests/main.cpp | 41 +- tests/functional_tests/wallet_scanner.cpp | 638 +++++++++++++ tests/functional_tests/wallet_scanner.h | 183 ++++ 35 files changed, 3359 insertions(+), 58 deletions(-) create mode 100644 src/async/mutex.h create mode 100644 src/seraphis_impl/enote_finding_context_legacy.cpp create mode 100644 src/seraphis_impl/enote_finding_context_legacy.h create mode 100644 src/seraphis_mocks/mock_http_client_pool.cpp create mode 100644 src/seraphis_mocks/mock_http_client_pool.h create mode 100644 src/seraphis_mocks/scan_context_async_mock.cpp create mode 100644 src/seraphis_mocks/scan_context_async_mock.h create mode 100644 tests/core_tests/enote_scanning.cpp create mode 100644 tests/core_tests/enote_scanning.h create mode 100644 tests/functional_tests/wallet_scanner.cpp create mode 100644 tests/functional_tests/wallet_scanner.h diff --git a/src/async/CMakeLists.txt b/src/async/CMakeLists.txt index e1cfd512ee..d2c2dac567 100644 --- a/src/async/CMakeLists.txt +++ b/src/async/CMakeLists.txt @@ -41,6 +41,7 @@ monero_add_library(async target_link_libraries(async PUBLIC common + epee PRIVATE ${EXTRA_LIBRARIES}) diff --git a/src/async/mutex.h b/src/async/mutex.h new file mode 100644 index 0000000000..5c4d0a7bd1 --- /dev/null +++ b/src/async/mutex.h @@ -0,0 +1,96 @@ +// Copyright (c) 2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +/// mutex + +#pragma once + +//local headers +#include "misc_language.h" +#include "misc_log_ex.h" + +//third-party headers + +//standard headers +#include +#include +#include +#include + +//forward declarations + + +// Lock the mutex and then unlock it at the end of the local scope +// - if it fails to unlock (either it's already unlocked or owned by a different thread), the exception is ignored +#define SCOPE_LOCK_MUTEX(mutex) \ + mutex.lock(); \ + auto scope_exit_handler_##mutex = epee::misc_utils::create_scope_leave_handler([this](){ \ + CHECK_AND_ASSERT_THROW_MES(mutex.unlock(), "failed to unlock " + std::string(#mutex)); \ + }) + +namespace async +{ + +/// mutex +class Mutex final +{ +public: + /// disable copy/move + Mutex& operator=(Mutex&&) = delete; + +//member functions + /// Lock the mutex and claim ownership + void lock() + { + m_mutex.lock(); + m_mutex_owner.store(std::this_thread::get_id(), std::memory_order_relaxed); + } + + /// Release ownership and unlock the mutex. If this thread does not own the lock already, returns false. + bool unlock() + { + if (!thread_owns_lock()) + return false; + m_mutex_owner.store(std::thread::id{}, std::memory_order_relaxed); + m_mutex.unlock(); + return true; + } + + /// Check if the given thread owns the mutex + bool thread_owns_lock(const std::thread::id thread_id = std::this_thread::get_id()) const + { + return thread_id == m_mutex_owner.load(std::memory_order_relaxed); + } + +private: +//member variables + std::mutex m_mutex; + std::atomic m_mutex_owner{std::thread::id{}}; +}; + +} //namespace asyc diff --git a/src/async/token_queue.h b/src/async/token_queue.h index 4e72fefa24..4eaaecdd64 100644 --- a/src/async/token_queue.h +++ b/src/async/token_queue.h @@ -51,7 +51,8 @@ enum class TokenQueueResult : unsigned char SUCCESS, QUEUE_EMPTY, TRY_LOCK_FAIL, - SHUTTING_DOWN + SHUTTING_DOWN, + QUEUE_NOT_EMPTY }; /// async token queue @@ -116,7 +117,27 @@ class TokenQueue final m_queue.pop_front(); return TokenQueueResult::SUCCESS; } + /// try to remove the minimum element + TokenQueueResult try_remove_min(TokenT &token_out) + { + // try to lock the queue, then check if there are any elements + std::unique_lock lock{m_mutex, std::try_to_lock}; + if (!lock.owns_lock()) + return TokenQueueResult::TRY_LOCK_FAIL; + if (m_queue.size() == 0) + return TokenQueueResult::QUEUE_EMPTY; + // find the min element + auto min_elem = m_queue.begin(); + for (auto it = m_queue.begin(); it != m_queue.end(); ++it) + { + if (*it < *min_elem) + min_elem = it; + } + token_out = std::move(*min_elem); + m_queue.erase(min_elem); + return TokenQueueResult::SUCCESS; + } /// shut down the queue void shut_down() { @@ -126,6 +147,15 @@ class TokenQueue final } m_condvar.notify_all(); } + /// reset the queue (queue must already be empty) + TokenQueueResult reset() + { + std::lock_guard lock{m_mutex}; + if (!m_queue.empty()) + return TokenQueueResult::QUEUE_NOT_EMPTY; + m_is_shutting_down = false; + return TokenQueueResult::SUCCESS; + } private: //member variables diff --git a/src/common/rpc_client.h b/src/common/rpc_client.h index 4bc75a1be9..7a206b4c92 100644 --- a/src/common/rpc_client.h +++ b/src/common/rpc_client.h @@ -59,6 +59,18 @@ namespace tools ); } + t_rpc_client( + const std::string &daemon_addr + , boost::optional user + , epee::net_utils::ssl_options_t ssl_options + ) + : m_http_client{} + { + m_http_client.set_server( + daemon_addr, std::move(user), std::move(ssl_options) + ); + } + template bool basic_json_rpc_request( T_req & req @@ -114,6 +126,33 @@ namespace tools } } + template + bool basic_rpc_request( + T_req & req + , T_res & res + , std::string const & relative_url + ) + { + t_http_connection connection(&m_http_client); + + bool ok = connection.is_open(); + if (!ok) + { + fail_msg_writer() << "Couldn't connect to daemon: " << m_http_client.get_host() << ":" << m_http_client.get_port(); + return false; + } + ok = epee::net_utils::invoke_http_json(relative_url, req, res, m_http_client, t_http_connection::TIMEOUT()); + if (!ok) + { + fail_msg_writer() << "basic_rpc_request: Daemon request failed"; + return false; + } + else + { + return true; + } + } + template bool rpc_request( T_req & req @@ -148,4 +187,75 @@ namespace tools return connection.is_open(); } }; + + class t_daemon_rpc_client final + { + private: + t_rpc_client m_rpc_client; + public: + t_daemon_rpc_client( + const std::string &daemon_addr + , const boost::optional &daemon_login + , const epee::net_utils::ssl_options_t ssl_support + ) + : m_rpc_client{daemon_addr, daemon_login, ssl_support} + { + } + + cryptonote::COMMAND_RPC_GET_HEIGHT::response get_height() + { + cryptonote::COMMAND_RPC_GET_HEIGHT::request req = AUTO_VAL_INIT(req); + cryptonote::COMMAND_RPC_GET_HEIGHT::response res = AUTO_VAL_INIT(res); + CHECK_AND_ASSERT_THROW_MES(m_rpc_client.basic_rpc_request(req, res, "/get_height"), "failed to get height"); + return res; + } + + cryptonote::COMMAND_RPC_POP_BLOCKS::response pop_blocks(uint64_t nblocks = 1) + { + cryptonote::COMMAND_RPC_POP_BLOCKS::request req = AUTO_VAL_INIT(req); + cryptonote::COMMAND_RPC_POP_BLOCKS::response res = AUTO_VAL_INIT(res); + req.nblocks = nblocks; + CHECK_AND_ASSERT_THROW_MES(m_rpc_client.basic_rpc_request(req, res, "/pop_blocks"), "failed to pop blocks"); + return res; + } + + cryptonote::COMMAND_RPC_GET_TRANSACTIONS::response get_transactions(const std::vector &txs_hashes) + { + cryptonote::COMMAND_RPC_GET_TRANSACTIONS::request req = AUTO_VAL_INIT(req); + cryptonote::COMMAND_RPC_GET_TRANSACTIONS::response res = AUTO_VAL_INIT(res); + req.txs_hashes = txs_hashes; + req.decode_as_json = false; + CHECK_AND_ASSERT_THROW_MES(m_rpc_client.basic_rpc_request(req, res, "/get_transactions"), "failed to get transactions"); + return res; + } + + cryptonote::COMMAND_RPC_FLUSH_TRANSACTION_POOL::response flush_txpool() + { + cryptonote::COMMAND_RPC_FLUSH_TRANSACTION_POOL::request req = AUTO_VAL_INIT(req); + cryptonote::COMMAND_RPC_FLUSH_TRANSACTION_POOL::response res = AUTO_VAL_INIT(res); + CHECK_AND_ASSERT_THROW_MES(m_rpc_client.basic_json_rpc_request(req, res, "flush_txpool"), "failed to flush txpool"); + return res; + } + + cryptonote::COMMAND_RPC_GENERATEBLOCKS::response generateblocks(const std::string &address, uint64_t amount_of_blocks) + { + cryptonote::COMMAND_RPC_GENERATEBLOCKS::request req = AUTO_VAL_INIT(req); + cryptonote::COMMAND_RPC_GENERATEBLOCKS::response res = AUTO_VAL_INIT(res); + req.amount_of_blocks = amount_of_blocks; + req.wallet_address = address; + req.prev_block = ""; + req.starting_nonce = 0; + CHECK_AND_ASSERT_THROW_MES(m_rpc_client.basic_json_rpc_request(req, res, "generateblocks"), "failed to generate blocks"); + return res; + } + + cryptonote::COMMAND_RPC_GET_LAST_BLOCK_HEADER::response get_last_block_header() + { + cryptonote::COMMAND_RPC_GET_LAST_BLOCK_HEADER::request req = AUTO_VAL_INIT(req); + cryptonote::COMMAND_RPC_GET_LAST_BLOCK_HEADER::response res = AUTO_VAL_INIT(res); + req.client = ""; + CHECK_AND_ASSERT_THROW_MES(m_rpc_client.basic_json_rpc_request(req, res, "get_last_block_header"), "failed to get last block header"); + return res; + } + }; } diff --git a/src/cryptonote_core/blockchain.cpp b/src/cryptonote_core/blockchain.cpp index 8d34f0e858..fe33757e4d 100644 --- a/src/cryptonote_core/blockchain.cpp +++ b/src/cryptonote_core/blockchain.cpp @@ -2774,8 +2774,9 @@ bool Blockchain::find_blockchain_supplement(const uint64_t req_start_block, cons if(req_start_block > 0) { // if requested height is higher than our chain, return false -- we can't help - top_hash = m_db->top_block_hash(&total_height); - ++total_height; + uint64_t top_height; + top_hash = m_db->top_block_hash(&top_height); + total_height = top_height + 1; if (req_start_block >= total_height) { return false; @@ -2791,8 +2792,9 @@ bool Blockchain::find_blockchain_supplement(const uint64_t req_start_block, cons } db_rtxn_guard rtxn_guard(m_db); - top_hash = m_db->top_block_hash(&total_height); - ++total_height; + uint64_t top_height; + top_hash = m_db->top_block_hash(&top_height); + total_height = top_height + 1; blocks.reserve(std::min(std::min(max_block_count, (size_t)10000), (size_t)(total_height - start_height))); CHECK_AND_ASSERT_MES(m_db->get_blocks_from(start_height, 3, max_block_count, max_tx_count, FIND_BLOCKCHAIN_SUPPLEMENT_MAX_SIZE, blocks, pruned, true, get_miner_tx_hash), false, "Error getting blocks"); diff --git a/src/ringct/rctTypes.h b/src/ringct/rctTypes.h index 247f25fffb..20b952c5e6 100644 --- a/src/ringct/rctTypes.h +++ b/src/ringct/rctTypes.h @@ -84,6 +84,7 @@ namespace rct { return bytes[i]; } bool operator==(const key &k) const { return !crypto_verify_32(bytes, k.bytes); } + bool operator!=(const key &k) const { return crypto_verify_32(bytes, k.bytes); } unsigned char bytes[32]; }; typedef std::vector keyV; //vector of keys diff --git a/src/rpc/core_rpc_server.cpp b/src/rpc/core_rpc_server.cpp index 7fcd1e6d7c..35a3d5bb91 100644 --- a/src/rpc/core_rpc_server.cpp +++ b/src/rpc/core_rpc_server.cpp @@ -725,12 +725,12 @@ namespace cryptonote } } - size_t max_blocks = COMMAND_RPC_GET_BLOCKS_FAST_MAX_BLOCK_COUNT; + size_t max_blocks = req.max_block_count > 0 + ? std::min(req.max_block_count, (uint64_t)COMMAND_RPC_GET_BLOCKS_FAST_MAX_BLOCK_COUNT) + : COMMAND_RPC_GET_BLOCKS_FAST_MAX_BLOCK_COUNT; if (m_rpc_payment) { - max_blocks = res.credits / COST_PER_BLOCK; - if (max_blocks > COMMAND_RPC_GET_BLOCKS_FAST_MAX_BLOCK_COUNT) - max_blocks = COMMAND_RPC_GET_BLOCKS_FAST_MAX_BLOCK_COUNT; + max_blocks = std::min((size_t)(res.credits / COST_PER_BLOCK), max_blocks); if (max_blocks == 0) { res.status = CORE_RPC_STATUS_PAYMENT_REQUIRED; diff --git a/src/rpc/core_rpc_server_commands_defs.h b/src/rpc/core_rpc_server_commands_defs.h index 1c8525d453..15a84a13e6 100644 --- a/src/rpc/core_rpc_server_commands_defs.h +++ b/src/rpc/core_rpc_server_commands_defs.h @@ -80,7 +80,7 @@ namespace cryptonote #define CORE_RPC_STATUS_NOT_MINING "NOT MINING" #define CORE_RPC_STATUS_PAYMENT_REQUIRED "PAYMENT REQUIRED" -inline const std::string get_rpc_status(const bool trusted_daemon, const std::string &s) +static inline const std::string get_rpc_status(const bool trusted_daemon, const std::string &s) { if (trusted_daemon) return s; @@ -191,6 +191,8 @@ inline const std::string get_rpc_status(const bool trusted_daemon, const std::st bool no_miner_tx; bool high_height_ok; uint64_t pool_info_since; + uint64_t max_block_count; + BEGIN_KV_SERIALIZE_MAP() KV_SERIALIZE_PARENT(rpc_access_request_base) KV_SERIALIZE_OPT(requested_info, (uint8_t)0) @@ -200,6 +202,7 @@ inline const std::string get_rpc_status(const bool trusted_daemon, const std::st KV_SERIALIZE_OPT(no_miner_tx, false) KV_SERIALIZE_OPT(high_height_ok, false) // default false maintains backwards compatibility for clients that relied on failure on high height KV_SERIALIZE_OPT(pool_info_since, (uint64_t)0) + KV_SERIALIZE_OPT(max_block_count, (uint64_t)0) END_KV_SERIALIZE_MAP() }; typedef epee::misc_utils::struct_init request; diff --git a/src/rpc/daemon_messages.h b/src/rpc/daemon_messages.h index 83546fce3d..27dcc191f1 100644 --- a/src/rpc/daemon_messages.h +++ b/src/rpc/daemon_messages.h @@ -92,6 +92,7 @@ BEGIN_RPC_MESSAGE_CLASS(GetBlocksFast); RPC_MESSAGE_MEMBER(std::list, block_ids); RPC_MESSAGE_MEMBER(uint64_t, start_height); RPC_MESSAGE_MEMBER(bool, prune); + RPC_MESSAGE_MEMBER(uint64_t, max_block_count); END_RPC_MESSAGE_REQUEST; BEGIN_RPC_MESSAGE_RESPONSE; RPC_MESSAGE_MEMBER(std::vector, blocks); diff --git a/src/seraphis_impl/CMakeLists.txt b/src/seraphis_impl/CMakeLists.txt index bee0f2ca75..f5bd8aa066 100644 --- a/src/seraphis_impl/CMakeLists.txt +++ b/src/seraphis_impl/CMakeLists.txt @@ -28,6 +28,7 @@ set(seraphis_impl_sources checkpoint_cache.cpp + enote_finding_context_legacy.cpp enote_store.cpp enote_store_payment_validator.cpp enote_store_utils.cpp diff --git a/src/seraphis_impl/enote_finding_context_legacy.cpp b/src/seraphis_impl/enote_finding_context_legacy.cpp new file mode 100644 index 0000000000..fe3bc880f7 --- /dev/null +++ b/src/seraphis_impl/enote_finding_context_legacy.cpp @@ -0,0 +1,192 @@ +// Copyright (c) 2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +//paired header +#include "enote_finding_context_legacy.h" + +//local headers +#include "async/misc_utils.h" +#include "device/device.hpp" +#include "seraphis_main/contextual_enote_record_types.h" +#include "seraphis_main/scan_balance_recovery_utils.h" + +//third party headers + +//standard headers + + +#undef MONERO_DEFAULT_LOG_CATEGORY +#define MONERO_DEFAULT_LOG_CATEGORY "seraphis" + +namespace sp +{ +//------------------------------------------------------------------------------------------------------------------- +void EnoteFindingContextLegacySimple::view_scan_chunk(const LegacyUnscannedChunk &legacy_unscanned_chunk, + sp::scanning::ChunkData &chunk_data_out) const +{ + for (const auto &blk : legacy_unscanned_chunk) + { + for (const auto &tx : blk.unscanned_txs) + { + // Identify owned enotes + if (tx.enotes.size() > 0) + { + std::list collected_records; + sp::scanning::try_find_legacy_enotes_in_tx(m_legacy_base_spend_pubkey, + m_legacy_subaddress_map, + m_legacy_view_privkey, + blk.block_index, + blk.block_timestamp, + tx.transaction_id, + tx.total_enotes_before_tx, + tx.unlock_time, + tx.tx_memo, + tx.enotes, + sp::SpEnoteOriginStatus::ONCHAIN, + hw::get_device("default"), + collected_records); + + chunk_data_out.basic_records_per_tx[tx.transaction_id] = std::move(collected_records); + } + else + { + // always add an entry for tx in the legacy basic records map (since we save key images for every tx) + chunk_data_out.basic_records_per_tx[tx.transaction_id] = std::list{}; + } + + // Collect key images + sp::SpContextualKeyImageSetV1 collected_key_images; + if (sp::scanning::try_collect_key_images_from_tx(blk.block_index, + blk.block_timestamp, + tx.transaction_id, + tx.legacy_key_images, + std::vector()/*sp_key_images*/, + sp::SpEnoteSpentStatus::SPENT_ONCHAIN, + collected_key_images)) + { + chunk_data_out.contextual_key_images.emplace_back(std::move(collected_key_images)); + } + } + } +} +//------------------------------------------------------------------------------------------------------------------- +void EnoteFindingContextLegacyMultithreaded::view_scan_chunk( + const LegacyUnscannedChunk &legacy_unscanned_chunk, + sp::scanning::ChunkData &chunk_data_out) const +{ + // 1. make join signal + async::join_signal_t join_signal{m_threadpool.make_join_signal()}; + + // 2. get join token + async::join_token_t join_token{m_threadpool.get_join_token(join_signal)}; + + // 3. prepare vector we'll use to collect results across threads + std::uint64_t num_txs = 0; + for (const auto &blk : legacy_unscanned_chunk) + num_txs += blk.unscanned_txs.size(); + std::vector>> basic_records_per_tx; + basic_records_per_tx.resize(num_txs); + + // 4. submit tasks to join on + std::uint64_t idx = 0; + for (const auto &blk : legacy_unscanned_chunk) + { + for (const auto &tx : blk.unscanned_txs) + { + // Identify owned enotes + if (tx.enotes.size() > 0) + { + auto task = + [ + this, + &blk, + &tx, + &basic_records_per_tx, + l_idx = idx, + l_join_token = join_token + ]() -> async::TaskVariant + { + std::list collected_records; + sp::scanning::try_find_legacy_enotes_in_tx(m_legacy_base_spend_pubkey, + m_legacy_subaddress_map, + m_legacy_view_privkey, + blk.block_index, + blk.block_timestamp, + tx.transaction_id, + tx.total_enotes_before_tx, + tx.unlock_time, + tx.tx_memo, + tx.enotes, + sp::SpEnoteOriginStatus::ONCHAIN, + hw::get_device("default"), + collected_records); + + basic_records_per_tx[l_idx] = {tx.transaction_id, std::move(collected_records)}; + return boost::none; + }; + + // submit to the threadpool + m_threadpool.submit(async::make_simple_task(async::DefaultPriorityLevels::MEDIUM, std::move(task))); + } + else + { + // always add an entry for tx in the legacy basic records map (since we save key images for every tx) + basic_records_per_tx[idx] = {tx.transaction_id, std::list{}}; + } + + // Collect key images + sp::SpContextualKeyImageSetV1 collected_key_images; + if (sp::scanning::try_collect_key_images_from_tx(blk.block_index, + blk.block_timestamp, + tx.transaction_id, + tx.legacy_key_images, + std::vector()/*sp_key_images*/, + sp::SpEnoteSpentStatus::SPENT_ONCHAIN, + collected_key_images)) + { + chunk_data_out.contextual_key_images.emplace_back(std::move(collected_key_images)); + } + + ++idx; + } + } + + // 5. get join condition + async::join_condition_t join_condition{ + m_threadpool.get_join_condition(std::move(join_signal), std::move(join_token)) + }; + + // 6. join the tasks + m_threadpool.work_while_waiting(std::move(join_condition)); + + // 7. set results + for (auto &brpt : basic_records_per_tx) + chunk_data_out.basic_records_per_tx.emplace(std::move(brpt)); +} +//------------------------------------------------------------------------------------------------------------------- +} //namespace sp diff --git a/src/seraphis_impl/enote_finding_context_legacy.h b/src/seraphis_impl/enote_finding_context_legacy.h new file mode 100644 index 0000000000..40a7faaa30 --- /dev/null +++ b/src/seraphis_impl/enote_finding_context_legacy.h @@ -0,0 +1,127 @@ +// Copyright (c) 2022, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// Dependency injectors for the find-received step of enote scanning. Intended to be stateless. + +#pragma once + +//local headers +#include "async/threadpool.h" +#include "crypto/crypto.h" +#include "cryptonote_basic/subaddress_index.h" +#include "ringct/rctTypes.h" +#include "seraphis_main/enote_finding_context.h" +#include "seraphis_main/scan_core_types.h" + +//third party headers + +//standard headers +#include +#include + +//forward declarations + + +namespace sp +{ + +//// +// EnoteFindingContextLegacySimple +// - find owned enotes from legacy view scanning using actual chain data +// - scans each tx in a chunk of blocks serially in order +/// +class EnoteFindingContextLegacySimple final : public sp::EnoteFindingContextLegacy +{ +public: +//constructors + EnoteFindingContextLegacySimple(const rct::key &legacy_base_spend_pubkey, + const std::unordered_map &legacy_subaddress_map, + const crypto::secret_key &legacy_view_privkey) : + m_legacy_base_spend_pubkey{legacy_base_spend_pubkey}, + m_legacy_subaddress_map{legacy_subaddress_map}, + m_legacy_view_privkey{legacy_view_privkey} + { + } + +//overloaded operators + /// disable copy/move (this is a scoped manager [reference wrapper]) + EnoteFindingContextLegacySimple& operator=(sp::EnoteFindingContextLegacy&&) = delete; + +//member functions + /// scans a chunk of blocks to find basic enote records + void view_scan_chunk(const LegacyUnscannedChunk &legacy_unscanned_chunk, + sp::scanning::ChunkData &chunk_data_out) const override; + +//member variables +private: + const rct::key &m_legacy_base_spend_pubkey; + // TODO: implement subaddress lookahead + const std::unordered_map &m_legacy_subaddress_map; + const crypto::secret_key &m_legacy_view_privkey; +}; + +//// +// EnoteFindingContextLegacyMultithreaded +// - find owned enotes from legacy view scanning using actual chain data +// - scanning each individual tx is a task that gets submitted to threadpool +/// +class EnoteFindingContextLegacyMultithreaded final : public sp::EnoteFindingContextLegacy +{ +public: +//constructors + EnoteFindingContextLegacyMultithreaded(const rct::key &legacy_base_spend_pubkey, + const std::unordered_map &legacy_subaddress_map, + const crypto::secret_key &legacy_view_privkey, + async::Threadpool &threadpool) : + m_legacy_base_spend_pubkey{legacy_base_spend_pubkey}, + m_legacy_subaddress_map{legacy_subaddress_map}, + m_legacy_view_privkey{legacy_view_privkey}, + m_threadpool(threadpool) + { + } + +//overloaded operators + /// disable copy/move (this is a scoped manager [reference wrapper]) + EnoteFindingContextLegacyMultithreaded& operator=(EnoteFindingContextLegacyMultithreaded&&) = delete; + +//member functions + /// scans a chunk of blocks to find basic enote records + void view_scan_chunk(const LegacyUnscannedChunk &legacy_unscanned_chunk, + sp::scanning::ChunkData &chunk_data_out) const override; + +//member variables +private: + const rct::key &m_legacy_base_spend_pubkey; + // TODO: implement subaddress lookahead + const std::unordered_map &m_legacy_subaddress_map; + const crypto::secret_key &m_legacy_view_privkey; + + async::Threadpool &m_threadpool; +}; + +} //namespace sp diff --git a/src/seraphis_main/enote_finding_context.h b/src/seraphis_main/enote_finding_context.h index b1c10dfc89..5bee7b5617 100644 --- a/src/seraphis_main/enote_finding_context.h +++ b/src/seraphis_main/enote_finding_context.h @@ -31,6 +31,9 @@ #pragma once //local headers +#include "async/threadpool.h" +#include "crypto/crypto.h" +#include "ringct/rctTypes.h" #include "seraphis_main/scan_core_types.h" #include "seraphis_main/scan_ledger_chunk.h" @@ -84,4 +87,57 @@ class EnoteFindingContextLedger const std::uint64_t chunk_max_size) const = 0; }; +//// +/// LegacyUnscannedTransaction: a transaction that is ready to be legacy view scanned +/// +struct LegacyUnscannedTransaction final +{ + rct::key transaction_id; + uint64_t unlock_time; + sp::TxExtra tx_memo; + uint64_t total_enotes_before_tx; + std::vector enotes; + std::vector legacy_key_images; +}; + +//// +/// LegacyUnscannedBlock: a block that is ready to be legacy view scanned +// - the txs are expected to be ordered as they appear in the block, where +// the first tx is the miner tx +/// +struct LegacyUnscannedBlock final +{ + uint64_t block_index; + uint64_t block_timestamp; + rct::key block_hash; + rct::key prev_block_hash; + std::vector unscanned_txs; +}; + +//// +/// LegacyUnscannedChunk: a chunk of blocks ready to be legacy view scanned +// - the blocks are expected to match their order on-chain +/// +using LegacyUnscannedChunk = std::vector; + +//// +// EnoteFindingContextLegacy +// - takes in chunks of blocks and produces chunks of owned enotes (from view scanning) +/// +class EnoteFindingContextLegacy +{ +public: +//destructor + virtual ~EnoteFindingContextLegacy() = default; + +//overloaded operators + /// disable copy/move (this is a virtual base class) + EnoteFindingContextLegacy& operator=(EnoteFindingContextLegacy&&) = delete; + +//member functions + /// scans a chunk of blocks to find basic enote records + virtual void view_scan_chunk(const LegacyUnscannedChunk &legacy_unscanned_chunk, + sp::scanning::ChunkData &chunk_data_out) const = 0; +}; + } //namespace sp diff --git a/src/seraphis_main/scan_machine.cpp b/src/seraphis_main/scan_machine.cpp index bdb02b9222..1243ee0aed 100644 --- a/src/seraphis_main/scan_machine.cpp +++ b/src/seraphis_main/scan_machine.cpp @@ -71,13 +71,22 @@ enum class ContiguityCheckResult : unsigned char // fixed back-off were used it could take many fullscan attempts to find the point of divergence //------------------------------------------------------------------------------------------------------------------- static std::uint64_t get_reorg_avoidance_depth(const std::uint64_t reorg_avoidance_increment, + const bool force_reorg_avoidance_increment, const std::uint64_t num_reorg_avoidance_backoffs) { - // 1. start at a depth of zero + // 1. start at a depth of zero (unless `force_reorg_avoidance_increment` is true) // - this allows us to avoid accidentally reorging your data store if the scanning backend only has a portion // of the blocks in your initial reorg avoidance depth range available when 'get chunk' is called (in the case // where there wasn't actually a reorg and the backend is just catching up) - if (num_reorg_avoidance_backoffs == 0) + // - for example, if your refresh index is 100, your start index is 150, and your avoidance increment is 10, if + // you start scanning but your scan context has only processed blocks 0-145 (out of let's say 200), then if we + // include a reorg avoidance backoff for the first processed chunk, then blocks [140, ..) will be requested + // from the scan context and only [140, 145] might be returned; when the chunk consumer gets those blocks, it + // will think blocks 145-150 were reorged away; a scan context *could* avoid returning a partial chunk if it + // knows there are unprocessed blocks in the chunk, but for robustness we can't assume all contexts will do so + // - `force_reorg_avoidance_increment` is useful when pointing to a daemon that fails if we request a start block + // higher than chain tip, so our initial request for blocks starts at a block the daemon knows + if (!force_reorg_avoidance_increment && num_reorg_avoidance_backoffs == 0) return 0; // 2. check that the increment is not 0 @@ -87,18 +96,22 @@ static std::uint64_t get_reorg_avoidance_depth(const std::uint64_t reorg_avoidan "reorg avoidance increment."); // 3. 10 ^ (num requests - 1) * increment - return math::saturating_mul(math::uint_pow(10, num_reorg_avoidance_backoffs - 1), reorg_avoidance_increment, -1); + const std::uint64_t exponent = math::saturating_sub(num_reorg_avoidance_backoffs, 1, 0); + return math::saturating_mul(math::uint_pow(10, exponent), reorg_avoidance_increment, -1); } //------------------------------------------------------------------------------------------------------------------- //------------------------------------------------------------------------------------------------------------------- static std::uint64_t get_estimated_start_scan_index(const std::uint64_t reorg_avoidance_increment, + const bool force_reorg_avoidance_increment, const std::uint64_t num_reorg_avoidance_backoffs, const std::uint64_t lowest_scannable_index, const std::uint64_t desired_start_index) { // 1. set reorg avoidance depth const std::uint64_t reorg_avoidance_depth{ - get_reorg_avoidance_depth(reorg_avoidance_increment, num_reorg_avoidance_backoffs) + get_reorg_avoidance_depth(reorg_avoidance_increment, + force_reorg_avoidance_increment, + num_reorg_avoidance_backoffs) }; // 2. initial block to scan = max(desired first block - reorg depth, chunk consumer's min scan index) @@ -107,6 +120,7 @@ static std::uint64_t get_estimated_start_scan_index(const std::uint64_t reorg_av //------------------------------------------------------------------------------------------------------------------- //------------------------------------------------------------------------------------------------------------------- static void set_initial_contiguity_marker(const std::uint64_t reorg_avoidance_increment, + const bool force_reorg_avoidance_increment, const std::uint64_t num_reorg_avoidance_backoffs, const ChunkConsumer &chunk_consumer, ContiguityMarker &contiguity_marker_out) @@ -115,6 +129,7 @@ static void set_initial_contiguity_marker(const std::uint64_t reorg_avoidance_in // - this is only an estimate since the chunk consumer may not have the block at this exact index cached const std::uint64_t estimated_start_scan_index{ get_estimated_start_scan_index(reorg_avoidance_increment, + force_reorg_avoidance_increment, num_reorg_avoidance_backoffs, chunk_consumer.refresh_index(), chunk_consumer.desired_first_block()) @@ -435,6 +450,7 @@ static ScanMachineState handle_need_fullscan(const ScanMachineNeedFullscan &stat // 1. set initial contiguity marker ContiguityMarker start_scan_contiguity_marker; set_initial_contiguity_marker(state.metadata.config.reorg_avoidance_increment, + state.metadata.config.force_reorg_avoidance_increment, state.metadata.fullscan_attempts, //exponential backoff as function of fullscan attempts, starting at 0 chunk_consumer, start_scan_contiguity_marker); @@ -464,6 +480,7 @@ static ScanMachineState handle_need_partialscan(const ScanMachineNeedPartialscan // 1. set initial contiguity marker ContiguityMarker start_scan_contiguity_marker; set_initial_contiguity_marker(state.metadata.config.reorg_avoidance_increment, + state.metadata.config.force_reorg_avoidance_increment, 1, //in partial scans always back off by just one reorg avoidance increment chunk_consumer, start_scan_contiguity_marker); diff --git a/src/seraphis_main/scan_machine_types.h b/src/seraphis_main/scan_machine_types.h index b14d556cb9..448e9249e5 100644 --- a/src/seraphis_main/scan_machine_types.h +++ b/src/seraphis_main/scan_machine_types.h @@ -75,6 +75,8 @@ struct ScanMachineConfig final /// increment for avoiding reorgs /// - each fullscan attempt looks (10^attempts * increment) blocks below the requested start index std::uint64_t reorg_avoidance_increment{10}; + /// force using the reorg_avoidance_increment from the start + bool force_reorg_avoidance_increment{false}; /// max number of blocks per ledger chunk /// - this is only a hint, the downstream scanning context is free to ignore it std::uint64_t max_chunk_size_hint{100}; diff --git a/src/seraphis_mocks/CMakeLists.txt b/src/seraphis_mocks/CMakeLists.txt index ed4796678c..19e01ec48c 100644 --- a/src/seraphis_mocks/CMakeLists.txt +++ b/src/seraphis_mocks/CMakeLists.txt @@ -32,6 +32,7 @@ set(seraphis_mocks_sources jamtis_mock_keys.cpp legacy_mock_keys.cpp make_mock_tx.cpp + mock_http_client_pool.cpp mock_ledger_context.cpp mock_offchain_context.cpp mock_send_receive.cpp @@ -39,6 +40,7 @@ set(seraphis_mocks_sources mock_tx_builders_legacy_inputs.cpp mock_tx_builders_outputs.cpp scan_chunk_consumer_mocks.cpp + scan_context_async_mock.cpp tx_input_selector_mocks.cpp) monero_find_all_headers(seraphis_mocks_headers, "${CMAKE_CURRENT_SOURCE_DIR}") @@ -49,11 +51,13 @@ monero_add_library(seraphis_mocks target_link_libraries(seraphis_mocks PUBLIC + async cncrypto common cryptonote_basic device epee + net ringct seraphis_core seraphis_crypto diff --git a/src/seraphis_mocks/mock_http_client_pool.cpp b/src/seraphis_mocks/mock_http_client_pool.cpp new file mode 100644 index 0000000000..9fcfb767a8 --- /dev/null +++ b/src/seraphis_mocks/mock_http_client_pool.cpp @@ -0,0 +1,85 @@ +// Copyright (c) 2022, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// NOT FOR PRODUCTION + +//local headers +#include "mock_http_client_pool.h" + +//third party headers + +//standard headers + +//forward declarations + + +namespace sp +{ +namespace mocks +{ +//------------------------------------------------------------------------------------------------------------------- +std::size_t ClientConnectionPool::acquire_unused_http_client() +{ + std::lock_guard lock{m_http_client_pool_mutex}; + + // TODO: if m_http_client_pool.size() >= m_max_connections, wait until unused + + // Check if we have any clients available for use + for (std::size_t i = 0; i < m_http_client_pool.size(); ++i) + { + if (m_http_client_pool[i].in_use) + continue; + + m_http_client_pool[i].in_use = true; + return i; + } + + // If we don't find an unused client, add a new one to the pool + m_http_client_pool.emplace_back(pool_http_client_t{ + .in_use = true, + .http_client = std::unique_ptr(new net::http::client()) + }); + + if (!m_proxy.empty()) + m_http_client_pool.back().http_client->set_proxy(m_proxy); + m_http_client_pool.back().http_client->set_server(m_daemon_address, m_daemon_login, m_ssl_support); + + return m_http_client_pool.size() - 1; +} +//------------------------------------------------------------------------------------------------------------------- +void ClientConnectionPool::release_http_client(size_t http_client_index) +{ + // Make the connection available for use again + std::lock_guard lock{m_http_client_pool_mutex}; + CHECK_AND_ASSERT_THROW_MES(m_http_client_pool.size() > http_client_index, + "http client connection is unknown to the pool"); + m_http_client_pool[http_client_index].in_use = false; +} +//------------------------------------------------------------------------------------------------------------------- +} //namespace mocks +} //namespace sp diff --git a/src/seraphis_mocks/mock_http_client_pool.h b/src/seraphis_mocks/mock_http_client_pool.h new file mode 100644 index 0000000000..97951e21fd --- /dev/null +++ b/src/seraphis_mocks/mock_http_client_pool.h @@ -0,0 +1,147 @@ +// Copyright (c) 2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// NOT FOR PRODUCTION +// TODO for production-ready: +// 1. Implement close_connections. +// 2. Require the pool respect max_connections. + +#pragma once + +//local headers +#include "misc_language.h" +#include "net/http.h" +#include "net/http_client.h" +#include "rpc/core_rpc_server_commands_defs.h" +#include "storages/http_abstract_invoke.h" + +//third party headers + +//standard headers +#include +#include +#include + +//forward declarations + + +namespace sp +{ +namespace mocks +{ +//// +// ClientConnectionPool +// - wraps a pool of network client connections to enable concurrent requests +/// +class ClientConnectionPool +{ +public: +//constructors + ClientConnectionPool( + const std::string &daemon_address, + const boost::optional daemon_login = boost::none, + const epee::net_utils::ssl_options_t ssl_support = epee::net_utils::ssl_support_t::e_ssl_support_autodetect, + const std::string &proxy = "", + const std::size_t max_connections = 20): + m_daemon_address{daemon_address}, + m_daemon_login{daemon_login}, + m_ssl_support(ssl_support), + m_proxy{proxy}, + m_max_connections{max_connections} + { + m_http_client_pool.reserve(max_connections); + } + + ~ClientConnectionPool() + { + this->close_connections(); + } + +//member functions + enum http_mode { JSON, BIN, JSON_RPC }; + + /// Use an http client from the pool to make an RPC request to the daemon + template + bool rpc_command(const http_mode &mode, + const std::string &command_name, + const typename COMMAND_TYPE::request &req, + typename COMMAND_TYPE::response &res) + { + // Acquire an http client from the connection pool + std::size_t http_client_index = this->acquire_unused_http_client(); + CHECK_AND_ASSERT_THROW_MES(m_http_client_pool.size() > http_client_index, "http client index is too high"); + auto scope_exit_handler = epee::misc_utils::create_scope_leave_handler([this, http_client_index]{ + this->release_http_client(http_client_index); + }); + + // Do the RPC command + LOG_PRINT_L2("Invoking " << command_name << " with http client " << http_client_index); + pool_http_client_t &http_client = m_http_client_pool[http_client_index]; + switch (mode) + { + case BIN: return epee::net_utils::invoke_http_bin(command_name, req, res, *http_client.http_client); + case JSON: return epee::net_utils::invoke_http_json(command_name, req, res, *http_client.http_client); + case JSON_RPC: return epee::net_utils::invoke_http_json_rpc("/json_rpc", command_name, req, res, *http_client.http_client); + default: { MERROR("Unknown http_mode: " << mode); return false; } + } + } + + /** + * brief: *close_connections* - close connections except num specified + * param: max_keep_alive - the max number of connections to keep alive (0 means close all) + */ + // TODO: implement + void close_connections(const std::size_t max_keep_alive = 0) { /* no-op */ }; + +private: + /// If an http client is available, acquires it. If none are available, initializes a new http client. + std::size_t acquire_unused_http_client(); + + /// Make http client available for use again. + void release_http_client(std::size_t http_client_index); + +//member variables +private: + const std::string m_daemon_address; + const boost::optional m_daemon_login; + const epee::net_utils::ssl_options_t m_ssl_support; + const std::string m_proxy; + + const std::size_t m_max_connections; + + struct pool_http_client_t { + bool in_use; + std::unique_ptr http_client; + }; + + // TODO: investigate possibility of removing the need for this internal mutex and avoid lock contention + mutable std::mutex m_http_client_pool_mutex; + mutable std::vector m_http_client_pool; +}; +} //namespace mocks +} //namespace sp diff --git a/src/seraphis_mocks/mock_ledger_context.cpp b/src/seraphis_mocks/mock_ledger_context.cpp index 2c705efb7c..79def24229 100644 --- a/src/seraphis_mocks/mock_ledger_context.cpp +++ b/src/seraphis_mocks/mock_ledger_context.cpp @@ -1,4 +1,4 @@ -// Copyright (c) 2022, The Monero Project +// Copyright (c) 2024, The Monero Project // // All rights reserved. // diff --git a/src/seraphis_mocks/scan_context_async_mock.cpp b/src/seraphis_mocks/scan_context_async_mock.cpp new file mode 100644 index 0000000000..cb08163e25 --- /dev/null +++ b/src/seraphis_mocks/scan_context_async_mock.cpp @@ -0,0 +1,903 @@ +// Copyright (c) 2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +//paired header +#include "scan_context_async_mock.h" + +//local headers +#include "async/misc_utils.h" +#include "cryptonote_basic/cryptonote_format_utils.h" +#include "misc_language.h" +#include "misc_log_ex.h" +#include "net/http.h" +#include "seraphis_impl/scan_ledger_chunk_simple.h" +#include "seraphis_main/contextual_enote_record_types.h" +#include "seraphis_main/enote_finding_context.h" +#include "seraphis_main/enote_record_utils_legacy.h" +#include "seraphis_main/scan_core_types.h" +#include "seraphis_main/scan_misc_utils.h" +#include "seraphis_main/scan_balance_recovery_utils.h" +#include "storages/http_abstract_invoke.h" +#include "wallet/wallet_errors.h" + +//standard headers +#include +#include +#include +#include + +//third party headers +#include + +#undef MONERO_DEFAULT_LOG_CATEGORY +#define MONERO_DEFAULT_LOG_CATEGORY "seraphis_impl" + +namespace sp +{ +namespace scanning +{ +namespace mocks +{ +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static void validate_get_blocks_res(const ChunkRequest &req, + const cryptonote::COMMAND_RPC_GET_BLOCKS_FAST::response &res) +{ + THROW_WALLET_EXCEPTION_IF(res.blocks.size() != res.output_indices.size(), tools::error::get_blocks_error, + "mismatched blocks (" + boost::lexical_cast(res.blocks.size()) + ") and output_indices (" + + boost::lexical_cast(res.output_indices.size()) + ") sizes from daemon"); + + for (std::size_t i = 0; i < res.blocks.size(); ++i) + { + const std::size_t num_txs = res.blocks[i].txs.size() + 1; // Add 1 for miner tx + const std::size_t num_output_indices = res.output_indices[i].indices.size(); + + THROW_WALLET_EXCEPTION_IF(num_txs != num_output_indices, tools::error::get_blocks_error, + "mismatched block txs (" + boost::lexical_cast(num_txs) + ") and output_indices" + + " (" + boost::lexical_cast(num_output_indices) + ") sizes from daemon"); + } + + if (!res.blocks.empty()) + { + // current height == (top block index + 1) + THROW_WALLET_EXCEPTION_IF(req.start_index >= res.current_height, tools::error::get_blocks_error, + "returned non-empty blocks in getblocks.bin but requested start index is >= chain height"); + } + else + { + // We expect to have scanned to the tip + THROW_WALLET_EXCEPTION_IF(req.start_index < res.current_height, tools::error::get_blocks_error, + "no blocks returned in getblocks.bin but requested start index is < chain height"); + + // Scanner is not designed to support retrieving empty chunks when no top block hash is returned (i.e. when + // pointing to an older daemon version) + THROW_WALLET_EXCEPTION_IF(res.top_block_hash == crypto::null_hash, tools::error::wallet_internal_error, + "did not expect empty chunk when top block hash is null"); + } +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static std::uint64_t get_total_output_count_before_tx(const std::vector &output_indices) +{ + // total_output_count_before_tx == global output index of first output in tx. + // Some txs have no enotes, in which case we set this value to 0 as it isn't useful. + // TODO: pre-RCT outputs yield incorrect values here but this is only used for spending + // need https://github.com/UkoeHB/monero/pull/40 in order to handle pre-RCT outputs + return !output_indices.empty() ? output_indices[0] : 0; +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static void prepare_unscanned_legacy_transaction(const crypto::hash &tx_hash, + const cryptonote::transaction &tx, + const std::uint64_t total_output_count_before_tx, + sp::LegacyUnscannedTransaction &unscanned_tx_out) +{ + unscanned_tx_out = LegacyUnscannedTransaction{}; + + unscanned_tx_out.transaction_id = rct::hash2rct(tx_hash); + unscanned_tx_out.total_enotes_before_tx = total_output_count_before_tx; + unscanned_tx_out.unlock_time = tx.unlock_time; + + unscanned_tx_out.tx_memo = sp::TxExtra( + (const unsigned char *) tx.extra.data(), + (const unsigned char *) tx.extra.data() + tx.extra.size() + ); + + sp::legacy_outputs_to_enotes(tx, unscanned_tx_out.enotes); + + unscanned_tx_out.legacy_key_images.reserve(tx.vin.size()); + for (const auto &in: tx.vin) + { + if (in.type() != typeid(cryptonote::txin_to_key)) + continue; + const auto &txin = boost::get(in); + unscanned_tx_out.legacy_key_images.emplace_back(txin.k_image); + } +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static bool is_terminal_chunk(const sp::scanning::ChunkContext &context, const std::uint64_t end_scan_index) +{ + if (sp::scanning::chunk_context_is_empty(context)) + { + MDEBUG("Chunk context is empty starting at " << context.start_index); + return true; + } + + // is the chunk the terminal chunk in the chain + const std::uint64_t current_chunk_end_index{context.start_index + sp::scanning::chunk_size(context)}; + if (current_chunk_end_index >= end_scan_index) + { + MDEBUG("Chunk context end index: " << current_chunk_end_index + << " (end_scan_index=" << end_scan_index << ")"); + return true; + } + + return false; +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static void rpc_get_blocks_internal(const ChunkRequest &chunk_request, + const std::function &rpc_get_blocks, + const std::uint64_t max_get_blocks_attempts, + const bool trusted_daemon, + const bool high_height_ok, + cryptonote::COMMAND_RPC_GET_BLOCKS_FAST::response &res_out) +{ + cryptonote::COMMAND_RPC_GET_BLOCKS_FAST::request req = AUTO_VAL_INIT(req); + + req.start_height = chunk_request.start_index; + req.max_block_count = chunk_request.requested_chunk_size; + req.high_height_ok = high_height_ok; + req.prune = true; + req.no_miner_tx = false; + + bool r = false; + std::size_t try_count = 0; + do + { + ++try_count; + try + { + MDEBUG("Pulling blocks at req start height: " << req.start_height << " (try_count=" << try_count << ")"); + r = rpc_get_blocks(req, res_out); + const std::string status = cryptonote::get_rpc_status(trusted_daemon, res_out.status); + THROW_ON_RPC_RESPONSE_ERROR(r, {}, res_out, "getblocks.bin", tools::error::get_blocks_error, status); + validate_get_blocks_res(chunk_request, res_out); + } + catch (tools::error::deprecated_rpc_access&) + { + // No need to retry + std::rethrow_exception(std::current_exception()); + } + catch (...) + { + r = false; + if (try_count >= max_get_blocks_attempts) + std::rethrow_exception(std::current_exception()); + } + } while (!r && try_count < max_get_blocks_attempts); + + THROW_WALLET_EXCEPTION_IF(!r, tools::error::wallet_internal_error, "failed to get blocks"); + + MDEBUG("Pulled blocks: requested start height " << req.start_height << ", count " << res_out.blocks.size() + << ", node height " << res_out.current_height << ", top hash " << res_out.top_block_hash + << ", pool info " << static_cast(res_out.pool_info_extent)); +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static void prepare_unscanned_block(const cryptonote::block_complete_entry &res_block_entry, + const std::size_t block_idx, + const std::vector &output_indices, + sp::LegacyUnscannedBlock &unscanned_block_out) +{ + unscanned_block_out.unscanned_txs.resize(1 + res_block_entry.txs.size()); // Add 1 for miner tx + + // Parse block + cryptonote::block block; + bool r = cryptonote::parse_and_validate_block_from_blob(res_block_entry.block, block); + THROW_WALLET_EXCEPTION_IF(!r, tools::error::wallet_internal_error, + "failed to parse block blob" + std::to_string(block_idx)); + + THROW_WALLET_EXCEPTION_IF(res_block_entry.txs.size() != block.tx_hashes.size(), + tools::error::wallet_internal_error, "mismatched num txs to hashes at block" + std::to_string(block_idx)); + + unscanned_block_out.block_index = cryptonote::get_block_height(block); + unscanned_block_out.block_timestamp = block.timestamp; + unscanned_block_out.block_hash = rct::hash2rct(cryptonote::get_block_hash(block)); + unscanned_block_out.prev_block_hash = rct::hash2rct(block.prev_id); + + THROW_WALLET_EXCEPTION_IF(output_indices.size() != unscanned_block_out.unscanned_txs.size(), + tools::error::wallet_internal_error, "mismatched size of output indices to unscanned txs"); + + // Prepare miner tx + crypto::hash miner_tx_hash = cryptonote::get_transaction_hash(block.miner_tx); + prepare_unscanned_legacy_transaction(miner_tx_hash, + block.miner_tx, + get_total_output_count_before_tx(output_indices[0].indices), + unscanned_block_out.unscanned_txs[0]); + + // Prepare non-miner txs + for (std::size_t tx_idx = 0; tx_idx < res_block_entry.txs.size(); ++tx_idx) + { + const std::size_t unscanned_tx_idx = 1 + tx_idx; + auto &unscanned_tx = unscanned_block_out.unscanned_txs[unscanned_tx_idx]; + + cryptonote::transaction tx; + r = cryptonote::parse_and_validate_tx_base_from_blob(res_block_entry.txs[tx_idx].blob, tx); + THROW_WALLET_EXCEPTION_IF(!r, tools::error::wallet_internal_error, + "failed to parse tx blob at index " + std::to_string(tx_idx)); + + prepare_unscanned_legacy_transaction(block.tx_hashes[tx_idx], + std::move(tx), + get_total_output_count_before_tx(output_indices[unscanned_tx_idx].indices), + unscanned_tx); + } +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static void parse_rpc_get_blocks(const ChunkRequest &chunk_request, + const cryptonote::COMMAND_RPC_GET_BLOCKS_FAST::response &res, + sp::scanning::ChunkContext &chunk_context_out, + sp::LegacyUnscannedChunk &unscanned_chunk_out) +{ + validate_get_blocks_res(chunk_request, res); + + chunk_context_out.block_ids.clear(); + unscanned_chunk_out.clear(); + + // Older daemons can return more blocks than requested because they did not support a max_block_count req param. + // The scanner expects requested_chunk_size blocks however, so we only care about the blocks up until that point. + // Note the scanner can also return *fewer* blocks than requested if at chain tip or the chunk exceeded max size. + const std::uint64_t num_blocks = std::min((std::uint64_t)res.blocks.size(), chunk_request.requested_chunk_size); + unscanned_chunk_out.resize(num_blocks); + chunk_context_out.block_ids.reserve(num_blocks); + + if (num_blocks == 0) + { + // must have requested the tip of the chain + chunk_context_out.prefix_block_id = rct::hash2rct(res.top_block_hash); + chunk_context_out.start_index = res.current_height; // current height == (top block index + 1) + return; + } + + // Parse blocks and txs + for (std::size_t block_idx = 0; block_idx < num_blocks; ++block_idx) + { + const auto &res_block_entry = res.blocks[block_idx]; + const auto &output_indices = res.output_indices[block_idx].indices; + auto &unscanned_block_out = unscanned_chunk_out[block_idx]; + + prepare_unscanned_block(res_block_entry, + block_idx, + output_indices, + unscanned_block_out); + + // Set chunk context data + chunk_context_out.block_ids.emplace_back(unscanned_block_out.block_hash); + if (block_idx == 0) + { + chunk_context_out.prefix_block_id = unscanned_block_out.prev_block_hash; + chunk_context_out.start_index = unscanned_block_out.block_index; + } + } +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +bool AsyncScanContextLegacy::check_launch_next_task() const +{ + THROW_WALLET_EXCEPTION_IF(!m_pending_queue_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the pending queue mutex"); + + if (!m_scanner_ready.load(std::memory_order_relaxed)) + { + MDEBUG("Pending queue is not available for use, no tasks can be launched"); + return false; + } + + const std::uint64_t num_blocks_in_chain = m_num_blocks_in_chain.load(std::memory_order_relaxed); + if (num_blocks_in_chain != 0 && m_scan_index.load(std::memory_order_relaxed) >= num_blocks_in_chain) + { + MDEBUG("Scan tasks are scheduled to scan to chain tip, not launching another task"); + return false; + } + + if (m_num_pending_chunks.load(std::memory_order_relaxed) >= m_config.pending_chunk_queue_size) + { + MDEBUG("Pending queue is already at max capacity"); + return false; + } + + // We use a separate counter for scanning chunks so we don't overload memory. + // Continuously fetching chunks while the scanner is backstopped can overload memory. + if (m_num_scanning_chunks.load(std::memory_order_relaxed) >= m_config.pending_chunk_queue_size) + { + MDEBUG("Scanning queue is already at max capacity"); + return false; + } + + return true; +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::try_fill_gap(bool chunk_is_terminal_chunk, + const std::uint64_t &requested_chunk_size, + const sp::scanning::ChunkContext &chunk_context) +{ + if (chunk_is_terminal_chunk) + return; + + // If chunk was smaller than requested, will need to fill the gap + const std::size_t chunk_size = sp::scanning::chunk_size(chunk_context); + + THROW_WALLET_EXCEPTION_IF(chunk_size > requested_chunk_size, tools::error::wallet_internal_error, + "chunk context is larger than requested"); + + const std::uint64_t gap = requested_chunk_size - chunk_size; + + // No gap, nothing to fill + if (gap == 0) + return; + + MDEBUG("There was a " << gap << " block gap at chunk request starting at " << chunk_context.start_index); + + const std::uint64_t gap_start_index = chunk_context.start_index + chunk_size; + + if (m_config.pending_chunk_queue_size > 1) + { + // Launch a new task to fill the gap + const ChunkRequest next_chunk_request{ + .start_index = gap_start_index, + .requested_chunk_size = gap + }; + + SCOPE_LOCK_MUTEX(m_pending_queue_mutex); + this->push_next_chunk_task(next_chunk_request); + } + else + { + // Pull scan index back to the start of the gap for next task. + // - For serial scan contexts (when `m_config.pending_chunk_queue_size == 1`), + // we can't launch a gap-filler task. Instead, we say the next serial task will start + // at the gap start index. + m_scan_index.store(gap_start_index, std::memory_order_relaxed); + } +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::update_chain_state(const sp::scanning::ChunkContext &chunk_context, + const std::uint64_t num_blocks_in_chain, + const rct::key &top_block_hash, + bool &chunk_is_terminal_chunk_out) +{ + std::lock_guard lock{m_chain_state_mutex}; + + MDEBUG("Updating chain state"); + + // Update the chain tip. + // - It's possible the chain tip will get set to a stale value here if a stale RPC request returns after a fresh + // one. That's ok. It will get corrected either by another request coming in later, or on the next scan pass. + // - Unless the scanner is checking difficulty, technically there's no way for it to know which chain is actually + // the highest difficulty chain anyway, so it has to trust whatever value comes in here. + // TODO: only update if difficulty > last known difficulty; needs a change to the daemon RPC + if (num_blocks_in_chain != m_num_blocks_in_chain.load(std::memory_order_relaxed) + || top_block_hash != m_top_block_hash) + { + m_num_blocks_in_chain.store(num_blocks_in_chain, std::memory_order_relaxed); + + // Note: the top block hash can be null if pointing to an older daemon + m_top_block_hash = top_block_hash; + + MDEBUG("Updated m_num_blocks_in_chain to " << num_blocks_in_chain + << " (m_top_block_hash=" << top_block_hash << ")"); + } + + // Check if it's the scanner's terminal chunk (empty chunk context or reached tip of the chain) + const std::uint64_t n_blocks_in_chain = m_num_blocks_in_chain.load(std::memory_order_relaxed); + chunk_is_terminal_chunk_out = is_terminal_chunk(chunk_context, n_blocks_in_chain); + + // Use the terminal chunk to update the top block hash if the chunk isn't empty. + // - This is required if the daemon RPC did NOT provide the top block hash (e.g. when pointing to an older + // daemon), in which case we have to use the last block ID in the terminal chunk to set the top block hash. + if (chunk_is_terminal_chunk_out && !chunk_context.block_ids.empty()) + { + m_top_block_hash = chunk_context.block_ids[chunk_context.block_ids.size() - 1]; + MDEBUG("Used terminal chunk to update top_block_hash " << m_top_block_hash + << " (num_blocks_in_chain=" << n_blocks_in_chain << ")"); + } + + // Sanity check expected values at terminal chunk + if (chunk_is_terminal_chunk_out) + { + // The m_scan_index must be at the tip or later (if the async scanner scheduled chunk tasks way beyond tip) + THROW_WALLET_EXCEPTION_IF(m_scan_index.load(std::memory_order_relaxed) < n_blocks_in_chain, + tools::error::wallet_internal_error, + "scan index is < m_num_blocks_in_chain even though we encountered the terminal chunk"); + + THROW_WALLET_EXCEPTION_IF(n_blocks_in_chain == 0, tools::error::wallet_internal_error, + "expected >0 num blocks in the chain at terminal chunk"); + + THROW_WALLET_EXCEPTION_IF(m_top_block_hash == rct::hash2rct(crypto::null_hash), + tools::error::wallet_internal_error, "expected top block hash to be set at terminal chunk"); + } +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::handle_chunk_request(const ChunkRequest &chunk_request, + sp::scanning::ChunkContext &chunk_context_out, + LegacyUnscannedChunk &unscanned_chunk_out, + bool &chunk_is_terminal_chunk_out) +{ + // Query daemon for chunk of blocks + cryptonote::COMMAND_RPC_GET_BLOCKS_FAST::response res = AUTO_VAL_INIT(res); + { + async::fanout_token_t fanout_token{m_threadpool.launch_temporary_worker()}; + rpc_get_blocks_internal(chunk_request, + rpc_get_blocks, + m_config.max_get_blocks_attempts, + m_config.trusted_daemon, + m_config.high_height_ok, + res); + } + + // Parse the result + parse_rpc_get_blocks(chunk_request, + res, + chunk_context_out, + unscanned_chunk_out); + + // Update scanner's known top block height and hash + this->update_chain_state(chunk_context_out, + res.current_height, // current height == (top block index + 1) + rct::hash2rct(res.top_block_hash), + chunk_is_terminal_chunk_out); + + // Check if the chunk was smaller than requested and fill gap if needed + this->try_fill_gap(chunk_is_terminal_chunk_out, + chunk_request.requested_chunk_size, + chunk_context_out); +} +//------------------------------------------------------------------------------------------------------------------- +async::TaskVariant AsyncScanContextLegacy::chunk_task(const ChunkRequest &chunk_request, + std::shared_future &context_stop_flag, + std::shared_future &data_stop_flag, + std::shared_ptr> &chunk_context_ptr_out, + std::shared_ptr> &chunk_data_ptr_out, + async::join_token_t &context_join_token_out) +{ + // Check if canceled + if (async::future_is_ready(context_stop_flag)) + { + m_num_pending_chunks.fetch_sub(1, std::memory_order_relaxed); + return boost::none; + } + + // Get the chunk from the daemon and prepare to scan + sp::scanning::ChunkContext chunk_context{}; + LegacyUnscannedChunk unscanned_chunk{}; + bool chunk_is_terminal_chunk = false; + try + { + this->handle_chunk_request(chunk_request, + chunk_context, + unscanned_chunk, + chunk_is_terminal_chunk); + } + catch (...) + { + LOG_ERROR("Failed to get chunk context at start index " << chunk_request.start_index); + chunk_context_ptr_out->set_exception(std::move(std::current_exception())); + m_num_pending_chunks.fetch_sub(1, std::memory_order_relaxed); + return boost::none; + } + + // Finished retrieving the chunk + chunk_context_ptr_out->set_value(std::move(chunk_context)); + context_join_token_out = nullptr; + m_num_pending_chunks.fetch_sub(1, std::memory_order_relaxed); + + // Check if canceled + if (async::future_is_ready(data_stop_flag)) + return boost::none; + + // launch the next task if we expect more and the queue has room + this->try_launch_next_chunk_task(chunk_is_terminal_chunk); + + // Retrieved the chunk, now need to scan it + m_num_scanning_chunks.fetch_add(1, std::memory_order_relaxed); + + // find-received-scan raw data + // - note: process chunk data can 'do nothing' if the chunk is empty (i.e. don't launch any tasks) + sp::scanning::ChunkData chunk_data; + try { m_enote_finding_context.view_scan_chunk(unscanned_chunk, chunk_data); } + catch (...) + { + LOG_ERROR("Failed to view scan chunk at start index " << chunk_request.start_index); + chunk_data_ptr_out->set_exception(std::move(std::current_exception())); + m_num_scanning_chunks.fetch_sub(1, std::memory_order_relaxed); + return boost::none; + } + + // Finished scanning the chunk + chunk_data_ptr_out->set_value(std::move(chunk_data)); + m_num_scanning_chunks.fetch_sub(1, std::memory_order_relaxed); + + MDEBUG("Finished scanning chunk starting at " << chunk_request.start_index); + + this->try_launch_next_chunk_task(chunk_is_terminal_chunk); + + return boost::none; +} +//------------------------------------------------------------------------------------------------------------------- +PendingChunk AsyncScanContextLegacy::launch_chunk_task(const ChunkRequest &chunk_request) +{ + THROW_WALLET_EXCEPTION_IF(!m_pending_queue_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the pending queue mutex"); + + MDEBUG("Launching chunk task at " << chunk_request.start_index + << " (requested_chunk_size=" << chunk_request.requested_chunk_size << ")"); + + // prepare chunk task + std::promise context_stop_signal{}; + std::promise data_stop_signal{}; + std::promise chunk_context_handle{}; + std::promise chunk_data_handle{}; + std::shared_future chunk_context_future = chunk_context_handle.get_future().share(); + std::shared_future chunk_data_future = chunk_data_handle.get_future().share(); + async::join_signal_t context_join_signal = m_threadpool.make_join_signal(); + async::join_signal_t data_join_signal = m_threadpool.make_join_signal(); + async::join_token_t context_join_token = m_threadpool.get_join_token(context_join_signal); + async::join_token_t data_join_token = m_threadpool.get_join_token(data_join_signal); + + auto task = + [ + this, + l_chunk_request = chunk_request, + l_context_stop_flag = context_stop_signal.get_future().share(), + l_data_stop_flag = data_stop_signal.get_future().share(), + l_chunk_context = std::make_shared>(std::move(chunk_context_handle)), + l_chunk_data = std::make_shared>(std::move(chunk_data_handle)), + l_context_join_token = context_join_token, + l_data_join_token = data_join_token + ]() mutable -> async::TaskVariant + { + return this->chunk_task(l_chunk_request, + l_context_stop_flag, + l_data_stop_flag, + l_chunk_context, + l_chunk_data, + l_context_join_token); + }; + + // launch the task + m_num_pending_chunks.fetch_add(1, std::memory_order_relaxed); + m_threadpool.submit(async::make_simple_task(async::DefaultPriorityLevels::MEDIUM, std::move(task))); + + // return pending chunk for caller to deal with as needed + async::join_condition_t chunk_context_join_condition{ + m_threadpool.get_join_condition(std::move(context_join_signal), std::move(context_join_token)) + }; + + async::join_condition_t chunk_data_join_condition{ + m_threadpool.get_join_condition(std::move(data_join_signal), std::move(data_join_token)) + }; + + return PendingChunk{ + .chunk_request = chunk_request, + .pending_context = sp::scanning::PendingChunkContext{ + .stop_signal = std::move(context_stop_signal), + .chunk_context = std::move(chunk_context_future), + .context_join_condition = std::move(chunk_context_join_condition) + }, + .pending_data = sp::scanning::PendingChunkData{ + .stop_signal = std::move(data_stop_signal), + .chunk_data = std::move(chunk_data_future), + .data_join_condition = std::move(chunk_data_join_condition) + } + }; +} +//------------------------------------------------------------------------------------------------------------------- +bool AsyncScanContextLegacy::try_launch_next_chunk_task() +{ + THROW_WALLET_EXCEPTION_IF(!m_pending_queue_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the pending queue mutex"); + + if (!this->check_launch_next_task()) + return false; + + // Advance the scanner's scanning index + const std::uint64_t start_index = m_scan_index.fetch_add(m_max_chunk_size_hint); + + const ChunkRequest next_chunk_request{ + .start_index = start_index, + .requested_chunk_size = m_max_chunk_size_hint + }; + + return this->push_next_chunk_task(next_chunk_request); +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::try_launch_next_chunk_task(const bool chunk_is_terminal_chunk) +{ + // Don't need to launch the next task if found the terminal chunk, we're done! + if (chunk_is_terminal_chunk) + return; + SCOPE_LOCK_MUTEX(m_pending_queue_mutex); + this->try_launch_next_chunk_task(); +} +//------------------------------------------------------------------------------------------------------------------- +bool AsyncScanContextLegacy::push_next_chunk_task(const ChunkRequest &next_chunk_request) +{ + THROW_WALLET_EXCEPTION_IF(!m_pending_queue_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the pending queue mutex"); + + if (!m_scanner_ready.load(std::memory_order_relaxed)) + { + MDEBUG("Pending queue is not available for use, not pushing next chunk task"); + return false; + } + + auto task = this->launch_chunk_task(next_chunk_request); + m_pending_chunk_queue.force_push(std::move(task)); + + return true; +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::handle_terminal_chunk() +{ + THROW_WALLET_EXCEPTION_IF(!m_async_scan_context_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the async scan context mutex"); + + // Clear up everything left in the queue + this->close_and_clear_pending_queue(); + + // Make sure we scanned to current tip + if (m_last_scanned_index == m_num_blocks_in_chain.load(std::memory_order_relaxed)) + { + // We're good to go + MDEBUG("We're prepared for the end condition, we scanned to " << m_last_scanned_index); + m_scanner_finished = true; + } + else + { + // The chain must have advanced since we started scanning, restart scanning from the highest scan + MDEBUG("The chain advanced since we started scanning, restart from last scan"); + SCOPE_LOCK_MUTEX(m_pending_queue_mutex); + this->start_scanner(m_last_scanned_index, m_max_chunk_size_hint); + } +} +//------------------------------------------------------------------------------------------------------------------- +std::unique_ptr AsyncScanContextLegacy::handle_end_condition() +{ + THROW_WALLET_EXCEPTION_IF(!m_async_scan_context_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the async scan context mutex"); + THROW_WALLET_EXCEPTION_IF(!m_pending_queue_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the pending queue mutex"); + + const std::uint64_t num_blocks_in_chain = m_num_blocks_in_chain.load(std::memory_order_relaxed); + + MDEBUG("No pending chunks remaining, num blocks in chain " << num_blocks_in_chain + << ", top hash " << m_top_block_hash << " , last scanned index " << m_last_scanned_index); + + THROW_WALLET_EXCEPTION_IF(!m_scanner_finished, tools::error::wallet_internal_error, + "finished scanning but m_scanner_finished is not set"); + + THROW_WALLET_EXCEPTION_IF(num_blocks_in_chain == 0, tools::error::wallet_internal_error, + "finished scanning but num blocks in chain not set"); + + THROW_WALLET_EXCEPTION_IF(m_top_block_hash == rct::hash2rct(crypto::null_hash), + tools::error::wallet_internal_error, "finished scanning but top block hash not set"); + + THROW_WALLET_EXCEPTION_IF(m_last_scanned_index != num_blocks_in_chain, + tools::error::wallet_internal_error, "finished scanning but did not scan to the tip of the chain"); + + // Use an empty chunk to indicate to the caller the scanner is finished + sp::scanning::ChunkContext empty_terminal_chunk{ + .prefix_block_id = m_top_block_hash, + .start_index = num_blocks_in_chain, + .block_ids = {} + }; + + return std::make_unique(std::move(empty_terminal_chunk)); +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::close_and_clear_pending_queue() +{ + THROW_WALLET_EXCEPTION_IF(!m_async_scan_context_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the async scan context mutex"); + + // TODO: implement a faster cancel (adding ability to cancel http requests would be significant) + MDEBUG("Waiting until pending queue clears"); + + // Don't allow scheduling any more chunk tasks until the scanner is restarted + m_scanner_ready.store(false, std::memory_order_relaxed); + + m_pending_chunk_queue.shut_down(); + + // Send stop signals to all pending tasks and drain the queue + std::vector drained_chunks; + PendingChunk pending_chunk; + auto pending_chunk_res = m_pending_chunk_queue.force_pop(pending_chunk); + while (pending_chunk_res != async::TokenQueueResult::SHUTTING_DOWN) + { + THROW_WALLET_EXCEPTION_IF(pending_chunk_res != async::TokenQueueResult::SUCCESS, + tools::error::wallet_internal_error, "Failed to clear pending chunks"); + + // Send stop signals + pending_chunk.pending_context.stop_signal.set_value(); + pending_chunk.pending_data.stop_signal.set_value(); + + // Push the pending chunk into our drain catcher + drained_chunks.push_back(std::move(pending_chunk)); + + // Get the next pending chunk if there is one + pending_chunk_res = m_pending_chunk_queue.force_pop(pending_chunk); + } + + // Wait until all work from the queue is done + while (!drained_chunks.empty()) + { + auto &clear_chunk = drained_chunks.back(); + MDEBUG("Waiting to clear onchain chunk starting at " << clear_chunk.chunk_request.start_index); + + // Wait until **data** join condition is set, we're not waiting on just the contexts + m_threadpool.work_while_waiting(clear_chunk.pending_data.data_join_condition, + async::DefaultPriorityLevels::MAX); + + drained_chunks.pop_back(); + } + + MDEBUG("Pending queue cleared"); +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::start_scanner(const std::uint64_t start_index, + const std::uint64_t max_chunk_size_hint) +{ + THROW_WALLET_EXCEPTION_IF(!m_async_scan_context_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the async scan context mutex"); + THROW_WALLET_EXCEPTION_IF(!m_pending_queue_mutex.thread_owns_lock(), tools::error::wallet_internal_error, + "this thread does not own the pending queue mutex"); + + MDEBUG("Starting scanner from index " << start_index); + + THROW_WALLET_EXCEPTION_IF(m_pending_chunk_queue.reset() != async::TokenQueueResult::SUCCESS, + tools::error::wallet_internal_error, "Pending queue failed to reset"); + + m_max_chunk_size_hint = max_chunk_size_hint; + m_scanner_ready.store(true, std::memory_order_relaxed); + m_scanner_finished = false; + + m_num_pending_chunks.store(0, std::memory_order_relaxed); + m_num_scanning_chunks.store(0, std::memory_order_relaxed); + m_scan_index.store(start_index, std::memory_order_relaxed); + m_last_scanned_index = start_index; + + m_num_blocks_in_chain.store(0, std::memory_order_relaxed); + m_top_block_hash = rct::hash2rct(crypto::null_hash); + + // launch tasks until the queue fills up + while (this->try_launch_next_chunk_task()) {}; +} +//------------------------------------------------------------------------------------------------------------------- +void AsyncScanContextLegacy::begin_scanning_from_index(const std::uint64_t start_index, + const std::uint64_t max_chunk_size_hint) +{ + SCOPE_LOCK_MUTEX(m_async_scan_context_mutex); + + // Wait for any pending chunks to finish if there are any (it's possible the caller detected a reorg and wants + // to restart scanning from the reorged block) + this->close_and_clear_pending_queue(); + + SCOPE_LOCK_MUTEX(m_pending_queue_mutex); + this->start_scanner(start_index, max_chunk_size_hint); +} +//------------------------------------------------------------------------------------------------------------------- +std::unique_ptr AsyncScanContextLegacy::get_onchain_chunk() +{ + SCOPE_LOCK_MUTEX(m_async_scan_context_mutex); + THROW_WALLET_EXCEPTION_IF(!m_scanner_ready.load(std::memory_order_relaxed) && !m_scanner_finished, + tools::error::wallet_internal_error, "scanner is not ready for use and not finished scanning yet"); + + // Get the chunk with the lowest requested start index + PendingChunk oldest_chunk; + { + SCOPE_LOCK_MUTEX(m_pending_queue_mutex); + + // Explicitly remove the min element (instead of the first element) because chunks might not be in the queue + // in chain order. If we needed to fill a gap (try_fill_gap), the pending chunk gets pushed to the end + // of the queue even though the requested start index may be lower than pending chunks already in the queue. + async::TokenQueueResult oldest_chunk_result = m_pending_chunk_queue.try_remove_min(oldest_chunk); + if (oldest_chunk_result == async::TokenQueueResult::QUEUE_EMPTY) + { + // We should be done scanning now + return this->handle_end_condition(); + } + THROW_WALLET_EXCEPTION_IF(oldest_chunk_result != async::TokenQueueResult::SUCCESS, + tools::error::wallet_internal_error, "Failed to remove earliest onchain chunk"); + + THROW_WALLET_EXCEPTION_IF(m_scanner_finished, tools::error::wallet_internal_error, + "expected empty queue upon handling terminal chunk"); + } + + THROW_WALLET_EXCEPTION_IF(!m_scanner_ready.load(std::memory_order_relaxed), + tools::error::wallet_internal_error, "scanner is not ready for use"); + + sp::scanning::mocks::ChunkRequest &oldest_request = oldest_chunk.chunk_request; + sp::scanning::PendingChunkContext &oldest_pending_context = oldest_chunk.pending_context; + MDEBUG("Waiting for onchain chunk starting at " << oldest_request.start_index); + + THROW_WALLET_EXCEPTION_IF(oldest_request.start_index != m_last_scanned_index, + tools::error::wallet_internal_error, "Chunk has index that is higher than expected"); + + // Wait until the earliest chunk context is ready + m_threadpool.work_while_waiting(oldest_pending_context.context_join_condition, + async::DefaultPriorityLevels::MAX); + + MDEBUG("Done waiting for onchain chunk starting at " << oldest_request.start_index); + + // Expect the earliest chunk context to be ready + THROW_WALLET_EXCEPTION_IF(!async::future_is_ready(oldest_pending_context.chunk_context), + tools::error::wallet_internal_error, "Earliest onchain chunk context is not ready"); + + // If there was an exception fetching the chunk context, .get() will throw it here + sp::scanning::ChunkContext oldest_context = std::move(oldest_pending_context.chunk_context.get()); + m_last_scanned_index = oldest_context.start_index + sp::scanning::chunk_size(oldest_context); + + // Handle the terminal chunk + const std::uint64_t num_blocks_in_chain = m_num_blocks_in_chain.load(std::memory_order_relaxed); + if (is_terminal_chunk(oldest_context, num_blocks_in_chain)) + { + MDEBUG("Encountered potential terminal chunk starting at " << oldest_context.start_index + << " (expected to start at " << oldest_request.start_index << ")"); + this->handle_terminal_chunk(); + } + + // We're ready to return the pending chunk now + std::vector pending_chunk_data; + pending_chunk_data.emplace_back(std::move(oldest_chunk.pending_data)); + + if (num_blocks_in_chain > 0) + LOG_PRINT_L0("Block " << m_last_scanned_index << " / " << num_blocks_in_chain); + + return std::make_unique(m_threadpool, + std::move(oldest_chunk.pending_context), + std::move(pending_chunk_data), + std::vector{rct::zero()}); +} +//------------------------------------------------------------------------------------------------------------------- +AsyncScanContextLegacy::~AsyncScanContextLegacy() +{ + SCOPE_LOCK_MUTEX(m_async_scan_context_mutex); + + // All tasks with copies of `this` are tracked in the pending queue. When the pending queue returns empty (after + // draining and working on all removed tasks), we know that there are no lingering tasks with copies of `this`. + this->close_and_clear_pending_queue(); +} +//------------------------------------------------------------------------------------------------------------------- +} //namespace scanning +} //namespace scanning +} //namespace sp diff --git a/src/seraphis_mocks/scan_context_async_mock.h b/src/seraphis_mocks/scan_context_async_mock.h new file mode 100644 index 0000000000..ff19355a5b --- /dev/null +++ b/src/seraphis_mocks/scan_context_async_mock.h @@ -0,0 +1,242 @@ +// Copyright (c) 2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +// NOT FOR PRODUCTION +// TODO for production-ready: +// 1. Pool handling. +// 2. Pre-RCT output handling. +// 3. Ability to stop the scanner (terminate_scanning) + +// Simple implementations of enote scanning contexts. + +#pragma once + +//local headers +#include "async/mutex.h" +#include "async/threadpool.h" +#include "common/variant.h" +#include "crypto/hash.h" +#include "cryptonote_basic/cryptonote_basic.h" +#include "cryptonote_config.h" +#include "ringct/rctTypes.h" +#include "rpc/core_rpc_server_commands_defs.h" +#include "seraphis_main/enote_finding_context.h" +#include "seraphis_main/scan_context.h" +#include "seraphis_main/scan_core_types.h" +#include "seraphis_main/scan_ledger_chunk.h" +#include "seraphis_main/scan_machine_types.h" +#include "seraphis_impl/scan_ledger_chunk_async.h" + +//third party headers + +//standard headers +#include +#include +#include +#include +#include +#include + +//forward declarations + + +namespace sp +{ +namespace scanning +{ +namespace mocks +{ +//------------------------------------------------------------------------------------------------------------------- +struct ChunkRequest final +{ + std::uint64_t start_index; + std::uint64_t requested_chunk_size; +}; +//------------------------------------------------------------------------------------------------------------------- +struct PendingChunk final +{ + ChunkRequest chunk_request; + sp::scanning::PendingChunkContext pending_context; + sp::scanning::PendingChunkData pending_data; +}; + +static inline bool operator<(const PendingChunk &pending_chunk1, const PendingChunk &pending_chunk2) +{ + return pending_chunk1.chunk_request.start_index < pending_chunk2.chunk_request.start_index; +} +//------------------------------------------------------------------------------------------------------------------- +struct AsyncScanContextLegacyConfig final +{ + /// max number of chunks that will be requested at once in a queue + const std::uint64_t pending_chunk_queue_size{10}; + /// maximum number of times to retry fetching blocks from daemon on failure + std::uint64_t max_get_blocks_attempts{3}; + /// whether or not user trusts the daemon's results + bool trusted_daemon{false}; + /// whether or not the daemon returns a successful response to getblocks.bin when the request includes a height + /// that is higher than chain tip + bool high_height_ok{true}; +}; +//------------------------------------------------------------------------------------------------------------------- +//// +// WARNING: if the chunk size increment exceeds the max chunk size obtainable from the raw chunk data source, then +// this will be less efficient because it will need to 'gap fill' continuously. To maximize efficiency, +// either make sure the scanner is pointing to a daemon that supports the max_block_count req param, +// or use a pending_chunk_queue_size of 1 and a multithreaded enote finding context. +/// +class AsyncScanContextLegacy final : public ScanContextLedger +{ +public: + AsyncScanContextLegacy(const AsyncScanContextLegacyConfig &config, + sp::EnoteFindingContextLegacy &enote_finding_context, + async::Threadpool &threadpool, + const std::function &rpc_get_blocks) : + m_config{config}, + m_enote_finding_context{enote_finding_context}, + m_scanner_ready{false}, + m_threadpool{threadpool}, + rpc_get_blocks{rpc_get_blocks} + { + assert(config.pending_chunk_queue_size > 0); + assert(config.max_get_blocks_attempts > 0); + } + + ~AsyncScanContextLegacy(); + + /// disable copy/move (this is a scoped manager [reference wrapper]) + AsyncScanContextLegacy& operator=(AsyncScanContextLegacy&&) = delete; + + /// Kick off the scanner starting from the provided index + void begin_scanning_from_index(const std::uint64_t start_index, + const std::uint64_t max_chunk_size_hint) override; + + /// Get the next chunk from the scanner. Must call begin_scanning_from_index once before get_onchain_chunk + std::unique_ptr get_onchain_chunk() override; + + // TODO: implement below functions + /// stop the current scanning process (should be no-throw no-fail) + void terminate_scanning() override { /* no-op */ } + /// test if scanning has been aborted + bool is_aborted() const override { return false; } + +private: + /// abstracted function that gets blocks via RPC request + const std::function &rpc_get_blocks; + + /// reset scanner state and kick off scanner + void start_scanner(const std::uint64_t start_index, + const std::uint64_t max_chunk_size_hint); + + /// launch task to get a chunk of blocks, returns a pending chunk composed of futures + PendingChunk launch_chunk_task(const ChunkRequest &chunk_request); + + // do the actual task handling chunks and resolve chunk promises as soon as ready + async::TaskVariant chunk_task(const ChunkRequest &chunk_request, + std::shared_future &context_stop_flag_out, + std::shared_future &data_stop_flag_out, + std::shared_ptr> &chunk_context_ptr_out, + std::shared_ptr> &chunk_data_ptr_out, + async::join_token_t &context_join_token_out); + + /// close queue to further tasks and wait until all tasks in the queue have completed + void close_and_clear_pending_queue(); + + /// check if we should launch the next task to get the next chunk of blocks + bool check_launch_next_task() const; + + /// launch task to get the next chunk of blocks advancing the scanner's scan_index + bool try_launch_next_chunk_task(); + + /// launch the next chunk task if we should + void try_launch_next_chunk_task(bool chunk_is_terminal_chunk); + + /// push the next chunk task into the pending queue + bool push_next_chunk_task(const ChunkRequest &next_chunk_request); + + /// if a chunk is smaller than requested, need to fill gap to next chunk + void try_fill_gap(bool chunk_is_terminal_chunk, + const std::uint64_t &requested_chunk_size, + const sp::scanning::ChunkContext &chunk_context); + + /// fetch chunk from daemon, parse it into chunk context and unscanned chunk + void handle_chunk_request(const ChunkRequest &chunk_request, + sp::scanning::ChunkContext &chunk_context_out, + LegacyUnscannedChunk &unscanned_chunk_out, + bool &chunk_is_terminal_chunk_out); + + /// update the async scan context's known chain height and top block hash + void update_chain_state(const sp::scanning::ChunkContext &chunk_context, + const std::uint64_t num_blocks_in_chain, + const rct::key &top_block_hash, + bool &chunk_is_terminal_chunk_out); + + /// once the scanner reaches the terminal chunk, prepare it for the end condition + void handle_terminal_chunk(); + + /// return an empty chunk with prev block hash set + std::unique_ptr handle_end_condition(); +private: + /// config options + const AsyncScanContextLegacyConfig &m_config; + std::uint64_t m_max_chunk_size_hint{(uint64_t)COMMAND_RPC_GET_BLOCKS_FAST_MAX_BLOCK_COUNT}; + + /// finding context used to view scan enotes + const sp::EnoteFindingContextLegacy &m_enote_finding_context; + + /// pending chunks + async::TokenQueue m_pending_chunk_queue{}; + std::atomic m_scanner_ready{false}; + bool m_scanner_finished{false}; + + /// scanner state + std::atomic m_num_pending_chunks{0}; + std::atomic m_num_scanning_chunks{0}; + std::atomic m_scan_index{0}; + std::uint64_t m_last_scanned_index{0}; + std::uint64_t m_end_scan_index{0}; + + /// chain state known to async scanner + std::atomic m_num_blocks_in_chain{0}; + rct::key m_top_block_hash{rct::hash2rct(crypto::null_hash)}; + + /// threading helpers + async::Threadpool &m_threadpool; + async::Mutex m_async_scan_context_mutex; + async::Mutex m_pending_queue_mutex; + std::mutex m_chain_state_mutex; +}; +//------------------------------------------------------------------------------------------------------------------- +} //namespace mocks +} //namespace scanning +} //namespace sp diff --git a/src/seraphis_mocks/seraphis_mocks.h b/src/seraphis_mocks/seraphis_mocks.h index a2875328a1..3d6ccf1e3e 100644 --- a/src/seraphis_mocks/seraphis_mocks.h +++ b/src/seraphis_mocks/seraphis_mocks.h @@ -44,6 +44,7 @@ #include "mock_tx_builders_inputs.h" #include "mock_tx_builders_legacy_inputs.h" #include "mock_tx_builders_outputs.h" +#include "scan_context_async_mock.h" #include "scan_chunk_consumer_mocks.h" #include "tx_fee_calculator_mocks.h" #include "tx_input_selection_output_context_mocks.h" diff --git a/src/wallet/wallet2.cpp b/src/wallet/wallet2.cpp index 6b269ad98e..bdd3ced742 100644 --- a/src/wallet/wallet2.cpp +++ b/src/wallet/wallet2.cpp @@ -15329,22 +15329,6 @@ void wallet2::on_device_progress(const hw::device_progress& event) m_callback->on_device_progress(event); } //---------------------------------------------------------------------------------------------------- -void wallet2::throw_on_rpc_response_error(bool r, const epee::json_rpc::error &error, const std::string &status, const char *method) const -{ - // Treat all RPC payment access errors the same, whether payment is actually required or not - THROW_WALLET_EXCEPTION_IF(error.code == CORE_RPC_ERROR_CODE_INVALID_CLIENT, tools::error::deprecated_rpc_access, method); - THROW_WALLET_EXCEPTION_IF(error.code, tools::error::wallet_coded_rpc_error, method, error.code, get_rpc_server_error_message(error.code)); - THROW_WALLET_EXCEPTION_IF(!r, tools::error::no_connection_to_daemon, method); - // empty string -> not connection - THROW_WALLET_EXCEPTION_IF(status.empty(), tools::error::no_connection_to_daemon, method); - - THROW_WALLET_EXCEPTION_IF(status == CORE_RPC_STATUS_BUSY, tools::error::daemon_busy, method); - THROW_WALLET_EXCEPTION_IF(status == CORE_RPC_STATUS_PAYMENT_REQUIRED, tools::error::deprecated_rpc_access, method); - // Deprecated RPC payment access endpoints would set status to "Client signature does not verify for " - THROW_WALLET_EXCEPTION_IF(status.compare(0, 16, "Client signature") == 0, tools::error::deprecated_rpc_access, method); -} -//---------------------------------------------------------------------------------------------------- - bool wallet2::save_to_file(const std::string& path_to_file, const std::string& raw, bool is_printable) const { if (is_printable || m_export_format == ExportFormat::Binary) diff --git a/src/wallet/wallet2.h b/src/wallet/wallet2.h index e2ccdf9da5..69cb94223f 100644 --- a/src/wallet/wallet2.h +++ b/src/wallet/wallet2.h @@ -75,15 +75,6 @@ #undef MONERO_DEFAULT_LOG_CATEGORY #define MONERO_DEFAULT_LOG_CATEGORY "wallet.wallet2" -#define THROW_ON_RPC_RESPONSE_ERROR(r, error, res, method, ...) \ - do { \ - throw_on_rpc_response_error(r, error, res.status, method); \ - THROW_WALLET_EXCEPTION_IF(res.status != CORE_RPC_STATUS_OK, ## __VA_ARGS__); \ - } while(0) - -#define THROW_ON_RPC_RESPONSE_ERROR_GENERIC(r, err, res, method) \ - THROW_ON_RPC_RESPONSE_ERROR(r, err, res, method, tools::error::wallet_generic_rpc_error, method, res.status) - class Serialization_portability_wallet_Test; class wallet_accessor_test; namespace multisig { class multisig_account; } @@ -1911,8 +1902,6 @@ namespace tools boost::optional on_device_passphrase_request(bool & on_device); void on_device_progress(const hw::device_progress& event); - void throw_on_rpc_response_error(bool r, const epee::json_rpc::error &error, const std::string &status, const char *method) const; - bool should_expand(const cryptonote::subaddress_index &index) const; bool spends_one_of_ours(const cryptonote::transaction &tx) const; diff --git a/src/wallet/wallet_errors.h b/src/wallet/wallet_errors.h index 4bbf050080..937a767c51 100644 --- a/src/wallet/wallet_errors.h +++ b/src/wallet/wallet_errors.h @@ -37,7 +37,9 @@ #include "cryptonote_basic/cryptonote_format_utils.h" #include "cryptonote_core/cryptonote_tx_utils.h" +#include "net/jsonrpc_structs.h" #include "rpc/core_rpc_server_commands_defs.h" +#include "rpc/core_rpc_server_error_codes.h" #include "include_base_utils.h" @@ -1040,3 +1042,33 @@ namespace tools LOG_ERROR(#cond << ". THROW EXCEPTION: " << #err_type); \ tools::error::throw_wallet_ex(std::string(__FILE__ ":" STRINGIZE(__LINE__)), ## __VA_ARGS__); \ } + +namespace tools +{ + namespace error + { + static inline void throw_on_rpc_response_error(bool r, const epee::json_rpc::error &error, const std::string &status, const char *method) + { + // Treat all RPC payment access errors the same, whether payment is actually required or not + THROW_WALLET_EXCEPTION_IF(error.code == CORE_RPC_ERROR_CODE_INVALID_CLIENT, tools::error::deprecated_rpc_access, method); + THROW_WALLET_EXCEPTION_IF(error.code, tools::error::wallet_coded_rpc_error, method, error.code, get_rpc_server_error_message(error.code)); + THROW_WALLET_EXCEPTION_IF(!r, tools::error::no_connection_to_daemon, method); + // empty string -> not connection + THROW_WALLET_EXCEPTION_IF(status.empty(), tools::error::no_connection_to_daemon, method); + + THROW_WALLET_EXCEPTION_IF(status == CORE_RPC_STATUS_BUSY, tools::error::daemon_busy, method); + THROW_WALLET_EXCEPTION_IF(status == CORE_RPC_STATUS_PAYMENT_REQUIRED, tools::error::deprecated_rpc_access, method); + // Deprecated RPC payment access endpoints would set status to "Client signature does not verify for " + THROW_WALLET_EXCEPTION_IF(status.compare(0, 16, "Client signature") == 0, tools::error::deprecated_rpc_access, method); + } + } +} + +#define THROW_ON_RPC_RESPONSE_ERROR(r, err, res, method, ...) \ + do { \ + tools::error::throw_on_rpc_response_error(r, err, res.status, method); \ + THROW_WALLET_EXCEPTION_IF(res.status != CORE_RPC_STATUS_OK, ## __VA_ARGS__); \ + } while(0) + +#define THROW_ON_RPC_RESPONSE_ERROR_GENERIC(r, err, res, method) \ + THROW_ON_RPC_RESPONSE_ERROR(r, err, res, method, tools::error::wallet_generic_rpc_error, method, res.status) diff --git a/tests/core_tests/CMakeLists.txt b/tests/core_tests/CMakeLists.txt index 4e94992abc..39e5d0c3c2 100644 --- a/tests/core_tests/CMakeLists.txt +++ b/tests/core_tests/CMakeLists.txt @@ -35,6 +35,7 @@ set(core_tests_sources chaingen001.cpp chaingen_main.cpp double_spend.cpp + enote_scanning.cpp integer_overflow.cpp multisig.cpp ring_signature_1.cpp @@ -57,6 +58,7 @@ set(core_tests_headers chaingen_tests_list.h double_spend.h double_spend.inl + enote_scanning.h integer_overflow.h multisig.h ring_signature_1.h @@ -81,6 +83,8 @@ target_link_libraries(core_tests version epee device + seraphis_core + seraphis_main wallet ${CMAKE_THREAD_LIBS_INIT} ${EXTRA_LIBRARIES}) diff --git a/tests/core_tests/chaingen_main.cpp b/tests/core_tests/chaingen_main.cpp index 9601a0ff5b..345d9264a6 100644 --- a/tests/core_tests/chaingen_main.cpp +++ b/tests/core_tests/chaingen_main.cpp @@ -303,6 +303,13 @@ int main(int argc, char* argv[]) GENERATE_AND_PLAY(gen_block_low_coinbase); + GENERATE_AND_PLAY(gen_enote_v1_pre_rct); + GENERATE_AND_PLAY(gen_enote_v1_coinbase); + GENERATE_AND_PLAY(gen_enote_v2_rct_full_size_encoded_amount); + GENERATE_AND_PLAY(gen_enote_v3_rct_compact_encoded_amount); + GENERATE_AND_PLAY(gen_enote_v4_coinbase_view_tags); + GENERATE_AND_PLAY(gen_enote_v5_rct_view_tags); + el::Level level = (failed_tests.empty() ? el::Level::Info : el::Level::Error); if (!list_tests) { diff --git a/tests/core_tests/chaingen_tests_list.h b/tests/core_tests/chaingen_tests_list.h index 595c80552c..e8d592cee3 100644 --- a/tests/core_tests/chaingen_tests_list.h +++ b/tests/core_tests/chaingen_tests_list.h @@ -45,6 +45,7 @@ #include "bulletproofs.h" #include "bulletproof_plus.h" #include "rct2.h" +#include "enote_scanning.h" /************************************************************************/ /* */ /************************************************************************/ diff --git a/tests/core_tests/enote_scanning.cpp b/tests/core_tests/enote_scanning.cpp new file mode 100644 index 0000000000..fa1ee88d8c --- /dev/null +++ b/tests/core_tests/enote_scanning.cpp @@ -0,0 +1,271 @@ +// Copyright (c) 2014-2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +// +// Parts of this file are originally copyright (c) 2012-2013 The Cryptonote developers + +#include "chaingen.h" +#include "device/device.hpp" +#include "enote_scanning.h" +#include "ringct/rctSigs.h" +#include "seraphis_core/legacy_enote_types.h" +#include "seraphis_main/enote_record_utils_legacy.h" + +using namespace epee; +using namespace crypto; +using namespace cryptonote; + +//---------------------------------------------------------------------------------------------------------------------- +// Helpers +//---------------------------------------------------------------------------------------------------------------------- +bool gen_enote_tx_validation_base::generate_with_full(std::vector& events, + size_t mixin, uint64_t amount_paid, uint8_t hf_version, bool use_rct, const rct::RCTConfig &rct_config, bool use_view_tags, + const std::function &post_tx) const +{ + uint64_t ts_start = 1338224400; + + GENERATE_ACCOUNT(miner_account); + MAKE_GENESIS_BLOCK(events, blk_0, miner_account, ts_start); + + // create 1 miner account, and have it mine the next block + const cryptonote::block *prev_block = &blk_0; + const size_t num_blocks = 1 + CRYPTONOTE_MINED_MONEY_UNLOCK_WINDOW + mixin; + std::vector blocks; + blocks.resize(num_blocks); + CHECK_AND_ASSERT_MES(generator.construct_block_manually(blocks[0], *prev_block, miner_account, + test_generator::bf_major_ver | test_generator::bf_minor_ver | test_generator::bf_timestamp | test_generator::bf_hf_version, + 2, 2, prev_block->timestamp + DIFFICULTY_BLOCKS_ESTIMATE_TIMESPAN * 2, // v2 has blocks twice as long + crypto::hash(), 0, transaction(), std::vector(), 0, 0, 2), + false, "Failed to generate block"); + events.push_back(blocks[0]); + + // mine enough blocks to be able to spend + cryptonote::block blk_r, blk_last; + { + blk_last = blocks[0]; + for (size_t i = 1; i < num_blocks; ++i) + { + CHECK_AND_ASSERT_MES(generator.construct_block_manually(blocks[i], blk_last, miner_account, + test_generator::bf_major_ver | test_generator::bf_minor_ver | test_generator::bf_timestamp | test_generator::bf_hf_version, + 2, 2, blk_last.timestamp + DIFFICULTY_BLOCKS_ESTIMATE_TIMESPAN * 2, // v2 has blocks twice as long + crypto::hash(), 0, cryptonote::transaction(), std::vector(), 0, 0, 2), + false, "Failed to generate block"); + events.push_back(blocks[i]); + blk_last = blocks[i]; + } + blk_r = blk_last; + } + + // create tx from this miner in another block + static const uint64_t input_amounts_available[] = {5000000000000, 30000000000000, 100000000000, 80000000000}; + std::vector sources; + + sources.resize(1); + tx_source_entry& src = sources.back(); + + const uint64_t needed_amount = src.amount = input_amounts_available[0]; + + size_t real_index_in_tx = 0; + for (size_t m = 0; m <= mixin; ++m) { + size_t index_in_tx = 0; + for (size_t i = 0; i < blocks[m].miner_tx.vout.size(); ++i) + if (blocks[m].miner_tx.vout[i].amount == needed_amount) + index_in_tx = i; + CHECK_AND_ASSERT_MES(blocks[m].miner_tx.vout[index_in_tx].amount == needed_amount, false, "Expected amount not found"); + src.push_output(m, boost::get(blocks[m].miner_tx.vout[index_in_tx].target).key, src.amount); + if (m == 0) + real_index_in_tx = index_in_tx; + } + src.real_out_tx_key = cryptonote::get_tx_pub_key_from_extra(blocks[0].miner_tx); + src.real_output = 0; + src.real_output_in_tx_index = real_index_in_tx; + src.mask = rct::identity(); + src.rct = false; + + cryptonote::transaction tx; + cryptonote::block blk_txes; + + // fill outputs entry + tx_destination_entry td; + td.addr = miner_account.get_keys().m_account_address; + std::vector destinations; + td.amount = amount_paid; + destinations.push_back(td); + destinations.push_back(td); + + crypto::secret_key tx_key; + std::vector additional_tx_keys; + std::unordered_map subaddresses; + subaddresses[miner_account.get_keys().m_account_address.m_spend_public_key] = {0,0}; + bool r = construct_tx_and_get_tx_key(miner_account.get_keys(), subaddresses, sources, destinations, cryptonote::account_public_address{}, std::vector(), tx, 0, tx_key, additional_tx_keys, use_rct, rct_config, use_view_tags); + CHECK_AND_ASSERT_MES(r, false, "failed to construct transaction"); + + if (post_tx && !post_tx(tx)) + { + MDEBUG("post_tx returned failure"); + return false; + } + + LOG_PRINT_L0("Test tx: " << obj_to_json_str(tx)); + events.push_back(tx); + + CHECK_AND_ASSERT_MES(generator.construct_block_manually(blk_txes, blocks.back(), miner_account, + test_generator::bf_major_ver | test_generator::bf_minor_ver | test_generator::bf_timestamp | test_generator::bf_tx_hashes | test_generator::bf_hf_version | test_generator::bf_max_outs | test_generator::bf_tx_fees, + hf_version, hf_version, blocks.back().timestamp + DIFFICULTY_BLOCKS_ESTIMATE_TIMESPAN * 2, // v2 has blocks twice as long + crypto::hash(), 0, cryptonote::transaction(), std::vector{cryptonote::get_transaction_hash(tx)}, 0, 6, hf_version, get_tx_fee(tx)), + false, "Failed to generate block"); + events.push_back(blk_txes); + + return true; +} +//---------------------------------------------------------------------------------------------------------------------- +template +static bool check_enotes(const cryptonote::transaction &tx) +{ + CHECK_AND_ASSERT_MES(tx.vout.size() > 0, false, "unexpected number of tx outs"); + + std::vector enotes; + sp::legacy_outputs_to_enotes(tx, enotes); + + CHECK_AND_ASSERT_MES(tx.vout.size() == enotes.size(), false, "outputs <> enotes size doesn't match"); + + // Assert the enotes are the expected type + for (const auto &enote : enotes) + { + const T *enote_ptr = enote.try_unwrap(); + CHECK_AND_ASSERT_MES(enote_ptr, false, "unexpected enote type"); + } + + return true; +} +//---------------------------------------------------------------------------------------------------------------------- +//---------------------------------------------------------------------------------------------------------------------- +// Tests +//---------------------------------------------------------------------------------------------------------------------- +bool gen_enote_v1_pre_rct::generate(std::vector& events) const +{ + const int mixin = 2; + const uint64_t amount_paid = 10000; + + bool use_rct = false; + bool use_view_tags = false; + auto post_tx = [](const cryptonote::transaction &tx) + { + return check_enotes(tx); + }; + + return generate_with_full(events, mixin, amount_paid, 4, use_rct, {}, use_view_tags, post_tx); +} +//---------------------------------------------------------------------------------------------------------------------- +bool gen_enote_v1_coinbase::generate(std::vector& events) const +{ + uint64_t ts_start = 1338224400; + + GENERATE_ACCOUNT(miner_account); + MAKE_GENESIS_BLOCK(events, blk_0, miner_account, ts_start); + + const cryptonote::block *prev_block = &blk_0; + cryptonote::block block; + CHECK_AND_ASSERT_MES(generator.construct_block_manually(block, *prev_block, miner_account, + test_generator::bf_major_ver | test_generator::bf_minor_ver | test_generator::bf_timestamp | test_generator::bf_hf_version, + 2, 2, prev_block->timestamp + DIFFICULTY_BLOCKS_ESTIMATE_TIMESPAN * 2, // v2 has blocks twice as long + crypto::hash(), 0, transaction(), std::vector(), 0, 0, 2), + false, "Failed to generate block"); + events.push_back(block); + + return check_enotes(block.miner_tx); +} +//---------------------------------------------------------------------------------------------------------------------- +bool gen_enote_v2_rct_full_size_encoded_amount::generate(std::vector& events) const +{ + const int mixin = 2; + const uint64_t amount_paid = 10000; + + bool use_rct = true; + const rct::RCTConfig rct_config { rct::RangeProofBorromean, 0 }; + bool use_view_tags = false; + + auto post_tx = [](const cryptonote::transaction &tx) + { + return check_enotes(tx); + }; + + return generate_with_full(events, mixin, amount_paid, 4, use_rct, rct_config, use_view_tags, post_tx); +} +//---------------------------------------------------------------------------------------------------------------------- +bool gen_enote_v3_rct_compact_encoded_amount::generate(std::vector& events) const +{ + const int mixin = 10; + const uint64_t amount_paid = 10000; + + bool use_rct = true; + const rct::RCTConfig rct_config { rct::RangeProofPaddedBulletproof, 2 }; + bool use_view_tags = false; + + auto post_tx = [](const cryptonote::transaction &tx) + { + return check_enotes(tx); + }; + + return generate_with_full(events, mixin, amount_paid, 11, use_rct, rct_config, use_view_tags, post_tx); +} +//---------------------------------------------------------------------------------------------------------------------- +bool gen_enote_v4_coinbase_view_tags::generate(std::vector& events) const +{ + uint64_t ts_start = 1338224400; + + GENERATE_ACCOUNT(miner_account); + MAKE_GENESIS_BLOCK(events, blk_0, miner_account, ts_start); + + const cryptonote::block *prev_block = &blk_0; + cryptonote::block block; + CHECK_AND_ASSERT_MES(generator.construct_block_manually(block, *prev_block, miner_account, + test_generator::bf_major_ver | test_generator::bf_minor_ver | test_generator::bf_timestamp | test_generator::bf_hf_version, + HF_VERSION_VIEW_TAGS, HF_VERSION_VIEW_TAGS, prev_block->timestamp + DIFFICULTY_BLOCKS_ESTIMATE_TIMESPAN * 2, // v2 has blocks twice as long + crypto::hash(), 0, transaction(), std::vector(), 0, 0, HF_VERSION_VIEW_TAGS), + false, "Failed to generate block"); + events.push_back(block); + + return check_enotes(block.miner_tx); +} +//---------------------------------------------------------------------------------------------------------------------- +bool gen_enote_v5_rct_view_tags::generate(std::vector& events) const +{ + const int mixin = 15; + const uint64_t amount_paid = 10000; + + bool use_rct = true; + const rct::RCTConfig rct_config { rct::RangeProofPaddedBulletproof, 4 }; + bool use_view_tags = true; + + auto post_tx = [](const cryptonote::transaction &tx) + { + return check_enotes(tx); + }; + + return generate_with_full(events, mixin, amount_paid, HF_VERSION_VIEW_TAGS, use_rct, rct_config, use_view_tags, post_tx); +} diff --git a/tests/core_tests/enote_scanning.h b/tests/core_tests/enote_scanning.h new file mode 100644 index 0000000000..26b784e655 --- /dev/null +++ b/tests/core_tests/enote_scanning.h @@ -0,0 +1,138 @@ +// Copyright (c) 2014-2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +// +// Parts of this file are originally copyright (c) 2012-2013 The Cryptonote developers + +#pragma once +#include "chaingen.h" + +struct gen_enote_tx_validation_base : public test_chain_unit_base +{ + gen_enote_tx_validation_base() + : m_invalid_tx_index(0) + , m_invalid_block_index(0) + { + REGISTER_CALLBACK_METHOD(gen_enote_tx_validation_base, mark_invalid_tx); + REGISTER_CALLBACK_METHOD(gen_enote_tx_validation_base, mark_invalid_block); + } + + bool check_tx_verification_context(const cryptonote::tx_verification_context& tvc, bool tx_added, size_t event_idx, const cryptonote::transaction& /*tx*/) + { + if (m_invalid_tx_index == event_idx) + return tvc.m_verifivation_failed; + else + return !tvc.m_verifivation_failed && tx_added; + } + + bool check_block_verification_context(const cryptonote::block_verification_context& bvc, size_t event_idx, const cryptonote::block& /*block*/) + { + if (m_invalid_block_index == event_idx) + return bvc.m_verifivation_failed; + else + return !bvc.m_verifivation_failed; + } + + bool mark_invalid_block(cryptonote::core& /*c*/, size_t ev_index, const std::vector& /*events*/) + { + m_invalid_block_index = ev_index + 1; + return true; + } + + bool mark_invalid_tx(cryptonote::core& /*c*/, size_t ev_index, const std::vector& /*events*/) + { + m_invalid_tx_index = ev_index + 1; + return true; + } + + bool generate_with_full(std::vector& events, + size_t mixin, uint64_t amount_paid, uint8_t hf_version, bool use_rct, const rct::RCTConfig &rct_config, bool use_view_tags, + const std::function &post_tx) const; + +private: + size_t m_invalid_tx_index; + size_t m_invalid_block_index; +}; + +template<> +struct get_test_options { + const std::pair hard_forks[4] = {std::make_pair(1, 0), std::make_pair(2, 1), std::make_pair(4, 64), std::make_pair(0, 0)}; + const cryptonote::test_options test_options = { + hard_forks, 0 + }; +}; + +struct gen_enote_v1_pre_rct : public gen_enote_tx_validation_base +{ + bool generate(std::vector& events) const; +}; +template<> struct get_test_options: public get_test_options {}; + +struct gen_enote_v1_coinbase : public gen_enote_tx_validation_base +{ + bool generate(std::vector& events) const; +}; +template<> struct get_test_options: public get_test_options {}; + +struct gen_enote_v2_rct_full_size_encoded_amount : public gen_enote_tx_validation_base +{ + bool generate(std::vector& events) const; +}; +template<> struct get_test_options: public get_test_options {}; + +struct gen_enote_v3_rct_compact_encoded_amount : public gen_enote_tx_validation_base +{ + bool generate(std::vector& events) const; +}; +template<> struct get_test_options { + const std::pair hard_forks[4] = {std::make_pair(1, 0), std::make_pair(2, 1), std::make_pair(11, 72), std::make_pair(0, 0)}; + const cryptonote::test_options test_options = { + hard_forks, 0 + }; +}; + +struct gen_enote_v4_coinbase_view_tags : public gen_enote_tx_validation_base +{ + bool generate(std::vector& events) const; +}; +template<> struct get_test_options { + const std::pair hard_forks[3] = {std::make_pair(1, 0), std::make_pair(HF_VERSION_VIEW_TAGS, 1), std::make_pair(0, 0)}; + const cryptonote::test_options test_options = { + hard_forks, 0 + }; +}; + +struct gen_enote_v5_rct_view_tags : public gen_enote_tx_validation_base +{ + bool generate(std::vector& events) const; +}; +template<> struct get_test_options { + const std::pair hard_forks[4] = {std::make_pair(1, 0), std::make_pair(2, 1), std::make_pair(HF_VERSION_VIEW_TAGS, 77), std::make_pair(0, 0)}; + const cryptonote::test_options test_options = { + hard_forks, 0 + }; +}; diff --git a/tests/functional_tests/CMakeLists.txt b/tests/functional_tests/CMakeLists.txt index 62185a0dba..0fef131b75 100644 --- a/tests/functional_tests/CMakeLists.txt +++ b/tests/functional_tests/CMakeLists.txt @@ -29,25 +29,34 @@ set(functional_tests_sources main.cpp transactions_flow_test.cpp - transactions_generation_from_blockchain.cpp) + transactions_generation_from_blockchain.cpp + wallet_scanner.cpp) set(functional_tests_headers transactions_flow_test.h - transactions_generation_from_blockchain.h) + transactions_generation_from_blockchain.h + wallet_scanner.h) monero_add_minimal_executable(functional_tests ${functional_tests_sources} ${functional_tests_headers}) target_link_libraries(functional_tests PRIVATE + cryptonote_basic cryptonote_core wallet common cncrypto epee + rpc + seraphis_core + seraphis_impl + seraphis_main + seraphis_mocks ${Boost_REGEX_LIBRARY} ${Boost_PROGRAM_OPTIONS_LIBRARY} ${CMAKE_THREAD_LIBS_INIT} + ${EPEE_READLINE} ${EXTRA_LIBRARIES}) set(make_test_signature_sources diff --git a/tests/functional_tests/functional_tests_rpc.py b/tests/functional_tests/functional_tests_rpc.py index e483352a43..c44a2682db 100755 --- a/tests/functional_tests/functional_tests_rpc.py +++ b/tests/functional_tests/functional_tests_rpc.py @@ -15,6 +15,11 @@ 'http_digest_auth', 'integrated_address', 'k_anonymity', 'mining', 'multisig', 'p2p', 'proofs', 'rpc_payment', 'sign_message', 'transfer', 'txpool', 'uri', 'validate_address', 'wallet' ] +CPP_TESTS = set(['wallet_scanner']) + +for cpp_test in CPP_TESTS: + DEFAULT_TESTS.append(cpp_test) + try: python = sys.argv[1] srcdir = sys.argv[2] @@ -148,8 +153,12 @@ def kill(): try: print('[TEST STARTED] ' + test) sys.stdout.flush() - cmd = [python, srcdir + '/' + test + ".py"] + + python_test = [python, srcdir + '/' + test + ".py"] + cpp_test = [FUNCTIONAL_TESTS_DIRECTORY + '/functional_tests', '--' + test] + cmd = cpp_test if test in CPP_TESTS else python_test subprocess.check_call(cmd) + PASS.append(test) print('[TEST PASSED] ' + test) except: diff --git a/tests/functional_tests/main.cpp b/tests/functional_tests/main.cpp index 5ec40203ca..79ce746ebc 100644 --- a/tests/functional_tests/main.cpp +++ b/tests/functional_tests/main.cpp @@ -29,26 +29,30 @@ // Parts of this file are originally copyright (c) 2012-2013 The Cryptonote developers #include +#include #include "include_base_utils.h" #include "string_tools.h" using namespace epee; #include "common/command_line.h" +#include "common/scoped_message_writer.h" #include "common/util.h" #include "transactions_flow_test.h" +#include "wallet_scanner.h" namespace po = boost::program_options; namespace { const command_line::arg_descriptor arg_test_transactions_flow = {"test_transactions_flow", ""}; + const command_line::arg_descriptor arg_wallet_scanner = {"wallet_scanner", ""}; const command_line::arg_descriptor arg_working_folder = {"working-folder", "", "."}; const command_line::arg_descriptor arg_source_wallet = {"source-wallet", "", "", true}; const command_line::arg_descriptor arg_dest_wallet = {"dest-wallet", "", "", true}; - const command_line::arg_descriptor arg_daemon_addr_a = {"daemon-addr-a", "", "127.0.0.1:8080"}; - const command_line::arg_descriptor arg_daemon_addr_b = {"daemon-addr-b", "", "127.0.0.1:8082"}; + const command_line::arg_descriptor arg_daemon_addr_a = {"daemon-addr-a", "", "127.0.0.1:18180"}; + const command_line::arg_descriptor arg_daemon_addr_b = {"daemon-addr-b", "", "127.0.0.1:18182"}; const command_line::arg_descriptor arg_transfer_amount = {"transfer_amount", "", 60000000000000}; const command_line::arg_descriptor arg_mix_in_factor = {"mix-in-factor", "", 15}; @@ -59,18 +63,20 @@ namespace int main(int argc, char* argv[]) { - TRY_ENTRY(); + try + { tools::on_startup(); string_tools::set_module_name_and_folder(argv[0]); //set up logging options - mlog_configure(mlog_get_default_log_path("functional_tests.log"), true); + mlog_configure(mlog_get_default_log_path("functional_tests.log"), false); mlog_set_log_level(3); po::options_description desc_options("Allowed options"); command_line::add_arg(desc_options, command_line::arg_help); command_line::add_arg(desc_options, arg_test_transactions_flow); + command_line::add_arg(desc_options, arg_wallet_scanner); command_line::add_arg(desc_options, arg_working_folder); command_line::add_arg(desc_options, arg_source_wallet); @@ -118,21 +124,30 @@ int main(int argc, char* argv[]) size_t repeat_count = command_line::get_arg(vm, arg_test_repeat_count); for(size_t i = 0; i != repeat_count; i++) - if(!transactions_flow_test(working_folder, path_source_wallet, path_target_wallet, daemon_addr_a, daemon_addr_b, amount_to_transfer, mix_in_factor, transactions_count, transactions_per_second)) - break; + { + if (!transactions_flow_test(working_folder, path_source_wallet, path_target_wallet, daemon_addr_a, daemon_addr_b, amount_to_transfer, mix_in_factor, transactions_count, transactions_per_second)) + return 1; + } + } - std::string s; - std::cin >> s; - + if (command_line::get_arg(vm, arg_wallet_scanner)) + { + std::string daemon_addr = command_line::get_arg(vm, arg_daemon_addr_a); + test::WalletScannerTest wallet_scanner{daemon_addr}; + if (!wallet_scanner.run()) + return 1; + } + } + catch (const std::exception &e) + { + tools::fail_msg_writer() << e.what(); return 1; } - else + catch (...) { - std::cout << desc_options << std::endl; + tools::fail_msg_writer() << "exception in functional tests"; return 1; } - CATCH_ENTRY_L0("main", 1); - return 0; } diff --git a/tests/functional_tests/wallet_scanner.cpp b/tests/functional_tests/wallet_scanner.cpp new file mode 100644 index 0000000000..cb82c2d511 --- /dev/null +++ b/tests/functional_tests/wallet_scanner.cpp @@ -0,0 +1,638 @@ +// Copyright (c) 2014-2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +// +// Parts of this file are originally copyright (c) 2012-2013 The Cryptonote developers + +//paired header +#include "wallet_scanner.h" + +//local headers +#include "misc_language.h" +#include "rpc/core_rpc_server_commands_defs.h" +#include "seraphis_core/legacy_core_utils.h" +#include "seraphis_impl/enote_finding_context_legacy.h" +#include "seraphis_impl/enote_store.h" +#include "seraphis_impl/enote_store_utils.h" +#include "seraphis_impl/scan_context_simple.h" +#include "seraphis_impl/scan_process_basic.h" +#include "seraphis_main/contextual_enote_record_types.h" +#include "seraphis_main/scan_machine_types.h" +#include "seraphis_mocks/scan_chunk_consumer_mocks.h" + +//third party headers +#include +#include + +//standard headers +#include + +namespace test +{ +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +const std::size_t SENDR_WALLET_IDX = 0; +const std::size_t RECVR_WALLET_IDX = 1; +const std::size_t NUM_WALLETS = 2; + +const std::uint64_t FAKE_OUTS_COUNT = 15; +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static std::unique_ptr generate_wallet(const std::string &daemon_addr, + const boost::optional &daemon_login, + const epee::net_utils::ssl_options_t ssl_support) +{ + std::unique_ptr wal(new tools::wallet2( + /*network*/ cryptonote::MAINNET, + /*kdf rounds*/ 1, + /*unattended keeps spend key decrypted*/ true + )); + + wal->init(daemon_addr, daemon_login, "", 0UL, true/*trusted_daemon*/, ssl_support); + wal->allow_mismatched_daemon_version(true); + wal->set_refresh_from_block_height(1); // setting to 1 skips height estimate in wal->generate() + + // Generate wallet in memory with empty wallet file name + wal->generate("", ""); + + return wal; +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +static void add_default_subaddresses(const rct::key &legacy_base_spend_pubkey, + const crypto::secret_key &legacy_view_privkey, + std::unordered_map &legacy_subaddress_map) +{ + const uint32_t SUBADDR_MAJOR_DEFAULT_LOOKAHEAD = 50; + const uint32_t SUBADDR_MINOR_DEFAULT_LOOKAHEAD = 200; + + for (uint32_t i = 0; i < SUBADDR_MAJOR_DEFAULT_LOOKAHEAD; ++i) + { + for (uint32_t j = 0; j < SUBADDR_MINOR_DEFAULT_LOOKAHEAD; ++j) + { + const cryptonote::subaddress_index subaddr_index{i, j}; + + rct::key legacy_subaddress_spendkey; + sp::make_legacy_subaddress_spendkey(legacy_base_spend_pubkey, + legacy_view_privkey, + subaddr_index, + hw::get_device("default"), + legacy_subaddress_spendkey); + + legacy_subaddress_map[legacy_subaddress_spendkey] = subaddr_index; + } + } +}; +//------------------------------------------------------------------------------------------------------------------- +static SpTestScanConfig get_default_sp_scan_config() +{ + // Default config pointing to updated daemon + sp::scanning::ScanMachineConfig scan_machine_config{ + .reorg_avoidance_increment = 1, + // the lower the max hint is, the quicker feedback gets to the user on scanner progress + .max_chunk_size_hint = 20, + .max_partialscan_attempts = 0 + }; + + const std::uint64_t updated_pending_chunk_queue_size = std::min( + (std::uint64_t)(boost::thread::hardware_concurrency() + 2), + static_cast(10)); + + sp::scanning::mocks::AsyncScanContextLegacyConfig scan_context_config{ + .pending_chunk_queue_size = updated_pending_chunk_queue_size, + .max_get_blocks_attempts = 3, + .trusted_daemon = true, + .high_height_ok = true + }; + + return { + .scan_machine_config = std::move(scan_machine_config), + .scan_context_config = std::move(scan_context_config) + }; +} +//------------------------------------------------------------------------------------------------------------------- +static SpTestScanConfig get_single_member_queue_sp_scan_config() +{ + // Setting up the config that we'd use to point to a daemon that is not running the updates necessary to speed up + // the async scanner + sp::scanning::ScanMachineConfig backwards_compatible_scan_machine_config{ + // since older daemons ban clients that request a height > chain height, give cushion to be safe + .reorg_avoidance_increment = 3, + // be safe by making sure we always start the index below last known height + .force_reorg_avoidance_increment = true, + // an older daemon won't respect this max chunk size hint + .max_chunk_size_hint = 1000, + .max_partialscan_attempts = 3 + }; + + sp::scanning::mocks::AsyncScanContextLegacyConfig backwards_compatible_scan_context_config{ + // won't do any "gap filling" inside the async scanner + .pending_chunk_queue_size = 1, + .max_get_blocks_attempts = 3, + .trusted_daemon = true, + // older daemon configs don't support requesting too high of a height + .high_height_ok = false + }; + + return { + .scan_machine_config = std::move(backwards_compatible_scan_machine_config), + .scan_context_config = std::move(backwards_compatible_scan_context_config) + }; +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::reset() +{ + printf("Resetting blockchain\n"); + std::uint64_t height = this->daemon()->get_height().height; + this->daemon()->pop_blocks(height - 1); + this->daemon()->flush_txpool(); +} +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::mine(const std::size_t wallet_idx, const std::uint64_t num_blocks) +{ + const std::string addr = this->wallet(wallet_idx)->get_account().get_public_address_str(cryptonote::MAINNET); + this->daemon()->generateblocks(addr, num_blocks); +} +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::transfer(const std::size_t wallet_idx, + const cryptonote::account_public_address &dest_addr, + const bool is_subaddress, + const std::uint64_t amount_to_transfer, + cryptonote::transaction &tx_out) +{ + std::vector dsts; + dsts.reserve(1); + + cryptonote::tx_destination_entry de; + de.addr = dest_addr; + de.is_subaddress = is_subaddress; + de.amount = amount_to_transfer; + dsts.push_back(de); + + std::vector ptx; + ptx = this->wallet(wallet_idx)->create_transactions_2(dsts, FAKE_OUTS_COUNT, 0, 0, std::vector(), 0, {}); + CHECK_AND_ASSERT_THROW_MES(ptx.size() == 1, "unexpected num pending txs"); + this->wallet(wallet_idx)->commit_tx(ptx[0]); + + tx_out = std::move(ptx[0].tx); +} +//------------------------------------------------------------------------------------------------------------------- +std::uint64_t WalletScannerTest::mine_tx(const crypto::hash &tx_hash, const std::string &miner_addr_str) +{ + const std::string txs_hash = epee::string_tools::pod_to_hex(tx_hash); + + // Make sure tx is in the pool + auto res = this->daemon()->get_transactions(std::vector{txs_hash}); + CHECK_AND_ASSERT_THROW_MES(res.txs.size() == 1 && res.txs[0].tx_hash == txs_hash && res.txs[0].in_pool, + "tx not found in pool"); + + // Mine the tx + const std::uint64_t height = this->daemon()->generateblocks(miner_addr_str, 1).height; + + // Make sure tx was mined + res = this->daemon()->get_transactions(std::vector{txs_hash}); + CHECK_AND_ASSERT_THROW_MES(res.txs.size() == 1 && res.txs[0].tx_hash == txs_hash + && res.txs[0].block_height == height, "tx not yet mined"); + + return this->daemon()->get_last_block_header().block_header.reward; +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::check_wallet2_scan(const ExpectedScanResults &res) +{ + auto &sendr_wallet = this->wallet(SENDR_WALLET_IDX); + auto &recvr_wallet = this->wallet(RECVR_WALLET_IDX); + + sendr_wallet->refresh(true); + recvr_wallet->refresh(true); + const std::uint64_t sendr_final_balance = sendr_wallet->balance(0, true); + const std::uint64_t recvr_final_balance = recvr_wallet->balance(0, true); + + CHECK_AND_ASSERT_THROW_MES(sendr_final_balance == res.sendr_expected_balance, + "sendr_wallet has unexpected balance"); + CHECK_AND_ASSERT_THROW_MES(recvr_final_balance == res.recvr_expected_balance, + "recvr_wallet has unexpected balance"); + + // Find all transfers with matching tx hash + tools::wallet2::transfer_container recvr_wallet_incoming_transfers; + recvr_wallet->get_transfers(recvr_wallet_incoming_transfers); + + std::uint64_t received_amount = 0; + auto it = recvr_wallet_incoming_transfers.begin(); + const auto end = recvr_wallet_incoming_transfers.end(); + const auto is_same_hash = [l_tx_hash = res.tx_hash](const tools::wallet2::transfer_details& td) + { return td.m_txid == l_tx_hash; }; + while ((it = std::find_if(it, end, is_same_hash)) != end) + { + CHECK_AND_ASSERT_THROW_MES(it->m_block_height > 0, "recvr_wallet did not see tx in chain"); + received_amount += it->m_amount; + it++; + } + CHECK_AND_ASSERT_THROW_MES(received_amount == res.transfer_amount, + "recvr_wallet did not receive correct amount"); +} +//------------------------------------------------------------------------------------------------------------------- +boost::multiprecision::uint128_t WalletScannerTest::sp_scan_chain(const std::size_t wallet_idx, + const SpTestScanConfig &config) +{ + auto &wallet = this->wallet(wallet_idx); + + // Set up keys + const rct::key legacy_base_spend_pubkey = rct::pk2rct( + wallet->get_account().get_keys().m_account_address.m_spend_public_key + ); + const crypto::secret_key legacy_spend_privkey = wallet->get_account().get_keys().m_spend_secret_key; + const crypto::secret_key legacy_view_privkey = wallet->get_account().get_keys().m_view_secret_key; + + // Set up subaddress map with default lookahead + std::unordered_map legacy_subaddress_map{}; + add_default_subaddresses(legacy_base_spend_pubkey, legacy_view_privkey, legacy_subaddress_map); + + // Set up the getblocks.bin RPC requester + ConnectionPoolWrapper conn_pool_locker{*this}; + const std::function rpc_get_blocks = + [&conn_pool_locker]( + const cryptonote::COMMAND_RPC_GET_BLOCKS_FAST::request &req, + cryptonote::COMMAND_RPC_GET_BLOCKS_FAST::response &res + ) + { + return conn_pool_locker.conn_pool()->rpc_command( + sp::mocks::ClientConnectionPool::http_mode::BIN, + "/getblocks.bin", + req, + res); + }; + + sp::EnoteFindingContextLegacySimple enote_finding_context{legacy_base_spend_pubkey, + legacy_subaddress_map, + legacy_view_privkey}; + + sp::scanning::mocks::AsyncScanContextLegacy scan_context_ledger{config.scan_context_config, + enote_finding_context, + async::get_default_threadpool(), + rpc_get_blocks}; + + sp::SpEnoteStore user_enote_store{ + /*refresh_index*/ 1, + /*first_sp_allowed_block_in_chain*/ std::uint64_t(-1), + /*default_spendable_age*/ CRYPTONOTE_DEFAULT_TX_SPENDABLE_AGE + }; + + sp::mocks::ChunkConsumerMockLegacy chunk_consumer{legacy_base_spend_pubkey, + legacy_spend_privkey, + legacy_view_privkey, + user_enote_store}; + + sp::scanning::ScanContextNonLedgerDummy scan_context_nonledger{}; + + bool r = sp::refresh_enote_store(config.scan_machine_config, + scan_context_nonledger, + scan_context_ledger, + chunk_consumer); + CHECK_AND_ASSERT_THROW_MES(r, "Failed to refresh enote store"); + + return sp::get_balance(user_enote_store, + {sp::SpEnoteOriginStatus::ONCHAIN, sp::SpEnoteOriginStatus::UNCONFIRMED}, + {sp::SpEnoteSpentStatus::SPENT_ONCHAIN, sp::SpEnoteSpentStatus::SPENT_UNCONFIRMED}); +} +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::check_seraphis_scan(const ExpectedScanResults &res) +{ + const std::vector test_configs = { + get_default_sp_scan_config(), + get_single_member_queue_sp_scan_config() + }; + + for (const auto &config : test_configs) + { + const auto sp_balance_sendr_wallet = this->sp_scan_chain(SENDR_WALLET_IDX, config); + const auto sp_balance_recvr_wallet = this->sp_scan_chain(RECVR_WALLET_IDX, config); + + const auto sendr_expected_balance = boost::multiprecision::uint128_t(res.sendr_expected_balance); + const auto recvr_expected_balance = boost::multiprecision::uint128_t(res.recvr_expected_balance); + + CHECK_AND_ASSERT_THROW_MES(sp_balance_sendr_wallet == sendr_expected_balance, + "sendr_wallet Seraphis lib balance incorrect"); + CHECK_AND_ASSERT_THROW_MES(sp_balance_recvr_wallet == recvr_expected_balance, + "recvr_wallet Seraphis lib balance incorrect"); + } +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +ExpectedScanResults WalletScannerTest::init_normal_transfer_test() +{ + auto &sendr_wallet = this->wallet(SENDR_WALLET_IDX); + auto &recvr_wallet = this->wallet(RECVR_WALLET_IDX); + + // Assert sendr_wallet has enough money to send to recvr_wallet + std::uint64_t amount_to_transfer = 1000000000000; + sendr_wallet->refresh(true); + recvr_wallet->refresh(true); + CHECK_AND_ASSERT_THROW_MES(sendr_wallet->unlocked_balance(0, true) > (amount_to_transfer*2)/*2x for fee*/, + "sendr_wallet does not have enough money"); + + // Save initial state + std::uint64_t sendr_init_balance = sendr_wallet->balance(0, true); + std::uint64_t recvr_init_balance = recvr_wallet->balance(0, true); + + // Send from sendr_wallet to recvr_wallet's primary adddress + cryptonote::transaction tx; + cryptonote::account_public_address dest_addr = recvr_wallet->get_account().get_keys().m_account_address; + this->transfer(SENDR_WALLET_IDX, dest_addr, false/*is_subaddress*/, amount_to_transfer, tx); + std::uint64_t fee = cryptonote::get_tx_fee(tx); + crypto::hash tx_hash = cryptonote::get_transaction_hash(tx); + + // Mine the tx + std::string sender_addr = sendr_wallet->get_account().get_public_address_str(cryptonote::MAINNET); + std::uint64_t block_reward = this->mine_tx(tx_hash, sender_addr); + + // Calculate expected balances + std::uint64_t sendr_expected_balance = sendr_init_balance - amount_to_transfer - fee + block_reward; + std::uint64_t recvr_expected_balance = recvr_init_balance + amount_to_transfer; + + return ExpectedScanResults{ + .sendr_expected_balance = sendr_expected_balance, + .recvr_expected_balance = recvr_expected_balance, + .tx_hash = std::move(tx_hash), + .transfer_amount = amount_to_transfer + }; +} +//------------------------------------------------------------------------------------------------------------------- +ExpectedScanResults WalletScannerTest::init_sweep_single_test() +{ + auto &sendr_wallet = this->wallet(SENDR_WALLET_IDX); + auto &recvr_wallet = this->wallet(RECVR_WALLET_IDX); + + sendr_wallet->refresh(true); + recvr_wallet->refresh(true); + + // Find a spendable output + crypto::key_image ki; + std::uint64_t amount; + { + tools::wallet2::transfer_container tc; + sendr_wallet->get_transfers(tc); + bool found = false; + for (const auto &td : tc) + { + if (td.m_amount > 0 && !td.m_spent && sendr_wallet->is_transfer_unlocked(td)) + { + ki = td.m_key_image; + amount = td.m_amount; + found = true; + break; + } + } + CHECK_AND_ASSERT_THROW_MES(found, "did not find spendable output"); + } + + // Save initial state + std::uint64_t sendr_init_balance = sendr_wallet->balance(0, true); + std::uint64_t recvr_init_balance = recvr_wallet->balance(0, true); + + // Sweep single output from sendr_wallet to recvr_wallet so no change + cryptonote::transaction tx; + { + std::vector ptx = sendr_wallet->create_transactions_single(ki, + recvr_wallet->get_account().get_keys().m_account_address, + false /*is_subaddress*/, + 1 /*outputs*/, + FAKE_OUTS_COUNT, + 0 /*unlock_time*/, + 0 /*priority*/, + std::vector() /*extra*/ + ); + CHECK_AND_ASSERT_THROW_MES(ptx.size() == 1, "unexpected num pending txs"); + sendr_wallet->commit_tx(ptx[0]); + tx = std::move(ptx[0].tx); + } + std::uint64_t fee = cryptonote::get_tx_fee(tx); + crypto::hash tx_hash = cryptonote::get_transaction_hash(tx); + + // Mine the tx + const std::string sender_addr = sendr_wallet->get_account().get_public_address_str(cryptonote::MAINNET); + std::uint64_t block_reward = this->mine_tx(tx_hash, sender_addr); + + // Calculate expected balances + std::uint64_t sendr_expected_balance = sendr_init_balance - amount + block_reward; + std::uint64_t recvr_expected_balance = recvr_init_balance + (amount - fee); + + return ExpectedScanResults{ + .sendr_expected_balance = sendr_expected_balance, + .recvr_expected_balance = recvr_expected_balance, + .tx_hash = std::move(tx_hash), + .transfer_amount = (amount - fee) + }; +} +//------------------------------------------------------------------------------------------------------------------- +ExpectedScanResults WalletScannerTest::init_subaddress_transfer_test() +{ + auto &sendr_wallet = this->wallet(SENDR_WALLET_IDX); + auto &recvr_wallet = this->wallet(RECVR_WALLET_IDX); + + // Assert sendr_wallet has enough money to send to recvr_wallet + std::uint64_t amount_to_transfer = 1000000000000; + sendr_wallet->refresh(true); + recvr_wallet->refresh(true); + CHECK_AND_ASSERT_THROW_MES(sendr_wallet->unlocked_balance(0, true) > (amount_to_transfer*2)/*2x for fee*/, + "sendr_wallet does not have enough money"); + + // Save initial state + std::uint64_t sendr_init_balance = sendr_wallet->balance(0, true); + std::uint64_t recvr_init_balance = recvr_wallet->balance(0, true); + + // Send from sendr_wallet to recvr_wallet subaddress major idx 0, minor idx 1 + cryptonote::transaction tx; + cryptonote::account_public_address dest_addr = recvr_wallet->get_subaddress({0, 1}); + this->transfer(SENDR_WALLET_IDX, dest_addr, true/*is_subaddress*/, amount_to_transfer, tx); + std::uint64_t fee = cryptonote::get_tx_fee(tx); + crypto::hash tx_hash = cryptonote::get_transaction_hash(tx); + + // Mine the tx + const std::string sender_addr = sendr_wallet->get_account().get_public_address_str(cryptonote::MAINNET); + std::uint64_t block_reward = this->mine_tx(tx_hash, sender_addr); + + // Calculate expected balances + std::uint64_t sendr_expected_balance = sendr_init_balance - amount_to_transfer - fee + block_reward; + std::uint64_t recvr_expected_balance = recvr_init_balance + amount_to_transfer; + + return ExpectedScanResults{ + .sendr_expected_balance = sendr_expected_balance, + .recvr_expected_balance = recvr_expected_balance, + .tx_hash = std::move(tx_hash), + .transfer_amount = amount_to_transfer + }; +} +//------------------------------------------------------------------------------------------------------------------- +ExpectedScanResults WalletScannerTest::init_multiple_subaddresses_test() +{ + auto &sendr_wallet = this->wallet(SENDR_WALLET_IDX); + auto &recvr_wallet = this->wallet(RECVR_WALLET_IDX); + + // Assert sendr_wallet has enough money to send to recvr_wallet + std::uint64_t amount_to_transfer = 1000000000000; + sendr_wallet->refresh(true); + recvr_wallet->refresh(true); + CHECK_AND_ASSERT_THROW_MES(sendr_wallet->unlocked_balance(0, true) > (amount_to_transfer*2)/*2x for fee*/, + "sendr_wallet does not have enough money"); + + // Save initial state + std::uint64_t sendr_init_balance = sendr_wallet->balance(0, true); + std::uint64_t recvr_init_balance = recvr_wallet->balance(0, true); + + // Send from sendr_wallet to 2 recvr_wallet subaddresses + cryptonote::transaction tx; + { + const uint32_t num_subaddress = 2; + + std::vector dsts; + dsts.reserve(num_subaddress); + for (uint32_t i = 1; i <= num_subaddress; ++i) + { + cryptonote::tx_destination_entry de; + de.addr = recvr_wallet->get_subaddress({0, i}); + de.is_subaddress = true; + de.amount = amount_to_transfer / num_subaddress; + dsts.push_back(de); + } + + std::vector ptx; + ptx = sendr_wallet->create_transactions_2(dsts, FAKE_OUTS_COUNT, 0, 0, std::vector(), 0, {}); + CHECK_AND_ASSERT_THROW_MES(ptx.size() == 1, "unexpected num pending txs"); + sendr_wallet->commit_tx(ptx[0]); + + tx = std::move(ptx[0].tx); + + // Ensure tx has correct num additional pub keys + const auto additional_pub_keys = cryptonote::get_additional_tx_pub_keys_from_extra(tx); + CHECK_AND_ASSERT_THROW_MES(additional_pub_keys.size() == (num_subaddress + 1), + "unexpected num additional pub keys"); + } + std::uint64_t fee = cryptonote::get_tx_fee(tx); + crypto::hash tx_hash = cryptonote::get_transaction_hash(tx); + + // Mine the tx + const std::string sender_addr = sendr_wallet->get_account().get_public_address_str(cryptonote::MAINNET); + std::uint64_t block_reward = this->mine_tx(tx_hash, sender_addr); + + // Use wallet2 to scan tx and make sure it's in the chain + std::uint64_t sendr_expected_balance = sendr_init_balance - amount_to_transfer - fee + block_reward; + std::uint64_t recvr_expected_balance = recvr_init_balance + amount_to_transfer; + + return ExpectedScanResults{ + .sendr_expected_balance = sendr_expected_balance, + .recvr_expected_balance = recvr_expected_balance, + .tx_hash = std::move(tx_hash), + .transfer_amount = amount_to_transfer + }; +} +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +// Tests +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::check_normal_transfer() +{ + printf("Checking normal transfer\n"); + const ExpectedScanResults res = this->init_normal_transfer_test(); + + this->check_wallet2_scan(res); + this->check_seraphis_scan(res); +} +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::check_sweep_single() +{ + printf("Checking sweep single\n"); + const ExpectedScanResults res = this->init_sweep_single_test(); + + this->check_wallet2_scan(res); + this->check_seraphis_scan(res); +} +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::check_subaddress_transfer() +{ + printf("Checking transfer to subaddress\n"); + const ExpectedScanResults res = this->init_subaddress_transfer_test(); + + this->check_wallet2_scan(res); + this->check_seraphis_scan(res); +} +//------------------------------------------------------------------------------------------------------------------- +void WalletScannerTest::check_multiple_subaddresses_transfer() +{ + printf("Checking transfer to multiple subaddresses\n"); + const ExpectedScanResults res = this->init_multiple_subaddresses_test(); + + this->check_wallet2_scan(res); + this->check_seraphis_scan(res); +} +//------------------------------------------------------------------------------------------------------------------- +bool WalletScannerTest::run() +{ + SCOPE_LOCK_MUTEX(m_wallets_mutex); + SCOPE_LOCK_MUTEX(m_daemon_mutex); + + // Reset chain + this->reset(); + + // Mine to sender + printf("Mining to sender wallet\n"); + this->mine(SENDR_WALLET_IDX, 80); + + // Run the tests + this->check_normal_transfer(); + this->check_sweep_single(); + this->check_subaddress_transfer(); + this->check_multiple_subaddresses_transfer(); + + // TODO: add test that advances chain AFTER scanner starts (use conditional variables) + // TODO: add reorg tests (both after scanning and while scanning) + + return true; +} +//------------------------------------------------------------------------------------------------------------------- +WalletScannerTest::WalletScannerTest(const std::string &daemon_addr): + m_daemon_addr(daemon_addr) +{ + const boost::optional daemon_login = boost::none; + const epee::net_utils::ssl_options_t ssl_support = epee::net_utils::ssl_support_t::e_ssl_support_disabled; + + m_daemon = std::make_unique(m_daemon_addr, daemon_login, ssl_support); + + m_wallets.reserve(NUM_WALLETS); + for (std::size_t i = 0; i < NUM_WALLETS; ++i) + { + m_wallets.push_back(generate_wallet(m_daemon_addr, daemon_login, ssl_support)); + } + + m_conn_pool = std::make_unique(m_daemon_addr, daemon_login, ssl_support); +} +//------------------------------------------------------------------------------------------------------------------- +} //namespace test diff --git a/tests/functional_tests/wallet_scanner.h b/tests/functional_tests/wallet_scanner.h new file mode 100644 index 0000000000..ef0ed55b5f --- /dev/null +++ b/tests/functional_tests/wallet_scanner.h @@ -0,0 +1,183 @@ +// Copyright (c) 2014-2024, The Monero Project +// +// All rights reserved. +// +// Redistribution and use in source and binary forms, with or without modification, are +// permitted provided that the following conditions are met: +// +// 1. Redistributions of source code must retain the above copyright notice, this list of +// conditions and the following disclaimer. +// +// 2. Redistributions in binary form must reproduce the above copyright notice, this list +// of conditions and the following disclaimer in the documentation and/or other +// materials provided with the distribution. +// +// 3. Neither the name of the copyright holder nor the names of its contributors may be +// used to endorse or promote products derived from this software without specific +// prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY +// EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +// MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +// THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +// PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +// INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +// STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF +// THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +// +// Parts of this file are originally copyright (c) 2012-2013 The Cryptonote developers + +//local headers +#include "async/mutex.h" +#include "common/rpc_client.h" +#include "crypto/hash.h" +#include "cryptonote_basic/cryptonote_format_utils.h" +#include "seraphis_main/scan_machine_types.h" +#include "seraphis_mocks/mock_http_client_pool.h" +#include "seraphis_mocks/scan_context_async_mock.h" +#include "wallet/wallet2.h" + +//third party headers +#include + +//standard headers +#include +#include +#include + +//forward declarations +namespace test { class ConnectionPoolWrapper; }; + + +namespace test +{ +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +struct ExpectedScanResults final +{ + std::uint64_t sendr_expected_balance; + std::uint64_t recvr_expected_balance; + crypto::hash tx_hash; + std::uint64_t transfer_amount; +}; +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +struct SpTestScanConfig final +{ + sp::scanning::ScanMachineConfig scan_machine_config; + sp::scanning::mocks::AsyncScanContextLegacyConfig scan_context_config; +}; +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +class WalletScannerTest final +{ + friend class ConnectionPoolWrapper; +public: +//constructor + WalletScannerTest(const std::string &daemon_addr); + + /// disable copy/move (this is a scoped manager [reference wrapper]) + WalletScannerTest& operator=(WalletScannerTest&&) = delete; + + /// Run the suite of wallet scanner tests + bool run(); +private: +//tests + void check_normal_transfer(); + void check_sweep_single(); + void check_subaddress_transfer(); + void check_multiple_subaddresses_transfer(); + +//test helpers + ExpectedScanResults init_normal_transfer_test(); + ExpectedScanResults init_sweep_single_test(); + ExpectedScanResults init_subaddress_transfer_test(); + ExpectedScanResults init_multiple_subaddresses_test(); + + /// Make sure the wallet2 scanner yields expected results + void check_wallet2_scan(const ExpectedScanResults &res); + + /// Make sure the Seraphis scanner yields expected results + void check_seraphis_scan(const ExpectedScanResults &res); + + /// Use the Seraphis lib to scan the chain and return wallet balance + boost::multiprecision::uint128_t sp_scan_chain(const std::size_t wallet_idx, const SpTestScanConfig &config); + +//utility helper functions + void reset(); + void mine(const std::size_t wallet_idx, const std::uint64_t num_blocks); + std::uint64_t mine_tx(const crypto::hash &tx_hash, const std::string &miner_addr_str); + void transfer(const std::size_t wallet_idx, + const cryptonote::account_public_address &dest_addr, + const bool is_subaddress, + const std::uint64_t amount_to_transfer, + cryptonote::transaction &tx_out); + +//accessors to resources + std::unique_ptr &daemon() + { + CHECK_AND_ASSERT_THROW_MES(m_daemon_mutex.thread_owns_lock(), "thread does not own daemon mutex"); + return m_daemon; + }; + + std::unique_ptr &wallet(const std::size_t idx) + { + CHECK_AND_ASSERT_THROW_MES(m_wallets_mutex.thread_owns_lock(), "thread does not own wallets mutex"); + CHECK_AND_ASSERT_THROW_MES(idx <= m_wallets.size(), "too high wallet idx"); + return m_wallets[idx]; + }; + + std::unique_ptr &conn_pool(const std::thread::id thread_id) + { + CHECK_AND_ASSERT_THROW_MES(m_conn_pool_mutex.thread_owns_lock(thread_id), + "thread does not own connection pool mutex"); + return m_conn_pool; + }; + +private: +//member variables + const std::string &m_daemon_addr; + + // Resources that are expected to be accessed through the accessor functions above + std::unique_ptr m_daemon; + std::vector> m_wallets; + std::unique_ptr m_conn_pool; + + // Mutexes for resources + async::Mutex m_wallets_mutex; + async::Mutex m_daemon_mutex; + async::Mutex m_conn_pool_mutex; +}; +//------------------------------------------------------------------------------------------------------------------- +//------------------------------------------------------------------------------------------------------------------- +class ConnectionPoolWrapper final +{ +public: + ConnectionPoolWrapper(WalletScannerTest &wst): + m_wst(wst), + m_thread_owner_id(std::this_thread::get_id()) + { + m_wst.m_conn_pool_mutex.lock(); + }; + + /// disable copy/move (this is a scoped manager [reference wrapper]) + ConnectionPoolWrapper& operator=(ConnectionPoolWrapper&&) = delete; + + ~ConnectionPoolWrapper() + { + // Close all open connections but 1, so that we keep a connection open for future RPC calls + m_wst.conn_pool(m_thread_owner_id)->close_connections(1); + m_wst.m_conn_pool_mutex.unlock(); + }; + + std::unique_ptr &conn_pool() + { + return m_wst.conn_pool(m_thread_owner_id); + }; +private: + WalletScannerTest &m_wst; + const std::thread::id m_thread_owner_id; +}; +//------------------------------------------------------------------------------------------------------------------- +}; // test