Merge bitcoin/bitcoin#30110: refactor: TxDownloadManager + fuzzing

0f4bc63585 [fuzz] txdownloadman and txdownload_impl (glozow)
699643f23a [unit test] MempoolRejectedTx (glozow)
fa584cbe72 [p2p] add TxDownloadOptions bool to make TxRequestTracker deterministic (glozow)
f803c8ce8d [p2p] filter 1p1c for child txid in recent rejects (glozow)
5269d57e6d [p2p] don't process orphan if in recent rejects (glozow)
2266eba43a [p2p] don't find 1p1cs for reconsiderable txns that are AlreadyHaveTx (glozow)
fa7027d0fc [refactor] add CheckIsEmpty and GetOrphanTransactions, remove access to TxDownloadMan internals (glozow)
969b07237b [refactor] wrap {Have,Get}TxToReconsider in txdownload (glozow)
f150fb94e7 [refactor] make AlreadyHaveTx and Find1P1CPackage private to TxDownloadImpl (glozow)
1e08195135 [refactor] move new tx logic to txdownload (glozow)
257568eab5 [refactor] move invalid package processing to TxDownload (glozow)
c4ce0c1218 [refactor] move invalid tx processing to TxDownload (glozow)
c6b21749ca [refactor] move valid tx processing to TxDownload (glozow)
a8cf3b6e84 [refactor] move Find1P1CPackage to txdownload (glozow)
f497414ce7 [refactor] put peerman tasks at the end of ProcessInvalidTx (glozow)
6797bc42a7 [p2p] restrict RecursiveDynamicUsage of orphans added to vExtraTxnForCompact (glozow)
798cc8f5aa [refactor] move Find1P1CPackage into ProcessInvalidTx (glozow)
416fbc952b [refactor] move new orphan handling to ProcessInvalidTx (glozow)
c8e67b9169 [refactor] move ProcessInvalidTx and ProcessValidTx definitions down (glozow)
3a41926d1b [refactor] move notfound processing to txdownload (glozow)
042a97ce7f [refactor] move tx inv/getdata handling to txdownload (glozow)
58e09f244b [p2p] don't log tx invs when in IBD (glozow)
288865338f [refactor] rename maybe_add_extra_compact_tx to first_time_failure (glozow)
f48d36cd97 [refactor] move peer (dis)connection logic to TxDownload (glozow)
f61d9e4b4b [refactor] move AlreadyHaveTx to TxDownload (glozow)
84e4ef843d [txdownload] add read-only reference to mempool (glozow)
af918349de [refactor] move ValidationInterface functions to TxDownloadManager (glozow)
f6c860efb1 [doc] fix typo in m_lazy_recent_confirmed_transactions doc (glozow)
5f9004e155 [refactor] add TxDownloadManager wrapping TxOrphanage, TxRequestTracker, and bloom filters (glozow)

Pull request description:

  Part of #27463.

  This PR does 3 things:

  (1) It modularizes transaction download logic into a `TxDownloadManager`. Transaction download logic refers to the process of deciding what transactions to request, download, and validate.[1] There should be no behavior changes. Using `--color_moved=dimmed_zebra -w` may help.
  (2) It adds unit and fuzz (🪄) testing for transaction download.
  (3) It makes a few small behavioral changes:
  - Stop (debug-only) logging tx invs during IBD
  - Just like all other transactions, require orphans have RecursiveDynamicUsage < 100k before adding to vExtraTxnForCompact
  - Don't return a 1p1c that contains a parent or child in recent rejects. Don't process any orphan already in recent rejects. These cases should not happen in actual node operation; it's just to allow tighter sanity checks during fuzzing.

  There are several benefits to this interface, such as:
  - Unit test coverage and fuzzing for logic that currently isn't feasible to test as thoroughly (without lots of overhead) and/or currently only lightly tested through `assert_debug_log` (not good) in functional tests.
  - When we add more functionality (e.g. package relay messages, more robust orphan handling), the vast majority of it will be within `TxDownloadManager` instead of `PeerManager`, making it easier to review and test. See #28031 for what this looks like.
  - `PeerManager` will no longer know anything about / have access to `TxOrphanage`, `TxRequestTracker` or the rejection caches. Its primary interface with `TxDownloadManager` would be much simpler:
      - Passing on  `ValidationInterface` callbacks
      - Telling `txdownloadman` when a peer {connects, disconnects}
      - Telling `txdownloadman`when a {transaction, package} is {accepted, rejected} from mempool
      - Telling `txdownloadman` when invs, notfounds, and txs are received.
      - Getting instructions on what to download.
      - Getting instructions on what {transactions, packages, orphans} to validate.
      - Get whether a peer `HaveMoreWork` for the `ProessMessages` loop
  - (todo) Thread-safety can be handled internally.

  [1]: This module is concerned with tx *download*, not upload. It excludes transaction announcements/gossip which happens after we download/accept a transaction. Txreconciliation (erlay) is excluded from this module, as it only relates to deciding which `inv`s to send or helping the other peer decide which `inv`s to send. It is independent from this logic.

ACKs for top commit:
  achow101:
    light ACK 0f4bc63585
  theStack:
    ACK 0f4bc63585
  instagibbs:
    reACK 0f4bc63585
  naumenkogs:
    ACK 0f4bc63585

Tree-SHA512: 84ab8ef8a0fc705eb829d7f7d6885f28944aaa42b03172f256a42605677b3e783919bb900d4e3b8589f85a0c387dfbd972bcd61d252d44a88c6aaa90e4bf920f
This commit is contained in:
Ava Chow 2024-10-29 14:41:12 -04:00
commit 7b66815b16
No known key found for this signature in database
GPG key ID: 17565732E08E5E41
10 changed files with 1771 additions and 549 deletions

View file

