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.

299 lines
8.1 KiB

  1. package statesync
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "sync"
  7. "github.com/tendermint/tendermint/internal/p2p"
  8. "github.com/tendermint/tendermint/light/provider"
  9. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  10. proto "github.com/tendermint/tendermint/proto/tendermint/types"
  11. "github.com/tendermint/tendermint/types"
  12. )
  13. var (
  14. errNoConnectedPeers = errors.New("no available peers to dispatch request to")
  15. errUnsolicitedResponse = errors.New("unsolicited light block response")
  16. errPeerAlreadyBusy = errors.New("peer is already processing a request")
  17. errDisconnected = errors.New("dispatcher disconnected")
  18. )
  19. // A Dispatcher multiplexes concurrent requests by multiple peers for light blocks.
  20. // Only one request per peer can be sent at a time. Subsequent concurrent requests will
  21. // report an error from the LightBlock method.
  22. // NOTE: It is not the responsibility of the dispatcher to verify the light blocks.
  23. type Dispatcher struct {
  24. // the channel with which to send light block requests on
  25. requestCh chan<- p2p.Envelope
  26. closeCh chan struct{}
  27. mtx sync.Mutex
  28. // all pending calls that have been dispatched and are awaiting an answer
  29. calls map[types.NodeID]chan *types.LightBlock
  30. }
  31. func NewDispatcher(requestCh chan<- p2p.Envelope) *Dispatcher {
  32. return &Dispatcher{
  33. requestCh: requestCh,
  34. closeCh: make(chan struct{}),
  35. calls: make(map[types.NodeID]chan *types.LightBlock),
  36. }
  37. }
  38. // LightBlock uses the request channel to fetch a light block from a given peer
  39. // tracking, the call and waiting for the reactor to pass back the response. A nil
  40. // LightBlock response is used to signal that the peer doesn't have the requested LightBlock.
  41. func (d *Dispatcher) LightBlock(ctx context.Context, height int64, peer types.NodeID) (*types.LightBlock, error) {
  42. // dispatch the request to the peer
  43. callCh, err := d.dispatch(peer, height)
  44. if err != nil {
  45. return nil, err
  46. }
  47. // clean up the call after a response is returned
  48. defer func() {
  49. d.mtx.Lock()
  50. defer d.mtx.Unlock()
  51. if call, ok := d.calls[peer]; ok {
  52. delete(d.calls, peer)
  53. close(call)
  54. }
  55. }()
  56. // wait for a response, cancel or timeout
  57. select {
  58. case resp := <-callCh:
  59. return resp, nil
  60. case <-ctx.Done():
  61. return nil, ctx.Err()
  62. case <-d.closeCh:
  63. return nil, errDisconnected
  64. }
  65. }
  66. // dispatch takes a peer and allocates it a channel so long as it's not already
  67. // busy and the receiving channel is still running. It then dispatches the message
  68. func (d *Dispatcher) dispatch(peer types.NodeID, height int64) (chan *types.LightBlock, error) {
  69. d.mtx.Lock()
  70. defer d.mtx.Unlock()
  71. select {
  72. case <-d.closeCh:
  73. return nil, errDisconnected
  74. default:
  75. }
  76. ch := make(chan *types.LightBlock, 1)
  77. // check if a request for the same peer has already been made
  78. if _, ok := d.calls[peer]; ok {
  79. close(ch)
  80. return ch, errPeerAlreadyBusy
  81. }
  82. d.calls[peer] = ch
  83. // send request
  84. d.requestCh <- p2p.Envelope{
  85. To: peer,
  86. Message: &ssproto.LightBlockRequest{
  87. Height: uint64(height),
  88. },
  89. }
  90. return ch, nil
  91. }
  92. // Respond allows the underlying process which receives requests on the
  93. // requestCh to respond with the respective light block. A nil response is used to
  94. // represent that the receiver of the request does not have a light block at that height.
  95. func (d *Dispatcher) Respond(lb *proto.LightBlock, peer types.NodeID) error {
  96. d.mtx.Lock()
  97. defer d.mtx.Unlock()
  98. // check that the response came from a request
  99. answerCh, ok := d.calls[peer]
  100. if !ok {
  101. // this can also happen if the response came in after the timeout
  102. return errUnsolicitedResponse
  103. }
  104. // If lb is nil we take that to mean that the peer didn't have the requested light
  105. // block and thus pass on the nil to the caller.
  106. if lb == nil {
  107. answerCh <- nil
  108. return nil
  109. }
  110. block, err := types.LightBlockFromProto(lb)
  111. if err != nil {
  112. return err
  113. }
  114. answerCh <- block
  115. return nil
  116. }
  117. // Close shuts down the dispatcher and cancels any pending calls awaiting responses.
  118. // Peers awaiting responses that have not arrived are delivered a nil block.
  119. func (d *Dispatcher) Close() {
  120. d.mtx.Lock()
  121. defer d.mtx.Unlock()
  122. close(d.closeCh)
  123. for peer, call := range d.calls {
  124. delete(d.calls, peer)
  125. close(call)
  126. }
  127. }
  128. func (d *Dispatcher) Done() <-chan struct{} {
  129. return d.closeCh
  130. }
  131. //----------------------------------------------------------------
  132. // BlockProvider is a p2p based light provider which uses a dispatcher connected
  133. // to the state sync reactor to serve light blocks to the light client
  134. //
  135. // TODO: This should probably be moved over to the light package but as we're
  136. // not yet officially supporting p2p light clients we'll leave this here for now.
  137. //
  138. // NOTE: BlockProvider will return an error with concurrent calls. However, we don't
  139. // need a mutex because a light client (and the backfill process) will never call a
  140. // method more than once at the same time
  141. type BlockProvider struct {
  142. peer types.NodeID
  143. chainID string
  144. dispatcher *Dispatcher
  145. }
  146. // Creates a block provider which implements the light client Provider interface.
  147. func NewBlockProvider(peer types.NodeID, chainID string, dispatcher *Dispatcher) *BlockProvider {
  148. return &BlockProvider{
  149. peer: peer,
  150. chainID: chainID,
  151. dispatcher: dispatcher,
  152. }
  153. }
  154. // LightBlock fetches a light block from the peer at a specified height returning either a
  155. // light block or an appropriate error.
  156. func (p *BlockProvider) LightBlock(ctx context.Context, height int64) (*types.LightBlock, error) {
  157. lb, err := p.dispatcher.LightBlock(ctx, height, p.peer)
  158. switch err {
  159. case nil:
  160. if lb == nil {
  161. return nil, provider.ErrLightBlockNotFound
  162. }
  163. case context.DeadlineExceeded, context.Canceled:
  164. return nil, err
  165. case errPeerAlreadyBusy:
  166. return nil, provider.ErrLightBlockNotFound
  167. default:
  168. return nil, provider.ErrUnreliableProvider{Reason: err.Error()}
  169. }
  170. // check that the height requested is the same one returned
  171. if lb.Height != height {
  172. return nil, provider.ErrBadLightBlock{
  173. Reason: fmt.Errorf("expected height %d, got height %d", height, lb.Height),
  174. }
  175. }
  176. // perform basic validation
  177. if err := lb.ValidateBasic(p.chainID); err != nil {
  178. return nil, provider.ErrBadLightBlock{Reason: err}
  179. }
  180. return lb, nil
  181. }
  182. // ReportEvidence should allow for the light client to report any light client
  183. // attacks. This is a no op as there currently isn't a way to wire this up to
  184. // the evidence reactor (we should endeavor to do this in the future but for now
  185. // it's not critical for backwards verification)
  186. func (p *BlockProvider) ReportEvidence(ctx context.Context, ev types.Evidence) error {
  187. return nil
  188. }
  189. // String implements stringer interface
  190. func (p *BlockProvider) String() string { return string(p.peer) }
  191. //----------------------------------------------------------------
  192. // peerList is a rolling list of peers. This is used to distribute the load of
  193. // retrieving blocks over all the peers the reactor is connected to
  194. type peerList struct {
  195. mtx sync.Mutex
  196. peers []types.NodeID
  197. waiting []chan types.NodeID
  198. }
  199. func newPeerList() *peerList {
  200. return &peerList{
  201. peers: make([]types.NodeID, 0),
  202. waiting: make([]chan types.NodeID, 0),
  203. }
  204. }
  205. func (l *peerList) Len() int {
  206. l.mtx.Lock()
  207. defer l.mtx.Unlock()
  208. return len(l.peers)
  209. }
  210. func (l *peerList) Pop(ctx context.Context) types.NodeID {
  211. l.mtx.Lock()
  212. if len(l.peers) == 0 {
  213. // if we don't have any peers in the list we block until a peer is
  214. // appended
  215. wait := make(chan types.NodeID, 1)
  216. l.waiting = append(l.waiting, wait)
  217. // unlock whilst waiting so that the list can be appended to
  218. l.mtx.Unlock()
  219. select {
  220. case peer := <-wait:
  221. return peer
  222. case <-ctx.Done():
  223. return ""
  224. }
  225. }
  226. peer := l.peers[0]
  227. l.peers = l.peers[1:]
  228. l.mtx.Unlock()
  229. return peer
  230. }
  231. func (l *peerList) Append(peer types.NodeID) {
  232. l.mtx.Lock()
  233. defer l.mtx.Unlock()
  234. if len(l.waiting) > 0 {
  235. wait := l.waiting[0]
  236. l.waiting = l.waiting[1:]
  237. wait <- peer
  238. close(wait)
  239. } else {
  240. l.peers = append(l.peers, peer)
  241. }
  242. }
  243. func (l *peerList) Remove(peer types.NodeID) {
  244. l.mtx.Lock()
  245. defer l.mtx.Unlock()
  246. for i, p := range l.peers {
  247. if p == peer {
  248. l.peers = append(l.peers[:i], l.peers[i+1:]...)
  249. return
  250. }
  251. }
  252. }
  253. func (l *peerList) All() []types.NodeID {
  254. l.mtx.Lock()
  255. defer l.mtx.Unlock()
  256. return l.peers
  257. }