0
0
Fork 0
mirror of https://github.com/bitcoin/bitcoin.git synced 2025-02-02 09:46:52 -05:00

Utilize anti-DoS headers download strategy

Avoid permanently storing headers from a peer, unless the headers are part of a
chain with sufficiently high work. This prevents memory attacks using low-work
headers.

Designed and co-authored with Pieter Wuille.
This commit is contained in:
Suhas Daftuar 2022-02-09 09:38:52 -05:00
parent ed470940cd
commit 551a8d957c
9 changed files with 884 additions and 12 deletions

View file

@ -151,6 +151,7 @@ BITCOIN_CORE_H = \
external_signer.h \
flatfile.h \
fs.h \
headerssync.h \
httprpc.h \
httpserver.h \
i2p.h \
@ -361,6 +362,7 @@ libbitcoin_node_a_SOURCES = \
dbwrapper.cpp \
deploymentstatus.cpp \
flatfile.cpp \
headerssync.cpp \
httprpc.cpp \
httpserver.cpp \
i2p.cpp \

317
src/headerssync.cpp Normal file
View file

@ -0,0 +1,317 @@
// Copyright (c) 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 <headerssync.h>
#include <logging.h>
#include <pow.h>
#include <timedata.h>
#include <util/check.h>
// The two constants below are computed using the simulation script on
// https://gist.github.com/sipa/016ae445c132cdf65a2791534dfb7ae1
//! Store a commitment to a header every HEADER_COMMITMENT_PERIOD blocks.
constexpr size_t HEADER_COMMITMENT_PERIOD{584};
//! Only feed headers to validation once this many headers on top have been
//! received and validated against commitments.
constexpr size_t REDOWNLOAD_BUFFER_SIZE{13959}; // 13959/584 = ~23.9 commitments
// Our memory analysis assumes 48 bytes for a CompressedHeader (so we should
// re-calculate parameters if we compress further)
static_assert(sizeof(CompressedHeader) == 48);
HeadersSyncState::HeadersSyncState(NodeId id, const Consensus::Params& consensus_params,
const CBlockIndex* chain_start, const arith_uint256& minimum_required_work) :
m_id(id), m_consensus_params(consensus_params),
m_chain_start(chain_start),
m_minimum_required_work(minimum_required_work),
m_current_chain_work(chain_start->nChainWork),
m_commit_offset(GetRand<unsigned>(HEADER_COMMITMENT_PERIOD)),
m_last_header_received(m_chain_start->GetBlockHeader()),
m_current_height(chain_start->nHeight)
{
// Estimate the number of blocks that could possibly exist on the peer's
// chain *right now* using 6 blocks/second (fastest blockrate given the MTP
// rule) times the number of seconds from the last allowed block until
// today. This serves as a memory bound on how many commitments we might
// store from this peer, and we can safely give up syncing if the peer
// exceeds this bound, because it's not possible for a consensus-valid
// chain to be longer than this (at the current time -- in the future we
// could try again, if necessary, to sync a longer chain).
m_max_commitments = 6*(Ticks<std::chrono::seconds>(GetAdjustedTime() - NodeSeconds{std::chrono::seconds{chain_start->GetMedianTimePast()}}) + MAX_FUTURE_BLOCK_TIME) / HEADER_COMMITMENT_PERIOD;
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync started with peer=%d: height=%i, max_commitments=%i, min_work=%s\n", m_id, m_current_height, m_max_commitments, m_minimum_required_work.ToString());
}
/** Free any memory in use, and mark this object as no longer usable. This is
* required to guarantee that we won't reuse this object with the same
* SaltedTxidHasher for another sync. */
void HeadersSyncState::Finalize()
{
Assume(m_download_state != State::FINAL);
m_header_commitments = {};
m_last_header_received.SetNull();
m_redownloaded_headers = {};
m_redownload_buffer_last_hash.SetNull();
m_redownload_buffer_first_prev_hash.SetNull();
m_process_all_remaining_headers = false;
m_current_height = 0;
m_download_state = State::FINAL;
}
/** Process the next batch of headers received from our peer.
* Validate and store commitments, and compare total chainwork to our target to
* see if we can switch to REDOWNLOAD mode. */
HeadersSyncState::ProcessingResult HeadersSyncState::ProcessNextHeaders(const
std::vector<CBlockHeader>& received_headers, const bool full_headers_message)
{
ProcessingResult ret;
Assume(!received_headers.empty());
if (received_headers.empty()) return ret;
Assume(m_download_state != State::FINAL);
if (m_download_state == State::FINAL) return ret;
if (m_download_state == State::PRESYNC) {
// During PRESYNC, we minimally validate block headers and
// occasionally add commitments to them, until we reach our work
// threshold (at which point m_download_state is updated to REDOWNLOAD).
ret.success = ValidateAndStoreHeadersCommitments(received_headers);
if (ret.success) {
if (full_headers_message || m_download_state == State::REDOWNLOAD) {
// A full headers message means the peer may have more to give us;
// also if we just switched to REDOWNLOAD then we need to re-request
// headers from the beginning.
ret.request_more = true;
} else {
Assume(m_download_state == State::PRESYNC);
// If we're in PRESYNC and we get a non-full headers
// message, then the peer's chain has ended and definitely doesn't
// have enough work, so we can stop our sync.
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: incomplete headers message at height=%i (presync phase)\n", m_id, m_current_height);
}
}
} else if (m_download_state == State::REDOWNLOAD) {
// During REDOWNLOAD, we compare our stored commitments to what we
// receive, and add headers to our redownload buffer. When the buffer
// gets big enough (meaning that we've checked enough commitments),
// we'll return a batch of headers to the caller for processing.
ret.success = true;
for (const auto& hdr : received_headers) {
if (!ValidateAndStoreRedownloadedHeader(hdr)) {
// Something went wrong -- the peer gave us an unexpected chain.
// We could consider looking at the reason for failure and
// punishing the peer, but for now just give up on sync.
ret.success = false;
break;
}
}
if (ret.success) {
// Return any headers that are ready for acceptance.
ret.pow_validated_headers = PopHeadersReadyForAcceptance();
// If we hit our target blockhash, then all remaining headers will be
// returned and we can clear any leftover internal state.
if (m_redownloaded_headers.empty() && m_process_all_remaining_headers) {
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync complete with peer=%d: releasing all at height=%i (redownload phase)\n", m_id, m_redownload_buffer_last_height);
} else if (full_headers_message) {
// If the headers message is full, we need to request more.
ret.request_more = true;
} else {
// For some reason our peer gave us a high-work chain, but is now
// declining to serve us that full chain again. Give up.
// Note that there's no more processing to be done with these
// headers, so we can still return success.
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: incomplete headers message at height=%i (redownload phase)\n", m_id, m_redownload_buffer_last_height);
}
}
}
if (!(ret.success && ret.request_more)) Finalize();
return ret;
}
bool HeadersSyncState::ValidateAndStoreHeadersCommitments(const std::vector<CBlockHeader>& headers)
{
// The caller should not give us an empty set of headers.
Assume(headers.size() > 0);
if (headers.size() == 0) return true;
Assume(m_download_state == State::PRESYNC);
if (m_download_state != State::PRESYNC) return false;
if (headers[0].hashPrevBlock != m_last_header_received.GetHash()) {
// Somehow our peer gave us a header that doesn't connect.
// This might be benign -- perhaps our peer reorged away from the chain
// they were on. Give up on this sync for now (likely we will start a
// new sync with a new starting point).
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: non-continuous headers at height=%i (presync phase)\n", m_id, m_current_height);
return false;
}
// If it does connect, (minimally) validate and occasionally store
// commitments.
for (const auto& hdr : headers) {
if (!ValidateAndProcessSingleHeader(hdr)) {
return false;
}
}
if (m_current_chain_work >= m_minimum_required_work) {
m_redownloaded_headers.clear();
m_redownload_buffer_last_height = m_chain_start->nHeight;
m_redownload_buffer_first_prev_hash = m_chain_start->GetBlockHash();
m_redownload_buffer_last_hash = m_chain_start->GetBlockHash();
m_redownload_chain_work = m_chain_start->nChainWork;
m_download_state = State::REDOWNLOAD;
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync transition with peer=%d: reached sufficient work at height=%i, redownloading from height=%i\n", m_id, m_current_height, m_redownload_buffer_last_height);
}
return true;
}
bool HeadersSyncState::ValidateAndProcessSingleHeader(const CBlockHeader& current)
{
Assume(m_download_state == State::PRESYNC);
if (m_download_state != State::PRESYNC) return false;
int next_height = m_current_height + 1;
// Verify that the difficulty isn't growing too fast; an adversary with
// limited hashing capability has a greater chance of producing a high
// work chain if they compress the work into as few blocks as possible,
// so don't let anyone give a chain that would violate the difficulty
// adjustment maximum.
if (!PermittedDifficultyTransition(m_consensus_params, next_height,
m_last_header_received.nBits, current.nBits)) {
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: invalid difficulty transition at height=%i (presync phase)\n", m_id, next_height);
return false;
}
if (next_height % HEADER_COMMITMENT_PERIOD == m_commit_offset) {
// Add a commitment.
m_header_commitments.push_back(m_hasher(current.GetHash()) & 1);
if (m_header_commitments.size() > m_max_commitments) {
// The peer's chain is too long; give up.
// It's possible the chain grew since we started the sync; so
// potentially we could succeed in syncing the peer's chain if we
// try again later.
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: exceeded max commitments at height=%i (presync phase)\n", m_id, next_height);
return false;
}
}
m_current_chain_work += GetBlockProof(CBlockIndex(current));
m_last_header_received = current;
m_current_height = next_height;
return true;
}
bool HeadersSyncState::ValidateAndStoreRedownloadedHeader(const CBlockHeader& header)
{
Assume(m_download_state == State::REDOWNLOAD);
if (m_download_state != State::REDOWNLOAD) return false;
int64_t next_height = m_redownload_buffer_last_height + 1;
// Ensure that we're working on a header that connects to the chain we're
// downloading.
if (header.hashPrevBlock != m_redownload_buffer_last_hash) {
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: non-continuous headers at height=%i (redownload phase)\n", m_id, next_height);
return false;
}
// Check that the difficulty adjustments are within our tolerance:
uint32_t previous_nBits{0};
if (!m_redownloaded_headers.empty()) {
previous_nBits = m_redownloaded_headers.back().nBits;
} else {
previous_nBits = m_chain_start->nBits;
}
if (!PermittedDifficultyTransition(m_consensus_params, next_height,
previous_nBits, header.nBits)) {
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: invalid difficulty transition at height=%i (redownload phase)\n", m_id, next_height);
return false;
}
// Track work on the redownloaded chain
m_redownload_chain_work += GetBlockProof(CBlockIndex(header));
if (m_redownload_chain_work >= m_minimum_required_work) {
m_process_all_remaining_headers = true;
}
// If we're at a header for which we previously stored a commitment, verify
// it is correct. Failure will result in aborting download.
// Also, don't check commitments once we've gotten to our target blockhash;
// it's possible our peer has extended its chain between our first sync and
// our second, and we don't want to return failure after we've seen our
// target blockhash just because we ran out of commitments.
if (!m_process_all_remaining_headers && next_height % HEADER_COMMITMENT_PERIOD == m_commit_offset) {
if (m_header_commitments.size() == 0) {
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: commitment overrun at height=%i (redownload phase)\n", m_id, next_height);
// Somehow our peer managed to feed us a different chain and
// we've run out of commitments.
return false;
}
bool commitment = m_hasher(header.GetHash()) & 1;
bool expected_commitment = m_header_commitments.front();
m_header_commitments.pop_front();
if (commitment != expected_commitment) {
LogPrint(BCLog::HEADERSSYNC, "Initial headers sync aborted with peer=%d: commitment mismatch at height=%i (redownload phase)\n", m_id, next_height);
return false;
}
}
// Store this header for later processing.
m_redownloaded_headers.push_back(header);
m_redownload_buffer_last_height = next_height;
m_redownload_buffer_last_hash = header.GetHash();
return true;
}
std::vector<CBlockHeader> HeadersSyncState::PopHeadersReadyForAcceptance()
{
std::vector<CBlockHeader> ret;
Assume(m_download_state == State::REDOWNLOAD);
if (m_download_state != State::REDOWNLOAD) return ret;
while (m_redownloaded_headers.size() > REDOWNLOAD_BUFFER_SIZE ||
(m_redownloaded_headers.size() > 0 && m_process_all_remaining_headers)) {
ret.emplace_back(m_redownloaded_headers.front().GetFullHeader(m_redownload_buffer_first_prev_hash));
m_redownloaded_headers.pop_front();
m_redownload_buffer_first_prev_hash = ret.back().GetHash();
}
return ret;
}
CBlockLocator HeadersSyncState::NextHeadersRequestLocator() const
{
Assume(m_download_state != State::FINAL);
if (m_download_state == State::FINAL) return {};
auto chain_start_locator = LocatorEntries(m_chain_start);
std::vector<uint256> locator;
if (m_download_state == State::PRESYNC) {
// During pre-synchronization, we continue from the last header received.
locator.push_back(m_last_header_received.GetHash());
}
if (m_download_state == State::REDOWNLOAD) {
// During redownload, we will download from the last received header that we stored.
locator.push_back(m_redownload_buffer_last_hash);
}
locator.insert(locator.end(), chain_start_locator.begin(), chain_start_locator.end());
return CBlockLocator{std::move(locator)};
}

