From 3eb1307df0a38ac4ea52995fbb03ead37387b41e Mon Sep 17 00:00:00 2001 From: glozow Date: Tue, 25 Jul 2023 14:21:20 +0100 Subject: [PATCH 1/7] guard TxRequest and rejection caches with new mutex We need to synchronize between various tx download structures. TxRequest does not inherently need cs_main for synchronization, and it's not appropriate to lock all of the tx download logic under cs_main. --- src/net_processing.cpp | 89 +++++++++++++++++++++++++++--------------- 1 file changed, 57 insertions(+), 32 deletions(-) diff --git a/src/net_processing.cpp b/src/net_processing.cpp index d674758abd8..95ef67b160c 100644 --- a/src/net_processing.cpp +++ b/src/net_processing.cpp @@ -490,9 +490,9 @@ public: /** Overridden from CValidationInterface. */ void BlockConnected(ChainstateRole role, const std::shared_ptr& pblock, const CBlockIndex* pindexConnected) override - EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, !m_tx_download_mutex); void BlockDisconnected(const std::shared_ptr &block, const CBlockIndex* pindex) override - EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, !m_tx_download_mutex); void UpdatedBlockTip(const CBlockIndex *pindexNew, const CBlockIndex *pindexFork, bool fInitialDownload) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex); void BlockChecked(const CBlock& block, const BlockValidationState& state) override @@ -501,13 +501,13 @@ public: EXCLUSIVE_LOCKS_REQUIRED(!m_most_recent_block_mutex); /** Implement NetEventsInterface */ - void InitializeNode(const CNode& node, ServiceFlags our_services) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex); - void FinalizeNode(const CNode& node) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_headers_presync_mutex); + void InitializeNode(const CNode& node, ServiceFlags our_services) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_tx_download_mutex); + void FinalizeNode(const CNode& node) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_headers_presync_mutex, !m_tx_download_mutex); bool HasAllDesirableServiceFlags(ServiceFlags services) const override; bool ProcessMessages(CNode* pfrom, std::atomic& interrupt) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex, !m_tx_download_mutex); bool SendMessages(CNode* pto) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, g_msgproc_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, g_msgproc_mutex, !m_tx_download_mutex); /** Implement PeerManager */ void StartScheduledTasks(CScheduler& scheduler) override; @@ -526,7 +526,7 @@ public: void UnitTestMisbehaving(NodeId peer_id) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex) { Misbehaving(*Assert(GetPeerRef(peer_id)), ""); }; void ProcessMessage(CNode& pfrom, const std::string& msg_type, DataStream& vRecv, const std::chrono::microseconds time_received, const std::atomic& interruptMsgProc) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex, !m_tx_download_mutex); void UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds) override; ServiceFlags GetDesirableServiceFlags(ServiceFlags services) const override; @@ -585,12 +585,12 @@ private: * Updates m_txrequest, m_recent_rejects, m_recent_rejects_reconsiderable, m_orphanage, and vExtraTxnForCompact. */ void ProcessInvalidTx(NodeId nodeid, const CTransactionRef& tx, const TxValidationState& result, bool maybe_add_extra_compact_tx) - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, cs_main); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex); /** Handle a transaction whose result was MempoolAcceptResult::ResultType::VALID. * Updates m_txrequest, m_orphanage, and vExtraTxnForCompact. Also queues the tx for relay. */ void ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, const std::list& replaced_transactions) - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, cs_main); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex); struct PackageToValidate { const Package m_txns; @@ -620,13 +620,13 @@ private: * individual transactions, and caches rejection for the package as a group. */ void ProcessPackageResult(const PackageToValidate& package_to_validate, const PackageMempoolAcceptResult& package_result) - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, cs_main); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex); /** Look for a child of this transaction in the orphanage to form a 1-parent-1-child package, * skipping any combinations that have already been tried. Return the resulting package along with * the senders of its respective transactions, or std::nullopt if no package is found. */ std::optional Find1P1CPackage(const CTransactionRef& ptx, NodeId nodeid) - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, cs_main); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex); /** * Reconsider orphan transactions after a parent has been accepted to the mempool. @@ -640,7 +640,7 @@ private: * will be empty. */ bool ProcessOrphanTx(Peer& peer) - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, !m_tx_download_mutex); /** Process a single headers message from a peer. * @@ -722,7 +722,7 @@ private: * peer. The announcement parameters are decided in PeerManager and then * passed to TxRequestTracker. */ void AddTxAnnouncement(const CNode& node, const GenTxid& gtxid, std::chrono::microseconds current_time) - EXCLUSIVE_LOCKS_REQUIRED(::cs_main); + EXCLUSIVE_LOCKS_REQUIRED(::cs_main, m_tx_download_mutex); /** Send a message to a peer */ void PushMessage(CNode& node, CSerializedNetMsg&& msg) const { m_connman.PushMessage(&node, std::move(msg)); } @@ -770,7 +770,19 @@ private: BanMan* const m_banman; ChainstateManager& m_chainman; CTxMemPool& m_mempool; - TxRequestTracker m_txrequest GUARDED_BY(::cs_main); + + /** Synchronizes tx download including TxRequestTracker, rejection filters, and TxOrphanage. + * Lock invariants: + * - A txhash (txid or wtxid) in m_txrequest is not also in m_orphanage. + * - A txhash (txid or wtxid) in m_txrequest is not also in m_recent_rejects. + * - A txhash (txid or wtxid) in m_txrequest is not also in m_recent_rejects_reconsiderable. + * - A txhash (txid or wtxid) in m_txrequest is not also in m_recent_confirmed_transactions. + * - Each data structure's limits hold (m_orphanage max size, m_txrequest per-peer limits, etc). + * + * m_tx_download_mutex must be acquired before mempool.cs + */ + Mutex m_tx_download_mutex; + TxRequestTracker m_txrequest GUARDED_BY(m_tx_download_mutex); std::unique_ptr m_txreconciliation; /** The height of the best chain */ @@ -851,7 +863,7 @@ private: * chain tip has changed. * */ bool AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable) - EXCLUSIVE_LOCKS_REQUIRED(cs_main, !m_recent_confirmed_transactions_mutex); + EXCLUSIVE_LOCKS_REQUIRED(cs_main, !m_recent_confirmed_transactions_mutex, m_tx_download_mutex); /** * Filter for transactions that were recently rejected by the mempool. @@ -887,10 +899,10 @@ private: * * Memory used: 1.3 MB */ - CRollingBloomFilter m_recent_rejects GUARDED_BY(::cs_main){120'000, 0.000'001}; + CRollingBloomFilter m_recent_rejects GUARDED_BY(m_tx_download_mutex){120'000, 0.000'001}; /** Block hash of chain tip the last time we reset m_recent_rejects and * m_recent_rejects_reconsiderable. */ - uint256 hashRecentRejectsChainTip GUARDED_BY(cs_main); + uint256 hashRecentRejectsChainTip GUARDED_BY(m_tx_download_mutex); /** * Filter for: @@ -912,7 +924,7 @@ private: * * Parameters are picked to be the same as m_recent_rejects, with the same rationale. */ - CRollingBloomFilter m_recent_rejects_reconsiderable GUARDED_BY(::cs_main){120'000, 0.000'001}; + CRollingBloomFilter m_recent_rejects_reconsiderable GUARDED_BY(m_tx_download_mutex){120'000, 0.000'001}; /* * Filter for transactions that have been recently confirmed. @@ -1067,7 +1079,7 @@ private: int m_peers_downloading_from GUARDED_BY(cs_main) = 0; /** Storage for orphan information */ - TxOrphanage m_orphanage; + TxOrphanage m_orphanage GUARDED_BY(m_tx_download_mutex); void AddToCompactExtraTransactions(const CTransactionRef& tx) EXCLUSIVE_LOCKS_REQUIRED(g_msgproc_mutex); @@ -1630,7 +1642,8 @@ void PeerManagerImpl::PushNodeVersion(CNode& pnode, const Peer& peer) void PeerManagerImpl::AddTxAnnouncement(const CNode& node, const GenTxid& gtxid, std::chrono::microseconds current_time) { - AssertLockHeld(::cs_main); // For m_txrequest + AssertLockHeld(::cs_main); // for State + AssertLockHeld(m_tx_download_mutex); // For m_txrequest NodeId nodeid = node.GetId(); if (!node.HasPermission(NetPermissionFlags::Relay) && m_txrequest.Count(nodeid) >= MAX_PEER_TX_ANNOUNCEMENTS) { // Too many queued announcements from this peer @@ -1666,8 +1679,11 @@ void PeerManagerImpl::InitializeNode(const CNode& node, ServiceFlags our_service { NodeId nodeid = node.GetId(); { - LOCK(cs_main); + LOCK(cs_main); // For m_node_states m_node_states.emplace_hint(m_node_states.end(), std::piecewise_construct, std::forward_as_tuple(nodeid), std::forward_as_tuple(node.IsInboundConn())); + } + { + LOCK(m_tx_download_mutex); assert(m_txrequest.Count(nodeid) == 0); } @@ -1735,8 +1751,11 @@ void PeerManagerImpl::FinalizeNode(const CNode& node) } } } - m_orphanage.EraseForPeer(nodeid); - m_txrequest.DisconnectedPeer(nodeid); + { + LOCK(m_tx_download_mutex); + m_orphanage.EraseForPeer(nodeid); + m_txrequest.DisconnectedPeer(nodeid); + } if (m_txreconciliation) m_txreconciliation->ForgetPeer(nodeid); m_num_preferred_download_peers -= state->fPreferredDownload; m_peers_downloading_from -= (!state->vBlocksInFlight.empty()); @@ -1753,6 +1772,7 @@ void PeerManagerImpl::FinalizeNode(const CNode& node) assert(m_peers_downloading_from == 0); assert(m_outbound_peers_with_protect_from_disconnect == 0); assert(m_wtxid_relay_peers == 0); + LOCK(m_tx_download_mutex); assert(m_txrequest.Size() == 0); assert(m_orphanage.Size() == 0); } @@ -2084,6 +2104,7 @@ void PeerManagerImpl::BlockConnected( if (role == ChainstateRole::BACKGROUND) { return; } + LOCK(m_tx_download_mutex); m_orphanage.EraseForBlock(*pblock); { @@ -2096,7 +2117,6 @@ void PeerManagerImpl::BlockConnected( } } { - LOCK(cs_main); for (const auto& ptx : pblock->vtx) { m_txrequest.ForgetTxHash(ptx->GetHash()); m_txrequest.ForgetTxHash(ptx->GetWitnessHash()); @@ -2254,6 +2274,9 @@ void PeerManagerImpl::BlockChecked(const CBlock& block, const BlockValidationSta bool PeerManagerImpl::AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable) { + AssertLockHeld(::cs_main); + AssertLockHeld(m_tx_download_mutex); + if (m_chainman.ActiveChain().Tip()->GetBlockHash() != hashRecentRejectsChainTip) { // If the chain tip has changed previously rejected transactions // might be now valid, e.g. due to a nLockTime'd tx becoming valid, @@ -3154,7 +3177,7 @@ void PeerManagerImpl::ProcessInvalidTx(NodeId nodeid, const CTransactionRef& ptx { AssertLockNotHeld(m_peer_mutex); AssertLockHeld(g_msgproc_mutex); - AssertLockHeld(cs_main); + AssertLockHeld(m_tx_download_mutex); LogDebug(BCLog::MEMPOOLREJ, "%s (wtxid=%s) from peer=%d was not accepted: %s\n", ptx->GetHash().ToString(), @@ -3219,7 +3242,7 @@ void PeerManagerImpl::ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, c { AssertLockNotHeld(m_peer_mutex); AssertLockHeld(g_msgproc_mutex); - AssertLockHeld(cs_main); + AssertLockHeld(m_tx_download_mutex); // As this version of the transaction was acceptable, we can forget about any requests for it. // No-op if the tx is not in txrequest. @@ -3247,7 +3270,7 @@ void PeerManagerImpl::ProcessPackageResult(const PackageToValidate& package_to_v { AssertLockNotHeld(m_peer_mutex); AssertLockHeld(g_msgproc_mutex); - AssertLockHeld(cs_main); + AssertLockHeld(m_tx_download_mutex); const auto& package = package_to_validate.m_txns; const auto& senders = package_to_validate.m_senders; @@ -3303,7 +3326,7 @@ std::optional PeerManagerImpl::Find1P1CPacka { AssertLockNotHeld(m_peer_mutex); AssertLockHeld(g_msgproc_mutex); - AssertLockHeld(cs_main); + AssertLockHeld(m_tx_download_mutex); const auto& parent_wtxid{ptx->GetWitnessHash()}; @@ -3356,7 +3379,7 @@ std::optional PeerManagerImpl::Find1P1CPacka bool PeerManagerImpl::ProcessOrphanTx(Peer& peer) { AssertLockHeld(g_msgproc_mutex); - LOCK(cs_main); + LOCK2(::cs_main, m_tx_download_mutex); CTransactionRef porphanTx = nullptr; @@ -4173,7 +4196,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, const bool reject_tx_invs{RejectIncomingTxs(pfrom)}; - LOCK(cs_main); + LOCK2(cs_main, m_tx_download_mutex); const auto current_time{GetTime()}; uint256* best_block{nullptr}; @@ -4506,7 +4529,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, const uint256& hash = peer->m_wtxid_relay ? wtxid : txid; AddKnownTx(*peer, hash); - LOCK(cs_main); + LOCK2(cs_main, m_tx_download_mutex); m_txrequest.ReceivedResponse(pfrom.GetId(), txid); if (tx.HasWitness()) m_txrequest.ReceivedResponse(pfrom.GetId(), wtxid); @@ -5263,7 +5286,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, std::vector vInv; vRecv >> vInv; if (vInv.size() <= MAX_PEER_TX_ANNOUNCEMENTS + MAX_BLOCKS_IN_TRANSIT_PER_PEER) { - LOCK(::cs_main); + LOCK(m_tx_download_mutex); for (CInv &inv : vInv) { if (inv.IsGenTxMsg()) { // If we receive a NOTFOUND message for a tx we requested, mark the announcement for it as @@ -5388,6 +5411,7 @@ bool PeerManagerImpl::ProcessMessages(CNode* pfrom, std::atomic& interrupt // by another peer that was already processed; in that case, // the extra work may not be noticed, possibly resulting in an // unnecessary 100ms delay) + LOCK(m_tx_download_mutex); if (m_orphanage.HaveTxToReconsider(peer->m_id)) fMoreWork = true; } catch (const std::exception& e) { LogPrint(BCLog::NET, "%s(%s, %u bytes): Exception '%s' (%s) caught\n", __func__, SanitizeString(msg.m_type), msg.m_message_size, e.what(), typeid(e).name()); @@ -6281,6 +6305,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto) // // Message: getdata (transactions) // + LOCK(m_tx_download_mutex); std::vector> expired; auto requestable = m_txrequest.GetRequestable(pto->GetId(), current_time, &expired); for (const auto& entry : expired) { From 36f170d87924e50d0ff9be2a1b0f2a8f13950a9b Mon Sep 17 00:00:00 2001 From: glozow Date: Fri, 29 Sep 2023 13:58:16 +0100 Subject: [PATCH 2/7] add ValidationInterface::ActiveTipChange This is a synchronous callback notifying clients of all tip changes. It allows clients to respond to a new block immediately after it is connected. The synchronicity is important for things like m_recent_rejects, in which a transaction's validity can change (rejected vs accepted) when this event is processed. For example, the transaction might have a timelock condition that has just been met. This is distinct from something like m_recent_confirmed_transactions, in which the validation outcome is the same (valid vs already-have), so it does not need to be reset immediately. --- src/validation.cpp | 14 +++++++++++++- src/validationinterface.cpp | 6 ++++++ src/validationinterface.h | 5 +++++ 3 files changed, 24 insertions(+), 1 deletion(-) diff --git a/src/validation.cpp b/src/validation.cpp index 988df3802a1..52bff9bac26 100644 --- a/src/validation.cpp +++ b/src/validation.cpp @@ -3488,6 +3488,7 @@ bool Chainstate::ActivateBestChain(BlockValidationState& state, std::shared_ptr< { LOCK(cs_main); + { // Lock transaction pool for at least as long as it takes for connectTrace to be consumed LOCK(MempoolMutex()); const bool was_in_ibd = m_chainman.IsInitialBlockDownload(); @@ -3564,7 +3565,12 @@ bool Chainstate::ActivateBestChain(BlockValidationState& state, std::shared_ptr< break; } } - } + } // release MempoolMutex + // Notify external listeners about the new tip, even if pindexFork == pindexNewTip. + if (m_chainman.m_options.signals && this == &m_chainman.ActiveChainstate()) { + m_chainman.m_options.signals->ActiveTipChange(pindexNewTip, m_chainman.IsInitialBlockDownload()); + } + } // release cs_main // When we reach this point, we switched to a new tip (stored in pindexNewTip). if (exited_ibd) { @@ -3783,6 +3789,12 @@ bool Chainstate::InvalidateBlock(BlockValidationState& state, CBlockIndex* pinde // distinguish user-initiated invalidateblock changes from other // changes. (void)m_chainman.GetNotifications().blockTip(GetSynchronizationState(m_chainman.IsInitialBlockDownload(), m_chainman.m_blockman.m_blockfiles_indexed), *to_mark_failed->pprev); + + // Fire ActiveTipChange now for the current chain tip to make sure clients are notified. + // ActivateBestChain may call this as well, but not necessarily. + if (m_chainman.m_options.signals) { + m_chainman.m_options.signals->ActiveTipChange(m_chain.Tip(), m_chainman.IsInitialBlockDownload()); + } } return true; } diff --git a/src/validationinterface.cpp b/src/validationinterface.cpp index 579444a065a..f5baa16c99b 100644 --- a/src/validationinterface.cpp +++ b/src/validationinterface.cpp @@ -183,6 +183,12 @@ void ValidationSignals::UpdatedBlockTip(const CBlockIndex *pindexNew, const CBlo fInitialDownload); } +void ValidationSignals::ActiveTipChange(const CBlockIndex *new_tip, bool is_ibd) +{ + LOG_EVENT("%s: new block hash=%s block height=%d", __func__, new_tip->GetBlockHash().ToString(), new_tip->nHeight); + m_internals->Iterate([&](CValidationInterface& callbacks) { callbacks.ActiveTipChange(new_tip, is_ibd); }); +} + void ValidationSignals::TransactionAddedToMempool(const NewMempoolTransactionInfo& tx, uint64_t mempool_sequence) { auto event = [tx, mempool_sequence, this] { diff --git a/src/validationinterface.h b/src/validationinterface.h index 6f49a73c93c..3cc3566a605 100644 --- a/src/validationinterface.h +++ b/src/validationinterface.h @@ -61,6 +61,10 @@ protected: * Called on a background thread. Only called for the active chainstate. */ virtual void UpdatedBlockTip(const CBlockIndex *pindexNew, const CBlockIndex *pindexFork, bool fInitialDownload) {} + /** + * Notifies listeners any time the block chain tip changes, synchronously. + */ + virtual void ActiveTipChange(const CBlockIndex* new_tip, bool is_ibd) {}; /** * Notifies listeners of a transaction having been added to mempool. * @@ -214,6 +218,7 @@ public: void SyncWithValidationInterfaceQueue() LOCKS_EXCLUDED(cs_main); void UpdatedBlockTip(const CBlockIndex *, const CBlockIndex *, bool fInitialDownload); + void ActiveTipChange(const CBlockIndex*, bool); void TransactionAddedToMempool(const NewMempoolTransactionInfo&, uint64_t mempool_sequence); void TransactionRemovedFromMempool(const CTransactionRef&, MemPoolRemovalReason, uint64_t mempool_sequence); void MempoolTransactionsRemovedForBlock(const std::vector&, unsigned int nBlockHeight); From 18a43552509603ddf83b752fd7b4b973ba1dcf82 Mon Sep 17 00:00:00 2001 From: glozow Date: Thu, 16 May 2024 12:27:40 +0100 Subject: [PATCH 3/7] update recent_rejects filters on ActiveTipChange Resetting m_recent_rejects once per block is more efficient than comparing hashRecentRejectsChainTip with the chain tip every time we call AlreadyHaveTx. We keep hashRecentRejectsChainTip for now to assert that updates happen correctly; it is removed in the next commit. --- src/net_processing.cpp | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/src/net_processing.cpp b/src/net_processing.cpp index 95ef67b160c..62c4df91a5a 100644 --- a/src/net_processing.cpp +++ b/src/net_processing.cpp @@ -489,6 +489,8 @@ public: CTxMemPool& pool, node::Warnings& warnings, Options opts); /** Overridden from CValidationInterface. */ + void ActiveTipChange(const CBlockIndex* new_tip, bool) override + EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, !m_tx_download_mutex); void BlockConnected(ChainstateRole role, const std::shared_ptr& pblock, const CBlockIndex* pindexConnected) override EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, !m_tx_download_mutex); void BlockDisconnected(const std::shared_ptr &block, const CBlockIndex* pindex) override @@ -2074,6 +2076,22 @@ void PeerManagerImpl::StartScheduledTasks(CScheduler& scheduler) scheduler.scheduleFromNow([&] { ReattemptInitialBroadcast(scheduler); }, delta); } +void PeerManagerImpl::ActiveTipChange(const CBlockIndex* new_tip, bool is_ibd) +{ + AssertLockNotHeld(m_mempool.cs); + AssertLockNotHeld(m_tx_download_mutex); + + if (!is_ibd) { + LOCK(m_tx_download_mutex); + // If the chain tip has changed, previously rejected transactions might now be valid, e.g. due + // to a timelock. Reset the rejection filters to give those transactions another chance if we + // see them again. + m_recent_rejects.reset(); + m_recent_rejects_reconsiderable.reset(); + hashRecentRejectsChainTip = new_tip->GetBlockHash(); + } +} + /** * Evict orphan txn pool entries based on a newly connected * block, remember the recently confirmed transactions, and delete tracked @@ -2277,7 +2295,11 @@ bool PeerManagerImpl::AlreadyHaveTx(const GenTxid& gtxid, bool include_reconside AssertLockHeld(::cs_main); AssertLockHeld(m_tx_download_mutex); - if (m_chainman.ActiveChain().Tip()->GetBlockHash() != hashRecentRejectsChainTip) { + // Since recent_rejects is updated whenever the tip changes, and hashRecentRejectsChainTip is + // not set until the first time it is called outside of IBD, hashRecentRejectsChainTip should + // always be up to date with the current chain tip. + if (!Assume(hashRecentRejectsChainTip == uint256::ZERO || + hashRecentRejectsChainTip == m_chainman.ActiveChain().Tip()->GetBlockHash())) { // If the chain tip has changed previously rejected transactions // might be now valid, e.g. due to a nLockTime'd tx becoming valid, // or a double-spend. Reset the rejects filter and give those From 723ea0f9a5b5e3f3f58ea049a98299ff0ebde468 Mon Sep 17 00:00:00 2001 From: glozow Date: Fri, 29 Sep 2023 14:03:51 +0100 Subject: [PATCH 4/7] remove obsoleted hashRecentRejectsChainTip This also means AlreadyHaveTx no longer needs cs_main held. --- src/net_processing.cpp | 23 +---------------------- 1 file changed, 1 insertion(+), 22 deletions(-) diff --git a/src/net_processing.cpp b/src/net_processing.cpp index 62c4df91a5a..e7012650165 100644 --- a/src/net_processing.cpp +++ b/src/net_processing.cpp @@ -861,11 +861,9 @@ private: * - m_recent_rejects * - m_recent_rejects_reconsiderable (if include_reconsiderable = true) * - m_recent_confirmed_transactions - * Also responsible for resetting m_recent_rejects and m_recent_rejects_reconsiderable if the - * chain tip has changed. * */ bool AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable) - EXCLUSIVE_LOCKS_REQUIRED(cs_main, !m_recent_confirmed_transactions_mutex, m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, m_tx_download_mutex); /** * Filter for transactions that were recently rejected by the mempool. @@ -902,9 +900,6 @@ private: * Memory used: 1.3 MB */ CRollingBloomFilter m_recent_rejects GUARDED_BY(m_tx_download_mutex){120'000, 0.000'001}; - /** Block hash of chain tip the last time we reset m_recent_rejects and - * m_recent_rejects_reconsiderable. */ - uint256 hashRecentRejectsChainTip GUARDED_BY(m_tx_download_mutex); /** * Filter for: @@ -2088,7 +2083,6 @@ void PeerManagerImpl::ActiveTipChange(const CBlockIndex* new_tip, bool is_ibd) // see them again. m_recent_rejects.reset(); m_recent_rejects_reconsiderable.reset(); - hashRecentRejectsChainTip = new_tip->GetBlockHash(); } } @@ -2292,23 +2286,8 @@ void PeerManagerImpl::BlockChecked(const CBlock& block, const BlockValidationSta bool PeerManagerImpl::AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable) { - AssertLockHeld(::cs_main); AssertLockHeld(m_tx_download_mutex); - // Since recent_rejects is updated whenever the tip changes, and hashRecentRejectsChainTip is - // not set until the first time it is called outside of IBD, hashRecentRejectsChainTip should - // always be up to date with the current chain tip. - if (!Assume(hashRecentRejectsChainTip == uint256::ZERO || - hashRecentRejectsChainTip == m_chainman.ActiveChain().Tip()->GetBlockHash())) { - // If the chain tip has changed previously rejected transactions - // might be now valid, e.g. due to a nLockTime'd tx becoming valid, - // or a double-spend. Reset the rejects filter and give those - // txs a second chance. - hashRecentRejectsChainTip = m_chainman.ActiveChain().Tip()->GetBlockHash(); - m_recent_rejects.reset(); - m_recent_rejects_reconsiderable.reset(); - } - const uint256& hash = gtxid.GetHash(); if (gtxid.IsWtxid()) { From 61745c7451ec64b26c74f672c688e82efb3b96aa Mon Sep 17 00:00:00 2001 From: glozow Date: Tue, 16 Apr 2024 14:58:42 +0100 Subject: [PATCH 5/7] lock m_recent_confirmed_transactions using m_tx_download_mutex --- src/net_processing.cpp | 43 +++++++++++++++++------------------------- 1 file changed, 17 insertions(+), 26 deletions(-) diff --git a/src/net_processing.cpp b/src/net_processing.cpp index e7012650165..c2419947631 100644 --- a/src/net_processing.cpp +++ b/src/net_processing.cpp @@ -490,11 +490,11 @@ public: /** Overridden from CValidationInterface. */ void ActiveTipChange(const CBlockIndex* new_tip, bool) override - EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, !m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_tx_download_mutex); void BlockConnected(ChainstateRole role, const std::shared_ptr& pblock, const CBlockIndex* pindexConnected) override - EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, !m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_tx_download_mutex); void BlockDisconnected(const std::shared_ptr &block, const CBlockIndex* pindex) override - EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, !m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_tx_download_mutex); void UpdatedBlockTip(const CBlockIndex *pindexNew, const CBlockIndex *pindexFork, bool fInitialDownload) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex); void BlockChecked(const CBlock& block, const BlockValidationState& state) override @@ -507,9 +507,9 @@ public: void FinalizeNode(const CNode& node) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_headers_presync_mutex, !m_tx_download_mutex); bool HasAllDesirableServiceFlags(ServiceFlags services) const override; bool ProcessMessages(CNode* pfrom, std::atomic& interrupt) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex, !m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex, !m_tx_download_mutex); bool SendMessages(CNode* pto) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, g_msgproc_mutex, !m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_most_recent_block_mutex, g_msgproc_mutex, !m_tx_download_mutex); /** Implement PeerManager */ void StartScheduledTasks(CScheduler& scheduler) override; @@ -528,7 +528,7 @@ public: void UnitTestMisbehaving(NodeId peer_id) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex) { Misbehaving(*Assert(GetPeerRef(peer_id)), ""); }; void ProcessMessage(CNode& pfrom, const std::string& msg_type, DataStream& vRecv, const std::chrono::microseconds time_received, const std::atomic& interruptMsgProc) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex, !m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex, g_msgproc_mutex, !m_tx_download_mutex); void UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds) override; ServiceFlags GetDesirableServiceFlags(ServiceFlags services) const override; @@ -863,7 +863,7 @@ private: * - m_recent_confirmed_transactions * */ bool AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable) - EXCLUSIVE_LOCKS_REQUIRED(!m_recent_confirmed_transactions_mutex, m_tx_download_mutex); + EXCLUSIVE_LOCKS_REQUIRED(m_tx_download_mutex); /** * Filter for transactions that were recently rejected by the mempool. @@ -938,8 +938,7 @@ private: * transaction per day that would be inadvertently ignored (which is the * same probability that we have in the reject filter). */ - Mutex m_recent_confirmed_transactions_mutex; - CRollingBloomFilter m_recent_confirmed_transactions GUARDED_BY(m_recent_confirmed_transactions_mutex){48'000, 0.000'001}; + CRollingBloomFilter m_recent_confirmed_transactions GUARDED_BY(m_tx_download_mutex){48'000, 0.000'001}; /** * For sending `inv`s to inbound peers, we use a single (exponentially @@ -2119,20 +2118,15 @@ void PeerManagerImpl::BlockConnected( LOCK(m_tx_download_mutex); m_orphanage.EraseForBlock(*pblock); - { - LOCK(m_recent_confirmed_transactions_mutex); - for (const auto& ptx : pblock->vtx) { - m_recent_confirmed_transactions.insert(ptx->GetHash().ToUint256()); - if (ptx->HasWitness()) { - m_recent_confirmed_transactions.insert(ptx->GetWitnessHash().ToUint256()); - } + for (const auto& ptx : pblock->vtx) { + m_recent_confirmed_transactions.insert(ptx->GetHash().ToUint256()); + if (ptx->HasWitness()) { + m_recent_confirmed_transactions.insert(ptx->GetWitnessHash().ToUint256()); } } - { - for (const auto& ptx : pblock->vtx) { - m_txrequest.ForgetTxHash(ptx->GetHash()); - m_txrequest.ForgetTxHash(ptx->GetWitnessHash()); - } + for (const auto& ptx : pblock->vtx) { + m_txrequest.ForgetTxHash(ptx->GetHash()); + m_txrequest.ForgetTxHash(ptx->GetWitnessHash()); } } @@ -2146,7 +2140,7 @@ void PeerManagerImpl::BlockDisconnected(const std::shared_ptr &blo // block's worth of transactions in it, but that should be fine, since // presumably the most common case of relaying a confirmed transaction // should be just after a new block containing it is found. - LOCK(m_recent_confirmed_transactions_mutex); + LOCK(m_tx_download_mutex); m_recent_confirmed_transactions.reset(); } @@ -2310,10 +2304,7 @@ bool PeerManagerImpl::AlreadyHaveTx(const GenTxid& gtxid, bool include_reconside if (include_reconsiderable && m_recent_rejects_reconsiderable.contains(hash)) return true; - { - LOCK(m_recent_confirmed_transactions_mutex); - if (m_recent_confirmed_transactions.contains(hash)) return true; - } + if (m_recent_confirmed_transactions.contains(hash)) return true; return m_recent_rejects.contains(hash) || m_mempool.exists(gtxid); } From 6ff84069a5dd92303ed2ec28f0ec7c96bbda3938 Mon Sep 17 00:00:00 2001 From: glozow Date: Thu, 16 May 2024 10:32:51 +0100 Subject: [PATCH 6/7] remove obsoleted TxOrphanage::m_mutex The TxOrphanage is now guarded externally by m_tx_download_mutex. --- src/test/orphanage_tests.cpp | 6 ++---- src/txorphanage.cpp | 22 -------------------- src/txorphanage.h | 40 ++++++++++++++++-------------------- 3 files changed, 20 insertions(+), 48 deletions(-) diff --git a/src/test/orphanage_tests.cpp b/src/test/orphanage_tests.cpp index 3459aa9f0eb..082d090d7cb 100644 --- a/src/test/orphanage_tests.cpp +++ b/src/test/orphanage_tests.cpp @@ -21,15 +21,13 @@ BOOST_FIXTURE_TEST_SUITE(orphanage_tests, TestingSetup) class TxOrphanageTest : public TxOrphanage { public: - inline size_t CountOrphans() const EXCLUSIVE_LOCKS_REQUIRED(!m_mutex) + inline size_t CountOrphans() const { - LOCK(m_mutex); return m_orphans.size(); } - CTransactionRef RandomOrphan() EXCLUSIVE_LOCKS_REQUIRED(!m_mutex) + CTransactionRef RandomOrphan() { - LOCK(m_mutex); std::map::iterator it; it = m_orphans.lower_bound(Wtxid::FromUint256(InsecureRand256())); if (it == m_orphans.end()) diff --git a/src/txorphanage.cpp b/src/txorphanage.cpp index 3eaf53939df..29f1c19c17b 100644 --- a/src/txorphanage.cpp +++ b/src/txorphanage.cpp @@ -20,8 +20,6 @@ static constexpr auto ORPHAN_TX_EXPIRE_INTERVAL{5min}; bool TxOrphanage::AddTx(const CTransactionRef& tx, NodeId peer) { - LOCK(m_mutex); - const Txid& hash = tx->GetHash(); const Wtxid& wtxid = tx->GetWitnessHash(); if (m_orphans.count(wtxid)) @@ -55,13 +53,11 @@ bool TxOrphanage::AddTx(const CTransactionRef& tx, NodeId peer) int TxOrphanage::EraseTx(const Wtxid& wtxid) { - LOCK(m_mutex); return EraseTxNoLock(wtxid); } int TxOrphanage::EraseTxNoLock(const Wtxid& wtxid) { - AssertLockHeld(m_mutex); std::map::iterator it = m_orphans.find(wtxid); if (it == m_orphans.end()) return 0; @@ -97,8 +93,6 @@ int TxOrphanage::EraseTxNoLock(const Wtxid& wtxid) void TxOrphanage::EraseForPeer(NodeId peer) { - LOCK(m_mutex); - m_peer_work_set.erase(peer); int nErased = 0; @@ -116,8 +110,6 @@ void TxOrphanage::EraseForPeer(NodeId peer) void TxOrphanage::LimitOrphans(unsigned int max_orphans, FastRandomContext& rng) { - LOCK(m_mutex); - unsigned int nEvicted = 0; auto nNow{Now()}; if (m_next_sweep <= nNow) { @@ -150,9 +142,6 @@ void TxOrphanage::LimitOrphans(unsigned int max_orphans, FastRandomContext& rng) void TxOrphanage::AddChildrenToWorkSet(const CTransaction& tx) { - LOCK(m_mutex); - - for (unsigned int i = 0; i < tx.vout.size(); i++) { const auto it_by_prev = m_outpoint_to_orphan_it.find(COutPoint(tx.GetHash(), i)); if (it_by_prev != m_outpoint_to_orphan_it.end()) { @@ -171,14 +160,11 @@ void TxOrphanage::AddChildrenToWorkSet(const CTransaction& tx) bool TxOrphanage::HaveTx(const Wtxid& wtxid) const { - LOCK(m_mutex); return m_orphans.count(wtxid); } CTransactionRef TxOrphanage::GetTxToReconsider(NodeId peer) { - LOCK(m_mutex); - auto work_set_it = m_peer_work_set.find(peer); if (work_set_it != m_peer_work_set.end()) { auto& work_set = work_set_it->second; @@ -197,8 +183,6 @@ CTransactionRef TxOrphanage::GetTxToReconsider(NodeId peer) bool TxOrphanage::HaveTxToReconsider(NodeId peer) { - LOCK(m_mutex); - auto work_set_it = m_peer_work_set.find(peer); if (work_set_it != m_peer_work_set.end()) { auto& work_set = work_set_it->second; @@ -209,8 +193,6 @@ bool TxOrphanage::HaveTxToReconsider(NodeId peer) void TxOrphanage::EraseForBlock(const CBlock& block) { - LOCK(m_mutex); - std::vector vOrphanErase; for (const CTransactionRef& ptx : block.vtx) { @@ -239,8 +221,6 @@ void TxOrphanage::EraseForBlock(const CBlock& block) std::vector TxOrphanage::GetChildrenFromSamePeer(const CTransactionRef& parent, NodeId nodeid) const { - LOCK(m_mutex); - // First construct a vector of iterators to ensure we do not return duplicates of the same tx // and so we can sort by nTimeExpire. std::vector iters; @@ -281,8 +261,6 @@ std::vector TxOrphanage::GetChildrenFromSamePeer(const CTransac std::vector> TxOrphanage::GetChildrenFromDifferentPeer(const CTransactionRef& parent, NodeId nodeid) const { - LOCK(m_mutex); - // First construct vector of iterators to ensure we do not return duplicates of the same tx. std::vector iters; diff --git a/src/txorphanage.h b/src/txorphanage.h index 3083c8467f3..9917266749a 100644 --- a/src/txorphanage.h +++ b/src/txorphanage.h @@ -22,55 +22,51 @@ class TxOrphanage { public: /** Add a new orphan transaction */ - bool AddTx(const CTransactionRef& tx, NodeId peer) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + bool AddTx(const CTransactionRef& tx, NodeId peer); /** Check if we already have an orphan transaction (by wtxid only) */ - bool HaveTx(const Wtxid& wtxid) const EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + bool HaveTx(const Wtxid& wtxid) const; /** Extract a transaction from a peer's work set * Returns nullptr if there are no transactions to work on. * Otherwise returns the transaction reference, and removes * it from the work set. */ - CTransactionRef GetTxToReconsider(NodeId peer) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + CTransactionRef GetTxToReconsider(NodeId peer); /** Erase an orphan by wtxid */ - int EraseTx(const Wtxid& wtxid) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + int EraseTx(const Wtxid& wtxid); /** Erase all orphans announced by a peer (eg, after that peer disconnects) */ - void EraseForPeer(NodeId peer) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + void EraseForPeer(NodeId peer); /** Erase all orphans included in or invalidated by a new block */ - void EraseForBlock(const CBlock& block) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + void EraseForBlock(const CBlock& block); /** Limit the orphanage to the given maximum */ - void LimitOrphans(unsigned int max_orphans, FastRandomContext& rng) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + void LimitOrphans(unsigned int max_orphans, FastRandomContext& rng); /** Add any orphans that list a particular tx as a parent into the from peer's work set */ - void AddChildrenToWorkSet(const CTransaction& tx) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex);; + void AddChildrenToWorkSet(const CTransaction& tx); /** Does this peer have any work to do? */ - bool HaveTxToReconsider(NodeId peer) EXCLUSIVE_LOCKS_REQUIRED(!m_mutex);; + bool HaveTxToReconsider(NodeId peer); /** Get all children that spend from this tx and were received from nodeid. Sorted from most * recent to least recent. */ - std::vector GetChildrenFromSamePeer(const CTransactionRef& parent, NodeId nodeid) const EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + std::vector GetChildrenFromSamePeer(const CTransactionRef& parent, NodeId nodeid) const; /** Get all children that spend from this tx but were not received from nodeid. Also return * which peer provided each tx. */ - std::vector> GetChildrenFromDifferentPeer(const CTransactionRef& parent, NodeId nodeid) const EXCLUSIVE_LOCKS_REQUIRED(!m_mutex); + std::vector> GetChildrenFromDifferentPeer(const CTransactionRef& parent, NodeId nodeid) const; /** Return how many entries exist in the orphange */ - size_t Size() EXCLUSIVE_LOCKS_REQUIRED(!m_mutex) + size_t Size() { - LOCK(m_mutex); return m_orphans.size(); } protected: - /** Guards orphan transactions */ - mutable Mutex m_mutex; - struct OrphanTx { CTransactionRef tx; NodeId fromPeer; @@ -80,10 +76,10 @@ protected: /** Map from wtxid to orphan transaction record. Limited by * -maxorphantx/DEFAULT_MAX_ORPHAN_TRANSACTIONS */ - std::map m_orphans GUARDED_BY(m_mutex); + std::map m_orphans; /** Which peer provided the orphans that need to be reconsidered */ - std::map> m_peer_work_set GUARDED_BY(m_mutex); + std::map> m_peer_work_set; using OrphanMap = decltype(m_orphans); @@ -98,16 +94,16 @@ protected: /** Index from the parents' COutPoint into the m_orphans. Used * to remove orphan transactions from the m_orphans */ - std::map> m_outpoint_to_orphan_it GUARDED_BY(m_mutex); + std::map> m_outpoint_to_orphan_it; /** Orphan transactions in vector for quick random eviction */ - std::vector m_orphan_list GUARDED_BY(m_mutex); + std::vector m_orphan_list; /** Erase an orphan by wtxid */ - int EraseTxNoLock(const Wtxid& wtxid) EXCLUSIVE_LOCKS_REQUIRED(m_mutex); + int EraseTxNoLock(const Wtxid& wtxid); /** Timestamp for the next scheduled sweep of expired orphans */ - NodeSeconds m_next_sweep GUARDED_BY(m_mutex){0s}; + NodeSeconds m_next_sweep{0s}; }; #endif // BITCOIN_TXORPHANAGE_H From c85accecafc20f6a6ae94bdf6cdd3ba9747218fd Mon Sep 17 00:00:00 2001 From: glozow Date: Thu, 23 May 2024 13:25:41 +0100 Subject: [PATCH 7/7] [refactor] delete EraseTxNoLock, just use EraseTx --- src/txorphanage.cpp | 13 ++++--------- src/txorphanage.h | 3 --- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/src/txorphanage.cpp b/src/txorphanage.cpp index 29f1c19c17b..df9b96e64d7 100644 --- a/src/txorphanage.cpp +++ b/src/txorphanage.cpp @@ -52,11 +52,6 @@ bool TxOrphanage::AddTx(const CTransactionRef& tx, NodeId peer) } int TxOrphanage::EraseTx(const Wtxid& wtxid) -{ - return EraseTxNoLock(wtxid); -} - -int TxOrphanage::EraseTxNoLock(const Wtxid& wtxid) { std::map::iterator it = m_orphans.find(wtxid); if (it == m_orphans.end()) @@ -102,7 +97,7 @@ void TxOrphanage::EraseForPeer(NodeId peer) // increment to avoid iterator becoming invalid after erasure const auto& [wtxid, orphan] = *iter++; if (orphan.fromPeer == peer) { - nErased += EraseTxNoLock(wtxid); + nErased += EraseTx(wtxid); } } if (nErased > 0) LogPrint(BCLog::TXPACKAGES, "Erased %d orphan transaction(s) from peer=%d\n", nErased, peer); @@ -121,7 +116,7 @@ void TxOrphanage::LimitOrphans(unsigned int max_orphans, FastRandomContext& rng) { std::map::iterator maybeErase = iter++; if (maybeErase->second.nTimeExpire <= nNow) { - nErased += EraseTxNoLock(maybeErase->second.tx->GetWitnessHash()); + nErased += EraseTx(maybeErase->second.tx->GetWitnessHash()); } else { nMinExpTime = std::min(maybeErase->second.nTimeExpire, nMinExpTime); } @@ -134,7 +129,7 @@ void TxOrphanage::LimitOrphans(unsigned int max_orphans, FastRandomContext& rng) { // Evict a random orphan: size_t randompos = rng.randrange(m_orphan_list.size()); - EraseTxNoLock(m_orphan_list[randompos]->second.tx->GetWitnessHash()); + EraseTx(m_orphan_list[randompos]->second.tx->GetWitnessHash()); ++nEvicted; } if (nEvicted > 0) LogPrint(BCLog::TXPACKAGES, "orphanage overflow, removed %u tx\n", nEvicted); @@ -213,7 +208,7 @@ void TxOrphanage::EraseForBlock(const CBlock& block) if (vOrphanErase.size()) { int nErased = 0; for (const auto& orphanHash : vOrphanErase) { - nErased += EraseTxNoLock(orphanHash); + nErased += EraseTx(orphanHash); } LogPrint(BCLog::TXPACKAGES, "Erased %d orphan transaction(s) included or conflicted by block\n", nErased); } diff --git a/src/txorphanage.h b/src/txorphanage.h index 9917266749a..207b79e0094 100644 --- a/src/txorphanage.h +++ b/src/txorphanage.h @@ -99,9 +99,6 @@ protected: /** Orphan transactions in vector for quick random eviction */ std::vector m_orphan_list; - /** Erase an orphan by wtxid */ - int EraseTxNoLock(const Wtxid& wtxid); - /** Timestamp for the next scheduled sweep of expired orphans */ NodeSeconds m_next_sweep{0s}; };