diff options
Diffstat (limited to 'src/net_processing.cpp')
-rw-r--r-- | src/net_processing.cpp | 517 |
1 files changed, 468 insertions, 49 deletions
diff --git a/src/net_processing.cpp b/src/net_processing.cpp index 64c2a29245..74700580ad 100644 --- a/src/net_processing.cpp +++ b/src/net_processing.cpp @@ -14,6 +14,7 @@ #include <consensus/validation.h> #include <deploymentstatus.h> #include <hash.h> +#include <headerssync.h> #include <index/blockfilterindex.h> #include <merkleblock.h> #include <netbase.h> @@ -289,7 +290,7 @@ struct Peer { * non-wtxid-relay peers, wtxid for wtxid-relay peers). We use the * mempool to sort transactions in dependency order before relay, so * this does not have to be sorted. */ - std::set<uint256> m_tx_inventory_to_send; + std::set<uint256> m_tx_inventory_to_send GUARDED_BY(m_tx_inventory_mutex); /** Whether the peer has requested us to send our complete mempool. Only * permitted if the peer has NetPermissionFlags::Mempool. See BIP35. */ bool m_send_mempool GUARDED_BY(m_tx_inventory_mutex){false}; @@ -370,6 +371,9 @@ struct Peer { /** Set of txids to reconsider once their parent transactions have been accepted **/ std::set<uint256> m_orphan_work_set GUARDED_BY(g_cs_orphans); + /** Whether we've sent this peer a getheaders in response to an inv prior to initial-headers-sync completing */ + bool m_inv_triggered_getheaders_before_sync{false}; + /** Protects m_getdata_requests **/ Mutex m_getdata_requests_mutex; /** Work queue of items requested by this peer **/ @@ -378,6 +382,15 @@ struct Peer { /** Time of the last getheaders message to this peer */ NodeClock::time_point m_last_getheaders_timestamp{}; + /** Protects m_headers_sync **/ + Mutex m_headers_sync_mutex; + /** Headers-sync state for this peer (eg for initial sync, or syncing large + * reorgs) **/ + std::unique_ptr<HeadersSyncState> m_headers_sync PT_GUARDED_BY(m_headers_sync_mutex) GUARDED_BY(m_headers_sync_mutex) {}; + + /** Whether we've sent our peer a sendheaders message. **/ + std::atomic<bool> m_sent_sendheaders{false}; + explicit Peer(NodeId id, ServiceFlags our_services) : m_id{id} , m_our_services{our_services} @@ -500,9 +513,9 @@ public: /** Implement NetEventsInterface */ void InitializeNode(CNode& node, ServiceFlags our_services) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex); - void FinalizeNode(const CNode& node) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex); + void FinalizeNode(const CNode& node) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_headers_presync_mutex); bool ProcessMessages(CNode* pfrom, std::atomic<bool>& interrupt) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex); bool SendMessages(CNode* pto) override EXCLUSIVE_LOCKS_REQUIRED(pto->cs_sendProcessing) EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex); @@ -519,7 +532,7 @@ public: void UnitTestMisbehaving(NodeId peer_id, int howmuch) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex) { Misbehaving(*Assert(GetPeerRef(peer_id)), howmuch, ""); }; void ProcessMessage(CNode& pfrom, const std::string& msg_type, CDataStream& vRecv, const std::chrono::microseconds time_received, const std::atomic<bool>& interruptMsgProc) override - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_recent_confirmed_transactions_mutex, !m_most_recent_block_mutex, !m_headers_presync_mutex); void UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds) override; private: @@ -578,18 +591,70 @@ private: void ProcessOrphanTx(std::set<uint256>& orphan_work_set) EXCLUSIVE_LOCKS_REQUIRED(cs_main, g_cs_orphans) EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex); - /** Process a single headers message from a peer. */ + /** Process a single headers message from a peer. + * + * @param[in] pfrom CNode of the peer + * @param[in] peer The peer sending us the headers + * @param[in] headers The headers received. Note that this may be modified within ProcessHeadersMessage. + * @param[in] via_compact_block Whether this header came in via compact block handling. + */ void ProcessHeadersMessage(CNode& pfrom, Peer& peer, - const std::vector<CBlockHeader>& headers, + std::vector<CBlockHeader>&& headers, bool via_compact_block) - EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex); + EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex, !m_headers_presync_mutex); /** Various helpers for headers processing, invoked by ProcessHeadersMessage() */ + /** Return true if headers are continuous and have valid proof-of-work (DoS points assigned on failure) */ + bool CheckHeadersPoW(const std::vector<CBlockHeader>& headers, const Consensus::Params& consensusParams, Peer& peer); + /** Calculate an anti-DoS work threshold for headers chains */ + arith_uint256 GetAntiDoSWorkThreshold(); /** Deal with state tracking and headers sync for peers that send the * occasional non-connecting header (this can happen due to BIP 130 headers * announcements for blocks interacting with the 2hr (MAX_FUTURE_BLOCK_TIME) rule). */ void HandleFewUnconnectingHeaders(CNode& pfrom, Peer& peer, const std::vector<CBlockHeader>& headers); /** Return true if the headers connect to each other, false otherwise */ bool CheckHeadersAreContinuous(const std::vector<CBlockHeader>& headers) const; + /** Try to continue a low-work headers sync that has already begun. + * Assumes the caller has already verified the headers connect, and has + * checked that each header satisfies the proof-of-work target included in + * the header. + * @param[in] peer The peer we're syncing with. + * @param[in] pfrom CNode of the peer + * @param[in,out] headers The headers to be processed. + * @return True if the passed in headers were successfully processed + * as the continuation of a low-work headers sync in progress; + * false otherwise. + * If false, the passed in headers will be returned back to + * the caller. + * If true, the returned headers may be empty, indicating + * there is no more work for the caller to do; or the headers + * may be populated with entries that have passed anti-DoS + * checks (and therefore may be validated for block index + * acceptance by the caller). + */ + bool IsContinuationOfLowWorkHeadersSync(Peer& peer, CNode& pfrom, + std::vector<CBlockHeader>& headers) + EXCLUSIVE_LOCKS_REQUIRED(peer.m_headers_sync_mutex, !m_headers_presync_mutex); + /** Check work on a headers chain to be processed, and if insufficient, + * initiate our anti-DoS headers sync mechanism. + * + * @param[in] peer The peer whose headers we're processing. + * @param[in] pfrom CNode of the peer + * @param[in] chain_start_header Where these headers connect in our index. + * @param[in,out] headers The headers to be processed. + * + * @return True if chain was low work and a headers sync was + * initiated (and headers will be empty after calling); false + * otherwise. + */ + bool TryLowWorkHeadersSync(Peer& peer, CNode& pfrom, + const CBlockIndex* chain_start_header, + std::vector<CBlockHeader>& headers) + EXCLUSIVE_LOCKS_REQUIRED(!peer.m_headers_sync_mutex, !m_peer_mutex, !m_headers_presync_mutex); + + /** Return true if the given header is an ancestor of + * m_chainman.m_best_header or our current tip */ + bool IsAncestorOfBestHeaderOrTip(const CBlockIndex* header) EXCLUSIVE_LOCKS_REQUIRED(cs_main); + /** Request further headers from this peer with a given locator. * We don't issue a getheaders message if we have a recent one outstanding. * This returns true if a getheaders is actually sent, and false otherwise. @@ -620,6 +685,9 @@ private: /** Send `addr` messages on a regular schedule. */ void MaybeSendAddr(CNode& node, Peer& peer, std::chrono::microseconds current_time); + /** Send a single `sendheaders` message, after we have completed headers sync with a peer. */ + void MaybeSendSendHeaders(CNode& node, Peer& peer); + /** Relay (gossip) an address to a few randomly chosen nodes. * * @param[in] originator The id of the peer that sent us the address. We don't want to relay it back. @@ -645,7 +713,7 @@ private: std::atomic<int> m_best_height{-1}; /** Next time to check for stale tip */ - std::chrono::seconds m_stale_tip_check_time{0s}; + std::chrono::seconds m_stale_tip_check_time GUARDED_BY(cs_main){0s}; /** Whether this node is running in -blocksonly mode */ const bool m_ignore_incoming_txs; @@ -654,7 +722,7 @@ private: /** Whether we've completed initial sync yet, for determining when to turn * on extra block-relay-only peers. */ - bool m_initial_sync_finished{false}; + bool m_initial_sync_finished GUARDED_BY(cs_main){false}; /** Protects m_peer_map. This mutex must not be locked while holding a lock * on any of the mutexes inside a Peer object. */ @@ -682,6 +750,9 @@ private: /** Number of nodes with fSyncStarted. */ int nSyncStarted GUARDED_BY(cs_main) = 0; + /** Hash of the last block we received via INV */ + uint256 m_last_block_inv_triggering_headers_sync{}; + /** * Sources of received blocks, saved to be able punish them when processing * happens afterwards. @@ -773,6 +844,24 @@ private: std::shared_ptr<const CBlockHeaderAndShortTxIDs> m_most_recent_compact_block GUARDED_BY(m_most_recent_block_mutex); uint256 m_most_recent_block_hash GUARDED_BY(m_most_recent_block_mutex); + // Data about the low-work headers synchronization, aggregated from all peers' HeadersSyncStates. + /** Mutex guarding the other m_headers_presync_* variables. */ + Mutex m_headers_presync_mutex; + /** A type to represent statistics about a peer's low-work headers sync. + * + * - The first field is the total verified amount of work in that synchronization. + * - The second is: + * - nullopt: the sync is in REDOWNLOAD phase (phase 2). + * - {height, timestamp}: the sync has the specified tip height and block timestamp (phase 1). + */ + using HeadersPresyncStats = std::pair<arith_uint256, std::optional<std::pair<int64_t, uint32_t>>>; + /** Statistics for all peers in low-work headers sync. */ + std::map<NodeId, HeadersPresyncStats> m_headers_presync_stats GUARDED_BY(m_headers_presync_mutex) {}; + /** The peer with the most-work entry in m_headers_presync_stats. */ + NodeId m_headers_presync_bestpeer GUARDED_BY(m_headers_presync_mutex) {-1}; + /** The m_headers_presync_stats improved, and needs signalling. */ + std::atomic_bool m_headers_presync_should_signal{false}; + /** Height of the highest block announced using BIP 152 high-bandwidth mode. */ int m_highest_fast_announce{0}; @@ -810,7 +899,7 @@ private: EXCLUSIVE_LOCKS_REQUIRED(!m_most_recent_block_mutex, peer.m_getdata_requests_mutex) LOCKS_EXCLUDED(::cs_main); /** Process a new block. Perform any post-processing housekeeping */ - void ProcessBlock(CNode& node, const std::shared_ptr<const CBlock>& block, bool force_processing); + void ProcessBlock(CNode& node, const std::shared_ptr<const CBlock>& block, bool force_processing, bool min_pow_checked); /** Relay map (txid or wtxid -> CTransactionRef) */ typedef std::map<uint256, CTransactionRef> MapRelay; @@ -1145,7 +1234,7 @@ bool PeerManagerImpl::TipMayBeStale() bool PeerManagerImpl::CanDirectFetch() { - return m_chainman.ActiveChain().Tip()->GetBlockTime() > GetAdjustedTime() - m_chainparams.GetConsensus().nPowTargetSpacing * 20; + return m_chainman.ActiveChain().Tip()->Time() > GetAdjustedTime() - m_chainparams.GetConsensus().PowTargetSpacing() * 20; } static bool PeerHasHeader(CNodeState *state, const CBlockIndex *pindex) EXCLUSIVE_LOCKS_REQUIRED(cs_main) @@ -1431,6 +1520,10 @@ void PeerManagerImpl::FinalizeNode(const CNode& node) // fSuccessfullyConnected set. m_addrman.Connected(node.addr); } + { + LOCK(m_headers_presync_mutex); + m_headers_presync_stats.erase(nodeid); + } LogPrint(BCLog::NET, "Cleared nodestate for peer=%d\n", nodeid); } @@ -1495,6 +1588,12 @@ bool PeerManagerImpl::GetNodeStateStats(NodeId nodeid, CNodeStateStats& stats) c stats.m_addr_processed = peer->m_addr_processed.load(); stats.m_addr_rate_limited = peer->m_addr_rate_limited.load(); stats.m_addr_relay_enabled = peer->m_addr_relay_enabled.load(); + { + LOCK(peer->m_headers_sync_mutex); + if (peer->m_headers_sync) { + stats.presync_height = peer->m_headers_sync->GetPresyncHeight(); + } + } return true; } @@ -1538,6 +1637,10 @@ bool PeerManagerImpl::MaybePunishNodeForBlock(NodeId nodeid, const BlockValidati switch (state.GetResult()) { case BlockValidationResult::BLOCK_RESULT_UNSET: break; + case BlockValidationResult::BLOCK_HEADER_LOW_WORK: + // We didn't try to process the block because the header chain may have + // too little work. + break; // The node is providing invalid data: case BlockValidationResult::BLOCK_CONSENSUS: case BlockValidationResult::BLOCK_MUTATED: @@ -2257,6 +2360,35 @@ void PeerManagerImpl::SendBlockTransactions(CNode& pfrom, Peer& peer, const CBlo m_connman.PushMessage(&pfrom, msgMaker.Make(NetMsgType::BLOCKTXN, resp)); } +bool PeerManagerImpl::CheckHeadersPoW(const std::vector<CBlockHeader>& headers, const Consensus::Params& consensusParams, Peer& peer) +{ + // Do these headers have proof-of-work matching what's claimed? + if (!HasValidProofOfWork(headers, consensusParams)) { + Misbehaving(peer, 100, "header with invalid proof of work"); + return false; + } + + // Are these headers connected to each other? + if (!CheckHeadersAreContinuous(headers)) { + Misbehaving(peer, 20, "non-continuous headers sequence"); + return false; + } + return true; +} + +arith_uint256 PeerManagerImpl::GetAntiDoSWorkThreshold() +{ + arith_uint256 near_chaintip_work = 0; + LOCK(cs_main); + if (m_chainman.ActiveChain().Tip() != nullptr) { + const CBlockIndex *tip = m_chainman.ActiveChain().Tip(); + // Use a 144 block buffer, so that we'll accept headers that fork from + // near our tip. + near_chaintip_work = tip->nChainWork - std::min<arith_uint256>(144*GetBlockProof(*tip), tip->nChainWork); + } + return std::max(near_chaintip_work, arith_uint256(nMinimumChainWork)); +} + /** * Special handling for unconnecting headers that might be part of a block * announcement. @@ -2279,7 +2411,7 @@ void PeerManagerImpl::HandleFewUnconnectingHeaders(CNode& pfrom, Peer& peer, nodestate->nUnconnectingHeaders++; // Try to fill in the missing headers. - if (MaybeSendGetHeaders(pfrom, m_chainman.ActiveChain().GetLocator(m_chainman.m_best_header), peer)) { + if (MaybeSendGetHeaders(pfrom, GetLocator(m_chainman.m_best_header), peer)) { LogPrint(BCLog::NET, "received header %s: missing prev block %s, sending getheaders (%d) to end (peer=%d, nUnconnectingHeaders=%d)\n", headers[0].GetHash().ToString(), headers[0].hashPrevBlock.ToString(), @@ -2310,6 +2442,146 @@ bool PeerManagerImpl::CheckHeadersAreContinuous(const std::vector<CBlockHeader>& return true; } +bool PeerManagerImpl::IsContinuationOfLowWorkHeadersSync(Peer& peer, CNode& pfrom, std::vector<CBlockHeader>& headers) +{ + if (peer.m_headers_sync) { + auto result = peer.m_headers_sync->ProcessNextHeaders(headers, headers.size() == MAX_HEADERS_RESULTS); + if (result.request_more) { + auto locator = peer.m_headers_sync->NextHeadersRequestLocator(); + // If we were instructed to ask for a locator, it should not be empty. + Assume(!locator.vHave.empty()); + if (!locator.vHave.empty()) { + // It should be impossible for the getheaders request to fail, + // because we should have cleared the last getheaders timestamp + // when processing the headers that triggered this call. But + // it may be possible to bypass this via compactblock + // processing, so check the result before logging just to be + // safe. + bool sent_getheaders = MaybeSendGetHeaders(pfrom, locator, peer); + if (sent_getheaders) { + LogPrint(BCLog::NET, "more getheaders (from %s) to peer=%d\n", + locator.vHave.front().ToString(), pfrom.GetId()); + } else { + LogPrint(BCLog::NET, "error sending next getheaders (from %s) to continue sync with peer=%d\n", + locator.vHave.front().ToString(), pfrom.GetId()); + } + } + } + + if (peer.m_headers_sync->GetState() == HeadersSyncState::State::FINAL) { + peer.m_headers_sync.reset(nullptr); + + // Delete this peer's entry in m_headers_presync_stats. + // If this is m_headers_presync_bestpeer, it will be replaced later + // by the next peer that triggers the else{} branch below. + LOCK(m_headers_presync_mutex); + m_headers_presync_stats.erase(pfrom.GetId()); + } else { + // Build statistics for this peer's sync. + HeadersPresyncStats stats; + stats.first = peer.m_headers_sync->GetPresyncWork(); + if (peer.m_headers_sync->GetState() == HeadersSyncState::State::PRESYNC) { + stats.second = {peer.m_headers_sync->GetPresyncHeight(), + peer.m_headers_sync->GetPresyncTime()}; + } + + // Update statistics in stats. + LOCK(m_headers_presync_mutex); + m_headers_presync_stats[pfrom.GetId()] = stats; + auto best_it = m_headers_presync_stats.find(m_headers_presync_bestpeer); + bool best_updated = false; + if (best_it == m_headers_presync_stats.end()) { + // If the cached best peer is outdated, iterate over all remaining ones (including + // newly updated one) to find the best one. + NodeId peer_best{-1}; + const HeadersPresyncStats* stat_best{nullptr}; + for (const auto& [peer, stat] : m_headers_presync_stats) { + if (!stat_best || stat > *stat_best) { + peer_best = peer; + stat_best = &stat; + } + } + m_headers_presync_bestpeer = peer_best; + best_updated = (peer_best == pfrom.GetId()); + } else if (best_it->first == pfrom.GetId() || stats > best_it->second) { + // pfrom was and remains the best peer, or pfrom just became best. + m_headers_presync_bestpeer = pfrom.GetId(); + best_updated = true; + } + if (best_updated && stats.second.has_value()) { + // If the best peer updated, and it is in its first phase, signal. + m_headers_presync_should_signal = true; + } + } + + if (result.success) { + // We only overwrite the headers passed in if processing was + // successful. + headers.swap(result.pow_validated_headers); + } + + return result.success; + } + // Either we didn't have a sync in progress, or something went wrong + // processing these headers, or we are returning headers to the caller to + // process. + return false; +} + +bool PeerManagerImpl::TryLowWorkHeadersSync(Peer& peer, CNode& pfrom, const CBlockIndex* chain_start_header, std::vector<CBlockHeader>& headers) +{ + // Calculate the total work on this chain. + arith_uint256 total_work = chain_start_header->nChainWork + CalculateHeadersWork(headers); + + // Our dynamic anti-DoS threshold (minimum work required on a headers chain + // before we'll store it) + arith_uint256 minimum_chain_work = GetAntiDoSWorkThreshold(); + + // Avoid DoS via low-difficulty-headers by only processing if the headers + // are part of a chain with sufficient work. + if (total_work < minimum_chain_work) { + // Only try to sync with this peer if their headers message was full; + // otherwise they don't have more headers after this so no point in + // trying to sync their too-little-work chain. + if (headers.size() == MAX_HEADERS_RESULTS) { + // Note: we could advance to the last header in this set that is + // known to us, rather than starting at the first header (which we + // may already have); however this is unlikely to matter much since + // ProcessHeadersMessage() already handles the case where all + // headers in a received message are already known and are + // ancestors of m_best_header or chainActive.Tip(), by skipping + // this logic in that case. So even if the first header in this set + // of headers is known, some header in this set must be new, so + // advancing to the first unknown header would be a small effect. + LOCK(peer.m_headers_sync_mutex); + peer.m_headers_sync.reset(new HeadersSyncState(peer.m_id, m_chainparams.GetConsensus(), + chain_start_header, minimum_chain_work)); + + // Now a HeadersSyncState object for tracking this synchronization is created, + // process the headers using it as normal. + return IsContinuationOfLowWorkHeadersSync(peer, pfrom, headers); + } else { + LogPrint(BCLog::NET, "Ignoring low-work chain (height=%u) from peer=%d\n", chain_start_header->nHeight + headers.size(), pfrom.GetId()); + // Since this is a low-work headers chain, no further processing is required. + headers = {}; + return true; + } + } + return false; +} + +bool PeerManagerImpl::IsAncestorOfBestHeaderOrTip(const CBlockIndex* header) +{ + if (header == nullptr) { + return false; + } else if (m_chainman.m_best_header != nullptr && header == m_chainman.m_best_header->GetAncestor(header->nHeight)) { + return true; + } else if (m_chainman.ActiveChain().Contains(header)) { + return true; + } + return false; +} + bool PeerManagerImpl::MaybeSendGetHeaders(CNode& pfrom, const CBlockLocator& locator, Peer& peer) { const CNetMsgMaker msgMaker(pfrom.GetCommonVersion()); @@ -2455,21 +2727,73 @@ void PeerManagerImpl::UpdatePeerStateForReceivedHeaders(CNode& pfrom, } void PeerManagerImpl::ProcessHeadersMessage(CNode& pfrom, Peer& peer, - const std::vector<CBlockHeader>& headers, + std::vector<CBlockHeader>&& headers, bool via_compact_block) { - const CNetMsgMaker msgMaker(pfrom.GetCommonVersion()); size_t nCount = headers.size(); if (nCount == 0) { // Nothing interesting. Stop asking this peers for more headers. + // If we were in the middle of headers sync, receiving an empty headers + // message suggests that the peer suddenly has nothing to give us + // (perhaps it reorged to our chain). Clear download state for this peer. + LOCK(peer.m_headers_sync_mutex); + if (peer.m_headers_sync) { + peer.m_headers_sync.reset(nullptr); + LOCK(m_headers_presync_mutex); + m_headers_presync_stats.erase(pfrom.GetId()); + } + return; + } + + // Before we do any processing, make sure these pass basic sanity checks. + // We'll rely on headers having valid proof-of-work further down, as an + // anti-DoS criteria (note: this check is required before passing any + // headers into HeadersSyncState). + if (!CheckHeadersPoW(headers, m_chainparams.GetConsensus(), peer)) { + // Misbehaving() calls are handled within CheckHeadersPoW(), so we can + // just return. (Note that even if a header is announced via compact + // block, the header itself should be valid, so this type of error can + // always be punished.) return; } const CBlockIndex *pindexLast = nullptr; + // We'll set already_validated_work to true if these headers are + // successfully processed as part of a low-work headers sync in progress + // (either in PRESYNC or REDOWNLOAD phase). + // If true, this will mean that any headers returned to us (ie during + // REDOWNLOAD) can be validated without further anti-DoS checks. + bool already_validated_work = false; + + // If we're in the middle of headers sync, let it do its magic. + bool have_headers_sync = false; + { + LOCK(peer.m_headers_sync_mutex); + + already_validated_work = IsContinuationOfLowWorkHeadersSync(peer, pfrom, headers); + + // The headers we passed in may have been: + // - untouched, perhaps if no headers-sync was in progress, or some + // failure occurred + // - erased, such as if the headers were successfully processed and no + // additional headers processing needs to take place (such as if we + // are still in PRESYNC) + // - replaced with headers that are now ready for validation, such as + // during the REDOWNLOAD phase of a low-work headers sync. + // So just check whether we still have headers that we need to process, + // or not. + if (headers.empty()) { + return; + } + + have_headers_sync = !!peer.m_headers_sync; + } + // Do these headers connect to something in our block index? - bool headers_connect_blockindex{WITH_LOCK(::cs_main, return m_chainman.m_blockman.LookupBlockIndex(headers[0].hashPrevBlock) != nullptr)}; + const CBlockIndex *chain_start_header{WITH_LOCK(::cs_main, return m_chainman.m_blockman.LookupBlockIndex(headers[0].hashPrevBlock))}; + bool headers_connect_blockindex{chain_start_header != nullptr}; if (!headers_connect_blockindex) { if (nCount <= MAX_BLOCKS_TO_ANNOUNCE) { @@ -2483,28 +2807,58 @@ void PeerManagerImpl::ProcessHeadersMessage(CNode& pfrom, Peer& peer, return; } + // If the headers we received are already in memory and an ancestor of + // m_best_header or our tip, skip anti-DoS checks. These headers will not + // use any more memory (and we are not leaking information that could be + // used to fingerprint us). + const CBlockIndex *last_received_header{nullptr}; + { + LOCK(cs_main); + last_received_header = m_chainman.m_blockman.LookupBlockIndex(headers.back().GetHash()); + if (IsAncestorOfBestHeaderOrTip(last_received_header)) { + already_validated_work = true; + } + } + + // If our peer has NetPermissionFlags::NoBan privileges, then bypass our + // anti-DoS logic (this saves bandwidth when we connect to a trusted peer + // on startup). + if (pfrom.HasPermission(NetPermissionFlags::NoBan)) { + already_validated_work = true; + } + // At this point, the headers connect to something in our block index. - if (!CheckHeadersAreContinuous(headers)) { - Misbehaving(peer, 20, "non-continuous headers sequence"); + // Do anti-DoS checks to determine if we should process or store for later + // processing. + if (!already_validated_work && TryLowWorkHeadersSync(peer, pfrom, + chain_start_header, headers)) { + // If we successfully started a low-work headers sync, then there + // should be no headers to process any further. + Assume(headers.empty()); return; } + // At this point, we have a set of headers with sufficient work on them + // which can be processed. + // If we don't have the last header, then this peer will have given us // something new (if these headers are valid). - bool received_new_header{WITH_LOCK(::cs_main, return m_chainman.m_blockman.LookupBlockIndex(headers.back().GetHash()) == nullptr)}; + bool received_new_header{last_received_header != nullptr}; + // Now process all the headers. BlockValidationState state; - if (!m_chainman.ProcessNewBlockHeaders(headers, state, &pindexLast)) { + if (!m_chainman.ProcessNewBlockHeaders(headers, /*min_pow_checked=*/true, state, &pindexLast)) { if (state.IsInvalid()) { MaybePunishNodeForBlock(pfrom.GetId(), state, via_compact_block, "invalid header received"); return; } } + Assume(pindexLast); - // Consider fetching more headers. - if (nCount == MAX_HEADERS_RESULTS) { + // Consider fetching more headers if we are not using our headers-sync mechanism. + if (nCount == MAX_HEADERS_RESULTS && !have_headers_sync) { // Headers message had its maximum size; the peer may have more headers. - if (MaybeSendGetHeaders(pfrom, m_chainman.ActiveChain().GetLocator(pindexLast), peer)) { + if (MaybeSendGetHeaders(pfrom, GetLocator(pindexLast), peer)) { LogPrint(BCLog::NET, "more getheaders (%d) to end to peer=%d (startheight:%d)\n", pindexLast->nHeight, pfrom.GetId(), peer.m_starting_height); } @@ -2765,10 +3119,10 @@ void PeerManagerImpl::ProcessGetCFCheckPt(CNode& node, Peer& peer, CDataStream& m_connman.PushMessage(&node, std::move(msg)); } -void PeerManagerImpl::ProcessBlock(CNode& node, const std::shared_ptr<const CBlock>& block, bool force_processing) +void PeerManagerImpl::ProcessBlock(CNode& node, const std::shared_ptr<const CBlock>& block, bool force_processing, bool min_pow_checked) { bool new_block{false}; - m_chainman.ProcessNewBlock(block, force_processing, &new_block); + m_chainman.ProcessNewBlock(block, force_processing, min_pow_checked, &new_block); if (new_block) { node.m_last_block_time = GetTime<std::chrono::seconds>(); } else { @@ -3026,13 +3380,6 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, pfrom.ConnectionTypeAsString()); } - if (pfrom.GetCommonVersion() >= SENDHEADERS_VERSION) { - // Tell our peer we prefer to receive headers rather than inv's - // We send this to non-NODE NETWORK peers as well, because even - // non-NODE NETWORK peers can announce blocks (such as pruning - // nodes) - m_connman.PushMessage(&pfrom, msgMaker.Make(NetMsgType::SENDHEADERS)); - } if (pfrom.GetCommonVersion() >= SHORT_IDS_BLOCKS_VERSION) { // Tell our peer we are willing to provide version 2 cmpctblocks. // However, we do not request new block announcements using @@ -3240,8 +3587,9 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, UpdateBlockAvailability(pfrom.GetId(), inv.hash); if (!fAlreadyHave && !fImporting && !fReindex && !IsBlockRequested(inv.hash)) { // Headers-first is the primary method of announcement on - // the network. If a node fell back to sending blocks by inv, - // it's probably for a re-org. The final block hash + // the network. If a node fell back to sending blocks by + // inv, it may be for a re-org, or because we haven't + // completed initial headers sync. The final block hash // provided should be the highest, so send a getheaders and // then fetch the blocks we need to catch up. best_block = &inv.hash; @@ -3266,10 +3614,30 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, } if (best_block != nullptr) { - if (MaybeSendGetHeaders(pfrom, m_chainman.ActiveChain().GetLocator(m_chainman.m_best_header), *peer)) { - LogPrint(BCLog::NET, "getheaders (%d) %s to peer=%d\n", - m_chainman.m_best_header->nHeight, best_block->ToString(), - pfrom.GetId()); + // If we haven't started initial headers-sync with this peer, then + // consider sending a getheaders now. On initial startup, there's a + // reliability vs bandwidth tradeoff, where we are only trying to do + // initial headers sync with one peer at a time, with a long + // timeout (at which point, if the sync hasn't completed, we will + // disconnect the peer and then choose another). In the meantime, + // as new blocks are found, we are willing to add one new peer per + // block to sync with as well, to sync quicker in the case where + // our initial peer is unresponsive (but less bandwidth than we'd + // use if we turned on sync with all peers). + CNodeState& state{*Assert(State(pfrom.GetId()))}; + if (state.fSyncStarted || (!peer->m_inv_triggered_getheaders_before_sync && *best_block != m_last_block_inv_triggering_headers_sync)) { + if (MaybeSendGetHeaders(pfrom, GetLocator(m_chainman.m_best_header), *peer)) { + LogPrint(BCLog::NET, "getheaders (%d) %s to peer=%d\n", + m_chainman.m_best_header->nHeight, best_block->ToString(), + pfrom.GetId()); + } + if (!state.fSyncStarted) { + peer->m_inv_triggered_getheaders_before_sync = true; + // Update the last block hash that triggered a new headers + // sync, so that we don't turn on headers sync with more + // than 1 new peer every new block. + m_last_block_inv_triggering_headers_sync = *best_block; + } } } @@ -3722,12 +4090,17 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, { LOCK(cs_main); - if (!m_chainman.m_blockman.LookupBlockIndex(cmpctblock.header.hashPrevBlock)) { + const CBlockIndex* prev_block = m_chainman.m_blockman.LookupBlockIndex(cmpctblock.header.hashPrevBlock); + if (!prev_block) { // Doesn't connect (or is genesis), instead of DoSing in AcceptBlockHeader, request deeper headers if (!m_chainman.ActiveChainstate().IsInitialBlockDownload()) { - MaybeSendGetHeaders(pfrom, m_chainman.ActiveChain().GetLocator(m_chainman.m_best_header), *peer); + MaybeSendGetHeaders(pfrom, GetLocator(m_chainman.m_best_header), *peer); } return; + } else if (prev_block->nChainWork + CalculateHeadersWork({cmpctblock.header}) < GetAntiDoSWorkThreshold()) { + // If we get a low-work header in a compact block, we can ignore it. + LogPrint(BCLog::NET, "Ignoring low-work compact block from peer %d\n", pfrom.GetId()); + return; } if (!m_chainman.m_blockman.LookupBlockIndex(cmpctblock.header.GetHash())) { @@ -3737,7 +4110,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, const CBlockIndex *pindex = nullptr; BlockValidationState state; - if (!m_chainman.ProcessNewBlockHeaders({cmpctblock.header}, state, &pindex)) { + if (!m_chainman.ProcessNewBlockHeaders({cmpctblock.header}, /*min_pow_checked=*/true, state, &pindex)) { if (state.IsInvalid()) { MaybePunishNodeForBlock(pfrom.GetId(), state, /*via_compact_block=*/true, "invalid header via cmpctblock"); return; @@ -3904,7 +4277,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, // we have a chain with at least nMinimumChainWork), and we ignore // compact blocks with less work than our tip, it is safe to treat // reconstructed compact blocks as having been requested. - ProcessBlock(pfrom, pblock, /*force_processing=*/true); + ProcessBlock(pfrom, pblock, /*force_processing=*/true, /*min_pow_checked=*/true); LOCK(cs_main); // hold cs_main for CBlockIndex::IsValid() if (pindex->IsValid(BLOCK_VALID_TRANSACTIONS)) { // Clear download state for this block, which is in @@ -3987,7 +4360,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, // disk-space attacks), but this should be safe due to the // protections in the compact block handler -- see related comment // in compact block optimistic reconstruction handling. - ProcessBlock(pfrom, pblock, /*force_processing=*/true); + ProcessBlock(pfrom, pblock, /*force_processing=*/true, /*min_pow_checked=*/true); } return; } @@ -4018,7 +4391,23 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, ReadCompactSize(vRecv); // ignore tx count; assume it is 0. } - return ProcessHeadersMessage(pfrom, *peer, headers, /*via_compact_block=*/false); + ProcessHeadersMessage(pfrom, *peer, std::move(headers), /*via_compact_block=*/false); + + // Check if the headers presync progress needs to be reported to validation. + // This needs to be done without holding the m_headers_presync_mutex lock. + if (m_headers_presync_should_signal.exchange(false)) { + HeadersPresyncStats stats; + { + LOCK(m_headers_presync_mutex); + auto it = m_headers_presync_stats.find(m_headers_presync_bestpeer); + if (it != m_headers_presync_stats.end()) stats = it->second; + } + if (stats.second) { + m_chainman.ReportHeadersPresync(stats.first, stats.second->first, stats.second->second); + } + } + + return; } if (msg_type == NetMsgType::BLOCK) @@ -4036,6 +4425,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, bool forceProcessing = false; const uint256 hash(pblock->GetHash()); + bool min_pow_checked = false; { LOCK(cs_main); // Always process the block if we requested it, since we may @@ -4046,8 +4436,14 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type, // which peers send us compact blocks, so the race between here and // cs_main in ProcessNewBlock is fine. mapBlockSource.emplace(hash, std::make_pair(pfrom.GetId(), true)); + + // Check work on this block against our anti-dos thresholds. + const CBlockIndex* prev_block = m_chainman.m_blockman.LookupBlockIndex(pblock->hashPrevBlock); + if (prev_block && prev_block->nChainWork + CalculateHeadersWork({pblock->GetBlockHeader()}) >= GetAntiDoSWorkThreshold()) { + min_pow_checked = true; + } } - ProcessBlock(pfrom, pblock, forceProcessing); + ProcessBlock(pfrom, pblock, forceProcessing, min_pow_checked); return; } @@ -4475,7 +4871,7 @@ void PeerManagerImpl::ConsiderEviction(CNode& pto, Peer& peer, std::chrono::seco // getheaders in-flight already, in which case the peer should // still respond to us with a sufficiently high work chain tip. MaybeSendGetHeaders(pto, - m_chainman.ActiveChain().GetLocator(state.m_chain_sync.m_work_header->pprev), + GetLocator(state.m_chain_sync.m_work_header->pprev), peer); LogPrint(BCLog::NET, "sending getheaders to outbound peer=%d to verify chain work (current best known block:%s, benchmark blockhash: %s)\n", pto.GetId(), state.pindexBestKnownBlock != nullptr ? state.pindexBestKnownBlock->GetBlockHash().ToString() : "<none>", state.m_chain_sync.m_work_header->GetBlockHash().ToString()); state.m_chain_sync.m_sent_getheaders = true; @@ -4732,6 +5128,27 @@ void PeerManagerImpl::MaybeSendAddr(CNode& node, Peer& peer, std::chrono::micros } } +void PeerManagerImpl::MaybeSendSendHeaders(CNode& node, Peer& peer) +{ + // Delay sending SENDHEADERS (BIP 130) until we're done with an + // initial-headers-sync with this peer. Receiving headers announcements for + // new blocks while trying to sync their headers chain is problematic, + // because of the state tracking done. + if (!peer.m_sent_sendheaders && node.GetCommonVersion() >= SENDHEADERS_VERSION) { + LOCK(cs_main); + CNodeState &state = *State(node.GetId()); + if (state.pindexBestKnownBlock != nullptr && + state.pindexBestKnownBlock->nChainWork > nMinimumChainWork) { + // Tell our peer we prefer to receive headers rather than inv's + // We send this to non-NODE NETWORK peers as well, because even + // non-NODE NETWORK peers can announce blocks (such as pruning + // nodes) + m_connman.PushMessage(&node, CNetMsgMaker(node.GetCommonVersion()).Make(NetMsgType::SENDHEADERS)); + peer.m_sent_sendheaders = true; + } + } +} + void PeerManagerImpl::MaybeSendFeefilter(CNode& pto, Peer& peer, std::chrono::microseconds current_time) { if (m_ignore_incoming_txs) return; @@ -4853,6 +5270,8 @@ bool PeerManagerImpl::SendMessages(CNode* pto) MaybeSendAddr(*pto, *peer, current_time); + MaybeSendSendHeaders(*pto, *peer); + { LOCK(cs_main); @@ -4886,7 +5305,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto) if (!state.fSyncStarted && CanServeBlocks(*peer) && !fImporting && !fReindex) { // Only actively request headers from a single peer, unless we're close to today. - if ((nSyncStarted == 0 && sync_blocks_and_headers_from_peer) || m_chainman.m_best_header->GetBlockTime() > GetAdjustedTime() - 24 * 60 * 60) { + if ((nSyncStarted == 0 && sync_blocks_and_headers_from_peer) || m_chainman.m_best_header->Time() > GetAdjustedTime() - 24h) { const CBlockIndex* pindexStart = m_chainman.m_best_header; /* If possible, start at the block preceding the currently best known header. This ensures that we always get a @@ -4897,7 +5316,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto) got back an empty response. */ if (pindexStart->pprev) pindexStart = pindexStart->pprev; - if (MaybeSendGetHeaders(*pto, m_chainman.ActiveChain().GetLocator(pindexStart), *peer)) { + if (MaybeSendGetHeaders(*pto, GetLocator(pindexStart), *peer)) { LogPrint(BCLog::NET, "initial getheaders (%d) to peer=%d (startheight:%d)\n", pindexStart->nHeight, pto->GetId(), peer->m_starting_height); state.fSyncStarted = true; @@ -4906,7 +5325,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto) // Convert HEADERS_DOWNLOAD_TIMEOUT_PER_HEADER to microseconds before scaling // to maintain precision std::chrono::microseconds{HEADERS_DOWNLOAD_TIMEOUT_PER_HEADER} * - (GetAdjustedTime() - m_chainman.m_best_header->GetBlockTime()) / consensusParams.nPowTargetSpacing + Ticks<std::chrono::seconds>(GetAdjustedTime() - m_chainman.m_best_header->Time()) / consensusParams.nPowTargetSpacing ); nSyncStarted++; } @@ -5223,7 +5642,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto) // Check for headers sync timeouts if (state.fSyncStarted && state.m_headers_sync_timeout < std::chrono::microseconds::max()) { // Detect whether this is a stalling initial-headers-sync peer - if (m_chainman.m_best_header->GetBlockTime() <= GetAdjustedTime() - 24 * 60 * 60) { + if (m_chainman.m_best_header->Time() <= GetAdjustedTime() - 24h) { if (current_time > state.m_headers_sync_timeout && nSyncStarted == 1 && (m_num_preferred_download_peers - state.fPreferredDownload >= 1)) { // Disconnect a peer (without NetPermissionFlags::NoBan permission) if it is our only sync peer, // and we have others we could be using instead. |