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.

619 lines
17 KiB

9 years ago
9 years ago
8 years ago
7 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
7 years ago
9 years ago
9 years ago
8 years ago
7 years ago
9 years ago
7 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
8 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
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
7 years ago
p2p: introduce peerConn to simplify peer creation (#1226) * expose AuthEnc in the P2P config if AuthEnc is true, dialed peers must have a node ID in the address and it must match the persistent pubkey from the secret handshake. Refs #1157 * fixes after my own review * fix docs * fix build failure ``` p2p/pex/pex_reactor_test.go:288:88: cannot use seed.NodeInfo().NetAddress() (type *p2p.NetAddress) as type string in array or slice literal ``` * p2p: introduce peerConn to simplify peer creation * Introduce `peerConn` containing the known fields of `peer` * `peer` only created in `sw.addPeer` once handshake is complete and NodeInfo is checked * Eliminates some mutable variables and makes the code flow better * Simplifies the `newXxxPeer` funcs * Use ID instead of PubKey where possible. * SetPubKeyFilter -> SetIDFilter * nodeInfo.Validate takes ID * remove peer.PubKey() * persistent node ids * fixes from review * test: use ip_plus_id.sh more * fix invalid memory panic during fast_sync test ``` 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: panic: runtime error: invalid memory address or nil pointer dereference 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: [signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0x98dd3e] 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: goroutine 3432 [running]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.newOutboundPeerConn(0xc423fd1380, 0xc420933e00, 0x1, 0x1239a60, 0 xc420128c40, 0x2, 0x42caf6, 0xc42001f300, 0xc422831d98, 0xc4227951c0, ...) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/peer.go:123 +0x31e 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).addOutboundPeerWithConfig(0xc4200ad040, 0xc423fd1380, 0 xc420933e00, 0xc423f48801, 0x28, 0x2) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:455 +0x12b 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).DialPeerWithAddress(0xc4200ad040, 0xc423fd1380, 0x1, 0x 0, 0x0) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:371 +0xdc 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).reconnectToPeer(0xc4200ad040, 0x123e000, 0xc42007bb00) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:290 +0x25f 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: created by github.com/tendermint/tendermint/p2p.(*Switch).StopPeerForError 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:256 +0x1b7 ```
7 years ago
p2p: introduce peerConn to simplify peer creation (#1226) * expose AuthEnc in the P2P config if AuthEnc is true, dialed peers must have a node ID in the address and it must match the persistent pubkey from the secret handshake. Refs #1157 * fixes after my own review * fix docs * fix build failure ``` p2p/pex/pex_reactor_test.go:288:88: cannot use seed.NodeInfo().NetAddress() (type *p2p.NetAddress) as type string in array or slice literal ``` * p2p: introduce peerConn to simplify peer creation * Introduce `peerConn` containing the known fields of `peer` * `peer` only created in `sw.addPeer` once handshake is complete and NodeInfo is checked * Eliminates some mutable variables and makes the code flow better * Simplifies the `newXxxPeer` funcs * Use ID instead of PubKey where possible. * SetPubKeyFilter -> SetIDFilter * nodeInfo.Validate takes ID * remove peer.PubKey() * persistent node ids * fixes from review * test: use ip_plus_id.sh more * fix invalid memory panic during fast_sync test ``` 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: panic: runtime error: invalid memory address or nil pointer dereference 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: [signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0x98dd3e] 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: goroutine 3432 [running]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.newOutboundPeerConn(0xc423fd1380, 0xc420933e00, 0x1, 0x1239a60, 0 xc420128c40, 0x2, 0x42caf6, 0xc42001f300, 0xc422831d98, 0xc4227951c0, ...) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/peer.go:123 +0x31e 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).addOutboundPeerWithConfig(0xc4200ad040, 0xc423fd1380, 0 xc420933e00, 0xc423f48801, 0x28, 0x2) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:455 +0x12b 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).DialPeerWithAddress(0xc4200ad040, 0xc423fd1380, 0x1, 0x 0, 0x0) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:371 +0xdc 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).reconnectToPeer(0xc4200ad040, 0x123e000, 0xc42007bb00) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:290 +0x25f 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: created by github.com/tendermint/tendermint/p2p.(*Switch).StopPeerForError 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:256 +0x1b7 ```
7 years ago
p2p: introduce peerConn to simplify peer creation (#1226) * expose AuthEnc in the P2P config if AuthEnc is true, dialed peers must have a node ID in the address and it must match the persistent pubkey from the secret handshake. Refs #1157 * fixes after my own review * fix docs * fix build failure ``` p2p/pex/pex_reactor_test.go:288:88: cannot use seed.NodeInfo().NetAddress() (type *p2p.NetAddress) as type string in array or slice literal ``` * p2p: introduce peerConn to simplify peer creation * Introduce `peerConn` containing the known fields of `peer` * `peer` only created in `sw.addPeer` once handshake is complete and NodeInfo is checked * Eliminates some mutable variables and makes the code flow better * Simplifies the `newXxxPeer` funcs * Use ID instead of PubKey where possible. * SetPubKeyFilter -> SetIDFilter * nodeInfo.Validate takes ID * remove peer.PubKey() * persistent node ids * fixes from review * test: use ip_plus_id.sh more * fix invalid memory panic during fast_sync test ``` 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: panic: runtime error: invalid memory address or nil pointer dereference 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: [signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0x98dd3e] 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: goroutine 3432 [running]: 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.newOutboundPeerConn(0xc423fd1380, 0xc420933e00, 0x1, 0x1239a60, 0 xc420128c40, 0x2, 0x42caf6, 0xc42001f300, 0xc422831d98, 0xc4227951c0, ...) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/peer.go:123 +0x31e 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).addOutboundPeerWithConfig(0xc4200ad040, 0xc423fd1380, 0 xc420933e00, 0xc423f48801, 0x28, 0x2) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:455 +0x12b 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).DialPeerWithAddress(0xc4200ad040, 0xc423fd1380, 0x1, 0x 0, 0x0) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:371 +0xdc 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: github.com/tendermint/tendermint/p2p.(*Switch).reconnectToPeer(0xc4200ad040, 0x123e000, 0xc42007bb00) 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:290 +0x25f 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: created by github.com/tendermint/tendermint/p2p.(*Switch).StopPeerForError 2018-02-21T06:30:05Z box887.localdomain docker/local_testnet_4[14907]: #011/go/src/github.com/tendermint/tendermint/p2p/switch.go:256 +0x1b7 ```
7 years ago
9 years ago
9 years ago
9 years ago
9 years ago
9 years ago
  1. package pex
  2. import (
  3. "fmt"
  4. "math/rand"
  5. "reflect"
  6. "sort"
  7. "sync"
  8. "time"
  9. "github.com/tendermint/go-amino"
  10. cmn "github.com/tendermint/tmlibs/common"
  11. "github.com/tendermint/tendermint/p2p"
  12. "github.com/tendermint/tendermint/p2p/conn"
  13. )
  14. type Peer = p2p.Peer
  15. const (
  16. // PexChannel is a channel for PEX messages
  17. PexChannel = byte(0x00)
  18. maxPexMessageSize = 1048576 // 1MB
  19. // ensure we have enough peers
  20. defaultEnsurePeersPeriod = 30 * time.Second
  21. defaultMinNumOutboundPeers = 10
  22. // Seed/Crawler constants
  23. // TODO:
  24. // We want seeds to only advertise good peers.
  25. // Peers are marked by external mechanisms.
  26. // We need a config value that can be set to be
  27. // on the order of how long it would take before a good
  28. // peer is marked good.
  29. defaultSeedDisconnectWaitPeriod = 2 * time.Minute // disconnect after this
  30. defaultCrawlPeerInterval = 2 * time.Minute // dont redial for this. TODO: back-off
  31. defaultCrawlPeersPeriod = 30 * time.Second // check some peers every this
  32. maxAttemptsToDial = 16 // ~ 35h in total (last attempt - 18h)
  33. )
  34. // PEXReactor handles PEX (peer exchange) and ensures that an
  35. // adequate number of peers are connected to the switch.
  36. //
  37. // It uses `AddrBook` (address book) to store `NetAddress`es of the peers.
  38. //
  39. // ## Preventing abuse
  40. //
  41. // Only accept pexAddrsMsg from peers we sent a corresponding pexRequestMsg too.
  42. // Only accept one pexRequestMsg every ~defaultEnsurePeersPeriod.
  43. type PEXReactor struct {
  44. p2p.BaseReactor
  45. book AddrBook
  46. config *PEXReactorConfig
  47. ensurePeersPeriod time.Duration
  48. // maps to prevent abuse
  49. requestsSent *cmn.CMap // ID->struct{}: unanswered send requests
  50. lastReceivedRequests *cmn.CMap // ID->time.Time: last time peer requested from us
  51. attemptsToDial sync.Map // address (string) -> {number of attempts (int), last time dialed (time.Time)}
  52. }
  53. // PEXReactorConfig holds reactor specific configuration data.
  54. type PEXReactorConfig struct {
  55. // Seed/Crawler mode
  56. SeedMode bool
  57. // Seeds is a list of addresses reactor may use
  58. // if it can't connect to peers in the addrbook.
  59. Seeds []string
  60. }
  61. type _attemptsToDial struct {
  62. number int
  63. lastDialed time.Time
  64. }
  65. // NewPEXReactor creates new PEX reactor.
  66. func NewPEXReactor(b AddrBook, config *PEXReactorConfig) *PEXReactor {
  67. r := &PEXReactor{
  68. book: b,
  69. config: config,
  70. ensurePeersPeriod: defaultEnsurePeersPeriod,
  71. requestsSent: cmn.NewCMap(),
  72. lastReceivedRequests: cmn.NewCMap(),
  73. }
  74. r.BaseReactor = *p2p.NewBaseReactor("PEXReactor", r)
  75. return r
  76. }
  77. // OnStart implements BaseService
  78. func (r *PEXReactor) OnStart() error {
  79. if err := r.BaseReactor.OnStart(); err != nil {
  80. return err
  81. }
  82. err := r.book.Start()
  83. if err != nil && err != cmn.ErrAlreadyStarted {
  84. return err
  85. }
  86. // return err if user provided a bad seed address
  87. if err := r.checkSeeds(); err != nil {
  88. return err
  89. }
  90. // Check if this node should run
  91. // in seed/crawler mode
  92. if r.config.SeedMode {
  93. go r.crawlPeersRoutine()
  94. } else {
  95. go r.ensurePeersRoutine()
  96. }
  97. return nil
  98. }
  99. // OnStop implements BaseService
  100. func (r *PEXReactor) OnStop() {
  101. r.BaseReactor.OnStop()
  102. r.book.Stop()
  103. }
  104. // GetChannels implements Reactor
  105. func (r *PEXReactor) GetChannels() []*conn.ChannelDescriptor {
  106. return []*conn.ChannelDescriptor{
  107. {
  108. ID: PexChannel,
  109. Priority: 1,
  110. SendQueueCapacity: 10,
  111. },
  112. }
  113. }
  114. // AddPeer implements Reactor by adding peer to the address book (if inbound)
  115. // or by requesting more addresses (if outbound).
  116. func (r *PEXReactor) AddPeer(p Peer) {
  117. if p.IsOutbound() {
  118. // For outbound peers, the address is already in the books -
  119. // either via DialPeersAsync or r.Receive.
  120. // Ask it for more peers if we need.
  121. if r.book.NeedMoreAddrs() {
  122. r.RequestAddrs(p)
  123. }
  124. } else {
  125. // For inbound peers, the peer is its own source,
  126. // and its NodeInfo has already been validated.
  127. // Let the ensurePeersRoutine handle asking for more
  128. // peers when we need - we don't trust inbound peers as much.
  129. addr := p.NodeInfo().NetAddress()
  130. r.book.AddAddress(addr, addr)
  131. }
  132. }
  133. // RemovePeer implements Reactor.
  134. func (r *PEXReactor) RemovePeer(p Peer, reason interface{}) {
  135. id := string(p.ID())
  136. r.requestsSent.Delete(id)
  137. r.lastReceivedRequests.Delete(id)
  138. }
  139. // Receive implements Reactor by handling incoming PEX messages.
  140. func (r *PEXReactor) Receive(chID byte, src Peer, msgBytes []byte) {
  141. msg, err := DecodeMessage(msgBytes)
  142. if err != nil {
  143. r.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
  144. r.Switch.StopPeerForError(src, err)
  145. return
  146. }
  147. r.Logger.Debug("Received message", "src", src, "chId", chID, "msg", msg)
  148. switch msg := msg.(type) {
  149. case *pexRequestMessage:
  150. // Check we're not receiving too many requests
  151. if err := r.receiveRequest(src); err != nil {
  152. r.Switch.StopPeerForError(src, err)
  153. return
  154. }
  155. // Seeds disconnect after sending a batch of addrs
  156. if r.config.SeedMode {
  157. // TODO: should we be more selective ?
  158. r.SendAddrs(src, r.book.GetSelection())
  159. r.Switch.StopPeerGracefully(src)
  160. } else {
  161. r.SendAddrs(src, r.book.GetSelection())
  162. }
  163. case *pexAddrsMessage:
  164. // If we asked for addresses, add them to the book
  165. if err := r.ReceiveAddrs(msg.Addrs, src); err != nil {
  166. r.Switch.StopPeerForError(src, err)
  167. return
  168. }
  169. default:
  170. r.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
  171. }
  172. }
  173. func (r *PEXReactor) receiveRequest(src Peer) error {
  174. id := string(src.ID())
  175. v := r.lastReceivedRequests.Get(id)
  176. if v == nil {
  177. // initialize with empty time
  178. lastReceived := time.Time{}
  179. r.lastReceivedRequests.Set(id, lastReceived)
  180. return nil
  181. }
  182. lastReceived := v.(time.Time)
  183. if lastReceived.Equal(time.Time{}) {
  184. // first time gets a free pass. then we start tracking the time
  185. lastReceived = time.Now()
  186. r.lastReceivedRequests.Set(id, lastReceived)
  187. return nil
  188. }
  189. now := time.Now()
  190. if now.Sub(lastReceived) < r.ensurePeersPeriod/3 {
  191. return fmt.Errorf("Peer (%v) is sending too many PEX requests. Disconnecting", src.ID())
  192. }
  193. r.lastReceivedRequests.Set(id, now)
  194. return nil
  195. }
  196. // RequestAddrs asks peer for more addresses if we do not already
  197. // have a request out for this peer.
  198. func (r *PEXReactor) RequestAddrs(p Peer) {
  199. id := string(p.ID())
  200. if r.requestsSent.Has(id) {
  201. return
  202. }
  203. r.requestsSent.Set(id, struct{}{})
  204. p.Send(PexChannel, cdc.MustMarshalBinary(&pexRequestMessage{}))
  205. }
  206. // ReceiveAddrs adds the given addrs to the addrbook if theres an open
  207. // request for this peer and deletes the open request.
  208. // If there's no open request for the src peer, it returns an error.
  209. func (r *PEXReactor) ReceiveAddrs(addrs []*p2p.NetAddress, src Peer) error {
  210. id := string(src.ID())
  211. if !r.requestsSent.Has(id) {
  212. return cmn.NewError("Received unsolicited pexAddrsMessage")
  213. }
  214. r.requestsSent.Delete(id)
  215. srcAddr := src.NodeInfo().NetAddress()
  216. for _, netAddr := range addrs {
  217. if netAddr != nil {
  218. r.book.AddAddress(netAddr, srcAddr)
  219. }
  220. }
  221. return nil
  222. }
  223. // SendAddrs sends addrs to the peer.
  224. func (r *PEXReactor) SendAddrs(p Peer, netAddrs []*p2p.NetAddress) {
  225. p.Send(PexChannel, cdc.MustMarshalBinary(&pexAddrsMessage{Addrs: netAddrs}))
  226. }
  227. // SetEnsurePeersPeriod sets period to ensure peers connected.
  228. func (r *PEXReactor) SetEnsurePeersPeriod(d time.Duration) {
  229. r.ensurePeersPeriod = d
  230. }
  231. // Ensures that sufficient peers are connected. (continuous)
  232. func (r *PEXReactor) ensurePeersRoutine() {
  233. var (
  234. seed = rand.New(rand.NewSource(time.Now().UnixNano()))
  235. jitter = seed.Int63n(r.ensurePeersPeriod.Nanoseconds())
  236. )
  237. // Randomize first round of communication to avoid thundering herd.
  238. // If no potential peers are present directly start connecting so we guarantee
  239. // swift setup with the help of configured seeds.
  240. if r.hasPotentialPeers() {
  241. time.Sleep(time.Duration(jitter))
  242. }
  243. // fire once immediately.
  244. // ensures we dial the seeds right away if the book is empty
  245. r.ensurePeers()
  246. // fire periodically
  247. ticker := time.NewTicker(r.ensurePeersPeriod)
  248. for {
  249. select {
  250. case <-ticker.C:
  251. r.ensurePeers()
  252. case <-r.Quit():
  253. ticker.Stop()
  254. return
  255. }
  256. }
  257. }
  258. // ensurePeers ensures that sufficient peers are connected. (once)
  259. //
  260. // heuristic that we haven't perfected yet, or, perhaps is manually edited by
  261. // the node operator. It should not be used to compute what addresses are
  262. // already connected or not.
  263. func (r *PEXReactor) ensurePeers() {
  264. var (
  265. out, in, dial = r.Switch.NumPeers()
  266. numToDial = defaultMinNumOutboundPeers - (out + dial)
  267. )
  268. r.Logger.Info(
  269. "Ensure peers",
  270. "numOutPeers", out,
  271. "numInPeers", in,
  272. "numDialing", dial,
  273. "numToDial", numToDial,
  274. )
  275. if numToDial <= 0 {
  276. return
  277. }
  278. // bias to prefer more vetted peers when we have fewer connections.
  279. // not perfect, but somewhate ensures that we prioritize connecting to more-vetted
  280. // NOTE: range here is [10, 90]. Too high ?
  281. newBias := cmn.MinInt(out, 8)*10 + 10
  282. toDial := make(map[p2p.ID]*p2p.NetAddress)
  283. // Try maxAttempts times to pick numToDial addresses to dial
  284. maxAttempts := numToDial * 3
  285. for i := 0; i < maxAttempts && len(toDial) < numToDial; i++ {
  286. try := r.book.PickAddress(newBias)
  287. if try == nil {
  288. continue
  289. }
  290. if _, selected := toDial[try.ID]; selected {
  291. continue
  292. }
  293. if dialling := r.Switch.IsDialing(try.ID); dialling {
  294. continue
  295. }
  296. if connected := r.Switch.Peers().Has(try.ID); connected {
  297. continue
  298. }
  299. r.Logger.Info("Will dial address", "addr", try)
  300. toDial[try.ID] = try
  301. }
  302. // Dial picked addresses
  303. for _, addr := range toDial {
  304. go r.dialPeer(addr)
  305. }
  306. // If we need more addresses, pick a random peer and ask for more.
  307. if r.book.NeedMoreAddrs() {
  308. peers := r.Switch.Peers().List()
  309. peersCount := len(peers)
  310. if peersCount > 0 {
  311. peer := peers[rand.Int()%peersCount] // nolint: gas
  312. r.Logger.Info("We need more addresses. Sending pexRequest to random peer", "peer", peer)
  313. r.RequestAddrs(peer)
  314. }
  315. }
  316. // If we are not connected to nor dialing anybody, fallback to dialing a seed.
  317. if out+in+dial+len(toDial) == 0 {
  318. r.Logger.Info("No addresses to dial nor connected peers. Falling back to seeds")
  319. r.dialSeeds()
  320. }
  321. }
  322. func (r *PEXReactor) dialPeer(addr *p2p.NetAddress) {
  323. var attempts int
  324. var lastDialed time.Time
  325. if lAttempts, attempted := r.attemptsToDial.Load(addr.DialString()); attempted {
  326. attempts = lAttempts.(_attemptsToDial).number
  327. lastDialed = lAttempts.(_attemptsToDial).lastDialed
  328. }
  329. if attempts > maxAttemptsToDial {
  330. r.Logger.Error("Reached max attempts to dial", "addr", addr, "attempts", attempts)
  331. r.book.MarkBad(addr)
  332. return
  333. }
  334. // exponential backoff if it's not our first attempt to dial given address
  335. if attempts > 0 {
  336. jitterSeconds := time.Duration(rand.Float64() * float64(time.Second)) // 1s == (1e9 ns)
  337. backoffDuration := jitterSeconds + ((1 << uint(attempts)) * time.Second)
  338. sinceLastDialed := time.Since(lastDialed)
  339. if sinceLastDialed < backoffDuration {
  340. r.Logger.Debug("Too early to dial", "addr", addr, "backoff_duration", backoffDuration, "last_dialed", lastDialed, "time_since", sinceLastDialed)
  341. return
  342. }
  343. }
  344. err := r.Switch.DialPeerWithAddress(addr, false)
  345. if err != nil {
  346. r.Logger.Error("Dialing failed", "addr", addr, "err", err, "attempts", attempts)
  347. // TODO: detect more "bad peer" scenarios
  348. if _, ok := err.(p2p.ErrSwitchAuthenticationFailure); ok {
  349. r.book.MarkBad(addr)
  350. } else {
  351. r.book.MarkAttempt(addr)
  352. }
  353. // record attempt
  354. r.attemptsToDial.Store(addr.DialString(), _attemptsToDial{attempts + 1, time.Now()})
  355. } else {
  356. // cleanup any history
  357. r.attemptsToDial.Delete(addr.DialString())
  358. }
  359. }
  360. // check seed addresses are well formed
  361. func (r *PEXReactor) checkSeeds() error {
  362. lSeeds := len(r.config.Seeds)
  363. if lSeeds == 0 {
  364. return nil
  365. }
  366. _, errs := p2p.NewNetAddressStrings(r.config.Seeds)
  367. for _, err := range errs {
  368. if err != nil {
  369. return err
  370. }
  371. }
  372. return nil
  373. }
  374. // randomly dial seeds until we connect to one or exhaust them
  375. func (r *PEXReactor) dialSeeds() {
  376. lSeeds := len(r.config.Seeds)
  377. if lSeeds == 0 {
  378. return
  379. }
  380. seedAddrs, _ := p2p.NewNetAddressStrings(r.config.Seeds)
  381. perm := rand.Perm(lSeeds)
  382. // perm := r.Switch.rng.Perm(lSeeds)
  383. for _, i := range perm {
  384. // dial a random seed
  385. seedAddr := seedAddrs[i]
  386. err := r.Switch.DialPeerWithAddress(seedAddr, false)
  387. if err == nil {
  388. return
  389. }
  390. r.Switch.Logger.Error("Error dialing seed", "err", err, "seed", seedAddr)
  391. }
  392. r.Switch.Logger.Error("Couldn't connect to any seeds")
  393. }
  394. // AttemptsToDial returns the number of attempts to dial specific address. It
  395. // returns 0 if never attempted or successfully connected.
  396. func (r *PEXReactor) AttemptsToDial(addr *p2p.NetAddress) int {
  397. lAttempts, attempted := r.attemptsToDial.Load(addr.DialString())
  398. if attempted {
  399. return lAttempts.(_attemptsToDial).number
  400. } else {
  401. return 0
  402. }
  403. }
  404. //----------------------------------------------------------
  405. // Explores the network searching for more peers. (continuous)
  406. // Seed/Crawler Mode causes this node to quickly disconnect
  407. // from peers, except other seed nodes.
  408. func (r *PEXReactor) crawlPeersRoutine() {
  409. // Do an initial crawl
  410. r.crawlPeers()
  411. // Fire periodically
  412. ticker := time.NewTicker(defaultCrawlPeersPeriod)
  413. for {
  414. select {
  415. case <-ticker.C:
  416. r.attemptDisconnects()
  417. r.crawlPeers()
  418. case <-r.Quit():
  419. return
  420. }
  421. }
  422. }
  423. // hasPotentialPeers indicates if there is a potential peer to connect to, by
  424. // consulting the Switch as well as the AddrBook.
  425. func (r *PEXReactor) hasPotentialPeers() bool {
  426. out, in, dial := r.Switch.NumPeers()
  427. return out+in+dial > 0 && len(r.book.ListOfKnownAddresses()) > 0
  428. }
  429. // crawlPeerInfo handles temporary data needed for the
  430. // network crawling performed during seed/crawler mode.
  431. type crawlPeerInfo struct {
  432. // The listening address of a potential peer we learned about
  433. Addr *p2p.NetAddress
  434. // The last time we attempt to reach this address
  435. LastAttempt time.Time
  436. // The last time we successfully reached this address
  437. LastSuccess time.Time
  438. }
  439. // oldestFirst implements sort.Interface for []crawlPeerInfo
  440. // based on the LastAttempt field.
  441. type oldestFirst []crawlPeerInfo
  442. func (of oldestFirst) Len() int { return len(of) }
  443. func (of oldestFirst) Swap(i, j int) { of[i], of[j] = of[j], of[i] }
  444. func (of oldestFirst) Less(i, j int) bool { return of[i].LastAttempt.Before(of[j].LastAttempt) }
  445. // getPeersToCrawl returns addresses of potential peers that we wish to validate.
  446. // NOTE: The status information is ordered as described above.
  447. func (r *PEXReactor) getPeersToCrawl() []crawlPeerInfo {
  448. var of oldestFirst
  449. // TODO: be more selective
  450. addrs := r.book.ListOfKnownAddresses()
  451. for _, addr := range addrs {
  452. if len(addr.ID()) == 0 {
  453. continue // dont use peers without id
  454. }
  455. of = append(of, crawlPeerInfo{
  456. Addr: addr.Addr,
  457. LastAttempt: addr.LastAttempt,
  458. LastSuccess: addr.LastSuccess,
  459. })
  460. }
  461. sort.Sort(of)
  462. return of
  463. }
  464. // crawlPeers will crawl the network looking for new peer addresses. (once)
  465. func (r *PEXReactor) crawlPeers() {
  466. peerInfos := r.getPeersToCrawl()
  467. now := time.Now()
  468. // Use addresses we know of to reach additional peers
  469. for _, pi := range peerInfos {
  470. // Do not attempt to connect with peers we recently dialed
  471. if now.Sub(pi.LastAttempt) < defaultCrawlPeerInterval {
  472. continue
  473. }
  474. // Otherwise, attempt to connect with the known address
  475. err := r.Switch.DialPeerWithAddress(pi.Addr, false)
  476. if err != nil {
  477. r.book.MarkAttempt(pi.Addr)
  478. continue
  479. }
  480. }
  481. // Crawl the connected peers asking for more addresses
  482. for _, pi := range peerInfos {
  483. // We will wait a minimum period of time before crawling peers again
  484. if now.Sub(pi.LastAttempt) >= defaultCrawlPeerInterval {
  485. peer := r.Switch.Peers().Get(pi.Addr.ID)
  486. if peer != nil {
  487. r.RequestAddrs(peer)
  488. }
  489. }
  490. }
  491. }
  492. // attemptDisconnects checks if we've been with each peer long enough to disconnect
  493. func (r *PEXReactor) attemptDisconnects() {
  494. for _, peer := range r.Switch.Peers().List() {
  495. status := peer.Status()
  496. if status.Duration < defaultSeedDisconnectWaitPeriod {
  497. continue
  498. }
  499. if peer.IsPersistent() {
  500. continue
  501. }
  502. r.Switch.StopPeerGracefully(peer)
  503. }
  504. }
  505. //-----------------------------------------------------------------------------
  506. // Messages
  507. // PexMessage is a primary type for PEX messages. Underneath, it could contain
  508. // either pexRequestMessage, or pexAddrsMessage messages.
  509. type PexMessage interface{}
  510. func RegisterPexMessage(cdc *amino.Codec) {
  511. cdc.RegisterInterface((*PexMessage)(nil), nil)
  512. cdc.RegisterConcrete(&pexRequestMessage{}, "tendermint/p2p/PexRequestMessage", nil)
  513. cdc.RegisterConcrete(&pexAddrsMessage{}, "tendermint/p2p/PexAddrsMessage", nil)
  514. }
  515. // DecodeMessage implements interface registered above.
  516. func DecodeMessage(bz []byte) (msg PexMessage, err error) {
  517. err = cdc.UnmarshalBinary(bz, &msg)
  518. return
  519. }
  520. /*
  521. A pexRequestMessage requests additional peer addresses.
  522. */
  523. type pexRequestMessage struct {
  524. }
  525. func (m *pexRequestMessage) String() string {
  526. return "[pexRequest]"
  527. }
  528. /*
  529. A message with announced peer addresses.
  530. */
  531. type pexAddrsMessage struct {
  532. Addrs []*p2p.NetAddress
  533. }
  534. func (m *pexAddrsMessage) String() string {
  535. return fmt.Sprintf("[pexAddrs %v]", m.Addrs)
  536. }