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.

1102 lines
31 KiB

p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
p2p: make PeerManager.DialNext() and EvictNext() block (#5947) See #5936 and #5938 for background. The plan was initially to have `DialNext()` and `EvictNext()` return a channel. However, implementing this became unnecessarily complicated and error-prone. As an example, the channel would be both consumed and populated (via method calls) by the same driving method (e.g. `Router.dialPeers()`) which could easily cause deadlocks where a method call blocked while sending on the channel that the caller itself was responsible for consuming (but couldn't since it was busy making the method call). It would also require a set of goroutines in the peer manager that would interact with the goroutines in the router in non-obvious ways, and fully populating the channel on startup could cause deadlocks with other startup tasks. Several issues like these made the solution hard to reason about. I therefore simply made `DialNext()` and `EvictNext()` block until the next peer was available, using internal triggers to wake these methods up in a non-blocking fashion when any relevant state changes occurred. This proved much simpler to reason about, since there are no goroutines in the peer manager (except for trivial retry timers), nor any blocking channel sends, and it instead relies entirely on the existing goroutine structure of the router for concurrency. This also happens to be the same pattern used by the `Transport.Accept()` API, following Go stdlib conventions, so all router goroutines end up using a consistent pattern as well.
4 years ago
  1. package p2p
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "io"
  7. "math/rand"
  8. "net"
  9. "runtime"
  10. "sync"
  11. "time"
  12. "github.com/gogo/protobuf/proto"
  13. "github.com/tendermint/tendermint/crypto"
  14. "github.com/tendermint/tendermint/libs/log"
  15. "github.com/tendermint/tendermint/libs/service"
  16. "github.com/tendermint/tendermint/types"
  17. )
  18. const queueBufferDefault = 32
  19. // ChannelID is an arbitrary channel ID.
  20. type ChannelID uint16
  21. // Envelope contains a message with sender/receiver routing info.
  22. type Envelope struct {
  23. From types.NodeID // sender (empty if outbound)
  24. To types.NodeID // receiver (empty if inbound)
  25. Broadcast bool // send to all connected peers (ignores To)
  26. Message proto.Message // message payload
  27. // channelID is for internal Router use, set on outbound messages to inform
  28. // the sendPeer() goroutine which transport channel to use.
  29. //
  30. // FIXME: If we migrate the Transport API to a byte-oriented multi-stream
  31. // API, this will no longer be necessary since each channel will be mapped
  32. // onto a stream during channel/peer setup. See:
  33. // https://github.com/tendermint/spec/pull/227
  34. channelID ChannelID
  35. }
  36. // PeerError is a peer error reported via Channel.Error.
  37. //
  38. // FIXME: This currently just disconnects the peer, which is too simplistic.
  39. // For example, some errors should be logged, some should cause disconnects,
  40. // and some should ban the peer.
  41. //
  42. // FIXME: This should probably be replaced by a more general PeerBehavior
  43. // concept that can mark good and bad behavior and contributes to peer scoring.
  44. // It should possibly also allow reactors to request explicit actions, e.g.
  45. // disconnection or banning, in addition to doing this based on aggregates.
  46. type PeerError struct {
  47. NodeID types.NodeID
  48. Err error
  49. }
  50. // Channel is a bidirectional channel to exchange Protobuf messages with peers,
  51. // wrapped in Envelope to specify routing info (i.e. sender/receiver).
  52. type Channel struct {
  53. ID ChannelID
  54. In <-chan Envelope // inbound messages (peers to reactors)
  55. Out chan<- Envelope // outbound messages (reactors to peers)
  56. Error chan<- PeerError // peer error reporting
  57. messageType proto.Message // the channel's message type, used for unmarshaling
  58. closeCh chan struct{}
  59. closeOnce sync.Once
  60. }
  61. // NewChannel creates a new channel. It is primarily for internal and test
  62. // use, reactors should use Router.OpenChannel().
  63. func NewChannel(
  64. id ChannelID,
  65. messageType proto.Message,
  66. inCh <-chan Envelope,
  67. outCh chan<- Envelope,
  68. errCh chan<- PeerError,
  69. ) *Channel {
  70. return &Channel{
  71. ID: id,
  72. messageType: messageType,
  73. In: inCh,
  74. Out: outCh,
  75. Error: errCh,
  76. closeCh: make(chan struct{}),
  77. }
  78. }
  79. // Close closes the channel. Future sends on Out and Error will panic. The In
  80. // channel remains open to avoid having to synchronize Router senders, which
  81. // should use Done() to detect channel closure instead.
  82. func (c *Channel) Close() {
  83. c.closeOnce.Do(func() {
  84. close(c.closeCh)
  85. close(c.Out)
  86. close(c.Error)
  87. })
  88. }
  89. // Done returns a channel that's closed when Channel.Close() is called.
  90. func (c *Channel) Done() <-chan struct{} {
  91. return c.closeCh
  92. }
  93. // Wrapper is a Protobuf message that can contain a variety of inner messages
  94. // (e.g. via oneof fields). If a Channel's message type implements Wrapper, the
  95. // Router will automatically wrap outbound messages and unwrap inbound messages,
  96. // such that reactors do not have to do this themselves.
  97. type Wrapper interface {
  98. proto.Message
  99. // Wrap will take a message and wrap it in this one if possible.
  100. Wrap(proto.Message) error
  101. // Unwrap will unwrap the inner message contained in this message.
  102. Unwrap() (proto.Message, error)
  103. }
  104. // RouterOptions specifies options for a Router.
  105. type RouterOptions struct {
  106. // ResolveTimeout is the timeout for resolving NodeAddress URLs.
  107. // 0 means no timeout.
  108. ResolveTimeout time.Duration
  109. // DialTimeout is the timeout for dialing a peer. 0 means no timeout.
  110. DialTimeout time.Duration
  111. // HandshakeTimeout is the timeout for handshaking with a peer. 0 means
  112. // no timeout.
  113. HandshakeTimeout time.Duration
  114. // QueueType must be "wdrr" (Weighed Deficit Round Robin), "priority", or
  115. // "fifo". Defaults to "fifo".
  116. QueueType string
  117. // MaxIncomingConnectionAttempts rate limits the number of incoming connection
  118. // attempts per IP address. Defaults to 100.
  119. MaxIncomingConnectionAttempts uint
  120. // IncomingConnectionWindow describes how often an IP address
  121. // can attempt to create a new connection. Defaults to 10
  122. // milliseconds, and cannot be less than 1 millisecond.
  123. IncomingConnectionWindow time.Duration
  124. // FilterPeerByIP is used by the router to inject filtering
  125. // behavior for new incoming connections. The router passes
  126. // the remote IP of the incoming connection the port number as
  127. // arguments. Functions should return an error to reject the
  128. // peer.
  129. FilterPeerByIP func(context.Context, net.IP, uint16) error
  130. // FilterPeerByID is used by the router to inject filtering
  131. // behavior for new incoming connections. The router passes
  132. // the NodeID of the node before completing the connection,
  133. // but this occurs after the handshake is complete. Filter by
  134. // IP address to filter before the handshake. Functions should
  135. // return an error to reject the peer.
  136. FilterPeerByID func(context.Context, types.NodeID) error
  137. // DialSleep controls the amount of time that the router
  138. // sleeps between dialing peers. If not set, a default value
  139. // is used that sleeps for a (random) amount of time up to 3
  140. // seconds between submitting each peer to be dialed.
  141. DialSleep func(context.Context)
  142. // NumConcrruentDials controls how many parallel go routines
  143. // are used to dial peers. This defaults to the value of
  144. // runtime.NumCPU.
  145. NumConcurrentDials func() int
  146. }
  147. const (
  148. queueTypeFifo = "fifo"
  149. queueTypePriority = "priority"
  150. queueTypeWDRR = "wdrr"
  151. )
  152. // Validate validates router options.
  153. func (o *RouterOptions) Validate() error {
  154. switch o.QueueType {
  155. case "":
  156. o.QueueType = queueTypeFifo
  157. case queueTypeFifo, queueTypeWDRR, queueTypePriority:
  158. // passI me
  159. default:
  160. return fmt.Errorf("queue type %q is not supported", o.QueueType)
  161. }
  162. switch {
  163. case o.IncomingConnectionWindow == 0:
  164. o.IncomingConnectionWindow = 100 * time.Millisecond
  165. case o.IncomingConnectionWindow < time.Millisecond:
  166. return fmt.Errorf("incomming connection window must be grater than 1m [%s]",
  167. o.IncomingConnectionWindow)
  168. }
  169. if o.MaxIncomingConnectionAttempts == 0 {
  170. o.MaxIncomingConnectionAttempts = 100
  171. }
  172. return nil
  173. }
  174. // Router manages peer connections and routes messages between peers and reactor
  175. // channels. It takes a PeerManager for peer lifecycle management (e.g. which
  176. // peers to dial and when) and a set of Transports for connecting and
  177. // communicating with peers.
  178. //
  179. // On startup, three main goroutines are spawned to maintain peer connections:
  180. //
  181. // dialPeers(): in a loop, calls PeerManager.DialNext() to get the next peer
  182. // address to dial and spawns a goroutine that dials the peer, handshakes
  183. // with it, and begins to route messages if successful.
  184. //
  185. // acceptPeers(): in a loop, waits for an inbound connection via
  186. // Transport.Accept() and spawns a goroutine that handshakes with it and
  187. // begins to route messages if successful.
  188. //
  189. // evictPeers(): in a loop, calls PeerManager.EvictNext() to get the next
  190. // peer to evict, and disconnects it by closing its message queue.
  191. //
  192. // When a peer is connected, an outbound peer message queue is registered in
  193. // peerQueues, and routePeer() is called to spawn off two additional goroutines:
  194. //
  195. // sendPeer(): waits for an outbound message from the peerQueues queue,
  196. // marshals it, and passes it to the peer transport which delivers it.
  197. //
  198. // receivePeer(): waits for an inbound message from the peer transport,
  199. // unmarshals it, and passes it to the appropriate inbound channel queue
  200. // in channelQueues.
  201. //
  202. // When a reactor opens a channel via OpenChannel, an inbound channel message
  203. // queue is registered in channelQueues, and a channel goroutine is spawned:
  204. //
  205. // routeChannel(): waits for an outbound message from the channel, looks
  206. // up the recipient peer's outbound message queue in peerQueues, and submits
  207. // the message to it.
  208. //
  209. // All channel sends in the router are blocking. It is the responsibility of the
  210. // queue interface in peerQueues and channelQueues to prioritize and drop
  211. // messages as appropriate during contention to prevent stalls and ensure good
  212. // quality of service.
  213. type Router struct {
  214. *service.BaseService
  215. logger log.Logger
  216. metrics *Metrics
  217. options RouterOptions
  218. nodeInfo types.NodeInfo
  219. privKey crypto.PrivKey
  220. peerManager *PeerManager
  221. chDescs []ChannelDescriptor
  222. transports []Transport
  223. connTracker connectionTracker
  224. protocolTransports map[Protocol]Transport
  225. stopCh chan struct{} // signals Router shutdown
  226. peerMtx sync.RWMutex
  227. peerQueues map[types.NodeID]queue // outbound messages per peer for all channels
  228. // the channels that the peer queue has open
  229. peerChannels map[types.NodeID]channelIDs
  230. queueFactory func(int) queue
  231. // FIXME: We don't strictly need to use a mutex for this if we seal the
  232. // channels on router start. This depends on whether we want to allow
  233. // dynamic channels in the future.
  234. channelMtx sync.RWMutex
  235. channelQueues map[ChannelID]queue // inbound messages from all peers to a single channel
  236. channelMessages map[ChannelID]proto.Message
  237. }
  238. // NewRouter creates a new Router. The given Transports must already be
  239. // listening on appropriate interfaces, and will be closed by the Router when it
  240. // stops.
  241. func NewRouter(
  242. logger log.Logger,
  243. metrics *Metrics,
  244. nodeInfo types.NodeInfo,
  245. privKey crypto.PrivKey,
  246. peerManager *PeerManager,
  247. transports []Transport,
  248. options RouterOptions,
  249. ) (*Router, error) {
  250. if err := options.Validate(); err != nil {
  251. return nil, err
  252. }
  253. router := &Router{
  254. logger: logger,
  255. metrics: metrics,
  256. nodeInfo: nodeInfo,
  257. privKey: privKey,
  258. connTracker: newConnTracker(
  259. options.MaxIncomingConnectionAttempts,
  260. options.IncomingConnectionWindow,
  261. ),
  262. chDescs: make([]ChannelDescriptor, 0),
  263. transports: transports,
  264. protocolTransports: map[Protocol]Transport{},
  265. peerManager: peerManager,
  266. options: options,
  267. stopCh: make(chan struct{}),
  268. channelQueues: map[ChannelID]queue{},
  269. channelMessages: map[ChannelID]proto.Message{},
  270. peerQueues: map[types.NodeID]queue{},
  271. peerChannels: make(map[types.NodeID]channelIDs),
  272. }
  273. router.BaseService = service.NewBaseService(logger, "router", router)
  274. qf, err := router.createQueueFactory()
  275. if err != nil {
  276. return nil, err
  277. }
  278. router.queueFactory = qf
  279. for _, transport := range transports {
  280. for _, protocol := range transport.Protocols() {
  281. if _, ok := router.protocolTransports[protocol]; !ok {
  282. router.protocolTransports[protocol] = transport
  283. }
  284. }
  285. }
  286. return router, nil
  287. }
  288. func (r *Router) createQueueFactory() (func(int) queue, error) {
  289. switch r.options.QueueType {
  290. case queueTypeFifo:
  291. return newFIFOQueue, nil
  292. case queueTypePriority:
  293. return func(size int) queue {
  294. if size%2 != 0 {
  295. size++
  296. }
  297. q := newPQScheduler(r.logger, r.metrics, r.chDescs, uint(size)/2, uint(size)/2, defaultCapacity)
  298. q.start()
  299. return q
  300. }, nil
  301. case queueTypeWDRR:
  302. return func(size int) queue {
  303. if size%2 != 0 {
  304. size++
  305. }
  306. q := newWDRRScheduler(r.logger, r.metrics, r.chDescs, uint(size)/2, uint(size)/2, defaultCapacity)
  307. q.start()
  308. return q
  309. }, nil
  310. default:
  311. return nil, fmt.Errorf("cannot construct queue of type %q", r.options.QueueType)
  312. }
  313. }
  314. // OpenChannel opens a new channel for the given message type. The caller must
  315. // close the channel when done, before stopping the Router. messageType is the
  316. // type of message passed through the channel (used for unmarshaling), which can
  317. // implement Wrapper to automatically (un)wrap multiple message types in a
  318. // wrapper message. The caller may provide a size to make the channel buffered,
  319. // which internally makes the inbound, outbound, and error channel buffered.
  320. func (r *Router) OpenChannel(chDesc ChannelDescriptor, messageType proto.Message, size int) (*Channel, error) {
  321. r.channelMtx.Lock()
  322. defer r.channelMtx.Unlock()
  323. id := ChannelID(chDesc.ID)
  324. if _, ok := r.channelQueues[id]; ok {
  325. return nil, fmt.Errorf("channel %v already exists", id)
  326. }
  327. r.chDescs = append(r.chDescs, chDesc)
  328. queue := r.queueFactory(size)
  329. outCh := make(chan Envelope, size)
  330. errCh := make(chan PeerError, size)
  331. channel := NewChannel(id, messageType, queue.dequeue(), outCh, errCh)
  332. var wrapper Wrapper
  333. if w, ok := messageType.(Wrapper); ok {
  334. wrapper = w
  335. }
  336. r.channelQueues[id] = queue
  337. r.channelMessages[id] = messageType
  338. // add the channel to the nodeInfo if it's not already there.
  339. r.nodeInfo.AddChannel(uint16(chDesc.ID))
  340. go func() {
  341. defer func() {
  342. r.channelMtx.Lock()
  343. delete(r.channelQueues, id)
  344. delete(r.channelMessages, id)
  345. r.channelMtx.Unlock()
  346. queue.close()
  347. }()
  348. r.routeChannel(id, outCh, errCh, wrapper)
  349. }()
  350. return channel, nil
  351. }
  352. // routeChannel receives outbound channel messages and routes them to the
  353. // appropriate peer. It also receives peer errors and reports them to the peer
  354. // manager. It returns when either the outbound channel or error channel is
  355. // closed, or the Router is stopped. wrapper is an optional message wrapper
  356. // for messages, see Wrapper for details.
  357. func (r *Router) routeChannel(
  358. chID ChannelID,
  359. outCh <-chan Envelope,
  360. errCh <-chan PeerError,
  361. wrapper Wrapper,
  362. ) {
  363. for {
  364. select {
  365. case envelope, ok := <-outCh:
  366. if !ok {
  367. return
  368. }
  369. // Mark the envelope with the channel ID to allow sendPeer() to pass
  370. // it on to Transport.SendMessage().
  371. envelope.channelID = chID
  372. // wrap the message in a wrapper message, if requested
  373. if wrapper != nil {
  374. msg := proto.Clone(wrapper)
  375. if err := msg.(Wrapper).Wrap(envelope.Message); err != nil {
  376. r.Logger.Error("failed to wrap message", "channel", chID, "err", err)
  377. continue
  378. }
  379. envelope.Message = msg
  380. }
  381. // collect peer queues to pass the message via
  382. var queues []queue
  383. if envelope.Broadcast {
  384. r.peerMtx.RLock()
  385. queues = make([]queue, 0, len(r.peerQueues))
  386. for nodeID, q := range r.peerQueues {
  387. peerChs := r.peerChannels[nodeID]
  388. // check whether the peer is receiving on that channel
  389. if _, ok := peerChs[chID]; ok {
  390. queues = append(queues, q)
  391. }
  392. }
  393. r.peerMtx.RUnlock()
  394. } else {
  395. r.peerMtx.RLock()
  396. q, ok := r.peerQueues[envelope.To]
  397. contains := false
  398. if ok {
  399. peerChs := r.peerChannels[envelope.To]
  400. // check whether the peer is receiving on that channel
  401. _, contains = peerChs[chID]
  402. }
  403. r.peerMtx.RUnlock()
  404. if !ok {
  405. r.logger.Debug("dropping message for unconnected peer", "peer", envelope.To, "channel", chID)
  406. continue
  407. }
  408. if !contains {
  409. // reactor tried to send a message across a channel that the
  410. // peer doesn't have available. This is a known issue due to
  411. // how peer subscriptions work:
  412. // https://github.com/tendermint/tendermint/issues/6598
  413. continue
  414. }
  415. queues = []queue{q}
  416. }
  417. // send message to peers
  418. for _, q := range queues {
  419. start := time.Now().UTC()
  420. select {
  421. case q.enqueue() <- envelope:
  422. r.metrics.RouterPeerQueueSend.Observe(time.Since(start).Seconds())
  423. case <-q.closed():
  424. r.logger.Debug("dropping message for unconnected peer", "peer", envelope.To, "channel", chID)
  425. case <-r.stopCh:
  426. return
  427. }
  428. }
  429. case peerError, ok := <-errCh:
  430. if !ok {
  431. return
  432. }
  433. r.logger.Error("peer error, evicting", "peer", peerError.NodeID, "err", peerError.Err)
  434. r.peerManager.Errored(peerError.NodeID, peerError.Err)
  435. case <-r.stopCh:
  436. return
  437. }
  438. }
  439. }
  440. func (r *Router) numConccurentDials() int {
  441. if r.options.NumConcurrentDials == nil {
  442. return runtime.NumCPU()
  443. }
  444. return r.options.NumConcurrentDials()
  445. }
  446. func (r *Router) filterPeersIP(ctx context.Context, ip net.IP, port uint16) error {
  447. if r.options.FilterPeerByIP == nil {
  448. return nil
  449. }
  450. return r.options.FilterPeerByIP(ctx, ip, port)
  451. }
  452. func (r *Router) filterPeersID(ctx context.Context, id types.NodeID) error {
  453. if r.options.FilterPeerByID == nil {
  454. return nil
  455. }
  456. return r.options.FilterPeerByID(ctx, id)
  457. }
  458. func (r *Router) dialSleep(ctx context.Context) {
  459. if r.options.DialSleep == nil {
  460. // nolint:gosec // G404: Use of weak random number generator
  461. timer := time.NewTimer(time.Duration(rand.Int63n(dialRandomizerIntervalMilliseconds)) * time.Millisecond)
  462. defer timer.Stop()
  463. select {
  464. case <-ctx.Done():
  465. case <-timer.C:
  466. }
  467. return
  468. }
  469. r.options.DialSleep(ctx)
  470. }
  471. // acceptPeers accepts inbound connections from peers on the given transport,
  472. // and spawns goroutines that route messages to/from them.
  473. func (r *Router) acceptPeers(transport Transport) {
  474. r.logger.Debug("starting accept routine", "transport", transport)
  475. ctx := r.stopCtx()
  476. for {
  477. conn, err := transport.Accept()
  478. switch err {
  479. case nil:
  480. case io.EOF:
  481. r.logger.Debug("stopping accept routine", "transport", transport)
  482. return
  483. default:
  484. r.logger.Error("failed to accept connection", "transport", transport, "err", err)
  485. return
  486. }
  487. incomingIP := conn.RemoteEndpoint().IP
  488. if err := r.connTracker.AddConn(incomingIP); err != nil {
  489. closeErr := conn.Close()
  490. r.logger.Debug("rate limiting incoming peer",
  491. "err", err,
  492. "ip", incomingIP.String(),
  493. "close_err", closeErr,
  494. )
  495. return
  496. }
  497. // Spawn a goroutine for the handshake, to avoid head-of-line blocking.
  498. go r.openConnection(ctx, conn)
  499. }
  500. }
  501. func (r *Router) openConnection(ctx context.Context, conn Connection) {
  502. defer conn.Close()
  503. defer r.connTracker.RemoveConn(conn.RemoteEndpoint().IP)
  504. re := conn.RemoteEndpoint()
  505. incomingIP := re.IP
  506. if err := r.filterPeersIP(ctx, incomingIP, re.Port); err != nil {
  507. r.logger.Debug("peer filtered by IP", "ip", incomingIP.String(), "err", err)
  508. return
  509. }
  510. // FIXME: The peer manager may reject the peer during Accepted()
  511. // after we've handshaked with the peer (to find out which peer it
  512. // is). However, because the handshake has no ack, the remote peer
  513. // will think the handshake was successful and start sending us
  514. // messages.
  515. //
  516. // This can cause problems in tests, where a disconnection can cause
  517. // the local node to immediately redial, while the remote node may
  518. // not have completed the disconnection yet and therefore reject the
  519. // reconnection attempt (since it thinks we're still connected from
  520. // before).
  521. //
  522. // The Router should do the handshake and have a final ack/fail
  523. // message to make sure both ends have accepted the connection, such
  524. // that it can be coordinated with the peer manager.
  525. peerInfo, _, err := r.handshakePeer(ctx, conn, "")
  526. switch {
  527. case errors.Is(err, context.Canceled):
  528. return
  529. case err != nil:
  530. r.logger.Error("peer handshake failed", "endpoint", conn, "err", err)
  531. return
  532. }
  533. if err := r.filterPeersID(ctx, peerInfo.NodeID); err != nil {
  534. r.logger.Debug("peer filtered by node ID", "node", peerInfo.NodeID, "err", err)
  535. return
  536. }
  537. if err := r.runWithPeerMutex(func() error { return r.peerManager.Accepted(peerInfo.NodeID) }); err != nil {
  538. r.logger.Error("failed to accept connection",
  539. "op", "incoming/accepted", "peer", peerInfo.NodeID, "err", err)
  540. return
  541. }
  542. r.routePeer(peerInfo.NodeID, conn, toChannelIDs(peerInfo.Channels))
  543. }
  544. // dialPeers maintains outbound connections to peers by dialing them.
  545. func (r *Router) dialPeers() {
  546. r.logger.Debug("starting dial routine")
  547. ctx := r.stopCtx()
  548. addresses := make(chan NodeAddress)
  549. wg := &sync.WaitGroup{}
  550. // Start a limited number of goroutines to dial peers in
  551. // parallel. the goal is to avoid starting an unbounded number
  552. // of goroutines thereby spamming the network, but also being
  553. // able to add peers at a reasonable pace, though the number
  554. // is somewhat arbitrary. The action is further throttled by a
  555. // sleep after sending to the addresses channel.
  556. for i := 0; i < r.numConccurentDials(); i++ {
  557. wg.Add(1)
  558. go func() {
  559. defer wg.Done()
  560. for {
  561. select {
  562. case <-ctx.Done():
  563. return
  564. case address := <-addresses:
  565. r.connectPeer(ctx, address)
  566. }
  567. }
  568. }()
  569. }
  570. LOOP:
  571. for {
  572. address, err := r.peerManager.DialNext(ctx)
  573. switch {
  574. case errors.Is(err, context.Canceled):
  575. r.logger.Debug("stopping dial routine")
  576. break LOOP
  577. case err != nil:
  578. r.logger.Error("failed to find next peer to dial", "err", err)
  579. break LOOP
  580. }
  581. select {
  582. case addresses <- address:
  583. // this jitters the frequency that we call
  584. // DialNext and prevents us from attempting to
  585. // create connections too quickly.
  586. r.dialSleep(ctx)
  587. continue
  588. case <-ctx.Done():
  589. close(addresses)
  590. break LOOP
  591. }
  592. }
  593. wg.Wait()
  594. }
  595. func (r *Router) connectPeer(ctx context.Context, address NodeAddress) {
  596. conn, err := r.dialPeer(ctx, address)
  597. switch {
  598. case errors.Is(err, context.Canceled):
  599. return
  600. case err != nil:
  601. r.logger.Error("failed to dial peer", "peer", address, "err", err)
  602. if err = r.peerManager.DialFailed(address); err != nil {
  603. r.logger.Error("failed to report dial failure", "peer", address, "err", err)
  604. }
  605. return
  606. }
  607. peerInfo, _, err := r.handshakePeer(ctx, conn, address.NodeID)
  608. switch {
  609. case errors.Is(err, context.Canceled):
  610. conn.Close()
  611. return
  612. case err != nil:
  613. r.logger.Error("failed to handshake with peer", "peer", address, "err", err)
  614. if err = r.peerManager.DialFailed(address); err != nil {
  615. r.logger.Error("failed to report dial failure", "peer", address, "err", err)
  616. }
  617. conn.Close()
  618. return
  619. }
  620. if err := r.runWithPeerMutex(func() error { return r.peerManager.Dialed(address) }); err != nil {
  621. r.logger.Error("failed to dial peer",
  622. "op", "outgoing/dialing", "peer", address.NodeID, "err", err)
  623. conn.Close()
  624. return
  625. }
  626. // routePeer (also) calls connection close
  627. go r.routePeer(address.NodeID, conn, toChannelIDs(peerInfo.Channels))
  628. }
  629. func (r *Router) getOrMakeQueue(peerID types.NodeID, channels channelIDs) queue {
  630. r.peerMtx.Lock()
  631. defer r.peerMtx.Unlock()
  632. if peerQueue, ok := r.peerQueues[peerID]; ok {
  633. return peerQueue
  634. }
  635. peerQueue := r.queueFactory(queueBufferDefault)
  636. r.peerQueues[peerID] = peerQueue
  637. r.peerChannels[peerID] = channels
  638. return peerQueue
  639. }
  640. // dialPeer connects to a peer by dialing it.
  641. func (r *Router) dialPeer(ctx context.Context, address NodeAddress) (Connection, error) {
  642. resolveCtx := ctx
  643. if r.options.ResolveTimeout > 0 {
  644. var cancel context.CancelFunc
  645. resolveCtx, cancel = context.WithTimeout(resolveCtx, r.options.ResolveTimeout)
  646. defer cancel()
  647. }
  648. r.logger.Debug("resolving peer address", "peer", address)
  649. endpoints, err := address.Resolve(resolveCtx)
  650. switch {
  651. case err != nil:
  652. return nil, fmt.Errorf("failed to resolve address %q: %w", address, err)
  653. case len(endpoints) == 0:
  654. return nil, fmt.Errorf("address %q did not resolve to any endpoints", address)
  655. }
  656. for _, endpoint := range endpoints {
  657. transport, ok := r.protocolTransports[endpoint.Protocol]
  658. if !ok {
  659. r.logger.Error("no transport found for protocol", "endpoint", endpoint)
  660. continue
  661. }
  662. dialCtx := ctx
  663. if r.options.DialTimeout > 0 {
  664. var cancel context.CancelFunc
  665. dialCtx, cancel = context.WithTimeout(dialCtx, r.options.DialTimeout)
  666. defer cancel()
  667. }
  668. // FIXME: When we dial and handshake the peer, we should pass it
  669. // appropriate address(es) it can use to dial us back. It can't use our
  670. // remote endpoint, since TCP uses different port numbers for outbound
  671. // connections than it does for inbound. Also, we may need to vary this
  672. // by the peer's endpoint, since e.g. a peer on 192.168.0.0 can reach us
  673. // on a private address on this endpoint, but a peer on the public
  674. // Internet can't and needs a different public address.
  675. conn, err := transport.Dial(dialCtx, endpoint)
  676. if err != nil {
  677. r.logger.Error("failed to dial endpoint", "peer", address.NodeID, "endpoint", endpoint, "err", err)
  678. } else {
  679. r.logger.Debug("dialed peer", "peer", address.NodeID, "endpoint", endpoint)
  680. return conn, nil
  681. }
  682. }
  683. return nil, errors.New("all endpoints failed")
  684. }
  685. // handshakePeer handshakes with a peer, validating the peer's information. If
  686. // expectID is given, we check that the peer's info matches it.
  687. func (r *Router) handshakePeer(
  688. ctx context.Context,
  689. conn Connection,
  690. expectID types.NodeID,
  691. ) (types.NodeInfo, crypto.PubKey, error) {
  692. if r.options.HandshakeTimeout > 0 {
  693. var cancel context.CancelFunc
  694. ctx, cancel = context.WithTimeout(ctx, r.options.HandshakeTimeout)
  695. defer cancel()
  696. }
  697. peerInfo, peerKey, err := conn.Handshake(ctx, r.nodeInfo, r.privKey)
  698. if err != nil {
  699. return peerInfo, peerKey, err
  700. }
  701. if err = peerInfo.Validate(); err != nil {
  702. return peerInfo, peerKey, fmt.Errorf("invalid handshake NodeInfo: %w", err)
  703. }
  704. if types.NodeIDFromPubKey(peerKey) != peerInfo.NodeID {
  705. return peerInfo, peerKey, fmt.Errorf("peer's public key did not match its node ID %q (expected %q)",
  706. peerInfo.NodeID, types.NodeIDFromPubKey(peerKey))
  707. }
  708. if expectID != "" && expectID != peerInfo.NodeID {
  709. return peerInfo, peerKey, fmt.Errorf("expected to connect with peer %q, got %q",
  710. expectID, peerInfo.NodeID)
  711. }
  712. if err := r.nodeInfo.CompatibleWith(peerInfo); err != nil {
  713. return peerInfo, peerKey, ErrRejected{
  714. err: err,
  715. id: peerInfo.ID(),
  716. isIncompatible: true,
  717. }
  718. }
  719. return peerInfo, peerKey, nil
  720. }
  721. func (r *Router) runWithPeerMutex(fn func() error) error {
  722. r.peerMtx.Lock()
  723. defer r.peerMtx.Unlock()
  724. return fn()
  725. }
  726. // routePeer routes inbound and outbound messages between a peer and the reactor
  727. // channels. It will close the given connection and send queue when done, or if
  728. // they are closed elsewhere it will cause this method to shut down and return.
  729. func (r *Router) routePeer(peerID types.NodeID, conn Connection, channels channelIDs) {
  730. r.metrics.Peers.Add(1)
  731. r.peerManager.Ready(peerID)
  732. sendQueue := r.getOrMakeQueue(peerID, channels)
  733. defer func() {
  734. r.peerMtx.Lock()
  735. delete(r.peerQueues, peerID)
  736. delete(r.peerChannels, peerID)
  737. r.peerMtx.Unlock()
  738. sendQueue.close()
  739. r.peerManager.Disconnected(peerID)
  740. r.metrics.Peers.Add(-1)
  741. }()
  742. r.logger.Info("peer connected", "peer", peerID, "endpoint", conn)
  743. errCh := make(chan error, 2)
  744. go func() {
  745. errCh <- r.receivePeer(peerID, conn)
  746. }()
  747. go func() {
  748. errCh <- r.sendPeer(peerID, conn, sendQueue)
  749. }()
  750. err := <-errCh
  751. _ = conn.Close()
  752. sendQueue.close()
  753. if e := <-errCh; err == nil {
  754. // The first err was nil, so we update it with the second err, which may
  755. // or may not be nil.
  756. err = e
  757. }
  758. switch err {
  759. case nil, io.EOF:
  760. r.logger.Info("peer disconnected", "peer", peerID, "endpoint", conn)
  761. default:
  762. r.logger.Error("peer failure", "peer", peerID, "endpoint", conn, "err", err)
  763. }
  764. }
  765. // receivePeer receives inbound messages from a peer, deserializes them and
  766. // passes them on to the appropriate channel.
  767. func (r *Router) receivePeer(peerID types.NodeID, conn Connection) error {
  768. for {
  769. chID, bz, err := conn.ReceiveMessage()
  770. if err != nil {
  771. return err
  772. }
  773. r.channelMtx.RLock()
  774. queue, ok := r.channelQueues[chID]
  775. messageType := r.channelMessages[chID]
  776. r.channelMtx.RUnlock()
  777. if !ok {
  778. r.logger.Debug("dropping message for unknown channel", "peer", peerID, "channel", chID)
  779. continue
  780. }
  781. msg := proto.Clone(messageType)
  782. if err := proto.Unmarshal(bz, msg); err != nil {
  783. r.logger.Error("message decoding failed, dropping message", "peer", peerID, "err", err)
  784. continue
  785. }
  786. if wrapper, ok := msg.(Wrapper); ok {
  787. msg, err = wrapper.Unwrap()
  788. if err != nil {
  789. r.logger.Error("failed to unwrap message", "err", err)
  790. continue
  791. }
  792. }
  793. start := time.Now().UTC()
  794. select {
  795. case queue.enqueue() <- Envelope{From: peerID, Message: msg}:
  796. r.metrics.PeerReceiveBytesTotal.With(
  797. "chID", fmt.Sprint(chID),
  798. "peer_id", string(peerID)).Add(float64(proto.Size(msg)))
  799. r.metrics.RouterChannelQueueSend.Observe(time.Since(start).Seconds())
  800. r.logger.Debug("received message", "peer", peerID, "message", msg)
  801. case <-queue.closed():
  802. r.logger.Debug("channel closed, dropping message", "peer", peerID, "channel", chID)
  803. case <-r.stopCh:
  804. return nil
  805. }
  806. }
  807. }
  808. // sendPeer sends queued messages to a peer.
  809. func (r *Router) sendPeer(peerID types.NodeID, conn Connection, peerQueue queue) error {
  810. for {
  811. start := time.Now().UTC()
  812. select {
  813. case envelope := <-peerQueue.dequeue():
  814. r.metrics.RouterPeerQueueRecv.Observe(time.Since(start).Seconds())
  815. if envelope.Message == nil {
  816. r.logger.Error("dropping nil message", "peer", peerID)
  817. continue
  818. }
  819. bz, err := proto.Marshal(envelope.Message)
  820. if err != nil {
  821. r.logger.Error("failed to marshal message", "peer", peerID, "err", err)
  822. continue
  823. }
  824. _, err = conn.SendMessage(envelope.channelID, bz)
  825. if err != nil {
  826. return err
  827. }
  828. r.logger.Debug("sent message", "peer", envelope.To, "message", envelope.Message)
  829. case <-peerQueue.closed():
  830. return nil
  831. case <-r.stopCh:
  832. return nil
  833. }
  834. }
  835. }
  836. // evictPeers evicts connected peers as requested by the peer manager.
  837. func (r *Router) evictPeers() {
  838. r.logger.Debug("starting evict routine")
  839. ctx := r.stopCtx()
  840. for {
  841. peerID, err := r.peerManager.EvictNext(ctx)
  842. switch {
  843. case errors.Is(err, context.Canceled):
  844. r.logger.Debug("stopping evict routine")
  845. return
  846. case err != nil:
  847. r.logger.Error("failed to find next peer to evict", "err", err)
  848. return
  849. }
  850. r.logger.Info("evicting peer", "peer", peerID)
  851. r.peerMtx.RLock()
  852. queue, ok := r.peerQueues[peerID]
  853. r.peerMtx.RUnlock()
  854. if ok {
  855. queue.close()
  856. }
  857. }
  858. }
  859. // NodeInfo returns a copy of the current NodeInfo. Used for testing.
  860. func (r *Router) NodeInfo() types.NodeInfo {
  861. return r.nodeInfo.Copy()
  862. }
  863. // OnStart implements service.Service.
  864. func (r *Router) OnStart() error {
  865. netAddr, _ := r.nodeInfo.NetAddress()
  866. r.Logger.Info(
  867. "starting router",
  868. "node_id", r.nodeInfo.NodeID,
  869. "channels", r.nodeInfo.Channels,
  870. "listen_addr", r.nodeInfo.ListenAddr,
  871. "net_addr", netAddr,
  872. )
  873. go r.dialPeers()
  874. go r.evictPeers()
  875. for _, transport := range r.transports {
  876. go r.acceptPeers(transport)
  877. }
  878. return nil
  879. }
  880. // OnStop implements service.Service.
  881. //
  882. // All channels must be closed by OpenChannel() callers before stopping the
  883. // router, to prevent blocked channel sends in reactors. Channels are not closed
  884. // here, since that would cause any reactor senders to panic, so it is the
  885. // sender's responsibility.
  886. func (r *Router) OnStop() {
  887. // Signal router shutdown.
  888. close(r.stopCh)
  889. // Close transport listeners (unblocks Accept calls).
  890. for _, transport := range r.transports {
  891. if err := transport.Close(); err != nil {
  892. r.logger.Error("failed to close transport", "transport", transport, "err", err)
  893. }
  894. }
  895. // Collect all remaining queues, and wait for them to close.
  896. queues := []queue{}
  897. r.channelMtx.RLock()
  898. for _, q := range r.channelQueues {
  899. queues = append(queues, q)
  900. }
  901. r.channelMtx.RUnlock()
  902. r.peerMtx.RLock()
  903. for _, q := range r.peerQueues {
  904. queues = append(queues, q)
  905. }
  906. r.peerMtx.RUnlock()
  907. for _, q := range queues {
  908. <-q.closed()
  909. }
  910. }
  911. // stopCtx returns a new context that is canceled when the router stops.
  912. func (r *Router) stopCtx() context.Context {
  913. ctx, cancel := context.WithCancel(context.Background())
  914. go func() {
  915. <-r.stopCh
  916. cancel()
  917. }()
  918. return ctx
  919. }
  920. type channelIDs map[ChannelID]struct{}
  921. func toChannelIDs(bytes []byte) channelIDs {
  922. c := make(map[ChannelID]struct{}, len(bytes))
  923. for _, b := range bytes {
  924. c[ChannelID(b)] = struct{}{}
  925. }
  926. return c
  927. }