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.

676 lines
19 KiB

  1. package p2p
  2. import (
  3. "fmt"
  4. "math"
  5. "sync"
  6. "time"
  7. "github.com/tendermint/tendermint/config"
  8. cmn "github.com/tendermint/tendermint/libs/common"
  9. "github.com/tendermint/tendermint/p2p/conn"
  10. )
  11. const (
  12. // wait a random amount of time from this interval
  13. // before dialing peers or reconnecting to help prevent DoS
  14. dialRandomizerIntervalMilliseconds = 3000
  15. // repeatedly try to reconnect for a few minutes
  16. // ie. 5 * 20 = 100s
  17. reconnectAttempts = 20
  18. reconnectInterval = 5 * time.Second
  19. // then move into exponential backoff mode for ~1day
  20. // ie. 3**10 = 16hrs
  21. reconnectBackOffAttempts = 10
  22. reconnectBackOffBaseSeconds = 3
  23. )
  24. // MConnConfig returns an MConnConfig with fields updated
  25. // from the P2PConfig.
  26. func MConnConfig(cfg *config.P2PConfig) conn.MConnConfig {
  27. mConfig := conn.DefaultMConnConfig()
  28. mConfig.FlushThrottle = cfg.FlushThrottleTimeout
  29. mConfig.SendRate = cfg.SendRate
  30. mConfig.RecvRate = cfg.RecvRate
  31. mConfig.MaxPacketMsgPayloadSize = cfg.MaxPacketMsgPayloadSize
  32. return mConfig
  33. }
  34. //-----------------------------------------------------------------------------
  35. // An AddrBook represents an address book from the pex package, which is used
  36. // to store peer addresses.
  37. type AddrBook interface {
  38. AddAddress(addr *NetAddress, src *NetAddress) error
  39. AddOurAddress(*NetAddress)
  40. OurAddress(*NetAddress) bool
  41. MarkGood(*NetAddress)
  42. RemoveAddress(*NetAddress)
  43. HasAddress(*NetAddress) bool
  44. Save()
  45. }
  46. // PeerFilterFunc to be implemented by filter hooks after a new Peer has been
  47. // fully setup.
  48. type PeerFilterFunc func(IPeerSet, Peer) error
  49. //-----------------------------------------------------------------------------
  50. // Switch handles peer connections and exposes an API to receive incoming messages
  51. // on `Reactors`. Each `Reactor` is responsible for handling incoming messages of one
  52. // or more `Channels`. So while sending outgoing messages is typically performed on the peer,
  53. // incoming messages are received on the reactor.
  54. type Switch struct {
  55. cmn.BaseService
  56. config *config.P2PConfig
  57. reactors map[string]Reactor
  58. chDescs []*conn.ChannelDescriptor
  59. reactorsByCh map[byte]Reactor
  60. peers *PeerSet
  61. dialing *cmn.CMap
  62. reconnecting *cmn.CMap
  63. nodeInfo NodeInfo // our node info
  64. nodeKey *NodeKey // our node privkey
  65. addrBook AddrBook
  66. transport Transport
  67. filterTimeout time.Duration
  68. peerFilters []PeerFilterFunc
  69. rng *cmn.Rand // seed for randomizing dial times and orders
  70. metrics *Metrics
  71. }
  72. // SwitchOption sets an optional parameter on the Switch.
  73. type SwitchOption func(*Switch)
  74. // NewSwitch creates a new Switch with the given config.
  75. func NewSwitch(
  76. cfg *config.P2PConfig,
  77. transport Transport,
  78. options ...SwitchOption,
  79. ) *Switch {
  80. sw := &Switch{
  81. config: cfg,
  82. reactors: make(map[string]Reactor),
  83. chDescs: make([]*conn.ChannelDescriptor, 0),
  84. reactorsByCh: make(map[byte]Reactor),
  85. peers: NewPeerSet(),
  86. dialing: cmn.NewCMap(),
  87. reconnecting: cmn.NewCMap(),
  88. metrics: NopMetrics(),
  89. transport: transport,
  90. filterTimeout: defaultFilterTimeout,
  91. }
  92. // Ensure we have a completely undeterministic PRNG.
  93. sw.rng = cmn.NewRand()
  94. sw.BaseService = *cmn.NewBaseService(nil, "P2P Switch", sw)
  95. for _, option := range options {
  96. option(sw)
  97. }
  98. return sw
  99. }
  100. // SwitchFilterTimeout sets the timeout used for peer filters.
  101. func SwitchFilterTimeout(timeout time.Duration) SwitchOption {
  102. return func(sw *Switch) { sw.filterTimeout = timeout }
  103. }
  104. // SwitchPeerFilters sets the filters for rejection of new peers.
  105. func SwitchPeerFilters(filters ...PeerFilterFunc) SwitchOption {
  106. return func(sw *Switch) { sw.peerFilters = filters }
  107. }
  108. // WithMetrics sets the metrics.
  109. func WithMetrics(metrics *Metrics) SwitchOption {
  110. return func(sw *Switch) { sw.metrics = metrics }
  111. }
  112. //---------------------------------------------------------------------
  113. // Switch setup
  114. // AddReactor adds the given reactor to the switch.
  115. // NOTE: Not goroutine safe.
  116. func (sw *Switch) AddReactor(name string, reactor Reactor) Reactor {
  117. // Validate the reactor.
  118. // No two reactors can share the same channel.
  119. reactorChannels := reactor.GetChannels()
  120. for _, chDesc := range reactorChannels {
  121. chID := chDesc.ID
  122. if sw.reactorsByCh[chID] != nil {
  123. cmn.PanicSanity(fmt.Sprintf("Channel %X has multiple reactors %v & %v", chID, sw.reactorsByCh[chID], reactor))
  124. }
  125. sw.chDescs = append(sw.chDescs, chDesc)
  126. sw.reactorsByCh[chID] = reactor
  127. }
  128. sw.reactors[name] = reactor
  129. reactor.SetSwitch(sw)
  130. return reactor
  131. }
  132. // Reactors returns a map of reactors registered on the switch.
  133. // NOTE: Not goroutine safe.
  134. func (sw *Switch) Reactors() map[string]Reactor {
  135. return sw.reactors
  136. }
  137. // Reactor returns the reactor with the given name.
  138. // NOTE: Not goroutine safe.
  139. func (sw *Switch) Reactor(name string) Reactor {
  140. return sw.reactors[name]
  141. }
  142. // SetNodeInfo sets the switch's NodeInfo for checking compatibility and handshaking with other nodes.
  143. // NOTE: Not goroutine safe.
  144. func (sw *Switch) SetNodeInfo(nodeInfo NodeInfo) {
  145. sw.nodeInfo = nodeInfo
  146. }
  147. // NodeInfo returns the switch's NodeInfo.
  148. // NOTE: Not goroutine safe.
  149. func (sw *Switch) NodeInfo() NodeInfo {
  150. return sw.nodeInfo
  151. }
  152. // SetNodeKey sets the switch's private key for authenticated encryption.
  153. // NOTE: Not goroutine safe.
  154. func (sw *Switch) SetNodeKey(nodeKey *NodeKey) {
  155. sw.nodeKey = nodeKey
  156. }
  157. //---------------------------------------------------------------------
  158. // Service start/stop
  159. // OnStart implements BaseService. It starts all the reactors and peers.
  160. func (sw *Switch) OnStart() error {
  161. // Start reactors
  162. for _, reactor := range sw.reactors {
  163. err := reactor.Start()
  164. if err != nil {
  165. return cmn.ErrorWrap(err, "failed to start %v", reactor)
  166. }
  167. }
  168. // Start accepting Peers.
  169. go sw.acceptRoutine()
  170. return nil
  171. }
  172. // OnStop implements BaseService. It stops all peers and reactors.
  173. func (sw *Switch) OnStop() {
  174. // Stop peers
  175. for _, p := range sw.peers.List() {
  176. p.Stop()
  177. if sw.peers.Remove(p) {
  178. sw.metrics.Peers.Add(float64(-1))
  179. }
  180. }
  181. // Stop reactors
  182. sw.Logger.Debug("Switch: Stopping reactors")
  183. for _, reactor := range sw.reactors {
  184. reactor.Stop()
  185. }
  186. }
  187. //---------------------------------------------------------------------
  188. // Peers
  189. // Broadcast runs a go routine for each attempted send, which will block trying
  190. // to send for defaultSendTimeoutSeconds. Returns a channel which receives
  191. // success values for each attempted send (false if times out). Channel will be
  192. // closed once msg bytes are sent to all peers (or time out).
  193. //
  194. // NOTE: Broadcast uses goroutines, so order of broadcast may not be preserved.
  195. func (sw *Switch) Broadcast(chID byte, msgBytes []byte) chan bool {
  196. successChan := make(chan bool, len(sw.peers.List()))
  197. sw.Logger.Debug("Broadcast", "channel", chID, "msgBytes", fmt.Sprintf("%X", msgBytes))
  198. var wg sync.WaitGroup
  199. for _, peer := range sw.peers.List() {
  200. wg.Add(1)
  201. go func(peer Peer) {
  202. defer wg.Done()
  203. success := peer.Send(chID, msgBytes)
  204. successChan <- success
  205. }(peer)
  206. }
  207. go func() {
  208. wg.Wait()
  209. close(successChan)
  210. }()
  211. return successChan
  212. }
  213. // NumPeers returns the count of outbound/inbound and outbound-dialing peers.
  214. func (sw *Switch) NumPeers() (outbound, inbound, dialing int) {
  215. peers := sw.peers.List()
  216. for _, peer := range peers {
  217. if peer.IsOutbound() {
  218. outbound++
  219. } else {
  220. inbound++
  221. }
  222. }
  223. dialing = sw.dialing.Size()
  224. return
  225. }
  226. // MaxNumOutboundPeers returns a maximum number of outbound peers.
  227. func (sw *Switch) MaxNumOutboundPeers() int {
  228. return sw.config.MaxNumOutboundPeers
  229. }
  230. // Peers returns the set of peers that are connected to the switch.
  231. func (sw *Switch) Peers() IPeerSet {
  232. return sw.peers
  233. }
  234. // StopPeerForError disconnects from a peer due to external error.
  235. // If the peer is persistent, it will attempt to reconnect.
  236. // TODO: make record depending on reason.
  237. func (sw *Switch) StopPeerForError(peer Peer, reason interface{}) {
  238. sw.Logger.Error("Stopping peer for error", "peer", peer, "err", reason)
  239. sw.stopAndRemovePeer(peer, reason)
  240. if peer.IsPersistent() {
  241. addr := peer.OriginalAddr()
  242. if addr == nil {
  243. // FIXME: persistent peers can't be inbound right now.
  244. // self-reported address for inbound persistent peers
  245. addr = peer.NodeInfo().NetAddress()
  246. }
  247. go sw.reconnectToPeer(addr)
  248. }
  249. }
  250. // StopPeerGracefully disconnects from a peer gracefully.
  251. // TODO: handle graceful disconnects.
  252. func (sw *Switch) StopPeerGracefully(peer Peer) {
  253. sw.Logger.Info("Stopping peer gracefully")
  254. sw.stopAndRemovePeer(peer, nil)
  255. }
  256. func (sw *Switch) stopAndRemovePeer(peer Peer, reason interface{}) {
  257. if sw.peers.Remove(peer) {
  258. sw.metrics.Peers.Add(float64(-1))
  259. }
  260. peer.Stop()
  261. for _, reactor := range sw.reactors {
  262. reactor.RemovePeer(peer, reason)
  263. }
  264. }
  265. // reconnectToPeer tries to reconnect to the addr, first repeatedly
  266. // with a fixed interval, then with exponential backoff.
  267. // If no success after all that, it stops trying, and leaves it
  268. // to the PEX/Addrbook to find the peer with the addr again
  269. // NOTE: this will keep trying even if the handshake or auth fails.
  270. // TODO: be more explicit with error types so we only retry on certain failures
  271. // - ie. if we're getting ErrDuplicatePeer we can stop
  272. // because the addrbook got us the peer back already
  273. func (sw *Switch) reconnectToPeer(addr *NetAddress) {
  274. if sw.reconnecting.Has(string(addr.ID)) {
  275. return
  276. }
  277. sw.reconnecting.Set(string(addr.ID), addr)
  278. defer sw.reconnecting.Delete(string(addr.ID))
  279. start := time.Now()
  280. sw.Logger.Info("Reconnecting to peer", "addr", addr)
  281. for i := 0; i < reconnectAttempts; i++ {
  282. if !sw.IsRunning() {
  283. return
  284. }
  285. if sw.IsDialingOrExistingAddress(addr) {
  286. sw.Logger.Debug("Peer connection has been established or dialed while we waiting next try", "addr", addr)
  287. return
  288. }
  289. err := sw.DialPeerWithAddress(addr, true)
  290. if err == nil {
  291. return // success
  292. }
  293. sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "addr", addr)
  294. // sleep a set amount
  295. sw.randomSleep(reconnectInterval)
  296. continue
  297. }
  298. sw.Logger.Error("Failed to reconnect to peer. Beginning exponential backoff",
  299. "addr", addr, "elapsed", time.Since(start))
  300. for i := 0; i < reconnectBackOffAttempts; i++ {
  301. if !sw.IsRunning() {
  302. return
  303. }
  304. // sleep an exponentially increasing amount
  305. sleepIntervalSeconds := math.Pow(reconnectBackOffBaseSeconds, float64(i))
  306. sw.randomSleep(time.Duration(sleepIntervalSeconds) * time.Second)
  307. err := sw.DialPeerWithAddress(addr, true)
  308. if err == nil {
  309. return // success
  310. }
  311. sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "addr", addr)
  312. }
  313. sw.Logger.Error("Failed to reconnect to peer. Giving up", "addr", addr, "elapsed", time.Since(start))
  314. }
  315. // SetAddrBook allows to set address book on Switch.
  316. func (sw *Switch) SetAddrBook(addrBook AddrBook) {
  317. sw.addrBook = addrBook
  318. }
  319. // MarkPeerAsGood marks the given peer as good when it did something useful
  320. // like contributed to consensus.
  321. func (sw *Switch) MarkPeerAsGood(peer Peer) {
  322. if sw.addrBook != nil {
  323. sw.addrBook.MarkGood(peer.NodeInfo().NetAddress())
  324. }
  325. }
  326. //---------------------------------------------------------------------
  327. // Dialing
  328. // DialPeersAsync dials a list of peers asynchronously in random order (optionally, making them persistent).
  329. // Used to dial peers from config on startup or from unsafe-RPC (trusted sources).
  330. // TODO: remove addrBook arg since it's now set on the switch
  331. func (sw *Switch) DialPeersAsync(addrBook AddrBook, peers []string, persistent bool) error {
  332. netAddrs, errs := NewNetAddressStrings(peers)
  333. // only log errors, dial correct addresses
  334. for _, err := range errs {
  335. sw.Logger.Error("Error in peer's address", "err", err)
  336. }
  337. ourAddr := sw.nodeInfo.NetAddress()
  338. // TODO: this code feels like it's in the wrong place.
  339. // The integration tests depend on the addrBook being saved
  340. // right away but maybe we can change that. Recall that
  341. // the addrBook is only written to disk every 2min
  342. if addrBook != nil {
  343. // add peers to `addrBook`
  344. for _, netAddr := range netAddrs {
  345. // do not add our address or ID
  346. if !netAddr.Same(ourAddr) {
  347. if err := addrBook.AddAddress(netAddr, ourAddr); err != nil {
  348. sw.Logger.Error("Can't add peer's address to addrbook", "err", err)
  349. }
  350. }
  351. }
  352. // Persist some peers to disk right away.
  353. // NOTE: integration tests depend on this
  354. addrBook.Save()
  355. }
  356. // permute the list, dial them in random order.
  357. perm := sw.rng.Perm(len(netAddrs))
  358. for i := 0; i < len(perm); i++ {
  359. go func(i int) {
  360. j := perm[i]
  361. addr := netAddrs[j]
  362. if addr.Same(ourAddr) {
  363. sw.Logger.Debug("Ignore attempt to connect to ourselves", "addr", addr, "ourAddr", ourAddr)
  364. return
  365. }
  366. sw.randomSleep(0)
  367. if sw.IsDialingOrExistingAddress(addr) {
  368. sw.Logger.Debug("Ignore attempt to connect to an existing peer", "addr", addr)
  369. return
  370. }
  371. err := sw.DialPeerWithAddress(addr, persistent)
  372. if err != nil {
  373. switch err.(type) {
  374. case ErrSwitchConnectToSelf, ErrSwitchDuplicatePeerID:
  375. sw.Logger.Debug("Error dialing peer", "err", err)
  376. default:
  377. sw.Logger.Error("Error dialing peer", "err", err)
  378. }
  379. }
  380. }(i)
  381. }
  382. return nil
  383. }
  384. // DialPeerWithAddress dials the given peer and runs sw.addPeer if it connects and authenticates successfully.
  385. // If `persistent == true`, the switch will always try to reconnect to this peer if the connection ever fails.
  386. func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) error {
  387. sw.dialing.Set(string(addr.ID), addr)
  388. defer sw.dialing.Delete(string(addr.ID))
  389. return sw.addOutboundPeerWithConfig(addr, sw.config, persistent)
  390. }
  391. // sleep for interval plus some random amount of ms on [0, dialRandomizerIntervalMilliseconds]
  392. func (sw *Switch) randomSleep(interval time.Duration) {
  393. r := time.Duration(sw.rng.Int63n(dialRandomizerIntervalMilliseconds)) * time.Millisecond
  394. time.Sleep(r + interval)
  395. }
  396. // IsDialingOrExistingAddress returns true if switch has a peer with the given
  397. // address or dialing it at the moment.
  398. func (sw *Switch) IsDialingOrExistingAddress(addr *NetAddress) bool {
  399. return sw.dialing.Has(string(addr.ID)) ||
  400. sw.peers.Has(addr.ID) ||
  401. (!sw.config.AllowDuplicateIP && sw.peers.HasIP(addr.IP))
  402. }
  403. func (sw *Switch) acceptRoutine() {
  404. for {
  405. p, err := sw.transport.Accept(peerConfig{
  406. chDescs: sw.chDescs,
  407. onPeerError: sw.StopPeerForError,
  408. reactorsByCh: sw.reactorsByCh,
  409. metrics: sw.metrics,
  410. })
  411. if err != nil {
  412. switch err.(type) {
  413. case ErrRejected:
  414. rErr := err.(ErrRejected)
  415. if rErr.IsSelf() {
  416. // Remove the given address from the address book and add to our addresses
  417. // to avoid dialing in the future.
  418. addr := rErr.Addr()
  419. sw.addrBook.RemoveAddress(&addr)
  420. sw.addrBook.AddOurAddress(&addr)
  421. }
  422. sw.Logger.Info(
  423. "Inbound Peer rejected",
  424. "err", err,
  425. "numPeers", sw.peers.Size(),
  426. )
  427. continue
  428. case *ErrTransportClosed:
  429. sw.Logger.Error(
  430. "Stopped accept routine, as transport is closed",
  431. "numPeers", sw.peers.Size(),
  432. )
  433. default:
  434. sw.Logger.Error(
  435. "Accept on transport errored",
  436. "err", err,
  437. "numPeers", sw.peers.Size(),
  438. )
  439. // We could instead have a retry loop around the acceptRoutine,
  440. // but that would need to stop and let the node shutdown eventually.
  441. // So might as well panic and let process managers restart the node.
  442. // There's no point in letting the node run without the acceptRoutine,
  443. // since it won't be able to accept new connections.
  444. panic(fmt.Errorf("accept routine exited: %v", err))
  445. }
  446. break
  447. }
  448. // Ignore connection if we already have enough peers.
  449. _, in, _ := sw.NumPeers()
  450. if in >= sw.config.MaxNumInboundPeers {
  451. sw.Logger.Info(
  452. "Ignoring inbound connection: already have enough inbound peers",
  453. "address", p.NodeInfo().NetAddress().String(),
  454. "have", in,
  455. "max", sw.config.MaxNumInboundPeers,
  456. )
  457. _ = p.Stop()
  458. continue
  459. }
  460. if err := sw.addPeer(p); err != nil {
  461. _ = p.Stop()
  462. sw.Logger.Info(
  463. "Ignoring inbound connection: error while adding peer",
  464. "err", err,
  465. "id", p.ID(),
  466. )
  467. }
  468. }
  469. }
  470. // dial the peer; make secret connection; authenticate against the dialed ID;
  471. // add the peer.
  472. // if dialing fails, start the reconnect loop. If handhsake fails, its over.
  473. // If peer is started succesffuly, reconnectLoop will start when
  474. // StopPeerForError is called
  475. func (sw *Switch) addOutboundPeerWithConfig(
  476. addr *NetAddress,
  477. cfg *config.P2PConfig,
  478. persistent bool,
  479. ) error {
  480. sw.Logger.Info("Dialing peer", "address", addr)
  481. // XXX(xla): Remove the leakage of test concerns in implementation.
  482. if cfg.TestDialFail {
  483. go sw.reconnectToPeer(addr)
  484. return fmt.Errorf("dial err (peerConfig.DialFail == true)")
  485. }
  486. p, err := sw.transport.Dial(*addr, peerConfig{
  487. chDescs: sw.chDescs,
  488. onPeerError: sw.StopPeerForError,
  489. persistent: persistent,
  490. reactorsByCh: sw.reactorsByCh,
  491. metrics: sw.metrics,
  492. })
  493. if err != nil {
  494. switch e := err.(type) {
  495. case ErrRejected:
  496. if e.IsSelf() {
  497. // Remove the given address from the address book and add to our addresses
  498. // to avoid dialing in the future.
  499. sw.addrBook.RemoveAddress(addr)
  500. sw.addrBook.AddOurAddress(addr)
  501. return err
  502. }
  503. }
  504. // retry persistent peers after
  505. // any dial error besides IsSelf()
  506. if persistent {
  507. go sw.reconnectToPeer(addr)
  508. }
  509. return err
  510. }
  511. if err := sw.addPeer(p); err != nil {
  512. _ = p.Stop()
  513. return err
  514. }
  515. return nil
  516. }
  517. func (sw *Switch) filterPeer(p Peer) error {
  518. // Avoid duplicate
  519. if sw.peers.Has(p.ID()) {
  520. return ErrRejected{id: p.ID(), isDuplicate: true}
  521. }
  522. errc := make(chan error, len(sw.peerFilters))
  523. for _, f := range sw.peerFilters {
  524. go func(f PeerFilterFunc, p Peer, errc chan<- error) {
  525. errc <- f(sw.peers, p)
  526. }(f, p, errc)
  527. }
  528. for i := 0; i < cap(errc); i++ {
  529. select {
  530. case err := <-errc:
  531. if err != nil {
  532. return ErrRejected{id: p.ID(), err: err, isFiltered: true}
  533. }
  534. case <-time.After(sw.filterTimeout):
  535. return ErrFilterTimeout{}
  536. }
  537. }
  538. return nil
  539. }
  540. // addPeer starts up the Peer and adds it to the Switch.
  541. func (sw *Switch) addPeer(p Peer) error {
  542. if err := sw.filterPeer(p); err != nil {
  543. return err
  544. }
  545. p.SetLogger(sw.Logger.With("peer", p.NodeInfo().NetAddress()))
  546. // All good. Start peer
  547. if sw.IsRunning() {
  548. if err := sw.startInitPeer(p); err != nil {
  549. return err
  550. }
  551. }
  552. // Add the peer to .peers.
  553. // We start it first so that a peer in the list is safe to Stop.
  554. // It should not err since we already checked peers.Has().
  555. if err := sw.peers.Add(p); err != nil {
  556. return err
  557. }
  558. sw.Logger.Info("Added peer", "peer", p)
  559. sw.metrics.Peers.Add(float64(1))
  560. return nil
  561. }
  562. func (sw *Switch) startInitPeer(p Peer) error {
  563. err := p.Start() // spawn send/recv routines
  564. if err != nil {
  565. // Should never happen
  566. sw.Logger.Error(
  567. "Error starting peer",
  568. "err", err,
  569. "peer", p,
  570. )
  571. return err
  572. }
  573. for _, reactor := range sw.reactors {
  574. reactor.AddPeer(p)
  575. }
  576. return nil
  577. }