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.

662 lines
19 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
8 years ago
9 years ago
8 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
8 years ago
9 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
9 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
9 years ago
7 years ago
9 years ago
9 years ago
9 years ago
7 years ago
9 years ago
9 years ago
8 years ago
9 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
9 years ago
8 years ago
7 years ago
7 years ago
9 years ago
9 years ago
9 years ago
9 years ago
8 years ago
9 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
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
8 years ago
9 years ago
7 years ago
9 years ago
8 years ago
9 years ago
9 years ago
9 years ago
9 years ago
7 years ago
7 years ago
7 years ago
8 years ago
8 years ago
8 years ago
8 years ago
  1. package p2p
  2. import (
  3. "fmt"
  4. "math"
  5. "math/rand"
  6. "net"
  7. "time"
  8. "github.com/pkg/errors"
  9. crypto "github.com/tendermint/go-crypto"
  10. cfg "github.com/tendermint/tendermint/config"
  11. cmn "github.com/tendermint/tmlibs/common"
  12. "github.com/tendermint/tmlibs/log"
  13. )
  14. const (
  15. // wait a random amount of time from this interval
  16. // before dialing seeds or reconnecting to help prevent DoS
  17. dialRandomizerIntervalMilliseconds = 3000
  18. // repeatedly try to reconnect for a few minutes
  19. // ie. 5 * 20 = 100s
  20. reconnectAttempts = 20
  21. reconnectInterval = 5 * time.Second
  22. // then move into exponential backoff mode for ~1day
  23. // ie. 3**10 = 16hrs
  24. reconnectBackOffAttempts = 10
  25. reconnectBackOffBaseSeconds = 3
  26. )
  27. type Reactor interface {
  28. cmn.Service // Start, Stop
  29. SetSwitch(*Switch)
  30. GetChannels() []*ChannelDescriptor
  31. AddPeer(peer Peer)
  32. RemovePeer(peer Peer, reason interface{})
  33. Receive(chID byte, peer Peer, msgBytes []byte) // CONTRACT: msgBytes are not nil
  34. }
  35. //--------------------------------------
  36. type BaseReactor struct {
  37. cmn.BaseService // Provides Start, Stop, .Quit
  38. Switch *Switch
  39. }
  40. func NewBaseReactor(name string, impl Reactor) *BaseReactor {
  41. return &BaseReactor{
  42. BaseService: *cmn.NewBaseService(nil, name, impl),
  43. Switch: nil,
  44. }
  45. }
  46. func (br *BaseReactor) SetSwitch(sw *Switch) {
  47. br.Switch = sw
  48. }
  49. func (_ *BaseReactor) GetChannels() []*ChannelDescriptor { return nil }
  50. func (_ *BaseReactor) AddPeer(peer Peer) {}
  51. func (_ *BaseReactor) RemovePeer(peer Peer, reason interface{}) {}
  52. func (_ *BaseReactor) Receive(chID byte, peer Peer, msgBytes []byte) {}
  53. //-----------------------------------------------------------------------------
  54. /*
  55. The `Switch` handles peer connections and exposes an API to receive incoming messages
  56. on `Reactors`. Each `Reactor` is responsible for handling incoming messages of one
  57. or more `Channels`. So while sending outgoing messages is typically performed on the peer,
  58. incoming messages are received on the reactor.
  59. */
  60. type Switch struct {
  61. cmn.BaseService
  62. config *cfg.P2PConfig
  63. peerConfig *PeerConfig
  64. listeners []Listener
  65. reactors map[string]Reactor
  66. chDescs []*ChannelDescriptor
  67. reactorsByCh map[byte]Reactor
  68. peers *PeerSet
  69. dialing *cmn.CMap
  70. nodeInfo *NodeInfo // our node info
  71. nodeKey *NodeKey // our node privkey
  72. peerIDTarget []byte
  73. filterConnByAddr func(net.Addr) error
  74. filterConnByPubKey func(crypto.PubKey) error
  75. rng *rand.Rand // seed for randomizing dial times and orders
  76. }
  77. var (
  78. ErrSwitchDuplicatePeer = errors.New("Duplicate peer")
  79. )
  80. func NewSwitch(config *cfg.P2PConfig) *Switch {
  81. sw := &Switch{
  82. config: config,
  83. peerConfig: DefaultPeerConfig(),
  84. reactors: make(map[string]Reactor),
  85. chDescs: make([]*ChannelDescriptor, 0),
  86. reactorsByCh: make(map[byte]Reactor),
  87. peers: NewPeerSet(),
  88. dialing: cmn.NewCMap(),
  89. nodeInfo: nil,
  90. }
  91. // Ensure we have a completely undeterministic PRNG. cmd.RandInt64() draws
  92. // from a seed that's initialized with OS entropy on process start.
  93. sw.rng = rand.New(rand.NewSource(cmn.RandInt64()))
  94. // TODO: collapse the peerConfig into the config ?
  95. sw.peerConfig.MConfig.flushThrottle = time.Duration(config.FlushThrottleTimeout) * time.Millisecond
  96. sw.peerConfig.MConfig.SendRate = config.SendRate
  97. sw.peerConfig.MConfig.RecvRate = config.RecvRate
  98. sw.peerConfig.MConfig.maxMsgPacketPayloadSize = config.MaxMsgPacketPayloadSize
  99. sw.BaseService = *cmn.NewBaseService(nil, "P2P Switch", sw)
  100. return sw
  101. }
  102. // AddReactor adds the given reactor to the switch.
  103. // NOTE: Not goroutine safe.
  104. func (sw *Switch) AddReactor(name string, reactor Reactor) Reactor {
  105. // Validate the reactor.
  106. // No two reactors can share the same channel.
  107. reactorChannels := reactor.GetChannels()
  108. for _, chDesc := range reactorChannels {
  109. chID := chDesc.ID
  110. if sw.reactorsByCh[chID] != nil {
  111. cmn.PanicSanity(fmt.Sprintf("Channel %X has multiple reactors %v & %v", chID, sw.reactorsByCh[chID], reactor))
  112. }
  113. sw.chDescs = append(sw.chDescs, chDesc)
  114. sw.reactorsByCh[chID] = reactor
  115. }
  116. sw.reactors[name] = reactor
  117. reactor.SetSwitch(sw)
  118. return reactor
  119. }
  120. // Reactors returns a map of reactors registered on the switch.
  121. // NOTE: Not goroutine safe.
  122. func (sw *Switch) Reactors() map[string]Reactor {
  123. return sw.reactors
  124. }
  125. // Reactor returns the reactor with the given name.
  126. // NOTE: Not goroutine safe.
  127. func (sw *Switch) Reactor(name string) Reactor {
  128. return sw.reactors[name]
  129. }
  130. // AddListener adds the given listener to the switch for listening to incoming peer connections.
  131. // NOTE: Not goroutine safe.
  132. func (sw *Switch) AddListener(l Listener) {
  133. sw.listeners = append(sw.listeners, l)
  134. }
  135. // Listeners returns the list of listeners the switch listens on.
  136. // NOTE: Not goroutine safe.
  137. func (sw *Switch) Listeners() []Listener {
  138. return sw.listeners
  139. }
  140. // IsListening returns true if the switch has at least one listener.
  141. // NOTE: Not goroutine safe.
  142. func (sw *Switch) IsListening() bool {
  143. return len(sw.listeners) > 0
  144. }
  145. // SetNodeInfo sets the switch's NodeInfo for checking compatibility and handshaking with other nodes.
  146. // NOTE: Not goroutine safe.
  147. func (sw *Switch) SetNodeInfo(nodeInfo *NodeInfo) {
  148. sw.nodeInfo = nodeInfo
  149. }
  150. // NodeInfo returns the switch's NodeInfo.
  151. // NOTE: Not goroutine safe.
  152. func (sw *Switch) NodeInfo() *NodeInfo {
  153. return sw.nodeInfo
  154. }
  155. // SetNodeKey sets the switch's private key for authenticated encryption.
  156. // NOTE: Overwrites sw.nodeInfo.PubKey.
  157. // NOTE: Not goroutine safe.
  158. func (sw *Switch) SetNodeKey(nodeKey *NodeKey) {
  159. sw.nodeKey = nodeKey
  160. if sw.nodeInfo != nil {
  161. sw.nodeInfo.PubKey = nodeKey.PubKey()
  162. }
  163. }
  164. // SetPeerIDTarget sets the target for incoming peer ID's -
  165. // the ID must be less than the target
  166. func (sw *Switch) SetPeerIDTarget(target []byte) {
  167. sw.peerIDTarget = target
  168. }
  169. // OnStart implements BaseService. It starts all the reactors, peers, and listeners.
  170. func (sw *Switch) OnStart() error {
  171. // Start reactors
  172. for _, reactor := range sw.reactors {
  173. err := reactor.Start()
  174. if err != nil {
  175. return errors.Wrapf(err, "failed to start %v", reactor)
  176. }
  177. }
  178. // Start listeners
  179. for _, listener := range sw.listeners {
  180. go sw.listenerRoutine(listener)
  181. }
  182. return nil
  183. }
  184. // OnStop implements BaseService. It stops all listeners, peers, and reactors.
  185. func (sw *Switch) OnStop() {
  186. // Stop listeners
  187. for _, listener := range sw.listeners {
  188. listener.Stop()
  189. }
  190. sw.listeners = nil
  191. // Stop peers
  192. for _, peer := range sw.peers.List() {
  193. peer.Stop()
  194. sw.peers.Remove(peer)
  195. }
  196. // Stop reactors
  197. sw.Logger.Debug("Switch: Stopping reactors")
  198. for _, reactor := range sw.reactors {
  199. reactor.Stop()
  200. }
  201. }
  202. // addPeer checks the given peer's validity, performs a handshake, and adds the
  203. // peer to the switch and to all registered reactors.
  204. // NOTE: This performs a blocking handshake before the peer is added.
  205. // NOTE: If error is returned, caller is responsible for calling peer.CloseConn()
  206. func (sw *Switch) addPeer(peer *peer) error {
  207. // Avoid self
  208. if sw.nodeInfo.PubKey.Equals(peer.PubKey().Wrap()) {
  209. return errors.New("Ignoring connection from self")
  210. }
  211. // Filter peer against white list
  212. if err := sw.FilterConnByAddr(peer.Addr()); err != nil {
  213. return err
  214. }
  215. if err := sw.FilterConnByPubKey(peer.PubKey()); err != nil {
  216. return err
  217. }
  218. if err := peer.HandshakeTimeout(sw.nodeInfo, time.Duration(sw.peerConfig.HandshakeTimeout*time.Second)); err != nil {
  219. return err
  220. }
  221. // Avoid duplicate
  222. if sw.peers.Has(peer.ID()) {
  223. return ErrSwitchDuplicatePeer
  224. }
  225. // Check version, chain id
  226. if err := sw.nodeInfo.CompatibleWith(peer.NodeInfo()); err != nil {
  227. return err
  228. }
  229. // Start peer
  230. if sw.IsRunning() {
  231. sw.startInitPeer(peer)
  232. }
  233. // Add the peer to .peers.
  234. // We start it first so that a peer in the list is safe to Stop.
  235. // It should not err since we already checked peers.Has().
  236. if err := sw.peers.Add(peer); err != nil {
  237. return err
  238. }
  239. sw.Logger.Info("Added peer", "peer", peer)
  240. return nil
  241. }
  242. // FilterConnByAddr returns an error if connecting to the given address is forbidden.
  243. func (sw *Switch) FilterConnByAddr(addr net.Addr) error {
  244. if sw.filterConnByAddr != nil {
  245. return sw.filterConnByAddr(addr)
  246. }
  247. return nil
  248. }
  249. // FilterConnByPubKey returns an error if connecting to the given public key is forbidden.
  250. func (sw *Switch) FilterConnByPubKey(pubkey crypto.PubKey) error {
  251. if sw.filterConnByPubKey != nil {
  252. return sw.filterConnByPubKey(pubkey)
  253. }
  254. return nil
  255. }
  256. // SetAddrFilter sets the function for filtering connections by address.
  257. func (sw *Switch) SetAddrFilter(f func(net.Addr) error) {
  258. sw.filterConnByAddr = f
  259. }
  260. // SetPubKeyFilter sets the function for filtering connections by public key.
  261. func (sw *Switch) SetPubKeyFilter(f func(crypto.PubKey) error) {
  262. sw.filterConnByPubKey = f
  263. }
  264. func (sw *Switch) startInitPeer(peer *peer) {
  265. err := peer.Start() // spawn send/recv routines
  266. if err != nil {
  267. // Should never happen
  268. sw.Logger.Error("Error starting peer", "peer", peer, "err", err)
  269. }
  270. for _, reactor := range sw.reactors {
  271. reactor.AddPeer(peer)
  272. }
  273. }
  274. // DialSeeds dials a list of seeds asynchronously in random order.
  275. func (sw *Switch) DialSeeds(addrBook *AddrBook, seeds []string) error {
  276. netAddrs, errs := NewNetAddressStrings(seeds)
  277. // TODO: IDs
  278. for _, err := range errs {
  279. sw.Logger.Error("Error in seed's address", "err", err)
  280. }
  281. if addrBook != nil {
  282. // add seeds to `addrBook`
  283. ourAddrS := sw.nodeInfo.ListenAddr
  284. ourAddr, _ := NewNetAddressString(ourAddrS)
  285. for _, netAddr := range netAddrs {
  286. // do not add ourselves
  287. if netAddr.Equals(ourAddr) {
  288. continue
  289. }
  290. addrBook.AddAddress(netAddr, ourAddr)
  291. }
  292. addrBook.Save()
  293. }
  294. // permute the list, dial them in random order.
  295. perm := sw.rng.Perm(len(netAddrs))
  296. for i := 0; i < len(perm); i++ {
  297. go func(i int) {
  298. sw.randomSleep(0)
  299. j := perm[i]
  300. sw.dialSeed(netAddrs[j])
  301. }(i)
  302. }
  303. return nil
  304. }
  305. // sleep for interval plus some random amount of ms on [0, dialRandomizerIntervalMilliseconds]
  306. func (sw *Switch) randomSleep(interval time.Duration) {
  307. r := time.Duration(sw.rng.Int63n(dialRandomizerIntervalMilliseconds)) * time.Millisecond
  308. time.Sleep(r + interval)
  309. }
  310. func (sw *Switch) dialSeed(addr *NetAddress) {
  311. peer, err := sw.DialPeerWithAddress(addr, true)
  312. if err != nil {
  313. sw.Logger.Error("Error dialing seed", "err", err)
  314. } else {
  315. sw.Logger.Info("Connected to seed", "peer", peer)
  316. }
  317. }
  318. // DialPeerWithAddress dials the given peer and runs sw.addPeer if it connects successfully.
  319. // If `persistent == true`, the switch will always try to reconnect to this peer if the connection ever fails.
  320. func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) (Peer, error) {
  321. sw.dialing.Set(string(addr.ID), addr)
  322. defer sw.dialing.Delete(string(addr.ID))
  323. sw.Logger.Info("Dialing peer", "address", addr)
  324. peer, err := newOutboundPeer(addr, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, sw.peerConfig)
  325. if err != nil {
  326. sw.Logger.Error("Failed to dial peer", "address", addr, "err", err)
  327. return nil, err
  328. }
  329. peer.SetLogger(sw.Logger.With("peer", addr))
  330. if persistent {
  331. peer.makePersistent()
  332. }
  333. err = sw.addPeer(peer)
  334. if err != nil {
  335. sw.Logger.Error("Failed to add peer", "address", addr, "err", err)
  336. peer.CloseConn()
  337. return nil, err
  338. }
  339. sw.Logger.Info("Dialed and added peer", "address", addr, "peer", peer)
  340. return peer, nil
  341. }
  342. // IsDialing returns true if the switch is currently dialing the given ID.
  343. func (sw *Switch) IsDialing(id ID) bool {
  344. return sw.dialing.Has(string(id))
  345. }
  346. // Broadcast runs a go routine for each attempted send, which will block
  347. // trying to send for defaultSendTimeoutSeconds. Returns a channel
  348. // which receives success values for each attempted send (false if times out).
  349. // NOTE: Broadcast uses goroutines, so order of broadcast may not be preserved.
  350. // TODO: Something more intelligent.
  351. func (sw *Switch) Broadcast(chID byte, msg interface{}) chan bool {
  352. successChan := make(chan bool, len(sw.peers.List()))
  353. sw.Logger.Debug("Broadcast", "channel", chID, "msg", msg)
  354. for _, peer := range sw.peers.List() {
  355. go func(peer Peer) {
  356. success := peer.Send(chID, msg)
  357. successChan <- success
  358. }(peer)
  359. }
  360. return successChan
  361. }
  362. // NumPeers returns the count of outbound/inbound and outbound-dialing peers.
  363. func (sw *Switch) NumPeers() (outbound, inbound, dialing int) {
  364. peers := sw.peers.List()
  365. for _, peer := range peers {
  366. if peer.IsOutbound() {
  367. outbound++
  368. } else {
  369. inbound++
  370. }
  371. }
  372. dialing = sw.dialing.Size()
  373. return
  374. }
  375. // Peers returns the set of peers that are connected to the switch.
  376. func (sw *Switch) Peers() IPeerSet {
  377. return sw.peers
  378. }
  379. // StopPeerForError disconnects from a peer due to external error.
  380. // If the peer is persistent, it will attempt to reconnect.
  381. // TODO: make record depending on reason.
  382. func (sw *Switch) StopPeerForError(peer Peer, reason interface{}) {
  383. sw.Logger.Error("Stopping peer for error", "peer", peer, "err", reason)
  384. sw.stopAndRemovePeer(peer, reason)
  385. if peer.IsPersistent() {
  386. go sw.reconnectToPeer(peer)
  387. }
  388. }
  389. // reconnectToPeer tries to reconnect to the peer, first repeatedly
  390. // with a fixed interval, then with exponential backoff.
  391. // If no success after all that, it stops trying, and leaves it
  392. // to the PEX/Addrbook to find the peer again
  393. func (sw *Switch) reconnectToPeer(peer Peer) {
  394. netAddr, _ := NewNetAddressString(peer.NodeInfo().RemoteAddr)
  395. netAddr.ID = peer.ID() // TODO: handle above
  396. start := time.Now()
  397. sw.Logger.Info("Reconnecting to peer", "peer", peer)
  398. for i := 0; i < reconnectAttempts; i++ {
  399. if !sw.IsRunning() {
  400. return
  401. }
  402. peer, err := sw.DialPeerWithAddress(netAddr, true)
  403. if err != nil {
  404. sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "peer", peer)
  405. // sleep a set amount
  406. sw.randomSleep(reconnectInterval)
  407. continue
  408. } else {
  409. sw.Logger.Info("Reconnected to peer", "peer", peer)
  410. return
  411. }
  412. }
  413. sw.Logger.Error("Failed to reconnect to peer. Beginning exponential backoff",
  414. "peer", peer, "elapsed", time.Since(start))
  415. for i := 0; i < reconnectBackOffAttempts; i++ {
  416. if !sw.IsRunning() {
  417. return
  418. }
  419. // sleep an exponentially increasing amount
  420. sleepIntervalSeconds := math.Pow(reconnectBackOffBaseSeconds, float64(i))
  421. sw.randomSleep(time.Duration(sleepIntervalSeconds) * time.Second)
  422. peer, err := sw.DialPeerWithAddress(netAddr, true)
  423. if err != nil {
  424. sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "err", err, "peer", peer)
  425. continue
  426. } else {
  427. sw.Logger.Info("Reconnected to peer", "peer", peer)
  428. return
  429. }
  430. }
  431. sw.Logger.Error("Failed to reconnect to peer. Giving up", "peer", peer, "elapsed", time.Since(start))
  432. }
  433. // StopPeerGracefully disconnects from a peer gracefully.
  434. // TODO: handle graceful disconnects.
  435. func (sw *Switch) StopPeerGracefully(peer Peer) {
  436. sw.Logger.Info("Stopping peer gracefully")
  437. sw.stopAndRemovePeer(peer, nil)
  438. }
  439. func (sw *Switch) stopAndRemovePeer(peer Peer, reason interface{}) {
  440. sw.peers.Remove(peer)
  441. peer.Stop()
  442. for _, reactor := range sw.reactors {
  443. reactor.RemovePeer(peer, reason)
  444. }
  445. }
  446. func (sw *Switch) listenerRoutine(l Listener) {
  447. for {
  448. inConn, ok := <-l.Connections()
  449. if !ok {
  450. break
  451. }
  452. // ignore connection if we already have enough
  453. maxPeers := sw.config.MaxNumPeers
  454. if maxPeers <= sw.peers.Size() {
  455. sw.Logger.Info("Ignoring inbound connection: already have enough peers", "address", inConn.RemoteAddr().String(), "numPeers", sw.peers.Size(), "max", maxPeers)
  456. continue
  457. }
  458. // New inbound connection!
  459. err := sw.addPeerWithConnectionAndConfig(inConn, sw.peerConfig)
  460. if err != nil {
  461. sw.Logger.Info("Ignoring inbound connection: error while adding peer", "address", inConn.RemoteAddr().String(), "err", err)
  462. continue
  463. }
  464. // NOTE: We don't yet have the listening port of the
  465. // remote (if they have a listener at all).
  466. // The peerHandshake will handle that.
  467. }
  468. // cleanup
  469. }
  470. //------------------------------------------------------------------
  471. // Connects switches via arbitrary net.Conn. Used for testing.
  472. // MakeConnectedSwitches returns n switches, connected according to the connect func.
  473. // If connect==Connect2Switches, the switches will be fully connected.
  474. // initSwitch defines how the i'th switch should be initialized (ie. with what reactors).
  475. // NOTE: panics if any switch fails to start.
  476. func MakeConnectedSwitches(cfg *cfg.P2PConfig, n int, initSwitch func(int, *Switch) *Switch, connect func([]*Switch, int, int)) []*Switch {
  477. switches := make([]*Switch, n)
  478. for i := 0; i < n; i++ {
  479. switches[i] = makeSwitch(cfg, i, "testing", "123.123.123", initSwitch)
  480. }
  481. if err := StartSwitches(switches); err != nil {
  482. panic(err)
  483. }
  484. for i := 0; i < n; i++ {
  485. for j := i + 1; j < n; j++ {
  486. connect(switches, i, j)
  487. }
  488. }
  489. return switches
  490. }
  491. // Connect2Switches will connect switches i and j via net.Pipe().
  492. // Blocks until a connection is established.
  493. // NOTE: caller ensures i and j are within bounds.
  494. func Connect2Switches(switches []*Switch, i, j int) {
  495. switchI := switches[i]
  496. switchJ := switches[j]
  497. c1, c2 := netPipe()
  498. doneCh := make(chan struct{})
  499. go func() {
  500. err := switchI.addPeerWithConnection(c1)
  501. if err != nil {
  502. panic(err)
  503. }
  504. doneCh <- struct{}{}
  505. }()
  506. go func() {
  507. err := switchJ.addPeerWithConnection(c2)
  508. if err != nil {
  509. panic(err)
  510. }
  511. doneCh <- struct{}{}
  512. }()
  513. <-doneCh
  514. <-doneCh
  515. }
  516. // StartSwitches calls sw.Start() for each given switch.
  517. // It returns the first encountered error.
  518. func StartSwitches(switches []*Switch) error {
  519. for _, s := range switches {
  520. err := s.Start() // start switch and reactors
  521. if err != nil {
  522. return err
  523. }
  524. }
  525. return nil
  526. }
  527. func makeSwitch(cfg *cfg.P2PConfig, i int, network, version string, initSwitch func(int, *Switch) *Switch) *Switch {
  528. // new switch, add reactors
  529. // TODO: let the config be passed in?
  530. nodeKey := &NodeKey{
  531. PrivKey: crypto.GenPrivKeyEd25519().Wrap(),
  532. }
  533. s := initSwitch(i, NewSwitch(cfg))
  534. s.SetNodeInfo(&NodeInfo{
  535. PubKey: nodeKey.PubKey(),
  536. Moniker: cmn.Fmt("switch%d", i),
  537. Network: network,
  538. Version: version,
  539. RemoteAddr: cmn.Fmt("%v:%v", network, rand.Intn(64512)+1023),
  540. ListenAddr: cmn.Fmt("%v:%v", network, rand.Intn(64512)+1023),
  541. })
  542. s.SetNodeKey(nodeKey)
  543. s.SetLogger(log.TestingLogger())
  544. return s
  545. }
  546. func (sw *Switch) addPeerWithConnection(conn net.Conn) error {
  547. peer, err := newInboundPeer(conn, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, sw.peerConfig)
  548. if err != nil {
  549. if err := conn.Close(); err != nil {
  550. sw.Logger.Error("Error closing connection", "err", err)
  551. }
  552. return err
  553. }
  554. peer.SetLogger(sw.Logger.With("peer", conn.RemoteAddr()))
  555. if err = sw.addPeer(peer); err != nil {
  556. peer.CloseConn()
  557. return err
  558. }
  559. return nil
  560. }
  561. func (sw *Switch) addPeerWithConnectionAndConfig(conn net.Conn, config *PeerConfig) error {
  562. peer, err := newInboundPeer(conn, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodeKey.PrivKey, config)
  563. if err != nil {
  564. if err := conn.Close(); err != nil {
  565. sw.Logger.Error("Error closing connection", "err", err)
  566. }
  567. return err
  568. }
  569. peer.SetLogger(sw.Logger.With("peer", conn.RemoteAddr()))
  570. if err = sw.addPeer(peer); err != nil {
  571. peer.CloseConn()
  572. return err
  573. }
  574. return nil
  575. }