Merge bitcoin/bitcoin#24543: net processing: Move remaining globals into PeerManagerImpl

778343a379 scripted-diff: Rename PeerManagerImpl members (dergoegge)
91c339243e [net processing] Move nHighestFastAnnounce into PeerManagerImpl (dergoegge)
10b83e2aa3 [net processing] Move block cache state into PeerManagerImpl (dergoegge)
a4c55a93ef [net processing] Inline and simplify UpdatePreferredDownload (dergoegge)
490c08f96a [net processing] Move nPreferredDownload into PeerManagerImpl (dergoegge)
a292df283a [net processing] Move mapNodeState into PeerManagerImpl (dergoegge)
37ecaf3e7a [net processing] Move CNodeState declaration above PeerManagerImpl (dergoegge)

Pull request description:

  This PR moves the remaining net processing globals into `PeerManagerImpl`. This will make testing the peer manager in isolation easier and also acts as a code clean up.

ACKs for top commit:
  jnewbery:
    Code review ACK 778343a379
  MarcoFalke:
    ACK 778343a379 🗒

Tree-SHA512: 4f22105d1de37b94c3ef349f38784a30cf8d450d394a6a7849e5bd78940a71e3edbffa3d25e8efb35d7f698fd255f199de7bd4c33e23af5621a6e4e67ed43cb5
24.x
MacroFake 3 years ago
commit 5d53cf3878
No known key found for this signature in database
GPG Key ID: CE2B75697E69A548

