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.

1391 lines
42 KiB

  1. package p2p
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "math"
  7. "math/rand"
  8. "sort"
  9. "sync"
  10. "time"
  11. "github.com/gogo/protobuf/proto"
  12. "github.com/google/orderedcode"
  13. dbm "github.com/tendermint/tm-db"
  14. tmsync "github.com/tendermint/tendermint/internal/libs/sync"
  15. p2pproto "github.com/tendermint/tendermint/proto/tendermint/p2p"
  16. "github.com/tendermint/tendermint/types"
  17. )
  18. const (
  19. // retryNever is returned by retryDelay() when retries are disabled.
  20. retryNever time.Duration = math.MaxInt64
  21. )
  22. // PeerStatus is a peer status.
  23. //
  24. // The peer manager has many more internal states for a peer (e.g. dialing,
  25. // connected, evicting, and so on), which are tracked separately. PeerStatus is
  26. // for external use outside of the peer manager.
  27. type PeerStatus string
  28. const (
  29. PeerStatusUp PeerStatus = "up" // connected and ready
  30. PeerStatusDown PeerStatus = "down" // disconnected
  31. PeerStatusGood PeerStatus = "good" // peer observed as good
  32. PeerStatusBad PeerStatus = "bad" // peer observed as bad
  33. )
  34. // PeerScore is a numeric score assigned to a peer (higher is better).
  35. type PeerScore uint8
  36. const (
  37. PeerScorePersistent PeerScore = math.MaxUint8 // persistent peers
  38. )
  39. // PeerUpdate is a peer update event sent via PeerUpdates.
  40. type PeerUpdate struct {
  41. NodeID types.NodeID
  42. Status PeerStatus
  43. }
  44. // PeerUpdates is a peer update subscription with notifications about peer
  45. // events (currently just status changes).
  46. type PeerUpdates struct {
  47. routerUpdatesCh chan PeerUpdate
  48. reactorUpdatesCh chan PeerUpdate
  49. closeCh chan struct{}
  50. closeOnce sync.Once
  51. }
  52. // NewPeerUpdates creates a new PeerUpdates subscription. It is primarily for
  53. // internal use, callers should typically use PeerManager.Subscribe(). The
  54. // subscriber must call Close() when done.
  55. func NewPeerUpdates(updatesCh chan PeerUpdate, buf int) *PeerUpdates {
  56. return &PeerUpdates{
  57. reactorUpdatesCh: updatesCh,
  58. routerUpdatesCh: make(chan PeerUpdate, buf),
  59. closeCh: make(chan struct{}),
  60. }
  61. }
  62. // Updates returns a channel for consuming peer updates.
  63. func (pu *PeerUpdates) Updates() <-chan PeerUpdate {
  64. return pu.reactorUpdatesCh
  65. }
  66. // SendUpdate pushes information about a peer into the routing layer,
  67. // presumably from a peer.
  68. func (pu *PeerUpdates) SendUpdate(update PeerUpdate) {
  69. select {
  70. case <-pu.closeCh:
  71. case pu.routerUpdatesCh <- update:
  72. }
  73. }
  74. // Close closes the peer updates subscription.
  75. func (pu *PeerUpdates) Close() {
  76. pu.closeOnce.Do(func() {
  77. // NOTE: We don't close updatesCh since multiple goroutines may be
  78. // sending on it. The PeerManager senders will select on closeCh as well
  79. // to avoid blocking on a closed subscription.
  80. close(pu.closeCh)
  81. })
  82. }
  83. // Done returns a channel that is closed when the subscription is closed.
  84. func (pu *PeerUpdates) Done() <-chan struct{} {
  85. return pu.closeCh
  86. }
  87. // PeerManagerOptions specifies options for a PeerManager.
  88. type PeerManagerOptions struct {
  89. // PersistentPeers are peers that we want to maintain persistent connections
  90. // to. These will be scored higher than other peers, and if
  91. // MaxConnectedUpgrade is non-zero any lower-scored peers will be evicted if
  92. // necessary to make room for these.
  93. PersistentPeers []types.NodeID
  94. // MaxPeers is the maximum number of peers to track information about, i.e.
  95. // store in the peer store. When exceeded, the lowest-scored unconnected peers
  96. // will be deleted. 0 means no limit.
  97. MaxPeers uint16
  98. // MaxConnected is the maximum number of connected peers (inbound and
  99. // outbound). 0 means no limit.
  100. MaxConnected uint16
  101. // MaxConnectedUpgrade is the maximum number of additional connections to
  102. // use for probing any better-scored peers to upgrade to when all connection
  103. // slots are full. 0 disables peer upgrading.
  104. //
  105. // For example, if we are already connected to MaxConnected peers, but we
  106. // know or learn about better-scored peers (e.g. configured persistent
  107. // peers) that we are not connected too, then we can probe these peers by
  108. // using up to MaxConnectedUpgrade connections, and once connected evict the
  109. // lowest-scored connected peers. This also works for inbound connections,
  110. // i.e. if a higher-scored peer attempts to connect to us, we can accept
  111. // the connection and evict a lower-scored peer.
  112. MaxConnectedUpgrade uint16
  113. // MinRetryTime is the minimum time to wait between retries. Retry times
  114. // double for each retry, up to MaxRetryTime. 0 disables retries.
  115. MinRetryTime time.Duration
  116. // MaxRetryTime is the maximum time to wait between retries. 0 means
  117. // no maximum, in which case the retry time will keep doubling.
  118. MaxRetryTime time.Duration
  119. // MaxRetryTimePersistent is the maximum time to wait between retries for
  120. // peers listed in PersistentPeers. 0 uses MaxRetryTime instead.
  121. MaxRetryTimePersistent time.Duration
  122. // RetryTimeJitter is the upper bound of a random interval added to
  123. // retry times, to avoid thundering herds. 0 disables jitter.
  124. RetryTimeJitter time.Duration
  125. // PeerScores sets fixed scores for specific peers. It is mainly used
  126. // for testing. A score of 0 is ignored.
  127. PeerScores map[types.NodeID]PeerScore
  128. // PrivatePeerIDs defines a set of NodeID objects which the PEX reactor will
  129. // consider private and never gossip.
  130. PrivatePeers map[types.NodeID]struct{}
  131. // persistentPeers provides fast PersistentPeers lookups. It is built
  132. // by optimize().
  133. persistentPeers map[types.NodeID]bool
  134. }
  135. // Validate validates the options.
  136. func (o *PeerManagerOptions) Validate() error {
  137. for _, id := range o.PersistentPeers {
  138. if err := id.Validate(); err != nil {
  139. return fmt.Errorf("invalid PersistentPeer ID %q: %w", id, err)
  140. }
  141. }
  142. for id := range o.PrivatePeers {
  143. if err := id.Validate(); err != nil {
  144. return fmt.Errorf("invalid private peer ID %q: %w", id, err)
  145. }
  146. }
  147. if o.MaxConnected > 0 && len(o.PersistentPeers) > int(o.MaxConnected) {
  148. return fmt.Errorf("number of persistent peers %v can't exceed MaxConnected %v",
  149. len(o.PersistentPeers), o.MaxConnected)
  150. }
  151. if o.MaxPeers > 0 {
  152. if o.MaxConnected == 0 || o.MaxConnected+o.MaxConnectedUpgrade > o.MaxPeers {
  153. return fmt.Errorf(
  154. "MaxConnected %v and MaxConnectedUpgrade %v can't exceed MaxPeers %v",
  155. o.MaxConnected, o.MaxConnectedUpgrade, o.MaxPeers)
  156. }
  157. }
  158. if o.MaxRetryTime > 0 {
  159. if o.MinRetryTime == 0 {
  160. return errors.New("can't set MaxRetryTime without MinRetryTime")
  161. }
  162. if o.MinRetryTime > o.MaxRetryTime {
  163. return fmt.Errorf("MinRetryTime %v is greater than MaxRetryTime %v",
  164. o.MinRetryTime, o.MaxRetryTime)
  165. }
  166. }
  167. if o.MaxRetryTimePersistent > 0 {
  168. if o.MinRetryTime == 0 {
  169. return errors.New("can't set MaxRetryTimePersistent without MinRetryTime")
  170. }
  171. if o.MinRetryTime > o.MaxRetryTimePersistent {
  172. return fmt.Errorf(
  173. "MinRetryTime %v is greater than MaxRetryTimePersistent %v",
  174. o.MinRetryTime, o.MaxRetryTimePersistent)
  175. }
  176. }
  177. return nil
  178. }
  179. // isPersistentPeer checks if a peer is in PersistentPeers. It will panic
  180. // if called before optimize().
  181. func (o *PeerManagerOptions) isPersistent(id types.NodeID) bool {
  182. if o.persistentPeers == nil {
  183. panic("isPersistentPeer() called before optimize()")
  184. }
  185. return o.persistentPeers[id]
  186. }
  187. // optimize optimizes operations by pregenerating lookup structures. It's a
  188. // separate method instead of memoizing during calls to avoid dealing with
  189. // concurrency and mutex overhead.
  190. func (o *PeerManagerOptions) optimize() {
  191. o.persistentPeers = make(map[types.NodeID]bool, len(o.PersistentPeers))
  192. for _, p := range o.PersistentPeers {
  193. o.persistentPeers[p] = true
  194. }
  195. }
  196. // PeerManager manages peer lifecycle information, using a peerStore for
  197. // underlying storage. Its primary purpose is to determine which peer to connect
  198. // to next (including retry timers), make sure a peer only has a single active
  199. // connection (either inbound or outbound), and evict peers to make room for
  200. // higher-scored peers. It does not manage actual connections (this is handled
  201. // by the Router), only the peer lifecycle state.
  202. //
  203. // For an outbound connection, the flow is as follows:
  204. // - DialNext: return a peer address to dial, mark peer as dialing.
  205. // - DialFailed: report a dial failure, unmark as dialing.
  206. // - Dialed: report a dial success, unmark as dialing and mark as connected
  207. // (errors if already connected, e.g. by Accepted).
  208. // - Ready: report routing is ready, mark as ready and broadcast PeerStatusUp.
  209. // - Disconnected: report peer disconnect, unmark as connected and broadcasts
  210. // PeerStatusDown.
  211. //
  212. // For an inbound connection, the flow is as follows:
  213. // - Accepted: report inbound connection success, mark as connected (errors if
  214. // already connected, e.g. by Dialed).
  215. // - Ready: report routing is ready, mark as ready and broadcast PeerStatusUp.
  216. // - Disconnected: report peer disconnect, unmark as connected and broadcasts
  217. // PeerStatusDown.
  218. //
  219. // When evicting peers, either because peers are explicitly scheduled for
  220. // eviction or we are connected to too many peers, the flow is as follows:
  221. // - EvictNext: if marked evict and connected, unmark evict and mark evicting.
  222. // If beyond MaxConnected, pick lowest-scored peer and mark evicting.
  223. // - Disconnected: unmark connected, evicting, evict, and broadcast a
  224. // PeerStatusDown peer update.
  225. //
  226. // If all connection slots are full (at MaxConnections), we can use up to
  227. // MaxConnectionsUpgrade additional connections to probe any higher-scored
  228. // unconnected peers, and if we reach them (or they reach us) we allow the
  229. // connection and evict a lower-scored peer. We mark the lower-scored peer as
  230. // upgrading[from]=to to make sure no other higher-scored peers can claim the
  231. // same one for an upgrade. The flow is as follows:
  232. // - Accepted: if upgrade is possible, mark connected and add lower-scored to evict.
  233. // - DialNext: if upgrade is possible, mark upgrading[from]=to and dialing.
  234. // - DialFailed: unmark upgrading[from]=to and dialing.
  235. // - Dialed: unmark upgrading[from]=to and dialing, mark as connected, add
  236. // lower-scored to evict.
  237. // - EvictNext: pick peer from evict, mark as evicting.
  238. // - Disconnected: unmark connected, upgrading[from]=to, evict, evicting.
  239. type PeerManager struct {
  240. selfID types.NodeID
  241. options PeerManagerOptions
  242. rand *rand.Rand
  243. dialWaker *tmsync.Waker // wakes up DialNext() on relevant peer changes
  244. evictWaker *tmsync.Waker // wakes up EvictNext() on relevant peer changes
  245. closeCh chan struct{} // signal channel for Close()
  246. closeOnce sync.Once
  247. mtx sync.Mutex
  248. store *peerStore
  249. subscriptions map[*PeerUpdates]*PeerUpdates // keyed by struct identity (address)
  250. dialing map[types.NodeID]bool // peers being dialed (DialNext → Dialed/DialFail)
  251. upgrading map[types.NodeID]types.NodeID // peers claimed for upgrade (DialNext → Dialed/DialFail)
  252. connected map[types.NodeID]bool // connected peers (Dialed/Accepted → Disconnected)
  253. ready map[types.NodeID]bool // ready peers (Ready → Disconnected)
  254. evict map[types.NodeID]bool // peers scheduled for eviction (Connected → EvictNext)
  255. evicting map[types.NodeID]bool // peers being evicted (EvictNext → Disconnected)
  256. }
  257. // NewPeerManager creates a new peer manager.
  258. func NewPeerManager(selfID types.NodeID, peerDB dbm.DB, options PeerManagerOptions) (*PeerManager, error) {
  259. if selfID == "" {
  260. return nil, errors.New("self ID not given")
  261. }
  262. if err := options.Validate(); err != nil {
  263. return nil, err
  264. }
  265. options.optimize()
  266. store, err := newPeerStore(peerDB)
  267. if err != nil {
  268. return nil, err
  269. }
  270. peerManager := &PeerManager{
  271. selfID: selfID,
  272. options: options,
  273. rand: rand.New(rand.NewSource(time.Now().UnixNano())), // nolint:gosec
  274. dialWaker: tmsync.NewWaker(),
  275. evictWaker: tmsync.NewWaker(),
  276. closeCh: make(chan struct{}),
  277. store: store,
  278. dialing: map[types.NodeID]bool{},
  279. upgrading: map[types.NodeID]types.NodeID{},
  280. connected: map[types.NodeID]bool{},
  281. ready: map[types.NodeID]bool{},
  282. evict: map[types.NodeID]bool{},
  283. evicting: map[types.NodeID]bool{},
  284. subscriptions: map[*PeerUpdates]*PeerUpdates{},
  285. }
  286. if err = peerManager.configurePeers(); err != nil {
  287. return nil, err
  288. }
  289. if err = peerManager.prunePeers(); err != nil {
  290. return nil, err
  291. }
  292. return peerManager, nil
  293. }
  294. // configurePeers configures peers in the peer store with ephemeral runtime
  295. // configuration, e.g. PersistentPeers. It also removes ourself, if we're in the
  296. // peer store. The caller must hold the mutex lock.
  297. func (m *PeerManager) configurePeers() error {
  298. if err := m.store.Delete(m.selfID); err != nil {
  299. return err
  300. }
  301. configure := map[types.NodeID]bool{}
  302. for _, id := range m.options.PersistentPeers {
  303. configure[id] = true
  304. }
  305. for id := range m.options.PeerScores {
  306. configure[id] = true
  307. }
  308. for id := range configure {
  309. if peer, ok := m.store.Get(id); ok {
  310. if err := m.store.Set(m.configurePeer(peer)); err != nil {
  311. return err
  312. }
  313. }
  314. }
  315. return nil
  316. }
  317. // configurePeer configures a peer with ephemeral runtime configuration.
  318. func (m *PeerManager) configurePeer(peer peerInfo) peerInfo {
  319. peer.Persistent = m.options.isPersistent(peer.ID)
  320. peer.FixedScore = m.options.PeerScores[peer.ID]
  321. return peer
  322. }
  323. // newPeerInfo creates a peerInfo for a new peer.
  324. func (m *PeerManager) newPeerInfo(id types.NodeID) peerInfo {
  325. peerInfo := peerInfo{
  326. ID: id,
  327. AddressInfo: map[NodeAddress]*peerAddressInfo{},
  328. }
  329. return m.configurePeer(peerInfo)
  330. }
  331. // prunePeers removes low-scored peers from the peer store if it contains more
  332. // than MaxPeers peers. The caller must hold the mutex lock.
  333. func (m *PeerManager) prunePeers() error {
  334. if m.options.MaxPeers == 0 || m.store.Size() <= int(m.options.MaxPeers) {
  335. return nil
  336. }
  337. ranked := m.store.Ranked()
  338. for i := len(ranked) - 1; i >= 0; i-- {
  339. peerID := ranked[i].ID
  340. switch {
  341. case m.store.Size() <= int(m.options.MaxPeers):
  342. return nil
  343. case m.dialing[peerID]:
  344. case m.connected[peerID]:
  345. default:
  346. if err := m.store.Delete(peerID); err != nil {
  347. return err
  348. }
  349. }
  350. }
  351. return nil
  352. }
  353. // Add adds a peer to the manager, given as an address. If the peer already
  354. // exists, the address is added to it if it isn't already present. This will push
  355. // low scoring peers out of the address book if it exceeds the maximum size.
  356. func (m *PeerManager) Add(address NodeAddress) (bool, error) {
  357. if err := address.Validate(); err != nil {
  358. return false, err
  359. }
  360. if address.NodeID == m.selfID {
  361. return false, fmt.Errorf("can't add self (%v) to peer store", m.selfID)
  362. }
  363. m.mtx.Lock()
  364. defer m.mtx.Unlock()
  365. peer, ok := m.store.Get(address.NodeID)
  366. if !ok {
  367. peer = m.newPeerInfo(address.NodeID)
  368. }
  369. _, ok = peer.AddressInfo[address]
  370. // if we already have the peer address, there's no need to continue
  371. if ok {
  372. return false, nil
  373. }
  374. // else add the new address
  375. peer.AddressInfo[address] = &peerAddressInfo{Address: address}
  376. if err := m.store.Set(peer); err != nil {
  377. return false, err
  378. }
  379. if err := m.prunePeers(); err != nil {
  380. return true, err
  381. }
  382. m.dialWaker.Wake()
  383. return true, nil
  384. }
  385. // PeerRatio returns the ratio of peer addresses stored to the maximum size.
  386. func (m *PeerManager) PeerRatio() float64 {
  387. m.mtx.Lock()
  388. defer m.mtx.Unlock()
  389. if m.options.MaxPeers == 0 {
  390. return 0
  391. }
  392. return float64(m.store.Size()) / float64(m.options.MaxPeers)
  393. }
  394. // DialNext finds an appropriate peer address to dial, and marks it as dialing.
  395. // If no peer is found, or all connection slots are full, it blocks until one
  396. // becomes available. The caller must call Dialed() or DialFailed() for the
  397. // returned peer.
  398. func (m *PeerManager) DialNext(ctx context.Context) (NodeAddress, error) {
  399. for {
  400. address, err := m.TryDialNext()
  401. if err != nil || (address != NodeAddress{}) {
  402. return address, err
  403. }
  404. select {
  405. case <-m.dialWaker.Sleep():
  406. case <-ctx.Done():
  407. return NodeAddress{}, ctx.Err()
  408. }
  409. }
  410. }
  411. // TryDialNext is equivalent to DialNext(), but immediately returns an empty
  412. // address if no peers or connection slots are available.
  413. func (m *PeerManager) TryDialNext() (NodeAddress, error) {
  414. m.mtx.Lock()
  415. defer m.mtx.Unlock()
  416. // We allow dialing MaxConnected+MaxConnectedUpgrade peers. Including
  417. // MaxConnectedUpgrade allows us to probe additional peers that have a
  418. // higher score than any other peers, and if successful evict it.
  419. if m.options.MaxConnected > 0 && len(m.connected)+len(m.dialing) >=
  420. int(m.options.MaxConnected)+int(m.options.MaxConnectedUpgrade) {
  421. return NodeAddress{}, nil
  422. }
  423. for _, peer := range m.store.Ranked() {
  424. if m.dialing[peer.ID] || m.connected[peer.ID] {
  425. continue
  426. }
  427. for _, addressInfo := range peer.AddressInfo {
  428. if time.Since(addressInfo.LastDialFailure) < m.retryDelay(addressInfo.DialFailures, peer.Persistent) {
  429. continue
  430. }
  431. // We now have an eligible address to dial. If we're full but have
  432. // upgrade capacity (as checked above), we find a lower-scored peer
  433. // we can replace and mark it as upgrading so noone else claims it.
  434. //
  435. // If we don't find one, there is no point in trying additional
  436. // peers, since they will all have the same or lower score than this
  437. // peer (since they're ordered by score via peerStore.Ranked).
  438. if m.options.MaxConnected > 0 && len(m.connected) >= int(m.options.MaxConnected) {
  439. upgradeFromPeer := m.findUpgradeCandidate(peer.ID, peer.Score())
  440. if upgradeFromPeer == "" {
  441. return NodeAddress{}, nil
  442. }
  443. m.upgrading[upgradeFromPeer] = peer.ID
  444. }
  445. m.dialing[peer.ID] = true
  446. return addressInfo.Address, nil
  447. }
  448. }
  449. return NodeAddress{}, nil
  450. }
  451. // DialFailed reports a failed dial attempt. This will make the peer available
  452. // for dialing again when appropriate (possibly after a retry timeout).
  453. //
  454. // FIXME: This should probably delete or mark bad addresses/peers after some time.
  455. func (m *PeerManager) DialFailed(address NodeAddress) error {
  456. m.mtx.Lock()
  457. defer m.mtx.Unlock()
  458. delete(m.dialing, address.NodeID)
  459. for from, to := range m.upgrading {
  460. if to == address.NodeID {
  461. delete(m.upgrading, from) // Unmark failed upgrade attempt.
  462. }
  463. }
  464. peer, ok := m.store.Get(address.NodeID)
  465. if !ok { // Peer may have been removed while dialing, ignore.
  466. return nil
  467. }
  468. addressInfo, ok := peer.AddressInfo[address]
  469. if !ok {
  470. return nil // Assume the address has been removed, ignore.
  471. }
  472. addressInfo.LastDialFailure = time.Now().UTC()
  473. addressInfo.DialFailures++
  474. if err := m.store.Set(peer); err != nil {
  475. return err
  476. }
  477. // We spawn a goroutine that notifies DialNext() again when the retry
  478. // timeout has elapsed, so that we can consider dialing it again. We
  479. // calculate the retry delay outside the goroutine, since it must hold
  480. // the mutex lock.
  481. if d := m.retryDelay(addressInfo.DialFailures, peer.Persistent); d != 0 && d != retryNever {
  482. go func() {
  483. // Use an explicit timer with deferred cleanup instead of
  484. // time.After(), to avoid leaking goroutines on PeerManager.Close().
  485. timer := time.NewTimer(d)
  486. defer timer.Stop()
  487. select {
  488. case <-timer.C:
  489. m.dialWaker.Wake()
  490. case <-m.closeCh:
  491. }
  492. }()
  493. } else {
  494. m.dialWaker.Wake()
  495. }
  496. return nil
  497. }
  498. // Dialed marks a peer as successfully dialed. Any further connections will be
  499. // rejected, and once disconnected the peer may be dialed again.
  500. func (m *PeerManager) Dialed(address NodeAddress) error {
  501. m.mtx.Lock()
  502. defer m.mtx.Unlock()
  503. delete(m.dialing, address.NodeID)
  504. var upgradeFromPeer types.NodeID
  505. for from, to := range m.upgrading {
  506. if to == address.NodeID {
  507. delete(m.upgrading, from)
  508. upgradeFromPeer = from
  509. // Don't break, just in case this peer was marked as upgrading for
  510. // multiple lower-scored peers (shouldn't really happen).
  511. }
  512. }
  513. if address.NodeID == m.selfID {
  514. return fmt.Errorf("rejecting connection to self (%v)", address.NodeID)
  515. }
  516. if m.connected[address.NodeID] {
  517. return fmt.Errorf("peer %v is already connected", address.NodeID)
  518. }
  519. if m.options.MaxConnected > 0 && len(m.connected) >= int(m.options.MaxConnected) {
  520. if upgradeFromPeer == "" || len(m.connected) >=
  521. int(m.options.MaxConnected)+int(m.options.MaxConnectedUpgrade) {
  522. return fmt.Errorf("already connected to maximum number of peers")
  523. }
  524. }
  525. peer, ok := m.store.Get(address.NodeID)
  526. if !ok {
  527. return fmt.Errorf("peer %q was removed while dialing", address.NodeID)
  528. }
  529. now := time.Now().UTC()
  530. peer.LastConnected = now
  531. if addressInfo, ok := peer.AddressInfo[address]; ok {
  532. addressInfo.DialFailures = 0
  533. addressInfo.LastDialSuccess = now
  534. // If not found, assume address has been removed.
  535. }
  536. if err := m.store.Set(peer); err != nil {
  537. return err
  538. }
  539. if upgradeFromPeer != "" && m.options.MaxConnected > 0 &&
  540. len(m.connected) >= int(m.options.MaxConnected) {
  541. // Look for an even lower-scored peer that may have appeared since we
  542. // started the upgrade.
  543. if p, ok := m.store.Get(upgradeFromPeer); ok {
  544. if u := m.findUpgradeCandidate(p.ID, p.Score()); u != "" {
  545. upgradeFromPeer = u
  546. }
  547. }
  548. m.evict[upgradeFromPeer] = true
  549. }
  550. m.connected[peer.ID] = true
  551. m.evictWaker.Wake()
  552. return nil
  553. }
  554. // Accepted marks an incoming peer connection successfully accepted. If the peer
  555. // is already connected or we don't allow additional connections then this will
  556. // return an error.
  557. //
  558. // If full but MaxConnectedUpgrade is non-zero and the incoming peer is
  559. // better-scored than any existing peers, then we accept it and evict a
  560. // lower-scored peer.
  561. //
  562. // NOTE: We can't take an address here, since e.g. TCP uses a different port
  563. // number for outbound traffic than inbound traffic, so the peer's endpoint
  564. // wouldn't necessarily be an appropriate address to dial.
  565. //
  566. // FIXME: When we accept a connection from a peer, we should register that
  567. // peer's address in the peer store so that we can dial it later. In order to do
  568. // that, we'll need to get the remote address after all, but as noted above that
  569. // can't be the remote endpoint since that will usually have the wrong port
  570. // number.
  571. func (m *PeerManager) Accepted(peerID types.NodeID) error {
  572. m.mtx.Lock()
  573. defer m.mtx.Unlock()
  574. if peerID == m.selfID {
  575. return fmt.Errorf("rejecting connection from self (%v)", peerID)
  576. }
  577. if m.connected[peerID] {
  578. return fmt.Errorf("peer %q is already connected", peerID)
  579. }
  580. if m.options.MaxConnected > 0 &&
  581. len(m.connected) >= int(m.options.MaxConnected)+int(m.options.MaxConnectedUpgrade) {
  582. return fmt.Errorf("already connected to maximum number of peers")
  583. }
  584. peer, ok := m.store.Get(peerID)
  585. if !ok {
  586. peer = m.newPeerInfo(peerID)
  587. }
  588. // If all connections slots are full, but we allow upgrades (and we checked
  589. // above that we have upgrade capacity), then we can look for a lower-scored
  590. // peer to replace and if found accept the connection anyway and evict it.
  591. var upgradeFromPeer types.NodeID
  592. if m.options.MaxConnected > 0 && len(m.connected) >= int(m.options.MaxConnected) {
  593. upgradeFromPeer = m.findUpgradeCandidate(peer.ID, peer.Score())
  594. if upgradeFromPeer == "" {
  595. return fmt.Errorf("already connected to maximum number of peers")
  596. }
  597. }
  598. peer.LastConnected = time.Now().UTC()
  599. if err := m.store.Set(peer); err != nil {
  600. return err
  601. }
  602. m.connected[peerID] = true
  603. if upgradeFromPeer != "" {
  604. m.evict[upgradeFromPeer] = true
  605. }
  606. m.evictWaker.Wake()
  607. return nil
  608. }
  609. // Ready marks a peer as ready, broadcasting status updates to subscribers. The
  610. // peer must already be marked as connected. This is separate from Dialed() and
  611. // Accepted() to allow the router to set up its internal queues before reactors
  612. // start sending messages.
  613. func (m *PeerManager) Ready(peerID types.NodeID) {
  614. m.mtx.Lock()
  615. defer m.mtx.Unlock()
  616. if m.connected[peerID] {
  617. m.ready[peerID] = true
  618. m.broadcast(PeerUpdate{
  619. NodeID: peerID,
  620. Status: PeerStatusUp,
  621. })
  622. }
  623. }
  624. // EvictNext returns the next peer to evict (i.e. disconnect). If no evictable
  625. // peers are found, the call will block until one becomes available.
  626. func (m *PeerManager) EvictNext(ctx context.Context) (types.NodeID, error) {
  627. for {
  628. id, err := m.TryEvictNext()
  629. if err != nil || id != "" {
  630. return id, err
  631. }
  632. select {
  633. case <-m.evictWaker.Sleep():
  634. case <-ctx.Done():
  635. return "", ctx.Err()
  636. }
  637. }
  638. }
  639. // TryEvictNext is equivalent to EvictNext, but immediately returns an empty
  640. // node ID if no evictable peers are found.
  641. func (m *PeerManager) TryEvictNext() (types.NodeID, error) {
  642. m.mtx.Lock()
  643. defer m.mtx.Unlock()
  644. // If any connected peers are explicitly scheduled for eviction, we return a
  645. // random one.
  646. for peerID := range m.evict {
  647. delete(m.evict, peerID)
  648. if m.connected[peerID] && !m.evicting[peerID] {
  649. m.evicting[peerID] = true
  650. return peerID, nil
  651. }
  652. }
  653. // If we're below capacity, we don't need to evict anything.
  654. if m.options.MaxConnected == 0 ||
  655. len(m.connected)-len(m.evicting) <= int(m.options.MaxConnected) {
  656. return "", nil
  657. }
  658. // If we're above capacity (shouldn't really happen), just pick the
  659. // lowest-ranked peer to evict.
  660. ranked := m.store.Ranked()
  661. for i := len(ranked) - 1; i >= 0; i-- {
  662. peer := ranked[i]
  663. if m.connected[peer.ID] && !m.evicting[peer.ID] {
  664. m.evicting[peer.ID] = true
  665. return peer.ID, nil
  666. }
  667. }
  668. return "", nil
  669. }
  670. // Disconnected unmarks a peer as connected, allowing it to be dialed or
  671. // accepted again as appropriate.
  672. func (m *PeerManager) Disconnected(peerID types.NodeID) {
  673. m.mtx.Lock()
  674. defer m.mtx.Unlock()
  675. ready := m.ready[peerID]
  676. delete(m.connected, peerID)
  677. delete(m.upgrading, peerID)
  678. delete(m.evict, peerID)
  679. delete(m.evicting, peerID)
  680. delete(m.ready, peerID)
  681. if ready {
  682. m.broadcast(PeerUpdate{
  683. NodeID: peerID,
  684. Status: PeerStatusDown,
  685. })
  686. }
  687. m.dialWaker.Wake()
  688. }
  689. // Errored reports a peer error, causing the peer to be evicted if it's
  690. // currently connected.
  691. //
  692. // FIXME: This should probably be replaced with a peer behavior API, see
  693. // PeerError comments for more details.
  694. //
  695. // FIXME: This will cause the peer manager to immediately try to reconnect to
  696. // the peer, which is probably not always what we want.
  697. func (m *PeerManager) Errored(peerID types.NodeID, err error) {
  698. m.mtx.Lock()
  699. defer m.mtx.Unlock()
  700. if m.connected[peerID] {
  701. m.evict[peerID] = true
  702. }
  703. m.evictWaker.Wake()
  704. }
  705. // Advertise returns a list of peer addresses to advertise to a peer.
  706. //
  707. // FIXME: This is fairly naïve and only returns the addresses of the
  708. // highest-ranked peers.
  709. func (m *PeerManager) Advertise(peerID types.NodeID, limit uint16) []NodeAddress {
  710. m.mtx.Lock()
  711. defer m.mtx.Unlock()
  712. addresses := make([]NodeAddress, 0, limit)
  713. for _, peer := range m.store.Ranked() {
  714. if peer.ID == peerID {
  715. continue
  716. }
  717. for nodeAddr, addressInfo := range peer.AddressInfo {
  718. if len(addresses) >= int(limit) {
  719. return addresses
  720. }
  721. // only add non-private NodeIDs
  722. if _, ok := m.options.PrivatePeers[nodeAddr.NodeID]; !ok {
  723. addresses = append(addresses, addressInfo.Address)
  724. }
  725. }
  726. }
  727. return addresses
  728. }
  729. // Subscribe subscribes to peer updates. The caller must consume the peer
  730. // updates in a timely fashion and close the subscription when done, otherwise
  731. // the PeerManager will halt.
  732. func (m *PeerManager) Subscribe() *PeerUpdates {
  733. // FIXME: We use a size 1 buffer here. When we broadcast a peer update
  734. // we have to loop over all of the subscriptions, and we want to avoid
  735. // having to block and wait for a context switch before continuing on
  736. // to the next subscriptions. This also prevents tail latencies from
  737. // compounding. Limiting it to 1 means that the subscribers are still
  738. // reasonably in sync. However, this should probably be benchmarked.
  739. peerUpdates := NewPeerUpdates(make(chan PeerUpdate, 1), 1)
  740. m.Register(peerUpdates)
  741. return peerUpdates
  742. }
  743. // Register allows you to inject a custom PeerUpdate instance into the
  744. // PeerManager, rather than relying on the instance constructed by the
  745. // Subscribe method, which wraps the functionality of the Register
  746. // method.
  747. //
  748. // The caller must consume the peer updates from this PeerUpdates
  749. // instance in a timely fashion and close the subscription when done,
  750. // otherwise the PeerManager will halt.
  751. func (m *PeerManager) Register(peerUpdates *PeerUpdates) {
  752. m.mtx.Lock()
  753. m.subscriptions[peerUpdates] = peerUpdates
  754. m.mtx.Unlock()
  755. go func() {
  756. for {
  757. select {
  758. case <-peerUpdates.closeCh:
  759. return
  760. case <-m.closeCh:
  761. return
  762. case pu := <-peerUpdates.routerUpdatesCh:
  763. m.processPeerEvent(pu)
  764. }
  765. }
  766. }()
  767. go func() {
  768. select {
  769. case <-peerUpdates.Done():
  770. m.mtx.Lock()
  771. delete(m.subscriptions, peerUpdates)
  772. m.mtx.Unlock()
  773. case <-m.closeCh:
  774. }
  775. }()
  776. }
  777. func (m *PeerManager) processPeerEvent(pu PeerUpdate) {
  778. m.mtx.Lock()
  779. defer m.mtx.Unlock()
  780. if _, ok := m.store.peers[pu.NodeID]; !ok {
  781. m.store.peers[pu.NodeID] = &peerInfo{}
  782. }
  783. switch pu.Status {
  784. case PeerStatusBad:
  785. m.store.peers[pu.NodeID].MutableScore--
  786. case PeerStatusGood:
  787. m.store.peers[pu.NodeID].MutableScore++
  788. }
  789. }
  790. // broadcast broadcasts a peer update to all subscriptions. The caller must
  791. // already hold the mutex lock, to make sure updates are sent in the same order
  792. // as the PeerManager processes them, but this means subscribers must be
  793. // responsive at all times or the entire PeerManager will halt.
  794. //
  795. // FIXME: Consider using an internal channel to buffer updates while also
  796. // maintaining order if this is a problem.
  797. func (m *PeerManager) broadcast(peerUpdate PeerUpdate) {
  798. for _, sub := range m.subscriptions {
  799. // We have to check closeCh separately first, otherwise there's a 50%
  800. // chance the second select will send on a closed subscription.
  801. select {
  802. case <-sub.closeCh:
  803. continue
  804. default:
  805. }
  806. select {
  807. case sub.reactorUpdatesCh <- peerUpdate:
  808. case <-sub.closeCh:
  809. }
  810. }
  811. }
  812. // Close closes the peer manager, releasing resources (i.e. goroutines).
  813. func (m *PeerManager) Close() {
  814. m.closeOnce.Do(func() {
  815. close(m.closeCh)
  816. })
  817. }
  818. // Addresses returns all known addresses for a peer, primarily for testing.
  819. // The order is arbitrary.
  820. func (m *PeerManager) Addresses(peerID types.NodeID) []NodeAddress {
  821. m.mtx.Lock()
  822. defer m.mtx.Unlock()
  823. addresses := []NodeAddress{}
  824. if peer, ok := m.store.Get(peerID); ok {
  825. for _, addressInfo := range peer.AddressInfo {
  826. addresses = append(addresses, addressInfo.Address)
  827. }
  828. }
  829. return addresses
  830. }
  831. // Peers returns all known peers, primarily for testing. The order is arbitrary.
  832. func (m *PeerManager) Peers() []types.NodeID {
  833. m.mtx.Lock()
  834. defer m.mtx.Unlock()
  835. peers := []types.NodeID{}
  836. for _, peer := range m.store.Ranked() {
  837. peers = append(peers, peer.ID)
  838. }
  839. return peers
  840. }
  841. // Scores returns the peer scores for all known peers, primarily for testing.
  842. func (m *PeerManager) Scores() map[types.NodeID]PeerScore {
  843. m.mtx.Lock()
  844. defer m.mtx.Unlock()
  845. scores := map[types.NodeID]PeerScore{}
  846. for _, peer := range m.store.Ranked() {
  847. scores[peer.ID] = peer.Score()
  848. }
  849. return scores
  850. }
  851. // Status returns the status for a peer, primarily for testing.
  852. func (m *PeerManager) Status(id types.NodeID) PeerStatus {
  853. m.mtx.Lock()
  854. defer m.mtx.Unlock()
  855. switch {
  856. case m.ready[id]:
  857. return PeerStatusUp
  858. default:
  859. return PeerStatusDown
  860. }
  861. }
  862. // findUpgradeCandidate looks for a lower-scored peer that we could evict
  863. // to make room for the given peer. Returns an empty ID if none is found.
  864. // If the peer is already being upgraded to, we return that same upgrade.
  865. // The caller must hold the mutex lock.
  866. func (m *PeerManager) findUpgradeCandidate(id types.NodeID, score PeerScore) types.NodeID {
  867. for from, to := range m.upgrading {
  868. if to == id {
  869. return from
  870. }
  871. }
  872. ranked := m.store.Ranked()
  873. for i := len(ranked) - 1; i >= 0; i-- {
  874. candidate := ranked[i]
  875. switch {
  876. case candidate.Score() >= score:
  877. return "" // no further peers can be scored lower, due to sorting
  878. case !m.connected[candidate.ID]:
  879. case m.evict[candidate.ID]:
  880. case m.evicting[candidate.ID]:
  881. case m.upgrading[candidate.ID] != "":
  882. default:
  883. return candidate.ID
  884. }
  885. }
  886. return ""
  887. }
  888. // retryDelay calculates a dial retry delay using exponential backoff, based on
  889. // retry settings in PeerManagerOptions. If retries are disabled (i.e.
  890. // MinRetryTime is 0), this returns retryNever (i.e. an infinite retry delay).
  891. // The caller must hold the mutex lock (for m.rand which is not thread-safe).
  892. func (m *PeerManager) retryDelay(failures uint32, persistent bool) time.Duration {
  893. if failures == 0 {
  894. return 0
  895. }
  896. if m.options.MinRetryTime == 0 {
  897. return retryNever
  898. }
  899. maxDelay := m.options.MaxRetryTime
  900. if persistent && m.options.MaxRetryTimePersistent > 0 {
  901. maxDelay = m.options.MaxRetryTimePersistent
  902. }
  903. delay := m.options.MinRetryTime * time.Duration(math.Pow(2, float64(failures-1)))
  904. if maxDelay > 0 && delay > maxDelay {
  905. delay = maxDelay
  906. }
  907. if m.options.RetryTimeJitter > 0 {
  908. delay += time.Duration(m.rand.Int63n(int64(m.options.RetryTimeJitter)))
  909. }
  910. return delay
  911. }
  912. // GetHeight returns a peer's height, as reported via SetHeight, or 0 if the
  913. // peer or height is unknown.
  914. //
  915. // FIXME: This is a temporary workaround to share state between the consensus
  916. // and mempool reactors, carried over from the legacy P2P stack. Reactors should
  917. // not have dependencies on each other, instead tracking this themselves.
  918. func (m *PeerManager) GetHeight(peerID types.NodeID) int64 {
  919. m.mtx.Lock()
  920. defer m.mtx.Unlock()
  921. peer, _ := m.store.Get(peerID)
  922. return peer.Height
  923. }
  924. // SetHeight stores a peer's height, making it available via GetHeight.
  925. //
  926. // FIXME: This is a temporary workaround to share state between the consensus
  927. // and mempool reactors, carried over from the legacy P2P stack. Reactors should
  928. // not have dependencies on each other, instead tracking this themselves.
  929. func (m *PeerManager) SetHeight(peerID types.NodeID, height int64) error {
  930. m.mtx.Lock()
  931. defer m.mtx.Unlock()
  932. peer, ok := m.store.Get(peerID)
  933. if !ok {
  934. peer = m.newPeerInfo(peerID)
  935. }
  936. peer.Height = height
  937. return m.store.Set(peer)
  938. }
  939. // peerStore stores information about peers. It is not thread-safe, assuming it
  940. // is only used by PeerManager which handles concurrency control. This allows
  941. // the manager to execute multiple operations atomically via its own mutex.
  942. //
  943. // The entire set of peers is kept in memory, for performance. It is loaded
  944. // from disk on initialization, and any changes are written back to disk
  945. // (without fsync, since we can afford to lose recent writes).
  946. type peerStore struct {
  947. db dbm.DB
  948. peers map[types.NodeID]*peerInfo
  949. ranked []*peerInfo // cache for Ranked(), nil invalidates cache
  950. }
  951. // newPeerStore creates a new peer store, loading all persisted peers from the
  952. // database into memory.
  953. func newPeerStore(db dbm.DB) (*peerStore, error) {
  954. if db == nil {
  955. return nil, errors.New("no database provided")
  956. }
  957. store := &peerStore{db: db}
  958. if err := store.loadPeers(); err != nil {
  959. return nil, err
  960. }
  961. return store, nil
  962. }
  963. // loadPeers loads all peers from the database into memory.
  964. func (s *peerStore) loadPeers() error {
  965. peers := map[types.NodeID]*peerInfo{}
  966. start, end := keyPeerInfoRange()
  967. iter, err := s.db.Iterator(start, end)
  968. if err != nil {
  969. return err
  970. }
  971. defer iter.Close()
  972. for ; iter.Valid(); iter.Next() {
  973. // FIXME: We may want to tolerate failures here, by simply logging
  974. // the errors and ignoring the faulty peer entries.
  975. msg := new(p2pproto.PeerInfo)
  976. if err := proto.Unmarshal(iter.Value(), msg); err != nil {
  977. return fmt.Errorf("invalid peer Protobuf data: %w", err)
  978. }
  979. peer, err := peerInfoFromProto(msg)
  980. if err != nil {
  981. return fmt.Errorf("invalid peer data: %w", err)
  982. }
  983. peers[peer.ID] = peer
  984. }
  985. if iter.Error() != nil {
  986. return iter.Error()
  987. }
  988. s.peers = peers
  989. s.ranked = nil // invalidate cache if populated
  990. return nil
  991. }
  992. // Get fetches a peer. The boolean indicates whether the peer existed or not.
  993. // The returned peer info is a copy, and can be mutated at will.
  994. func (s *peerStore) Get(id types.NodeID) (peerInfo, bool) {
  995. peer, ok := s.peers[id]
  996. return peer.Copy(), ok
  997. }
  998. // Set stores peer data. The input data will be copied, and can safely be reused
  999. // by the caller.
  1000. func (s *peerStore) Set(peer peerInfo) error {
  1001. if err := peer.Validate(); err != nil {
  1002. return err
  1003. }
  1004. peer = peer.Copy()
  1005. // FIXME: We may want to optimize this by avoiding saving to the database
  1006. // if there haven't been any changes to persisted fields.
  1007. bz, err := peer.ToProto().Marshal()
  1008. if err != nil {
  1009. return err
  1010. }
  1011. if err = s.db.Set(keyPeerInfo(peer.ID), bz); err != nil {
  1012. return err
  1013. }
  1014. if current, ok := s.peers[peer.ID]; !ok || current.Score() != peer.Score() {
  1015. // If the peer is new, or its score changes, we invalidate the Ranked() cache.
  1016. s.peers[peer.ID] = &peer
  1017. s.ranked = nil
  1018. } else {
  1019. // Otherwise, since s.ranked contains pointers to the old data and we
  1020. // want those pointers to remain valid with the new data, we have to
  1021. // update the existing pointer address.
  1022. *current = peer
  1023. }
  1024. return nil
  1025. }
  1026. // Delete deletes a peer, or does nothing if it does not exist.
  1027. func (s *peerStore) Delete(id types.NodeID) error {
  1028. if _, ok := s.peers[id]; !ok {
  1029. return nil
  1030. }
  1031. if err := s.db.Delete(keyPeerInfo(id)); err != nil {
  1032. return err
  1033. }
  1034. delete(s.peers, id)
  1035. s.ranked = nil
  1036. return nil
  1037. }
  1038. // List retrieves all peers in an arbitrary order. The returned data is a copy,
  1039. // and can be mutated at will.
  1040. func (s *peerStore) List() []peerInfo {
  1041. peers := make([]peerInfo, 0, len(s.peers))
  1042. for _, peer := range s.peers {
  1043. peers = append(peers, peer.Copy())
  1044. }
  1045. return peers
  1046. }
  1047. // Ranked returns a list of peers ordered by score (better peers first). Peers
  1048. // with equal scores are returned in an arbitrary order. The returned list must
  1049. // not be mutated or accessed concurrently by the caller, since it returns
  1050. // pointers to internal peerStore data for performance.
  1051. //
  1052. // Ranked is used to determine both which peers to dial, which ones to evict,
  1053. // and which ones to delete completely.
  1054. //
  1055. // FIXME: For now, we simply maintain a cache in s.ranked which is invalidated
  1056. // by setting it to nil, but if necessary we should use a better data structure
  1057. // for this (e.g. a heap or ordered map).
  1058. //
  1059. // FIXME: The scoring logic is currently very naïve, see peerInfo.Score().
  1060. func (s *peerStore) Ranked() []*peerInfo {
  1061. if s.ranked != nil {
  1062. return s.ranked
  1063. }
  1064. s.ranked = make([]*peerInfo, 0, len(s.peers))
  1065. for _, peer := range s.peers {
  1066. s.ranked = append(s.ranked, peer)
  1067. }
  1068. sort.Slice(s.ranked, func(i, j int) bool {
  1069. // FIXME: If necessary, consider precomputing scores before sorting,
  1070. // to reduce the number of Score() calls.
  1071. return s.ranked[i].Score() > s.ranked[j].Score()
  1072. })
  1073. return s.ranked
  1074. }
  1075. // Size returns the number of peers in the peer store.
  1076. func (s *peerStore) Size() int {
  1077. return len(s.peers)
  1078. }
  1079. // peerInfo contains peer information stored in a peerStore.
  1080. type peerInfo struct {
  1081. ID types.NodeID
  1082. AddressInfo map[NodeAddress]*peerAddressInfo
  1083. LastConnected time.Time
  1084. // These fields are ephemeral, i.e. not persisted to the database.
  1085. Persistent bool
  1086. Height int64
  1087. FixedScore PeerScore // mainly for tests
  1088. MutableScore int64 // updated by router
  1089. }
  1090. // peerInfoFromProto converts a Protobuf PeerInfo message to a peerInfo,
  1091. // erroring if the data is invalid.
  1092. func peerInfoFromProto(msg *p2pproto.PeerInfo) (*peerInfo, error) {
  1093. p := &peerInfo{
  1094. ID: types.NodeID(msg.ID),
  1095. AddressInfo: map[NodeAddress]*peerAddressInfo{},
  1096. }
  1097. if msg.LastConnected != nil {
  1098. p.LastConnected = *msg.LastConnected
  1099. }
  1100. for _, a := range msg.AddressInfo {
  1101. addressInfo, err := peerAddressInfoFromProto(a)
  1102. if err != nil {
  1103. return nil, err
  1104. }
  1105. p.AddressInfo[addressInfo.Address] = addressInfo
  1106. }
  1107. return p, p.Validate()
  1108. }
  1109. // ToProto converts the peerInfo to p2pproto.PeerInfo for database storage. The
  1110. // Protobuf type only contains persisted fields, while ephemeral fields are
  1111. // discarded. The returned message may contain pointers to original data, since
  1112. // it is expected to be serialized immediately.
  1113. func (p *peerInfo) ToProto() *p2pproto.PeerInfo {
  1114. msg := &p2pproto.PeerInfo{
  1115. ID: string(p.ID),
  1116. LastConnected: &p.LastConnected,
  1117. }
  1118. for _, addressInfo := range p.AddressInfo {
  1119. msg.AddressInfo = append(msg.AddressInfo, addressInfo.ToProto())
  1120. }
  1121. if msg.LastConnected.IsZero() {
  1122. msg.LastConnected = nil
  1123. }
  1124. return msg
  1125. }
  1126. // Copy returns a deep copy of the peer info.
  1127. func (p *peerInfo) Copy() peerInfo {
  1128. if p == nil {
  1129. return peerInfo{}
  1130. }
  1131. c := *p
  1132. for i, addressInfo := range c.AddressInfo {
  1133. addressInfoCopy := addressInfo.Copy()
  1134. c.AddressInfo[i] = &addressInfoCopy
  1135. }
  1136. return c
  1137. }
  1138. // Score calculates a score for the peer. Higher-scored peers will be
  1139. // preferred over lower scores.
  1140. func (p *peerInfo) Score() PeerScore {
  1141. if p.FixedScore > 0 {
  1142. return p.FixedScore
  1143. }
  1144. if p.Persistent {
  1145. return PeerScorePersistent
  1146. }
  1147. if p.MutableScore <= 0 {
  1148. return 0
  1149. }
  1150. if p.MutableScore >= math.MaxUint8 {
  1151. return PeerScore(math.MaxUint8)
  1152. }
  1153. return PeerScore(p.MutableScore)
  1154. }
  1155. // Validate validates the peer info.
  1156. func (p *peerInfo) Validate() error {
  1157. if p.ID == "" {
  1158. return errors.New("no peer ID")
  1159. }
  1160. return nil
  1161. }
  1162. // peerAddressInfo contains information and statistics about a peer address.
  1163. type peerAddressInfo struct {
  1164. Address NodeAddress
  1165. LastDialSuccess time.Time
  1166. LastDialFailure time.Time
  1167. DialFailures uint32 // since last successful dial
  1168. }
  1169. // peerAddressInfoFromProto converts a Protobuf PeerAddressInfo message
  1170. // to a peerAddressInfo.
  1171. func peerAddressInfoFromProto(msg *p2pproto.PeerAddressInfo) (*peerAddressInfo, error) {
  1172. address, err := ParseNodeAddress(msg.Address)
  1173. if err != nil {
  1174. return nil, fmt.Errorf("invalid address %q: %w", address, err)
  1175. }
  1176. addressInfo := &peerAddressInfo{
  1177. Address: address,
  1178. DialFailures: msg.DialFailures,
  1179. }
  1180. if msg.LastDialSuccess != nil {
  1181. addressInfo.LastDialSuccess = *msg.LastDialSuccess
  1182. }
  1183. if msg.LastDialFailure != nil {
  1184. addressInfo.LastDialFailure = *msg.LastDialFailure
  1185. }
  1186. return addressInfo, addressInfo.Validate()
  1187. }
  1188. // ToProto converts the address into to a Protobuf message for serialization.
  1189. func (a *peerAddressInfo) ToProto() *p2pproto.PeerAddressInfo {
  1190. msg := &p2pproto.PeerAddressInfo{
  1191. Address: a.Address.String(),
  1192. LastDialSuccess: &a.LastDialSuccess,
  1193. LastDialFailure: &a.LastDialFailure,
  1194. DialFailures: a.DialFailures,
  1195. }
  1196. if msg.LastDialSuccess.IsZero() {
  1197. msg.LastDialSuccess = nil
  1198. }
  1199. if msg.LastDialFailure.IsZero() {
  1200. msg.LastDialFailure = nil
  1201. }
  1202. return msg
  1203. }
  1204. // Copy returns a copy of the address info.
  1205. func (a *peerAddressInfo) Copy() peerAddressInfo {
  1206. return *a
  1207. }
  1208. // Validate validates the address info.
  1209. func (a *peerAddressInfo) Validate() error {
  1210. return a.Address.Validate()
  1211. }
  1212. // Database key prefixes.
  1213. const (
  1214. prefixPeerInfo int64 = 1
  1215. )
  1216. // keyPeerInfo generates a peerInfo database key.
  1217. func keyPeerInfo(id types.NodeID) []byte {
  1218. key, err := orderedcode.Append(nil, prefixPeerInfo, string(id))
  1219. if err != nil {
  1220. panic(err)
  1221. }
  1222. return key
  1223. }
  1224. // keyPeerInfoRange generates start/end keys for the entire peerInfo key range.
  1225. func keyPeerInfoRange() ([]byte, []byte) {
  1226. start, err := orderedcode.Append(nil, prefixPeerInfo, "")
  1227. if err != nil {
  1228. panic(err)
  1229. }
  1230. end, err := orderedcode.Append(nil, prefixPeerInfo, orderedcode.Infinity)
  1231. if err != nil {
  1232. panic(err)
  1233. }
  1234. return start, end
  1235. }