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:
    ACK 8d8eeb422e 🔑
  jnewbery:
    utACK 8d8eeb422e
  mzumsande:
    Code Review ACK 8d8eeb422e

Tree-SHA512: e772eb2a0a85db346dd7b453a41011a12756fc7cbfda6a9ef6daa9633b9a47b9770ab3dc02377690f9d02127301c3905ff22905977f758bf90b17a9a35b37523
This commit is contained in:
MacroFake 2022-07-19 08:32:30 +02:00
commit 2bdce7f7ad
No known key found for this signature in database
GPG key ID: CE2B75697E69A548
13 changed files with 199 additions and 203 deletions

View file

@ -206,15 +206,13 @@ static std::vector<CAddress> ConvertSeeds(const std::vector<uint8_t> &vSeedsIn)
// Otherwise, return the unroutable 0.0.0.0 but filled in with
// the normal parameters, since the IP may be changed to a useful
// one by discovery.
CAddress GetLocalAddress(const CNetAddr *paddrPeer, ServiceFlags nLocalServices)
CService GetLocalAddress(const CNetAddr& addrPeer)
{
CAddress ret(CService(CNetAddr(),GetListenPort()), nLocalServices);
CService ret{CNetAddr(), GetListenPort()};
CService addr;
if (GetLocal(addr, paddrPeer))
{
ret = CAddress(addr, nLocalServices);
if (GetLocal(addr, &addrPeer)) {
ret = CService{addr};
}
ret.nTime = GetAdjustedTime();
return ret;
}
@ -233,35 +231,35 @@ bool IsPeerAddrLocalGood(CNode *pnode)
IsReachable(addrLocal.GetNetwork());
}
std::optional<CAddress> GetLocalAddrForPeer(CNode *pnode)
std::optional<CService> GetLocalAddrForPeer(CNode& node)
{
CAddress addrLocal = GetLocalAddress(&pnode->addr, pnode->GetLocalServices());
CService addrLocal{GetLocalAddress(node.addr)};
if (gArgs.GetBoolArg("-addrmantest", false)) {
// use IPv4 loopback during addrmantest
addrLocal = CAddress(CService(LookupNumeric("127.0.0.1", GetListenPort())), pnode->GetLocalServices());
addrLocal = CService(LookupNumeric("127.0.0.1", GetListenPort()));
}
// If discovery is enabled, sometimes give our peer the address it
// tells us that it sees us as in case it has a better idea of our
// address than we do.
FastRandomContext rng;
if (IsPeerAddrLocalGood(pnode) && (!addrLocal.IsRoutable() ||
if (IsPeerAddrLocalGood(&node) && (!addrLocal.IsRoutable() ||
rng.randbits((GetnScore(addrLocal) > LOCAL_MANUAL) ? 3 : 1) == 0))
{
if (pnode->IsInboundConn()) {
if (node.IsInboundConn()) {
// For inbound connections, assume both the address and the port
// as seen from the peer.
addrLocal = CAddress{pnode->GetAddrLocal(), addrLocal.nServices, addrLocal.nTime};
addrLocal = CService{node.GetAddrLocal()};
} else {
// For outbound connections, assume just the address as seen from
// the peer and leave the port in `addrLocal` as returned by
// `GetLocalAddress()` above. The peer has no way to observe our
// listening port when we have initiated the connection.
addrLocal.SetIP(pnode->GetAddrLocal());
addrLocal.SetIP(node.GetAddrLocal());
}
}
if (addrLocal.IsRoutable() || gArgs.GetBoolArg("-addrmantest", false))
{
LogPrint(BCLog::NET, "Advertising address %s to peer=%d\n", addrLocal.ToString(), pnode->GetId());
LogPrint(BCLog::NET, "Advertising address %s to peer=%d\n", addrLocal.ToString(), node.GetId());
return addrLocal;
}
// Address is unroutable. Don't advertise.
@ -543,7 +541,6 @@ CNode* CConnman::ConnectNode(CAddress addrConnect, const char *pszDest, bool fCo
addr_bind = GetBindAddress(*sock);
}
CNode* pnode = new CNode(id,
nLocalServices,
std::move(sock),
addrConnect,
CalculateKeyedNetGroup(addrConnect),
@ -603,7 +600,6 @@ Network CNode::ConnectedThroughNetwork() const
void CNode::CopyStats(CNodeStats& stats)
{
stats.nodeid = this->GetId();
X(nServices);
X(addr);
X(addrBind);
stats.m_network = ConnectedThroughNetwork();
@ -880,7 +876,7 @@ bool CConnman::AttemptToEvictConnection()
.m_min_ping_time = node->m_min_ping_time,
.m_last_block_time = node->m_last_block_time,
.m_last_tx_time = node->m_last_tx_time,
.fRelevantServices = HasAllDesirableServiceFlags(node->nServices),
.fRelevantServices = node->m_has_all_wanted_services,
.m_relay_txs = node->m_relays_txs.load(),
.fBloomFilter = node->m_bloom_filter_loaded.load(),
.nKeyedNetGroup = node->nKeyedNetGroup,
@ -1014,7 +1010,6 @@ void CConnman::CreateNodeFromAcceptedSocket(std::unique_ptr<Sock>&& sock,
const bool inbound_onion = std::find(m_onion_binds.begin(), m_onion_binds.end(), addr_bind) != m_onion_binds.end();
CNode* pnode = new CNode(id,
nodeServices,
std::move(sock),
addr,
CalculateKeyedNetGroup(addr),
@ -1026,7 +1021,7 @@ void CConnman::CreateNodeFromAcceptedSocket(std::unique_ptr<Sock>&& sock,
pnode->AddRef();
pnode->m_permissionFlags = permissionFlags;
pnode->m_prefer_evict = discouraged;
m_msgproc->InitializeNode(pnode);
m_msgproc->InitializeNode(*pnode, nodeServices);
LogPrint(BCLog::NET, "connection from %s accepted\n", addr.ToString());
@ -1964,7 +1959,7 @@ void CConnman::OpenNetworkConnection(const CAddress& addrConnect, bool fCountFai
if (grantOutbound)
grantOutbound->MoveTo(pnode->grantOutbound);
m_msgproc->InitializeNode(pnode);
m_msgproc->InitializeNode(*pnode, nLocalServices);
{
LOCK(m_nodes_mutex);
m_nodes.push_back(pnode);
@ -2708,7 +2703,10 @@ ServiceFlags CConnman::GetLocalServices() const
unsigned int CConnman::GetReceiveFloodSize() const { return nReceiveFloodSize; }
CNode::CNode(NodeId idIn, ServiceFlags nLocalServicesIn, std::shared_ptr<Sock> sock, const CAddress& addrIn, uint64_t nKeyedNetGroupIn, uint64_t nLocalHostNonceIn, const CAddress& addrBindIn, const std::string& addrNameIn, ConnectionType conn_type_in, bool inbound_onion)
CNode::CNode(NodeId idIn, std::shared_ptr<Sock> sock, const CAddress& addrIn,
uint64_t nKeyedNetGroupIn, uint64_t nLocalHostNonceIn,
const CAddress& addrBindIn, const std::string& addrNameIn,
ConnectionType conn_type_in, bool inbound_onion)
: m_sock{sock},
m_connected{GetTime<std::chrono::seconds>()},
addr(addrIn),
@ -2718,8 +2716,7 @@ CNode::CNode(NodeId idIn, ServiceFlags nLocalServicesIn, std::shared_ptr<Sock> s
nKeyedNetGroup(nKeyedNetGroupIn),
id(idIn),
nLocalHostNonce(nLocalHostNonceIn),
m_conn_type(conn_type_in),
nLocalServices(nLocalServicesIn)
m_conn_type(conn_type_in)
{
if (inbound_onion) assert(conn_type_in == ConnectionType::INBOUND);

View file

@ -144,8 +144,8 @@ enum
};
bool IsPeerAddrLocalGood(CNode *pnode);
/** Returns a local address that we should advertise to this peer */
std::optional<CAddress> GetLocalAddrForPeer(CNode *pnode);
/** Returns a local address that we should advertise to this peer. */
std::optional<CService> GetLocalAddrForPeer(CNode& node);
/**
* Mark a network as reachable or unreachable (no automatic connects to it)
@ -163,7 +163,7 @@ void RemoveLocal(const CService& addr);
bool SeenLocal(const CService& addr);
bool IsLocal(const CService& addr);
bool GetLocal(CService &addr, const CNetAddr *paddrPeer = nullptr);
CAddress GetLocalAddress(const CNetAddr *paddrPeer, ServiceFlags nLocalServices);
CService GetLocalAddress(const CNetAddr& addrPeer);
extern bool fDiscover;
@ -187,7 +187,6 @@ class CNodeStats
{
public:
NodeId nodeid;
ServiceFlags nServices;
std::chrono::seconds m_last_send;
std::chrono::seconds m_last_recv;
std::chrono::seconds m_last_tx_time;
@ -346,7 +345,6 @@ public:
std::unique_ptr<TransportSerializer> m_serializer;
NetPermissionFlags m_permissionFlags{NetPermissionFlags::None};
std::atomic<ServiceFlags> nServices{NODE_NONE};
/**
* Socket used for communication with the node.
@ -399,8 +397,6 @@ public:
bool HasPermission(NetPermissionFlags permission) const {
return NetPermissions::HasFlag(m_permissionFlags, permission);
}
bool fClient{false}; // set by version message
bool m_limited_node{false}; //after BIP159, set by version message
/** fSuccessfullyConnected is set to true on receiving VERACK from the peer. */
std::atomic_bool fSuccessfullyConnected{false};
// Setting fDisconnect to true will cause the node to be disconnected the
@ -484,6 +480,9 @@ public:
// Peer selected us as (compact blocks) high-bandwidth peer (BIP152)
std::atomic<bool> m_bip152_highbandwidth_from{false};
/** Whether this peer provides all services that we want. Used for eviction decisions */
std::atomic_bool m_has_all_wanted_services{false};
/** Whether we should relay transactions to this peer (their version
* message did not include fRelay=false and this is not a block-relay-only
* connection). This only changes from false to true. It will never change
@ -514,7 +513,10 @@ public:
* criterium in CConnman::AttemptToEvictConnection. */
std::atomic<std::chrono::microseconds> m_min_ping_time{std::chrono::microseconds::max()};
CNode(NodeId id, ServiceFlags nLocalServicesIn, std::shared_ptr<Sock> sock, const CAddress& addrIn, uint64_t nKeyedNetGroupIn, uint64_t nLocalHostNonceIn, const CAddress& addrBindIn, const std::string& addrNameIn, ConnectionType conn_type_in, bool inbound_onion);
CNode(NodeId id, std::shared_ptr<Sock> sock, const CAddress& addrIn,
uint64_t nKeyedNetGroupIn, uint64_t nLocalHostNonceIn,
const CAddress& addrBindIn, const std::string& addrNameIn,
ConnectionType conn_type_in, bool inbound_onion);
CNode(const CNode&) = delete;
CNode& operator=(const CNode&) = delete;
@ -572,11 +574,6 @@ public:
void CopyStats(CNodeStats& stats) EXCLUSIVE_LOCKS_REQUIRED(!m_subver_mutex, !m_addr_local_mutex, !cs_vSend, !cs_vRecv);
ServiceFlags GetLocalServices() const
{
return nLocalServices;
}
std::string ConnectionTypeAsString() const { return ::ConnectionTypeAsString(m_conn_type); }
/** A ping-pong round trip has completed successfully. Update latest and minimum ping times. */
@ -591,23 +588,6 @@ private:
const ConnectionType m_conn_type;
std::atomic<int> m_greatest_common_version{INIT_PROTO_VERSION};
//! Services offered to this peer.
//!
//! This is supplied by the parent CConnman during peer connection
//! (CConnman::ConnectNode()) from its attribute of the same name.
//!
//! This is 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 nLocalServices;
std::list<CNetMessage> vRecvMsg; // Used only by SocketHandler thread
// Our address, as reported by the peer
@ -625,7 +605,7 @@ class NetEventsInterface
{
public:
/** Initialize a peer (setup state, queue any initial messages) */
virtual void InitializeNode(CNode* pnode) = 0;
virtual void InitializeNode(CNode& node, ServiceFlags our_services) = 0;
/** Handle removal of a peer (clear state) */
virtual void FinalizeNode(const CNode& node) = 0;
@ -1035,16 +1015,14 @@ private:
std::map<uint64_t, CachedAddrResponse> m_addr_response_caches;
/**
* Services this instance offers.
* Services this node offers.
*
* This data is replicated in each CNode instance we create during peer
* connection (in ConnectNode()) under a member also called
* nLocalServices.
* This data is replicated in each Peer instance we create.
*
* This data is not marked const, but after being set it should not
* change. See the note in CNode::nLocalServices documentation.
* change.
*
* \sa CNode::nLocalServices
* \sa Peer::our_services
*/
ServiceFlags nLocalServices;

View file

@ -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();

View file

@ -34,6 +34,7 @@ struct CNodeStateStats {
uint64_t m_addr_processed = 0;
uint64_t m_addr_rate_limited = 0;
bool m_addr_relay_enabled{false};
ServiceFlags their_services;
};
class PeerManager : public CValidationInterface, public NetEventsInterface

View file

@ -1162,7 +1162,6 @@ void RPCConsole::updateDetailWidget()
if (!stats->nodeStats.addrLocal.empty())
peerAddrDetails += "<br />" + tr("via %1").arg(QString::fromStdString(stats->nodeStats.addrLocal));
ui->peerHeading->setText(peerAddrDetails);
ui->peerServices->setText(GUIUtil::formatServicesStr(stats->nodeStats.nServices));
QString bip152_hb_settings;
if (stats->nodeStats.m_bip152_highbandwidth_to) bip152_hb_settings = ts.to;
if (stats->nodeStats.m_bip152_highbandwidth_from) bip152_hb_settings += (bip152_hb_settings.isEmpty() ? ts.from : QLatin1Char('/') + ts.from);
@ -1197,6 +1196,7 @@ void RPCConsole::updateDetailWidget()
// This check fails for example if the lock was busy and
// nodeStateStats couldn't be fetched.
if (stats->fNodeStateStatsAvailable) {
ui->peerServices->setText(GUIUtil::formatServicesStr(stats->nodeStateStats.their_services));
// Sync height is init to -1
if (stats->nodeStateStats.nSyncHeight > -1) {
ui->peerSyncHeight->setText(QString("%1").arg(stats->nodeStateStats.nSyncHeight));

View file

@ -195,8 +195,9 @@ static RPCHelpMan getpeerinfo()
if (stats.m_mapped_as != 0) {
obj.pushKV("mapped_as", uint64_t(stats.m_mapped_as));
}
obj.pushKV("services", strprintf("%016x", stats.nServices));
obj.pushKV("servicesnames", GetServicesNames(stats.nServices));
ServiceFlags services{fStateStats ? statestats.their_services : ServiceFlags::NODE_NONE};
obj.pushKV("services", strprintf("%016x", services));
obj.pushKV("servicesnames", GetServicesNames(services));
obj.pushKV("lastsend", count_seconds(stats.m_last_send));
obj.pushKV("lastrecv", count_seconds(stats.m_last_recv));
obj.pushKV("last_transaction", count_seconds(stats.m_last_tx_time));

View file

@ -15,6 +15,7 @@
#include <serialize.h>
#include <test/util/net.h>
#include <test/util/setup_common.h>
#include <timedata.h>
#include <util/string.h>
#include <util/system.h>
#include <util/time.h>
@ -44,17 +45,15 @@ BOOST_FIXTURE_TEST_SUITE(denialofservice_tests, TestingSetup)
// work.
BOOST_AUTO_TEST_CASE(outbound_slow_chain_eviction)
{
auto connman = std::make_unique<CConnman>(0x1337, 0x1337, *m_node.addrman, *m_node.netgroupman);
ConnmanTestMsg& connman = static_cast<ConnmanTestMsg&>(*m_node.connman);
// Disable inactivity checks for this test to avoid interference
static_cast<ConnmanTestMsg*>(connman.get())->SetPeerConnectTimeout(99999s);
auto peerLogic = PeerManager::make(*connman, *m_node.addrman, nullptr,
*m_node.chainman, *m_node.mempool, false);
connman.SetPeerConnectTimeout(99999s);
PeerManager& peerman = *m_node.peerman;
// Mock an outbound peer
CAddress addr1(ip(0xa0b0c001), NODE_NONE);
NodeId id{0};
CNode dummyNode1{id++,
ServiceFlags(NODE_NETWORK | NODE_WITNESS),
/*sock=*/nullptr,
addr1,
/*nKeyedNetGroupIn=*/0,
@ -63,10 +62,16 @@ BOOST_AUTO_TEST_CASE(outbound_slow_chain_eviction)
/*addrNameIn=*/"",
ConnectionType::OUTBOUND_FULL_RELAY,
/*inbound_onion=*/false};
dummyNode1.SetCommonVersion(PROTOCOL_VERSION);
peerLogic->InitializeNode(&dummyNode1);
dummyNode1.fSuccessfullyConnected = true;
connman.Handshake(
/*node=*/dummyNode1,
/*successfully_connected=*/true,
/*remote_services=*/ServiceFlags(NODE_NETWORK | NODE_WITNESS),
/*local_services=*/ServiceFlags(NODE_NETWORK | NODE_WITNESS),
/*permission_flags=*/NetPermissionFlags::None,
/*version=*/PROTOCOL_VERSION,
/*relay_txs=*/true);
TestOnlyResetTimeData();
// This test requires that we have a chain with non-zero work.
{
@ -78,7 +83,7 @@ BOOST_AUTO_TEST_CASE(outbound_slow_chain_eviction)
// Test starts here
{
LOCK(dummyNode1.cs_sendProcessing);
BOOST_CHECK(peerLogic->SendMessages(&dummyNode1)); // should result in getheaders
BOOST_CHECK(peerman.SendMessages(&dummyNode1)); // should result in getheaders
}
{
LOCK(dummyNode1.cs_vSend);
@ -91,7 +96,7 @@ BOOST_AUTO_TEST_CASE(outbound_slow_chain_eviction)
SetMockTime(nStartTime+21*60);
{
LOCK(dummyNode1.cs_sendProcessing);
BOOST_CHECK(peerLogic->SendMessages(&dummyNode1)); // should result in getheaders
BOOST_CHECK(peerman.SendMessages(&dummyNode1)); // should result in getheaders
}
{
LOCK(dummyNode1.cs_vSend);
@ -101,18 +106,17 @@ BOOST_AUTO_TEST_CASE(outbound_slow_chain_eviction)
SetMockTime(nStartTime+24*60);
{
LOCK(dummyNode1.cs_sendProcessing);
BOOST_CHECK(peerLogic->SendMessages(&dummyNode1)); // should result in disconnect
BOOST_CHECK(peerman.SendMessages(&dummyNode1)); // should result in disconnect
}
BOOST_CHECK(dummyNode1.fDisconnect == true);
peerLogic->FinalizeNode(dummyNode1);
peerman.FinalizeNode(dummyNode1);
}
static void AddRandomOutboundPeer(NodeId& id, std::vector<CNode*>& vNodes, PeerManager& peerLogic, ConnmanTestMsg& connman, ConnectionType connType)
{
CAddress addr(ip(g_insecure_rand_ctx.randbits(32)), NODE_NONE);
vNodes.emplace_back(new CNode{id++,
ServiceFlags(NODE_NETWORK | NODE_WITNESS),
/*sock=*/nullptr,
addr,
/*nKeyedNetGroupIn=*/0,
@ -124,7 +128,7 @@ static void AddRandomOutboundPeer(NodeId& id, std::vector<CNode*>& vNodes, PeerM
CNode &node = *vNodes.back();
node.SetCommonVersion(PROTOCOL_VERSION);
peerLogic.InitializeNode(&node);
peerLogic.InitializeNode(node, ServiceFlags(NODE_NETWORK | NODE_WITNESS));
node.fSuccessfullyConnected = true;
connman.AddTestNode(node);
@ -292,7 +296,6 @@ BOOST_AUTO_TEST_CASE(peer_discouragement)
banman->ClearBanned();
NodeId id{0};
nodes[0] = new CNode{id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr[0],
/*nKeyedNetGroupIn=*/0,
@ -302,7 +305,7 @@ BOOST_AUTO_TEST_CASE(peer_discouragement)
ConnectionType::INBOUND,
/*inbound_onion=*/false};
nodes[0]->SetCommonVersion(PROTOCOL_VERSION);
peerLogic->InitializeNode(nodes[0]);
peerLogic->InitializeNode(*nodes[0], NODE_NETWORK);
nodes[0]->fSuccessfullyConnected = true;
connman->AddTestNode(*nodes[0]);
peerLogic->UnitTestMisbehaving(nodes[0]->GetId(), DISCOURAGEMENT_THRESHOLD); // Should be discouraged
@ -315,7 +318,6 @@ BOOST_AUTO_TEST_CASE(peer_discouragement)
BOOST_CHECK(!banman->IsDiscouraged(other_addr)); // Different address, not discouraged
nodes[1] = new CNode{id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr[1],
/*nKeyedNetGroupIn=*/1,
@ -325,7 +327,7 @@ BOOST_AUTO_TEST_CASE(peer_discouragement)
ConnectionType::INBOUND,
/*inbound_onion=*/false};
nodes[1]->SetCommonVersion(PROTOCOL_VERSION);
peerLogic->InitializeNode(nodes[1]);
peerLogic->InitializeNode(*nodes[1], NODE_NETWORK);
nodes[1]->fSuccessfullyConnected = true;
connman->AddTestNode(*nodes[1]);
peerLogic->UnitTestMisbehaving(nodes[1]->GetId(), DISCOURAGEMENT_THRESHOLD - 1);
@ -353,7 +355,6 @@ BOOST_AUTO_TEST_CASE(peer_discouragement)
// Make sure non-IP peers are discouraged and disconnected properly.
nodes[2] = new CNode{id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr[2],
/*nKeyedNetGroupIn=*/1,
@ -363,7 +364,7 @@ BOOST_AUTO_TEST_CASE(peer_discouragement)
ConnectionType::OUTBOUND_FULL_RELAY,
/*inbound_onion=*/false};
nodes[2]->SetCommonVersion(PROTOCOL_VERSION);
peerLogic->InitializeNode(nodes[2]);
peerLogic->InitializeNode(*nodes[2], NODE_NETWORK);
nodes[2]->fSuccessfullyConnected = true;
connman->AddTestNode(*nodes[2]);
peerLogic->UnitTestMisbehaving(nodes[2]->GetId(), DISCOURAGEMENT_THRESHOLD);
@ -398,7 +399,6 @@ BOOST_AUTO_TEST_CASE(DoS_bantime)
CAddress addr(ip(0xa0b0c001), NODE_NONE);
NodeId id{0};
CNode dummyNode{id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr,
/*nKeyedNetGroupIn=*/4,
@ -408,7 +408,7 @@ BOOST_AUTO_TEST_CASE(DoS_bantime)
ConnectionType::INBOUND,
/*inbound_onion=*/false};
dummyNode.SetCommonVersion(PROTOCOL_VERSION);
peerLogic->InitializeNode(&dummyNode);
peerLogic->InitializeNode(dummyNode, NODE_NETWORK);
dummyNode.fSuccessfullyConnected = true;
peerLogic->UnitTestMisbehaving(dummyNode.GetId(), DISCOURAGEMENT_THRESHOLD);

View file

@ -68,7 +68,6 @@ FUZZ_TARGET_INIT(net, initialize_net)
(void)node.GetAddrLocal();
(void)node.GetId();
(void)node.GetLocalNonce();
(void)node.GetLocalServices();
const int ref_count = node.GetRefCount();
assert(ref_count >= 0);
(void)node.GetCommonVersion();

View file

@ -294,6 +294,7 @@ void FillNode(FuzzedDataProvider& fuzzed_data_provider, ConnmanTestMsg& connman,
connman.Handshake(node,
/*successfully_connected=*/fuzzed_data_provider.ConsumeBool(),
/*remote_services=*/ConsumeWeakEnum(fuzzed_data_provider, ALL_SERVICE_FLAGS),
/*local_services=*/ConsumeWeakEnum(fuzzed_data_provider, ALL_SERVICE_FLAGS),
/*permission_flags=*/ConsumeWeakEnum(fuzzed_data_provider, ALL_NET_PERMISSION_FLAGS),
/*version=*/fuzzed_data_provider.ConsumeIntegralInRange<int32_t>(MIN_PEER_PROTO_VERSION, std::numeric_limits<int32_t>::max()),
/*relay_txs=*/fuzzed_data_provider.ConsumeBool());

View file

@ -296,7 +296,6 @@ template <bool ReturnUniquePtr = false>
auto ConsumeNode(FuzzedDataProvider& fuzzed_data_provider, const std::optional<NodeId>& node_id_in = std::nullopt) noexcept
{
const NodeId node_id = node_id_in.value_or(fuzzed_data_provider.ConsumeIntegralInRange<NodeId>(0, std::numeric_limits<NodeId>::max()));
const ServiceFlags local_services = ConsumeWeakEnum(fuzzed_data_provider, ALL_SERVICE_FLAGS);
const auto sock = std::make_shared<FuzzedSock>(fuzzed_data_provider);
const CAddress address = ConsumeAddress(fuzzed_data_provider);
const uint64_t keyed_net_group = fuzzed_data_provider.ConsumeIntegral<uint64_t>();
@ -307,7 +306,6 @@ auto ConsumeNode(FuzzedDataProvider& fuzzed_data_provider, const std::optional<N
const bool inbound_onion{conn_type == ConnectionType::INBOUND ? fuzzed_data_provider.ConsumeBool() : false};
if constexpr (ReturnUniquePtr) {
return std::make_unique<CNode>(node_id,
local_services,
sock,
address,
keyed_net_group,
@ -318,7 +316,6 @@ auto ConsumeNode(FuzzedDataProvider& fuzzed_data_provider, const std::optional<N
inbound_onion);
} else {
return CNode{node_id,
local_services,
sock,
address,
keyed_net_group,

View file

@ -58,7 +58,6 @@ BOOST_AUTO_TEST_CASE(cnode_simple_test)
std::string pszDest;
std::unique_ptr<CNode> pnode1 = std::make_unique<CNode>(id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr,
/*nKeyedNetGroupIn=*/0,
@ -77,7 +76,6 @@ BOOST_AUTO_TEST_CASE(cnode_simple_test)
BOOST_CHECK_EQUAL(pnode1->ConnectedThroughNetwork(), Network::NET_IPV4);
std::unique_ptr<CNode> pnode2 = std::make_unique<CNode>(id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr,
/*nKeyedNetGroupIn=*/1,
@ -96,7 +94,6 @@ BOOST_AUTO_TEST_CASE(cnode_simple_test)
BOOST_CHECK_EQUAL(pnode2->ConnectedThroughNetwork(), Network::NET_IPV4);
std::unique_ptr<CNode> pnode3 = std::make_unique<CNode>(id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr,
/*nKeyedNetGroupIn=*/0,
@ -115,7 +112,6 @@ BOOST_AUTO_TEST_CASE(cnode_simple_test)
BOOST_CHECK_EQUAL(pnode3->ConnectedThroughNetwork(), Network::NET_IPV4);
std::unique_ptr<CNode> pnode4 = std::make_unique<CNode>(id++,
NODE_NETWORK,
/*sock=*/nullptr,
addr,
/*nKeyedNetGroupIn=*/1,
@ -629,7 +625,6 @@ BOOST_AUTO_TEST_CASE(ipv4_peer_with_ipv6_addrMe_test)
ipv4AddrPeer.s_addr = 0xa0b0c001;
CAddress addr = CAddress(CService(ipv4AddrPeer, 7777), NODE_NETWORK);
std::unique_ptr<CNode> pnode = std::make_unique<CNode>(/*id=*/0,
NODE_NETWORK,
/*sock=*/nullptr,
addr,
/*nKeyedNetGroupIn=*/0,
@ -648,7 +643,7 @@ BOOST_AUTO_TEST_CASE(ipv4_peer_with_ipv6_addrMe_test)
pnode->SetAddrLocal(addrLocal);
// before patch, this causes undefined behavior detectable with clang's -fsanitize=memory
GetLocalAddrForPeer(&*pnode);
GetLocalAddrForPeer(*pnode);
// suppress no-checks-run warning; if this test fails, it's by triggering a sanitizer
BOOST_CHECK(1);
@ -675,19 +670,15 @@ BOOST_AUTO_TEST_CASE(get_local_addr_for_peer_port)
const uint16_t bind_port = 20001;
m_node.args->ForceSetArg("-bind", strprintf("3.4.5.6:%u", bind_port));
const uint32_t current_time = static_cast<uint32_t>(GetAdjustedTime());
SetMockTime(current_time);
// Our address:port as seen from the peer, completely different from the above.
in_addr peer_us_addr;
peer_us_addr.s_addr = htonl(0x02030405);
const CAddress peer_us{CService{peer_us_addr, 20002}, NODE_NETWORK, current_time};
const CService peer_us{peer_us_addr, 20002};
// Create a peer with a routable IPv4 address (outbound).
in_addr peer_out_in_addr;
peer_out_in_addr.s_addr = htonl(0x01020304);
CNode peer_out{/*id=*/0,
/*nLocalServicesIn=*/NODE_NETWORK,
/*sock=*/nullptr,
/*addrIn=*/CAddress{CService{peer_out_in_addr, 8333}, NODE_NETWORK},
/*nKeyedNetGroupIn=*/0,
@ -700,16 +691,15 @@ BOOST_AUTO_TEST_CASE(get_local_addr_for_peer_port)
peer_out.SetAddrLocal(peer_us);
// Without the fix peer_us:8333 is chosen instead of the proper peer_us:bind_port.
auto chosen_local_addr = GetLocalAddrForPeer(&peer_out);
auto chosen_local_addr = GetLocalAddrForPeer(peer_out);
BOOST_REQUIRE(chosen_local_addr);
const CAddress expected{CService{peer_us_addr, bind_port}, NODE_NETWORK, current_time};
const CService expected{peer_us_addr, bind_port};
BOOST_CHECK(*chosen_local_addr == expected);
// Create a peer with a routable IPv4 address (inbound).
in_addr peer_in_in_addr;
peer_in_in_addr.s_addr = htonl(0x05060708);
CNode peer_in{/*id=*/0,
/*nLocalServicesIn=*/NODE_NETWORK,
/*sock=*/nullptr,
/*addrIn=*/CAddress{CService{peer_in_in_addr, 8333}, NODE_NETWORK},
/*nKeyedNetGroupIn=*/0,
@ -722,7 +712,7 @@ BOOST_AUTO_TEST_CASE(get_local_addr_for_peer_port)
peer_in.SetAddrLocal(peer_us);
// Without the fix peer_us:8333 is chosen instead of the proper peer_us:peer_us.GetPort().
chosen_local_addr = GetLocalAddrForPeer(&peer_in);
chosen_local_addr = GetLocalAddrForPeer(peer_in);
BOOST_REQUIRE(chosen_local_addr);
BOOST_CHECK(*chosen_local_addr == peer_us);
@ -837,7 +827,6 @@ BOOST_AUTO_TEST_CASE(initial_advertise_from_version_message)
in_addr peer_in_addr;
peer_in_addr.s_addr = htonl(0x01020304);
CNode peer{/*id=*/0,
/*nLocalServicesIn=*/NODE_NETWORK,
/*sock=*/nullptr,
/*addrIn=*/CAddress{CService{peer_in_addr, 8333}, NODE_NETWORK},
/*nKeyedNetGroupIn=*/0,
@ -857,7 +846,7 @@ BOOST_AUTO_TEST_CASE(initial_advertise_from_version_message)
*static_cast<TestChainState*>(&m_node.chainman->ActiveChainstate());
chainstate.JumpOutOfIbd();
m_node.peerman->InitializeNode(&peer);
m_node.peerman->InitializeNode(peer, NODE_NETWORK);
std::atomic<bool> interrupt_dummy{false};
std::chrono::microseconds time_received_dummy{0};

View file

@ -16,6 +16,7 @@
void ConnmanTestMsg::Handshake(CNode& node,
bool successfully_connected,
ServiceFlags remote_services,
ServiceFlags local_services,
NetPermissionFlags permission_flags,
int32_t version,
bool relay_txs)
@ -24,7 +25,7 @@ void ConnmanTestMsg::Handshake(CNode& node,
auto& connman{*this};
const CNetMsgMaker mm{0};
peerman.InitializeNode(&node);
peerman.InitializeNode(node, local_services);
CSerializedNetMsg msg_version{
mm.Make(NetMsgType::VERSION,
@ -51,10 +52,10 @@ void ConnmanTestMsg::Handshake(CNode& node,
if (node.fDisconnect) return;
assert(node.nVersion == version);
assert(node.GetCommonVersion() == std::min(version, PROTOCOL_VERSION));
assert(node.nServices == remote_services);
CNodeStateStats statestats;
assert(peerman.GetNodeStateStats(node.GetId(), statestats));
assert(statestats.m_relay_txs == (relay_txs && !node.IsBlockOnlyConn()));
assert(statestats.their_services == remote_services);
node.m_permissionFlags = permission_flags;
if (successfully_connected) {
CSerializedNetMsg msg_verack{mm.Make(NetMsgType::VERACK)};

View file

@ -42,6 +42,7 @@ struct ConnmanTestMsg : public CConnman {
void Handshake(CNode& node,
bool successfully_connected,
ServiceFlags remote_services,
ServiceFlags local_services,
NetPermissionFlags permission_flags,
int32_t version,
bool relay_txs);