@ -334,6 +334,104 @@ struct Peer {
using PeerRef = std::shared_ptr<Peer>;
/**
* Maintain validation-specific state about nodes, protected by cs_main, instead
* by CNode's own locks. This simplifies asynchronous operation, where
* processing of incoming data is done after the ProcessMessage call returns,
* and we're no longer holding the node's locks.
*/
struct CNodeState {
//! The best known block we know this peer has announced.
const CBlockIndex* pindexBestKnownBlock{nullptr};
//! The hash of the last unknown block this peer has announced.
uint256 hashLastUnknownBlock{};
//! The last full block we both have.
const CBlockIndex* pindexLastCommonBlock{nullptr};
//! The best header we have sent our peer.
const CBlockIndex* pindexBestHeaderSent{nullptr};
//! Length of current-streak of unconnecting headers announcements
int nUnconnectingHeaders{0};
//! Whether we've started headers synchronization with this peer.
bool fSyncStarted{false};
//! When to potentially disconnect peer for stalling headers download
std::chrono::microseconds m_headers_sync_timeout{0us};
//! Since when we're stalling block download progress (in microseconds), or 0.
std::chrono::microseconds m_stalling_since{0us};
std::list<QueuedBlock> vBlocksInFlight;
//! When the first entry in vBlocksInFlight started downloading. Don't care when vBlocksInFlight is empty.
std::chrono::microseconds m_downloading_since{0us};
int nBlocksInFlight{0};
//! Whether we consider this a preferred download peer.
bool fPreferredDownload{false};
//! Whether this peer wants invs or headers (when possible) for block announcements.
bool fPreferHeaders{false};
//! Whether this peer wants invs or cmpctblocks (when possible) for block announcements.
bool fPreferHeaderAndIDs{false};
/**
* Whether this peer will send us cmpctblocks if we request them.
* This is not used to gate request logic, as we really only care about fSupportsDesiredCmpctVersion,
* but is used as a flag to "lock in" the version of compact blocks (fWantsCmpctWitness) we send.
*/
bool fProvidesHeaderAndIDs{false};
//! Whether this peer can give us witnesses
bool fHaveWitness{false};
//! Whether this peer wants witnesses in cmpctblocks/blocktxns
bool fWantsCmpctWitness{false};
/**
* If we've announced NODE_WITNESS to this peer: whether the peer sends witnesses in cmpctblocks/blocktxns,
* otherwise: whether this peer sends non-witnesses in cmpctblocks/blocktxns.
*/
bool fSupportsDesiredCmpctVersion{false};
/** State used to enforce CHAIN_SYNC_TIMEOUT and EXTRA_PEER_CHECK_INTERVAL logic.
*
* Both are only in effect for outbound, non-manual, non-protected connections.
* Any peer protected (m_protect = true) is not chosen for eviction. A peer is
* marked as protected if all of these are true:
* - its connection type is IsBlockOnlyConn() == false
* - it gave us a valid connecting header
* - we haven't reached MAX_OUTBOUND_PEERS_TO_PROTECT_FROM_DISCONNECT yet
* - its chain tip has at least as much work as ours
*
* CHAIN_SYNC_TIMEOUT: if a peer's best known block has less work than our tip,
* set a timeout CHAIN_SYNC_TIMEOUT in the future:
* - If at timeout their best known block now has more work than our tip
* when the timeout was set, then either reset the timeout or clear it
* (after comparing against our current tip's work)
* - If at timeout their best known block still has less work than our
* tip did when the timeout was set, then send a getheaders message,
* and set a shorter timeout, HEADERS_RESPONSE_TIME seconds in future.
* If their best known block is still behind when that new timeout is
* reached, disconnect.
*
* EXTRA_PEER_CHECK_INTERVAL: after each interval, if we have too many outbound peers,
* drop the outbound one that least recently announced us a new block.
*/
struct ChainSyncTimeoutState {
//! A timeout used for checking whether our peer has sufficiently synced
std::chrono::seconds m_timeout{0s};
//! A header with the work we require on our peer's chain
const CBlockIndex* m_work_header{nullptr};
//! After timeout is reached, set to true after sending getheaders
bool m_sent_getheaders{false};
//! Whether this peer is protected from disconnection due to a bad/slow chain
bool m_protect{false};
};
ChainSyncTimeoutState m_chain_sync;
//! Time of last new block announcement
int64_t m_last_block_announcement{0};
//! Whether this peer is an inbound connection
const bool m_is_inbound;
//! A rolling bloom filter of all announced tx CInvs to this peer.
CRollingBloomFilter m_recently_announced_invs = CRollingBloomFilter{INVENTORY_MAX_RECENT_RELAY, 0.000001};
CNodeState(bool is_inbound) : m_is_inbound(is_inbound) {}
};
class PeerManagerImpl final : public PeerManager
{
public:
@ -366,6 +464,7 @@ public:
void Misbehaving(const NodeId pnode, const int howmuch, const std::string& message) override;
void ProcessMessage(CNode& pfrom, const std::string& msg_type, CDataStream& vRecv,
const std::chrono::microseconds time_received, const std::atomic<bool>& interruptMsgProc) override;
void UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds) override;
private:
/** Consider evicting an outbound peer based on the amount of time they've been behind our tip */
@ -484,6 +583,16 @@ private:
*/
std::map<NodeId, PeerRef> m_peer_map GUARDED_BY(m_peer_mutex);
/** Map maintaining per-node state. */
std::map<NodeId, CNodeState> m_node_states GUARDED_BY(cs_main);
/** Get a pointer to a const CNodeState, used when not mutating the CNodeState object. */
const CNodeState* State(NodeId pnode) const EXCLUSIVE_LOCKS_REQUIRED(cs_main);
/** Get a pointer to a mutable CNodeState. */
CNodeState* State(NodeId pnode) EXCLUSIVE_LOCKS_REQUIRED(cs_main);
uint32_t GetFetchFlags(const CNode& pfrom) const EXCLUSIVE_LOCKS_REQUIRED(cs_main);
std::atomic<std::chrono::microseconds> m_next_inv_to_inbounds{0us};
/** Number of nodes with fSyncStarted. */
@ -503,6 +612,9 @@ private:
/** Number of outbound peers with m_chain_sync.m_protect. */
int m_outbound_peers_with_protect_from_disconnect GUARDED_BY(cs_main) = 0;
/** Number of preferable block download peers. */
int m_num_preferred_download_peers GUARDED_BY(cs_main){0};
bool AlreadyHaveTx(const GenTxid& gtxid) EXCLUSIVE_LOCKS_REQUIRED(cs_main);
/**
@ -569,6 +681,17 @@ private:
std::chrono::microseconds NextInvToInbounds(std::chrono::microseconds now,
std::chrono::seconds average_interval);
// All of the following cache a recent block, and are protected by m_most_recent_block_mutex
RecursiveMutex m_most_recent_block_mutex;
std::shared_ptr<const CBlock> m_most_recent_block GUARDED_BY(m_most_recent_block_mutex);
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);
bool m_most_recent_compact_block_has_witnesses GUARDED_BY(m_most_recent_block_mutex){false};
/** Height of the highest block announced using BIP 152 high-bandwidth mode. */
int m_highest_fast_announce{0};
/** Have we requested this block from a peer */
bool IsBlockRequested(const uint256& hash) EXCLUSIVE_LOCKS_REQUIRED(cs_main);
@ -710,122 +833,20 @@ private:
*/
bool SetupAddressRelay(const CNode& node, Peer& peer);
};
} // namespace
namespace {
/** Number of preferable block download peers. */
int nPreferredDownload GUARDED_BY(cs_main) = 0;
} // namespace
namespace {
/**
* Maintain validation-specific state about nodes, protected by cs_main, instead
* by CNode's own locks. This simplifies asynchronous operation, where
* processing of incoming data is done after the ProcessMessage call returns,
* and we're no longer holding the node's locks.
*/
struct CNodeState {
//! The best known block we know this peer has announced.
const CBlockIndex* pindexBestKnownBlock{nullptr};
//! The hash of the last unknown block this peer has announced.
uint256 hashLastUnknownBlock{};
//! The last full block we both have.
const CBlockIndex* pindexLastCommonBlock{nullptr};
//! The best header we have sent our peer.
const CBlockIndex* pindexBestHeaderSent{nullptr};
//! Length of current-streak of unconnecting headers announcements
int nUnconnectingHeaders{0};
//! Whether we've started headers synchronization with this peer.
bool fSyncStarted{false};
//! When to potentially disconnect peer for stalling headers download
std::chrono::microseconds m_headers_sync_timeout{0us};
//! Since when we're stalling block download progress (in microseconds), or 0.
std::chrono::microseconds m_stalling_since{0us};
std::list<QueuedBlock> vBlocksInFlight;
//! When the first entry in vBlocksInFlight started downloading. Don't care when vBlocksInFlight is empty.
std::chrono::microseconds m_downloading_since{0us};
int nBlocksInFlight{0};
//! Whether we consider this a preferred download peer.
bool fPreferredDownload{false};
//! Whether this peer wants invs or headers (when possible) for block announcements.
bool fPreferHeaders{false};
//! Whether this peer wants invs or cmpctblocks (when possible) for block announcements.
bool fPreferHeaderAndIDs{false};
/**
* Whether this peer will send us cmpctblocks if we request them.
* This is not used to gate request logic, as we really only care about fSupportsDesiredCmpctVersion,
* but is used as a flag to "lock in" the version of compact blocks (fWantsCmpctWitness) we send.
*/
bool fProvidesHeaderAndIDs{false};
//! Whether this peer can give us witnesses
bool fHaveWitness{false};
//! Whether this peer wants witnesses in cmpctblocks/blocktxns
bool fWantsCmpctWitness{false};
/**
* If we've announced NODE_WITNESS to this peer: whether the peer sends witnesses in cmpctblocks/blocktxns,
* otherwise: whether this peer sends non-witnesses in cmpctblocks/blocktxns.
*/
bool fSupportsDesiredCmpctVersion{false};
/** State used to enforce CHAIN_SYNC_TIMEOUT and EXTRA_PEER_CHECK_INTERVAL logic.
*
* Both are only in effect for outbound, non-manual, non-protected connections.
* Any peer protected (m_protect = true) is not chosen for eviction. A peer is
* marked as protected if all of these are true:
* - its connection type is IsBlockOnlyConn() == false
* - it gave us a valid connecting header
* - we haven't reached MAX_OUTBOUND_PEERS_TO_PROTECT_FROM_DISCONNECT yet
* - its chain tip has at least as much work as ours
*
* CHAIN_SYNC_TIMEOUT: if a peer's best known block has less work than our tip,
* set a timeout CHAIN_SYNC_TIMEOUT in the future:
* - If at timeout their best known block now has more work than our tip
* when the timeout was set, then either reset the timeout or clear it
* (after comparing against our current tip's work)
* - If at timeout their best known block still has less work than our
* tip did when the timeout was set, then send a getheaders message,
* and set a shorter timeout, HEADERS_RESPONSE_TIME seconds in future.
* If their best known block is still behind when that new timeout is
* reached, disconnect.
*
* EXTRA_PEER_CHECK_INTERVAL: after each interval, if we have too many outbound peers,
* drop the outbound one that least recently announced us a new block.
*/
struct ChainSyncTimeoutState {
//! A timeout used for checking whether our peer has sufficiently synced
std::chrono::seconds m_timeout{0s};
//! A header with the work we require on our peer's chain
const CBlockIndex* m_work_header{nullptr};
//! After timeout is reached, set to true after sending getheaders
bool m_sent_getheaders{false};
//! Whether this peer is protected from disconnection due to a bad/slow chain
bool m_protect{false};
};
ChainSyncTimeoutState m_chain_sync;
//! Time of last new block announcement
int64_t m_last_block_announcement{0};
//! Whether this peer is an inbound connection
const bool m_is_inbound;
//! A rolling bloom filter of all announced tx CInvs to this peer.
CRollingBloomFilter m_recently_announced_invs = CRollingBloomFilter{INVENTORY_MAX_RECENT_RELAY, 0.000001};
CNodeState(bool is_inbound) : m_is_inbound(is_inbound) {}
};
/** Map maintaining per-node state. */
static std::map<NodeId, CNodeState> mapNodeState GUARDED_BY(cs_main);
static CNodeState *State(NodeId pnode) EXCLUSIVE_LOCKS_REQUIRED(cs_main) {
std::map<NodeId, CNodeState>::iterator it = mapNodeState.find(pnode);
if (it == mapNodeState.end())
const CNodeState* PeerManagerImpl::State(NodeId pnode) const EXCLUSIVE_LOCKS_REQUIRED(cs_main)
{
std::map<NodeId, CNodeState>::const_iterator it = m_node_states.find(pnode);
if (it == m_node_states.end())
return nullptr;
return &it->second;
}
CNodeState* PeerManagerImpl::State(NodeId pnode) EXCLUSIVE_LOCKS_REQUIRED(cs_main)
{
return const_cast<CNodeState*>(std::as_const(*this).State(pnode));
}
/**
* Whether the peer supports the address. For example, a peer that does not
* implement BIP155 cannot receive Tor v3 addresses because it requires
@ -865,16 +886,6 @@ static void AddKnownTx(Peer& peer, const uint256& hash)
}
}
static void UpdatePreferredDownload(const CNode& node, CNodeState* state) EXCLUSIVE_LOCKS_REQUIRED(cs_main)
{
nPreferredDownload -= state->fPreferredDownload;
// Whether this node should be marked as a preferred download node.
state->fPreferredDownload = (!node.IsInboundConn() || node.HasPermission(NetPermissionFlags::NoBan)) && !node.IsAddrFetchConn() && !node.fClient;
nPreferredDownload += state->fPreferredDownload;
}
std::chrono::microseconds PeerManagerImpl::NextInvToInbounds(std::chrono::microseconds now,
std::chrono::seconds average_interval)
{
@ -1215,9 +1226,7 @@ void PeerManagerImpl::AddTxAnnouncement(const CNode& node, const GenTxid& gtxid,
m_txrequest.ReceivedInv(nodeid, gtxid, preferred, current_time + delay);
}
// This function is used for testing the stale tip eviction logic, see
// denialofservice_tests.cpp
void UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds)
void PeerManagerImpl::UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds)
{
LOCK(cs_main);
CNodeState *state = State(node);
@ -1229,7 +1238,7 @@ void PeerManagerImpl::InitializeNode(CNode *pnode)
NodeId nodeid = pnode->GetId();
{
LOCK(cs_main);
mapNodeState.emplace_hint(mapNodeState.end(), std::piecewise_construct, std::forward_as_tuple(nodeid), std::forward_as_tuple(pnode->IsInboundConn()));
m_node_states.emplace_hint(m_node_states.end(), std::piecewise_construct, std::forward_as_tuple(nodeid), std::forward_as_tuple(pnode->IsInboundConn()));
assert(m_txrequest.Count(nodeid) == 0);
}
PeerRef peer = std::make_shared<Peer>(nodeid, /*tx_relay=*/ !pnode->IsBlockOnlyConn());
@ -1291,18 +1300,18 @@ void PeerManagerImpl::FinalizeNode(const CNode& node)
}
WITH_LOCK(g_cs_orphans, m_orphanage.EraseForPeer(nodeid));
m_txrequest.DisconnectedPeer(nodeid);
nPreferredDownload -= state->fPreferredDownload;
m_num_preferred_download_peers -= state->fPreferredDownload;
m_peers_downloading_from -= (state->nBlocksInFlight != 0);
assert(m_peers_downloading_from >= 0);
m_outbound_peers_with_protect_from_disconnect -= state->m_chain_sync.m_protect;
assert(m_outbound_peers_with_protect_from_disconnect >= 0);
mapNodeState.erase(nodeid);
m_node_states.erase(nodeid);
if (mapNodeState.empty()) {
if (m_node_states.empty()) {
// Do a consistency check after the last peer is removed.
assert(mapBlocksInFlight.empty());
assert(nPreferredDownload == 0);
assert(m_num_preferred_download_peers == 0);
assert(m_peers_downloading_from == 0);
assert(m_outbound_peers_with_protect_from_disconnect == 0);
assert(m_wtxid_relay_peers == 0);
@ -1343,7 +1352,7 @@ bool PeerManagerImpl::GetNodeStateStats(NodeId nodeid, CNodeStateStats& stats) c
{
{
LOCK(cs_main);
CNodeState* state = State(nodeid);
const CNodeState* state = State(nodeid);
if (state == nullptr)
return false;
stats.nSyncHeight = state->pindexBestKnownBlock ? state->pindexBestKnownBlock->nHeight : -1;
@ -1617,13 +1626,6 @@ void PeerManagerImpl::BlockDisconnected(const std::shared_ptr<const CBlock> &blo
m_recent_confirmed_transactions.reset();
}
// All of the following cache a recent block, and are protected by cs_most_recent_block
static RecursiveMutex cs_most_recent_block;
static std::shared_ptr<const CBlock> most_recent_block GUARDED_BY(cs_most_recent_block);
static std::shared_ptr<const CBlockHeaderAndShortTxIDs> most_recent_compact_block GUARDED_BY(cs_most_recent_block);
static uint256 most_recent_block_hash GUARDED_BY(cs_most_recent_block);
static bool fWitnessesPresentInMostRecentCompactBlock GUARDED_BY(cs_most_recent_block);
/**
* Maintain state about the best-seen block and fast-announce a compact block
* to compatible peers.
@ -1635,10 +1637,9 @@ void PeerManagerImpl::NewPoWValidBlock(const CBlockIndex *pindex, const std::sha
LOCK(cs_main);
static int nHighestFastAnnounce = 0;
if (pindex->nHeight <= nHighestFastAnnounce)
if (pindex->nHeight <= m_highest_fast_announce)
return;
nHighestFastAnnounce = pindex->nHeight;
m_highest_fast_announce = pindex->nHeight;
bool fWitnessEnabled = DeploymentActiveAt(*pindex, m_chainparams.GetConsensus(), Consensus::DEPLOYMENT_SEGWIT);
uint256 hashBlock(pblock->GetHash());
@ -1646,11 +1647,11 @@ void PeerManagerImpl::NewPoWValidBlock(const CBlockIndex *pindex, const std::sha
std::async(std::launch::deferred, [&] { return msgMaker.Make(NetMsgType::CMPCTBLOCK, *pcmpctblock); })};
{
LOCK(cs_most_recent_block);
most_recent_block_hash = hashBlock;
most_recent_block = pblock;
most_recent_compact_block = pcmpctblock;
fWitnessesPresentInMostRecentCompactBlock = fWitnessEnabled;
LOCK(m_most_recent_block_mutex);
m_most_recent_block_hash = hashBlock;
m_most_recent_block = pblock;
m_most_recent_compact_block = pcmpctblock;
m_most_recent_compact_block_has_witnesses = fWitnessEnabled;
}
m_connman.ForEachNode([this, pindex, fWitnessEnabled, &lazy_ser, &hashBlock](CNode* pnode) EXCLUSIVE_LOCKS_REQUIRED(::cs_main) {
@ -1860,10 +1861,10 @@ void PeerManagerImpl::ProcessGetBlockData(CNode& pfrom, Peer& peer, const CInv&
std::shared_ptr<const CBlockHeaderAndShortTxIDs> a_recent_compact_block;
bool fWitnessesPresentInARecentCompactBlock;
{
LOCK(cs_most_recent_block);
a_recent_block = most_recent_block;
a_recent_compact_block = most_recent_compact_block;
fWitnessesPresentInARecentCompactBlock = fWitnessesPresentInMostRecentCompactBlock;
LOCK(m_most_recent_block_mutex);
a_recent_block = m_most_recent_block;
a_recent_compact_block = m_most_recent_compact_block;
fWitnessesPresentInARecentCompactBlock = m_most_recent_compact_block_has_witnesses;
}
bool need_activate_chain = false;
@ -2126,7 +2127,8 @@ void PeerManagerImpl::ProcessGetData(CNode& pfrom, Peer& peer, const std::atomic
}
}
static uint32_t GetFetchFlags(const CNode& pfrom) EXCLUSIVE_LOCKS_REQUIRED(cs_main) {
uint32_t PeerManagerImpl::GetFetchFlags(const CNode& pfrom) const EXCLUSIVE_LOCKS_REQUIRED(cs_main)
{
uint32_t nFetchFlags = 0;
if (State(pfrom.GetId())->fHaveWitness) {
nFetchFlags |= MSG_WITNESS_FLAG;
@ -2735,8 +2737,10 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
// Potentially mark this peer as a preferred download peer.
{
LOCK(cs_main);
UpdatePreferredDownload(pfrom, State(pfrom.GetId()));
LOCK(cs_main);
CNodeState* state = State(pfrom.GetId());
state->fPreferredDownload = (!pfrom.IsInboundConn() || pfrom.HasPermission(NetPermissionFlags::NoBan)) && !pfrom.IsAddrFetchConn() && !pfrom.fClient;
m_num_preferred_download_peers += state->fPreferredDownload;
}
// Self advertisement & GETADDR logic
@ -3159,8 +3163,8 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
{
std::shared_ptr<const CBlock> a_recent_block;
{
LOCK(cs_most_recent_block);
a_recent_block = most_recent_block;
LOCK(m_most_recent_block_mutex);
a_recent_block = m_most_recent_block;
}
BlockValidationState state;
if (!m_chainman.ActiveChainstate().ActivateBestChain(state, a_recent_block)) {
@ -3211,10 +3215,10 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
std::shared_ptr<const CBlock> recent_block;
{
LOCK(cs_most_recent_block);
if (most_recent_block_hash == req.blockhash)
recent_block = most_recent_block;
// Unlock cs_most_recent_block to avoid cs_main lock inversion
LOCK(m_most_recent_block_mutex);
if (m_most_recent_block_hash == req.blockhash)
recent_block = m_most_recent_block;
// Unlock m_most_recent_block_mutex to avoid cs_main lock inversion
}
if (recent_block) {
SendBlockTransactions(pfrom, *recent_block, req);
@ -4678,7 +4682,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
if (m_chainman.m_best_header == nullptr) {
m_chainman.m_best_header = m_chainman.ActiveChain().Tip();
}
bool fFetch = state.fPreferredDownload || (nPreferredDownload == 0 && !pto->fClient && !pto->IsAddrFetchConn()); // Download if this is a nice peer, or we have no nice peers and this one might do.
bool fFetch = state.fPreferredDownload || (m_num_preferred_download_peers == 0 && !pto->fClient && !pto->IsAddrFetchConn()); // Download if this is a nice peer, or we have no nice peers and this one might do.
if (!state.fSyncStarted && !pto->fClient && !fImporting && !fReindex) {
// Only actively request headers from a single peer, unless we're close to today.
if ((nSyncStarted == 0 && fFetch) || m_chainman.m_best_header->GetBlockTime() > GetAdjustedTime() - 24 * 60 * 60) {
@ -4783,12 +4787,12 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
bool fGotBlockFromCache = false;
{
LOCK(cs_most_recent_block);
if (most_recent_block_hash == pBestIndex->GetBlockHash()) {
if (state.fWantsCmpctWitness || !fWitnessesPresentInMostRecentCompactBlock)
m_connman.PushMessage(pto, msgMaker.Make(nSendFlags, NetMsgType::CMPCTBLOCK, *most_recent_compact_block));
LOCK(m_most_recent_block_mutex);
if (m_most_recent_block_hash == pBestIndex->GetBlockHash()) {
if (state.fWantsCmpctWitness || !m_most_recent_compact_block_has_witnesses)
m_connman.PushMessage(pto, msgMaker.Make(nSendFlags, NetMsgType::CMPCTBLOCK, *m_most_recent_compact_block));
else {
CBlockHeaderAndShortTxIDs cmpctblock(*most_recent_block, state.fWantsCmpctWitness);
CBlockHeaderAndShortTxIDs cmpctblock(*m_most_recent_block, state.fWantsCmpctWitness);
m_connman.PushMessage(pto, msgMaker.Make(nSendFlags, NetMsgType::CMPCTBLOCK, cmpctblock));
}
fGotBlockFromCache = true;
@ -5023,7 +5027,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
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 (current_time > state.m_headers_sync_timeout && nSyncStarted == 1 && (nPreferredDownload - state.fPreferredDownload >= 1)) {
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.
// Note: If all our peers are inbound, then we won't

@ -87,6 +87,9 @@ public:
/** Process a single message from a peer. Public for fuzz testing */
virtual void ProcessMessage(CNode& pfrom, const std::string& msg_type, CDataStream& vRecv,
const std::chrono::microseconds time_received, const std::atomic<bool>& interruptMsgProc) = 0;
/** This function is used for testing the stale tip eviction logic, see denialofservice_tests.cpp */
virtual void UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds) = 0;
};
#endif // BITCOIN_NET_PROCESSING_H

@ -32,8 +32,6 @@ static CService ip(uint32_t i)
return CService(CNetAddr(s), Params().GetDefaultPort());
}
void UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_seconds);
BOOST_FIXTURE_TEST_SUITE(denialofservice_tests, TestingSetup)
// Test eviction of an outbound peer whose chain never advances
@ -195,7 +193,7 @@ BOOST_AUTO_TEST_CASE(stale_tip_peer_management)
// Update the last announced block time for the last
// peer, and check that the next newest node gets evicted.
UpdateLastBlockAnnounceTime(vNodes.back()->GetId(), GetTime());
peerLogic->UpdateLastBlockAnnounceTime(vNodes.back()->GetId(), GetTime());
peerLogic->CheckForStaleTipAndEvictPeers();
for (int i = 0; i < max_outbound_full_relay - 1; ++i) {

Loading…
Cancel
Save