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.

408 lines
13 KiB

  1. package pex
  2. import (
  3. "context"
  4. "fmt"
  5. "sync"
  6. "time"
  7. "github.com/tendermint/tendermint/internal/p2p"
  8. "github.com/tendermint/tendermint/internal/p2p/conn"
  9. "github.com/tendermint/tendermint/libs/log"
  10. "github.com/tendermint/tendermint/libs/service"
  11. protop2p "github.com/tendermint/tendermint/proto/tendermint/p2p"
  12. "github.com/tendermint/tendermint/types"
  13. )
  14. var (
  15. _ service.Service = (*Reactor)(nil)
  16. _ p2p.Wrapper = (*protop2p.PexMessage)(nil)
  17. )
  18. const (
  19. // PexChannel is a channel for PEX messages
  20. PexChannel = 0x00
  21. // over-estimate of max NetAddress size
  22. // hexID (40) + IP (16) + Port (2) + Name (100) ...
  23. // NOTE: dont use massive DNS name ..
  24. maxAddressSize = 256
  25. // max addresses returned by GetSelection
  26. // NOTE: this must match "maxMsgSize"
  27. maxGetSelection = 250
  28. // NOTE: amplification factor!
  29. // small request results in up to maxMsgSize response
  30. maxMsgSize = maxAddressSize * maxGetSelection
  31. // the minimum time one peer can send another request to the same peer
  32. minReceiveRequestInterval = 100 * time.Millisecond
  33. // the maximum amount of addresses that can be included in a response
  34. maxAddresses = 100
  35. // How long to wait when there are no peers available before trying again
  36. noAvailablePeersWaitPeriod = 1 * time.Second
  37. // indicates the ping rate of the pex reactor when the peer store is full.
  38. // The reactor should still look to add new peers in order to flush out low
  39. // scoring peers that are still in the peer store
  40. fullCapacityInterval = 10 * time.Minute
  41. )
  42. // TODO: We should decide whether we want channel descriptors to be housed
  43. // within each reactor (as they are now) or, considering that the reactor doesn't
  44. // really need to care about the channel descriptors, if they should be housed
  45. // in the node module.
  46. func ChannelDescriptor() *conn.ChannelDescriptor {
  47. return &conn.ChannelDescriptor{
  48. ID: PexChannel,
  49. MessageType: new(protop2p.PexMessage),
  50. Priority: 1,
  51. SendQueueCapacity: 10,
  52. RecvMessageCapacity: maxMsgSize,
  53. RecvBufferCapacity: 128,
  54. }
  55. }
  56. // The peer exchange or PEX reactor supports the peer manager by sending
  57. // requests to other peers for addresses that can be given to the peer manager
  58. // and at the same time advertises addresses to peers that need more.
  59. //
  60. // The reactor is able to tweak the intensity of it's search by decreasing or
  61. // increasing the interval between each request. It tracks connected peers via
  62. // a linked list, sending a request to the node at the front of the list and
  63. // adding it to the back of the list once a response is received.
  64. type Reactor struct {
  65. service.BaseService
  66. logger log.Logger
  67. peerManager *p2p.PeerManager
  68. pexCh *p2p.Channel
  69. peerUpdates *p2p.PeerUpdates
  70. // list of available peers to loop through and send peer requests to
  71. availablePeers map[types.NodeID]struct{}
  72. mtx sync.RWMutex
  73. // requestsSent keeps track of which peers the PEX reactor has sent requests
  74. // to. This prevents the sending of spurious responses.
  75. // NOTE: If a node never responds, they will remain in this map until a
  76. // peer down status update is sent
  77. requestsSent map[types.NodeID]struct{}
  78. // lastReceivedRequests keeps track of when peers send a request to prevent
  79. // peers from sending requests too often (as defined by
  80. // minReceiveRequestInterval).
  81. lastReceivedRequests map[types.NodeID]time.Time
  82. // the total number of unique peers added
  83. totalPeers int
  84. }
  85. // NewReactor returns a reference to a new reactor.
  86. func NewReactor(
  87. ctx context.Context,
  88. logger log.Logger,
  89. peerManager *p2p.PeerManager,
  90. channelCreator p2p.ChannelCreator,
  91. peerUpdates *p2p.PeerUpdates,
  92. ) (*Reactor, error) {
  93. channel, err := channelCreator(ctx, ChannelDescriptor())
  94. if err != nil {
  95. return nil, err
  96. }
  97. r := &Reactor{
  98. logger: logger,
  99. peerManager: peerManager,
  100. pexCh: channel,
  101. peerUpdates: peerUpdates,
  102. availablePeers: make(map[types.NodeID]struct{}),
  103. requestsSent: make(map[types.NodeID]struct{}),
  104. lastReceivedRequests: make(map[types.NodeID]time.Time),
  105. }
  106. r.BaseService = *service.NewBaseService(logger, "PEX", r)
  107. return r, nil
  108. }
  109. // OnStart starts separate go routines for each p2p Channel and listens for
  110. // envelopes on each. In addition, it also listens for peer updates and handles
  111. // messages on that p2p channel accordingly. The caller must be sure to execute
  112. // OnStop to ensure the outbound p2p Channels are closed.
  113. func (r *Reactor) OnStart(ctx context.Context) error {
  114. go r.processPexCh(ctx)
  115. go r.processPeerUpdates(ctx)
  116. return nil
  117. }
  118. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  119. // blocking until they all exit.
  120. func (r *Reactor) OnStop() {}
  121. // processPexCh implements a blocking event loop where we listen for p2p
  122. // Envelope messages from the pexCh.
  123. func (r *Reactor) processPexCh(ctx context.Context) {
  124. incoming := make(chan *p2p.Envelope)
  125. go func() {
  126. defer close(incoming)
  127. iter := r.pexCh.Receive(ctx)
  128. for iter.Next(ctx) {
  129. select {
  130. case <-ctx.Done():
  131. return
  132. case incoming <- iter.Envelope():
  133. }
  134. }
  135. }()
  136. // Initially, we will request peers quickly to bootstrap. This duration
  137. // will be adjusted upward as knowledge of the network grows.
  138. var nextPeerRequest = minReceiveRequestInterval
  139. timer := time.NewTimer(0)
  140. defer timer.Stop()
  141. for {
  142. timer.Reset(nextPeerRequest)
  143. select {
  144. case <-ctx.Done():
  145. return
  146. case <-timer.C:
  147. // Send a request for more peer addresses.
  148. if err := r.sendRequestForPeers(ctx); err != nil {
  149. return
  150. // TODO(creachadair): Do we really want to stop processing the PEX
  151. // channel just because of an error here?
  152. }
  153. // Note we do not update the poll timer upon making a request, only
  154. // when we receive an update that updates our priors.
  155. case envelope, ok := <-incoming:
  156. if !ok {
  157. return // channel closed
  158. }
  159. // A request from another peer, or a response to one of our requests.
  160. dur, err := r.handlePexMessage(ctx, envelope)
  161. if err != nil {
  162. r.logger.Error("failed to process message",
  163. "ch_id", r.pexCh.ID, "envelope", envelope, "err", err)
  164. if serr := r.pexCh.SendError(ctx, p2p.PeerError{
  165. NodeID: envelope.From,
  166. Err: err,
  167. }); serr != nil {
  168. return
  169. }
  170. } else if dur != 0 {
  171. // We got a useful result; update the poll timer.
  172. nextPeerRequest = dur
  173. }
  174. }
  175. }
  176. }
  177. // processPeerUpdates initiates a blocking process where we listen for and handle
  178. // PeerUpdate messages. When the reactor is stopped, we will catch the signal and
  179. // close the p2p PeerUpdatesCh gracefully.
  180. func (r *Reactor) processPeerUpdates(ctx context.Context) {
  181. for {
  182. select {
  183. case <-ctx.Done():
  184. return
  185. case peerUpdate := <-r.peerUpdates.Updates():
  186. r.processPeerUpdate(peerUpdate)
  187. }
  188. }
  189. }
  190. // handlePexMessage handles envelopes sent from peers on the PexChannel.
  191. // If an update was received, a new polling interval is returned; otherwise the
  192. // duration is 0.
  193. func (r *Reactor) handlePexMessage(ctx context.Context, envelope *p2p.Envelope) (time.Duration, error) {
  194. logger := r.logger.With("peer", envelope.From)
  195. switch msg := envelope.Message.(type) {
  196. case *protop2p.PexRequest:
  197. // Verify that this peer hasn't sent us another request too recently.
  198. if err := r.markPeerRequest(envelope.From); err != nil {
  199. return 0, err
  200. }
  201. // Fetch peers from the peer manager, convert NodeAddresses into URL
  202. // strings, and send them back to the caller.
  203. nodeAddresses := r.peerManager.Advertise(envelope.From, maxAddresses)
  204. pexAddresses := make([]protop2p.PexAddress, len(nodeAddresses))
  205. for idx, addr := range nodeAddresses {
  206. pexAddresses[idx] = protop2p.PexAddress{
  207. URL: addr.String(),
  208. }
  209. }
  210. return 0, r.pexCh.Send(ctx, p2p.Envelope{
  211. To: envelope.From,
  212. Message: &protop2p.PexResponse{Addresses: pexAddresses},
  213. })
  214. case *protop2p.PexResponse:
  215. // Verify that this response corresponds to one of our pending requests.
  216. if err := r.markPeerResponse(envelope.From); err != nil {
  217. return 0, err
  218. }
  219. // Verify that the response does not exceed the safety limit.
  220. if len(msg.Addresses) > maxAddresses {
  221. return 0, fmt.Errorf("peer sent too many addresses (%d > maxiumum %d)",
  222. len(msg.Addresses), maxAddresses)
  223. }
  224. var numAdded int
  225. for _, pexAddress := range msg.Addresses {
  226. peerAddress, err := p2p.ParseNodeAddress(pexAddress.URL)
  227. if err != nil {
  228. continue
  229. }
  230. added, err := r.peerManager.Add(peerAddress)
  231. if err != nil {
  232. logger.Error("failed to add PEX address", "address", peerAddress, "err", err)
  233. continue
  234. }
  235. if added {
  236. numAdded++
  237. logger.Debug("added PEX address", "address", peerAddress)
  238. }
  239. }
  240. return r.calculateNextRequestTime(numAdded), nil
  241. default:
  242. return 0, fmt.Errorf("received unknown message: %T", msg)
  243. }
  244. }
  245. // processPeerUpdate processes a PeerUpdate. For added peers, PeerStatusUp, we
  246. // send a request for addresses.
  247. func (r *Reactor) processPeerUpdate(peerUpdate p2p.PeerUpdate) {
  248. r.logger.Debug("received PEX peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  249. r.mtx.Lock()
  250. defer r.mtx.Unlock()
  251. switch peerUpdate.Status {
  252. case p2p.PeerStatusUp:
  253. r.availablePeers[peerUpdate.NodeID] = struct{}{}
  254. case p2p.PeerStatusDown:
  255. delete(r.availablePeers, peerUpdate.NodeID)
  256. delete(r.requestsSent, peerUpdate.NodeID)
  257. delete(r.lastReceivedRequests, peerUpdate.NodeID)
  258. default:
  259. }
  260. }
  261. // sendRequestForPeers chooses a peer from the set of available peers and sends
  262. // that peer a request for more peer addresses. The chosen peer is moved into
  263. // the requestsSent bucket so that we will not attempt to contact them again
  264. // until they've replied or updated.
  265. func (r *Reactor) sendRequestForPeers(ctx context.Context) error {
  266. r.mtx.Lock()
  267. defer r.mtx.Unlock()
  268. if len(r.availablePeers) == 0 {
  269. // no peers are available
  270. r.logger.Debug("no available peers to send a PEX request to (retrying)")
  271. return nil
  272. }
  273. // Select an arbitrary peer from the available set.
  274. var peerID types.NodeID
  275. for peerID = range r.availablePeers {
  276. break
  277. }
  278. if err := r.pexCh.Send(ctx, p2p.Envelope{
  279. To: peerID,
  280. Message: &protop2p.PexRequest{},
  281. }); err != nil {
  282. return err
  283. }
  284. // Move the peer from available to pending.
  285. delete(r.availablePeers, peerID)
  286. r.requestsSent[peerID] = struct{}{}
  287. return nil
  288. }
  289. // calculateNextRequestTime selects how long we should wait before attempting
  290. // to send out another request for peer addresses.
  291. //
  292. // This implements a simplified proportional control mechanism to poll more
  293. // often when our knowledge of the network is incomplete, and less often as our
  294. // knowledge grows. To estimate our knowledge of the network, we use the
  295. // fraction of "new" peers (addresses we have not previously seen) to the total
  296. // so far observed. When we first join the network, this fraction will be close
  297. // to 1, meaning most new peers are "new" to us, and as we discover more peers,
  298. // the fraction will go toward zero.
  299. //
  300. // The minimum interval will be minReceiveRequestInterval to ensure we will not
  301. // request from any peer more often than we would allow them to do from us.
  302. func (r *Reactor) calculateNextRequestTime(added int) time.Duration {
  303. r.mtx.Lock()
  304. defer r.mtx.Unlock()
  305. r.totalPeers += added
  306. // If the peer store is nearly full, wait the maximum interval.
  307. if ratio := r.peerManager.PeerRatio(); ratio >= 0.95 {
  308. r.logger.Debug("Peer manager is nearly full",
  309. "sleep_period", fullCapacityInterval, "ratio", ratio)
  310. return fullCapacityInterval
  311. }
  312. // If there are no available peers to query, poll less aggressively.
  313. if len(r.availablePeers) == 0 {
  314. r.logger.Debug("No available peers to send a PEX request",
  315. "sleep_period", noAvailablePeersWaitPeriod)
  316. return noAvailablePeersWaitPeriod
  317. }
  318. // Reaching here, there are available peers to query and the peer store
  319. // still has space. Estimate our knowledge of the network from the latest
  320. // update and choose a new interval.
  321. base := float64(minReceiveRequestInterval) / float64(len(r.availablePeers))
  322. multiplier := float64(r.totalPeers+1) / float64(added+1) // +1 to avert zero division
  323. return time.Duration(base*multiplier*multiplier) + minReceiveRequestInterval
  324. }
  325. func (r *Reactor) markPeerRequest(peer types.NodeID) error {
  326. r.mtx.Lock()
  327. defer r.mtx.Unlock()
  328. if lastRequestTime, ok := r.lastReceivedRequests[peer]; ok {
  329. if d := time.Since(lastRequestTime); d < minReceiveRequestInterval {
  330. return fmt.Errorf("peer %v sent PEX request too soon (%v < minimum %v)",
  331. peer, d, minReceiveRequestInterval)
  332. }
  333. }
  334. r.lastReceivedRequests[peer] = time.Now()
  335. return nil
  336. }
  337. func (r *Reactor) markPeerResponse(peer types.NodeID) error {
  338. r.mtx.Lock()
  339. defer r.mtx.Unlock()
  340. // check if a request to this peer was sent
  341. if _, ok := r.requestsSent[peer]; !ok {
  342. return fmt.Errorf("peer sent a PEX response when none was requested (%v)", peer)
  343. }
  344. delete(r.requestsSent, peer)
  345. // attach to the back of the list so that the peer can be used again for
  346. // future requests
  347. r.availablePeers[peer] = struct{}{}
  348. return nil
  349. }