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.

500 lines
12 KiB

p2p: Implement PeerTransport This is the implementation for the design described in ADR 12[0]. It's the first step of a larger refactor of the p2p package as tracked in interface bundling all concerns of low-level connection handling and isolating the rest of peer lifecycle management from the specifics of the low-level internet protocols. Even if the swappable implementation will never be utilised, already the isolation of conn related code in one place will help with the reasoning about execution path and addressation of security sensitive issues surfaced through bounty programs and audits. We deliberately decided to not have Peer filtering and other management in the Transport, its sole responsibility is the translation of connections to Peers, handing those to the caller fully setup. It's the responsibility of the caller to reject those and or keep track. Peer filtering will take place in the Switch and can be inspected in a the following commit. This changeset additionally is an exercise in clean separation of logic and other infrastructural concerns like logging and instrumentation. By leveraging a clean and minimal interface. How this looks can be seen in a follow-up change. Design #2069[2] Refs #2067[3] Fixes #2047[4] Fixes #2046[5] changes: * describe Transport interface * implement new default Transport: MultiplexTransport * test MultiplexTransport with new constraints * implement ConnSet for concurrent management of net.Conn, synchronous to PeerSet * implement and expose duplicate IP filter * implemnt TransportOption for optional parametirisation [0] https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-012-peer-transport.md [1] https://github.com/tendermint/tendermint/issues/2067 [2] https://github.com/tendermint/tendermint/pull/2069 [3] https://github.com/tendermint/tendermint/issues/2067 [4] https://github.com/tendermint/tendermint/issues/2047 [5] https://github.com/tendermint/tendermint/issues/2046
6 years ago
p2p: Implement PeerTransport This is the implementation for the design described in ADR 12[0]. It's the first step of a larger refactor of the p2p package as tracked in interface bundling all concerns of low-level connection handling and isolating the rest of peer lifecycle management from the specifics of the low-level internet protocols. Even if the swappable implementation will never be utilised, already the isolation of conn related code in one place will help with the reasoning about execution path and addressation of security sensitive issues surfaced through bounty programs and audits. We deliberately decided to not have Peer filtering and other management in the Transport, its sole responsibility is the translation of connections to Peers, handing those to the caller fully setup. It's the responsibility of the caller to reject those and or keep track. Peer filtering will take place in the Switch and can be inspected in a the following commit. This changeset additionally is an exercise in clean separation of logic and other infrastructural concerns like logging and instrumentation. By leveraging a clean and minimal interface. How this looks can be seen in a follow-up change. Design #2069[2] Refs #2067[3] Fixes #2047[4] Fixes #2046[5] changes: * describe Transport interface * implement new default Transport: MultiplexTransport * test MultiplexTransport with new constraints * implement ConnSet for concurrent management of net.Conn, synchronous to PeerSet * implement and expose duplicate IP filter * implemnt TransportOption for optional parametirisation [0] https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-012-peer-transport.md [1] https://github.com/tendermint/tendermint/issues/2067 [2] https://github.com/tendermint/tendermint/pull/2069 [3] https://github.com/tendermint/tendermint/issues/2067 [4] https://github.com/tendermint/tendermint/issues/2047 [5] https://github.com/tendermint/tendermint/issues/2046
6 years ago
p2p: Implement PeerTransport This is the implementation for the design described in ADR 12[0]. It's the first step of a larger refactor of the p2p package as tracked in interface bundling all concerns of low-level connection handling and isolating the rest of peer lifecycle management from the specifics of the low-level internet protocols. Even if the swappable implementation will never be utilised, already the isolation of conn related code in one place will help with the reasoning about execution path and addressation of security sensitive issues surfaced through bounty programs and audits. We deliberately decided to not have Peer filtering and other management in the Transport, its sole responsibility is the translation of connections to Peers, handing those to the caller fully setup. It's the responsibility of the caller to reject those and or keep track. Peer filtering will take place in the Switch and can be inspected in a the following commit. This changeset additionally is an exercise in clean separation of logic and other infrastructural concerns like logging and instrumentation. By leveraging a clean and minimal interface. How this looks can be seen in a follow-up change. Design #2069[2] Refs #2067[3] Fixes #2047[4] Fixes #2046[5] changes: * describe Transport interface * implement new default Transport: MultiplexTransport * test MultiplexTransport with new constraints * implement ConnSet for concurrent management of net.Conn, synchronous to PeerSet * implement and expose duplicate IP filter * implemnt TransportOption for optional parametirisation [0] https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-012-peer-transport.md [1] https://github.com/tendermint/tendermint/issues/2067 [2] https://github.com/tendermint/tendermint/pull/2069 [3] https://github.com/tendermint/tendermint/issues/2067 [4] https://github.com/tendermint/tendermint/issues/2047 [5] https://github.com/tendermint/tendermint/issues/2046
6 years ago
p2p: Implement PeerTransport This is the implementation for the design described in ADR 12[0]. It's the first step of a larger refactor of the p2p package as tracked in interface bundling all concerns of low-level connection handling and isolating the rest of peer lifecycle management from the specifics of the low-level internet protocols. Even if the swappable implementation will never be utilised, already the isolation of conn related code in one place will help with the reasoning about execution path and addressation of security sensitive issues surfaced through bounty programs and audits. We deliberately decided to not have Peer filtering and other management in the Transport, its sole responsibility is the translation of connections to Peers, handing those to the caller fully setup. It's the responsibility of the caller to reject those and or keep track. Peer filtering will take place in the Switch and can be inspected in a the following commit. This changeset additionally is an exercise in clean separation of logic and other infrastructural concerns like logging and instrumentation. By leveraging a clean and minimal interface. How this looks can be seen in a follow-up change. Design #2069[2] Refs #2067[3] Fixes #2047[4] Fixes #2046[5] changes: * describe Transport interface * implement new default Transport: MultiplexTransport * test MultiplexTransport with new constraints * implement ConnSet for concurrent management of net.Conn, synchronous to PeerSet * implement and expose duplicate IP filter * implemnt TransportOption for optional parametirisation [0] https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-012-peer-transport.md [1] https://github.com/tendermint/tendermint/issues/2067 [2] https://github.com/tendermint/tendermint/pull/2069 [3] https://github.com/tendermint/tendermint/issues/2067 [4] https://github.com/tendermint/tendermint/issues/2047 [5] https://github.com/tendermint/tendermint/issues/2046
6 years ago
p2p: Implement PeerTransport This is the implementation for the design described in ADR 12[0]. It's the first step of a larger refactor of the p2p package as tracked in interface bundling all concerns of low-level connection handling and isolating the rest of peer lifecycle management from the specifics of the low-level internet protocols. Even if the swappable implementation will never be utilised, already the isolation of conn related code in one place will help with the reasoning about execution path and addressation of security sensitive issues surfaced through bounty programs and audits. We deliberately decided to not have Peer filtering and other management in the Transport, its sole responsibility is the translation of connections to Peers, handing those to the caller fully setup. It's the responsibility of the caller to reject those and or keep track. Peer filtering will take place in the Switch and can be inspected in a the following commit. This changeset additionally is an exercise in clean separation of logic and other infrastructural concerns like logging and instrumentation. By leveraging a clean and minimal interface. How this looks can be seen in a follow-up change. Design #2069[2] Refs #2067[3] Fixes #2047[4] Fixes #2046[5] changes: * describe Transport interface * implement new default Transport: MultiplexTransport * test MultiplexTransport with new constraints * implement ConnSet for concurrent management of net.Conn, synchronous to PeerSet * implement and expose duplicate IP filter * implemnt TransportOption for optional parametirisation [0] https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-012-peer-transport.md [1] https://github.com/tendermint/tendermint/issues/2067 [2] https://github.com/tendermint/tendermint/pull/2069 [3] https://github.com/tendermint/tendermint/issues/2067 [4] https://github.com/tendermint/tendermint/issues/2047 [5] https://github.com/tendermint/tendermint/issues/2046
6 years ago
  1. package p2p
  2. import (
  3. "context"
  4. "fmt"
  5. "net"
  6. "time"
  7. "github.com/tendermint/tendermint/config"
  8. "github.com/tendermint/tendermint/crypto"
  9. "github.com/tendermint/tendermint/p2p/conn"
  10. )
  11. const (
  12. defaultDialTimeout = time.Second
  13. defaultFilterTimeout = 5 * time.Second
  14. defaultHandshakeTimeout = 3 * time.Second
  15. )
  16. // IPResolver is a behaviour subset of net.Resolver.
  17. type IPResolver interface {
  18. LookupIPAddr(context.Context, string) ([]net.IPAddr, error)
  19. }
  20. // accept is the container to carry the upgraded connection and NodeInfo from an
  21. // asynchronously running routine to the Accept method.
  22. type accept struct {
  23. conn net.Conn
  24. nodeInfo NodeInfo
  25. err error
  26. }
  27. // peerConfig is used to bundle data we need to fully setup a Peer with an
  28. // MConn, provided by the caller of Accept and Dial (currently the Switch). This
  29. // a temporary measure until reactor setup is less dynamic and we introduce the
  30. // concept of PeerBehaviour to communicate about significant Peer lifecycle
  31. // events.
  32. // TODO(xla): Refactor out with more static Reactor setup and PeerBehaviour.
  33. type peerConfig struct {
  34. chDescs []*conn.ChannelDescriptor
  35. onPeerError func(Peer, interface{})
  36. outbound, persistent bool
  37. reactorsByCh map[byte]Reactor
  38. metrics *Metrics
  39. }
  40. // Transport emits and connects to Peers. The implementation of Peer is left to
  41. // the transport. Each transport is also responsible to filter establishing
  42. // peers specific to its domain.
  43. type Transport interface {
  44. // Accept returns a newly connected Peer.
  45. Accept(peerConfig) (Peer, error)
  46. // Dial connects to the Peer for the address.
  47. Dial(NetAddress, peerConfig) (Peer, error)
  48. }
  49. // transportLifecycle bundles the methods for callers to control start and stop
  50. // behaviour.
  51. type transportLifecycle interface {
  52. Close() error
  53. Listen(NetAddress) error
  54. }
  55. // ConnFilterFunc to be implemented by filter hooks after a new connection has
  56. // been established. The set of exisiting connections is passed along together
  57. // with all resolved IPs for the new connection.
  58. type ConnFilterFunc func(ConnSet, net.Conn, []net.IP) error
  59. // ConnDuplicateIPFilter resolves and keeps all ips for an incoming connection
  60. // and refuses new ones if they come from a known ip.
  61. func ConnDuplicateIPFilter() ConnFilterFunc {
  62. return func(cs ConnSet, c net.Conn, ips []net.IP) error {
  63. for _, ip := range ips {
  64. if cs.HasIP(ip) {
  65. return ErrRejected{
  66. conn: c,
  67. err: fmt.Errorf("IP<%v> already connected", ip),
  68. isDuplicate: true,
  69. }
  70. }
  71. }
  72. return nil
  73. }
  74. }
  75. // MultiplexTransportOption sets an optional parameter on the
  76. // MultiplexTransport.
  77. type MultiplexTransportOption func(*MultiplexTransport)
  78. // MultiplexTransportConnFilters sets the filters for rejection new connections.
  79. func MultiplexTransportConnFilters(
  80. filters ...ConnFilterFunc,
  81. ) MultiplexTransportOption {
  82. return func(mt *MultiplexTransport) { mt.connFilters = filters }
  83. }
  84. // MultiplexTransportFilterTimeout sets the timeout waited for filter calls to
  85. // return.
  86. func MultiplexTransportFilterTimeout(
  87. timeout time.Duration,
  88. ) MultiplexTransportOption {
  89. return func(mt *MultiplexTransport) { mt.filterTimeout = timeout }
  90. }
  91. // MultiplexTransportResolver sets the Resolver used for ip lokkups, defaults to
  92. // net.DefaultResolver.
  93. func MultiplexTransportResolver(resolver IPResolver) MultiplexTransportOption {
  94. return func(mt *MultiplexTransport) { mt.resolver = resolver }
  95. }
  96. // MultiplexTransport accepts and dials tcp connections and upgrades them to
  97. // multiplexed peers.
  98. type MultiplexTransport struct {
  99. listener net.Listener
  100. acceptc chan accept
  101. closec chan struct{}
  102. // Lookup table for duplicate ip and id checks.
  103. conns ConnSet
  104. connFilters []ConnFilterFunc
  105. dialTimeout time.Duration
  106. filterTimeout time.Duration
  107. handshakeTimeout time.Duration
  108. nodeInfo NodeInfo
  109. nodeKey NodeKey
  110. resolver IPResolver
  111. // TODO(xla): Those configs are still needed as we parameterise peerConn and
  112. // peer currently. All relevant configuration should be refactored into options
  113. // with sane defaults.
  114. mConfig conn.MConnConfig
  115. p2pConfig config.P2PConfig
  116. }
  117. // Test multiplexTransport for interface completeness.
  118. var _ Transport = (*MultiplexTransport)(nil)
  119. var _ transportLifecycle = (*MultiplexTransport)(nil)
  120. // NewMultiplexTransport returns a tcp connected multiplexed peer.
  121. func NewMultiplexTransport(
  122. nodeInfo NodeInfo,
  123. nodeKey NodeKey,
  124. ) *MultiplexTransport {
  125. return &MultiplexTransport{
  126. acceptc: make(chan accept),
  127. closec: make(chan struct{}),
  128. dialTimeout: defaultDialTimeout,
  129. filterTimeout: defaultFilterTimeout,
  130. handshakeTimeout: defaultHandshakeTimeout,
  131. mConfig: conn.DefaultMConnConfig(),
  132. nodeInfo: nodeInfo,
  133. nodeKey: nodeKey,
  134. conns: NewConnSet(),
  135. resolver: net.DefaultResolver,
  136. }
  137. }
  138. // Accept implements Transport.
  139. func (mt *MultiplexTransport) Accept(cfg peerConfig) (Peer, error) {
  140. select {
  141. // This case should never have any side-effectful/blocking operations to
  142. // ensure that quality peers are ready to be used.
  143. case a := <-mt.acceptc:
  144. if a.err != nil {
  145. return nil, a.err
  146. }
  147. cfg.outbound = false
  148. return mt.wrapPeer(a.conn, a.nodeInfo, cfg), nil
  149. case <-mt.closec:
  150. return nil, &ErrTransportClosed{}
  151. }
  152. }
  153. // Dial implements Transport.
  154. func (mt *MultiplexTransport) Dial(
  155. addr NetAddress,
  156. cfg peerConfig,
  157. ) (Peer, error) {
  158. c, err := addr.DialTimeout(mt.dialTimeout)
  159. if err != nil {
  160. return nil, err
  161. }
  162. // TODO(xla): Evaluate if we should apply filters if we explicitly dial.
  163. if err := mt.filterConn(c); err != nil {
  164. return nil, err
  165. }
  166. secretConn, nodeInfo, err := mt.upgrade(c)
  167. if err != nil {
  168. return nil, err
  169. }
  170. cfg.outbound = true
  171. p := mt.wrapPeer(secretConn, nodeInfo, cfg)
  172. return p, nil
  173. }
  174. // Close implements transportLifecycle.
  175. func (mt *MultiplexTransport) Close() error {
  176. close(mt.closec)
  177. if mt.listener != nil {
  178. return mt.listener.Close()
  179. }
  180. return nil
  181. }
  182. // Listen implements transportLifecycle.
  183. func (mt *MultiplexTransport) Listen(addr NetAddress) error {
  184. ln, err := net.Listen("tcp", addr.DialString())
  185. if err != nil {
  186. return err
  187. }
  188. mt.listener = ln
  189. go mt.acceptPeers()
  190. return nil
  191. }
  192. func (mt *MultiplexTransport) acceptPeers() {
  193. for {
  194. c, err := mt.listener.Accept()
  195. if err != nil {
  196. // If Close() has been called, silently exit.
  197. select {
  198. case _, ok := <-mt.closec:
  199. if !ok {
  200. return
  201. }
  202. default:
  203. // Transport is not closed
  204. }
  205. mt.acceptc <- accept{err: err}
  206. return
  207. }
  208. // Connection upgrade and filtering should be asynchronous to avoid
  209. // Head-of-line blocking[0].
  210. // Reference: https://github.com/tendermint/tendermint/issues/2047
  211. //
  212. // [0] https://en.wikipedia.org/wiki/Head-of-line_blocking
  213. go func(c net.Conn) {
  214. var (
  215. nodeInfo NodeInfo
  216. secretConn *conn.SecretConnection
  217. )
  218. err := mt.filterConn(c)
  219. if err == nil {
  220. secretConn, nodeInfo, err = mt.upgrade(c)
  221. }
  222. select {
  223. case mt.acceptc <- accept{secretConn, nodeInfo, err}:
  224. // Make the upgraded peer available.
  225. case <-mt.closec:
  226. // Give up if the transport was closed.
  227. _ = c.Close()
  228. return
  229. }
  230. }(c)
  231. }
  232. }
  233. func (mt *MultiplexTransport) cleanup(c net.Conn) error {
  234. mt.conns.Remove(c)
  235. return c.Close()
  236. }
  237. func (mt *MultiplexTransport) filterConn(c net.Conn) (err error) {
  238. defer func() {
  239. if err != nil {
  240. _ = c.Close()
  241. }
  242. }()
  243. // Reject if connection is already present.
  244. if mt.conns.Has(c) {
  245. return ErrRejected{conn: c, isDuplicate: true}
  246. }
  247. // Resolve ips for incoming conn.
  248. ips, err := resolveIPs(mt.resolver, c)
  249. if err != nil {
  250. return err
  251. }
  252. errc := make(chan error, len(mt.connFilters))
  253. for _, f := range mt.connFilters {
  254. go func(f ConnFilterFunc, c net.Conn, ips []net.IP, errc chan<- error) {
  255. errc <- f(mt.conns, c, ips)
  256. }(f, c, ips, errc)
  257. }
  258. for i := 0; i < cap(errc); i++ {
  259. select {
  260. case err := <-errc:
  261. if err != nil {
  262. return ErrRejected{conn: c, err: err, isFiltered: true}
  263. }
  264. case <-time.After(mt.filterTimeout):
  265. return ErrFilterTimeout{}
  266. }
  267. }
  268. mt.conns.Set(c, ips)
  269. return nil
  270. }
  271. func (mt *MultiplexTransport) upgrade(
  272. c net.Conn,
  273. ) (secretConn *conn.SecretConnection, nodeInfo NodeInfo, err error) {
  274. defer func() {
  275. if err != nil {
  276. _ = mt.cleanup(c)
  277. }
  278. }()
  279. secretConn, err = upgradeSecretConn(c, mt.handshakeTimeout, mt.nodeKey.PrivKey)
  280. if err != nil {
  281. return nil, NodeInfo{}, ErrRejected{
  282. conn: c,
  283. err: fmt.Errorf("secrect conn failed: %v", err),
  284. isAuthFailure: true,
  285. }
  286. }
  287. nodeInfo, err = handshake(secretConn, mt.handshakeTimeout, mt.nodeInfo)
  288. if err != nil {
  289. return nil, NodeInfo{}, ErrRejected{
  290. conn: c,
  291. err: fmt.Errorf("handshake failed: %v", err),
  292. isAuthFailure: true,
  293. }
  294. }
  295. if err := nodeInfo.Validate(); err != nil {
  296. return nil, NodeInfo{}, ErrRejected{
  297. conn: c,
  298. err: err,
  299. isNodeInfoInvalid: true,
  300. }
  301. }
  302. // Ensure connection key matches self reported key.
  303. if connID := PubKeyToID(secretConn.RemotePubKey()); connID != nodeInfo.ID {
  304. return nil, NodeInfo{}, ErrRejected{
  305. conn: c,
  306. id: connID,
  307. err: fmt.Errorf(
  308. "conn.ID (%v) NodeInfo.ID (%v) missmatch",
  309. connID,
  310. nodeInfo.ID,
  311. ),
  312. isAuthFailure: true,
  313. }
  314. }
  315. // Reject self.
  316. if mt.nodeInfo.ID == nodeInfo.ID {
  317. return nil, NodeInfo{}, ErrRejected{
  318. addr: *NewNetAddress(nodeInfo.ID, c.RemoteAddr()),
  319. conn: c,
  320. id: nodeInfo.ID,
  321. isSelf: true,
  322. }
  323. }
  324. if err := mt.nodeInfo.CompatibleWith(nodeInfo); err != nil {
  325. return nil, NodeInfo{}, ErrRejected{
  326. conn: c,
  327. err: err,
  328. id: nodeInfo.ID,
  329. isIncompatible: true,
  330. }
  331. }
  332. return secretConn, nodeInfo, nil
  333. }
  334. func (mt *MultiplexTransport) wrapPeer(
  335. c net.Conn,
  336. ni NodeInfo,
  337. cfg peerConfig,
  338. ) Peer {
  339. p := newPeer(
  340. peerConn{
  341. conn: c,
  342. config: &mt.p2pConfig,
  343. outbound: cfg.outbound,
  344. persistent: cfg.persistent,
  345. },
  346. mt.mConfig,
  347. ni,
  348. cfg.reactorsByCh,
  349. cfg.chDescs,
  350. cfg.onPeerError,
  351. PeerMetrics(cfg.metrics),
  352. )
  353. // Wait for Peer to Stop so we can cleanup.
  354. go func(c net.Conn) {
  355. <-p.Quit()
  356. _ = mt.cleanup(c)
  357. }(c)
  358. return p
  359. }
  360. func handshake(
  361. c net.Conn,
  362. timeout time.Duration,
  363. nodeInfo NodeInfo,
  364. ) (NodeInfo, error) {
  365. if err := c.SetDeadline(time.Now().Add(timeout)); err != nil {
  366. return NodeInfo{}, err
  367. }
  368. var (
  369. errc = make(chan error, 2)
  370. peerNodeInfo NodeInfo
  371. )
  372. go func(errc chan<- error, c net.Conn) {
  373. _, err := cdc.MarshalBinaryWriter(c, nodeInfo)
  374. errc <- err
  375. }(errc, c)
  376. go func(errc chan<- error, c net.Conn) {
  377. _, err := cdc.UnmarshalBinaryReader(
  378. c,
  379. &peerNodeInfo,
  380. int64(MaxNodeInfoSize()),
  381. )
  382. errc <- err
  383. }(errc, c)
  384. for i := 0; i < cap(errc); i++ {
  385. err := <-errc
  386. if err != nil {
  387. return NodeInfo{}, err
  388. }
  389. }
  390. return peerNodeInfo, c.SetDeadline(time.Time{})
  391. }
  392. func upgradeSecretConn(
  393. c net.Conn,
  394. timeout time.Duration,
  395. privKey crypto.PrivKey,
  396. ) (*conn.SecretConnection, error) {
  397. if err := c.SetDeadline(time.Now().Add(timeout)); err != nil {
  398. return nil, err
  399. }
  400. sc, err := conn.MakeSecretConnection(c, privKey)
  401. if err != nil {
  402. return nil, err
  403. }
  404. return sc, sc.SetDeadline(time.Time{})
  405. }
  406. func resolveIPs(resolver IPResolver, c net.Conn) ([]net.IP, error) {
  407. host, _, err := net.SplitHostPort(c.RemoteAddr().String())
  408. if err != nil {
  409. return nil, err
  410. }
  411. addrs, err := resolver.LookupIPAddr(context.Background(), host)
  412. if err != nil {
  413. return nil, err
  414. }
  415. ips := []net.IP{}
  416. for _, addr := range addrs {
  417. ips = append(ips, addr.IP)
  418. }
  419. return ips, nil
  420. }