268
src/headerssync.h Normal file
View file

@ -0,0 +1,268 @@
// Copyright (c) 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.
#ifndef BITCOIN_HEADERSSYNC_H
#define BITCOIN_HEADERSSYNC_H
#include <arith_uint256.h>
#include <chain.h>
#include <consensus/params.h>
#include <net.h> // For NodeId
#include <primitives/block.h>
#include <uint256.h>
#include <util/bitdeque.h>
#include <util/hasher.h>
#include <deque>
#include <vector>
// A compressed CBlockHeader, which leaves out the prevhash
struct CompressedHeader {
// header
int32_t nVersion{0};
uint256 hashMerkleRoot;
uint32_t nTime{0};
uint32_t nBits{0};
uint32_t nNonce{0};
CompressedHeader()
{
hashMerkleRoot.SetNull();
}
CompressedHeader(const CBlockHeader& header)
{
nVersion = header.nVersion;
hashMerkleRoot = header.hashMerkleRoot;
nTime = header.nTime;
nBits = header.nBits;
nNonce = header.nNonce;
}
CBlockHeader GetFullHeader(const uint256& hash_prev_block) {
CBlockHeader ret;
ret.nVersion = nVersion;
ret.hashPrevBlock = hash_prev_block;
ret.hashMerkleRoot = hashMerkleRoot;
ret.nTime = nTime;
ret.nBits = nBits;
ret.nNonce = nNonce;
return ret;
};
};
/** HeadersSyncState:
*
* We wish to download a peer's headers chain in a DoS-resistant way.
*
* The Bitcoin protocol does not offer an easy way to determine the work on a
* peer's chain. Currently, we can query a peer's headers by using a GETHEADERS
* message, and our peer can return a set of up to 2000 headers that connect to
* something we know. If a peer's chain has more than 2000 blocks, then we need
* a way to verify that the chain actually has enough work on it to be useful to
* us -- by being above our anti-DoS minimum-chain-work threshold -- before we
* commit to storing those headers in memory. Otherwise, it would be cheap for
* an attacker to waste all our memory by serving us low-work headers
* (particularly for a new node coming online for the first time).
*
* To prevent memory-DoS with low-work headers, while still always being
* able to reorg to whatever the most-work chain is, we require that a chain
* meet a work threshold before committing it to memory. We can do this by
* downloading a peer's headers twice, whenever we are not sure that the chain
* has sufficient work:
*
* - In the first download phase, called pre-synchronization, we can calculate
* the work on the chain as we go (just by checking the nBits value on each
* header, and validating the proof-of-work).
*
* - Once we have reached a header where the cumulative chain work is
* sufficient, we switch to downloading the headers a second time, this time
* processing them fully, and possibly storing them in memory.
*
* To prevent an attacker from using (eg) the honest chain to convince us that
* they have a high-work chain, but then feeding us an alternate set of
* low-difficulty headers in the second phase, we store commitments to the
* chain we see in the first download phase that we check in the second phase,
* as follows:
*
* - In phase 1 (presync), store 1 bit (using a salted hash function) for every
* N headers that we see. With a reasonable choice of N, this uses relatively
* little memory even for a very long chain.
*
* - In phase 2 (redownload), keep a lookahead buffer and only accept headers
* from that buffer into the block index (permanent memory usage) once they
* have some target number of verified commitments on top of them. With this
* parametrization, we can achieve a given security target for potential
* permanent memory usage, while choosing N to minimize memory use during the
* sync (temporary, per-peer storage).
*/
class HeadersSyncState {
public:
~HeadersSyncState() {}
enum class State {
/** PRESYNC means the peer has not yet demonstrated their chain has
* sufficient work and we're only building commitments to the chain they
* serve us. */
PRESYNC,
/** REDOWNLOAD means the peer has given us a high-enough-work chain,
* and now we're redownloading the headers we saw before and trying to
* accept them */
REDOWNLOAD,
/** We're done syncing with this peer and can discard any remaining state */
FINAL
};
/** Return the current state of our download */
State GetState() const { return m_download_state; }
/** Construct a HeadersSyncState object representing a headers sync via this
* download-twice mechanism).
*
* id: node id (for logging)
* consensus_params: parameters needed for difficulty adjustment validation
* chain_start: best known fork point that the peer's headers branch from
* minimum_required_work: amount of chain work required to accept the chain
*/
HeadersSyncState(NodeId id, const Consensus::Params& consensus_params,
const CBlockIndex* chain_start, const arith_uint256& minimum_required_work);
/** Result data structure for ProcessNextHeaders. */
struct ProcessingResult {
std::vector<CBlockHeader> pow_validated_headers;
bool success{false};
bool request_more{false};
};
/** Process a batch of headers, once a sync via this mechanism has started
*
* received_headers: headers that were received over the network for processing.
* Assumes the caller has already verified the headers
* are continuous, and has checked that each header
* satisfies the proof-of-work target included in the
* header (but not necessarily verified that the
* proof-of-work target is correct and passes consensus
* rules).
* full_headers_message: true if the message was at max capacity,
* indicating more headers may be available
* ProcessingResult.pow_validated_headers: will be filled in with any
* headers that the caller can fully process and
* validate now (because these returned headers are
* on a chain with sufficient work)
* ProcessingResult.success: set to false if an error is detected and the sync is
* aborted; true otherwise.
* ProcessingResult.request_more: if true, the caller is suggested to call
* NextHeadersRequestLocator and send a getheaders message using it.
*/
ProcessingResult ProcessNextHeaders(const std::vector<CBlockHeader>&
received_headers, bool full_headers_message);
/** Issue the next GETHEADERS message to our peer.
*
* This will return a locator appropriate for the current sync object, to continue the
* synchronization phase it is in.
*/
CBlockLocator NextHeadersRequestLocator() const;
private:
/** Clear out all download state that might be in progress (freeing any used
* memory), and mark this object as no longer usable.
*/
void Finalize();
/**
* Only called in PRESYNC.
* Validate the work on the headers we received from the network, and
* store commitments for later. Update overall state with successfully
* processed headers.
* On failure, this invokes Finalize() and returns false.
*/
bool ValidateAndStoreHeadersCommitments(const std::vector<CBlockHeader>& headers);
/** In PRESYNC, process and update state for a single header */
bool ValidateAndProcessSingleHeader(const CBlockHeader& current);
/** In REDOWNLOAD, check a header's commitment (if applicable) and add to
* buffer for later processing */
bool ValidateAndStoreRedownloadedHeader(const CBlockHeader& header);
/** Return a set of headers that satisfy our proof-of-work threshold */
std::vector<CBlockHeader> PopHeadersReadyForAcceptance();
private:
/** NodeId of the peer (used for log messages) **/
const NodeId m_id;
/** We use the consensus params in our anti-DoS calculations */
const Consensus::Params& m_consensus_params;
/** Store the last block in our block index that the peer's chain builds from */
const CBlockIndex* m_chain_start{nullptr};
/** Minimum work that we're looking for on this chain. */
const arith_uint256 m_minimum_required_work;
/** Work that we've seen so far on the peer's chain */
arith_uint256 m_current_chain_work;
/** m_hasher is a salted hasher for making our 1-bit commitments to headers we've seen. */
const SaltedTxidHasher m_hasher;
/** A queue of commitment bits, created during the 1st phase, and verified during the 2nd. */
bitdeque<> m_header_commitments;
/** The (secret) offset on the heights for which to create commitments.
*
* m_header_commitments entries are created at any height h for which
* (h % HEADER_COMMITMENT_PERIOD) == m_commit_offset. */
const unsigned m_commit_offset;
/** m_max_commitments is a bound we calculate on how long an honest peer's chain could be,
* given the MTP rule.
*
* Any peer giving us more headers than this will have its sync aborted. This serves as a
* memory bound on m_header_commitments. */
uint64_t m_max_commitments{0};
/** Store the latest header received while in PRESYNC (initialized to m_chain_start) */
CBlockHeader m_last_header_received;
/** Height of m_last_header_received */
int64_t m_current_height{0};
/** During phase 2 (REDOWNLOAD), we buffer redownloaded headers in memory
* until enough commitments have been verified; those are stored in
* m_redownloaded_headers */
std::deque<CompressedHeader> m_redownloaded_headers;
/** Height of last header in m_redownloaded_headers */
int64_t m_redownload_buffer_last_height{0};
/** Hash of last header in m_redownloaded_headers (initialized to
* m_chain_start). We have to cache it because we don't have hashPrevBlock
* available in a CompressedHeader.
*/
uint256 m_redownload_buffer_last_hash;
/** The hashPrevBlock entry for the first header in m_redownloaded_headers
* We need this to reconstruct the full header when it's time for
* processing.
*/
uint256 m_redownload_buffer_first_prev_hash;
/** The accumulated work on the redownloaded chain. */
arith_uint256 m_redownload_chain_work;
/** Set this to true once we encounter the target blockheader during phase
* 2 (REDOWNLOAD). At this point, we can process and store all remaining
* headers still in m_redownloaded_headers.
*/
bool m_process_all_remaining_headers{false};
/** Current state of our headers sync. */
State m_download_state{State::PRESYNC};
};
#endif // BITCOIN_HEADERSSYNC_H

