mirror of
https://github.com/bitcoin/bitcoin.git
synced 2025-11-11 06:28:31 +01:00
Merge bitcoin/bitcoin#25514: net processing: Move CNode::nServices and CNode::nLocalServices to Peer
8d8eeb422e[net processing] Remove CNode::nLocalServices (John Newbery)5961f8eea1[net] Return CService from GetLocalAddrForPeer and GetLocalAddress (dergoegge)d9079fe18d[net processing] Remove CNode::nServices (John Newbery)7d1c036934[net processing] Replace fHaveWitness with CanServeWitnesses() (John Newbery)f65e83d51b[net processing] Remove fClient and m_limited_node (John Newbery)fc5eb528f7[tests] Connect peer in outbound_slow_chain_eviction by sending p2p messages (John Newbery)1f52c47d5c[net processing] Add m_our_services and m_their_services to Peer (John Newbery) Pull request description: Another step in #19398. Which services we offer to a peer and which services they offer to us is application layer data and should not be stored on `CNode`. This is also a prerequisite for adding `PeerManager` unit tests (See #25515). ACKs for top commit: MarcoFalke: ACK8d8eeb422e🔑 jnewbery: utACK8d8eeb422emzumsande: Code Review ACK8d8eeb422eTree-SHA512: e772eb2a0a85db346dd7b453a41011a12756fc7cbfda6a9ef6daa9633b9a47b9770ab3dc02377690f9d02127301c3905ff22905977f758bf90b17a9a35b37523
This commit is contained in:
@@ -207,6 +207,23 @@ struct Peer {
|
||||
/** Same id as the CNode object for this peer */
|
||||
const NodeId m_id{0};
|
||||
|
||||
/** Services we offered to this peer.
|
||||
*
|
||||
* This is supplied by CConnman during peer initialization. It's const
|
||||
* because there is no protocol defined for renegotiating services
|
||||
* initially offered to a peer. The set of local services we offer should
|
||||
* not change after initialization.
|
||||
*
|
||||
* An interesting example of this is NODE_NETWORK and initial block
|
||||
* download: a node which starts up from scratch doesn't have any blocks
|
||||
* to serve, but still advertises NODE_NETWORK because it will eventually
|
||||
* fulfill this role after IBD completes. P2P code is written in such a
|
||||
* way that it can gracefully handle peers who don't make good on their
|
||||
* service advertisements. */
|
||||
const ServiceFlags m_our_services;
|
||||
/** Services this peer offered to us. */
|
||||
std::atomic<ServiceFlags> m_their_services{NODE_NONE};
|
||||
|
||||
/** Protects misbehavior data members */
|
||||
Mutex m_misbehavior_mutex;
|
||||
/** Accumulated misbehavior score for this peer */
|
||||
@@ -360,8 +377,9 @@ struct Peer {
|
||||
/** Time of the last getheaders message to this peer */
|
||||
NodeClock::time_point m_last_getheaders_timestamp{};
|
||||
|
||||
Peer(NodeId id)
|
||||
explicit Peer(NodeId id, ServiceFlags our_services)
|
||||
: m_id{id}
|
||||
, m_our_services{our_services}
|
||||
{}
|
||||
|
||||
private:
|
||||
@@ -410,8 +428,6 @@ struct CNodeState {
|
||||
bool m_requested_hb_cmpctblocks{false};
|
||||
/** Whether this peer will send us cmpctblocks if we request them. */
|
||||
bool m_provides_cmpctblocks{false};
|
||||
//! Whether this peer can give us witnesses
|
||||
bool fHaveWitness{false};
|
||||
|
||||
/** State used to enforce CHAIN_SYNC_TIMEOUT and EXTRA_PEER_CHECK_INTERVAL logic.
|
||||
*
|
||||
@@ -482,7 +498,7 @@ public:
|
||||
EXCLUSIVE_LOCKS_REQUIRED(!m_most_recent_block_mutex);
|
||||
|
||||
/** Implement NetEventsInterface */
|
||||
void InitializeNode(CNode* pnode) override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex);
|
||||
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);
|
||||
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);
|
||||
@@ -492,7 +508,8 @@ public:
|
||||
/** Implement PeerManager */
|
||||
void StartScheduledTasks(CScheduler& scheduler) override;
|
||||
void CheckForStaleTipAndEvictPeers() override;
|
||||
std::optional<std::string> FetchBlock(NodeId peer_id, const CBlockIndex& block_index) override;
|
||||
std::optional<std::string> FetchBlock(NodeId peer_id, const CBlockIndex& block_index) override
|
||||
EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex);
|
||||
bool GetNodeStateStats(NodeId nodeid, CNodeStateStats& stats) const override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex);
|
||||
bool IgnoresIncomingTxs() override { return m_ignore_incoming_txs; }
|
||||
void SendPings() override EXCLUSIVE_LOCKS_REQUIRED(!m_peer_mutex);
|
||||
@@ -578,7 +595,7 @@ private:
|
||||
*/
|
||||
bool MaybeSendGetHeaders(CNode& pfrom, const CBlockLocator& locator, Peer& peer);
|
||||
/** Potentially fetch blocks from this peer upon receipt of a new headers tip */
|
||||
void HeadersDirectFetchBlocks(CNode& pfrom, const CBlockIndex* pindexLast);
|
||||
void HeadersDirectFetchBlocks(CNode& pfrom, const Peer& peer, const CBlockIndex* pindexLast);
|
||||
/** Update peer state based on received headers message */
|
||||
void UpdatePeerStateForReceivedHeaders(CNode& pfrom, const CBlockIndex *pindexLast, bool received_new_header, bool may_have_more_headers);
|
||||
|
||||
@@ -657,7 +674,7 @@ private:
|
||||
/** 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);
|
||||
uint32_t GetFetchFlags(const Peer& peer) const;
|
||||
|
||||
std::atomic<std::chrono::microseconds> m_next_inv_to_inbounds{0us};
|
||||
|
||||
@@ -778,7 +795,7 @@ private:
|
||||
/** Update pindexLastCommonBlock and add not-in-flight missing successors to vBlocks, until it has
|
||||
* at most count entries.
|
||||
*/
|
||||
void FindNextBlocksToDownload(NodeId nodeid, unsigned int count, std::vector<const CBlockIndex*>& vBlocks, NodeId& nodeStaller) EXCLUSIVE_LOCKS_REQUIRED(cs_main);
|
||||
void FindNextBlocksToDownload(const Peer& peer, unsigned int count, std::vector<const CBlockIndex*>& vBlocks, NodeId& nodeStaller) EXCLUSIVE_LOCKS_REQUIRED(cs_main);
|
||||
|
||||
std::map<uint256, std::pair<NodeId, std::list<QueuedBlock>::iterator> > mapBlocksInFlight GUARDED_BY(cs_main);
|
||||
|
||||
@@ -848,6 +865,7 @@ private:
|
||||
*
|
||||
* May disconnect from the peer in the case of a bad request.
|
||||
*
|
||||
* @param[in] node The node that we received the request from
|
||||
* @param[in] peer The peer that we received the request from
|
||||
* @param[in] filter_type The filter type the request is for. Must be basic filters.
|
||||
* @param[in] start_height The start height for the request
|
||||
@@ -857,7 +875,7 @@ private:
|
||||
* @param[out] filter_index The filter index, if the request can be serviced.
|
||||
* @return True if the request can be serviced.
|
||||
*/
|
||||
bool PrepareBlockFilterRequest(CNode& peer,
|
||||
bool PrepareBlockFilterRequest(CNode& node, Peer& peer,
|
||||
BlockFilterType filter_type, uint32_t start_height,
|
||||
const uint256& stop_hash, uint32_t max_height_diff,
|
||||
const CBlockIndex*& stop_index,
|
||||
@@ -868,30 +886,33 @@ private:
|
||||
*
|
||||
* May disconnect from the peer in the case of a bad request.
|
||||
*
|
||||
* @param[in] node The node that we received the request from
|
||||
* @param[in] peer The peer that we received the request from
|
||||
* @param[in] vRecv The raw message received
|
||||
*/
|
||||
void ProcessGetCFilters(CNode& peer, CDataStream& vRecv);
|
||||
void ProcessGetCFilters(CNode& node, Peer& peer, CDataStream& vRecv);
|
||||
|
||||
/**
|
||||
* Handle a cfheaders request.
|
||||
*
|
||||
* May disconnect from the peer in the case of a bad request.
|
||||
*
|
||||
* @param[in] node The node that we received the request from
|
||||
* @param[in] peer The peer that we received the request from
|
||||
* @param[in] vRecv The raw message received
|
||||
*/
|
||||
void ProcessGetCFHeaders(CNode& peer, CDataStream& vRecv);
|
||||
void ProcessGetCFHeaders(CNode& node, Peer& peer, CDataStream& vRecv);
|
||||
|
||||
/**
|
||||
* Handle a getcfcheckpt request.
|
||||
*
|
||||
* May disconnect from the peer in the case of a bad request.
|
||||
*
|
||||
* @param[in] node The node that we received the request from
|
||||
* @param[in] peer The peer that we received the request from
|
||||
* @param[in] vRecv The raw message received
|
||||
*/
|
||||
void ProcessGetCFCheckPt(CNode& peer, CDataStream& vRecv);
|
||||
void ProcessGetCFCheckPt(CNode& node, Peer& peer, CDataStream& vRecv);
|
||||
|
||||
/** Checks if address relay is permitted with peer. If needed, initializes
|
||||
* the m_addr_known bloom filter and sets m_addr_relay_enabled to true.
|
||||
@@ -955,6 +976,26 @@ static void AddKnownTx(Peer& peer, const uint256& hash)
|
||||
tx_relay->m_tx_inventory_known_filter.insert(hash);
|
||||
}
|
||||
|
||||
/** Whether this peer can serve us blocks. */
|
||||
static bool CanServeBlocks(const Peer& peer)
|
||||
{
|
||||
return peer.m_their_services & (NODE_NETWORK|NODE_NETWORK_LIMITED);
|
||||
}
|
||||
|
||||
/** Whether this peer can only serve limited recent blocks (e.g. because
|
||||
* it prunes old blocks) */
|
||||
static bool IsLimitedPeer(const Peer& peer)
|
||||
{
|
||||
return (!(peer.m_their_services & NODE_NETWORK) &&
|
||||
(peer.m_their_services & NODE_NETWORK_LIMITED));
|
||||
}
|
||||
|
||||
/** Whether this peer can serve us witness data */
|
||||
static bool CanServeWitnesses(const Peer& peer)
|
||||
{
|
||||
return peer.m_their_services & NODE_WITNESS;
|
||||
}
|
||||
|
||||
std::chrono::microseconds PeerManagerImpl::NextInvToInbounds(std::chrono::microseconds now,
|
||||
std::chrono::seconds average_interval)
|
||||
{
|
||||
@@ -1148,17 +1189,17 @@ void PeerManagerImpl::UpdateBlockAvailability(NodeId nodeid, const uint256 &hash
|
||||
}
|
||||
}
|
||||
|
||||
void PeerManagerImpl::FindNextBlocksToDownload(NodeId nodeid, unsigned int count, std::vector<const CBlockIndex*>& vBlocks, NodeId& nodeStaller)
|
||||
void PeerManagerImpl::FindNextBlocksToDownload(const Peer& peer, unsigned int count, std::vector<const CBlockIndex*>& vBlocks, NodeId& nodeStaller)
|
||||
{
|
||||
if (count == 0)
|
||||
return;
|
||||
|
||||
vBlocks.reserve(vBlocks.size() + count);
|
||||
CNodeState *state = State(nodeid);
|
||||
CNodeState *state = State(peer.m_id);
|
||||
assert(state != nullptr);
|
||||
|
||||
// Make sure pindexBestKnownBlock is up to date, we'll need it.
|
||||
ProcessBlockAvailability(nodeid);
|
||||
ProcessBlockAvailability(peer.m_id);
|
||||
|
||||
if (state->pindexBestKnownBlock == nullptr || state->pindexBestKnownBlock->nChainWork < m_chainman.ActiveChain().Tip()->nChainWork || state->pindexBestKnownBlock->nChainWork < nMinimumChainWork) {
|
||||
// This peer has nothing interesting.
|
||||
@@ -1206,7 +1247,7 @@ void PeerManagerImpl::FindNextBlocksToDownload(NodeId nodeid, unsigned int count
|
||||
// We consider the chain that this peer is on invalid.
|
||||
return;
|
||||
}
|
||||
if (!State(nodeid)->fHaveWitness && DeploymentActiveAt(*pindex, m_chainman, Consensus::DEPLOYMENT_SEGWIT)) {
|
||||
if (!CanServeWitnesses(peer) && DeploymentActiveAt(*pindex, m_chainman, Consensus::DEPLOYMENT_SEGWIT)) {
|
||||
// We wouldn't download this block or its descendants from this peer.
|
||||
return;
|
||||
}
|
||||
@@ -1217,7 +1258,7 @@ void PeerManagerImpl::FindNextBlocksToDownload(NodeId nodeid, unsigned int count
|
||||
// The block is not already downloaded, and not yet in flight.
|
||||
if (pindex->nHeight > nWindowEnd) {
|
||||
// We reached the end of the window.
|
||||
if (vBlocks.size() == 0 && waitingfor != nodeid) {
|
||||
if (vBlocks.size() == 0 && waitingfor != peer.m_id) {
|
||||
// We aren't able to fetch anything, but we would be if the download window was one larger.
|
||||
nodeStaller = waitingfor;
|
||||
}
|
||||
@@ -1239,10 +1280,7 @@ void PeerManagerImpl::FindNextBlocksToDownload(NodeId nodeid, unsigned int count
|
||||
|
||||
void PeerManagerImpl::PushNodeVersion(CNode& pnode, const Peer& peer)
|
||||
{
|
||||
// Note that pnode->GetLocalServices() is a reflection of the local
|
||||
// services we were offering when the CNode object was created for this
|
||||
// peer.
|
||||
uint64_t my_services{pnode.GetLocalServices()};
|
||||
uint64_t my_services{peer.m_our_services};
|
||||
const int64_t nTime{count_seconds(GetTime<std::chrono::seconds>())};
|
||||
uint64_t nonce = pnode.GetLocalNonce();
|
||||
const int nNodeStartingHeight{m_best_height};
|
||||
@@ -1299,21 +1337,21 @@ void PeerManagerImpl::UpdateLastBlockAnnounceTime(NodeId node, int64_t time_in_s
|
||||
if (state) state->m_last_block_announcement = time_in_seconds;
|
||||
}
|
||||
|
||||
void PeerManagerImpl::InitializeNode(CNode *pnode)
|
||||
void PeerManagerImpl::InitializeNode(CNode& node, ServiceFlags our_services)
|
||||
{
|
||||
NodeId nodeid = pnode->GetId();
|
||||
NodeId nodeid = node.GetId();
|
||||
{
|
||||
LOCK(cs_main);
|
||||
m_node_states.emplace_hint(m_node_states.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(node.IsInboundConn()));
|
||||
assert(m_txrequest.Count(nodeid) == 0);
|
||||
}
|
||||
PeerRef peer = std::make_shared<Peer>(nodeid);
|
||||
PeerRef peer = std::make_shared<Peer>(nodeid, our_services);
|
||||
{
|
||||
LOCK(m_peer_mutex);
|
||||
m_peer_map.emplace_hint(m_peer_map.end(), nodeid, peer);
|
||||
}
|
||||
if (!pnode->IsInboundConn()) {
|
||||
PushNodeVersion(*pnode, *peer);
|
||||
if (!node.IsInboundConn()) {
|
||||
PushNodeVersion(node, *peer);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1431,6 +1469,7 @@ bool PeerManagerImpl::GetNodeStateStats(NodeId nodeid, CNodeStateStats& stats) c
|
||||
|
||||
PeerRef peer = GetPeerRef(nodeid);
|
||||
if (peer == nullptr) return false;
|
||||
stats.their_services = peer->m_their_services;
|
||||
stats.m_starting_height = peer->m_starting_height;
|
||||
// It is common for nodes with good ping times to suddenly become lagged,
|
||||
// due to a new block arriving or other large transfer.
|
||||
@@ -1585,12 +1624,14 @@ std::optional<std::string> PeerManagerImpl::FetchBlock(NodeId peer_id, const CBl
|
||||
if (fImporting) return "Importing...";
|
||||
if (fReindex) return "Reindexing...";
|
||||
|
||||
LOCK(cs_main);
|
||||
// Ensure this peer exists and hasn't been disconnected
|
||||
CNodeState* state = State(peer_id);
|
||||
if (state == nullptr) return "Peer does not exist";
|
||||
PeerRef peer = GetPeerRef(peer_id);
|
||||
if (peer == nullptr) return "Peer does not exist";
|
||||
|
||||
// Ignore pre-segwit peers
|
||||
if (!state->fHaveWitness) return "Pre-SegWit peer";
|
||||
if (!CanServeWitnesses(*peer)) return "Pre-SegWit peer";
|
||||
|
||||
LOCK(cs_main);
|
||||
|
||||
// Mark block as in-flight unless it already is (for this peer).
|
||||
// If a block was already in-flight for a different peer, its BLOCKTXN
|
||||
@@ -1974,7 +2015,7 @@ void PeerManagerImpl::ProcessGetBlockData(CNode& pfrom, Peer& peer, const CInv&
|
||||
}
|
||||
// Avoid leaking prune-height by never sending blocks below the NODE_NETWORK_LIMITED threshold
|
||||
if (!pfrom.HasPermission(NetPermissionFlags::NoBan) && (
|
||||
(((pfrom.GetLocalServices() & NODE_NETWORK_LIMITED) == NODE_NETWORK_LIMITED) && ((pfrom.GetLocalServices() & NODE_NETWORK) != NODE_NETWORK) && (m_chainman.ActiveChain().Tip()->nHeight - pindex->nHeight > (int)NODE_NETWORK_LIMITED_MIN_BLOCKS + 2 /* add two blocks buffer extension for possible races */) )
|
||||
(((peer.m_our_services & NODE_NETWORK_LIMITED) == NODE_NETWORK_LIMITED) && ((peer.m_our_services & NODE_NETWORK) != NODE_NETWORK) && (m_chainman.ActiveChain().Tip()->nHeight - pindex->nHeight > (int)NODE_NETWORK_LIMITED_MIN_BLOCKS + 2 /* add two blocks buffer extension for possible races */) )
|
||||
)) {
|
||||
LogPrint(BCLog::NET, "Ignore block request below NODE_NETWORK_LIMITED threshold, disconnect peer=%d\n", pfrom.GetId());
|
||||
//disconnect node and prevent it from stalling (would otherwise wait for the missing block)
|
||||
@@ -2191,10 +2232,10 @@ void PeerManagerImpl::ProcessGetData(CNode& pfrom, Peer& peer, const std::atomic
|
||||
}
|
||||
}
|
||||
|
||||
uint32_t PeerManagerImpl::GetFetchFlags(const CNode& pfrom) const EXCLUSIVE_LOCKS_REQUIRED(cs_main)
|
||||
uint32_t PeerManagerImpl::GetFetchFlags(const Peer& peer) const
|
||||
{
|
||||
uint32_t nFetchFlags = 0;
|
||||
if (State(pfrom.GetId())->fHaveWitness) {
|
||||
if (CanServeWitnesses(peer)) {
|
||||
nFetchFlags |= MSG_WITNESS_FLAG;
|
||||
}
|
||||
return nFetchFlags;
|
||||
@@ -2289,7 +2330,7 @@ bool PeerManagerImpl::MaybeSendGetHeaders(CNode& pfrom, const CBlockLocator& loc
|
||||
* We require that the given tip have at least as much work as our tip, and for
|
||||
* our current tip to be "close to synced" (see CanDirectFetch()).
|
||||
*/
|
||||
void PeerManagerImpl::HeadersDirectFetchBlocks(CNode& pfrom, const CBlockIndex* pindexLast)
|
||||
void PeerManagerImpl::HeadersDirectFetchBlocks(CNode& pfrom, const Peer& peer, const CBlockIndex* pindexLast)
|
||||
{
|
||||
const CNetMsgMaker msgMaker(pfrom.GetCommonVersion());
|
||||
|
||||
@@ -2304,7 +2345,7 @@ void PeerManagerImpl::HeadersDirectFetchBlocks(CNode& pfrom, const CBlockIndex*
|
||||
while (pindexWalk && !m_chainman.ActiveChain().Contains(pindexWalk) && vToFetch.size() <= MAX_BLOCKS_IN_TRANSIT_PER_PEER) {
|
||||
if (!(pindexWalk->nStatus & BLOCK_HAVE_DATA) &&
|
||||
!IsBlockRequested(pindexWalk->GetBlockHash()) &&
|
||||
(!DeploymentActiveAt(*pindexWalk, m_chainman, Consensus::DEPLOYMENT_SEGWIT) || State(pfrom.GetId())->fHaveWitness)) {
|
||||
(!DeploymentActiveAt(*pindexWalk, m_chainman, Consensus::DEPLOYMENT_SEGWIT) || CanServeWitnesses(peer))) {
|
||||
// We don't have this block, and it's not yet in flight.
|
||||
vToFetch.push_back(pindexWalk);
|
||||
}
|
||||
@@ -2326,7 +2367,7 @@ void PeerManagerImpl::HeadersDirectFetchBlocks(CNode& pfrom, const CBlockIndex*
|
||||
// Can't download any more from this peer
|
||||
break;
|
||||
}
|
||||
uint32_t nFetchFlags = GetFetchFlags(pfrom);
|
||||
uint32_t nFetchFlags = GetFetchFlags(peer);
|
||||
vGetData.push_back(CInv(MSG_BLOCK | nFetchFlags, pindex->GetBlockHash()));
|
||||
BlockRequested(pfrom.GetId(), *pindex);
|
||||
LogPrint(BCLog::NET, "Requesting block %s from peer=%d\n",
|
||||
@@ -2471,7 +2512,7 @@ void PeerManagerImpl::ProcessHeadersMessage(CNode& pfrom, Peer& peer,
|
||||
UpdatePeerStateForReceivedHeaders(pfrom, pindexLast, received_new_header, nCount == MAX_HEADERS_RESULTS);
|
||||
|
||||
// Consider immediately downloading blocks.
|
||||
HeadersDirectFetchBlocks(pfrom, pindexLast);
|
||||
HeadersDirectFetchBlocks(pfrom, peer, pindexLast);
|
||||
|
||||
return;
|
||||
}
|
||||
@@ -2555,7 +2596,7 @@ void PeerManagerImpl::ProcessOrphanTx(std::set<uint256>& orphan_work_set)
|
||||
}
|
||||
}
|
||||
|
||||
bool PeerManagerImpl::PrepareBlockFilterRequest(CNode& peer,
|
||||
bool PeerManagerImpl::PrepareBlockFilterRequest(CNode& node, Peer& peer,
|
||||
BlockFilterType filter_type, uint32_t start_height,
|
||||
const uint256& stop_hash, uint32_t max_height_diff,
|
||||
const CBlockIndex*& stop_index,
|
||||
@@ -2563,11 +2604,11 @@ bool PeerManagerImpl::PrepareBlockFilterRequest(CNode& peer,
|
||||
{
|
||||
const bool supported_filter_type =
|
||||
(filter_type == BlockFilterType::BASIC &&
|
||||
(peer.GetLocalServices() & NODE_COMPACT_FILTERS));
|
||||
(peer.m_our_services & NODE_COMPACT_FILTERS));
|
||||
if (!supported_filter_type) {
|
||||
LogPrint(BCLog::NET, "peer %d requested unsupported block filter type: %d\n",
|
||||
peer.GetId(), static_cast<uint8_t>(filter_type));
|
||||
peer.fDisconnect = true;
|
||||
node.GetId(), static_cast<uint8_t>(filter_type));
|
||||
node.fDisconnect = true;
|
||||
return false;
|
||||
}
|
||||
|
||||
@@ -2578,8 +2619,8 @@ bool PeerManagerImpl::PrepareBlockFilterRequest(CNode& peer,
|
||||
// Check that the stop block exists and the peer would be allowed to fetch it.
|
||||
if (!stop_index || !BlockRequestAllowed(stop_index)) {
|
||||
LogPrint(BCLog::NET, "peer %d requested invalid block hash: %s\n",
|
||||
peer.GetId(), stop_hash.ToString());
|
||||
peer.fDisconnect = true;
|
||||
node.GetId(), stop_hash.ToString());
|
||||
node.fDisconnect = true;
|
||||
return false;
|
||||
}
|
||||
}
|
||||
@@ -2588,14 +2629,14 @@ bool PeerManagerImpl::PrepareBlockFilterRequest(CNode& peer,
|
||||
if (start_height > stop_height) {
|
||||
LogPrint(BCLog::NET, "peer %d sent invalid getcfilters/getcfheaders with " /* Continued */
|
||||
"start height %d and stop height %d\n",
|
||||
peer.GetId(), start_height, stop_height);
|
||||
peer.fDisconnect = true;
|
||||
node.GetId(), start_height, stop_height);
|
||||
node.fDisconnect = true;
|
||||
return false;
|
||||
}
|
||||
if (stop_height - start_height >= max_height_diff) {
|
||||
LogPrint(BCLog::NET, "peer %d requested too many cfilters/cfheaders: %d / %d\n",
|
||||
peer.GetId(), stop_height - start_height + 1, max_height_diff);
|
||||
peer.fDisconnect = true;
|
||||
node.GetId(), stop_height - start_height + 1, max_height_diff);
|
||||
node.fDisconnect = true;
|
||||
return false;
|
||||
}
|
||||
|
||||
@@ -2608,7 +2649,7 @@ bool PeerManagerImpl::PrepareBlockFilterRequest(CNode& peer,
|
||||
return true;
|
||||
}
|
||||
|
||||
void PeerManagerImpl::ProcessGetCFilters(CNode& peer, CDataStream& vRecv)
|
||||
void PeerManagerImpl::ProcessGetCFilters(CNode& node,Peer& peer, CDataStream& vRecv)
|
||||
{
|
||||
uint8_t filter_type_ser;
|
||||
uint32_t start_height;
|
||||
@@ -2620,7 +2661,7 @@ void PeerManagerImpl::ProcessGetCFilters(CNode& peer, CDataStream& vRecv)
|
||||
|
||||
const CBlockIndex* stop_index;
|
||||
BlockFilterIndex* filter_index;
|
||||
if (!PrepareBlockFilterRequest(peer, filter_type, start_height, stop_hash,
|
||||
if (!PrepareBlockFilterRequest(node, peer, filter_type, start_height, stop_hash,
|
||||
MAX_GETCFILTERS_SIZE, stop_index, filter_index)) {
|
||||
return;
|
||||
}
|
||||
@@ -2633,13 +2674,13 @@ void PeerManagerImpl::ProcessGetCFilters(CNode& peer, CDataStream& vRecv)
|
||||
}
|
||||
|
||||
for (const auto& filter : filters) {
|
||||
CSerializedNetMsg msg = CNetMsgMaker(peer.GetCommonVersion())
|
||||
CSerializedNetMsg msg = CNetMsgMaker(node.GetCommonVersion())
|
||||
.Make(NetMsgType::CFILTER, filter);
|
||||
m_connman.PushMessage(&peer, std::move(msg));
|
||||
m_connman.PushMessage(&node, std::move(msg));
|
||||
}
|
||||
}
|
||||
|
||||
void PeerManagerImpl::ProcessGetCFHeaders(CNode& peer, CDataStream& vRecv)
|
||||
void PeerManagerImpl::ProcessGetCFHeaders(CNode& node, Peer& peer, CDataStream& vRecv)
|
||||
{
|
||||
uint8_t filter_type_ser;
|
||||
uint32_t start_height;
|
||||
@@ -2651,7 +2692,7 @@ void PeerManagerImpl::ProcessGetCFHeaders(CNode& peer, CDataStream& vRecv)
|
||||
|
||||
const CBlockIndex* stop_index;
|
||||
BlockFilterIndex* filter_index;
|
||||
if (!PrepareBlockFilterRequest(peer, filter_type, start_height, stop_hash,
|
||||
if (!PrepareBlockFilterRequest(node, peer, filter_type, start_height, stop_hash,
|
||||
MAX_GETCFHEADERS_SIZE, stop_index, filter_index)) {
|
||||
return;
|
||||
}
|
||||
@@ -2674,16 +2715,16 @@ void PeerManagerImpl::ProcessGetCFHeaders(CNode& peer, CDataStream& vRecv)
|
||||
return;
|
||||
}
|
||||
|
||||
CSerializedNetMsg msg = CNetMsgMaker(peer.GetCommonVersion())
|
||||
CSerializedNetMsg msg = CNetMsgMaker(node.GetCommonVersion())
|
||||
.Make(NetMsgType::CFHEADERS,
|
||||
filter_type_ser,
|
||||
stop_index->GetBlockHash(),
|
||||
prev_header,
|
||||
filter_hashes);
|
||||
m_connman.PushMessage(&peer, std::move(msg));
|
||||
m_connman.PushMessage(&node, std::move(msg));
|
||||
}
|
||||
|
||||
void PeerManagerImpl::ProcessGetCFCheckPt(CNode& peer, CDataStream& vRecv)
|
||||
void PeerManagerImpl::ProcessGetCFCheckPt(CNode& node, Peer& peer, CDataStream& vRecv)
|
||||
{
|
||||
uint8_t filter_type_ser;
|
||||
uint256 stop_hash;
|
||||
@@ -2694,7 +2735,7 @@ void PeerManagerImpl::ProcessGetCFCheckPt(CNode& peer, CDataStream& vRecv)
|
||||
|
||||
const CBlockIndex* stop_index;
|
||||
BlockFilterIndex* filter_index;
|
||||
if (!PrepareBlockFilterRequest(peer, filter_type, /*start_height=*/0, stop_hash,
|
||||
if (!PrepareBlockFilterRequest(node, peer, filter_type, /*start_height=*/0, stop_hash,
|
||||
/*max_height_diff=*/std::numeric_limits<uint32_t>::max(),
|
||||
stop_index, filter_index)) {
|
||||
return;
|
||||
@@ -2715,12 +2756,12 @@ void PeerManagerImpl::ProcessGetCFCheckPt(CNode& peer, CDataStream& vRecv)
|
||||
}
|
||||
}
|
||||
|
||||
CSerializedNetMsg msg = CNetMsgMaker(peer.GetCommonVersion())
|
||||
CSerializedNetMsg msg = CNetMsgMaker(node.GetCommonVersion())
|
||||
.Make(NetMsgType::CFCHECKPT,
|
||||
filter_type_ser,
|
||||
stop_index->GetBlockHash(),
|
||||
headers);
|
||||
m_connman.PushMessage(&peer, std::move(msg));
|
||||
m_connman.PushMessage(&node, std::move(msg));
|
||||
}
|
||||
|
||||
void PeerManagerImpl::ProcessBlock(CNode& node, const std::shared_ptr<const CBlock>& block, bool force_processing)
|
||||
@@ -2842,7 +2883,8 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
|
||||
m_connman.PushMessage(&pfrom, msg_maker.Make(NetMsgType::VERACK));
|
||||
|
||||
pfrom.nServices = nServices;
|
||||
pfrom.m_has_all_wanted_services = HasAllDesirableServiceFlags(nServices);
|
||||
peer->m_their_services = nServices;
|
||||
pfrom.SetAddrLocal(addrMe);
|
||||
{
|
||||
LOCK(pfrom.m_subver_mutex);
|
||||
@@ -2850,18 +2892,12 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
}
|
||||
peer->m_starting_height = starting_height;
|
||||
|
||||
// set nodes not relaying blocks and tx and not serving (parts) of the historical blockchain as "clients"
|
||||
pfrom.fClient = (!(nServices & NODE_NETWORK) && !(nServices & NODE_NETWORK_LIMITED));
|
||||
|
||||
// set nodes not capable of serving the complete blockchain history as "limited nodes"
|
||||
pfrom.m_limited_node = (!(nServices & NODE_NETWORK) && (nServices & NODE_NETWORK_LIMITED));
|
||||
|
||||
// We only initialize the m_tx_relay data structure if:
|
||||
// - this isn't an outbound block-relay-only connection; and
|
||||
// - fRelay=true or we're offering NODE_BLOOM to this peer
|
||||
// (NODE_BLOOM means that the peer may turn on tx relay later)
|
||||
if (!pfrom.IsBlockOnlyConn() &&
|
||||
(fRelay || (pfrom.GetLocalServices() & NODE_BLOOM))) {
|
||||
(fRelay || (peer->m_our_services & NODE_BLOOM))) {
|
||||
auto* const tx_relay = peer->SetTxRelay();
|
||||
{
|
||||
LOCK(tx_relay->m_bloom_filter_mutex);
|
||||
@@ -2870,17 +2906,11 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
if (fRelay) pfrom.m_relays_txs = true;
|
||||
}
|
||||
|
||||
if((nServices & NODE_WITNESS))
|
||||
{
|
||||
LOCK(cs_main);
|
||||
State(pfrom.GetId())->fHaveWitness = true;
|
||||
}
|
||||
|
||||
// Potentially mark this peer as a preferred download peer.
|
||||
{
|
||||
LOCK(cs_main);
|
||||
CNodeState* state = State(pfrom.GetId());
|
||||
state->fPreferredDownload = (!pfrom.IsInboundConn() || pfrom.HasPermission(NetPermissionFlags::NoBan)) && !pfrom.IsAddrFetchConn() && !pfrom.fClient;
|
||||
state->fPreferredDownload = (!pfrom.IsInboundConn() || pfrom.HasPermission(NetPermissionFlags::NoBan)) && !pfrom.IsAddrFetchConn() && CanServeBlocks(*peer);
|
||||
m_num_preferred_download_peers += state->fPreferredDownload;
|
||||
}
|
||||
|
||||
@@ -2899,7 +2929,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
// indicate to the peer that we will participate in addr relay.
|
||||
if (fListen && !m_chainman.ActiveChainstate().IsInitialBlockDownload())
|
||||
{
|
||||
CAddress addr = GetLocalAddress(&pfrom.addr, pfrom.GetLocalServices());
|
||||
CAddress addr{GetLocalAddress(pfrom.addr), peer->m_our_services, (uint32_t)GetAdjustedTime()};
|
||||
FastRandomContext insecure_rand;
|
||||
if (addr.IsRoutable())
|
||||
{
|
||||
@@ -3758,7 +3788,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
// We requested this block for some reason, but our mempool will probably be useless
|
||||
// so we just grab the block via normal getdata
|
||||
std::vector<CInv> vInv(1);
|
||||
vInv[0] = CInv(MSG_BLOCK | GetFetchFlags(pfrom), cmpctblock.header.GetHash());
|
||||
vInv[0] = CInv(MSG_BLOCK | GetFetchFlags(*peer), cmpctblock.header.GetHash());
|
||||
m_connman.PushMessage(&pfrom, msgMaker.Make(NetMsgType::GETDATA, vInv));
|
||||
}
|
||||
return;
|
||||
@@ -3794,7 +3824,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
} else if (status == READ_STATUS_FAILED) {
|
||||
// Duplicate txindexes, the block is now in-flight, so just request it
|
||||
std::vector<CInv> vInv(1);
|
||||
vInv[0] = CInv(MSG_BLOCK | GetFetchFlags(pfrom), cmpctblock.header.GetHash());
|
||||
vInv[0] = CInv(MSG_BLOCK | GetFetchFlags(*peer), cmpctblock.header.GetHash());
|
||||
m_connman.PushMessage(&pfrom, msgMaker.Make(NetMsgType::GETDATA, vInv));
|
||||
return;
|
||||
}
|
||||
@@ -3837,7 +3867,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
// We requested this block, but its far into the future, so our
|
||||
// mempool will probably be useless - request the block normally
|
||||
std::vector<CInv> vInv(1);
|
||||
vInv[0] = CInv(MSG_BLOCK | GetFetchFlags(pfrom), cmpctblock.header.GetHash());
|
||||
vInv[0] = CInv(MSG_BLOCK | GetFetchFlags(*peer), cmpctblock.header.GetHash());
|
||||
m_connman.PushMessage(&pfrom, msgMaker.Make(NetMsgType::GETDATA, vInv));
|
||||
return;
|
||||
} else {
|
||||
@@ -3921,7 +3951,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
} else if (status == READ_STATUS_FAILED) {
|
||||
// Might have collided, fall back to getdata now :(
|
||||
std::vector<CInv> invs;
|
||||
invs.push_back(CInv(MSG_BLOCK | GetFetchFlags(pfrom), resp.blockhash));
|
||||
invs.push_back(CInv(MSG_BLOCK | GetFetchFlags(*peer), resp.blockhash));
|
||||
m_connman.PushMessage(&pfrom, msgMaker.Make(NetMsgType::GETDATA, invs));
|
||||
} else {
|
||||
// Block is either okay, or possibly we received
|
||||
@@ -4061,7 +4091,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
}
|
||||
|
||||
if (msg_type == NetMsgType::MEMPOOL) {
|
||||
if (!(pfrom.GetLocalServices() & NODE_BLOOM) && !pfrom.HasPermission(NetPermissionFlags::Mempool))
|
||||
if (!(peer->m_our_services & NODE_BLOOM) && !pfrom.HasPermission(NetPermissionFlags::Mempool))
|
||||
{
|
||||
if (!pfrom.HasPermission(NetPermissionFlags::NoBan))
|
||||
{
|
||||
@@ -4164,7 +4194,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
}
|
||||
|
||||
if (msg_type == NetMsgType::FILTERLOAD) {
|
||||
if (!(pfrom.GetLocalServices() & NODE_BLOOM)) {
|
||||
if (!(peer->m_our_services & NODE_BLOOM)) {
|
||||
LogPrint(BCLog::NET, "filterload received despite not offering bloom services from peer=%d; disconnecting\n", pfrom.GetId());
|
||||
pfrom.fDisconnect = true;
|
||||
return;
|
||||
@@ -4189,7 +4219,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
}
|
||||
|
||||
if (msg_type == NetMsgType::FILTERADD) {
|
||||
if (!(pfrom.GetLocalServices() & NODE_BLOOM)) {
|
||||
if (!(peer->m_our_services & NODE_BLOOM)) {
|
||||
LogPrint(BCLog::NET, "filteradd received despite not offering bloom services from peer=%d; disconnecting\n", pfrom.GetId());
|
||||
pfrom.fDisconnect = true;
|
||||
return;
|
||||
@@ -4217,7 +4247,7 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
}
|
||||
|
||||
if (msg_type == NetMsgType::FILTERCLEAR) {
|
||||
if (!(pfrom.GetLocalServices() & NODE_BLOOM)) {
|
||||
if (!(peer->m_our_services & NODE_BLOOM)) {
|
||||
LogPrint(BCLog::NET, "filterclear received despite not offering bloom services from peer=%d; disconnecting\n", pfrom.GetId());
|
||||
pfrom.fDisconnect = true;
|
||||
return;
|
||||
@@ -4248,17 +4278,17 @@ void PeerManagerImpl::ProcessMessage(CNode& pfrom, const std::string& msg_type,
|
||||
}
|
||||
|
||||
if (msg_type == NetMsgType::GETCFILTERS) {
|
||||
ProcessGetCFilters(pfrom, vRecv);
|
||||
ProcessGetCFilters(pfrom, *peer, vRecv);
|
||||
return;
|
||||
}
|
||||
|
||||
if (msg_type == NetMsgType::GETCFHEADERS) {
|
||||
ProcessGetCFHeaders(pfrom, vRecv);
|
||||
ProcessGetCFHeaders(pfrom, *peer, vRecv);
|
||||
return;
|
||||
}
|
||||
|
||||
if (msg_type == NetMsgType::GETCFCHECKPT) {
|
||||
ProcessGetCFCheckPt(pfrom, vRecv);
|
||||
ProcessGetCFCheckPt(pfrom, *peer, vRecv);
|
||||
return;
|
||||
}
|
||||
|
||||
@@ -4654,9 +4684,10 @@ void PeerManagerImpl::MaybeSendAddr(CNode& node, Peer& peer, std::chrono::micros
|
||||
if (peer.m_next_local_addr_send != 0us) {
|
||||
peer.m_addr_known->reset();
|
||||
}
|
||||
if (std::optional<CAddress> local_addr = GetLocalAddrForPeer(&node)) {
|
||||
if (std::optional<CService> local_service = GetLocalAddrForPeer(node)) {
|
||||
CAddress local_addr{*local_service, peer.m_our_services, (uint32_t)GetAdjustedTime()};
|
||||
FastRandomContext insecure_rand;
|
||||
PushAddress(peer, *local_addr, insecure_rand);
|
||||
PushAddress(peer, local_addr, insecure_rand);
|
||||
}
|
||||
peer.m_next_local_addr_send = GetExponentialRand(current_time, AVG_LOCAL_ADDRESS_BROADCAST_INTERVAL);
|
||||
}
|
||||
@@ -4840,7 +4871,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
|
||||
bool sync_blocks_and_headers_from_peer = false;
|
||||
if (state.fPreferredDownload) {
|
||||
sync_blocks_and_headers_from_peer = true;
|
||||
} else if (!pto->fClient && !pto->IsAddrFetchConn()) {
|
||||
} else if (CanServeBlocks(*peer) && !pto->IsAddrFetchConn()) {
|
||||
// Typically this is an inbound peer. If we don't have any outbound
|
||||
// peers, or if we aren't downloading any blocks from such peers,
|
||||
// then allow block downloads from this peer, too.
|
||||
@@ -4855,7 +4886,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
|
||||
}
|
||||
}
|
||||
|
||||
if (!state.fSyncStarted && !pto->fClient && !fImporting && !fReindex) {
|
||||
if (!state.fSyncStarted && CanServeBlocks(*peer) && !fImporting && !fReindex) {
|
||||
// Only actively request headers from a single peer, unless we're close to today.
|
||||
if ((nSyncStarted == 0 && sync_blocks_and_headers_from_peer) || m_chainman.m_best_header->GetBlockTime() > GetAdjustedTime() - 24 * 60 * 60) {
|
||||
const CBlockIndex* pindexStart = m_chainman.m_best_header;
|
||||
@@ -5232,12 +5263,12 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
|
||||
// Message: getdata (blocks)
|
||||
//
|
||||
std::vector<CInv> vGetData;
|
||||
if (!pto->fClient && ((sync_blocks_and_headers_from_peer && !pto->m_limited_node) || !m_chainman.ActiveChainstate().IsInitialBlockDownload()) && state.nBlocksInFlight < MAX_BLOCKS_IN_TRANSIT_PER_PEER) {
|
||||
if (CanServeBlocks(*peer) && ((sync_blocks_and_headers_from_peer && !IsLimitedPeer(*peer)) || !m_chainman.ActiveChainstate().IsInitialBlockDownload()) && state.nBlocksInFlight < MAX_BLOCKS_IN_TRANSIT_PER_PEER) {
|
||||
std::vector<const CBlockIndex*> vToDownload;
|
||||
NodeId staller = -1;
|
||||
FindNextBlocksToDownload(pto->GetId(), MAX_BLOCKS_IN_TRANSIT_PER_PEER - state.nBlocksInFlight, vToDownload, staller);
|
||||
FindNextBlocksToDownload(*peer, MAX_BLOCKS_IN_TRANSIT_PER_PEER - state.nBlocksInFlight, vToDownload, staller);
|
||||
for (const CBlockIndex *pindex : vToDownload) {
|
||||
uint32_t nFetchFlags = GetFetchFlags(*pto);
|
||||
uint32_t nFetchFlags = GetFetchFlags(*peer);
|
||||
vGetData.push_back(CInv(MSG_BLOCK | nFetchFlags, pindex->GetBlockHash()));
|
||||
BlockRequested(pto->GetId(), *pindex);
|
||||
LogPrint(BCLog::NET, "Requesting block %s (%d) peer=%d\n", pindex->GetBlockHash().ToString(),
|
||||
@@ -5264,7 +5295,7 @@ bool PeerManagerImpl::SendMessages(CNode* pto)
|
||||
if (!AlreadyHaveTx(gtxid)) {
|
||||
LogPrint(BCLog::NET, "Requesting %s %s peer=%d\n", gtxid.IsWtxid() ? "wtx" : "tx",
|
||||
gtxid.GetHash().ToString(), pto->GetId());
|
||||
vGetData.emplace_back(gtxid.IsWtxid() ? MSG_WTX : (MSG_TX | GetFetchFlags(*pto)), gtxid.GetHash());
|
||||
vGetData.emplace_back(gtxid.IsWtxid() ? MSG_WTX : (MSG_TX | GetFetchFlags(*peer)), gtxid.GetHash());
|
||||
if (vGetData.size() >= MAX_GETDATA_SZ) {
|
||||
m_connman.PushMessage(pto, msgMaker.Make(NetMsgType::GETDATA, vGetData));
|
||||
vGetData.clear();
|
||||
|
||||
Reference in New Issue
Block a user