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.

722 lines
20 KiB

  1. package pex_test
  2. import (
  3. "context"
  4. "testing"
  5. "time"
  6. "github.com/stretchr/testify/require"
  7. "github.com/tendermint/tendermint/libs/log"
  8. "github.com/tendermint/tendermint/p2p"
  9. "github.com/tendermint/tendermint/p2p/p2ptest"
  10. "github.com/tendermint/tendermint/p2p/pex"
  11. proto "github.com/tendermint/tendermint/proto/tendermint/p2p"
  12. )
  13. const (
  14. checkFrequency = 500 * time.Millisecond
  15. defaultBufferSize = 2
  16. shortWait = 10 * time.Second
  17. longWait = 60 * time.Second
  18. firstNode = 0
  19. secondNode = 1
  20. thirdNode = 2
  21. fourthNode = 3
  22. )
  23. func TestReactorBasic(t *testing.T) {
  24. // start a network with one mock reactor and one "real" reactor
  25. testNet := setup(t, testOptions{
  26. MockNodes: 1,
  27. TotalNodes: 2,
  28. })
  29. testNet.connectAll(t)
  30. testNet.start(t)
  31. // assert that the mock node receives a request from the real node
  32. testNet.listenForRequest(t, secondNode, firstNode, shortWait)
  33. // assert that when a mock node sends a request it receives a response (and
  34. // the correct one)
  35. testNet.sendRequest(t, firstNode, secondNode, true)
  36. testNet.listenForResponse(t, secondNode, firstNode, shortWait, []proto.PexAddressV2(nil))
  37. }
  38. func TestReactorConnectFullNetwork(t *testing.T) {
  39. testNet := setup(t, testOptions{
  40. TotalNodes: 8,
  41. })
  42. // make every node be only connected with one other node (it actually ends up
  43. // being two because of two way connections but oh well)
  44. testNet.connectN(t, 1)
  45. testNet.start(t)
  46. // assert that all nodes add each other in the network
  47. for idx := 0; idx < len(testNet.nodes); idx++ {
  48. testNet.requireNumberOfPeers(t, idx, len(testNet.nodes)-1, longWait)
  49. }
  50. }
  51. func TestReactorSendsRequestsTooOften(t *testing.T) {
  52. testNet := setup(t, testOptions{
  53. MockNodes: 1,
  54. TotalNodes: 3,
  55. })
  56. testNet.connectAll(t)
  57. testNet.start(t)
  58. // firstNode sends two requests to the secondNode
  59. testNet.sendRequest(t, firstNode, secondNode, true)
  60. testNet.sendRequest(t, firstNode, secondNode, true)
  61. // assert that the secondNode evicts the first node (although they reconnect
  62. // straight away again)
  63. testNet.listenForPeerUpdate(t, secondNode, firstNode, p2p.PeerStatusDown, shortWait)
  64. // firstNode should still receive the address of the thirdNode by the secondNode
  65. expectedAddrs := testNet.getV2AddressesFor([]int{thirdNode})
  66. testNet.listenForResponse(t, secondNode, firstNode, shortWait, expectedAddrs)
  67. }
  68. func TestReactorSendsResponseWithoutRequest(t *testing.T) {
  69. testNet := setup(t, testOptions{
  70. MockNodes: 1,
  71. TotalNodes: 3,
  72. })
  73. testNet.connectAll(t)
  74. testNet.start(t)
  75. // firstNode sends the secondNode an unrequested response
  76. // NOTE: secondNode will send a request by default during startup so we send
  77. // two responses to counter that.
  78. testNet.sendResponse(t, firstNode, secondNode, []int{thirdNode}, true)
  79. testNet.sendResponse(t, firstNode, secondNode, []int{thirdNode}, true)
  80. // secondNode should evict the firstNode
  81. testNet.listenForPeerUpdate(t, secondNode, firstNode, p2p.PeerStatusDown, shortWait)
  82. }
  83. func TestReactorNeverSendsTooManyPeers(t *testing.T) {
  84. testNet := setup(t, testOptions{
  85. MockNodes: 1,
  86. TotalNodes: 2,
  87. })
  88. testNet.connectAll(t)
  89. testNet.start(t)
  90. testNet.addNodes(t, 110)
  91. nodes := make([]int, 110)
  92. for i := 0; i < len(nodes); i++ {
  93. nodes[i] = i + 2
  94. }
  95. testNet.addAddresses(t, secondNode, nodes)
  96. // first we check that even although we have 110 peers, honest pex reactors
  97. // only send 100 (test if secondNode sends firstNode 100 addresses)
  98. testNet.pingAndlistenForNAddresses(t, secondNode, firstNode, shortWait, 100)
  99. }
  100. func TestReactorErrorsOnReceivingTooManyPeers(t *testing.T) {
  101. testNet := setup(t, testOptions{
  102. MockNodes: 1,
  103. TotalNodes: 2,
  104. })
  105. testNet.connectAll(t)
  106. testNet.start(t)
  107. testNet.addNodes(t, 110)
  108. nodes := make([]int, 110)
  109. for i := 0; i < len(nodes); i++ {
  110. nodes[i] = i + 2
  111. }
  112. // now we send a response with more than 100 peers
  113. testNet.sendResponse(t, firstNode, secondNode, nodes, true)
  114. // secondNode should evict the firstNode
  115. testNet.listenForPeerUpdate(t, secondNode, firstNode, p2p.PeerStatusDown, shortWait)
  116. }
  117. func TestReactorSmallPeerStoreInALargeNetwork(t *testing.T) {
  118. testNet := setup(t, testOptions{
  119. TotalNodes: 16,
  120. MaxPeers: 8,
  121. MaxConnected: 6,
  122. BufferSize: 8,
  123. })
  124. testNet.connectN(t, 1)
  125. testNet.start(t)
  126. // test that all nodes reach full capacity
  127. for _, nodeID := range testNet.nodes {
  128. require.Eventually(t, func() bool {
  129. // nolint:scopelint
  130. return testNet.network.Nodes[nodeID].PeerManager.PeerRatio() >= 0.9
  131. }, longWait, checkFrequency)
  132. }
  133. }
  134. func TestReactorLargePeerStoreInASmallNetwork(t *testing.T) {
  135. testNet := setup(t, testOptions{
  136. TotalNodes: 10,
  137. MaxPeers: 100,
  138. MaxConnected: 100,
  139. BufferSize: 10,
  140. })
  141. testNet.connectN(t, 1)
  142. testNet.start(t)
  143. // assert that all nodes add each other in the network
  144. for idx := 0; idx < len(testNet.nodes); idx++ {
  145. testNet.requireNumberOfPeers(t, idx, len(testNet.nodes)-1, longWait)
  146. }
  147. }
  148. func TestReactorWithNetworkGrowth(t *testing.T) {
  149. testNet := setup(t, testOptions{
  150. TotalNodes: 5,
  151. BufferSize: 5,
  152. })
  153. testNet.connectAll(t)
  154. testNet.start(t)
  155. // assert that all nodes add each other in the network
  156. for idx := 0; idx < len(testNet.nodes); idx++ {
  157. testNet.requireNumberOfPeers(t, idx, len(testNet.nodes)-1, shortWait)
  158. }
  159. // now we inject 10 more nodes
  160. testNet.addNodes(t, 10)
  161. for i := 5; i < testNet.total; i++ {
  162. node := testNet.nodes[i]
  163. require.NoError(t, testNet.reactors[node].Start())
  164. require.True(t, testNet.reactors[node].IsRunning())
  165. // we connect all new nodes to a single entry point and check that the
  166. // node can distribute the addresses to all the others
  167. testNet.connectPeers(t, 0, i)
  168. }
  169. require.Len(t, testNet.reactors, 15)
  170. // assert that all nodes add each other in the network
  171. for idx := 0; idx < len(testNet.nodes); idx++ {
  172. testNet.requireNumberOfPeers(t, idx, len(testNet.nodes)-1, longWait)
  173. }
  174. }
  175. func TestReactorIntegrationWithLegacyHandleRequest(t *testing.T) {
  176. testNet := setup(t, testOptions{
  177. MockNodes: 1,
  178. TotalNodes: 3,
  179. })
  180. testNet.connectAll(t)
  181. testNet.start(t)
  182. t.Log(testNet.nodes)
  183. // mock node sends a V1 Pex message to the second node
  184. testNet.sendRequest(t, firstNode, secondNode, false)
  185. addrs := testNet.getAddressesFor(t, []int{thirdNode})
  186. testNet.listenForLegacyResponse(t, secondNode, firstNode, shortWait, addrs)
  187. }
  188. func TestReactorIntegrationWithLegacyHandleResponse(t *testing.T) {
  189. testNet := setup(t, testOptions{
  190. MockNodes: 1,
  191. TotalNodes: 4,
  192. BufferSize: 4,
  193. })
  194. testNet.connectPeers(t, firstNode, secondNode)
  195. testNet.connectPeers(t, firstNode, thirdNode)
  196. testNet.connectPeers(t, firstNode, fourthNode)
  197. testNet.start(t)
  198. testNet.listenForRequest(t, secondNode, firstNode, shortWait)
  199. // send a v1 response instead
  200. testNet.sendResponse(t, firstNode, secondNode, []int{thirdNode, fourthNode}, false)
  201. testNet.requireNumberOfPeers(t, secondNode, len(testNet.nodes)-1, shortWait)
  202. }
  203. type reactorTestSuite struct {
  204. network *p2ptest.Network
  205. logger log.Logger
  206. reactors map[p2p.NodeID]*pex.ReactorV2
  207. pexChannels map[p2p.NodeID]*p2p.Channel
  208. peerChans map[p2p.NodeID]chan p2p.PeerUpdate
  209. peerUpdates map[p2p.NodeID]*p2p.PeerUpdates
  210. nodes []p2p.NodeID
  211. mocks []p2p.NodeID
  212. total int
  213. opts testOptions
  214. }
  215. type testOptions struct {
  216. MockNodes int
  217. TotalNodes int
  218. BufferSize int
  219. MaxPeers uint16
  220. MaxConnected uint16
  221. }
  222. // setup setups a test suite with a network of nodes. Mocknodes represent the
  223. // hollow nodes that the test can listen and send on
  224. func setup(t *testing.T, opts testOptions) *reactorTestSuite {
  225. t.Helper()
  226. require.Greater(t, opts.TotalNodes, opts.MockNodes)
  227. if opts.BufferSize == 0 {
  228. opts.BufferSize = defaultBufferSize
  229. }
  230. networkOpts := p2ptest.NetworkOptions{
  231. NumNodes: opts.TotalNodes,
  232. BufferSize: opts.BufferSize,
  233. NodeOpts: p2ptest.NodeOptions{
  234. MaxPeers: opts.MaxPeers,
  235. MaxConnected: opts.MaxConnected,
  236. },
  237. }
  238. chBuf := opts.BufferSize
  239. realNodes := opts.TotalNodes - opts.MockNodes
  240. rts := &reactorTestSuite{
  241. logger: log.TestingLogger().With("testCase", t.Name()),
  242. network: p2ptest.MakeNetwork(t, networkOpts),
  243. reactors: make(map[p2p.NodeID]*pex.ReactorV2, realNodes),
  244. pexChannels: make(map[p2p.NodeID]*p2p.Channel, opts.TotalNodes),
  245. peerChans: make(map[p2p.NodeID]chan p2p.PeerUpdate, opts.TotalNodes),
  246. peerUpdates: make(map[p2p.NodeID]*p2p.PeerUpdates, opts.TotalNodes),
  247. total: opts.TotalNodes,
  248. opts: opts,
  249. }
  250. // NOTE: we don't assert that the channels get drained after stopping the
  251. // reactor
  252. rts.pexChannels = rts.network.MakeChannelsNoCleanup(
  253. t, p2p.ChannelID(pex.PexChannel), new(proto.PexMessage), chBuf,
  254. )
  255. idx := 0
  256. for nodeID := range rts.network.Nodes {
  257. rts.peerChans[nodeID] = make(chan p2p.PeerUpdate, chBuf)
  258. rts.peerUpdates[nodeID] = p2p.NewPeerUpdates(rts.peerChans[nodeID], chBuf)
  259. rts.network.Nodes[nodeID].PeerManager.Register(rts.peerUpdates[nodeID])
  260. // the first nodes in the array are always mock nodes
  261. if idx < opts.MockNodes {
  262. rts.mocks = append(rts.mocks, nodeID)
  263. } else {
  264. rts.reactors[nodeID] = pex.NewReactorV2(
  265. rts.logger.With("nodeID", nodeID),
  266. rts.network.Nodes[nodeID].PeerManager,
  267. rts.pexChannels[nodeID],
  268. rts.peerUpdates[nodeID],
  269. )
  270. }
  271. rts.nodes = append(rts.nodes, nodeID)
  272. idx++
  273. }
  274. require.Len(t, rts.reactors, realNodes)
  275. t.Cleanup(func() {
  276. for nodeID, reactor := range rts.reactors {
  277. if reactor.IsRunning() {
  278. require.NoError(t, reactor.Stop())
  279. require.False(t, reactor.IsRunning())
  280. }
  281. rts.pexChannels[nodeID].Close()
  282. rts.peerUpdates[nodeID].Close()
  283. }
  284. for _, nodeID := range rts.mocks {
  285. rts.pexChannels[nodeID].Close()
  286. rts.peerUpdates[nodeID].Close()
  287. }
  288. })
  289. return rts
  290. }
  291. // starts up the pex reactors for each node
  292. func (r *reactorTestSuite) start(t *testing.T) {
  293. t.Helper()
  294. for _, reactor := range r.reactors {
  295. require.NoError(t, reactor.Start())
  296. require.True(t, reactor.IsRunning())
  297. }
  298. }
  299. func (r *reactorTestSuite) addNodes(t *testing.T, nodes int) {
  300. t.Helper()
  301. for i := 0; i < nodes; i++ {
  302. node := r.network.MakeNode(t, p2ptest.NodeOptions{
  303. MaxPeers: r.opts.MaxPeers,
  304. MaxConnected: r.opts.MaxConnected,
  305. })
  306. r.network.Nodes[node.NodeID] = node
  307. nodeID := node.NodeID
  308. r.pexChannels[nodeID] = node.MakeChannelNoCleanup(
  309. t, p2p.ChannelID(pex.PexChannel), new(proto.PexMessage), r.opts.BufferSize,
  310. )
  311. r.peerChans[nodeID] = make(chan p2p.PeerUpdate, r.opts.BufferSize)
  312. r.peerUpdates[nodeID] = p2p.NewPeerUpdates(r.peerChans[nodeID], r.opts.BufferSize)
  313. r.network.Nodes[nodeID].PeerManager.Register(r.peerUpdates[nodeID])
  314. r.reactors[nodeID] = pex.NewReactorV2(
  315. r.logger.With("nodeID", nodeID),
  316. r.network.Nodes[nodeID].PeerManager,
  317. r.pexChannels[nodeID],
  318. r.peerUpdates[nodeID],
  319. )
  320. r.nodes = append(r.nodes, nodeID)
  321. r.total++
  322. }
  323. }
  324. func (r *reactorTestSuite) listenFor(
  325. t *testing.T,
  326. node p2p.NodeID,
  327. conditional func(msg p2p.Envelope) bool,
  328. assertion func(t *testing.T, msg p2p.Envelope) bool,
  329. waitPeriod time.Duration,
  330. ) {
  331. timesUp := time.After(waitPeriod)
  332. for {
  333. select {
  334. case envelope := <-r.pexChannels[node].In:
  335. if conditional(envelope) && assertion(t, envelope) {
  336. return
  337. }
  338. case <-timesUp:
  339. require.Fail(t, "timed out waiting for message",
  340. "node=%v, waitPeriod=%s", node, waitPeriod)
  341. }
  342. }
  343. }
  344. func (r *reactorTestSuite) listenForRequest(t *testing.T, fromNode, toNode int, waitPeriod time.Duration) {
  345. r.logger.Info("Listening for request", "from", fromNode, "to", toNode)
  346. to, from := r.checkNodePair(t, toNode, fromNode)
  347. conditional := func(msg p2p.Envelope) bool {
  348. _, ok := msg.Message.(*proto.PexRequestV2)
  349. return ok && msg.From == from
  350. }
  351. assertion := func(t *testing.T, msg p2p.Envelope) bool {
  352. require.Equal(t, &proto.PexRequestV2{}, msg.Message)
  353. return true
  354. }
  355. r.listenFor(t, to, conditional, assertion, waitPeriod)
  356. }
  357. func (r *reactorTestSuite) pingAndlistenForNAddresses(
  358. t *testing.T,
  359. fromNode, toNode int,
  360. waitPeriod time.Duration,
  361. addresses int,
  362. ) {
  363. r.logger.Info("Listening for addresses", "from", fromNode, "to", toNode)
  364. to, from := r.checkNodePair(t, toNode, fromNode)
  365. conditional := func(msg p2p.Envelope) bool {
  366. _, ok := msg.Message.(*proto.PexResponseV2)
  367. return ok && msg.From == from
  368. }
  369. assertion := func(t *testing.T, msg p2p.Envelope) bool {
  370. m, ok := msg.Message.(*proto.PexResponseV2)
  371. if !ok {
  372. require.Fail(t, "expected pex response v2")
  373. return true
  374. }
  375. // assert the same amount of addresses
  376. if len(m.Addresses) == addresses {
  377. return true
  378. }
  379. // if we didn't get the right length, we wait and send the
  380. // request again
  381. time.Sleep(300 * time.Millisecond)
  382. r.sendRequest(t, toNode, fromNode, true)
  383. return false
  384. }
  385. r.sendRequest(t, toNode, fromNode, true)
  386. r.listenFor(t, to, conditional, assertion, waitPeriod)
  387. }
  388. func (r *reactorTestSuite) listenForResponse(
  389. t *testing.T,
  390. fromNode, toNode int,
  391. waitPeriod time.Duration,
  392. addresses []proto.PexAddressV2,
  393. ) {
  394. r.logger.Info("Listening for response", "from", fromNode, "to", toNode)
  395. to, from := r.checkNodePair(t, toNode, fromNode)
  396. conditional := func(msg p2p.Envelope) bool {
  397. _, ok := msg.Message.(*proto.PexResponseV2)
  398. r.logger.Info("message", msg, "ok", ok)
  399. return ok && msg.From == from
  400. }
  401. assertion := func(t *testing.T, msg p2p.Envelope) bool {
  402. require.Equal(t, &proto.PexResponseV2{Addresses: addresses}, msg.Message)
  403. return true
  404. }
  405. r.listenFor(t, to, conditional, assertion, waitPeriod)
  406. }
  407. func (r *reactorTestSuite) listenForLegacyResponse(
  408. t *testing.T,
  409. fromNode, toNode int,
  410. waitPeriod time.Duration,
  411. addresses []proto.PexAddress,
  412. ) {
  413. r.logger.Info("Listening for response", "from", fromNode, "to", toNode)
  414. to, from := r.checkNodePair(t, toNode, fromNode)
  415. conditional := func(msg p2p.Envelope) bool {
  416. _, ok := msg.Message.(*proto.PexResponse)
  417. return ok && msg.From == from
  418. }
  419. assertion := func(t *testing.T, msg p2p.Envelope) bool {
  420. require.Equal(t, &proto.PexResponse{Addresses: addresses}, msg.Message)
  421. return true
  422. }
  423. r.listenFor(t, to, conditional, assertion, waitPeriod)
  424. }
  425. func (r *reactorTestSuite) listenForPeerUpdate(
  426. t *testing.T,
  427. onNode, withNode int,
  428. status p2p.PeerStatus,
  429. waitPeriod time.Duration,
  430. ) {
  431. on, with := r.checkNodePair(t, onNode, withNode)
  432. sub := r.network.Nodes[on].PeerManager.Subscribe()
  433. defer sub.Close()
  434. timesUp := time.After(waitPeriod)
  435. for {
  436. select {
  437. case peerUpdate := <-sub.Updates():
  438. if peerUpdate.NodeID == with {
  439. require.Equal(t, status, peerUpdate.Status)
  440. return
  441. }
  442. case <-timesUp:
  443. require.Fail(t, "timed out waiting for peer status", "%v with status %v",
  444. with, status)
  445. return
  446. }
  447. }
  448. }
  449. func (r *reactorTestSuite) getV2AddressesFor(nodes []int) []proto.PexAddressV2 {
  450. addresses := make([]proto.PexAddressV2, len(nodes))
  451. for idx, node := range nodes {
  452. nodeID := r.nodes[node]
  453. addresses[idx] = proto.PexAddressV2{
  454. URL: r.network.Nodes[nodeID].NodeAddress.String(),
  455. }
  456. }
  457. return addresses
  458. }
  459. func (r *reactorTestSuite) getAddressesFor(t *testing.T, nodes []int) []proto.PexAddress {
  460. addresses := make([]proto.PexAddress, len(nodes))
  461. for idx, node := range nodes {
  462. nodeID := r.nodes[node]
  463. nodeAddrs := r.network.Nodes[nodeID].NodeAddress
  464. endpoints, err := nodeAddrs.Resolve(context.Background())
  465. require.NoError(t, err)
  466. require.Len(t, endpoints, 1)
  467. addresses[idx] = proto.PexAddress{
  468. ID: string(nodeAddrs.NodeID),
  469. IP: endpoints[0].IP.String(),
  470. Port: uint32(endpoints[0].Port),
  471. }
  472. }
  473. return addresses
  474. }
  475. func (r *reactorTestSuite) sendRequest(t *testing.T, fromNode, toNode int, v2 bool) {
  476. to, from := r.checkNodePair(t, toNode, fromNode)
  477. if v2 {
  478. r.pexChannels[from].Out <- p2p.Envelope{
  479. To: to,
  480. Message: &proto.PexRequestV2{},
  481. }
  482. } else {
  483. r.pexChannels[from].Out <- p2p.Envelope{
  484. To: to,
  485. Message: &proto.PexRequest{},
  486. }
  487. }
  488. }
  489. func (r *reactorTestSuite) sendResponse(
  490. t *testing.T,
  491. fromNode, toNode int,
  492. withNodes []int,
  493. v2 bool,
  494. ) {
  495. from, to := r.checkNodePair(t, fromNode, toNode)
  496. if v2 {
  497. addrs := r.getV2AddressesFor(withNodes)
  498. r.pexChannels[from].Out <- p2p.Envelope{
  499. To: to,
  500. Message: &proto.PexResponseV2{
  501. Addresses: addrs,
  502. },
  503. }
  504. } else {
  505. addrs := r.getAddressesFor(t, withNodes)
  506. r.pexChannels[from].Out <- p2p.Envelope{
  507. To: to,
  508. Message: &proto.PexResponse{
  509. Addresses: addrs,
  510. },
  511. }
  512. }
  513. }
  514. func (r *reactorTestSuite) requireNumberOfPeers(
  515. t *testing.T,
  516. nodeIndex, numPeers int,
  517. waitPeriod time.Duration,
  518. ) {
  519. require.Eventuallyf(t, func() bool {
  520. actualNumPeers := len(r.network.Nodes[r.nodes[nodeIndex]].PeerManager.Peers())
  521. return actualNumPeers >= numPeers
  522. }, waitPeriod, checkFrequency, "peer failed to connect with the asserted amount of peers "+
  523. "index=%d, node=%q, waitPeriod=%s expected=%d actual=%d",
  524. nodeIndex, r.nodes[nodeIndex], waitPeriod, numPeers,
  525. len(r.network.Nodes[r.nodes[nodeIndex]].PeerManager.Peers()),
  526. )
  527. }
  528. func (r *reactorTestSuite) connectAll(t *testing.T) {
  529. r.connectN(t, r.total-1)
  530. }
  531. // connects all nodes with n other nodes
  532. func (r *reactorTestSuite) connectN(t *testing.T, n int) {
  533. if n >= r.total {
  534. require.Fail(t, "connectN: n must be less than the size of the network - 1")
  535. }
  536. for i := 0; i < r.total; i++ {
  537. for j := 0; j < n; j++ {
  538. r.connectPeers(t, i, (i+j+1)%r.total)
  539. }
  540. }
  541. }
  542. // connects node1 to node2
  543. func (r *reactorTestSuite) connectPeers(t *testing.T, sourceNode, targetNode int) {
  544. t.Helper()
  545. node1, node2 := r.checkNodePair(t, sourceNode, targetNode)
  546. r.logger.Info("connecting peers", "sourceNode", sourceNode, "targetNode", targetNode)
  547. n1 := r.network.Nodes[node1]
  548. if n1 == nil {
  549. require.Fail(t, "connectPeers: source node %v is not part of the testnet", node1)
  550. return
  551. }
  552. n2 := r.network.Nodes[node2]
  553. if n2 == nil {
  554. require.Fail(t, "connectPeers: target node %v is not part of the testnet", node2)
  555. return
  556. }
  557. sourceSub := n1.PeerManager.Subscribe()
  558. defer sourceSub.Close()
  559. targetSub := n2.PeerManager.Subscribe()
  560. defer targetSub.Close()
  561. sourceAddress := n1.NodeAddress
  562. r.logger.Debug("source address", "address", sourceAddress)
  563. targetAddress := n2.NodeAddress
  564. r.logger.Debug("target address", "address", targetAddress)
  565. added, err := n1.PeerManager.Add(targetAddress)
  566. require.NoError(t, err)
  567. if !added {
  568. r.logger.Debug("nodes already know about one another",
  569. "sourceNode", sourceNode, "targetNode", targetNode)
  570. return
  571. }
  572. select {
  573. case peerUpdate := <-targetSub.Updates():
  574. require.Equal(t, p2p.PeerUpdate{
  575. NodeID: node1,
  576. Status: p2p.PeerStatusUp,
  577. }, peerUpdate)
  578. r.logger.Debug("target connected with source")
  579. case <-time.After(time.Second):
  580. require.Fail(t, "timed out waiting for peer", "%v accepting %v",
  581. targetNode, sourceNode)
  582. }
  583. select {
  584. case peerUpdate := <-sourceSub.Updates():
  585. require.Equal(t, p2p.PeerUpdate{
  586. NodeID: node2,
  587. Status: p2p.PeerStatusUp,
  588. }, peerUpdate)
  589. r.logger.Debug("source connected with target")
  590. case <-time.After(time.Second):
  591. require.Fail(t, "timed out waiting for peer", "%v dialing %v",
  592. sourceNode, targetNode)
  593. }
  594. added, err = n2.PeerManager.Add(sourceAddress)
  595. require.NoError(t, err)
  596. require.True(t, added)
  597. }
  598. // nolint: unused
  599. func (r *reactorTestSuite) pexAddresses(t *testing.T, nodeIndices []int) []proto.PexAddress {
  600. var addresses []proto.PexAddress
  601. for _, i := range nodeIndices {
  602. if i < len(r.nodes) {
  603. require.Fail(t, "index for pex address is greater than number of nodes")
  604. }
  605. nodeAddrs := r.network.Nodes[r.nodes[i]].NodeAddress
  606. ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
  607. endpoints, err := nodeAddrs.Resolve(ctx)
  608. cancel()
  609. require.NoError(t, err)
  610. for _, endpoint := range endpoints {
  611. if endpoint.IP != nil {
  612. addresses = append(addresses, proto.PexAddress{
  613. ID: string(nodeAddrs.NodeID),
  614. IP: endpoint.IP.String(),
  615. Port: uint32(endpoint.Port),
  616. })
  617. }
  618. }
  619. }
  620. return addresses
  621. }
  622. func (r *reactorTestSuite) checkNodePair(t *testing.T, first, second int) (p2p.NodeID, p2p.NodeID) {
  623. require.NotEqual(t, first, second)
  624. require.Less(t, first, r.total)
  625. require.Less(t, second, r.total)
  626. return r.nodes[first], r.nodes[second]
  627. }
  628. func (r *reactorTestSuite) addAddresses(t *testing.T, node int, addrs []int) {
  629. peerManager := r.network.Nodes[r.nodes[node]].PeerManager
  630. for _, addr := range addrs {
  631. require.Less(t, addr, r.total)
  632. address := r.network.Nodes[r.nodes[addr]].NodeAddress
  633. added, err := peerManager.Add(address)
  634. require.NoError(t, err)
  635. require.True(t, added)
  636. }
  637. }