View file

@ -165,6 +165,7 @@ const CLogCategoryDesc LogCategories[] =
#endif
{BCLog::UTIL, "util"},
{BCLog::BLOCKSTORE, "blockstorage"},
{BCLog::HEADERSSYNC, "headerssync"},
{BCLog::ALL, "1"},
{BCLog::ALL, "all"},
};
@ -263,6 +264,8 @@ std::string LogCategoryToStr(BCLog::LogFlags category)
return "util";
case BCLog::LogFlags::BLOCKSTORE:
return "blockstorage";
case BCLog::LogFlags::HEADERSSYNC:
return "headerssync";
case BCLog::LogFlags::ALL:
return "all";
}

View file

@ -65,6 +65,7 @@ namespace BCLog {
#endif
UTIL = (1 << 25),
BLOCKSTORE = (1 << 26),
HEADERSSYNC = (1 << 27),
ALL = ~(uint32_t)0,
};
enum class Level {

View file

@ -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,12 @@ 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) {};
explicit Peer(NodeId id, ServiceFlags our_services)
: m_id{id}
, m_our_services{our_services}
@ -581,18 +588,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);
/** 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);
/** 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);
/** 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.
@ -2263,6 +2322,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.
@ -2316,6 +2404,104 @@ 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);
}
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 +2647,71 @@ 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);
}
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,16 +2725,38 @@ 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 (state.IsInvalid()) {
@ -2508,8 +2766,8 @@ void PeerManagerImpl::ProcessHeadersMessage(CNode& pfrom, Peer& peer,
}
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, GetLocator(pindexLast), peer)) {
LogPrint(BCLog::NET, "more getheaders (%d) to end to peer=%d (startheight:%d)\n",
@ -4046,7 +4304,7 @@ 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);
return ProcessHeadersMessage(pfrom, *peer, std::move(headers), /*via_compact_block=*/false);
}
if (msg_type == NetMsgType::BLOCK)

