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.

553 lines
16 KiB

9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
7 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
8 years ago
9 years ago
7 years ago
7 years ago
9 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
7 years ago
9 years ago
9 years ago
9 years ago
7 years ago
9 years ago
9 years ago
9 years ago
9 years ago
7 years ago
7 years ago
9 years ago
7 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
7 years ago
7 years ago
9 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
9 years ago
7 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
7 years ago
9 years ago
8 years ago
9 years ago
7 years ago
9 years ago
9 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
8 years ago
7 years ago
8 years ago
7 years ago
8 years ago
7 years ago
8 years ago
  1. package p2p
  2. import (
  3. "fmt"
  4. "math"
  5. "math/rand"
  6. "net"
  7. "sync"
  8. "time"
  9. "github.com/pkg/errors"
  10. crypto "github.com/tendermint/go-crypto"
  11. cfg "github.com/tendermint/tendermint/config"
  12. "github.com/tendermint/tendermint/p2p/conn"
  13. cmn "github.com/tendermint/tmlibs/common"
  14. )
  15. const (
  16. // wait a random amount of time from this interval
  17. // before dialing peers or reconnecting to help prevent DoS
  18. dialRandomizerIntervalMilliseconds = 3000
  19. // repeatedly try to reconnect for a few minutes
  20. // ie. 5 * 20 = 100s
  21. reconnectAttempts = 20
  22. reconnectInterval = 5 * time.Second
  23. // then move into exponential backoff mode for ~1day
  24. // ie. 3**10 = 16hrs
  25. reconnectBackOffAttempts = 10
  26. reconnectBackOffBaseSeconds = 3
  27. )
  28. //-----------------------------------------------------------------------------
  29. type AddrBook interface {
  30. AddAddress(addr *NetAddress, src *NetAddress) error
  31. Save()
  32. }
  33. //-----------------------------------------------------------------------------
  34. // `Switch` handles peer connections and exposes an API to receive incoming messages
  35. // on `Reactors`. Each `Reactor` is responsible for handling incoming messages of one
  36. // or more `Channels`. So while sending outgoing messages is typically performed on the peer,
  37. // incoming messages are received on the reactor.
  38. type Switch struct {
  39. cmn.BaseService
  40. config *cfg.P2PConfig
  41. peerConfig *PeerConfig
  42. listeners []Listener
  43. reactors map[string]Reactor
  44. chDescs []*conn.ChannelDescriptor
  45. reactorsByCh map[byte]Reactor
  46. peers *PeerSet
  47. dialing *cmn.CMap
  48. nodeInfo NodeInfo // our node info
  49. nodeKey *NodeKey // our node privkey
  50. filterConnByAddr func(net.Addr) error
  51. filterConnByPubKey func(crypto.PubKey) error
  52. rng *rand.Rand // seed for randomizing dial times and orders
  53. }
  54. func NewSwitch(config *cfg.P2PConfig) *Switch {
  55. sw := &Switch{
  56. config: config,
  57. peerConfig: DefaultPeerConfig(),
  58. reactors: make(map[string]Reactor),
  59. chDescs: make([]*conn.ChannelDescriptor, 0),
  60. reactorsByCh: make(map[byte]Reactor),
  61. peers: NewPeerSet(),
  62. dialing: cmn.NewCMap(),
  63. }
  64. // Ensure we have a completely undeterministic PRNG. cmd.RandInt64() draws
  65. // from a seed that's initialized with OS entropy on process start.
  66. sw.rng = rand.New(rand.NewSource(cmn.RandInt64()))
  67. // TODO: collapse the peerConfig into the config ?
  68. sw.peerConfig.MConfig.FlushThrottle = time.Duration(config.FlushThrottleTimeout) * time.Millisecond
  69. sw.peerConfig.MConfig.SendRate = config.SendRate
  70. sw.peerConfig.MConfig.RecvRate = config.RecvRate
  71. sw.peerConfig.MConfig.MaxMsgPacketPayloadSize = config.MaxMsgPacketPayloadSize
  72. sw.BaseService = *cmn.NewBaseService(nil, "P2P Switch", sw)
  73. return sw
  74. }
  75. //---------------------------------------------------------------------
  76. // Switch setup
  77. // AddReactor adds the given reactor to the switch.
  78. // NOTE: Not goroutine safe.
  79. func (sw *Switch) AddReactor(name string, reactor Reactor) Reactor {
  80. // Validate the reactor.
  81. // No two reactors can share the same channel.
  82. reactorChannels := reactor.GetChannels()
  83. for _, chDesc := range reactorChannels {
  84. chID := chDesc.ID
  85. if sw.reactorsByCh[chID] != nil {
  86. cmn.PanicSanity(fmt.Sprintf("Channel %X has multiple reactors %v & %v", chID, sw.reactorsByCh[chID], reactor))
  87. }
  88. sw.chDescs = append(sw.chDescs, chDesc)
  89. sw.reactorsByCh[chID] = reactor
  90. }
  91. sw.reactors[name] = reactor
  92. reactor.SetSwitch(sw)
  93. return reactor
  94. }
  95. // Reactors returns a map of reactors registered on the switch.
  96. // NOTE: Not goroutine safe.
  97. func (sw *Switch) Reactors() map[string]Reactor {
  98. return sw.reactors
  99. }
  100. // Reactor returns the reactor with the given name.
  101. // NOTE: Not goroutine safe.
  102. func (sw *Switch) Reactor(name string) Reactor {
  103. return sw.reactors[name]
  104. }
  105. // AddListener adds the given listener to the switch for listening to incoming peer connections.
  106. // NOTE: Not goroutine safe.
  107. func (sw *Switch) AddListener(l Listener) {
  108. sw.listeners = append(sw.listeners, l)
  109. }
  110. // Listeners returns the list of listeners the switch listens on.
  111. // NOTE: Not goroutine safe.
  112. func (sw *Switch) Listeners() []Listener {
  113. return sw.listeners
  114. }
  115. // IsListening returns true if the switch has at least one listener.
  116. // NOTE: Not goroutine safe.
  117. func (sw *Switch) IsListening() bool {
  118. return len(sw.listeners) > 0
  119. }
  120. // SetNodeInfo sets the switch's NodeInfo for checking compatibility and handshaking with other nodes.
  121. // NOTE: Not goroutine safe.
  122. func (sw *Switch) SetNodeInfo(nodeInfo NodeInfo) {
  123. sw.nodeInfo = nodeInfo
  124. }
  125. // NodeInfo returns the switch's NodeInfo.
  126. // NOTE: Not goroutine safe.
  127. func (sw *Switch) NodeInfo() NodeInfo {
  128. return sw.nodeInfo
  129. }
  130. // SetNodeKey sets the switch's private key for authenticated encryption.
  131. // NOTE: Not goroutine safe.
  132. func (sw *Switch) SetNodeKey(nodeKey *NodeKey) {
  133. sw.nodeKey = nodeKey
  134. }
  135. //---------------------------------------------------------------------
  136. // Service start/stop
  137. // OnStart implements BaseService. It starts all the reactors, peers, and listeners.
  138. func (sw *Switch) OnStart() error {
  139. // Start reactors
  140. for _, reactor := range sw.reactors {
  141. err := reactor.Start()
  142. if err != nil {
  143. return errors.Wrapf(err, "failed to start %v", reactor)
  144. }
  145. }
  146. // Start listeners
  147. for _, listener := range sw.listeners {
  148. go sw.listenerRoutine(listener)
  149. }
  150. return nil
  151. }
  152. // OnStop implements BaseService. It stops all listeners, peers, and reactors.
  153. func (sw *Switch) OnStop() {
  154. // Stop listeners
  155. for _, listener := range sw.listeners {
  156. listener.Stop()
  157. }
  158. sw.listeners = nil
  159. // Stop peers
  160. for _, peer := range sw.peers.List() {
  161. peer.Stop()
  162. sw.peers.Remove(peer)
  163. }
  164. // Stop reactors
  165. sw.Logger.Debug("Switch: Stopping reactors")
  166. for _, reactor := range sw.reactors {
  167. reactor.Stop()
  168. }
  169. }
  170. //---------------------------------------------------------------------
  171. // Peers
  172. // Broadcast runs a go routine for each attempted send, which will block trying
  173. // to send for defaultSendTimeoutSeconds. Returns a channel which receives
  174. // success values for each attempted send (false if times out). Channel will be
  175. // closed once msg send to all peers.
  176. //
  177. // NOTE: Broadcast uses goroutines, so order of broadcast may not be preserved.
  178. func (sw *Switch) Broadcast(chID byte, msg interface{}) chan bool {
  179. successChan := make(chan bool, len(sw.peers.List()))
  180. sw.Logger.Debug("Broadcast", "channel", chID, "msg", msg)
  181. var wg sync.WaitGroup
  182. for _, peer := range sw.peers.List() {
  183. wg.Add(1)
  184. go func(peer Peer) {
  185. defer wg.Done()
  186. success := peer.Send(chID, msg)
  187. successChan <- success
  188. }(peer)
  189. }
  190. go func() {
  191. wg.Wait()
  192. close(successChan)
  193. }()
  194. return successChan
  195. }
  196. // NumPeers returns the count of outbound/inbound and outbound-dialing peers.
  197. func (sw *Switch) NumPeers() (outbound, inbound, dialing int) {
  198. peers := sw.peers.List()
  199. for _, peer := range peers {
  200. if peer.IsOutbound() {
  201. outbound++
  202. } else {
  203. inbound++
  204. }
  205. }
  206. dialing = sw.dialing.Size()
  207. return
  208. }
  209. // Peers returns the set of peers that are connected to the switch.
  210. func (sw *Switch) Peers() IPeerSet {
  211. return sw.peers
  212. }
  213. // StopPeerForError disconnects from a peer due to external error.
  214. // If the peer is persistent, it will attempt to reconnect.
  215. // TODO: make record depending on reason.
  216. func (sw *Switch) StopPeerForError(peer Peer, reason interface{}) {
  217. sw.Logger.Error("Stopping peer for error", "peer", peer, "err", reason)
  218. sw.stopAndRemovePeer(peer, reason)
  219. if peer.IsPersistent() {
  220. go sw.reconnectToPeer(peer)
  221. }
  222. }
  223. // StopPeerGracefully disconnects from a peer gracefully.
  224. // TODO: handle graceful disconnects.
  225. func (sw *Switch) StopPeerGracefully(peer Peer) {
  226. sw.Logger.Info("Stopping peer gracefully")
  227. sw.stopAndRemovePeer(peer, nil)
  228. }
  229. func (sw *Switch) stopAndRemovePeer(peer Peer, reason interface{}) {
  230. sw.peers.Remove(peer)
  231. peer.Stop()
  232. for _, reactor := range sw.reactors {
  233. reactor.RemovePeer(peer, reason)
  234. }
  235. }
  236. // reconnectToPeer tries to reconnect to the peer, first repeatedly
  237. // with a fixed interval, then with exponential backoff.
  238. // If no success after all that, it stops trying, and leaves it
  239. // to the PEX/Addrbook to find the peer again
  240. func (sw *Switch) reconnectToPeer(peer Peer) {
  241. // NOTE this will connect to the self reported address,
  242. // not necessarily the original we dialed
  243. netAddr := peer.NodeInfo().NetAddress()
  244. start := time.Now()
  245. sw.Logger.Info("Reconnecting to peer", "peer", peer)
  246. for i := 0; i < reconnectAttempts; i++ {
  247. if !sw.IsRunning() {
  248. return
  249. }
  250. peer, err := sw.DialPeerWithAddress(netAddr, true)
  251. if err != nil {
  252. sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "peer", peer)
  253. // sleep a set amount
  254. sw.randomSleep(reconnectInterval)
  255. continue
  256. } else {
  257. sw.Logger.Info("Reconnected to peer", "peer", peer)
  258. return
  259. }
  260. }
  261. sw.Logger.Error("Failed to reconnect to peer. Beginning exponential backoff",
  262. "peer", peer, "elapsed", time.Since(start))
  263. for i := 0; i < reconnectBackOffAttempts; i++ {
  264. if !sw.IsRunning() {
  265. return
  266. }
  267. // sleep an exponentially increasing amount
  268. sleepIntervalSeconds := math.Pow(reconnectBackOffBaseSeconds, float64(i))
  269. sw.randomSleep(time.Duration(sleepIntervalSeconds) * time.Second)
  270. peer, err := sw.DialPeerWithAddress(netAddr, true)
  271. if err != nil {
  272. sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "peer", peer)
  273. continue
  274. } else {
  275. sw.Logger.Info("Reconnected to peer", "peer", peer)
  276. return
  277. }
  278. }
  279. sw.Logger.Error("Failed to reconnect to peer. Giving up", "peer", peer, "elapsed", time.Since(start))
  280. }
  281. //---------------------------------------------------------------------
  282. // Dialing
  283. // IsDialing returns true if the switch is currently dialing the given ID.
  284. func (sw *Switch) IsDialing(id ID) bool {
  285. return sw.dialing.Has(string(id))
  286. }
  287. // DialPeersAsync dials a list of peers asynchronously in random order (optionally, making them persistent).
  288. func (sw *Switch) DialPeersAsync(addrBook AddrBook, peers []string, persistent bool) error {
  289. netAddrs, errs := NewNetAddressStrings(peers)
  290. for _, err := range errs {
  291. sw.Logger.Error("Error in peer's address", "err", err)
  292. }
  293. if addrBook != nil {
  294. // add peers to `addrBook`
  295. ourAddr := sw.nodeInfo.NetAddress()
  296. for _, netAddr := range netAddrs {
  297. // do not add our address or ID
  298. if netAddr.Same(ourAddr) {
  299. continue
  300. }
  301. // TODO: move this out of here ?
  302. addrBook.AddAddress(netAddr, ourAddr)
  303. }
  304. // Persist some peers to disk right away.
  305. // NOTE: integration tests depend on this
  306. addrBook.Save()
  307. }
  308. // permute the list, dial them in random order.
  309. perm := sw.rng.Perm(len(netAddrs))
  310. for i := 0; i < len(perm); i++ {
  311. go func(i int) {
  312. sw.randomSleep(0)
  313. j := perm[i]
  314. peer, err := sw.DialPeerWithAddress(netAddrs[j], persistent)
  315. if err != nil {
  316. sw.Logger.Error("Error dialing peer", "err", err)
  317. } else {
  318. sw.Logger.Info("Connected to peer", "peer", peer)
  319. }
  320. }(i)
  321. }
  322. return nil
  323. }
  324. // DialPeerWithAddress dials the given peer and runs sw.addPeer if it connects and authenticates successfully.
  325. // If `persistent == true`, the switch will always try to reconnect to this peer if the connection ever fails.
  326. func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) (Peer, error) {
  327. sw.dialing.Set(string(addr.ID), addr)
  328. defer sw.dialing.Delete(string(addr.ID))
  329. return sw.addOutboundPeerWithConfig(addr, sw.peerConfig, persistent)
  330. }
  331. // sleep for interval plus some random amount of ms on [0, dialRandomizerIntervalMilliseconds]
  332. func (sw *Switch) randomSleep(interval time.Duration) {
  333. r := time.Duration(sw.rng.Int63n(dialRandomizerIntervalMilliseconds)) * time.Millisecond
  334. time.Sleep(r + interval)
  335. }
  336. //------------------------------------------------------------------------------------
  337. // Connection filtering
  338. // FilterConnByAddr returns an error if connecting to the given address is forbidden.
  339. func (sw *Switch) FilterConnByAddr(addr net.Addr) error {
  340. if sw.filterConnByAddr != nil {
  341. return sw.filterConnByAddr(addr)
  342. }
  343. return nil
  344. }
  345. // FilterConnByPubKey returns an error if connecting to the given public key is forbidden.
  346. func (sw *Switch) FilterConnByPubKey(pubkey crypto.PubKey) error {
  347. if sw.filterConnByPubKey != nil {
  348. return sw.filterConnByPubKey(pubkey)
  349. }
  350. return nil
  351. }
  352. // SetAddrFilter sets the function for filtering connections by address.
  353. func (sw *Switch) SetAddrFilter(f func(net.Addr) error) {
  354. sw.filterConnByAddr = f
  355. }
  356. // SetPubKeyFilter sets the function for filtering connections by public key.
  357. func (sw *Switch) SetPubKeyFilter(f func(crypto.PubKey) error) {
  358. sw.filterConnByPubKey = f
  359. }
  360. //------------------------------------------------------------------------------------
  361. func (sw *Switch) listenerRoutine(l Listener) {
  362. for {
  363. inConn, ok := <-l.Connections()
  364. if !ok {
  365. break
  366. }
  367. // ignore connection if we already have enough
  368. maxPeers := sw.config.MaxNumPeers
  369. if maxPeers <= sw.peers.Size() {
  370. sw.Logger.Info("Ignoring inbound connection: already have enough peers", "address", inConn.RemoteAddr().String(), "numPeers", sw.peers.Size(), "max", maxPeers)
  371. continue
  372. }
  373. // New inbound connection!
  374. err := sw.addInboundPeerWithConfig(inConn, sw.peerConfig)
  375. if err != nil {
  376. sw.Logger.Info("Ignoring inbound connection: error while adding peer", "address", inConn.RemoteAddr().String(), "err", err)
  377. continue
  378. }
  379. }
  380. // cleanup
  381. }
  382. func (sw *Switch) addInboundPeerWithConfig(conn net.Conn, config *PeerConfig) error {
  383. peer, err := newInboundPeer(conn, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, config)
  384. if err != nil {
  385. conn.Close() // peer is nil
  386. return err
  387. }
  388. peer.SetLogger(sw.Logger.With("peer", conn.RemoteAddr()))
  389. if err = sw.addPeer(peer); err != nil {
  390. peer.CloseConn()
  391. return err
  392. }
  393. return nil
  394. }
  395. // dial the peer; make secret connection; authenticate against the dialed ID;
  396. // add the peer.
  397. func (sw *Switch) addOutboundPeerWithConfig(addr *NetAddress, config *PeerConfig, persistent bool) (Peer, error) {
  398. sw.Logger.Info("Dialing peer", "address", addr)
  399. peer, err := newOutboundPeer(addr, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, config, persistent)
  400. if err != nil {
  401. sw.Logger.Error("Failed to dial peer", "address", addr, "err", err)
  402. return nil, err
  403. }
  404. peer.SetLogger(sw.Logger.With("peer", addr))
  405. // authenticate peer
  406. if addr.ID == "" {
  407. peer.Logger.Info("Dialed peer with unknown ID - unable to authenticate", "addr", addr)
  408. } else if addr.ID != peer.ID() {
  409. peer.CloseConn()
  410. return nil, ErrSwitchAuthenticationFailure{addr, peer.ID()}
  411. }
  412. err = sw.addPeer(peer)
  413. if err != nil {
  414. sw.Logger.Error("Failed to add peer", "address", addr, "err", err)
  415. peer.CloseConn()
  416. return nil, err
  417. }
  418. sw.Logger.Info("Dialed and added peer", "address", addr, "peer", peer)
  419. return peer, nil
  420. }
  421. // addPeer performs the Tendermint P2P handshake with a peer
  422. // that already has a SecretConnection. If all goes well,
  423. // it starts the peer and adds it to the switch.
  424. // NOTE: This performs a blocking handshake before the peer is added.
  425. // NOTE: If error is returned, caller is responsible for calling peer.CloseConn()
  426. func (sw *Switch) addPeer(peer *peer) error {
  427. // Avoid self
  428. if sw.nodeKey.ID() == peer.ID() {
  429. return ErrSwitchConnectToSelf
  430. }
  431. // Avoid duplicate
  432. if sw.peers.Has(peer.ID()) {
  433. return ErrSwitchDuplicatePeer
  434. }
  435. // Filter peer against white list
  436. if err := sw.FilterConnByAddr(peer.Addr()); err != nil {
  437. return err
  438. }
  439. if err := sw.FilterConnByPubKey(peer.PubKey()); err != nil {
  440. return err
  441. }
  442. // Exchange NodeInfo with the peer
  443. if err := peer.HandshakeTimeout(sw.nodeInfo, time.Duration(sw.peerConfig.HandshakeTimeout*time.Second)); err != nil {
  444. return err
  445. }
  446. // Validate the peers nodeInfo against the pubkey
  447. if err := peer.NodeInfo().Validate(peer.PubKey()); err != nil {
  448. return err
  449. }
  450. // Check version, chain id
  451. if err := sw.nodeInfo.CompatibleWith(peer.NodeInfo()); err != nil {
  452. return err
  453. }
  454. // All good. Start peer
  455. if sw.IsRunning() {
  456. sw.startInitPeer(peer)
  457. }
  458. // Add the peer to .peers.
  459. // We start it first so that a peer in the list is safe to Stop.
  460. // It should not err since we already checked peers.Has().
  461. if err := sw.peers.Add(peer); err != nil {
  462. return err
  463. }
  464. sw.Logger.Info("Added peer", "peer", peer)
  465. return nil
  466. }
  467. func (sw *Switch) startInitPeer(peer *peer) {
  468. err := peer.Start() // spawn send/recv routines
  469. if err != nil {
  470. // Should never happen
  471. sw.Logger.Error("Error starting peer", "peer", peer, "err", err)
  472. }
  473. for _, reactor := range sw.reactors {
  474. reactor.AddPeer(peer)
  475. }
  476. }