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.

356 lines
9.9 KiB

  1. package p2ptest
  2. import (
  3. "context"
  4. "math/rand"
  5. "testing"
  6. "time"
  7. "github.com/stretchr/testify/require"
  8. dbm "github.com/tendermint/tm-db"
  9. "github.com/tendermint/tendermint/crypto"
  10. "github.com/tendermint/tendermint/crypto/ed25519"
  11. "github.com/tendermint/tendermint/internal/p2p"
  12. "github.com/tendermint/tendermint/libs/log"
  13. "github.com/tendermint/tendermint/types"
  14. )
  15. // Network sets up an in-memory network that can be used for high-level P2P
  16. // testing. It creates an arbitrary number of nodes that are connected to each
  17. // other, and can open channels across all nodes with custom reactors.
  18. type Network struct {
  19. Nodes map[types.NodeID]*Node
  20. logger log.Logger
  21. memoryNetwork *p2p.MemoryNetwork
  22. cancel context.CancelFunc
  23. }
  24. // NetworkOptions is an argument structure to parameterize the
  25. // MakeNetwork function.
  26. type NetworkOptions struct {
  27. NumNodes int
  28. BufferSize int
  29. NodeOpts NodeOptions
  30. }
  31. type NodeOptions struct {
  32. MaxPeers uint16
  33. MaxConnected uint16
  34. }
  35. func (opts *NetworkOptions) setDefaults() {
  36. if opts.BufferSize == 0 {
  37. opts.BufferSize = 1
  38. }
  39. }
  40. // MakeNetwork creates a test network with the given number of nodes and
  41. // connects them to each other.
  42. func MakeNetwork(ctx context.Context, t *testing.T, opts NetworkOptions) *Network {
  43. opts.setDefaults()
  44. logger := log.TestingLogger()
  45. network := &Network{
  46. Nodes: map[types.NodeID]*Node{},
  47. logger: logger,
  48. memoryNetwork: p2p.NewMemoryNetwork(logger, opts.BufferSize),
  49. }
  50. for i := 0; i < opts.NumNodes; i++ {
  51. node := network.MakeNode(ctx, t, opts.NodeOpts)
  52. network.Nodes[node.NodeID] = node
  53. }
  54. return network
  55. }
  56. // Start starts the network by setting up a list of node addresses to dial in
  57. // addition to creating a peer update subscription for each node. Finally, all
  58. // nodes are connected to each other.
  59. func (n *Network) Start(ctx context.Context, t *testing.T) {
  60. ctx, n.cancel = context.WithCancel(ctx)
  61. t.Cleanup(n.cancel)
  62. // Set up a list of node addresses to dial, and a peer update subscription
  63. // for each node.
  64. dialQueue := []p2p.NodeAddress{}
  65. subs := map[types.NodeID]*p2p.PeerUpdates{}
  66. subctx, subcancel := context.WithCancel(ctx)
  67. defer subcancel()
  68. for _, node := range n.Nodes {
  69. dialQueue = append(dialQueue, node.NodeAddress)
  70. subs[node.NodeID] = node.PeerManager.Subscribe(subctx)
  71. }
  72. // For each node, dial the nodes that it still doesn't have a connection to
  73. // (either inbound or outbound), and wait for both sides to confirm the
  74. // connection via the subscriptions.
  75. for i, sourceAddress := range dialQueue {
  76. sourceNode := n.Nodes[sourceAddress.NodeID]
  77. sourceSub := subs[sourceAddress.NodeID]
  78. for _, targetAddress := range dialQueue[i+1:] { // nodes <i already connected
  79. targetNode := n.Nodes[targetAddress.NodeID]
  80. targetSub := subs[targetAddress.NodeID]
  81. added, err := sourceNode.PeerManager.Add(targetAddress)
  82. require.NoError(t, err)
  83. require.True(t, added)
  84. select {
  85. case <-ctx.Done():
  86. require.Fail(t, "operation canceled")
  87. case peerUpdate := <-sourceSub.Updates():
  88. require.Equal(t, p2p.PeerUpdate{
  89. NodeID: targetNode.NodeID,
  90. Status: p2p.PeerStatusUp,
  91. }, peerUpdate)
  92. case <-time.After(3 * time.Second):
  93. require.Fail(t, "timed out waiting for peer", "%v dialing %v",
  94. sourceNode.NodeID, targetNode.NodeID)
  95. }
  96. select {
  97. case <-ctx.Done():
  98. require.Fail(t, "operation canceled")
  99. case peerUpdate := <-targetSub.Updates():
  100. require.Equal(t, p2p.PeerUpdate{
  101. NodeID: sourceNode.NodeID,
  102. Status: p2p.PeerStatusUp,
  103. }, peerUpdate)
  104. case <-time.After(3 * time.Second):
  105. require.Fail(t, "timed out waiting for peer", "%v accepting %v",
  106. targetNode.NodeID, sourceNode.NodeID)
  107. }
  108. // Add the address to the target as well, so it's able to dial the
  109. // source back if that's even necessary.
  110. added, err = targetNode.PeerManager.Add(sourceAddress)
  111. require.NoError(t, err)
  112. require.True(t, added)
  113. }
  114. }
  115. }
  116. // NodeIDs returns the network's node IDs.
  117. func (n *Network) NodeIDs() []types.NodeID {
  118. ids := []types.NodeID{}
  119. for id := range n.Nodes {
  120. ids = append(ids, id)
  121. }
  122. return ids
  123. }
  124. // MakeChannels makes a channel on all nodes and returns them, automatically
  125. // doing error checks and cleanups.
  126. func (n *Network) MakeChannels(
  127. ctx context.Context,
  128. t *testing.T,
  129. chDesc *p2p.ChannelDescriptor,
  130. ) map[types.NodeID]*p2p.Channel {
  131. channels := map[types.NodeID]*p2p.Channel{}
  132. for _, node := range n.Nodes {
  133. channels[node.NodeID] = node.MakeChannel(ctx, t, chDesc)
  134. }
  135. return channels
  136. }
  137. // MakeChannelsNoCleanup makes a channel on all nodes and returns them,
  138. // automatically doing error checks. The caller must ensure proper cleanup of
  139. // all the channels.
  140. func (n *Network) MakeChannelsNoCleanup(
  141. ctx context.Context,
  142. t *testing.T,
  143. chDesc *p2p.ChannelDescriptor,
  144. ) map[types.NodeID]*p2p.Channel {
  145. channels := map[types.NodeID]*p2p.Channel{}
  146. for _, node := range n.Nodes {
  147. channels[node.NodeID] = node.MakeChannelNoCleanup(ctx, t, chDesc)
  148. }
  149. return channels
  150. }
  151. // RandomNode returns a random node.
  152. func (n *Network) RandomNode() *Node {
  153. nodes := make([]*Node, 0, len(n.Nodes))
  154. for _, node := range n.Nodes {
  155. nodes = append(nodes, node)
  156. }
  157. return nodes[rand.Intn(len(nodes))] // nolint:gosec
  158. }
  159. // Peers returns a node's peers (i.e. everyone except itself).
  160. func (n *Network) Peers(id types.NodeID) []*Node {
  161. peers := make([]*Node, 0, len(n.Nodes)-1)
  162. for _, peer := range n.Nodes {
  163. if peer.NodeID != id {
  164. peers = append(peers, peer)
  165. }
  166. }
  167. return peers
  168. }
  169. // Remove removes a node from the network, stopping it and waiting for all other
  170. // nodes to pick up the disconnection.
  171. func (n *Network) Remove(ctx context.Context, t *testing.T, id types.NodeID) {
  172. require.Contains(t, n.Nodes, id)
  173. node := n.Nodes[id]
  174. delete(n.Nodes, id)
  175. subs := []*p2p.PeerUpdates{}
  176. subctx, subcancel := context.WithCancel(ctx)
  177. defer subcancel()
  178. for _, peer := range n.Nodes {
  179. sub := peer.PeerManager.Subscribe(subctx)
  180. subs = append(subs, sub)
  181. }
  182. require.NoError(t, node.Transport.Close())
  183. node.cancel()
  184. if node.Router.IsRunning() {
  185. node.Router.Stop()
  186. node.Router.Wait()
  187. }
  188. for _, sub := range subs {
  189. RequireUpdate(t, sub, p2p.PeerUpdate{
  190. NodeID: node.NodeID,
  191. Status: p2p.PeerStatusDown,
  192. })
  193. }
  194. }
  195. // Node is a node in a Network, with a Router and a PeerManager.
  196. type Node struct {
  197. NodeID types.NodeID
  198. NodeInfo types.NodeInfo
  199. NodeAddress p2p.NodeAddress
  200. PrivKey crypto.PrivKey
  201. Router *p2p.Router
  202. PeerManager *p2p.PeerManager
  203. Transport *p2p.MemoryTransport
  204. cancel context.CancelFunc
  205. }
  206. // MakeNode creates a new Node configured for the network with a
  207. // running peer manager, but does not add it to the existing
  208. // network. Callers are responsible for updating peering relationships.
  209. func (n *Network) MakeNode(ctx context.Context, t *testing.T, opts NodeOptions) *Node {
  210. ctx, cancel := context.WithCancel(ctx)
  211. privKey := ed25519.GenPrivKey()
  212. nodeID := types.NodeIDFromPubKey(privKey.PubKey())
  213. nodeInfo := types.NodeInfo{
  214. NodeID: nodeID,
  215. ListenAddr: "0.0.0.0:0", // FIXME: We have to fake this for now.
  216. Moniker: string(nodeID),
  217. }
  218. transport := n.memoryNetwork.CreateTransport(nodeID)
  219. require.Len(t, transport.Endpoints(), 1, "transport not listening on 1 endpoint")
  220. peerManager, err := p2p.NewPeerManager(nodeID, dbm.NewMemDB(), p2p.PeerManagerOptions{
  221. MinRetryTime: 10 * time.Millisecond,
  222. MaxRetryTime: 100 * time.Millisecond,
  223. RetryTimeJitter: time.Millisecond,
  224. MaxPeers: opts.MaxPeers,
  225. MaxConnected: opts.MaxConnected,
  226. })
  227. require.NoError(t, err)
  228. router, err := p2p.NewRouter(
  229. ctx,
  230. n.logger,
  231. p2p.NopMetrics(),
  232. nodeInfo,
  233. privKey,
  234. peerManager,
  235. []p2p.Transport{transport},
  236. transport.Endpoints(),
  237. p2p.RouterOptions{DialSleep: func(_ context.Context) {}},
  238. )
  239. require.NoError(t, err)
  240. require.NoError(t, router.Start(ctx))
  241. t.Cleanup(func() {
  242. if router.IsRunning() {
  243. router.Stop()
  244. router.Wait()
  245. }
  246. require.NoError(t, transport.Close())
  247. cancel()
  248. })
  249. return &Node{
  250. NodeID: nodeID,
  251. NodeInfo: nodeInfo,
  252. NodeAddress: transport.Endpoints()[0].NodeAddress(nodeID),
  253. PrivKey: privKey,
  254. Router: router,
  255. PeerManager: peerManager,
  256. Transport: transport,
  257. cancel: cancel,
  258. }
  259. }
  260. // MakeChannel opens a channel, with automatic error handling and cleanup. On
  261. // test cleanup, it also checks that the channel is empty, to make sure
  262. // all expected messages have been asserted.
  263. func (n *Node) MakeChannel(
  264. ctx context.Context,
  265. t *testing.T,
  266. chDesc *p2p.ChannelDescriptor,
  267. ) *p2p.Channel {
  268. ctx, cancel := context.WithCancel(ctx)
  269. channel, err := n.Router.OpenChannel(ctx, chDesc)
  270. require.NoError(t, err)
  271. require.Contains(t, n.Router.NodeInfo().Channels, byte(chDesc.ID))
  272. t.Cleanup(func() {
  273. RequireEmpty(ctx, t, channel)
  274. cancel()
  275. })
  276. return channel
  277. }
  278. // MakeChannelNoCleanup opens a channel, with automatic error handling. The
  279. // caller must ensure proper cleanup of the channel.
  280. func (n *Node) MakeChannelNoCleanup(
  281. ctx context.Context,
  282. t *testing.T,
  283. chDesc *p2p.ChannelDescriptor,
  284. ) *p2p.Channel {
  285. channel, err := n.Router.OpenChannel(ctx, chDesc)
  286. require.NoError(t, err)
  287. return channel
  288. }
  289. // MakePeerUpdates opens a peer update subscription, with automatic cleanup.
  290. // It checks that all updates have been consumed during cleanup.
  291. func (n *Node) MakePeerUpdates(ctx context.Context, t *testing.T) *p2p.PeerUpdates {
  292. t.Helper()
  293. sub := n.PeerManager.Subscribe(ctx)
  294. t.Cleanup(func() {
  295. RequireNoUpdates(ctx, t, sub)
  296. })
  297. return sub
  298. }
  299. // MakePeerUpdatesNoRequireEmpty opens a peer update subscription, with automatic cleanup.
  300. // It does *not* check that all updates have been consumed, but will
  301. // close the update channel.
  302. func (n *Node) MakePeerUpdatesNoRequireEmpty(ctx context.Context, t *testing.T) *p2p.PeerUpdates {
  303. return n.PeerManager.Subscribe(ctx)
  304. }
  305. func MakeChannelDesc(chID p2p.ChannelID) *p2p.ChannelDescriptor {
  306. return &p2p.ChannelDescriptor{
  307. ID: chID,
  308. MessageType: &Message{},
  309. Priority: 5,
  310. SendQueueCapacity: 10,
  311. RecvMessageCapacity: 10,
  312. }
  313. }