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.

646 lines
18 KiB

  1. package p2p_test
  2. import (
  3. "context"
  4. "io"
  5. "net"
  6. "testing"
  7. "time"
  8. "github.com/fortytw2/leaktest"
  9. "github.com/stretchr/testify/assert"
  10. "github.com/stretchr/testify/require"
  11. "github.com/tendermint/tendermint/crypto/ed25519"
  12. "github.com/tendermint/tendermint/internal/p2p"
  13. "github.com/tendermint/tendermint/libs/bytes"
  14. "github.com/tendermint/tendermint/types"
  15. )
  16. // transportFactory is used to set up transports for tests.
  17. type transportFactory func(t *testing.T) p2p.Transport
  18. // testTransports is a registry of transport factories for withTransports().
  19. var testTransports = map[string]transportFactory{}
  20. // withTransports is a test helper that runs a test against all transports
  21. // registered in testTransports.
  22. func withTransports(t *testing.T, tester func(*testing.T, transportFactory)) {
  23. t.Helper()
  24. for name, transportFactory := range testTransports {
  25. transportFactory := transportFactory
  26. t.Run(name, func(t *testing.T) {
  27. t.Cleanup(leaktest.Check(t))
  28. tester(t, transportFactory)
  29. })
  30. }
  31. }
  32. func TestTransport_AcceptClose(t *testing.T) {
  33. // Just test accept unblock on close, happy path is tested widely elsewhere.
  34. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  35. a := makeTransport(t)
  36. // In-progress Accept should error on concurrent close.
  37. errCh := make(chan error, 1)
  38. go func() {
  39. time.Sleep(200 * time.Millisecond)
  40. errCh <- a.Close()
  41. }()
  42. _, err := a.Accept()
  43. require.Error(t, err)
  44. require.Equal(t, io.EOF, err)
  45. require.NoError(t, <-errCh)
  46. // Closed transport should return error immediately.
  47. _, err = a.Accept()
  48. require.Error(t, err)
  49. require.Equal(t, io.EOF, err)
  50. })
  51. }
  52. func TestTransport_DialEndpoints(t *testing.T) {
  53. ipTestCases := []struct {
  54. ip net.IP
  55. ok bool
  56. }{
  57. {net.IPv4zero, true},
  58. {net.IPv6zero, true},
  59. {nil, false},
  60. {net.IPv4bcast, false},
  61. {net.IPv4allsys, false},
  62. {[]byte{1, 2, 3}, false},
  63. {[]byte{1, 2, 3, 4, 5}, false},
  64. }
  65. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  66. a := makeTransport(t)
  67. endpoints := a.Endpoints()
  68. require.NotEmpty(t, endpoints)
  69. endpoint := endpoints[0]
  70. // Spawn a goroutine to simply accept any connections until closed.
  71. go func() {
  72. for {
  73. conn, err := a.Accept()
  74. if err != nil {
  75. return
  76. }
  77. _ = conn.Close()
  78. }
  79. }()
  80. // Dialing self should work.
  81. conn, err := a.Dial(ctx, endpoint)
  82. require.NoError(t, err)
  83. require.NoError(t, conn.Close())
  84. // Dialing empty endpoint should error.
  85. _, err = a.Dial(ctx, p2p.Endpoint{})
  86. require.Error(t, err)
  87. // Dialing without protocol should error.
  88. noProtocol := endpoint
  89. noProtocol.Protocol = ""
  90. _, err = a.Dial(ctx, noProtocol)
  91. require.Error(t, err)
  92. // Dialing with invalid protocol should error.
  93. fooProtocol := endpoint
  94. fooProtocol.Protocol = "foo"
  95. _, err = a.Dial(ctx, fooProtocol)
  96. require.Error(t, err)
  97. // Tests for networked endpoints (with IP).
  98. if len(endpoint.IP) > 0 && endpoint.Protocol != p2p.MemoryProtocol {
  99. for _, tc := range ipTestCases {
  100. tc := tc
  101. t.Run(tc.ip.String(), func(t *testing.T) {
  102. e := endpoint
  103. e.IP = tc.ip
  104. conn, err := a.Dial(ctx, e)
  105. if tc.ok {
  106. require.NoError(t, conn.Close())
  107. require.NoError(t, err)
  108. } else {
  109. require.Error(t, err, "endpoint=%s", e)
  110. }
  111. })
  112. }
  113. // Non-networked endpoints should error.
  114. noIP := endpoint
  115. noIP.IP = nil
  116. noIP.Port = 0
  117. noIP.Path = "foo"
  118. _, err := a.Dial(ctx, noIP)
  119. require.Error(t, err)
  120. } else {
  121. // Tests for non-networked endpoints (no IP).
  122. noPath := endpoint
  123. noPath.Path = ""
  124. _, err = a.Dial(ctx, noPath)
  125. require.Error(t, err)
  126. }
  127. })
  128. }
  129. func TestTransport_Dial(t *testing.T) {
  130. // Most just tests dial failures, happy path is tested widely elsewhere.
  131. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  132. a := makeTransport(t)
  133. b := makeTransport(t)
  134. require.NotEmpty(t, a.Endpoints())
  135. require.NotEmpty(t, b.Endpoints())
  136. aEndpoint := a.Endpoints()[0]
  137. bEndpoint := b.Endpoints()[0]
  138. // Context cancellation should error. We can't test timeouts since we'd
  139. // need a non-responsive endpoint.
  140. cancelCtx, cancel := context.WithCancel(ctx)
  141. cancel()
  142. _, err := a.Dial(cancelCtx, bEndpoint)
  143. require.Error(t, err)
  144. require.Equal(t, err, context.Canceled)
  145. // Unavailable endpoint should error.
  146. err = b.Close()
  147. require.NoError(t, err)
  148. _, err = a.Dial(ctx, bEndpoint)
  149. require.Error(t, err)
  150. // Dialing from a closed transport should still work.
  151. errCh := make(chan error, 1)
  152. go func() {
  153. conn, err := a.Accept()
  154. if err == nil {
  155. _ = conn.Close()
  156. }
  157. errCh <- err
  158. }()
  159. conn, err := b.Dial(ctx, aEndpoint)
  160. require.NoError(t, err)
  161. require.NoError(t, conn.Close())
  162. require.NoError(t, <-errCh)
  163. })
  164. }
  165. func TestTransport_Endpoints(t *testing.T) {
  166. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  167. a := makeTransport(t)
  168. b := makeTransport(t)
  169. // Both transports return valid and different endpoints.
  170. aEndpoints := a.Endpoints()
  171. bEndpoints := b.Endpoints()
  172. require.NotEmpty(t, aEndpoints)
  173. require.NotEmpty(t, bEndpoints)
  174. require.NotEqual(t, aEndpoints, bEndpoints)
  175. for _, endpoint := range append(aEndpoints, bEndpoints...) {
  176. err := endpoint.Validate()
  177. require.NoError(t, err, "invalid endpoint %q", endpoint)
  178. }
  179. // When closed, the transport should no longer return any endpoints.
  180. err := a.Close()
  181. require.NoError(t, err)
  182. require.Empty(t, a.Endpoints())
  183. require.NotEmpty(t, b.Endpoints())
  184. })
  185. }
  186. func TestTransport_Protocols(t *testing.T) {
  187. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  188. a := makeTransport(t)
  189. protocols := a.Protocols()
  190. endpoints := a.Endpoints()
  191. require.NotEmpty(t, protocols)
  192. require.NotEmpty(t, endpoints)
  193. for _, endpoint := range endpoints {
  194. require.Contains(t, protocols, endpoint.Protocol)
  195. }
  196. })
  197. }
  198. func TestTransport_String(t *testing.T) {
  199. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  200. a := makeTransport(t)
  201. require.NotEmpty(t, a.String())
  202. })
  203. }
  204. func TestConnection_Handshake(t *testing.T) {
  205. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  206. a := makeTransport(t)
  207. b := makeTransport(t)
  208. ab, ba := dialAccept(t, a, b)
  209. // A handshake should pass the given keys and NodeInfo.
  210. aKey := ed25519.GenPrivKey()
  211. aInfo := types.NodeInfo{
  212. NodeID: types.NodeIDFromPubKey(aKey.PubKey()),
  213. ProtocolVersion: types.ProtocolVersion{
  214. P2P: 1,
  215. Block: 2,
  216. App: 3,
  217. },
  218. ListenAddr: "listenaddr",
  219. Network: "network",
  220. Version: "1.2.3",
  221. Channels: bytes.HexBytes([]byte{0xf0, 0x0f}),
  222. Moniker: "moniker",
  223. Other: types.NodeInfoOther{
  224. TxIndex: "txindex",
  225. RPCAddress: "rpc.domain.com",
  226. },
  227. }
  228. bKey := ed25519.GenPrivKey()
  229. bInfo := types.NodeInfo{NodeID: types.NodeIDFromPubKey(bKey.PubKey())}
  230. errCh := make(chan error, 1)
  231. go func() {
  232. // Must use assert due to goroutine.
  233. peerInfo, peerKey, err := ba.Handshake(ctx, bInfo, bKey)
  234. if err == nil {
  235. assert.Equal(t, aInfo, peerInfo)
  236. assert.Equal(t, aKey.PubKey(), peerKey)
  237. }
  238. errCh <- err
  239. }()
  240. peerInfo, peerKey, err := ab.Handshake(ctx, aInfo, aKey)
  241. require.NoError(t, err)
  242. require.Equal(t, bInfo, peerInfo)
  243. require.Equal(t, bKey.PubKey(), peerKey)
  244. require.NoError(t, <-errCh)
  245. })
  246. }
  247. func TestConnection_HandshakeCancel(t *testing.T) {
  248. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  249. a := makeTransport(t)
  250. b := makeTransport(t)
  251. // Handshake should error on context cancellation.
  252. ab, ba := dialAccept(t, a, b)
  253. timeoutCtx, cancel := context.WithTimeout(ctx, 1*time.Minute)
  254. cancel()
  255. _, _, err := ab.Handshake(timeoutCtx, types.NodeInfo{}, ed25519.GenPrivKey())
  256. require.Error(t, err)
  257. require.Equal(t, context.Canceled, err)
  258. _ = ab.Close()
  259. _ = ba.Close()
  260. // Handshake should error on context timeout.
  261. ab, ba = dialAccept(t, a, b)
  262. timeoutCtx, cancel = context.WithTimeout(ctx, 200*time.Millisecond)
  263. defer cancel()
  264. _, _, err = ab.Handshake(timeoutCtx, types.NodeInfo{}, ed25519.GenPrivKey())
  265. require.Error(t, err)
  266. require.Equal(t, context.DeadlineExceeded, err)
  267. _ = ab.Close()
  268. _ = ba.Close()
  269. })
  270. }
  271. func TestConnection_FlushClose(t *testing.T) {
  272. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  273. a := makeTransport(t)
  274. b := makeTransport(t)
  275. ab, _ := dialAcceptHandshake(t, a, b)
  276. // FIXME: FlushClose should be removed (and replaced by separate Flush
  277. // and Close calls if necessary). We can't reliably test it, so we just
  278. // make sure it closes both ends and that it's idempotent.
  279. err := ab.FlushClose()
  280. require.NoError(t, err)
  281. _, _, err = ab.ReceiveMessage()
  282. require.Error(t, err)
  283. require.Equal(t, io.EOF, err)
  284. _, err = ab.SendMessage(chID, []byte("closed"))
  285. require.Error(t, err)
  286. require.Equal(t, io.EOF, err)
  287. err = ab.FlushClose()
  288. require.NoError(t, err)
  289. })
  290. }
  291. func TestConnection_LocalRemoteEndpoint(t *testing.T) {
  292. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  293. a := makeTransport(t)
  294. b := makeTransport(t)
  295. ab, ba := dialAcceptHandshake(t, a, b)
  296. // Local and remote connection endpoints correspond to each other.
  297. require.NotEmpty(t, ab.LocalEndpoint())
  298. require.NotEmpty(t, ba.LocalEndpoint())
  299. require.Equal(t, ab.LocalEndpoint(), ba.RemoteEndpoint())
  300. require.Equal(t, ab.RemoteEndpoint(), ba.LocalEndpoint())
  301. })
  302. }
  303. func TestConnection_SendReceive(t *testing.T) {
  304. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  305. a := makeTransport(t)
  306. b := makeTransport(t)
  307. ab, ba := dialAcceptHandshake(t, a, b)
  308. // Can send and receive a to b.
  309. ok, err := ab.SendMessage(chID, []byte("foo"))
  310. require.NoError(t, err)
  311. require.True(t, ok)
  312. ch, msg, err := ba.ReceiveMessage()
  313. require.NoError(t, err)
  314. require.Equal(t, []byte("foo"), msg)
  315. require.Equal(t, chID, ch)
  316. // Can send and receive b to a.
  317. _, err = ba.SendMessage(chID, []byte("bar"))
  318. require.NoError(t, err)
  319. _, msg, err = ab.ReceiveMessage()
  320. require.NoError(t, err)
  321. require.Equal(t, []byte("bar"), msg)
  322. // TrySendMessage also works.
  323. ok, err = ba.TrySendMessage(chID, []byte("try"))
  324. require.NoError(t, err)
  325. require.True(t, ok)
  326. ch, msg, err = ab.ReceiveMessage()
  327. require.NoError(t, err)
  328. require.Equal(t, []byte("try"), msg)
  329. require.Equal(t, chID, ch)
  330. // Connections should still be active after closing the transports.
  331. err = a.Close()
  332. require.NoError(t, err)
  333. err = b.Close()
  334. require.NoError(t, err)
  335. _, err = ab.SendMessage(chID, []byte("still here"))
  336. require.NoError(t, err)
  337. ch, msg, err = ba.ReceiveMessage()
  338. require.NoError(t, err)
  339. require.Equal(t, chID, ch)
  340. require.Equal(t, []byte("still here"), msg)
  341. // Close one side of the connection. Both sides should then error
  342. // with io.EOF when trying to send or receive.
  343. err = ba.Close()
  344. require.NoError(t, err)
  345. _, _, err = ab.ReceiveMessage()
  346. require.Error(t, err)
  347. require.Equal(t, io.EOF, err)
  348. _, err = ab.SendMessage(chID, []byte("closed"))
  349. require.Error(t, err)
  350. require.Equal(t, io.EOF, err)
  351. _, _, err = ba.ReceiveMessage()
  352. require.Error(t, err)
  353. require.Equal(t, io.EOF, err)
  354. _, err = ba.SendMessage(chID, []byte("closed"))
  355. require.Error(t, err)
  356. require.Equal(t, io.EOF, err)
  357. })
  358. }
  359. func TestConnection_Status(t *testing.T) {
  360. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  361. a := makeTransport(t)
  362. b := makeTransport(t)
  363. ab, _ := dialAcceptHandshake(t, a, b)
  364. // FIXME: This isn't implemented in all transports, so for now we just
  365. // check that it doesn't panic, which isn't really much of a test.
  366. ab.Status()
  367. })
  368. }
  369. func TestConnection_String(t *testing.T) {
  370. withTransports(t, func(t *testing.T, makeTransport transportFactory) {
  371. a := makeTransport(t)
  372. b := makeTransport(t)
  373. ab, _ := dialAccept(t, a, b)
  374. require.NotEmpty(t, ab.String())
  375. })
  376. }
  377. func TestEndpoint_NodeAddress(t *testing.T) {
  378. var (
  379. ip4 = []byte{1, 2, 3, 4}
  380. ip4in6 = net.IPv4(1, 2, 3, 4)
  381. ip6 = []byte{0xb1, 0x0c, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0x01}
  382. id = types.NodeID("00112233445566778899aabbccddeeff00112233")
  383. )
  384. testcases := []struct {
  385. endpoint p2p.Endpoint
  386. expect p2p.NodeAddress
  387. }{
  388. // Valid endpoints.
  389. {
  390. p2p.Endpoint{Protocol: "tcp", IP: ip4, Port: 8080, Path: "path"},
  391. p2p.NodeAddress{Protocol: "tcp", Hostname: "1.2.3.4", Port: 8080, Path: "path"},
  392. },
  393. {
  394. p2p.Endpoint{Protocol: "tcp", IP: ip4in6, Port: 8080, Path: "path"},
  395. p2p.NodeAddress{Protocol: "tcp", Hostname: "1.2.3.4", Port: 8080, Path: "path"},
  396. },
  397. {
  398. p2p.Endpoint{Protocol: "tcp", IP: ip6, Port: 8080, Path: "path"},
  399. p2p.NodeAddress{Protocol: "tcp", Hostname: "b10c::1", Port: 8080, Path: "path"},
  400. },
  401. {
  402. p2p.Endpoint{Protocol: "memory", Path: "foo"},
  403. p2p.NodeAddress{Protocol: "memory", Path: "foo"},
  404. },
  405. {
  406. p2p.Endpoint{Protocol: "memory", Path: string(id)},
  407. p2p.NodeAddress{Protocol: "memory", Path: string(id)},
  408. },
  409. // Partial (invalid) endpoints.
  410. {p2p.Endpoint{}, p2p.NodeAddress{}},
  411. {p2p.Endpoint{Protocol: "tcp"}, p2p.NodeAddress{Protocol: "tcp"}},
  412. {p2p.Endpoint{IP: net.IPv4(1, 2, 3, 4)}, p2p.NodeAddress{Hostname: "1.2.3.4"}},
  413. {p2p.Endpoint{Port: 8080}, p2p.NodeAddress{}},
  414. {p2p.Endpoint{Path: "path"}, p2p.NodeAddress{Path: "path"}},
  415. }
  416. for _, tc := range testcases {
  417. tc := tc
  418. t.Run(tc.endpoint.String(), func(t *testing.T) {
  419. // Without NodeID.
  420. expect := tc.expect
  421. require.Equal(t, expect, tc.endpoint.NodeAddress(""))
  422. // With NodeID.
  423. expect.NodeID = id
  424. require.Equal(t, expect, tc.endpoint.NodeAddress(expect.NodeID))
  425. })
  426. }
  427. }
  428. func TestEndpoint_String(t *testing.T) {
  429. var (
  430. ip4 = []byte{1, 2, 3, 4}
  431. ip4in6 = net.IPv4(1, 2, 3, 4)
  432. ip6 = []byte{0xb1, 0x0c, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0x01}
  433. nodeID = types.NodeID("00112233445566778899aabbccddeeff00112233")
  434. )
  435. testcases := []struct {
  436. endpoint p2p.Endpoint
  437. expect string
  438. }{
  439. // Non-networked endpoints.
  440. {p2p.Endpoint{Protocol: "memory", Path: string(nodeID)}, "memory:" + string(nodeID)},
  441. {p2p.Endpoint{Protocol: "file", Path: "foo"}, "file:///foo"},
  442. {p2p.Endpoint{Protocol: "file", Path: "👋"}, "file:///%F0%9F%91%8B"},
  443. // IPv4 endpoints.
  444. {p2p.Endpoint{Protocol: "tcp", IP: ip4}, "tcp://1.2.3.4"},
  445. {p2p.Endpoint{Protocol: "tcp", IP: ip4in6}, "tcp://1.2.3.4"},
  446. {p2p.Endpoint{Protocol: "tcp", IP: ip4, Port: 8080}, "tcp://1.2.3.4:8080"},
  447. {p2p.Endpoint{Protocol: "tcp", IP: ip4, Port: 8080, Path: "/path"}, "tcp://1.2.3.4:8080/path"},
  448. {p2p.Endpoint{Protocol: "tcp", IP: ip4, Path: "path/👋"}, "tcp://1.2.3.4/path/%F0%9F%91%8B"},
  449. // IPv6 endpoints.
  450. {p2p.Endpoint{Protocol: "tcp", IP: ip6}, "tcp://b10c::1"},
  451. {p2p.Endpoint{Protocol: "tcp", IP: ip6, Port: 8080}, "tcp://[b10c::1]:8080"},
  452. {p2p.Endpoint{Protocol: "tcp", IP: ip6, Port: 8080, Path: "/path"}, "tcp://[b10c::1]:8080/path"},
  453. {p2p.Endpoint{Protocol: "tcp", IP: ip6, Path: "path/👋"}, "tcp://b10c::1/path/%F0%9F%91%8B"},
  454. // Partial (invalid) endpoints.
  455. {p2p.Endpoint{}, ""},
  456. {p2p.Endpoint{Protocol: "tcp"}, "tcp:"},
  457. {p2p.Endpoint{IP: []byte{1, 2, 3, 4}}, "1.2.3.4"},
  458. {p2p.Endpoint{IP: []byte{0xb1, 0x0c, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0x01}}, "b10c::1"},
  459. {p2p.Endpoint{Port: 8080}, ""},
  460. {p2p.Endpoint{Path: "foo"}, "/foo"},
  461. }
  462. for _, tc := range testcases {
  463. tc := tc
  464. t.Run(tc.expect, func(t *testing.T) {
  465. require.Equal(t, tc.expect, tc.endpoint.String())
  466. })
  467. }
  468. }
  469. func TestEndpoint_Validate(t *testing.T) {
  470. var (
  471. ip4 = []byte{1, 2, 3, 4}
  472. ip4in6 = net.IPv4(1, 2, 3, 4)
  473. ip6 = []byte{0xb1, 0x0c, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0x01}
  474. )
  475. testcases := []struct {
  476. endpoint p2p.Endpoint
  477. expectValid bool
  478. }{
  479. // Valid endpoints.
  480. {p2p.Endpoint{Protocol: "tcp", IP: ip4}, true},
  481. {p2p.Endpoint{Protocol: "tcp", IP: ip4in6}, true},
  482. {p2p.Endpoint{Protocol: "tcp", IP: ip6}, true},
  483. {p2p.Endpoint{Protocol: "tcp", IP: ip4, Port: 8008}, true},
  484. {p2p.Endpoint{Protocol: "tcp", IP: ip4, Port: 8080, Path: "path"}, true},
  485. {p2p.Endpoint{Protocol: "memory", Path: "path"}, true},
  486. // Invalid endpoints.
  487. {p2p.Endpoint{}, false},
  488. {p2p.Endpoint{IP: ip4}, false},
  489. {p2p.Endpoint{Protocol: "tcp"}, false},
  490. {p2p.Endpoint{Protocol: "tcp", IP: []byte{1, 2, 3}}, false},
  491. {p2p.Endpoint{Protocol: "tcp", Port: 8080, Path: "path"}, false},
  492. }
  493. for _, tc := range testcases {
  494. tc := tc
  495. t.Run(tc.endpoint.String(), func(t *testing.T) {
  496. err := tc.endpoint.Validate()
  497. if tc.expectValid {
  498. require.NoError(t, err)
  499. } else {
  500. require.Error(t, err)
  501. }
  502. })
  503. }
  504. }
  505. // dialAccept is a helper that dials b from a and returns both sides of the
  506. // connection.
  507. func dialAccept(t *testing.T, a, b p2p.Transport) (p2p.Connection, p2p.Connection) {
  508. t.Helper()
  509. endpoints := b.Endpoints()
  510. require.NotEmpty(t, endpoints, "peer not listening on any endpoints")
  511. ctx, cancel := context.WithTimeout(ctx, time.Second)
  512. defer cancel()
  513. acceptCh := make(chan p2p.Connection, 1)
  514. errCh := make(chan error, 1)
  515. go func() {
  516. conn, err := b.Accept()
  517. errCh <- err
  518. acceptCh <- conn
  519. }()
  520. dialConn, err := a.Dial(ctx, endpoints[0])
  521. require.NoError(t, err)
  522. acceptConn := <-acceptCh
  523. require.NoError(t, <-errCh)
  524. t.Cleanup(func() {
  525. _ = dialConn.Close()
  526. _ = acceptConn.Close()
  527. })
  528. return dialConn, acceptConn
  529. }
  530. // dialAcceptHandshake is a helper that dials and handshakes b from a and
  531. // returns both sides of the connection.
  532. func dialAcceptHandshake(t *testing.T, a, b p2p.Transport) (p2p.Connection, p2p.Connection) {
  533. t.Helper()
  534. ab, ba := dialAccept(t, a, b)
  535. ctx, cancel := context.WithTimeout(ctx, time.Second)
  536. defer cancel()
  537. errCh := make(chan error, 1)
  538. go func() {
  539. privKey := ed25519.GenPrivKey()
  540. nodeInfo := types.NodeInfo{NodeID: types.NodeIDFromPubKey(privKey.PubKey())}
  541. _, _, err := ba.Handshake(ctx, nodeInfo, privKey)
  542. errCh <- err
  543. }()
  544. privKey := ed25519.GenPrivKey()
  545. nodeInfo := types.NodeInfo{NodeID: types.NodeIDFromPubKey(privKey.PubKey())}
  546. _, _, err := ab.Handshake(ctx, nodeInfo, privKey)
  547. require.NoError(t, err)
  548. timer := time.NewTimer(2 * time.Second)
  549. defer timer.Stop()
  550. select {
  551. case err := <-errCh:
  552. require.NoError(t, err)
  553. case <-timer.C:
  554. require.Fail(t, "handshake timed out")
  555. }
  556. return ab, ba
  557. }