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.

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