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.

521 lines
17 KiB

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