You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

556 lines
18 KiB

  1. package pex
  2. import (
  3. "context"
  4. "fmt"
  5. "runtime/debug"
  6. "sync"
  7. "time"
  8. "github.com/tendermint/tendermint/internal/p2p"
  9. "github.com/tendermint/tendermint/internal/p2p/conn"
  10. "github.com/tendermint/tendermint/libs/log"
  11. tmmath "github.com/tendermint/tendermint/libs/math"
  12. "github.com/tendermint/tendermint/libs/service"
  13. protop2p "github.com/tendermint/tendermint/proto/tendermint/p2p"
  14. "github.com/tendermint/tendermint/types"
  15. )
  16. var (
  17. _ service.Service = (*ReactorV2)(nil)
  18. _ p2p.Wrapper = (*protop2p.PexMessage)(nil)
  19. )
  20. // TODO: Consolidate with params file.
  21. // See https://github.com/tendermint/tendermint/issues/6371
  22. const (
  23. // PexChannel is a channel for PEX messages
  24. PexChannel = byte(0x00)
  25. // over-estimate of max NetAddress size
  26. // hexID (40) + IP (16) + Port (2) + Name (100) ...
  27. // NOTE: dont use massive DNS name ..
  28. maxAddressSize = 256
  29. // max addresses returned by GetSelection
  30. // NOTE: this must match "maxMsgSize"
  31. maxGetSelection = 250
  32. // NOTE: amplification factor!
  33. // small request results in up to maxMsgSize response
  34. maxMsgSize = maxAddressSize * maxGetSelection
  35. // the minimum time one peer can send another request to the same peer
  36. minReceiveRequestInterval = 100 * time.Millisecond
  37. // the maximum amount of addresses that can be included in a response
  38. maxAddresses uint16 = 100
  39. // allocated time to resolve a node address into a set of endpoints
  40. resolveTimeout = 3 * time.Second
  41. // How long to wait when there are no peers available before trying again
  42. noAvailablePeersWaitPeriod = 1 * time.Second
  43. // indicates the ping rate of the pex reactor when the peer store is full.
  44. // The reactor should still look to add new peers in order to flush out low
  45. // scoring peers that are still in the peer store
  46. fullCapacityInterval = 10 * time.Minute
  47. )
  48. // TODO: We should decide whether we want channel descriptors to be housed
  49. // within each reactor (as they are now) or, considering that the reactor doesn't
  50. // really need to care about the channel descriptors, if they should be housed
  51. // in the node module.
  52. func ChannelDescriptor() conn.ChannelDescriptor {
  53. return conn.ChannelDescriptor{
  54. ID: PexChannel,
  55. Priority: 1,
  56. SendQueueCapacity: 10,
  57. RecvMessageCapacity: maxMsgSize,
  58. RecvBufferCapacity: 32,
  59. MaxSendBytes: 200,
  60. }
  61. }
  62. // ReactorV2 is a PEX reactor for the new P2P stack. The legacy reactor
  63. // is Reactor.
  64. //
  65. // FIXME: Rename this when Reactor is removed, and consider moving to p2p/.
  66. //
  67. // The peer exchange or PEX reactor supports the peer manager by sending
  68. // requests to other peers for addresses that can be given to the peer manager
  69. // and at the same time advertises addresses to peers that need more.
  70. //
  71. // The reactor is able to tweak the intensity of it's search by decreasing or
  72. // increasing the interval between each request. It tracks connected peers via
  73. // a linked list, sending a request to the node at the front of the list and
  74. // adding it to the back of the list once a response is received.
  75. type ReactorV2 struct {
  76. service.BaseService
  77. peerManager *p2p.PeerManager
  78. pexCh *p2p.Channel
  79. peerUpdates *p2p.PeerUpdates
  80. closeCh chan struct{}
  81. // list of available peers to loop through and send peer requests to
  82. availablePeers map[types.NodeID]struct{}
  83. mtx sync.RWMutex
  84. // requestsSent keeps track of which peers the PEX reactor has sent requests
  85. // to. This prevents the sending of spurious responses.
  86. // NOTE: If a node never responds, they will remain in this map until a
  87. // peer down status update is sent
  88. requestsSent map[types.NodeID]struct{}
  89. // lastReceivedRequests keeps track of when peers send a request to prevent
  90. // peers from sending requests too often (as defined by
  91. // minReceiveRequestInterval).
  92. lastReceivedRequests map[types.NodeID]time.Time
  93. // the time when another request will be sent
  94. nextRequestTime time.Time
  95. // keep track of how many new peers to existing peers we have received to
  96. // extrapolate the size of the network
  97. newPeers uint32
  98. totalPeers uint32
  99. // discoveryRatio is the inverse ratio of new peers to old peers squared.
  100. // This is multiplied by the minimum duration to calculate how long to wait
  101. // between each request.
  102. discoveryRatio float32
  103. }
  104. // NewReactor returns a reference to a new reactor.
  105. func NewReactorV2(
  106. logger log.Logger,
  107. peerManager *p2p.PeerManager,
  108. pexCh *p2p.Channel,
  109. peerUpdates *p2p.PeerUpdates,
  110. ) *ReactorV2 {
  111. r := &ReactorV2{
  112. peerManager: peerManager,
  113. pexCh: pexCh,
  114. peerUpdates: peerUpdates,
  115. closeCh: make(chan struct{}),
  116. availablePeers: make(map[types.NodeID]struct{}),
  117. requestsSent: make(map[types.NodeID]struct{}),
  118. lastReceivedRequests: make(map[types.NodeID]time.Time),
  119. }
  120. r.BaseService = *service.NewBaseService(logger, "PEX", r)
  121. return r
  122. }
  123. // OnStart starts separate go routines for each p2p Channel and listens for
  124. // envelopes on each. In addition, it also listens for peer updates and handles
  125. // messages on that p2p channel accordingly. The caller must be sure to execute
  126. // OnStop to ensure the outbound p2p Channels are closed.
  127. func (r *ReactorV2) OnStart() error {
  128. go r.processPexCh()
  129. go r.processPeerUpdates()
  130. return nil
  131. }
  132. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  133. // blocking until they all exit.
  134. func (r *ReactorV2) OnStop() {
  135. // Close closeCh to signal to all spawned goroutines to gracefully exit. All
  136. // p2p Channels should execute Close().
  137. close(r.closeCh)
  138. // Wait for all p2p Channels to be closed before returning. This ensures we
  139. // can easily reason about synchronization of all p2p Channels and ensure no
  140. // panics will occur.
  141. <-r.pexCh.Done()
  142. <-r.peerUpdates.Done()
  143. }
  144. // processPexCh implements a blocking event loop where we listen for p2p
  145. // Envelope messages from the pexCh.
  146. func (r *ReactorV2) processPexCh() {
  147. defer r.pexCh.Close()
  148. for {
  149. select {
  150. case <-r.closeCh:
  151. r.Logger.Debug("stopped listening on PEX channel; closing...")
  152. return
  153. // outbound requests for new peers
  154. case <-r.waitUntilNextRequest():
  155. r.sendRequestForPeers()
  156. // inbound requests for new peers or responses to requests sent by this
  157. // reactor
  158. case envelope := <-r.pexCh.In:
  159. if err := r.handleMessage(r.pexCh.ID, envelope); err != nil {
  160. r.Logger.Error("failed to process message", "ch_id", r.pexCh.ID, "envelope", envelope, "err", err)
  161. r.pexCh.Error <- p2p.PeerError{
  162. NodeID: envelope.From,
  163. Err: err,
  164. }
  165. }
  166. }
  167. }
  168. }
  169. // processPeerUpdates initiates a blocking process where we listen for and handle
  170. // PeerUpdate messages. When the reactor is stopped, we will catch the signal and
  171. // close the p2p PeerUpdatesCh gracefully.
  172. func (r *ReactorV2) processPeerUpdates() {
  173. defer r.peerUpdates.Close()
  174. for {
  175. select {
  176. case peerUpdate := <-r.peerUpdates.Updates():
  177. r.processPeerUpdate(peerUpdate)
  178. case <-r.closeCh:
  179. r.Logger.Debug("stopped listening on peer updates channel; closing...")
  180. return
  181. }
  182. }
  183. }
  184. // handlePexMessage handles envelopes sent from peers on the PexChannel.
  185. func (r *ReactorV2) handlePexMessage(envelope p2p.Envelope) error {
  186. logger := r.Logger.With("peer", envelope.From)
  187. switch msg := envelope.Message.(type) {
  188. case *protop2p.PexRequest:
  189. // Check if the peer hasn't sent a prior request too close to this one
  190. // in time.
  191. if err := r.markPeerRequest(envelope.From); err != nil {
  192. return err
  193. }
  194. // parse and send the legacy PEX addresses
  195. pexAddresses := r.resolve(r.peerManager.Advertise(envelope.From, maxAddresses))
  196. r.pexCh.Out <- p2p.Envelope{
  197. To: envelope.From,
  198. Message: &protop2p.PexResponse{Addresses: pexAddresses},
  199. }
  200. case *protop2p.PexResponse:
  201. // check if the response matches a request that was made to that peer
  202. if err := r.markPeerResponse(envelope.From); err != nil {
  203. return err
  204. }
  205. // check the size of the response
  206. if len(msg.Addresses) > int(maxAddresses) {
  207. return fmt.Errorf("peer sent too many addresses (max: %d, got: %d)",
  208. maxAddresses,
  209. len(msg.Addresses),
  210. )
  211. }
  212. for _, pexAddress := range msg.Addresses {
  213. // no protocol is prefixed so we assume the default (mconn)
  214. peerAddress, err := p2p.ParseNodeAddress(
  215. fmt.Sprintf("%s@%s:%d", pexAddress.ID, pexAddress.IP, pexAddress.Port))
  216. if err != nil {
  217. continue
  218. }
  219. added, err := r.peerManager.Add(peerAddress)
  220. if err != nil {
  221. logger.Error("failed to add PEX address", "address", peerAddress, "err", err)
  222. }
  223. if added {
  224. r.newPeers++
  225. logger.Debug("added PEX address", "address", peerAddress)
  226. }
  227. r.totalPeers++
  228. }
  229. // V2 PEX MESSAGES
  230. case *protop2p.PexRequestV2:
  231. // check if the peer hasn't sent a prior request too close to this one
  232. // in time
  233. if err := r.markPeerRequest(envelope.From); err != nil {
  234. return err
  235. }
  236. // request peers from the peer manager and parse the NodeAddresses into
  237. // URL strings
  238. nodeAddresses := r.peerManager.Advertise(envelope.From, maxAddresses)
  239. pexAddressesV2 := make([]protop2p.PexAddressV2, len(nodeAddresses))
  240. for idx, addr := range nodeAddresses {
  241. pexAddressesV2[idx] = protop2p.PexAddressV2{
  242. URL: addr.String(),
  243. }
  244. }
  245. r.pexCh.Out <- p2p.Envelope{
  246. To: envelope.From,
  247. Message: &protop2p.PexResponseV2{Addresses: pexAddressesV2},
  248. }
  249. case *protop2p.PexResponseV2:
  250. // check if the response matches a request that was made to that peer
  251. if err := r.markPeerResponse(envelope.From); err != nil {
  252. return err
  253. }
  254. // check the size of the response
  255. if len(msg.Addresses) > int(maxAddresses) {
  256. return fmt.Errorf("peer sent too many addresses (max: %d, got: %d)",
  257. maxAddresses,
  258. len(msg.Addresses),
  259. )
  260. }
  261. for _, pexAddress := range msg.Addresses {
  262. peerAddress, err := p2p.ParseNodeAddress(pexAddress.URL)
  263. if err != nil {
  264. continue
  265. }
  266. added, err := r.peerManager.Add(peerAddress)
  267. if err != nil {
  268. logger.Error("failed to add V2 PEX address", "address", peerAddress, "err", err)
  269. }
  270. if added {
  271. r.newPeers++
  272. logger.Debug("added V2 PEX address", "address", peerAddress)
  273. }
  274. r.totalPeers++
  275. }
  276. default:
  277. return fmt.Errorf("received unknown message: %T", msg)
  278. }
  279. return nil
  280. }
  281. // resolve resolves a set of peer addresses into PEX addresses.
  282. //
  283. // FIXME: This is necessary because the current PEX protocol only supports
  284. // IP/port pairs, while the P2P stack uses NodeAddress URLs. The PEX protocol
  285. // should really use URLs too, to exchange DNS names instead of IPs and allow
  286. // different transport protocols (e.g. QUIC and MemoryTransport).
  287. //
  288. // FIXME: We may want to cache and parallelize this, but for now we'll just rely
  289. // on the operating system to cache it for us.
  290. func (r *ReactorV2) resolve(addresses []p2p.NodeAddress) []protop2p.PexAddress {
  291. limit := len(addresses)
  292. pexAddresses := make([]protop2p.PexAddress, 0, limit)
  293. for _, address := range addresses {
  294. ctx, cancel := context.WithTimeout(context.Background(), resolveTimeout)
  295. endpoints, err := address.Resolve(ctx)
  296. r.Logger.Debug("resolved node address", "endpoints", endpoints)
  297. cancel()
  298. if err != nil {
  299. r.Logger.Debug("failed to resolve address", "address", address, "err", err)
  300. continue
  301. }
  302. for _, endpoint := range endpoints {
  303. r.Logger.Debug("checking endpint", "IP", endpoint.IP, "Port", endpoint.Port)
  304. if len(pexAddresses) >= limit {
  305. return pexAddresses
  306. } else if endpoint.IP != nil {
  307. r.Logger.Debug("appending pex address")
  308. // PEX currently only supports IP-networked transports (as
  309. // opposed to e.g. p2p.MemoryTransport).
  310. //
  311. // FIXME: as the PEX address contains no information about the
  312. // protocol, we jam this into the ID. We won't need to this once
  313. // we support URLs
  314. pexAddresses = append(pexAddresses, protop2p.PexAddress{
  315. ID: string(address.NodeID),
  316. IP: endpoint.IP.String(),
  317. Port: uint32(endpoint.Port),
  318. })
  319. }
  320. }
  321. }
  322. return pexAddresses
  323. }
  324. // handleMessage handles an Envelope sent from a peer on a specific p2p Channel.
  325. // It will handle errors and any possible panics gracefully. A caller can handle
  326. // any error returned by sending a PeerError on the respective channel.
  327. func (r *ReactorV2) handleMessage(chID p2p.ChannelID, envelope p2p.Envelope) (err error) {
  328. defer func() {
  329. if e := recover(); e != nil {
  330. err = fmt.Errorf("panic in processing message: %v", e)
  331. r.Logger.Error(
  332. "recovering from processing message panic",
  333. "err", err,
  334. "stack", string(debug.Stack()),
  335. )
  336. }
  337. }()
  338. r.Logger.Debug("received PEX message", "peer", envelope.From)
  339. switch chID {
  340. case p2p.ChannelID(PexChannel):
  341. err = r.handlePexMessage(envelope)
  342. default:
  343. err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope)
  344. }
  345. return err
  346. }
  347. // processPeerUpdate processes a PeerUpdate. For added peers, PeerStatusUp, we
  348. // send a request for addresses.
  349. func (r *ReactorV2) processPeerUpdate(peerUpdate p2p.PeerUpdate) {
  350. r.Logger.Debug("received PEX peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  351. r.mtx.Lock()
  352. defer r.mtx.Unlock()
  353. switch peerUpdate.Status {
  354. case p2p.PeerStatusUp:
  355. r.availablePeers[peerUpdate.NodeID] = struct{}{}
  356. case p2p.PeerStatusDown:
  357. delete(r.availablePeers, peerUpdate.NodeID)
  358. delete(r.requestsSent, peerUpdate.NodeID)
  359. delete(r.lastReceivedRequests, peerUpdate.NodeID)
  360. default:
  361. }
  362. }
  363. func (r *ReactorV2) waitUntilNextRequest() <-chan time.Time {
  364. return time.After(time.Until(r.nextRequestTime))
  365. }
  366. // sendRequestForPeers pops the first peerID off the list and sends the
  367. // peer a request for more peer addresses. The function then moves the
  368. // peer into the requestsSent bucket and calculates when the next request
  369. // time should be
  370. func (r *ReactorV2) sendRequestForPeers() {
  371. r.mtx.Lock()
  372. defer r.mtx.Unlock()
  373. if len(r.availablePeers) == 0 {
  374. // no peers are available
  375. r.Logger.Debug("no available peers to send request to, waiting...")
  376. r.nextRequestTime = time.Now().Add(noAvailablePeersWaitPeriod)
  377. return
  378. }
  379. var peerID types.NodeID
  380. // use range to get a random peer.
  381. for peerID = range r.availablePeers {
  382. break
  383. }
  384. // The node accommodates for both pex systems
  385. if r.isLegacyPeer(peerID) {
  386. r.pexCh.Out <- p2p.Envelope{
  387. To: peerID,
  388. Message: &protop2p.PexRequest{},
  389. }
  390. } else {
  391. r.pexCh.Out <- p2p.Envelope{
  392. To: peerID,
  393. Message: &protop2p.PexRequestV2{},
  394. }
  395. }
  396. // remove the peer from the abvailable peers list and mark it in the requestsSent map
  397. delete(r.availablePeers, peerID)
  398. r.requestsSent[peerID] = struct{}{}
  399. r.calculateNextRequestTime()
  400. r.Logger.Debug("peer request sent", "next_request_time", r.nextRequestTime)
  401. }
  402. // calculateNextRequestTime implements something of a proportional controller
  403. // to estimate how often the reactor should be requesting new peer addresses.
  404. // The dependent variable in this calculation is the ratio of new peers to
  405. // all peers that the reactor receives. The interval is thus calculated as the
  406. // inverse squared. In the beginning, all peers should be new peers.
  407. // We expect this ratio to be near 1 and thus the interval to be as short
  408. // as possible. As the node becomes more familiar with the network the ratio of
  409. // new nodes will plummet to a very small number, meaning the interval expands
  410. // to its upper bound.
  411. // CONTRACT: Must use a write lock as nextRequestTime is updated
  412. func (r *ReactorV2) calculateNextRequestTime() {
  413. // check if the peer store is full. If so then there is no need
  414. // to send peer requests too often
  415. if ratio := r.peerManager.PeerRatio(); ratio >= 0.95 {
  416. r.Logger.Debug("peer manager near full ratio, sleeping...",
  417. "sleep_period", fullCapacityInterval, "ratio", ratio)
  418. r.nextRequestTime = time.Now().Add(fullCapacityInterval)
  419. return
  420. }
  421. // baseTime represents the shortest interval that we can send peer requests
  422. // in. For example if we have 10 peers and we can't send a message to the
  423. // same peer every 500ms, then we can send a request every 50ms. In practice
  424. // we use a safety margin of 2, ergo 100ms
  425. peers := tmmath.MinInt(len(r.availablePeers), 50)
  426. baseTime := minReceiveRequestInterval
  427. if peers > 0 {
  428. baseTime = minReceiveRequestInterval * 2 / time.Duration(peers)
  429. }
  430. if r.totalPeers > 0 || r.discoveryRatio == 0 {
  431. // find the ratio of new peers. NOTE: We add 1 to both sides to avoid
  432. // divide by zero problems
  433. ratio := float32(r.totalPeers+1) / float32(r.newPeers+1)
  434. // square the ratio in order to get non linear time intervals
  435. // NOTE: The longest possible interval for a network with 100 or more peers
  436. // where a node is connected to 50 of them is 2 minutes.
  437. r.discoveryRatio = ratio * ratio
  438. r.newPeers = 0
  439. r.totalPeers = 0
  440. }
  441. // NOTE: As ratio is always >= 1, discovery ratio is >= 1. Therefore we don't need to worry
  442. // about the next request time being less than the minimum time
  443. r.nextRequestTime = time.Now().Add(baseTime * time.Duration(r.discoveryRatio))
  444. }
  445. func (r *ReactorV2) markPeerRequest(peer types.NodeID) error {
  446. r.mtx.Lock()
  447. defer r.mtx.Unlock()
  448. if lastRequestTime, ok := r.lastReceivedRequests[peer]; ok {
  449. if time.Now().Before(lastRequestTime.Add(minReceiveRequestInterval)) {
  450. return fmt.Errorf("peer sent a request too close after a prior one. Minimum interval: %v",
  451. minReceiveRequestInterval)
  452. }
  453. }
  454. r.lastReceivedRequests[peer] = time.Now()
  455. return nil
  456. }
  457. func (r *ReactorV2) markPeerResponse(peer types.NodeID) error {
  458. r.mtx.Lock()
  459. defer r.mtx.Unlock()
  460. // check if a request to this peer was sent
  461. if _, ok := r.requestsSent[peer]; !ok {
  462. return fmt.Errorf("peer sent a PEX response when none was requested (%v)", peer)
  463. }
  464. delete(r.requestsSent, peer)
  465. // attach to the back of the list so that the peer can be used again for
  466. // future requests
  467. r.availablePeers[peer] = struct{}{}
  468. return nil
  469. }
  470. // all addresses must use a MCONN protocol for the peer to be considered part of the
  471. // legacy p2p pex system
  472. func (r *ReactorV2) isLegacyPeer(peer types.NodeID) bool {
  473. for _, addr := range r.peerManager.Addresses(peer) {
  474. if addr.Protocol != p2p.MConnProtocol {
  475. return false
  476. }
  477. }
  478. return true
  479. }