@ -246,6 +246,7 @@ add_library(bitcoin_node STATIC EXCLUDE_FROM_ALL
node/psbt.cpp node/psbt.cpp
node/timeoffsets.cpp node/timeoffsets.cpp
node/transaction.cpp node/transaction.cpp
node/txdownloadman_impl.cpp
node/txreconciliation.cpp node/txreconciliation.cpp
node/utxo_snapshot.cpp node/utxo_snapshot.cpp
node/warnings.cpp node/warnings.cpp

View file

@ -24,6 +24,7 @@
#include <netmessagemaker.h> #include <netmessagemaker.h>
#include <node/blockstorage.h> #include <node/blockstorage.h>
#include <node/timeoffsets.h> #include <node/timeoffsets.h>
#include <node/txdownloadman.h>
#include <node/txreconciliation.h> #include <node/txreconciliation.h>
#include <node/warnings.h> #include <node/warnings.h>
#include <policy/fees.h> #include <policy/fees.h>
@ -88,22 +89,6 @@ static constexpr auto PING_INTERVAL{2min};
static const unsigned int MAX_LOCATOR_SZ = 101; static const unsigned int MAX_LOCATOR_SZ = 101;
/** The maximum number of entries in an 'inv' protocol message */ /** The maximum number of entries in an 'inv' protocol message */
static const unsigned int MAX_INV_SZ = 50000; static const unsigned int MAX_INV_SZ = 50000;
/** Maximum number of in-flight transaction requests from a peer. It is not a hard limit, but the threshold at which
* point the OVERLOADED_PEER_TX_DELAY kicks in. */
static constexpr int32_t MAX_PEER_TX_REQUEST_IN_FLIGHT = 100;
/** Maximum number of transactions to consider for requesting, per peer. It provides a reasonable DoS limit to
* per-peer memory usage spent on announcements, while covering peers continuously sending INVs at the maximum
* rate (by our own policy, see INVENTORY_BROADCAST_PER_SECOND) for several minutes, while not receiving
* the actual transaction (from any peer) in response to requests for them. */
static constexpr int32_t MAX_PEER_TX_ANNOUNCEMENTS = 5000;
/** How long to delay requesting transactions via txids, if we have wtxid-relaying peers */
static constexpr auto TXID_RELAY_DELAY{2s};
/** How long to delay requesting transactions from non-preferred peers */
static constexpr auto NONPREF_PEER_TX_DELAY{2s};
/** How long to delay requesting transactions from overloaded peers (see MAX_PEER_TX_REQUEST_IN_FLIGHT). */
static constexpr auto OVERLOADED_PEER_TX_DELAY{2s};
/** How long to wait before downloading a transaction from an additional peer */
static constexpr auto GETDATA_TX_INTERVAL{60s};
/** Limit to avoid sending big packets. Not used in processing incoming GETDATA for compatibility */ /** Limit to avoid sending big packets. Not used in processing incoming GETDATA for compatibility */
static const unsigned int MAX_GETDATA_SZ = 1000; static const unsigned int MAX_GETDATA_SZ = 1000;
/** Number of blocks that can be requested at any given time from a single peer. */ /** Number of blocks that can be requested at any given time from a single peer. */
@ -155,7 +140,7 @@ static constexpr unsigned int INVENTORY_BROADCAST_TARGET = INVENTORY_BROADCAST_P
/** Maximum number of inventory items to send per transmission. */ /** Maximum number of inventory items to send per transmission. */
static constexpr unsigned int INVENTORY_BROADCAST_MAX = 1000; static constexpr unsigned int INVENTORY_BROADCAST_MAX = 1000;
static_assert(INVENTORY_BROADCAST_MAX >= INVENTORY_BROADCAST_TARGET, "INVENTORY_BROADCAST_MAX too low"); static_assert(INVENTORY_BROADCAST_MAX >= INVENTORY_BROADCAST_TARGET, "INVENTORY_BROADCAST_MAX too low");
static_assert(INVENTORY_BROADCAST_MAX <= MAX_PEER_TX_ANNOUNCEMENTS, "INVENTORY_BROADCAST_MAX too high"); static_assert(INVENTORY_BROADCAST_MAX <= node::MAX_PEER_TX_ANNOUNCEMENTS, "INVENTORY_BROADCAST_MAX too high");
/** Average delay between feefilter broadcasts in seconds. */ /** Average delay between feefilter broadcasts in seconds. */
static constexpr auto AVG_FEEFILTER_BROADCAST_INTERVAL{10min}; static constexpr auto AVG_FEEFILTER_BROADCAST_INTERVAL{10min};
/** Maximum feefilter broadcast delay after significant change. */ /** Maximum feefilter broadcast delay after significant change. */
@ -580,12 +565,18 @@ private:
bool MaybeDiscourageAndDisconnect(CNode& pnode, Peer& peer); bool MaybeDiscourageAndDisconnect(CNode& pnode, Peer& peer);
/** Handle a transaction whose result was not MempoolAcceptResult::ResultType::VALID. /** Handle a transaction whose result was not MempoolAcceptResult::ResultType::VALID.
* @param[in] maybe_add_extra_compact_tx Whether this tx should be added to vExtraTxnForCompact. * @param[in] first_time_failure Whether we should consider inserting into vExtraTxnForCompact, adding
* a new orphan to resolve, or looking for a package to submit.
* Set to true for transactions just received over p2p.
* Set to false if the tx has already been rejected before, * Set to false if the tx has already been rejected before,
* e.g. is an orphan, to avoid adding duplicate entries. * e.g. is already in the orphanage, to avoid adding duplicate entries.
* Updates m_txrequest, m_lazy_recent_rejects, m_lazy_recent_rejects_reconsiderable, m_orphanage, and vExtraTxnForCompact. */ * Updates m_txrequest, m_lazy_recent_rejects, m_lazy_recent_rejects_reconsiderable, m_orphanage, and vExtraTxnForCompact.
void ProcessInvalidTx(NodeId nodeid, const CTransactionRef& tx, const TxValidationState& result, *
bool maybe_add_extra_compact_tx) * @returns a PackageToValidate if this transaction has a reconsiderable failure and an eligible package was found,
* or std::nullopt otherwise.
*/
std::optional<node::PackageToValidate> ProcessInvalidTx(NodeId nodeid, const CTransactionRef& tx, const TxValidationState& result,
bool first_time_failure)
EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex); EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex);
/** Handle a transaction whose result was MempoolAcceptResult::ResultType::VALID. /** Handle a transaction whose result was MempoolAcceptResult::ResultType::VALID.
@ -593,40 +584,10 @@ private:
void ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, const std::list<CTransactionRef>& replaced_transactions) void ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, const std::list<CTransactionRef>& replaced_transactions)
EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex); EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex);
struct PackageToValidate {
const Package m_txns;
const std::vector<NodeId> m_senders;
/** Construct a 1-parent-1-child package. */
explicit PackageToValidate(const CTransactionRef& parent,
const CTransactionRef& child,
NodeId parent_sender,
NodeId child_sender) :
m_txns{parent, child},
m_senders {parent_sender, child_sender}
{}
std::string ToString() const {
Assume(m_txns.size() == 2);
return strprintf("parent %s (wtxid=%s, sender=%d) + child %s (wtxid=%s, sender=%d)",
m_txns.front()->GetHash().ToString(),
m_txns.front()->GetWitnessHash().ToString(),
m_senders.front(),
m_txns.back()->GetHash().ToString(),
m_txns.back()->GetWitnessHash().ToString(),
m_senders.back());
}
};
/** Handle the results of package validation: calls ProcessValidTx and ProcessInvalidTx for /** Handle the results of package validation: calls ProcessValidTx and ProcessInvalidTx for
* individual transactions, and caches rejection for the package as a group. * individual transactions, and caches rejection for the package as a group.
*/ */
void ProcessPackageResult(const PackageToValidate& package_to_validate, const PackageMempoolAcceptResult& package_result) void ProcessPackageResult(const node::PackageToValidate& package_to_validate, const PackageMempoolAcceptResult& package_result)
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<PackageToValidate> Find1P1CPackage(const CTransactionRef& ptx, NodeId nodeid)
EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex); EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, g_msgproc_mutex, m_tx_download_mutex);
/** /**
@ -719,12 +680,6 @@ private:
void SendBlockTransactions(CNode& pfrom, Peer& peer, const CBlock& block, const BlockTransactionsRequest& req); void SendBlockTransactions(CNode& pfrom, Peer& peer, const CBlock& block, const BlockTransactionsRequest& req);
/** Register with TxRequestTracker that an INV has been received from a
* 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, m_tx_download_mutex);
/** Send a message to a peer */ /** Send a message to a peer */
void PushMessage(CNode& node, CSerializedNetMsg&& msg) const { m_connman.PushMessage(&node, std::move(msg)); } void PushMessage(CNode& node, CSerializedNetMsg&& msg) const { m_connman.PushMessage(&node, std::move(msg)); }
template <typename... Args> template <typename... Args>
@ -781,7 +736,8 @@ private:
* - Each data structure's limits hold (m_orphanage max size, m_txrequest per-peer limits, etc). * - Each data structure's limits hold (m_orphanage max size, m_txrequest per-peer limits, etc).
*/ */
Mutex m_tx_download_mutex ACQUIRED_BEFORE(m_mempool.cs); Mutex m_tx_download_mutex ACQUIRED_BEFORE(m_mempool.cs);
TxRequestTracker m_txrequest GUARDED_BY(m_tx_download_mutex); node::TxDownloadManager m_txdownloadman GUARDED_BY(m_tx_download_mutex);
std::unique_ptr<TxReconciliationTracker> m_txreconciliation; std::unique_ptr<TxReconciliationTracker> m_txreconciliation;
/** The height of the best chain */ /** The height of the best chain */
@ -852,124 +808,6 @@ private:
/** Stalling timeout for blocks in IBD */ /** Stalling timeout for blocks in IBD */
std::atomic<std::chrono::seconds> m_block_stalling_timeout{BLOCK_STALLING_TIMEOUT_DEFAULT}; std::atomic<std::chrono::seconds> m_block_stalling_timeout{BLOCK_STALLING_TIMEOUT_DEFAULT};
/** Check whether we already have this gtxid in:
* - mempool
* - orphanage
* - m_lazy_recent_rejects
* - m_lazy_recent_rejects_reconsiderable (if include_reconsiderable = true)
* - m_lazy_recent_confirmed_transactions
* */
bool AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable)
EXCLUSIVE_LOCKS_REQUIRED(m_tx_download_mutex);
/**
* Filter for transactions that were recently rejected by the mempool.
* These are not rerequested until the chain tip changes, at which point
* the entire filter is reset.
*
* Without this filter we'd be re-requesting txs from each of our peers,
* increasing bandwidth consumption considerably. For instance, with 100
* peers, half of which relay a tx we don't accept, that might be a 50x
* bandwidth increase. A flooding attacker attempting to roll-over the
* filter using minimum-sized, 60byte, transactions might manage to send
* 1000/sec if we have fast peers, so we pick 120,000 to give our peers a
* two minute window to send invs to us.
*
* Decreasing the false positive rate is fairly cheap, so we pick one in a
* million to make it highly unlikely for users to have issues with this
* filter.
*
* We typically only add wtxids to this filter. For non-segwit
* transactions, the txid == wtxid, so this only prevents us from
* re-downloading non-segwit transactions when communicating with
* non-wtxidrelay peers -- which is important for avoiding malleation
* attacks that could otherwise interfere with transaction relay from
* non-wtxidrelay peers. For communicating with wtxidrelay peers, having
* the reject filter store wtxids is exactly what we want to avoid
* redownload of a rejected transaction.
*
* In cases where we can tell that a segwit transaction will fail
* validation no matter the witness, we may add the txid of such
* transaction to the filter as well. This can be helpful when
* communicating with txid-relay peers or if we were to otherwise fetch a
* transaction via txid (eg in our orphan handling).
*
* Memory used: 1.3 MB
*/
std::unique_ptr<CRollingBloomFilter> m_lazy_recent_rejects GUARDED_BY(m_tx_download_mutex){nullptr};
CRollingBloomFilter& RecentRejectsFilter() EXCLUSIVE_LOCKS_REQUIRED(m_tx_download_mutex)
{
AssertLockHeld(m_tx_download_mutex);
if (!m_lazy_recent_rejects) {
m_lazy_recent_rejects = std::make_unique<CRollingBloomFilter>(120'000, 0.000'001);
}
return *m_lazy_recent_rejects;
}
/**
* Filter for:
* (1) wtxids of transactions that were recently rejected by the mempool but are
* eligible for reconsideration if submitted with other transactions.
* (2) packages (see GetPackageHash) we have already rejected before and should not retry.
*
* Similar to m_lazy_recent_rejects, this filter is used to save bandwidth when e.g. all of our peers
* have larger mempools and thus lower minimum feerates than us.
*
* When a transaction's error is TxValidationResult::TX_RECONSIDERABLE (in a package or by
* itself), add its wtxid to this filter. When a package fails for any reason, add the combined
* hash to this filter.
*
* Upon receiving an announcement for a transaction, if it exists in this filter, do not
* download the txdata. When considering packages, if it exists in this filter, drop it.
*
* Reset this filter when the chain tip changes.
*
* Parameters are picked to be the same as m_lazy_recent_rejects, with the same rationale.
*/
std::unique_ptr<CRollingBloomFilter> m_lazy_recent_rejects_reconsiderable GUARDED_BY(m_tx_download_mutex){nullptr};
CRollingBloomFilter& RecentRejectsReconsiderableFilter() EXCLUSIVE_LOCKS_REQUIRED(m_tx_download_mutex)
{
AssertLockHeld(m_tx_download_mutex);
if (!m_lazy_recent_rejects_reconsiderable) {
m_lazy_recent_rejects_reconsiderable = std::make_unique<CRollingBloomFilter>(120'000, 0.000'001);
}
return *m_lazy_recent_rejects_reconsiderable;
}
/*
* Filter for transactions that have been recently confirmed.
* We use this to avoid requesting transactions that have already been
* confirnmed.
*
* Blocks don't typically have more than 4000 transactions, so this should
* be at least six blocks (~1 hr) worth of transactions that we can store,
* inserting both a txid and wtxid for every observed transaction.
* If the number of transactions appearing in a block goes up, or if we are
* seeing getdata requests more than an hour after initial announcement, we
* can increase this number.
* The false positive rate of 1/1M should come out to less than 1
* transaction per day that would be inadvertently ignored (which is the
* same probability that we have in the reject filter).
*/
std::unique_ptr<CRollingBloomFilter> m_lazy_recent_confirmed_transactions GUARDED_BY(m_tx_download_mutex){nullptr};
CRollingBloomFilter& RecentConfirmedTransactionsFilter() EXCLUSIVE_LOCKS_REQUIRED(m_tx_download_mutex)
{
AssertLockHeld(m_tx_download_mutex);
if (!m_lazy_recent_confirmed_transactions) {
m_lazy_recent_confirmed_transactions = std::make_unique<CRollingBloomFilter>(48'000, 0.000'001);
}
return *m_lazy_recent_confirmed_transactions;
}
/** /**
* For sending `inv`s to inbound peers, we use a single (exponentially * For sending `inv`s to inbound peers, we use a single (exponentially
* distributed) timer for all peers. If we used a separate timer for each * distributed) timer for all peers. If we used a separate timer for each
@ -1104,9 +942,6 @@ private:
/** Number of peers from which we're downloading blocks. */ /** Number of peers from which we're downloading blocks. */
int m_peers_downloading_from GUARDED_BY(cs_main) = 0; int m_peers_downloading_from GUARDED_BY(cs_main) = 0;
/** Storage for orphan information */
TxOrphanage m_orphanage GUARDED_BY(m_tx_download_mutex);
void AddToCompactExtraTransactions(const CTransactionRef& tx) EXCLUSIVE_LOCKS_REQUIRED(g_msgproc_mutex); void AddToCompactExtraTransactions(const CTransactionRef& tx) EXCLUSIVE_LOCKS_REQUIRED(g_msgproc_mutex);
/** Orphan/conflicted/etc transactions that are kept for compact block reconstruction. /** Orphan/conflicted/etc transactions that are kept for compact block reconstruction.
@ -1678,34 +1513,6 @@ 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 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
return;
}
const CNodeState* state = State(nodeid);
// Decide the TxRequestTracker parameters for this announcement:
// - "preferred": if fPreferredDownload is set (= outbound, or NetPermissionFlags::NoBan permission)
// - "reqtime": current time plus delays for:
// - NONPREF_PEER_TX_DELAY for announcements from non-preferred connections
// - TXID_RELAY_DELAY for txid announcements while wtxid peers are available
// - OVERLOADED_PEER_TX_DELAY for announcements from peers which have at least
// MAX_PEER_TX_REQUEST_IN_FLIGHT requests in flight (and don't have NetPermissionFlags::Relay).
auto delay{0us};
const bool preferred = state->fPreferredDownload;
if (!preferred) delay += NONPREF_PEER_TX_DELAY;
if (!gtxid.IsWtxid() && m_wtxid_relay_peers > 0) delay += TXID_RELAY_DELAY;
const bool overloaded = !node.HasPermission(NetPermissionFlags::Relay) &&
m_txrequest.CountInFlight(nodeid) >= MAX_PEER_TX_REQUEST_IN_FLIGHT;
if (overloaded) delay += OVERLOADED_PEER_TX_DELAY;
m_txrequest.ReceivedInv(nodeid, gtxid, preferred, current_time + delay);
}
void PeerManagerImpl::UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds) void PeerManagerImpl::UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds)
{ {
LOCK(cs_main); LOCK(cs_main);
@ -1720,10 +1527,7 @@ void PeerManagerImpl::InitializeNode(const CNode& node, ServiceFlags our_service
LOCK(cs_main); // For m_node_states LOCK(cs_main); // For m_node_states
m_node_states.try_emplace(m_node_states.end(), nodeid); m_node_states.try_emplace(m_node_states.end(), nodeid);
} }
{ WITH_LOCK(m_tx_download_mutex, m_txdownloadman.CheckIsEmpty(nodeid));
LOCK(m_tx_download_mutex);
assert(m_txrequest.Count(nodeid) == 0);
}
if (NetPermissions::HasFlag(node.m_permission_flags, NetPermissionFlags::BloomFilter)) { if (NetPermissions::HasFlag(node.m_permission_flags, NetPermissionFlags::BloomFilter)) {
our_services = static_cast<ServiceFlags>(our_services | NODE_BLOOM); our_services = static_cast<ServiceFlags>(our_services | NODE_BLOOM);
@ -1791,8 +1595,7 @@ void PeerManagerImpl::FinalizeNode(const CNode& node)
} }
{ {
LOCK(m_tx_download_mutex); LOCK(m_tx_download_mutex);
m_orphanage.EraseForPeer(nodeid); m_txdownloadman.DisconnectedPeer(nodeid);
m_txrequest.DisconnectedPeer(nodeid);
} }
if (m_txreconciliation) m_txreconciliation->ForgetPeer(nodeid); if (m_txreconciliation) m_txreconciliation->ForgetPeer(nodeid);
m_num_preferred_download_peers -= state->fPreferredDownload; m_num_preferred_download_peers -= state->fPreferredDownload;
@ -1810,9 +1613,7 @@ void PeerManagerImpl::FinalizeNode(const CNode& node)
assert(m_peers_downloading_from == 0); assert(m_peers_downloading_from == 0);
assert(m_outbound_peers_with_protect_from_disconnect == 0); assert(m_outbound_peers_with_protect_from_disconnect == 0);
assert(m_wtxid_relay_peers == 0); assert(m_wtxid_relay_peers == 0);
LOCK(m_tx_download_mutex); WITH_LOCK(m_tx_download_mutex, m_txdownloadman.CheckIsEmpty());
assert(m_txrequest.Size() == 0);
assert(m_orphanage.Size() == 0);
} }
} // cs_main } // cs_main
if (node.fSuccessfullyConnected && if (node.fSuccessfullyConnected &&
@ -1921,7 +1722,7 @@ bool PeerManagerImpl::GetNodeStateStats(NodeId nodeid, CNodeStateStats& stats) c
std::vector<TxOrphanage::OrphanTxBase> PeerManagerImpl::GetOrphanTransactions() std::vector<TxOrphanage::OrphanTxBase> PeerManagerImpl::GetOrphanTransactions()
{ {
LOCK(m_tx_download_mutex); LOCK(m_tx_download_mutex);
return m_orphanage.GetOrphanTransactions(); return m_txdownloadman.GetOrphanTransactions();
} }
PeerManagerInfo PeerManagerImpl::GetInfo() const PeerManagerInfo PeerManagerImpl::GetInfo() const
@ -2088,6 +1889,7 @@ PeerManagerImpl::PeerManagerImpl(CConnman& connman, AddrMan& addrman,
m_banman(banman), m_banman(banman),
m_chainman(chainman), m_chainman(chainman),
m_mempool(pool), m_mempool(pool),
m_txdownloadman(node::TxDownloadOptions{pool, m_rng, opts.max_orphan_txs, opts.deterministic_rng}),
m_warnings{warnings}, m_warnings{warnings},
m_opts{opts} m_opts{opts}
{ {
@ -2124,8 +1926,7 @@ void PeerManagerImpl::ActiveTipChange(const CBlockIndex& new_tip, bool is_ibd)
// If the chain tip has changed, previously rejected transactions might now be valid, e.g. due // 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 // to a timelock. Reset the rejection filters to give those transactions another chance if we
// see them again. // see them again.
RecentRejectsFilter().reset(); m_txdownloadman.ActiveTipChange();
RecentRejectsReconsiderableFilter().reset();
} }
} }
@ -2160,30 +1961,13 @@ void PeerManagerImpl::BlockConnected(
return; return;
} }
LOCK(m_tx_download_mutex); LOCK(m_tx_download_mutex);
m_orphanage.EraseForBlock(*pblock); m_txdownloadman.BlockConnected(pblock);
for (const auto& ptx : pblock->vtx) {
RecentConfirmedTransactionsFilter().insert(ptx->GetHash().ToUint256());
if (ptx->HasWitness()) {
RecentConfirmedTransactionsFilter().insert(ptx->GetWitnessHash().ToUint256());
}
m_txrequest.ForgetTxHash(ptx->GetHash());
m_txrequest.ForgetTxHash(ptx->GetWitnessHash());
}
} }
void PeerManagerImpl::BlockDisconnected(const std::shared_ptr<const CBlock> &block, const CBlockIndex* pindex) void PeerManagerImpl::BlockDisconnected(const std::shared_ptr<const CBlock> &block, const CBlockIndex* pindex)
{ {
// To avoid relay problems with transactions that were previously
// confirmed, clear our filter of recently confirmed transactions whenever
// there's a reorg.
// This means that in a 1-block reorg (where 1 block is disconnected and
// then another block reconnected), our filter will drop to having only one
// 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_tx_download_mutex); LOCK(m_tx_download_mutex);
RecentConfirmedTransactionsFilter().reset(); m_txdownloadman.BlockDisconnected();
} }
/** /**
@ -2319,38 +2103,6 @@ void PeerManagerImpl::BlockChecked(const CBlock& block, const BlockValidationSta
// Messages // Messages
// //
bool PeerManagerImpl::AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable)
{
AssertLockHeld(m_tx_download_mutex);
const uint256& hash = gtxid.GetHash();
if (gtxid.IsWtxid()) {
// Normal query by wtxid.
if (m_orphanage.HaveTx(Wtxid::FromUint256(hash))) return true;
} else {
// Never query by txid: it is possible that the transaction in the orphanage has the same
// txid but a different witness, which would give us a false positive result. If we decided
// not to request the transaction based on this result, an attacker could prevent us from
// downloading a transaction by intentionally creating a malleated version of it. While
// only one (or none!) of these transactions can ultimately be confirmed, we have no way of
// discerning which one that is, so the orphanage can store multiple transactions with the
// same txid.
//
// While we won't query by txid, we can try to "guess" what the wtxid is based on the txid.
// A non-segwit transaction's txid == wtxid. Query this txid "casted" to a wtxid. This will
// help us find non-segwit transactions, saving bandwidth, and should have no false positives.
if (m_orphanage.HaveTx(Wtxid::FromUint256(hash))) return true;
}
if (include_reconsiderable && RecentRejectsReconsiderableFilter().contains(hash)) return true;
if (RecentConfirmedTransactionsFilter().contains(hash)) return true;
return RecentRejectsFilter().contains(hash) || m_mempool.exists(gtxid);
}
bool PeerManagerImpl::AlreadyHaveBlock(const uint256& block_hash) bool PeerManagerImpl::AlreadyHaveBlock(const uint256& block_hash)
{ {
return m_chainman.m_blockman.LookupBlockIndex(block_hash) != nullptr; return m_chainman.m_blockman.LookupBlockIndex(block_hash) != nullptr;
@ -3206,70 +2958,33 @@ void PeerManagerImpl::ProcessHeadersMessage(CNode& pfrom, Peer& peer,
return; return;
} }
void PeerManagerImpl::ProcessInvalidTx(NodeId nodeid, const CTransactionRef& ptx, const TxValidationState& state, std::optional<node::PackageToValidate> PeerManagerImpl::ProcessInvalidTx(NodeId nodeid, const CTransactionRef& ptx, const TxValidationState& state,
bool maybe_add_extra_compact_tx) bool first_time_failure)
{ {
AssertLockNotHeld(m_peer_mutex); AssertLockNotHeld(m_peer_mutex);
AssertLockHeld(g_msgproc_mutex); AssertLockHeld(g_msgproc_mutex);
AssertLockHeld(m_tx_download_mutex); AssertLockHeld(m_tx_download_mutex);
PeerRef peer{GetPeerRef(nodeid)};
LogDebug(BCLog::MEMPOOLREJ, "%s (wtxid=%s) from peer=%d was not accepted: %s\n", LogDebug(BCLog::MEMPOOLREJ, "%s (wtxid=%s) from peer=%d was not accepted: %s\n",
ptx->GetHash().ToString(), ptx->GetHash().ToString(),
ptx->GetWitnessHash().ToString(), ptx->GetWitnessHash().ToString(),
nodeid, nodeid,
state.ToString()); state.ToString());
if (state.GetResult() == TxValidationResult::TX_MISSING_INPUTS) { const auto& [add_extra_compact_tx, unique_parents, package_to_validate] = m_txdownloadman.MempoolRejectedTx(ptx, state, nodeid, first_time_failure);
return;
} else if (state.GetResult() != TxValidationResult::TX_WITNESS_STRIPPED) { if (add_extra_compact_tx && RecursiveDynamicUsage(*ptx) < 100000) {
// We can add the wtxid of this transaction to our reject filter.
// Do not add txids of witness transactions or witness-stripped
// transactions to the filter, as they can have been malleated;
// adding such txids to the reject filter would potentially
// interfere with relay of valid transactions from peers that
// do not support wtxid-based relay. See
// https://github.com/bitcoin/bitcoin/issues/8279 for details.
// We can remove this restriction (and always add wtxids to
// the filter even for witness stripped transactions) once
// wtxid-based relay is broadly deployed.
// See also comments in https://github.com/bitcoin/bitcoin/pull/18044#discussion_r443419034
// for concerns around weakening security of unupgraded nodes
// if we start doing this too early.
if (state.GetResult() == TxValidationResult::TX_RECONSIDERABLE) {
// If the result is TX_RECONSIDERABLE, add it to m_lazy_recent_rejects_reconsiderable
// because we should not download or submit this transaction by itself again, but may
// submit it as part of a package later.
RecentRejectsReconsiderableFilter().insert(ptx->GetWitnessHash().ToUint256());
} else {
RecentRejectsFilter().insert(ptx->GetWitnessHash().ToUint256());
}
m_txrequest.ForgetTxHash(ptx->GetWitnessHash());
// If the transaction failed for TX_INPUTS_NOT_STANDARD,
// then we know that the witness was irrelevant to the policy
// failure, since this check depends only on the txid
// (the scriptPubKey being spent is covered by the txid).
// Add the txid to the reject filter to prevent repeated
// processing of this transaction in the event that child
// transactions are later received (resulting in
// parent-fetching by txid via the orphan-handling logic).
// We only add the txid if it differs from the wtxid, to avoid wasting entries in the
// rolling bloom filter.
if (state.GetResult() == TxValidationResult::TX_INPUTS_NOT_STANDARD && ptx->HasWitness()) {
RecentRejectsFilter().insert(ptx->GetHash().ToUint256());
m_txrequest.ForgetTxHash(ptx->GetHash());
}
if (maybe_add_extra_compact_tx && RecursiveDynamicUsage(*ptx) < 100000) {
AddToCompactExtraTransactions(ptx); AddToCompactExtraTransactions(ptx);
} }
for (const uint256& parent_txid : unique_parents) {
if (peer) AddKnownTx(*peer, parent_txid);
} }
MaybePunishNodeForTx(nodeid, state); MaybePunishNodeForTx(nodeid, state);
// If the tx failed in ProcessOrphanTx, it should be removed from the orphanage unless the return package_to_validate;
// tx was still missing inputs. If the tx was not in the orphanage, EraseTx does nothing and returns 0.
if (Assume(state.GetResult() != TxValidationResult::TX_MISSING_INPUTS) && m_orphanage.EraseTx(ptx->GetWitnessHash()) > 0) {
LogDebug(BCLog::TXPACKAGES, " removed orphan tx %s (wtxid=%s)\n", ptx->GetHash().ToString(), ptx->GetWitnessHash().ToString());
}
} }
void PeerManagerImpl::ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, const std::list<CTransactionRef>& replaced_transactions) void PeerManagerImpl::ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, const std::list<CTransactionRef>& replaced_transactions)
@ -3278,14 +2993,7 @@ void PeerManagerImpl::ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, c
AssertLockHeld(g_msgproc_mutex); AssertLockHeld(g_msgproc_mutex);
AssertLockHeld(m_tx_download_mutex); AssertLockHeld(m_tx_download_mutex);
// As this version of the transaction was acceptable, we can forget about any requests for it. m_txdownloadman.MempoolAcceptedTx(tx);
// No-op if the tx is not in txrequest.
m_txrequest.ForgetTxHash(tx->GetHash());
m_txrequest.ForgetTxHash(tx->GetWitnessHash());
m_orphanage.AddChildrenToWorkSet(*tx);
// If it came from the orphanage, remove it. No-op if the tx is not in txorphanage.
m_orphanage.EraseTx(tx->GetWitnessHash());
LogDebug(BCLog::MEMPOOL, "AcceptToMemoryPool: peer=%d: accepted %s (wtxid=%s) (poolsz %u txn, %u kB)\n", LogDebug(BCLog::MEMPOOL, "AcceptToMemoryPool: peer=%d: accepted %s (wtxid=%s) (poolsz %u txn, %u kB)\n",
nodeid, nodeid,
@ -3300,7 +3008,7 @@ void PeerManagerImpl::ProcessValidTx(NodeId nodeid, const CTransactionRef& tx, c
} }
} }
void PeerManagerImpl::ProcessPackageResult(const PackageToValidate& package_to_validate, const PackageMempoolAcceptResult& package_result) void PeerManagerImpl::ProcessPackageResult(const node::PackageToValidate& package_to_validate, const PackageMempoolAcceptResult& package_result)
{ {
AssertLockNotHeld(m_peer_mutex); AssertLockNotHeld(m_peer_mutex);
AssertLockHeld(g_msgproc_mutex); AssertLockHeld(g_msgproc_mutex);
@ -3310,7 +3018,7 @@ void PeerManagerImpl::ProcessPackageResult(const PackageToValidate& package_to_v
const auto& senders = package_to_validate.m_senders; const auto& senders = package_to_validate.m_senders;
if (package_result.m_state.IsInvalid()) { if (package_result.m_state.IsInvalid()) {
RecentRejectsReconsiderableFilter().insert(GetPackageHash(package)); m_txdownloadman.MempoolRejectedPackage(package);
} }
// We currently only expect to process 1-parent-1-child packages. Remove if this changes. // We currently only expect to process 1-parent-1-child packages. Remove if this changes.
if (!Assume(package.size() == 2)) return; if (!Assume(package.size() == 2)) return;
@ -3340,7 +3048,7 @@ void PeerManagerImpl::ProcessPackageResult(const PackageToValidate& package_to_v
// added there when added to the orphanage or rejected for TX_RECONSIDERABLE. // added there when added to the orphanage or rejected for TX_RECONSIDERABLE.
// This should be updated if package submission is ever used for transactions // This should be updated if package submission is ever used for transactions
// that haven't already been validated before. // that haven't already been validated before.
ProcessInvalidTx(nodeid, tx, tx_result.m_state, /*maybe_add_extra_compact_tx=*/false); ProcessInvalidTx(nodeid, tx, tx_result.m_state, /*first_time_failure=*/false);
break; break;
} }
case MempoolAcceptResult::ResultType::MEMPOOL_ENTRY: case MempoolAcceptResult::ResultType::MEMPOOL_ENTRY:
@ -3356,60 +3064,6 @@ void PeerManagerImpl::ProcessPackageResult(const PackageToValidate& package_to_v
} }
} }
std::optional<PeerManagerImpl::PackageToValidate> PeerManagerImpl::Find1P1CPackage(const CTransactionRef& ptx, NodeId nodeid)
{
AssertLockNotHeld(m_peer_mutex);
AssertLockHeld(g_msgproc_mutex);
AssertLockHeld(m_tx_download_mutex);
const auto& parent_wtxid{ptx->GetWitnessHash()};
Assume(RecentRejectsReconsiderableFilter().contains(parent_wtxid.ToUint256()));
// Prefer children from this peer. This helps prevent censorship attempts in which an attacker
// sends lots of fake children for the parent, and we (unluckily) keep selecting the fake
// children instead of the real one provided by the honest peer.
const auto cpfp_candidates_same_peer{m_orphanage.GetChildrenFromSamePeer(ptx, nodeid)};
// These children should be sorted from newest to oldest. In the (probably uncommon) case
// of children that replace each other, this helps us accept the highest feerate (probably the
// most recent) one efficiently.
for (const auto& child : cpfp_candidates_same_peer) {
Package maybe_cpfp_package{ptx, child};
if (!RecentRejectsReconsiderableFilter().contains(GetPackageHash(maybe_cpfp_package))) {
return PeerManagerImpl::PackageToValidate{ptx, child, nodeid, nodeid};
}
}
// If no suitable candidate from the same peer is found, also try children that were provided by
// a different peer. This is useful because sometimes multiple peers announce both transactions
// to us, and we happen to download them from different peers (we wouldn't have known that these
// 2 transactions are related). We still want to find 1p1c packages then.
//
// If we start tracking all announcers of orphans, we can restrict this logic to parent + child
// pairs in which both were provided by the same peer, i.e. delete this step.
const auto cpfp_candidates_different_peer{m_orphanage.GetChildrenFromDifferentPeer(ptx, nodeid)};
// Find the first 1p1c that hasn't already been rejected. We randomize the order to not
// create a bias that attackers can use to delay package acceptance.
//
// Create a random permutation of the indices.
std::vector<size_t> tx_indices(cpfp_candidates_different_peer.size());
std::iota(tx_indices.begin(), tx_indices.end(), 0);
std::shuffle(tx_indices.begin(), tx_indices.end(), m_rng);
for (const auto index : tx_indices) {
// If we already tried a package and failed for any reason, the combined hash was
// cached in m_lazy_recent_rejects_reconsiderable.
const auto [child_tx, child_sender] = cpfp_candidates_different_peer.at(index);
Package maybe_cpfp_package{ptx, child_tx};
if (!RecentRejectsReconsiderableFilter().contains(GetPackageHash(maybe_cpfp_package))) {
return PeerManagerImpl::PackageToValidate{ptx, child_tx, nodeid, child_sender};
}
}
return std::nullopt;
}
bool PeerManagerImpl::ProcessOrphanTx(Peer& peer) bool PeerManagerImpl::ProcessOrphanTx(Peer& peer)
{ {
AssertLockHeld(g_msgproc_mutex); AssertLockHeld(g_msgproc_mutex);
@ -3417,7 +3071,7 @@ bool PeerManagerImpl::ProcessOrphanTx(Peer& peer)
CTransactionRef porphanTx = nullptr; CTransactionRef porphanTx = nullptr;
while (CTransactionRef porphanTx = m_orphanage.GetTxToReconsider(peer.m_id)) { while (CTransactionRef porphanTx = m_txdownloadman.GetTxToReconsider(peer.m_id)) {
const MempoolAcceptResult result = m_chainman.ProcessTransaction(porphanTx); const MempoolAcceptResult result = m_chainman.ProcessTransaction(porphanTx);
const TxValidationState& state = result.m_state; const TxValidationState& state = result.m_state;
const Txid& orphanHash = porphanTx->GetHash(); const Txid& orphanHash = porphanTx->GetHash();
@ -3438,7 +3092,7 @@ bool PeerManagerImpl::ProcessOrphanTx(Peer& peer)
state.GetResult() != TxValidationResult::TX_UNKNOWN && state.GetResult() != TxValidationResult::TX_UNKNOWN &&
state.GetResult() != TxValidationResult::TX_NO_MEMPOOL && state.GetResult() != TxValidationResult::TX_NO_MEMPOOL &&
state.GetResult() != TxValidationResult::TX_RESULT_UNSET)) { state.GetResult() != TxValidationResult::TX_RESULT_UNSET)) {
ProcessInvalidTx(peer.m_id, porphanTx, state, /*maybe_add_extra_compact_tx=*/false); ProcessInvalidTx(peer.m_id, porphanTx, state, /*first_time_failure=*/false);
} }
return true; return true;
} }
@ -3999,6 +3653,16 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
tx_relay->m_next_inv_send_time == 0s)); tx_relay->m_next_inv_send_time == 0s));
} }
{
LOCK2(::cs_main, m_tx_download_mutex);
const CNodeState* state = State(pfrom.GetId());
m_txdownloadman.ConnectedPeer(pfrom.GetId(), node::TxDownloadConnectionInfo {
.m_preferred = state->fPreferredDownload,
.m_relay_permissions = pfrom.HasPermission(NetPermissionFlags::Relay),
.m_wtxid_relay = peer->m_wtxid_relay,
});
}
pfrom.fSuccessfullyConnected = true; pfrom.fSuccessfullyConnected = true;
return; return;
} }
@ -4268,12 +3932,11 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
return; return;
} }
const GenTxid gtxid = ToGenTxid(inv); const GenTxid gtxid = ToGenTxid(inv);
const bool fAlreadyHave = AlreadyHaveTx(gtxid, /*include_reconsiderable=*/true);
LogDebug(BCLog::NET, "got inv: %s %s peer=%d\n", inv.ToString(), fAlreadyHave ? "have" : "new", pfrom.GetId());
AddKnownTx(*peer, inv.hash); AddKnownTx(*peer, inv.hash);
if (!fAlreadyHave && !m_chainman.IsInitialBlockDownload()) {
AddTxAnnouncement(pfrom, gtxid, current_time); if (!m_chainman.IsInitialBlockDownload()) {
const bool fAlreadyHave{m_txdownloadman.AddTxAnnouncement(pfrom.GetId(), gtxid, current_time, /*p2p_inv=*/true)};
LogDebug(BCLog::NET, "got inv: %s %s peer=%d\n", inv.ToString(), fAlreadyHave ? "have" : "new", pfrom.GetId());
} }
} else { } else {
LogDebug(BCLog::NET, "Unknown inv type \"%s\" received from peer=%d\n", inv.ToString(), pfrom.GetId()); LogDebug(BCLog::NET, "Unknown inv type \"%s\" received from peer=%d\n", inv.ToString(), pfrom.GetId());
@ -4565,22 +4228,8 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
LOCK2(cs_main, m_tx_download_mutex); LOCK2(cs_main, m_tx_download_mutex);
m_txrequest.ReceivedResponse(pfrom.GetId(), txid); const auto& [should_validate, package_to_validate] = m_txdownloadman.ReceivedTx(pfrom.GetId(), ptx);
if (tx.HasWitness()) m_txrequest.ReceivedResponse(pfrom.GetId(), wtxid); if (!should_validate) {
// We do the AlreadyHaveTx() check using wtxid, rather than txid - in the
// absence of witness malleation, this is strictly better, because the
// recent rejects filter may contain the wtxid but rarely contains
// the txid of a segwit transaction that has been rejected.
// In the presence of witness malleation, it's possible that by only
// doing the check with wtxid, we could overlook a transaction which
// was confirmed with a different witness, or exists in our mempool
// with a different witness, but this has limited downside:
// mempool validation does its own lookup of whether we have the txid
// already; and an adversary can already relay us old transactions
// (older than our recency filter) if trying to DoS us, without any need
// for witness malleation.
if (AlreadyHaveTx(GenTxid::Wtxid(wtxid), /*include_reconsiderable=*/true)) {
if (pfrom.HasPermission(NetPermissionFlags::ForceRelay)) { if (pfrom.HasPermission(NetPermissionFlags::ForceRelay)) {
// Always relay transactions received from peers with forcerelay // Always relay transactions received from peers with forcerelay
// permission, even if they were already in the mempool, allowing // permission, even if they were already in the mempool, allowing
@ -4595,37 +4244,18 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
} }
} }
if (RecentRejectsReconsiderableFilter().contains(wtxid)) { if (package_to_validate) {
// When a transaction is already in m_lazy_recent_rejects_reconsiderable, we shouldn't submit
// it by itself again. However, look for a matching child in the orphanage, as it is
// possible that they succeed as a package.
LogDebug(BCLog::TXPACKAGES, "found tx %s (wtxid=%s) in reconsiderable rejects, looking for child in orphanage\n",
txid.ToString(), wtxid.ToString());
if (auto package_to_validate{Find1P1CPackage(ptx, pfrom.GetId())}) {
const auto package_result{ProcessNewPackage(m_chainman.ActiveChainstate(), m_mempool, package_to_validate->m_txns, /*test_accept=*/false, /*client_maxfeerate=*/std::nullopt)}; const auto package_result{ProcessNewPackage(m_chainman.ActiveChainstate(), m_mempool, package_to_validate->m_txns, /*test_accept=*/false, /*client_maxfeerate=*/std::nullopt)};
LogDebug(BCLog::TXPACKAGES, "package evaluation for %s: %s\n", package_to_validate->ToString(), LogDebug(BCLog::TXPACKAGES, "package evaluation for %s: %s\n", package_to_validate->ToString(),
package_result.m_state.IsValid() ? "package accepted" : "package rejected"); package_result.m_state.IsValid() ? "package accepted" : "package rejected");
ProcessPackageResult(package_to_validate.value(), package_result); ProcessPackageResult(package_to_validate.value(), package_result);
} }
}
// If a tx is detected by m_lazy_recent_rejects it is ignored. Because we haven't
// submitted the tx to our mempool, we won't have computed a DoS
// score for it or determined exactly why we consider it invalid.
//
// This means we won't penalize any peer subsequently relaying a DoSy
// tx (even if we penalized the first peer who gave it to us) because
// we have to account for m_lazy_recent_rejects showing false positives. In
// other words, we shouldn't penalize a peer if we aren't *sure* they
// submitted a DoSy tx.
//
// Note that m_lazy_recent_rejects doesn't just record DoSy or invalid
// transactions, but any tx not accepted by the mempool, which may be
// due to node policy (vs. consensus). So we can't blanket penalize a
// peer simply for relaying a tx that our m_lazy_recent_rejects has caught,
// regardless of false positives.
return; return;
} }
// ReceivedTx should not be telling us to validate the tx and a package.
Assume(!package_to_validate.has_value());
const MempoolAcceptResult result = m_chainman.ProcessTransaction(ptx); const MempoolAcceptResult result = m_chainman.ProcessTransaction(ptx);
const TxValidationState& state = result.m_state; const TxValidationState& state = result.m_state;
@ -4633,90 +4263,8 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
ProcessValidTx(pfrom.GetId(), ptx, result.m_replaced_transactions); ProcessValidTx(pfrom.GetId(), ptx, result.m_replaced_transactions);
pfrom.m_last_tx_time = GetTime<std::chrono::seconds>(); pfrom.m_last_tx_time = GetTime<std::chrono::seconds>();
} }
else if (state.GetResult() == TxValidationResult::TX_MISSING_INPUTS)
{
bool fRejectedParents = false; // It may be the case that the orphans parents have all been rejected
// Deduplicate parent txids, so that we don't have to loop over
// the same parent txid more than once down below.
std::vector<uint256> unique_parents;
unique_parents.reserve(tx.vin.size());
for (const CTxIn& txin : tx.vin) {
// We start with all parents, and then remove duplicates below.
unique_parents.push_back(txin.prevout.hash);
}
std::sort(unique_parents.begin(), unique_parents.end());
unique_parents.erase(std::unique(unique_parents.begin(), unique_parents.end()), unique_parents.end());
// Distinguish between parents in m_lazy_recent_rejects and m_lazy_recent_rejects_reconsiderable.
// We can tolerate having up to 1 parent in m_lazy_recent_rejects_reconsiderable since we
// submit 1p1c packages. However, fail immediately if any are in m_lazy_recent_rejects.
std::optional<uint256> rejected_parent_reconsiderable;
for (const uint256& parent_txid : unique_parents) {
if (RecentRejectsFilter().contains(parent_txid)) {
fRejectedParents = true;
break;
} else if (RecentRejectsReconsiderableFilter().contains(parent_txid) && !m_mempool.exists(GenTxid::Txid(parent_txid))) {
// More than 1 parent in m_lazy_recent_rejects_reconsiderable: 1p1c will not be
// sufficient to accept this package, so just give up here.
if (rejected_parent_reconsiderable.has_value()) {
fRejectedParents = true;
break;
}
rejected_parent_reconsiderable = parent_txid;
}
}
if (!fRejectedParents) {
const auto current_time{GetTime<std::chrono::microseconds>()};
for (const uint256& parent_txid : unique_parents) {
// Here, we only have the txid (and not wtxid) of the
// inputs, so we only request in txid mode, even for
// wtxidrelay peers.
// Eventually we should replace this with an improved
// protocol for getting all unconfirmed parents.
const auto gtxid{GenTxid::Txid(parent_txid)};
AddKnownTx(*peer, parent_txid);
// Exclude m_lazy_recent_rejects_reconsiderable: the missing parent may have been
// previously rejected for being too low feerate. This orphan might CPFP it.
if (!AlreadyHaveTx(gtxid, /*include_reconsiderable=*/false)) AddTxAnnouncement(pfrom, gtxid, current_time);
}
if (m_orphanage.AddTx(ptx, pfrom.GetId())) {
AddToCompactExtraTransactions(ptx);
}
// Once added to the orphan pool, a tx is considered AlreadyHave, and we shouldn't request it anymore.
m_txrequest.ForgetTxHash(tx.GetHash());
m_txrequest.ForgetTxHash(tx.GetWitnessHash());
// DoS prevention: do not allow m_orphanage to grow unbounded (see CVE-2012-3789)
m_orphanage.LimitOrphans(m_opts.max_orphan_txs, m_rng);
} else {
LogDebug(BCLog::MEMPOOL, "not keeping orphan with rejected parents %s (wtxid=%s)\n",
tx.GetHash().ToString(),
tx.GetWitnessHash().ToString());
// We will continue to reject this tx since it has rejected
// parents so avoid re-requesting it from other peers.
// Here we add both the txid and the wtxid, as we know that
// regardless of what witness is provided, we will not accept
// this, so we don't need to allow for redownload of this txid
// from any of our non-wtxidrelay peers.
RecentRejectsFilter().insert(tx.GetHash().ToUint256());
RecentRejectsFilter().insert(tx.GetWitnessHash().ToUint256());
m_txrequest.ForgetTxHash(tx.GetHash());
m_txrequest.ForgetTxHash(tx.GetWitnessHash());
}
}
if (state.IsInvalid()) { if (state.IsInvalid()) {
ProcessInvalidTx(pfrom.GetId(), ptx, state, /*maybe_add_extra_compact_tx=*/true); if (auto package_to_validate{ProcessInvalidTx(pfrom.GetId(), ptx, state, /*first_time_failure=*/true)}) {
}
// When a transaction fails for TX_RECONSIDERABLE, look for a matching child in the
// orphanage, as it is possible that they succeed as a package.
if (state.GetResult() == TxValidationResult::TX_RECONSIDERABLE) {
LogDebug(BCLog::TXPACKAGES, "tx %s (wtxid=%s) failed but reconsiderable, looking for child in orphanage\n",
txid.ToString(), wtxid.ToString());
if (auto package_to_validate{Find1P1CPackage(ptx, pfrom.GetId())}) {
const auto package_result{ProcessNewPackage(m_chainman.ActiveChainstate(), m_mempool, package_to_validate->m_txns, /*test_accept=*/false, /*client_maxfeerate=*/std::nullopt)}; const auto package_result{ProcessNewPackage(m_chainman.ActiveChainstate(), m_mempool, package_to_validate->m_txns, /*test_accept=*/false, /*client_maxfeerate=*/std::nullopt)};
LogDebug(BCLog::TXPACKAGES, "package evaluation for %s: %s\n", package_to_validate->ToString(), LogDebug(BCLog::TXPACKAGES, "package evaluation for %s: %s\n", package_to_validate->ToString(),
package_result.m_state.IsValid() ? "package accepted" : "package rejected"); package_result.m_state.IsValid() ? "package accepted" : "package rejected");
@ -5319,16 +4867,16 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
if (msg_type == NetMsgType::NOTFOUND) { if (msg_type == NetMsgType::NOTFOUND) {
std::vector<CInv> vInv; std::vector<CInv> vInv;
vRecv >> vInv; vRecv >> vInv;
if (vInv.size() <= MAX_PEER_TX_ANNOUNCEMENTS + MAX_BLOCKS_IN_TRANSIT_PER_PEER) { std::vector<uint256> tx_invs;
LOCK(m_tx_download_mutex); if (vInv.size() <= node::MAX_PEER_TX_ANNOUNCEMENTS + MAX_BLOCKS_IN_TRANSIT_PER_PEER) {
for (CInv &inv : vInv) { for (CInv &inv : vInv) {
if (inv.IsGenTxMsg()) { if (inv.IsGenTxMsg()) {
// If we receive a NOTFOUND message for a tx we requested, mark the announcement for it as tx_invs.emplace_back(inv.hash);
// completed in TxRequestTracker.
m_txrequest.ReceivedResponse(pfrom.GetId(), inv.hash);
} }
} }
} }
LOCK(m_tx_download_mutex);
m_txdownloadman.ReceivedNotFound(pfrom.GetId(), tx_invs);
return; return;
} }
@ -5447,7 +4995,7 @@ bool PeerManagerImpl::ProcessMessages(CNode* pfrom, std::atomic<bool>& interrupt
// the extra work may not be noticed, possibly resulting in an // the extra work may not be noticed, possibly resulting in an
// unnecessary 100ms delay) // unnecessary 100ms delay)
LOCK(m_tx_download_mutex); LOCK(m_tx_download_mutex);
if (m_orphanage.HaveTxToReconsider(peer->m_id)) fMoreWork = true; if (m_txdownloadman.HaveMoreWork(peer->m_id)) fMoreWork = true;
} catch (const std::exception& e) { } catch (const std::exception& e) {
LogDebug(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()); LogDebug(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());
} catch (...) { } catch (...) {
@ -6343,31 +5891,14 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
// //
{ {
LOCK(m_tx_download_mutex); LOCK(m_tx_download_mutex);
std::vector<std::pair<NodeId, GenTxid>> expired; for (const GenTxid& gtxid : m_txdownloadman.GetRequestsToSend(pto->GetId(), current_time)) {
auto requestable = m_txrequest.GetRequestable(pto->GetId(), current_time, &expired);
for (const auto& entry : expired) {
LogDebug(BCLog::NET, "timeout of inflight %s %s from peer=%d\n", entry.second.IsWtxid() ? "wtx" : "tx",
entry.second.GetHash().ToString(), entry.first);
}
for (const GenTxid& gtxid : requestable) {
// Exclude m_lazy_recent_rejects_reconsiderable: we may be requesting a missing parent
// that was previously rejected for being too low feerate.
if (!AlreadyHaveTx(gtxid, /*include_reconsiderable=*/false)) {
LogDebug(BCLog::NET, "Requesting %s %s peer=%d\n", gtxid.IsWtxid() ? "wtx" : "tx",
gtxid.GetHash().ToString(), pto->GetId());
vGetData.emplace_back(gtxid.IsWtxid() ? MSG_WTX : (MSG_TX | GetFetchFlags(*peer)), gtxid.GetHash()); vGetData.emplace_back(gtxid.IsWtxid() ? MSG_WTX : (MSG_TX | GetFetchFlags(*peer)), gtxid.GetHash());
if (vGetData.size() >= MAX_GETDATA_SZ) { if (vGetData.size() >= MAX_GETDATA_SZ) {
MakeAndPushMessage(*pto, NetMsgType::GETDATA, vGetData); MakeAndPushMessage(*pto, NetMsgType::GETDATA, vGetData);
vGetData.clear(); vGetData.clear();
} }
m_txrequest.RequestedTx(pto->GetId(), gtxid.GetHash(), current_time + GETDATA_TX_INTERVAL);
} else {
// We have already seen this transaction, no need to download. This is just a belt-and-suspenders, as
// this should already be called whenever a transaction becomes AlreadyHaveTx().
m_txrequest.ForgetTxHash(gtxid.GetHash());
} }
} }
} // release m_tx_download_mutex
if (!vGetData.empty()) if (!vGetData.empty())
MakeAndPushMessage(*pto, NetMsgType::GETDATA, vGetData); MakeAndPushMessage(*pto, NetMsgType::GETDATA, vGetData);

178
src/node/txdownloadman.h Normal file
View file

@ -0,0 +1,178 @@
// Copyright (c) 2024 The Bitcoin Core developers
// Distributed under the MIT software license, see the accompanying
// file COPYING or http://www.opensource.org/licenses/mit-license.php.
#ifndef BITCOIN_NODE_TXDOWNLOADMAN_H
#define BITCOIN_NODE_TXDOWNLOADMAN_H
#include <net.h>
#include <policy/packages.h>
#include <txorphanage.h>
#include <cstdint>
#include <memory>
class CBlock;
class CRollingBloomFilter;
class CTxMemPool;
class GenTxid;
class TxRequestTracker;
namespace node {
class TxDownloadManagerImpl;
/** Maximum number of in-flight transaction requests from a peer. It is not a hard limit, but the threshold at which
* point the OVERLOADED_PEER_TX_DELAY kicks in. */
static constexpr int32_t MAX_PEER_TX_REQUEST_IN_FLIGHT = 100;
/** Maximum number of transactions to consider for requesting, per peer. It provides a reasonable DoS limit to
* per-peer memory usage spent on announcements, while covering peers continuously sending INVs at the maximum
* rate (by our own policy, see INVENTORY_BROADCAST_PER_SECOND) for several minutes, while not receiving
* the actual transaction (from any peer) in response to requests for them. */
static constexpr int32_t MAX_PEER_TX_ANNOUNCEMENTS = 5000;
/** How long to delay requesting transactions via txids, if we have wtxid-relaying peers */
static constexpr auto TXID_RELAY_DELAY{2s};
/** How long to delay requesting transactions from non-preferred peers */
static constexpr auto NONPREF_PEER_TX_DELAY{2s};
/** How long to delay requesting transactions from overloaded peers (see MAX_PEER_TX_REQUEST_IN_FLIGHT). */
static constexpr auto OVERLOADED_PEER_TX_DELAY{2s};
/** How long to wait before downloading a transaction from an additional peer */
static constexpr auto GETDATA_TX_INTERVAL{60s};
struct TxDownloadOptions {
/** Read-only reference to mempool. */
const CTxMemPool& m_mempool;
/** RNG provided by caller. */
FastRandomContext& m_rng;
/** Maximum number of transactions allowed in orphanage. */
const uint32_t m_max_orphan_txs;
/** Instantiate TxRequestTracker as deterministic (used for tests). */
bool m_deterministic_txrequest{false};
};
struct TxDownloadConnectionInfo {
/** Whether this peer is preferred for transaction download. */
const bool m_preferred;
/** Whether this peer has Relay permissions. */
const bool m_relay_permissions;
/** Whether this peer supports wtxid relay. */
const bool m_wtxid_relay;
};
struct PackageToValidate {
Package m_txns;
std::vector<NodeId> m_senders;
/** Construct a 1-parent-1-child package. */
explicit PackageToValidate(const CTransactionRef& parent,
const CTransactionRef& child,
NodeId parent_sender,
NodeId child_sender) :
m_txns{parent, child},
m_senders{parent_sender, child_sender}
{}
// Move ctor
PackageToValidate(PackageToValidate&& other) : m_txns{std::move(other.m_txns)}, m_senders{std::move(other.m_senders)} {}
// Copy ctor
PackageToValidate(const PackageToValidate& other) = default;
// Move assignment
PackageToValidate& operator=(PackageToValidate&& other) {
this->m_txns = std::move(other.m_txns);
this->m_senders = std::move(other.m_senders);
return *this;
}
std::string ToString() const {
Assume(m_txns.size() == 2);
return strprintf("parent %s (wtxid=%s, sender=%d) + child %s (wtxid=%s, sender=%d)",
m_txns.front()->GetHash().ToString(),
m_txns.front()->GetWitnessHash().ToString(),
m_senders.front(),
m_txns.back()->GetHash().ToString(),
m_txns.back()->GetWitnessHash().ToString(),
m_senders.back());
}
};
struct RejectedTxTodo
{
bool m_should_add_extra_compact_tx;
std::vector<uint256> m_unique_parents;
std::optional<PackageToValidate> m_package_to_validate;
};
/**
* Class responsible for deciding what transactions to request and, once
* downloaded, whether and how to validate them. It is also responsible for
* deciding what transaction packages to validate and how to resolve orphan
* transactions. Its data structures include TxRequestTracker for scheduling
* requests, rolling bloom filters for remembering transactions that have
* already been {accepted, rejected, confirmed}, an orphanage, and a registry of
* each peer's transaction relay-related information.
*
* Caller needs to interact with TxDownloadManager:
* - ValidationInterface callbacks.
* - When a potential transaction relay peer connects or disconnects.
* - When a transaction or package is accepted or rejected from mempool
* - When a inv, notfound, or tx message is received
* - To get instructions for which getdata messages to send
*
* This class is not thread-safe. Access must be synchronized using an
* external mutex.
*/
class TxDownloadManager {
const std::unique_ptr<TxDownloadManagerImpl> m_impl;
public:
explicit TxDownloadManager(const TxDownloadOptions& options);
~TxDownloadManager();
// Responses to chain events. TxDownloadManager is not an actual client of ValidationInterface, these are called through PeerManager.
void ActiveTipChange();
void BlockConnected(const std::shared_ptr<const CBlock>& pblock);
void BlockDisconnected();
/** Creates a new PeerInfo. Saves the connection info to calculate tx announcement delays later. */
void ConnectedPeer(NodeId nodeid, const TxDownloadConnectionInfo& info);
/** Deletes all txrequest announcements and orphans for a given peer. */
void DisconnectedPeer(NodeId nodeid);
/** New inv has been received. May be added as a candidate to txrequest.
* @param[in] p2p_inv When true, only add this announcement if we don't already have the tx.
* Returns true if this was a dropped inv (p2p_inv=true and we already have the tx), false otherwise. */
bool AddTxAnnouncement(NodeId peer, const GenTxid& gtxid, std::chrono::microseconds now, bool p2p_inv);
/** Get getdata requests to send. */
std::vector<GenTxid> GetRequestsToSend(NodeId nodeid, std::chrono::microseconds current_time);
/** Should be called when a notfound for a tx has been received. */
void ReceivedNotFound(NodeId nodeid, const std::vector<uint256>& txhashes);
/** Respond to successful transaction submission to mempool */
void MempoolAcceptedTx(const CTransactionRef& tx);
/** Respond to transaction rejected from mempool */
RejectedTxTodo MempoolRejectedTx(const CTransactionRef& ptx, const TxValidationState& state, NodeId nodeid, bool first_time_failure);
/** Respond to package rejected from mempool */
void MempoolRejectedPackage(const Package& package);
/** Marks a tx as ReceivedResponse in txrequest and checks whether AlreadyHaveTx.
* Return a bool indicating whether this tx should be validated. If false, optionally, a
* PackageToValidate. */
std::pair<bool, std::optional<PackageToValidate>> ReceivedTx(NodeId nodeid, const CTransactionRef& ptx);
/** Whether there are any orphans to reconsider for this peer. */
bool HaveMoreWork(NodeId nodeid) const;
/** Returns next orphan tx to consider, or nullptr if none exist. */
CTransactionRef GetTxToReconsider(NodeId nodeid);
/** Check that all data structures are empty. */
void CheckIsEmpty() const;
/** Check that all data structures that track per-peer information have nothing for this peer. */
void CheckIsEmpty(NodeId nodeid) const;
/** Wrapper for TxOrphanage::GetOrphanTransactions */
std::vector<TxOrphanage::OrphanTxBase> GetOrphanTransactions() const;
};
} // namespace node
#endif // BITCOIN_NODE_TXDOWNLOADMAN_H

View file

@ -0,0 +1,536 @@
// Copyright (c) 2024
// Distributed under the MIT software license, see the accompanying
// file COPYING or http://www.opensource.org/licenses/mit-license.php.
#include <node/txdownloadman_impl.h>
#include <node/txdownloadman.h>
#include <chain.h>
#include <consensus/validation.h>
#include <logging.h>
#include <txmempool.h>
#include <validation.h>
#include <validationinterface.h>
namespace node {
// TxDownloadManager wrappers
TxDownloadManager::TxDownloadManager(const TxDownloadOptions& options) :
m_impl{std::make_unique<TxDownloadManagerImpl>(options)}
{}
TxDownloadManager::~TxDownloadManager() = default;
void TxDownloadManager::ActiveTipChange()
{
m_impl->ActiveTipChange();
}
void TxDownloadManager::BlockConnected(const std::shared_ptr<const CBlock>& pblock)
{
m_impl->BlockConnected(pblock);
}
void TxDownloadManager::BlockDisconnected()
{
m_impl->BlockDisconnected();
}
void TxDownloadManager::ConnectedPeer(NodeId nodeid, const TxDownloadConnectionInfo& info)
{
m_impl->ConnectedPeer(nodeid, info);
}
void TxDownloadManager::DisconnectedPeer(NodeId nodeid)
{
m_impl->DisconnectedPeer(nodeid);
}
bool TxDownloadManager::AddTxAnnouncement(NodeId peer, const GenTxid& gtxid, std::chrono::microseconds now, bool p2p_inv)
{
return m_impl->AddTxAnnouncement(peer, gtxid, now, p2p_inv);
}
std::vector<GenTxid> TxDownloadManager::GetRequestsToSend(NodeId nodeid, std::chrono::microseconds current_time)
{
return m_impl->GetRequestsToSend(nodeid, current_time);
}
void TxDownloadManager::ReceivedNotFound(NodeId nodeid, const std::vector<uint256>& txhashes)
{
m_impl->ReceivedNotFound(nodeid, txhashes);
}
void TxDownloadManager::MempoolAcceptedTx(const CTransactionRef& tx)
{
m_impl->MempoolAcceptedTx(tx);
}
RejectedTxTodo TxDownloadManager::MempoolRejectedTx(const CTransactionRef& ptx, const TxValidationState& state, NodeId nodeid, bool first_time_failure)
{
return m_impl->MempoolRejectedTx(ptx, state, nodeid, first_time_failure);
}
void TxDownloadManager::MempoolRejectedPackage(const Package& package)
{
m_impl->MempoolRejectedPackage(package);
}
std::pair<bool, std::optional<PackageToValidate>> TxDownloadManager::ReceivedTx(NodeId nodeid, const CTransactionRef& ptx)
{
return m_impl->ReceivedTx(nodeid, ptx);
}
bool TxDownloadManager::HaveMoreWork(NodeId nodeid) const
{
return m_impl->HaveMoreWork(nodeid);
}
CTransactionRef TxDownloadManager::GetTxToReconsider(NodeId nodeid)
{
return m_impl->GetTxToReconsider(nodeid);
}
void TxDownloadManager::CheckIsEmpty() const
{
m_impl->CheckIsEmpty();
}
void TxDownloadManager::CheckIsEmpty(NodeId nodeid) const
{
m_impl->CheckIsEmpty(nodeid);
}
std::vector<TxOrphanage::OrphanTxBase> TxDownloadManager::GetOrphanTransactions() const
{
return m_impl->GetOrphanTransactions();
}
// TxDownloadManagerImpl
void TxDownloadManagerImpl::ActiveTipChange()
{
RecentRejectsFilter().reset();
RecentRejectsReconsiderableFilter().reset();
}
void TxDownloadManagerImpl::BlockConnected(const std::shared_ptr<const CBlock>& pblock)
{
m_orphanage.EraseForBlock(*pblock);
for (const auto& ptx : pblock->vtx) {
RecentConfirmedTransactionsFilter().insert(ptx->GetHash().ToUint256());
if (ptx->HasWitness()) {
RecentConfirmedTransactionsFilter().insert(ptx->GetWitnessHash().ToUint256());
}
m_txrequest.ForgetTxHash(ptx->GetHash());
m_txrequest.ForgetTxHash(ptx->GetWitnessHash());
}
}
void TxDownloadManagerImpl::BlockDisconnected()
{
// To avoid relay problems with transactions that were previously
// confirmed, clear our filter of recently confirmed transactions whenever
// there's a reorg.
// This means that in a 1-block reorg (where 1 block is disconnected and
// then another block reconnected), our filter will drop to having only one
// 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.
RecentConfirmedTransactionsFilter().reset();
}
bool TxDownloadManagerImpl::AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable)
{
const uint256& hash = gtxid.GetHash();
if (gtxid.IsWtxid()) {
// Normal query by wtxid.
if (m_orphanage.HaveTx(Wtxid::FromUint256(hash))) return true;
} else {
// Never query by txid: it is possible that the transaction in the orphanage has the same
// txid but a different witness, which would give us a false positive result. If we decided
// not to request the transaction based on this result, an attacker could prevent us from
// downloading a transaction by intentionally creating a malleated version of it. While
// only one (or none!) of these transactions can ultimately be confirmed, we have no way of
// discerning which one that is, so the orphanage can store multiple transactions with the
// same txid.
//
// While we won't query by txid, we can try to "guess" what the wtxid is based on the txid.
// A non-segwit transaction's txid == wtxid. Query this txid "casted" to a wtxid. This will
// help us find non-segwit transactions, saving bandwidth, and should have no false positives.
if (m_orphanage.HaveTx(Wtxid::FromUint256(hash))) return true;
}
if (include_reconsiderable && RecentRejectsReconsiderableFilter().contains(hash)) return true;
if (RecentConfirmedTransactionsFilter().contains(hash)) return true;
return RecentRejectsFilter().contains(hash) || m_opts.m_mempool.exists(gtxid);
}
void TxDownloadManagerImpl::ConnectedPeer(NodeId nodeid, const TxDownloadConnectionInfo& info)
{
// If already connected (shouldn't happen in practice), exit early.
if (m_peer_info.contains(nodeid)) return;
m_peer_info.try_emplace(nodeid, info);
if (info.m_wtxid_relay) m_num_wtxid_peers += 1;
}
void TxDownloadManagerImpl::DisconnectedPeer(NodeId nodeid)
{
m_orphanage.EraseForPeer(nodeid);
m_txrequest.DisconnectedPeer(nodeid);
if (auto it = m_peer_info.find(nodeid); it != m_peer_info.end()) {
if (it->second.m_connection_info.m_wtxid_relay) m_num_wtxid_peers -= 1;
m_peer_info.erase(it);
}
}
bool TxDownloadManagerImpl::AddTxAnnouncement(NodeId peer, const GenTxid& gtxid, std::chrono::microseconds now, bool p2p_inv)
{
// If this is an inv received from a peer and we already have it, we can drop it.
// If this is a request for the parent of an orphan, we don't drop transactions that we already have. In particular,
// we *do* want to request parents that are in m_lazy_recent_rejects_reconsiderable, since they can be CPFP'd.
if (p2p_inv && AlreadyHaveTx(gtxid, /*include_reconsiderable=*/true)) return true;
auto it = m_peer_info.find(peer);
if (it == m_peer_info.end()) return false;
const auto& info = it->second.m_connection_info;
if (!info.m_relay_permissions && m_txrequest.Count(peer) >= MAX_PEER_TX_ANNOUNCEMENTS) {
// Too many queued announcements for this peer
return false;
}
// Decide the TxRequestTracker parameters for this announcement:
// - "preferred": if fPreferredDownload is set (= outbound, or NetPermissionFlags::NoBan permission)
// - "reqtime": current time plus delays for:
// - NONPREF_PEER_TX_DELAY for announcements from non-preferred connections
// - TXID_RELAY_DELAY for txid announcements while wtxid peers are available
// - OVERLOADED_PEER_TX_DELAY for announcements from peers which have at least
// MAX_PEER_TX_REQUEST_IN_FLIGHT requests in flight (and don't have NetPermissionFlags::Relay).
auto delay{0us};
if (!info.m_preferred) delay += NONPREF_PEER_TX_DELAY;
if (!gtxid.IsWtxid() && m_num_wtxid_peers > 0) delay += TXID_RELAY_DELAY;
const bool overloaded = !info.m_relay_permissions && m_txrequest.CountInFlight(peer) >= MAX_PEER_TX_REQUEST_IN_FLIGHT;
if (overloaded) delay += OVERLOADED_PEER_TX_DELAY;
m_txrequest.ReceivedInv(peer, gtxid, info.m_preferred, now + delay);
return false;
}
std::vector<GenTxid> TxDownloadManagerImpl::GetRequestsToSend(NodeId nodeid, std::chrono::microseconds current_time)
{
std::vector<GenTxid> requests;
std::vector<std::pair<NodeId, GenTxid>> expired;
auto requestable = m_txrequest.GetRequestable(nodeid, current_time, &expired);
for (const auto& entry : expired) {
LogDebug(BCLog::NET, "timeout of inflight %s %s from peer=%d\n", entry.second.IsWtxid() ? "wtx" : "tx",
entry.second.GetHash().ToString(), entry.first);
}
for (const GenTxid& gtxid : requestable) {
if (!AlreadyHaveTx(gtxid, /*include_reconsiderable=*/false)) {
LogDebug(BCLog::NET, "Requesting %s %s peer=%d\n", gtxid.IsWtxid() ? "wtx" : "tx",
gtxid.GetHash().ToString(), nodeid);
requests.emplace_back(gtxid);
m_txrequest.RequestedTx(nodeid, gtxid.GetHash(), current_time + GETDATA_TX_INTERVAL);
} else {
// We have already seen this transaction, no need to download. This is just a belt-and-suspenders, as
// this should already be called whenever a transaction becomes AlreadyHaveTx().
m_txrequest.ForgetTxHash(gtxid.GetHash());
}
}
return requests;
}
void TxDownloadManagerImpl::ReceivedNotFound(NodeId nodeid, const std::vector<uint256>& txhashes)
{
for (const auto& txhash : txhashes) {
// If we receive a NOTFOUND message for a tx we requested, mark the announcement for it as
// completed in TxRequestTracker.
m_txrequest.ReceivedResponse(nodeid, txhash);
}
}
std::optional<PackageToValidate> TxDownloadManagerImpl::Find1P1CPackage(const CTransactionRef& ptx, NodeId nodeid)
{
const auto& parent_wtxid{ptx->GetWitnessHash()};
Assume(RecentRejectsReconsiderableFilter().contains(parent_wtxid.ToUint256()));
// Prefer children from this peer. This helps prevent censorship attempts in which an attacker
// sends lots of fake children for the parent, and we (unluckily) keep selecting the fake
// children instead of the real one provided by the honest peer.
const auto cpfp_candidates_same_peer{m_orphanage.GetChildrenFromSamePeer(ptx, nodeid)};
// These children should be sorted from newest to oldest. In the (probably uncommon) case
// of children that replace each other, this helps us accept the highest feerate (probably the
// most recent) one efficiently.
for (const auto& child : cpfp_candidates_same_peer) {
Package maybe_cpfp_package{ptx, child};
if (!RecentRejectsReconsiderableFilter().contains(GetPackageHash(maybe_cpfp_package)) &&
!RecentRejectsFilter().contains(child->GetHash().ToUint256())) {
return PackageToValidate{ptx, child, nodeid, nodeid};
}
}
// If no suitable candidate from the same peer is found, also try children that were provided by
// a different peer. This is useful because sometimes multiple peers announce both transactions
// to us, and we happen to download them from different peers (we wouldn't have known that these
// 2 transactions are related). We still want to find 1p1c packages then.
//
// If we start tracking all announcers of orphans, we can restrict this logic to parent + child
// pairs in which both were provided by the same peer, i.e. delete this step.
const auto cpfp_candidates_different_peer{m_orphanage.GetChildrenFromDifferentPeer(ptx, nodeid)};
// Find the first 1p1c that hasn't already been rejected. We randomize the order to not
// create a bias that attackers can use to delay package acceptance.
//
// Create a random permutation of the indices.
std::vector<size_t> tx_indices(cpfp_candidates_different_peer.size());
std::iota(tx_indices.begin(), tx_indices.end(), 0);
std::shuffle(tx_indices.begin(), tx_indices.end(), m_opts.m_rng);
for (const auto index : tx_indices) {
// If we already tried a package and failed for any reason, the combined hash was
// cached in m_lazy_recent_rejects_reconsiderable.
const auto [child_tx, child_sender] = cpfp_candidates_different_peer.at(index);
Package maybe_cpfp_package{ptx, child_tx};
if (!RecentRejectsReconsiderableFilter().contains(GetPackageHash(maybe_cpfp_package)) &&
!RecentRejectsFilter().contains(child_tx->GetHash().ToUint256())) {
return PackageToValidate{ptx, child_tx, nodeid, child_sender};
}
}
return std::nullopt;
}
void TxDownloadManagerImpl::MempoolAcceptedTx(const CTransactionRef& tx)
{
// 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.
m_txrequest.ForgetTxHash(tx->GetHash());
m_txrequest.ForgetTxHash(tx->GetWitnessHash());
m_orphanage.AddChildrenToWorkSet(*tx);
// If it came from the orphanage, remove it. No-op if the tx is not in txorphanage.
m_orphanage.EraseTx(tx->GetWitnessHash());
}
node::RejectedTxTodo TxDownloadManagerImpl::MempoolRejectedTx(const CTransactionRef& ptx, const TxValidationState& state, NodeId nodeid, bool first_time_failure)
{
const CTransaction& tx{*ptx};
// Results returned to caller
// Whether we should call AddToCompactExtraTransactions at the end
bool add_extra_compact_tx{first_time_failure};
// Hashes to pass to AddKnownTx later
std::vector<uint256> unique_parents;
// Populated if failure is reconsiderable and eligible package is found.
std::optional<node::PackageToValidate> package_to_validate;
if (state.GetResult() == TxValidationResult::TX_MISSING_INPUTS) {
// Only process a new orphan if this is a first time failure, as otherwise it must be either
// already in orphanage or from 1p1c processing.
if (first_time_failure && !RecentRejectsFilter().contains(ptx->GetWitnessHash().ToUint256())) {
bool fRejectedParents = false; // It may be the case that the orphans parents have all been rejected
// Deduplicate parent txids, so that we don't have to loop over
// the same parent txid more than once down below.
unique_parents.reserve(tx.vin.size());
for (const CTxIn& txin : tx.vin) {
// We start with all parents, and then remove duplicates below.
unique_parents.push_back(txin.prevout.hash);
}
std::sort(unique_parents.begin(), unique_parents.end());
unique_parents.erase(std::unique(unique_parents.begin(), unique_parents.end()), unique_parents.end());
// Distinguish between parents in m_lazy_recent_rejects and m_lazy_recent_rejects_reconsiderable.
// We can tolerate having up to 1 parent in m_lazy_recent_rejects_reconsiderable since we
// submit 1p1c packages. However, fail immediately if any are in m_lazy_recent_rejects.
std::optional<uint256> rejected_parent_reconsiderable;
for (const uint256& parent_txid : unique_parents) {
if (RecentRejectsFilter().contains(parent_txid)) {
fRejectedParents = true;
break;
} else if (RecentRejectsReconsiderableFilter().contains(parent_txid) &&
!m_opts.m_mempool.exists(GenTxid::Txid(parent_txid))) {
// More than 1 parent in m_lazy_recent_rejects_reconsiderable: 1p1c will not be
// sufficient to accept this package, so just give up here.
if (rejected_parent_reconsiderable.has_value()) {
fRejectedParents = true;
break;
}
rejected_parent_reconsiderable = parent_txid;
}
}
if (!fRejectedParents) {
const auto current_time{GetTime<std::chrono::microseconds>()};
for (const uint256& parent_txid : unique_parents) {
// Here, we only have the txid (and not wtxid) of the
// inputs, so we only request in txid mode, even for
// wtxidrelay peers.
// Eventually we should replace this with an improved
// protocol for getting all unconfirmed parents.
const auto gtxid{GenTxid::Txid(parent_txid)};
// Exclude m_lazy_recent_rejects_reconsiderable: the missing parent may have been
// previously rejected for being too low feerate. This orphan might CPFP it.
if (!AlreadyHaveTx(gtxid, /*include_reconsiderable=*/false)) {
AddTxAnnouncement(nodeid, gtxid, current_time, /*p2p_inv=*/false);
}
}
// Potentially flip add_extra_compact_tx to false if AddTx returns false because the tx was already there
add_extra_compact_tx &= m_orphanage.AddTx(ptx, nodeid);
// Once added to the orphan pool, a tx is considered AlreadyHave, and we shouldn't request it anymore.
m_txrequest.ForgetTxHash(tx.GetHash());
m_txrequest.ForgetTxHash(tx.GetWitnessHash());
// DoS prevention: do not allow m_orphanage to grow unbounded (see CVE-2012-3789)
m_orphanage.LimitOrphans(m_opts.m_max_orphan_txs, m_opts.m_rng);
} else {
unique_parents.clear();
LogDebug(BCLog::MEMPOOL, "not keeping orphan with rejected parents %s (wtxid=%s)\n",
tx.GetHash().ToString(),
tx.GetWitnessHash().ToString());
// We will continue to reject this tx since it has rejected
// parents so avoid re-requesting it from other peers.
// Here we add both the txid and the wtxid, as we know that
// regardless of what witness is provided, we will not accept
// this, so we don't need to allow for redownload of this txid
// from any of our non-wtxidrelay peers.
RecentRejectsFilter().insert(tx.GetHash().ToUint256());
RecentRejectsFilter().insert(tx.GetWitnessHash().ToUint256());
m_txrequest.ForgetTxHash(tx.GetHash());
m_txrequest.ForgetTxHash(tx.GetWitnessHash());
}
}
} else if (state.GetResult() == TxValidationResult::TX_WITNESS_STRIPPED) {
add_extra_compact_tx = false;
} else {
// We can add the wtxid of this transaction to our reject filter.
// Do not add txids of witness transactions or witness-stripped
// transactions to the filter, as they can have been malleated;
// adding such txids to the reject filter would potentially
// interfere with relay of valid transactions from peers that
// do not support wtxid-based relay. See
// https://github.com/bitcoin/bitcoin/issues/8279 for details.
// We can remove this restriction (and always add wtxids to
// the filter even for witness stripped transactions) once
// wtxid-based relay is broadly deployed.
// See also comments in https://github.com/bitcoin/bitcoin/pull/18044#discussion_r443419034
// for concerns around weakening security of unupgraded nodes
// if we start doing this too early.
if (state.GetResult() == TxValidationResult::TX_RECONSIDERABLE) {
// If the result is TX_RECONSIDERABLE, add it to m_lazy_recent_rejects_reconsiderable
// because we should not download or submit this transaction by itself again, but may
// submit it as part of a package later.
RecentRejectsReconsiderableFilter().insert(ptx->GetWitnessHash().ToUint256());
if (first_time_failure) {
// When a transaction fails for TX_RECONSIDERABLE, look for a matching child in the
// orphanage, as it is possible that they succeed as a package.
LogDebug(BCLog::TXPACKAGES, "tx %s (wtxid=%s) failed but reconsiderable, looking for child in orphanage\n",
ptx->GetHash().ToString(), ptx->GetWitnessHash().ToString());
package_to_validate = Find1P1CPackage(ptx, nodeid);
}
} else {
RecentRejectsFilter().insert(ptx->GetWitnessHash().ToUint256());
}
m_txrequest.ForgetTxHash(ptx->GetWitnessHash());
// If the transaction failed for TX_INPUTS_NOT_STANDARD,
// then we know that the witness was irrelevant to the policy
// failure, since this check depends only on the txid
// (the scriptPubKey being spent is covered by the txid).
// Add the txid to the reject filter to prevent repeated
// processing of this transaction in the event that child
// transactions are later received (resulting in
// parent-fetching by txid via the orphan-handling logic).
// We only add the txid if it differs from the wtxid, to avoid wasting entries in the
// rolling bloom filter.
if (state.GetResult() == TxValidationResult::TX_INPUTS_NOT_STANDARD && ptx->HasWitness()) {
RecentRejectsFilter().insert(ptx->GetHash().ToUint256());
m_txrequest.ForgetTxHash(ptx->GetHash());
}
}
// If the tx failed in ProcessOrphanTx, it should be removed from the orphanage unless the
// tx was still missing inputs. If the tx was not in the orphanage, EraseTx does nothing and returns 0.
if (state.GetResult() != TxValidationResult::TX_MISSING_INPUTS && m_orphanage.EraseTx(ptx->GetWitnessHash()) > 0) {
LogDebug(BCLog::TXPACKAGES, " removed orphan tx %s (wtxid=%s)\n", ptx->GetHash().ToString(), ptx->GetWitnessHash().ToString());
}
return RejectedTxTodo{
.m_should_add_extra_compact_tx = add_extra_compact_tx,
.m_unique_parents = std::move(unique_parents),
.m_package_to_validate = std::move(package_to_validate)
};
}
void TxDownloadManagerImpl::MempoolRejectedPackage(const Package& package)
{
RecentRejectsReconsiderableFilter().insert(GetPackageHash(package));
}
std::pair<bool, std::optional<PackageToValidate>> TxDownloadManagerImpl::ReceivedTx(NodeId nodeid, const CTransactionRef& ptx)
{
const uint256& txid = ptx->GetHash();
const uint256& wtxid = ptx->GetWitnessHash();
// Mark that we have received a response
m_txrequest.ReceivedResponse(nodeid, txid);
if (ptx->HasWitness()) m_txrequest.ReceivedResponse(nodeid, wtxid);
// First check if we should drop this tx.
// We do the AlreadyHaveTx() check using wtxid, rather than txid - in the
// absence of witness malleation, this is strictly better, because the
// recent rejects filter may contain the wtxid but rarely contains
// the txid of a segwit transaction that has been rejected.
// In the presence of witness malleation, it's possible that by only
// doing the check with wtxid, we could overlook a transaction which
// was confirmed with a different witness, or exists in our mempool
// with a different witness, but this has limited downside:
// mempool validation does its own lookup of whether we have the txid
// already; and an adversary can already relay us old transactions
// (older than our recency filter) if trying to DoS us, without any need
// for witness malleation.
if (AlreadyHaveTx(GenTxid::Wtxid(wtxid), /*include_reconsiderable=*/false)) {
// If a tx is detected by m_lazy_recent_rejects it is ignored. Because we haven't
// submitted the tx to our mempool, we won't have computed a DoS
// score for it or determined exactly why we consider it invalid.
//
// This means we won't penalize any peer subsequently relaying a DoSy
// tx (even if we penalized the first peer who gave it to us) because
// we have to account for m_lazy_recent_rejects showing false positives. In
// other words, we shouldn't penalize a peer if we aren't *sure* they
// submitted a DoSy tx.
//
// Note that m_lazy_recent_rejects doesn't just record DoSy or invalid
// transactions, but any tx not accepted by the mempool, which may be
// due to node policy (vs. consensus). So we can't blanket penalize a
// peer simply for relaying a tx that our m_lazy_recent_rejects has caught,
// regardless of false positives.
return {false, std::nullopt};
} else if (RecentRejectsReconsiderableFilter().contains(wtxid)) {
// When a transaction is already in m_lazy_recent_rejects_reconsiderable, we shouldn't submit
// it by itself again. However, look for a matching child in the orphanage, as it is
// possible that they succeed as a package.
LogDebug(BCLog::TXPACKAGES, "found tx %s (wtxid=%s) in reconsiderable rejects, looking for child in orphanage\n",
txid.ToString(), wtxid.ToString());
return {false, Find1P1CPackage(ptx, nodeid)};
}
return {true, std::nullopt};
}
bool TxDownloadManagerImpl::HaveMoreWork(NodeId nodeid)
{
return m_orphanage.HaveTxToReconsider(nodeid);
}
CTransactionRef TxDownloadManagerImpl::GetTxToReconsider(NodeId nodeid)
{
return m_orphanage.GetTxToReconsider(nodeid);
}
void TxDownloadManagerImpl::CheckIsEmpty(NodeId nodeid)
{
assert(m_txrequest.Count(nodeid) == 0);
}
void TxDownloadManagerImpl::CheckIsEmpty()
{
assert(m_orphanage.Size() == 0);
assert(m_txrequest.Size() == 0);
assert(m_num_wtxid_peers == 0);
}
std::vector<TxOrphanage::OrphanTxBase> TxDownloadManagerImpl::GetOrphanTransactions() const
{
return m_orphanage.GetOrphanTransactions();
}
} // namespace node

View file

@ -0,0 +1,192 @@
// Copyright (c) 2024
// Distributed under the MIT software license, see the accompanying
// file COPYING or http://www.opensource.org/licenses/mit-license.php.
#ifndef BITCOIN_NODE_TXDOWNLOADMAN_IMPL_H
#define BITCOIN_NODE_TXDOWNLOADMAN_IMPL_H
#include <node/txdownloadman.h>
#include <common/bloom.h>
#include <consensus/validation.h>
#include <kernel/chain.h>
#include <net.h>
#include <primitives/transaction.h>
#include <policy/packages.h>
#include <txorphanage.h>
#include <txrequest.h>
class CTxMemPool;
namespace node {
class TxDownloadManagerImpl {
public:
TxDownloadOptions m_opts;
/** Manages unvalidated tx data (orphan transactions for which we are downloading ancestors). */
TxOrphanage m_orphanage;
/** Tracks candidates for requesting and downloading transaction data. */
TxRequestTracker m_txrequest;
/**
* Filter for transactions that were recently rejected by the mempool.
* These are not rerequested until the chain tip changes, at which point
* the entire filter is reset.
*
* Without this filter we'd be re-requesting txs from each of our peers,
* increasing bandwidth consumption considerably. For instance, with 100
* peers, half of which relay a tx we don't accept, that might be a 50x
* bandwidth increase. A flooding attacker attempting to roll-over the
* filter using minimum-sized, 60byte, transactions might manage to send
* 1000/sec if we have fast peers, so we pick 120,000 to give our peers a
* two minute window to send invs to us.
*
* Decreasing the false positive rate is fairly cheap, so we pick one in a
* million to make it highly unlikely for users to have issues with this
* filter.
*
* We typically only add wtxids to this filter. For non-segwit
* transactions, the txid == wtxid, so this only prevents us from
* re-downloading non-segwit transactions when communicating with
* non-wtxidrelay peers -- which is important for avoiding malleation
* attacks that could otherwise interfere with transaction relay from
* non-wtxidrelay peers. For communicating with wtxidrelay peers, having
* the reject filter store wtxids is exactly what we want to avoid
* redownload of a rejected transaction.
*
* In cases where we can tell that a segwit transaction will fail
* validation no matter the witness, we may add the txid of such
* transaction to the filter as well. This can be helpful when
* communicating with txid-relay peers or if we were to otherwise fetch a
* transaction via txid (eg in our orphan handling).
*
* Memory used: 1.3 MB
*/
std::unique_ptr<CRollingBloomFilter> m_lazy_recent_rejects{nullptr};
CRollingBloomFilter& RecentRejectsFilter()
{
if (!m_lazy_recent_rejects) {
m_lazy_recent_rejects = std::make_unique<CRollingBloomFilter>(120'000, 0.000'001);
}
return *m_lazy_recent_rejects;
}
/**
* Filter for:
* (1) wtxids of transactions that were recently rejected by the mempool but are
* eligible for reconsideration if submitted with other transactions.
* (2) packages (see GetPackageHash) we have already rejected before and should not retry.
*
* Similar to m_lazy_recent_rejects, this filter is used to save bandwidth when e.g. all of our peers
* have larger mempools and thus lower minimum feerates than us.
*
* When a transaction's error is TxValidationResult::TX_RECONSIDERABLE (in a package or by
* itself), add its wtxid to this filter. When a package fails for any reason, add the combined
* hash to this filter.
*
* Upon receiving an announcement for a transaction, if it exists in this filter, do not
* download the txdata. When considering packages, if it exists in this filter, drop it.
*
* Reset this filter when the chain tip changes.
*
* Parameters are picked to be the same as m_lazy_recent_rejects, with the same rationale.
*/
std::unique_ptr<CRollingBloomFilter> m_lazy_recent_rejects_reconsiderable{nullptr};
CRollingBloomFilter& RecentRejectsReconsiderableFilter()
{
if (!m_lazy_recent_rejects_reconsiderable) {
m_lazy_recent_rejects_reconsiderable = std::make_unique<CRollingBloomFilter>(120'000, 0.000'001);
}
return *m_lazy_recent_rejects_reconsiderable;
}
/*
* Filter for transactions that have been recently confirmed.
* We use this to avoid requesting transactions that have already been
* confirmed.
*
* Blocks don't typically have more than 4000 transactions, so this should
* be at least six blocks (~1 hr) worth of transactions that we can store,
* inserting both a txid and wtxid for every observed transaction.
* If the number of transactions appearing in a block goes up, or if we are
* seeing getdata requests more than an hour after initial announcement, we
* can increase this number.
* The false positive rate of 1/1M should come out to less than 1
* transaction per day that would be inadvertently ignored (which is the
* same probability that we have in the reject filter).
*/
std::unique_ptr<CRollingBloomFilter> m_lazy_recent_confirmed_transactions{nullptr};
CRollingBloomFilter& RecentConfirmedTransactionsFilter()
{
if (!m_lazy_recent_confirmed_transactions) {
m_lazy_recent_confirmed_transactions = std::make_unique<CRollingBloomFilter>(48'000, 0.000'001);
}
return *m_lazy_recent_confirmed_transactions;
}
TxDownloadManagerImpl(const TxDownloadOptions& options) : m_opts{options}, m_txrequest{options.m_deterministic_txrequest} {}
struct PeerInfo {
/** Information relevant to scheduling tx requests. */
const TxDownloadConnectionInfo m_connection_info;
PeerInfo(const TxDownloadConnectionInfo& info) : m_connection_info{info} {}
};
/** Information for all of the peers we may download transactions from. This is not necessarily
* all peers we are connected to (no block-relay-only and temporary connections). */
std::map<NodeId, PeerInfo> m_peer_info;
/** Number of wtxid relay peers we have in m_peer_info. */
uint32_t m_num_wtxid_peers{0};
void ActiveTipChange();
void BlockConnected(const std::shared_ptr<const CBlock>& pblock);
void BlockDisconnected();
/** Check whether we already have this gtxid in:
* - mempool
* - orphanage
* - m_recent_rejects
* - m_recent_rejects_reconsiderable (if include_reconsiderable = true)
* - m_recent_confirmed_transactions
* */
bool AlreadyHaveTx(const GenTxid& gtxid, bool include_reconsiderable);
void ConnectedPeer(NodeId nodeid, const TxDownloadConnectionInfo& info);
void DisconnectedPeer(NodeId nodeid);
/** New inv has been received. May be added as a candidate to txrequest. */
bool AddTxAnnouncement(NodeId peer, const GenTxid& gtxid, std::chrono::microseconds now, bool p2p_inv);
/** Get getdata requests to send. */
std::vector<GenTxid> GetRequestsToSend(NodeId nodeid, std::chrono::microseconds current_time);
/** Marks a tx as ReceivedResponse in txrequest. */
void ReceivedNotFound(NodeId nodeid, const std::vector<uint256>& txhashes);
/** 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<PackageToValidate> Find1P1CPackage(const CTransactionRef& ptx, NodeId nodeid);
void MempoolAcceptedTx(const CTransactionRef& tx);
RejectedTxTodo MempoolRejectedTx(const CTransactionRef& ptx, const TxValidationState& state, NodeId nodeid, bool first_time_failure);
void MempoolRejectedPackage(const Package& package);
std::pair<bool, std::optional<PackageToValidate>> ReceivedTx(NodeId nodeid, const CTransactionRef& ptx);
bool HaveMoreWork(NodeId nodeid);
CTransactionRef GetTxToReconsider(NodeId nodeid);
void CheckIsEmpty();
void CheckIsEmpty(NodeId nodeid);
std::vector<TxOrphanage::OrphanTxBase> GetOrphanTransactions() const;
};
} // namespace node
#endif // BITCOIN_NODE_TXDOWNLOADMAN_IMPL_H

View file

@ -125,6 +125,7 @@ add_executable(test_bitcoin
torcontrol_tests.cpp torcontrol_tests.cpp
transaction_tests.cpp transaction_tests.cpp
translation_tests.cpp translation_tests.cpp
txdownload_tests.cpp
txindex_tests.cpp txindex_tests.cpp
txpackage_tests.cpp txpackage_tests.cpp
txreconciliation_tests.cpp txreconciliation_tests.cpp

View file

@ -117,6 +117,7 @@ add_executable(fuzz
timeoffsets.cpp timeoffsets.cpp
torcontrol.cpp torcontrol.cpp
transaction.cpp transaction.cpp
txdownloadman.cpp
tx_in.cpp tx_in.cpp
tx_out.cpp tx_out.cpp
tx_pool.cpp tx_pool.cpp

View file

@ -0,0 +1,445 @@
// Copyright (c) 2023 The Bitcoin Core developers
// Distributed under the MIT software license, see the accompanying
// file COPYING or http://www.opensource.org/licenses/mit-license.php.
#include <consensus/validation.h>
#include <node/context.h>
#include <node/mempool_args.h>
#include <node/miner.h>
#include <node/txdownloadman.h>
#include <node/txdownloadman_impl.h>
#include <test/fuzz/FuzzedDataProvider.h>
#include <test/fuzz/fuzz.h>
#include <test/fuzz/util.h>
#include <test/fuzz/util/mempool.h>
#include <test/util/mining.h>
#include <test/util/script.h>
#include <test/util/setup_common.h>
#include <test/util/txmempool.h>
#include <util/hasher.h>
#include <util/rbf.h>
#include <txmempool.h>
#include <validation.h>
#include <validationinterface.h>
namespace {
const TestingSetup* g_setup;
constexpr size_t NUM_COINS{50};
COutPoint COINS[NUM_COINS];
static TxValidationResult TESTED_TX_RESULTS[] = {
// Skip TX_RESULT_UNSET
TxValidationResult::TX_CONSENSUS,
TxValidationResult::TX_RECENT_CONSENSUS_CHANGE,
TxValidationResult::TX_INPUTS_NOT_STANDARD,
TxValidationResult::TX_NOT_STANDARD,
TxValidationResult::TX_MISSING_INPUTS,
TxValidationResult::TX_PREMATURE_SPEND,
TxValidationResult::TX_WITNESS_MUTATED,
TxValidationResult::TX_WITNESS_STRIPPED,
TxValidationResult::TX_CONFLICT,
TxValidationResult::TX_MEMPOOL_POLICY,
// Skip TX_NO_MEMPOOL
TxValidationResult::TX_RECONSIDERABLE,
TxValidationResult::TX_UNKNOWN,
};
// Precomputed transactions. Some may conflict with each other.
std::vector<CTransactionRef> TRANSACTIONS;
// Limit the total number of peers because we don't expect coverage to change much with lots more peers.
constexpr int NUM_PEERS = 16;
// Precomputed random durations (positive and negative, each ~exponentially distributed).
std::chrono::microseconds TIME_SKIPS[128];
static CTransactionRef MakeTransactionSpending(const std::vector<COutPoint>& outpoints, size_t num_outputs, bool add_witness)
{
CMutableTransaction tx;
// If no outpoints are given, create a random one.
for (const auto& outpoint : outpoints) {
tx.vin.emplace_back(outpoint);
}
if (add_witness) {
tx.vin[0].scriptWitness.stack.push_back({1});
}
for (size_t o = 0; o < num_outputs; ++o) tx.vout.emplace_back(CENT, P2WSH_OP_TRUE);
return MakeTransactionRef(tx);
}
static std::vector<COutPoint> PickCoins(FuzzedDataProvider& fuzzed_data_provider)
{
std::vector<COutPoint> ret;
ret.push_back(fuzzed_data_provider.PickValueInArray(COINS));
LIMITED_WHILE(fuzzed_data_provider.ConsumeBool(), 10) {
ret.push_back(fuzzed_data_provider.PickValueInArray(COINS));
}
return ret;
}
void initialize()
{
static const auto testing_setup = MakeNoLogFileContext<const TestingSetup>();
g_setup = testing_setup.get();
for (uint32_t i = 0; i < uint32_t{NUM_COINS}; ++i) {
COINS[i] = COutPoint{Txid::FromUint256((HashWriter() << i).GetHash()), i};
}
size_t outpoints_index = 0;
// 2 transactions same txid different witness
{
auto tx1{MakeTransactionSpending({COINS[outpoints_index]}, /*num_outputs=*/5, /*add_witness=*/false)};
auto tx2{MakeTransactionSpending({COINS[outpoints_index]}, /*num_outputs=*/5, /*add_witness=*/true)};
Assert(tx1->GetHash() == tx2->GetHash());
TRANSACTIONS.emplace_back(tx1);
TRANSACTIONS.emplace_back(tx2);
outpoints_index += 1;
}
// 2 parents 1 child
{
auto tx_parent_1{MakeTransactionSpending({COINS[outpoints_index++]}, /*num_outputs=*/1, /*add_witness=*/true)};
TRANSACTIONS.emplace_back(tx_parent_1);
auto tx_parent_2{MakeTransactionSpending({COINS[outpoints_index++]}, /*num_outputs=*/1, /*add_witness=*/false)};
TRANSACTIONS.emplace_back(tx_parent_2);
TRANSACTIONS.emplace_back(MakeTransactionSpending({COutPoint{tx_parent_1->GetHash(), 0}, COutPoint{tx_parent_2->GetHash(), 0}},
/*num_outputs=*/1, /*add_witness=*/true));
}
// 1 parent 2 children
{
auto tx_parent{MakeTransactionSpending({COINS[outpoints_index++]}, /*num_outputs=*/2, /*add_witness=*/true)};
TRANSACTIONS.emplace_back(tx_parent);
TRANSACTIONS.emplace_back(MakeTransactionSpending({COutPoint{tx_parent->GetHash(), 0}},
/*num_outputs=*/1, /*add_witness=*/true));
TRANSACTIONS.emplace_back(MakeTransactionSpending({COutPoint{tx_parent->GetHash(), 1}},
/*num_outputs=*/1, /*add_witness=*/true));
}
// chain of 5 segwit
{
COutPoint& last_outpoint = COINS[outpoints_index++];
for (auto i{0}; i < 5; ++i) {
auto tx{MakeTransactionSpending({last_outpoint}, /*num_outputs=*/1, /*add_witness=*/true)};
TRANSACTIONS.emplace_back(tx);
last_outpoint = COutPoint{tx->GetHash(), 0};
}
}
// chain of 5 non-segwit
{
COutPoint& last_outpoint = COINS[outpoints_index++];
for (auto i{0}; i < 5; ++i) {
auto tx{MakeTransactionSpending({last_outpoint}, /*num_outputs=*/1, /*add_witness=*/false)};
TRANSACTIONS.emplace_back(tx);
last_outpoint = COutPoint{tx->GetHash(), 0};
}
}
// Also create a loose tx for each outpoint. Some of these transactions conflict with the above
// or have the same txid.
for (const auto& outpoint : COINS) {
TRANSACTIONS.emplace_back(MakeTransactionSpending({outpoint}, /*num_outputs=*/1, /*add_witness=*/true));
}
// Create random-looking time jumps
int i = 0;
// TIME_SKIPS[N] for N=0..15 is just N microseconds.
for (; i < 16; ++i) {
TIME_SKIPS[i] = std::chrono::microseconds{i};
}
// TIME_SKIPS[N] for N=16..127 has randomly-looking but roughly exponentially increasing values up to
// 198.416453 seconds.
for (; i < 128; ++i) {
int diff_bits = ((i - 10) * 2) / 9;
uint64_t diff = 1 + (CSipHasher(0, 0).Write(i).Finalize() >> (64 - diff_bits));
TIME_SKIPS[i] = TIME_SKIPS[i - 1] + std::chrono::microseconds{diff};
}
}
void CheckPackageToValidate(const node::PackageToValidate& package_to_validate, NodeId peer)
{
Assert(package_to_validate.m_senders.size() == 2);
Assert(package_to_validate.m_senders.front() == peer);
Assert(package_to_validate.m_senders.back() < NUM_PEERS);
// Package is a 1p1c
const auto& package = package_to_validate.m_txns;
Assert(IsChildWithParents(package));
Assert(package.size() == 2);
}
FUZZ_TARGET(txdownloadman, .init = initialize)
{
FuzzedDataProvider fuzzed_data_provider(buffer.data(), buffer.size());
// Initialize txdownloadman
bilingual_str error;
CTxMemPool pool{MemPoolOptionsForTest(g_setup->m_node), error};
const auto max_orphan_count = fuzzed_data_provider.ConsumeIntegralInRange<unsigned int>(0, 300);
FastRandomContext det_rand{true};
node::TxDownloadManager txdownloadman{node::TxDownloadOptions{pool, det_rand, max_orphan_count, true}};
std::chrono::microseconds time{244466666};
LIMITED_WHILE(fuzzed_data_provider.ConsumeBool(), 10000)
{
NodeId rand_peer = fuzzed_data_provider.ConsumeIntegralInRange<int64_t>(0, NUM_PEERS - 1);
// Transaction can be one of the premade ones or a randomly generated one
auto rand_tx = fuzzed_data_provider.ConsumeBool() ?
MakeTransactionSpending(PickCoins(fuzzed_data_provider),
/*num_outputs=*/fuzzed_data_provider.ConsumeIntegralInRange(1, 500),
/*add_witness=*/fuzzed_data_provider.ConsumeBool()) :
TRANSACTIONS.at(fuzzed_data_provider.ConsumeIntegralInRange<unsigned>(0, TRANSACTIONS.size() - 1));
CallOneOf(
fuzzed_data_provider,
[&] {
node::TxDownloadConnectionInfo info{
.m_preferred = fuzzed_data_provider.ConsumeBool(),
.m_relay_permissions = fuzzed_data_provider.ConsumeBool(),
.m_wtxid_relay = fuzzed_data_provider.ConsumeBool()
};
txdownloadman.ConnectedPeer(rand_peer, info);
},
[&] {
txdownloadman.DisconnectedPeer(rand_peer);
txdownloadman.CheckIsEmpty(rand_peer);
},
[&] {
txdownloadman.ActiveTipChange();
},
[&] {
CBlock block;
block.vtx.push_back(rand_tx);
txdownloadman.BlockConnected(std::make_shared<CBlock>(block));
},
[&] {
txdownloadman.BlockDisconnected();
},
[&] {
txdownloadman.MempoolAcceptedTx(rand_tx);
},
[&] {
TxValidationState state;
state.Invalid(fuzzed_data_provider.PickValueInArray(TESTED_TX_RESULTS), "");
bool first_time_failure{fuzzed_data_provider.ConsumeBool()};
node::RejectedTxTodo todo = txdownloadman.MempoolRejectedTx(rand_tx, state, rand_peer, first_time_failure);
Assert(first_time_failure || !todo.m_should_add_extra_compact_tx);
},
[&] {
GenTxid gtxid = fuzzed_data_provider.ConsumeBool() ?
GenTxid::Txid(rand_tx->GetHash()) :
GenTxid::Wtxid(rand_tx->GetWitnessHash());
txdownloadman.AddTxAnnouncement(rand_peer, gtxid, time, /*p2p_inv=*/fuzzed_data_provider.ConsumeBool());
},
[&] {
txdownloadman.GetRequestsToSend(rand_peer, time);
},
[&] {
txdownloadman.ReceivedTx(rand_peer, rand_tx);
const auto& [should_validate, maybe_package] = txdownloadman.ReceivedTx(rand_peer, rand_tx);
// The only possible results should be:
// - Don't validate the tx, no package.
// - Don't validate the tx, package.
// - Validate the tx, no package.
// The only combination that doesn't make sense is validate both tx and package.
Assert(!(should_validate && maybe_package.has_value()));
if (maybe_package.has_value()) CheckPackageToValidate(*maybe_package, rand_peer);
},
[&] {
txdownloadman.ReceivedNotFound(rand_peer, {rand_tx->GetWitnessHash()});
},
[&] {
const bool expect_work{txdownloadman.HaveMoreWork(rand_peer)};
const auto ptx = txdownloadman.GetTxToReconsider(rand_peer);
// expect_work=true doesn't necessarily mean the next item from the workset isn't a
// nullptr, as the transaction could have been removed from orphanage without being
// removed from the peer's workset.
if (ptx) {
// However, if there was a non-null tx in the workset, HaveMoreWork should have
// returned true.
Assert(expect_work);
}
}
);
// Jump forwards or backwards
auto time_skip = fuzzed_data_provider.PickValueInArray(TIME_SKIPS);
if (fuzzed_data_provider.ConsumeBool()) time_skip *= -1;
time += time_skip;
}
// Disconnect everybody, check that all data structures are empty.
for (NodeId nodeid = 0; nodeid < NUM_PEERS; ++nodeid) {
txdownloadman.DisconnectedPeer(nodeid);
txdownloadman.CheckIsEmpty(nodeid);
}
txdownloadman.CheckIsEmpty();
}
// Give node 0 relay permissions, and nobody else. This helps us remember who is a RelayPermissions
// peer without tracking anything (this is only for the txdownload_impl target).
static bool HasRelayPermissions(NodeId peer) { return peer == 0; }
static void CheckInvariants(const node::TxDownloadManagerImpl& txdownload_impl, size_t max_orphan_count)
{
const TxOrphanage& orphanage = txdownload_impl.m_orphanage;
// Orphanage usage should never exceed what is allowed
Assert(orphanage.Size() <= max_orphan_count);
// We should never have more than the maximum in-flight requests out for a peer.
for (NodeId peer = 0; peer < NUM_PEERS; ++peer) {
if (!HasRelayPermissions(peer)) {
Assert(txdownload_impl.m_txrequest.CountInFlight(peer) <= node::MAX_PEER_TX_REQUEST_IN_FLIGHT);
}
}
txdownload_impl.m_txrequest.SanityCheck();
}
FUZZ_TARGET(txdownloadman_impl, .init = initialize)
{
FuzzedDataProvider fuzzed_data_provider(buffer.data(), buffer.size());
// Initialize a TxDownloadManagerImpl
bilingual_str error;
CTxMemPool pool{MemPoolOptionsForTest(g_setup->m_node), error};
const auto max_orphan_count = fuzzed_data_provider.ConsumeIntegralInRange<unsigned int>(0, 300);
FastRandomContext det_rand{true};
node::TxDownloadManagerImpl txdownload_impl{node::TxDownloadOptions{pool, det_rand, max_orphan_count, true}};
std::chrono::microseconds time{244466666};
LIMITED_WHILE(fuzzed_data_provider.ConsumeBool(), 10000)
{
NodeId rand_peer = fuzzed_data_provider.ConsumeIntegralInRange<int64_t>(0, NUM_PEERS - 1);
// Transaction can be one of the premade ones or a randomly generated one
auto rand_tx = fuzzed_data_provider.ConsumeBool() ?
MakeTransactionSpending(PickCoins(fuzzed_data_provider),
/*num_outputs=*/fuzzed_data_provider.ConsumeIntegralInRange(1, 500),
/*add_witness=*/fuzzed_data_provider.ConsumeBool()) :
TRANSACTIONS.at(fuzzed_data_provider.ConsumeIntegralInRange<unsigned>(0, TRANSACTIONS.size() - 1));
CallOneOf(
fuzzed_data_provider,
[&] {
node::TxDownloadConnectionInfo info{
.m_preferred = fuzzed_data_provider.ConsumeBool(),
.m_relay_permissions = HasRelayPermissions(rand_peer),
.m_wtxid_relay = fuzzed_data_provider.ConsumeBool()
};
txdownload_impl.ConnectedPeer(rand_peer, info);
},
[&] {
txdownload_impl.DisconnectedPeer(rand_peer);
txdownload_impl.CheckIsEmpty(rand_peer);
},
[&] {
txdownload_impl.ActiveTipChange();
// After a block update, nothing should be in the rejection caches
for (const auto& tx : TRANSACTIONS) {
Assert(!txdownload_impl.RecentRejectsFilter().contains(tx->GetWitnessHash().ToUint256()));
Assert(!txdownload_impl.RecentRejectsFilter().contains(tx->GetHash().ToUint256()));
Assert(!txdownload_impl.RecentRejectsReconsiderableFilter().contains(tx->GetWitnessHash().ToUint256()));
Assert(!txdownload_impl.RecentRejectsReconsiderableFilter().contains(tx->GetHash().ToUint256()));
}
},
[&] {
CBlock block;
block.vtx.push_back(rand_tx);
txdownload_impl.BlockConnected(std::make_shared<CBlock>(block));
// Block transactions must be removed from orphanage
Assert(!txdownload_impl.m_orphanage.HaveTx(rand_tx->GetWitnessHash()));
},
[&] {
txdownload_impl.BlockDisconnected();
Assert(!txdownload_impl.RecentConfirmedTransactionsFilter().contains(rand_tx->GetWitnessHash().ToUint256()));
Assert(!txdownload_impl.RecentConfirmedTransactionsFilter().contains(rand_tx->GetHash().ToUint256()));
},
[&] {
txdownload_impl.MempoolAcceptedTx(rand_tx);
},
[&] {
TxValidationState state;
state.Invalid(fuzzed_data_provider.PickValueInArray(TESTED_TX_RESULTS), "");
bool first_time_failure{fuzzed_data_provider.ConsumeBool()};
bool reject_contains_wtxid{txdownload_impl.RecentRejectsFilter().contains(rand_tx->GetWitnessHash().ToUint256())};
node::RejectedTxTodo todo = txdownload_impl.MempoolRejectedTx(rand_tx, state, rand_peer, first_time_failure);
Assert(first_time_failure || !todo.m_should_add_extra_compact_tx);
if (!reject_contains_wtxid) Assert(todo.m_unique_parents.size() <= rand_tx->vin.size());
},
[&] {
GenTxid gtxid = fuzzed_data_provider.ConsumeBool() ?
GenTxid::Txid(rand_tx->GetHash()) :
GenTxid::Wtxid(rand_tx->GetWitnessHash());
txdownload_impl.AddTxAnnouncement(rand_peer, gtxid, time, /*p2p_inv=*/fuzzed_data_provider.ConsumeBool());
},
[&] {
const auto getdata_requests = txdownload_impl.GetRequestsToSend(rand_peer, time);
// TxDownloadManager should not be telling us to request things we already have.
// Exclude m_lazy_recent_rejects_reconsiderable because it may request low-feerate parent of orphan.
for (const auto& gtxid : getdata_requests) {
Assert(!txdownload_impl.AlreadyHaveTx(gtxid, /*include_reconsiderable=*/false));
}
},
[&] {
const auto& [should_validate, maybe_package] = txdownload_impl.ReceivedTx(rand_peer, rand_tx);
// The only possible results should be:
// - Don't validate the tx, no package.
// - Don't validate the tx, package.
// - Validate the tx, no package.
// The only combination that doesn't make sense is validate both tx and package.
Assert(!(should_validate && maybe_package.has_value()));
if (should_validate) {
Assert(!txdownload_impl.AlreadyHaveTx(GenTxid::Wtxid(rand_tx->GetWitnessHash()), /*include_reconsiderable=*/true));
}
if (maybe_package.has_value()) {
CheckPackageToValidate(*maybe_package, rand_peer);
const auto& package = maybe_package->m_txns;
// Parent is in m_lazy_recent_rejects_reconsiderable and child is in m_orphanage
Assert(txdownload_impl.RecentRejectsReconsiderableFilter().contains(rand_tx->GetWitnessHash().ToUint256()));
Assert(txdownload_impl.m_orphanage.HaveTx(maybe_package->m_txns.back()->GetWitnessHash()));
// Package has not been rejected
Assert(!txdownload_impl.RecentRejectsReconsiderableFilter().contains(GetPackageHash(package)));
// Neither is in m_lazy_recent_rejects
Assert(!txdownload_impl.RecentRejectsFilter().contains(package.front()->GetWitnessHash().ToUint256()));
Assert(!txdownload_impl.RecentRejectsFilter().contains(package.back()->GetWitnessHash().ToUint256()));
}
},
[&] {
txdownload_impl.ReceivedNotFound(rand_peer, {rand_tx->GetWitnessHash()});
},
[&] {
const bool expect_work{txdownload_impl.HaveMoreWork(rand_peer)};
const auto ptx{txdownload_impl.GetTxToReconsider(rand_peer)};
// expect_work=true doesn't necessarily mean the next item from the workset isn't a
// nullptr, as the transaction could have been removed from orphanage without being
// removed from the peer's workset.
if (ptx) {
// However, if there was a non-null tx in the workset, HaveMoreWork should have
// returned true.
Assert(expect_work);
Assert(txdownload_impl.AlreadyHaveTx(GenTxid::Wtxid(ptx->GetWitnessHash()), /*include_reconsiderable=*/false));
// Presumably we have validated this tx. Use "missing inputs" to keep it in the
// orphanage longer. Later iterations might call MempoolAcceptedTx or
// MempoolRejectedTx with a different error.
TxValidationState state_missing_inputs;
state_missing_inputs.Invalid(TxValidationResult::TX_MISSING_INPUTS, "");
txdownload_impl.MempoolRejectedTx(ptx, state_missing_inputs, rand_peer, fuzzed_data_provider.ConsumeBool());
}
}
);
// Jump ahead in time
time += fuzzed_data_provider.PickValueInArray(TIME_SKIPS);
CheckInvariants(txdownload_impl, max_orphan_count);
}
// Disconnect everybody, check that all data structures are empty.
for (NodeId nodeid = 0; nodeid < NUM_PEERS; ++nodeid) {
txdownload_impl.DisconnectedPeer(nodeid);
txdownload_impl.CheckIsEmpty(nodeid);
}
txdownload_impl.CheckIsEmpty();
}
} // namespace

View file

@ -0,0 +1,337 @@
// Copyright (c) 2011-2022 The Bitcoin Core developers
// Distributed under the MIT software license, see the accompanying
// file COPYING or http://www.opensource.org/licenses/mit-license.php.
#include <addresstype.h>
#include <consensus/validation.h>
#include <net_processing.h>
#include <node/txdownloadman_impl.h>
#include <primitives/transaction.h>
#include <script/script.h>
#include <test/util/random.h>
#include <test/util/setup_common.h>
#include <validation.h>
#include <array>
#include <boost/test/unit_test.hpp>
BOOST_FIXTURE_TEST_SUITE(txdownload_tests, TestingSetup)
struct Behaviors {
bool m_txid_in_rejects;
bool m_wtxid_in_rejects;
bool m_txid_in_rejects_recon;
bool m_wtxid_in_rejects_recon;
bool m_keep_for_compact;
bool m_ignore_inv_txid;
bool m_ignore_inv_wtxid;
// Constructor. We are passing and casting ints because they are more readable in a table (see all_expected_results).
Behaviors(bool txid_rejects, bool wtxid_rejects, bool txid_recon, bool wtxid_recon, bool keep, bool txid_inv, bool wtxid_inv) :
m_txid_in_rejects(txid_rejects),
m_wtxid_in_rejects(wtxid_rejects),
m_txid_in_rejects_recon(txid_recon),
m_wtxid_in_rejects_recon(wtxid_recon),
m_keep_for_compact(keep),
m_ignore_inv_txid(txid_inv),
m_ignore_inv_wtxid(wtxid_inv)
{}
void CheckEqual(const Behaviors& other, bool segwit)
{
BOOST_CHECK_EQUAL(other.m_wtxid_in_rejects, m_wtxid_in_rejects);
BOOST_CHECK_EQUAL(other.m_wtxid_in_rejects_recon, m_wtxid_in_rejects_recon);
BOOST_CHECK_EQUAL(other.m_keep_for_compact, m_keep_for_compact);
BOOST_CHECK_EQUAL(other.m_ignore_inv_wtxid, m_ignore_inv_wtxid);
// false negatives for nonsegwit transactions, since txid == wtxid.
if (segwit) {
BOOST_CHECK_EQUAL(other.m_txid_in_rejects, m_txid_in_rejects);
BOOST_CHECK_EQUAL(other.m_txid_in_rejects_recon, m_txid_in_rejects_recon);
BOOST_CHECK_EQUAL(other.m_ignore_inv_txid, m_ignore_inv_txid);
}
}
};
// Map from failure reason to expected behavior for a segwit tx that fails
// Txid and Wtxid are assumed to be different here. For a nonsegwit transaction, use the wtxid results.
static std::map<TxValidationResult, Behaviors> expected_behaviors{
{TxValidationResult::TX_CONSENSUS, {/*txid_rejects*/0,/*wtxid_rejects*/1,/*txid_recon*/0,/*wtxid_recon*/0,/*keep*/1,/*txid_inv*/0,/*wtxid_inv*/1}},
{TxValidationResult::TX_RECENT_CONSENSUS_CHANGE, { 0, 1, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_INPUTS_NOT_STANDARD, { 1, 1, 0, 0, 1, 1, 1}},
{TxValidationResult::TX_NOT_STANDARD, { 0, 1, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_MISSING_INPUTS, { 0, 0, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_PREMATURE_SPEND, { 0, 1, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_WITNESS_MUTATED, { 0, 1, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_WITNESS_STRIPPED, { 0, 0, 0, 0, 0, 0, 0}},
{TxValidationResult::TX_CONFLICT, { 0, 1, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_MEMPOOL_POLICY, { 0, 1, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_NO_MEMPOOL, { 0, 1, 0, 0, 1, 0, 1}},
{TxValidationResult::TX_RECONSIDERABLE, { 0, 0, 0, 1, 1, 0, 1}},
{TxValidationResult::TX_UNKNOWN, { 0, 1, 0, 0, 1, 0, 1}},
};
static bool CheckOrphanBehavior(node::TxDownloadManagerImpl& txdownload_impl, const CTransactionRef& tx, const node::RejectedTxTodo& ret, std::string& err_msg,
bool expect_orphan, bool expect_keep, unsigned int expected_parents)
{
// Missing inputs can never result in a PackageToValidate.
if (ret.m_package_to_validate.has_value()) {
err_msg = strprintf("returned a PackageToValidate on missing inputs");
return false;
}
if (expect_orphan != txdownload_impl.m_orphanage.HaveTx(tx->GetWitnessHash())) {
err_msg = strprintf("unexpectedly %s tx in orpanage", expect_orphan ? "did not find" : "found");
return false;
}
if (expect_keep != ret.m_should_add_extra_compact_tx) {
err_msg = strprintf("unexpectedly returned %s add to vExtraTxnForCompact", expect_keep ? "should not" : "should");
return false;
}
if (expected_parents != ret.m_unique_parents.size()) {
err_msg = strprintf("expected %u unique_parents, got %u", expected_parents, ret.m_unique_parents.size());
return false;
}
return true;
}
static CTransactionRef CreatePlaceholderTx(bool segwit)
{
// Each tx returned from here spends the previous one.
static Txid prevout_hash{};
CMutableTransaction mtx;
mtx.vin.emplace_back(prevout_hash, 0);
// This makes txid != wtxid
if (segwit) mtx.vin[0].scriptWitness.stack.push_back({1});
mtx.vout.emplace_back(CENT, CScript());
auto ptx = MakeTransactionRef(mtx);
prevout_hash = ptx->GetHash();
return ptx;
}
BOOST_FIXTURE_TEST_CASE(tx_rejection_types, TestChain100Setup)
{
CTxMemPool& pool = *Assert(m_node.mempool);
FastRandomContext det_rand{true};
node::TxDownloadOptions DEFAULT_OPTS{pool, det_rand, DEFAULT_MAX_ORPHAN_TRANSACTIONS, true};
// A new TxDownloadManagerImpl is created for each tx so we can just reuse the same one.
TxValidationState state;
NodeId nodeid{0};
std::chrono::microseconds now{GetTime()};
node::TxDownloadConnectionInfo connection_info{/*m_preferred=*/false, /*m_relay_permissions=*/false, /*m_wtxid_relay=*/true};
for (const auto segwit_parent : {true, false}) {
for (const auto segwit_child : {true, false}) {
const auto ptx_parent = CreatePlaceholderTx(segwit_parent);
const auto ptx_child = CreatePlaceholderTx(segwit_child);
const auto& parent_txid = ptx_parent->GetHash().ToUint256();
const auto& parent_wtxid = ptx_parent->GetWitnessHash().ToUint256();
const auto& child_txid = ptx_child->GetHash().ToUint256();
const auto& child_wtxid = ptx_child->GetWitnessHash().ToUint256();
for (const auto& [result, expected_behavior] : expected_behaviors) {
node::TxDownloadManagerImpl txdownload_impl{DEFAULT_OPTS};
txdownload_impl.ConnectedPeer(nodeid, connection_info);
// Parent failure
state.Invalid(result, "");
const auto& [keep, unique_txids, package_to_validate] = txdownload_impl.MempoolRejectedTx(ptx_parent, state, nodeid, /*first_time_failure=*/true);
// No distinction between txid and wtxid caching for nonsegwit transactions, so only test these specific
// behaviors for segwit transactions.
Behaviors actual_behavior{
/*txid_rejects=*/txdownload_impl.RecentRejectsFilter().contains(parent_txid),
/*wtxid_rejects=*/txdownload_impl.RecentRejectsFilter().contains(parent_wtxid),
/*txid_recon=*/txdownload_impl.RecentRejectsReconsiderableFilter().contains(parent_txid),
/*wtxid_recon=*/txdownload_impl.RecentRejectsReconsiderableFilter().contains(parent_wtxid),
/*keep=*/keep,
/*txid_inv=*/txdownload_impl.AddTxAnnouncement(nodeid, GenTxid::Txid(parent_txid), now, /*p2p_inv=*/true),
/*wtxid_inv=*/txdownload_impl.AddTxAnnouncement(nodeid, GenTxid::Wtxid(parent_wtxid), now, /*p2p_inv=*/true),
};
BOOST_TEST_MESSAGE("Testing behavior for " << result << (segwit_parent ? " segwit " : " nonsegwit"));
actual_behavior.CheckEqual(expected_behavior, /*segwit=*/segwit_parent);
// Later, a child of this transaction fails for missing inputs
state.Invalid(TxValidationResult::TX_MISSING_INPUTS, "");
txdownload_impl.MempoolRejectedTx(ptx_child, state, nodeid, /*first_time_failure=*/true);
// If parent (by txid) was rejected, child is too.
const bool parent_txid_rejected{segwit_parent ? expected_behavior.m_txid_in_rejects : expected_behavior.m_wtxid_in_rejects};
BOOST_CHECK_EQUAL(parent_txid_rejected, txdownload_impl.RecentRejectsFilter().contains(child_txid));
BOOST_CHECK_EQUAL(parent_txid_rejected, txdownload_impl.RecentRejectsFilter().contains(child_wtxid));
// Unless rejected, the child should be in orphanage.
BOOST_CHECK_EQUAL(!parent_txid_rejected, txdownload_impl.m_orphanage.HaveTx(ptx_child->GetWitnessHash()));
}
}
}
}
BOOST_FIXTURE_TEST_CASE(handle_missing_inputs, TestChain100Setup)
{
CTxMemPool& pool = *Assert(m_node.mempool);
FastRandomContext det_rand{true};
node::TxDownloadOptions DEFAULT_OPTS{pool, det_rand, DEFAULT_MAX_ORPHAN_TRANSACTIONS, true};
NodeId nodeid{1};
node::TxDownloadConnectionInfo DEFAULT_CONN{/*m_preferred=*/false, /*m_relay_permissions=*/false, /*m_wtxid_relay=*/true};
// We need mature coinbases
mineBlocks(20);
// Transactions with missing inputs are treated differently depending on how much we know about
// their parents.
CKey wallet_key = GenerateRandomKey();
CScript destination = GetScriptForDestination(PKHash(wallet_key.GetPubKey()));
// Amount for spending coinbase in a 1-in-1-out tx, at depth n, each time deducting 1000 from the amount as fees.
CAmount amount_depth_1{50 * COIN - 1000};
CAmount amount_depth_2{amount_depth_1 - 1000};
// Amount for spending coinbase in a 1-in-2-out tx, deducting 1000 in fees
CAmount amount_split_half{25 * COIN - 500};
int test_chain_height{100};
TxValidationState state_orphan;
state_orphan.Invalid(TxValidationResult::TX_MISSING_INPUTS, "");
// Transactions are not all submitted to mempool. Conserve the number of m_coinbase_txns we
// consume, and only increment this index number when we would conflict with an existing
// mempool transaction.
size_t coinbase_idx{0};
for (int decisions = 0; decisions < (1 << 4); ++decisions) {
auto mtx_single_parent = CreateValidMempoolTransaction(m_coinbase_txns[coinbase_idx], /*input_vout=*/0, test_chain_height, coinbaseKey, destination, amount_depth_1, /*submit=*/false);
auto single_parent = MakeTransactionRef(mtx_single_parent);
auto mtx_orphan = CreateValidMempoolTransaction(single_parent, /*input_vout=*/0, test_chain_height, wallet_key, destination, amount_depth_2, /*submit=*/false);
auto orphan = MakeTransactionRef(mtx_orphan);
node::TxDownloadManagerImpl txdownload_impl{DEFAULT_OPTS};
txdownload_impl.ConnectedPeer(nodeid, DEFAULT_CONN);
// Each bit of decisions tells us whether the parent is in a particular cache.
// It is definitely possible for a transaction to be in multiple caches. For example, it
// may have both a low feerate and found to violate some mempool policy when validated
// in a 1p1c.
const bool parent_recent_rej(decisions & 1);
const bool parent_recent_rej_recon((decisions >> 1) & 1);
const bool parent_recent_conf((decisions >> 2) & 1);
const bool parent_in_mempool((decisions >> 3) & 1);
if (parent_recent_rej) txdownload_impl.RecentRejectsFilter().insert(single_parent->GetHash().ToUint256());
if (parent_recent_rej_recon) txdownload_impl.RecentRejectsReconsiderableFilter().insert(single_parent->GetHash().ToUint256());
if (parent_recent_conf) txdownload_impl.RecentConfirmedTransactionsFilter().insert(single_parent->GetHash().ToUint256());
if (parent_in_mempool) {
const auto mempool_result = WITH_LOCK(::cs_main, return m_node.chainman->ProcessTransaction(single_parent));
BOOST_CHECK(mempool_result.m_result_type == MempoolAcceptResult::ResultType::VALID);
coinbase_idx += 1;
assert(coinbase_idx < m_coinbase_txns.size());
}
// Whether or not the transaction is added as an orphan depends solely on whether or not
// it's in RecentRejectsFilter. Specifically, the parent is allowed to be in
// RecentRejectsReconsiderableFilter, but it cannot be in RecentRejectsFilter.
const bool expect_keep_orphan = !parent_recent_rej;
const auto ret_1p1c = txdownload_impl.MempoolRejectedTx(orphan, state_orphan, nodeid, /*first_time_failure=*/true);
std::string err_msg;
const bool ok = CheckOrphanBehavior(txdownload_impl, orphan, ret_1p1c, err_msg,
/*expect_orphan=*/expect_keep_orphan, /*expect_keep=*/true, /*expected_parents=*/expect_keep_orphan ? 1 : 0);
BOOST_CHECK_MESSAGE(ok, err_msg);
}
// Orphan with multiple parents
{
std::vector<CTransactionRef> parents;
std::vector<COutPoint> outpoints;
int32_t num_parents{24};
for (int32_t i = 0; i < num_parents; ++i) {
assert(coinbase_idx < m_coinbase_txns.size());
auto mtx_parent = CreateValidMempoolTransaction(m_coinbase_txns[coinbase_idx++], /*input_vout=*/0, test_chain_height,
coinbaseKey, destination, amount_depth_1 + i, /*submit=*/false);
auto ptx_parent = MakeTransactionRef(mtx_parent);
parents.emplace_back(ptx_parent);
outpoints.emplace_back(ptx_parent->GetHash(), 0);
}
// Send all coins to 1 output.
auto mtx_orphan = CreateValidMempoolTransaction(parents, outpoints, test_chain_height, {wallet_key}, {{amount_depth_2 * num_parents, destination}}, /*submit=*/false);
auto orphan = MakeTransactionRef(mtx_orphan);
// 1 parent in RecentRejectsReconsiderableFilter, the rest are unknown
{
node::TxDownloadManagerImpl txdownload_impl{DEFAULT_OPTS};
txdownload_impl.ConnectedPeer(nodeid, DEFAULT_CONN);
txdownload_impl.RecentRejectsReconsiderableFilter().insert(parents[0]->GetHash().ToUint256());
const auto ret_1p1c_parent_reconsiderable = txdownload_impl.MempoolRejectedTx(orphan, state_orphan, nodeid, /*first_time_failure=*/true);
std::string err_msg;
const bool ok = CheckOrphanBehavior(txdownload_impl, orphan, ret_1p1c_parent_reconsiderable, err_msg,
/*expect_orphan=*/true, /*expect_keep=*/true, /*expected_parents=*/num_parents);
BOOST_CHECK_MESSAGE(ok, err_msg);
}
// 1 parent in RecentRejectsReconsiderableFilter, the rest are confirmed
{
node::TxDownloadManagerImpl txdownload_impl{DEFAULT_OPTS};
txdownload_impl.ConnectedPeer(nodeid, DEFAULT_CONN);
txdownload_impl.RecentRejectsReconsiderableFilter().insert(parents[0]->GetHash().ToUint256());
for (int32_t i = 1; i < num_parents; ++i) {
txdownload_impl.RecentConfirmedTransactionsFilter().insert(parents[i]->GetHash().ToUint256());
}
const auto ret_1recon_conf = txdownload_impl.MempoolRejectedTx(orphan, state_orphan, nodeid, /*first_time_failure=*/true);
std::string err_msg;
const bool ok = CheckOrphanBehavior(txdownload_impl, orphan, ret_1recon_conf, err_msg,
/*expect_orphan=*/true, /*expect_keep=*/true, /*expected_parents=*/num_parents);
BOOST_CHECK_MESSAGE(ok, err_msg);
}
// 1 parent in RecentRejectsReconsiderableFilter, 1 other in {RecentRejectsReconsiderableFilter, RecentRejectsFilter}
for (int i = 0; i < 2; ++i) {
node::TxDownloadManagerImpl txdownload_impl{DEFAULT_OPTS};
txdownload_impl.ConnectedPeer(nodeid, DEFAULT_CONN);
txdownload_impl.RecentRejectsReconsiderableFilter().insert(parents[1]->GetHash().ToUint256());
// Doesn't really matter which parent
auto& alreadyhave_parent = parents[0];
if (i == 0) {
txdownload_impl.RecentRejectsReconsiderableFilter().insert(alreadyhave_parent->GetHash().ToUint256());
} else if (i == 1) {
txdownload_impl.RecentRejectsFilter().insert(alreadyhave_parent->GetHash().ToUint256());
}
const auto ret_2_problems = txdownload_impl.MempoolRejectedTx(orphan, state_orphan, nodeid, /*first_time_failure=*/true);
std::string err_msg;
const bool ok = CheckOrphanBehavior(txdownload_impl, orphan, ret_2_problems, err_msg,
/*expect_orphan=*/false, /*expect_keep=*/true, /*expected_parents=*/0);
BOOST_CHECK_MESSAGE(ok, err_msg);
}
}
// Orphan with multiple inputs spending from a single parent
{
assert(coinbase_idx < m_coinbase_txns.size());
auto parent_2outputs = MakeTransactionRef(CreateValidMempoolTransaction({m_coinbase_txns[coinbase_idx]}, {{m_coinbase_txns[coinbase_idx]->GetHash(), 0}}, test_chain_height, {coinbaseKey},
{{amount_split_half, destination}, {amount_split_half, destination}}, /*submit=*/false));
auto orphan = MakeTransactionRef(CreateValidMempoolTransaction({parent_2outputs}, {{parent_2outputs->GetHash(), 0}, {parent_2outputs->GetHash(), 1}},
test_chain_height, {wallet_key}, {{amount_depth_2, destination}}, /*submit=*/false));
// Parent is in RecentRejectsReconsiderableFilter. Inputs will find it twice, but this
// should only counts as 1 parent in the filter.
{
node::TxDownloadManagerImpl txdownload_impl{DEFAULT_OPTS};
txdownload_impl.ConnectedPeer(nodeid, DEFAULT_CONN);
txdownload_impl.RecentRejectsReconsiderableFilter().insert(parent_2outputs->GetHash().ToUint256());
const auto ret_1p1c_2reconsiderable = txdownload_impl.MempoolRejectedTx(orphan, state_orphan, nodeid, /*first_time_failure=*/true);
std::string err_msg;
const bool ok = CheckOrphanBehavior(txdownload_impl, orphan, ret_1p1c_2reconsiderable, err_msg,
/*expect_orphan=*/true, /*expect_keep=*/true, /*expected_parents=*/1);
BOOST_CHECK_MESSAGE(ok, err_msg);
}
}
}
BOOST_AUTO_TEST_SUITE_END()

View file

@ -67,7 +67,7 @@ public:
std::vector<std::pair<CTransactionRef, NodeId>> GetChildrenFromDifferentPeer(const CTransactionRef& parent, NodeId nodeid) const; std::vector<std::pair<CTransactionRef, NodeId>> GetChildrenFromDifferentPeer(const CTransactionRef& parent, NodeId nodeid) const;
/** Return how many entries exist in the orphange */ /** Return how many entries exist in the orphange */
size_t Size() size_t Size() const
{ {
return m_orphans.size(); return m_orphans.size();
} }