aboutsummaryrefslogtreecommitdiff
path: root/src/net_processing.cpp
diff options
context:
space:
mode:
authorfanquake <fanquake@gmail.com>2022-08-30 15:34:10 +0100
committerfanquake <fanquake@gmail.com>2022-08-30 15:37:59 +0100
commite9035f867a36a430998e3811385958229ac79cf5 (patch)
treeccd1df3e2a12e7ed0dd143e59cc17bf7d7d71331 /src/net_processing.cpp
parentcfda740b332c77800f9bb2506d840dad3f4938c0 (diff)
parent3add23454624c4c79c9eebc060b6fbed4e3131a7 (diff)
downloadbitcoin-e9035f867a36a430998e3811385958229ac79cf5.tar.xz
Merge bitcoin/bitcoin#25717: p2p: Implement anti-DoS headers sync
3add23454624c4c79c9eebc060b6fbed4e3131a7 ui: show header pre-synchronization progress (Pieter Wuille) 738421c50f2dbd7395b50a5dbdf6168b07435e62 Emit NotifyHeaderTip signals for pre-synchronization progress (Pieter Wuille) 376086fc5a187f5b2ab3a0d1202ed4e6c22bdb50 Make validation interface capable of signalling header presync (Pieter Wuille) 93eae27031a65b4156df49015ae45b2b541b4e5a Test large reorgs with headerssync logic (Suhas Daftuar) 355547334f7d08640ee1fa291227356d61145d1a Track headers presync progress and log it (Pieter Wuille) 03712dddfbb9fe0dc7a2ead53c65106189f5c803 Expose HeadersSyncState::m_current_height in getpeerinfo() (Suhas Daftuar) 150a5486db50ff77c91765392149000029c8a309 Test headers sync using minchainwork threshold (Suhas Daftuar) 0b6aa826b53470c9cc8ef4a153fa710dce80882f Add unit test for HeadersSyncState (Suhas Daftuar) 83c6a0c5249c4ecbd11f7828c84a50fb473faba3 Reduce spurious messages during headers sync (Suhas Daftuar) ed6cddd98e32263fc116a4380af6d66da20da990 Require callers of AcceptBlockHeader() to perform anti-dos checks (Suhas Daftuar) 551a8d957c4c44afbd0d608fcdf7c6a4352babce Utilize anti-DoS headers download strategy (Suhas Daftuar) ed470940cddbeb40425960d51cefeec4948febe4 Add functions to construct locators without CChain (Pieter Wuille) 84852bb6bb3579e475ce78fe729fd125ddbc715f Add bitdeque, an std::deque<bool> analogue that does bit packing. (Pieter Wuille) 1d4cfa4272cf2c8b980cc8762c1ff2220d3e8d51 Add function to validate difficulty changes (Suhas Daftuar) Pull request description: New nodes starting up for the first time lack protection against DoS from low-difficulty headers. While checkpoints serve as our protection against headers that fork from the main chain below the known checkpointed values, this protection only applies to nodes that have been able to download the honest chain to the checkpointed heights. We can protect all nodes from DoS from low-difficulty headers by adopting a different strategy: before we commit to storing a header in permanent storage, first verify that the header is part of a chain that has sufficiently high work (either `nMinimumChainWork`, or something comparable to our tip). This means that we will download headers from a given peer twice: once to verify the work on the chain, and a second time when permanently storing the headers. The p2p protocol doesn't provide an easy way for us to ensure that we receive the same headers during the second download of peer's headers chain. To ensure that a peer doesn't (say) give us the main chain in phase 1 to trick us into permanently storing an alternate, low-work chain in phase 2, we store commitments to the headers during our first download, which we validate in the second download. Some parameters must be chosen for commitment size/frequency in phase 1, and validation of commitments in phase 2. In this PR, those parameters are chosen to both (a) minimize the per-peer memory usage that an attacker could utilize, and (b) bound the expected amount of permanent memory that an attacker could get us to use to be well-below the memory growth that we'd get from the honest chain (where we expect 1 new block header every 10 minutes). After this PR, we should be able to remove checkpoints from our code, which is a nice philosophical change for us to make as well, as there has been confusion over the years about the role checkpoints play in Bitcoin's consensus algorithm. Thanks to Pieter Wuille for collaborating on this design. ACKs for top commit: Sjors: re-tACK 3add23454624c4c79c9eebc060b6fbed4e3131a7 mzumsande: re-ACK 3add23454624c4c79c9eebc060b6fbed4e3131a7 sipa: re-ACK 3add23454624c4c79c9eebc060b6fbed4e3131a7 glozow: ACK 3add234546 Tree-SHA512: e7789d65f62f72141b8899eb4a2fb3d0621278394d2d7adaa004675250118f89a4e4cb42777fe56649d744ec445ad95141e10f6def65f0a58b7b35b2e654a875
Diffstat (limited to 'src/net_processing.cpp')
-rw-r--r--src/net_processing.cpp459
1 files changed, 422 insertions, 37 deletions
diff --git a/src/net_processing.cpp b/src/net_processing.cpp
index fff7f86d78..746e3bedde 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>
@@ -381,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}
@@ -503,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);
@@ -522,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:
@@ -581,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.
@@ -623,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.
@@ -779,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};
@@ -816,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;
@@ -1437,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);
}
@@ -1501,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;
}
@@ -1544,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:
@@ -2263,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.
@@ -2285,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(),
@@ -2316,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());
@@ -2461,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) {
@@ -2489,28 +2807,51 @@ 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;
+ }
+ }
+
// 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, WITH_LOCK(m_chainman.GetMutex(), return 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);
}
@@ -2771,10 +3112,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 {
@@ -3032,13 +3373,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
@@ -3285,7 +3619,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
// 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, m_chainman.ActiveChain().GetLocator(m_chainman.m_best_header), *peer)) {
+ 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());
@@ -3749,12 +4083,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())) {
@@ -3764,7 +4103,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;
@@ -3931,7 +4270,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
@@ -4014,7 +4353,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;
}
@@ -4045,7 +4384,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)
@@ -4063,6 +4418,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
@@ -4073,8 +4429,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;
}
@@ -4502,7 +4864,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;
@@ -4759,6 +5121,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;
@@ -4880,6 +5263,8 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
MaybeSendAddr(*pto, *peer, current_time);
+ MaybeSendSendHeaders(*pto, *peer);
+
{
LOCK(cs_main);
@@ -4924,7 +5309,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;