View file

@ -3432,6 +3432,22 @@ std::vector<unsigned char> ChainstateManager::GenerateCoinbaseCommitment(CBlock&
return commitment;
}
bool HasValidProofOfWork(const std::vector<CBlockHeader>& headers, const Consensus::Params& consensusParams)
{
return std::all_of(headers.cbegin(), headers.cend(),
[&](const auto& header) { return CheckProofOfWork(header.GetHash(), header.nBits, consensusParams);});
}
arith_uint256 CalculateHeadersWork(const std::vector<CBlockHeader>& headers)
{
arith_uint256 total_work{0};
for (const CBlockHeader& header : headers) {
CBlockIndex dummy(header);
total_work += GetBlockProof(dummy);
}
return total_work;
}
/** Context-dependent validity checks.
* By "context", we mean only the previous block headers, but not the UTXO
* set; UTXO-related validity checks are done in ConnectBlock().

View file

@ -340,6 +340,12 @@ bool TestBlockValidity(BlockValidationState& state,
bool fCheckPOW = true,
bool fCheckMerkleRoot = true) EXCLUSIVE_LOCKS_REQUIRED(cs_main);
/** Check with the proof of work on each blockheader matches the value in nBits */
bool HasValidProofOfWork(const std::vector<CBlockHeader>& headers, const Consensus::Params& consensusParams);
/** Return the sum of the work on a given set of headers */
arith_uint256 CalculateHeadersWork(const std::vector<CBlockHeader>& headers);
/** RAII wrapper for VerifyDB: Verify consistency of the block and coin databases */
class CVerifyDB {
public:

View file

@ -22,6 +22,7 @@ class RejectLowDifficultyHeadersTest(BitcoinTestFramework):
self.setup_clean_chain = True
self.chain = 'testnet3' # Use testnet chain because it has an early checkpoint
self.num_nodes = 2
self.extra_args = [["-minimumchainwork=0x0"], ["-minimumchainwork=0x0"]]
def add_options(self, parser):
parser.add_argument(
@ -62,7 +63,7 @@ class RejectLowDifficultyHeadersTest(BitcoinTestFramework):
self.log.info("Feed all fork headers (succeeds without checkpoint)")
# On node 0 it succeeds because checkpoints are disabled
self.restart_node(0, extra_args=['-nocheckpoints'])
self.restart_node(0, extra_args=['-nocheckpoints', "-minimumchainwork=0x0"])
peer_no_checkpoint = self.nodes[0].add_p2p_connection(P2PInterface())
peer_no_checkpoint.send_and_ping(msg_headers(self.headers_fork))
assert {