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.

843 lines
24 KiB

  1. package statesync
  2. import (
  3. "bytes"
  4. "context"
  5. "errors"
  6. "fmt"
  7. "reflect"
  8. "runtime/debug"
  9. "sort"
  10. "time"
  11. abci "github.com/tendermint/tendermint/abci/types"
  12. "github.com/tendermint/tendermint/config"
  13. tmsync "github.com/tendermint/tendermint/internal/libs/sync"
  14. "github.com/tendermint/tendermint/internal/p2p"
  15. "github.com/tendermint/tendermint/libs/log"
  16. "github.com/tendermint/tendermint/libs/service"
  17. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  18. "github.com/tendermint/tendermint/proxy"
  19. sm "github.com/tendermint/tendermint/state"
  20. "github.com/tendermint/tendermint/store"
  21. "github.com/tendermint/tendermint/types"
  22. )
  23. var (
  24. _ service.Service = (*Reactor)(nil)
  25. _ p2p.Wrapper = (*ssproto.Message)(nil)
  26. // ChannelShims contains a map of ChannelDescriptorShim objects, where each
  27. // object wraps a reference to a legacy p2p ChannelDescriptor and the corresponding
  28. // p2p proto.Message the new p2p Channel is responsible for handling.
  29. //
  30. //
  31. // TODO: Remove once p2p refactor is complete.
  32. // ref: https://github.com/tendermint/tendermint/issues/5670
  33. ChannelShims = map[p2p.ChannelID]*p2p.ChannelDescriptorShim{
  34. SnapshotChannel: {
  35. MsgType: new(ssproto.Message),
  36. Descriptor: &p2p.ChannelDescriptor{
  37. ID: byte(SnapshotChannel),
  38. Priority: 6,
  39. SendQueueCapacity: 10,
  40. RecvMessageCapacity: snapshotMsgSize,
  41. RecvBufferCapacity: 128,
  42. MaxSendBytes: 400,
  43. },
  44. },
  45. ChunkChannel: {
  46. MsgType: new(ssproto.Message),
  47. Descriptor: &p2p.ChannelDescriptor{
  48. ID: byte(ChunkChannel),
  49. Priority: 3,
  50. SendQueueCapacity: 4,
  51. RecvMessageCapacity: chunkMsgSize,
  52. RecvBufferCapacity: 128,
  53. MaxSendBytes: 400,
  54. },
  55. },
  56. LightBlockChannel: {
  57. MsgType: new(ssproto.Message),
  58. Descriptor: &p2p.ChannelDescriptor{
  59. ID: byte(LightBlockChannel),
  60. Priority: 2,
  61. SendQueueCapacity: 10,
  62. RecvMessageCapacity: lightBlockMsgSize,
  63. RecvBufferCapacity: 128,
  64. MaxSendBytes: 400,
  65. },
  66. },
  67. }
  68. )
  69. const (
  70. // SnapshotChannel exchanges snapshot metadata
  71. SnapshotChannel = p2p.ChannelID(0x60)
  72. // ChunkChannel exchanges chunk contents
  73. ChunkChannel = p2p.ChannelID(0x61)
  74. // LightBlockChannel exchanges light blocks
  75. LightBlockChannel = p2p.ChannelID(0x62)
  76. // recentSnapshots is the number of recent snapshots to send and receive per peer.
  77. recentSnapshots = 10
  78. // snapshotMsgSize is the maximum size of a snapshotResponseMessage
  79. snapshotMsgSize = int(4e6) // ~4MB
  80. // chunkMsgSize is the maximum size of a chunkResponseMessage
  81. chunkMsgSize = int(16e6) // ~16MB
  82. // lightBlockMsgSize is the maximum size of a lightBlockResponseMessage
  83. lightBlockMsgSize = int(1e7) // ~10MB
  84. // lightBlockResponseTimeout is how long the dispatcher waits for a peer to
  85. // return a light block
  86. lightBlockResponseTimeout = 30 * time.Second
  87. // maxLightBlockRequestRetries is the amount of retries acceptable before
  88. // the backfill process aborts
  89. maxLightBlockRequestRetries = 20
  90. )
  91. // SyncReactor defines an interface used for testing abilities of node.startStateSync.
  92. type SyncReactor interface {
  93. Sync(context.Context, StateProvider, time.Duration) (sm.State, error)
  94. Backfill(sm.State) error
  95. }
  96. // Reactor handles state sync, both restoring snapshots for the local node and
  97. // serving snapshots for other nodes.
  98. type Reactor struct {
  99. service.BaseService
  100. cfg config.StateSyncConfig
  101. stateStore sm.Store
  102. blockStore *store.BlockStore
  103. conn proxy.AppConnSnapshot
  104. connQuery proxy.AppConnQuery
  105. tempDir string
  106. snapshotCh *p2p.Channel
  107. chunkCh *p2p.Channel
  108. blockCh *p2p.Channel
  109. peerUpdates *p2p.PeerUpdates
  110. closeCh chan struct{}
  111. dispatcher *dispatcher
  112. // This will only be set when a state sync is in progress. It is used to feed
  113. // received snapshots and chunks into the sync.
  114. mtx tmsync.RWMutex
  115. syncer *syncer
  116. }
  117. // NewReactor returns a reference to a new state sync reactor, which implements
  118. // the service.Service interface. It accepts a logger, connections for snapshots
  119. // and querying, references to p2p Channels and a channel to listen for peer
  120. // updates on. Note, the reactor will close all p2p Channels when stopping.
  121. func NewReactor(
  122. cfg config.StateSyncConfig,
  123. logger log.Logger,
  124. conn proxy.AppConnSnapshot,
  125. connQuery proxy.AppConnQuery,
  126. snapshotCh, chunkCh, blockCh *p2p.Channel,
  127. peerUpdates *p2p.PeerUpdates,
  128. stateStore sm.Store,
  129. blockStore *store.BlockStore,
  130. tempDir string,
  131. ) *Reactor {
  132. r := &Reactor{
  133. cfg: cfg,
  134. conn: conn,
  135. connQuery: connQuery,
  136. snapshotCh: snapshotCh,
  137. chunkCh: chunkCh,
  138. blockCh: blockCh,
  139. peerUpdates: peerUpdates,
  140. closeCh: make(chan struct{}),
  141. tempDir: tempDir,
  142. dispatcher: newDispatcher(blockCh.Out, lightBlockResponseTimeout),
  143. stateStore: stateStore,
  144. blockStore: blockStore,
  145. }
  146. r.BaseService = *service.NewBaseService(logger, "StateSync", r)
  147. return r
  148. }
  149. // OnStart starts separate go routines for each p2p Channel and listens for
  150. // envelopes on each. In addition, it also listens for peer updates and handles
  151. // messages on that p2p channel accordingly. The caller must be sure to execute
  152. // OnStop to ensure the outbound p2p Channels are closed. No error is returned.
  153. func (r *Reactor) OnStart() error {
  154. // Listen for envelopes on the snapshot p2p Channel in a separate go-routine
  155. // as to not block or cause IO contention with the chunk p2p Channel. Note,
  156. // we do not launch a go-routine to handle individual envelopes as to not
  157. // have to deal with bounding workers or pools.
  158. go r.processSnapshotCh()
  159. // Listen for envelopes on the chunk p2p Channel in a separate go-routine
  160. // as to not block or cause IO contention with the snapshot p2p Channel. Note,
  161. // we do not launch a go-routine to handle individual envelopes as to not
  162. // have to deal with bounding workers or pools.
  163. go r.processChunkCh()
  164. go r.processBlockCh()
  165. go r.processPeerUpdates()
  166. r.dispatcher.start()
  167. return nil
  168. }
  169. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  170. // blocking until they all exit.
  171. func (r *Reactor) OnStop() {
  172. // tell the dispatcher to stop sending any more requests
  173. r.dispatcher.stop()
  174. // Close closeCh to signal to all spawned goroutines to gracefully exit. All
  175. // p2p Channels should execute Close().
  176. close(r.closeCh)
  177. // Wait for all p2p Channels to be closed before returning. This ensures we
  178. // can easily reason about synchronization of all p2p Channels and ensure no
  179. // panics will occur.
  180. <-r.snapshotCh.Done()
  181. <-r.chunkCh.Done()
  182. <-r.blockCh.Done()
  183. <-r.peerUpdates.Done()
  184. }
  185. // Sync runs a state sync, fetching snapshots and providing chunks to the
  186. // application. It also saves tendermint state and runs a backfill process to
  187. // retrieve the necessary amount of headers, commits and validators sets to be
  188. // able to process evidence and participate in consensus.
  189. func (r *Reactor) Sync(
  190. ctx context.Context,
  191. stateProvider StateProvider,
  192. discoveryTime time.Duration,
  193. ) (sm.State, error) {
  194. r.mtx.Lock()
  195. if r.syncer != nil {
  196. r.mtx.Unlock()
  197. return sm.State{}, errors.New("a state sync is already in progress")
  198. }
  199. if stateProvider == nil {
  200. r.mtx.Unlock()
  201. return sm.State{}, errors.New("the stateProvider should not be nil when doing the state sync")
  202. }
  203. r.syncer = newSyncer(
  204. r.cfg,
  205. r.Logger,
  206. r.conn,
  207. r.connQuery,
  208. stateProvider,
  209. r.snapshotCh.Out,
  210. r.chunkCh.Out,
  211. r.tempDir,
  212. )
  213. r.mtx.Unlock()
  214. requestSnapshotsHook := func() {
  215. // request snapshots from all currently connected peers
  216. r.snapshotCh.Out <- p2p.Envelope{
  217. Broadcast: true,
  218. Message: &ssproto.SnapshotsRequest{},
  219. }
  220. }
  221. state, commit, err := r.syncer.SyncAny(ctx, discoveryTime, requestSnapshotsHook)
  222. if err != nil {
  223. return sm.State{}, err
  224. }
  225. r.mtx.Lock()
  226. r.syncer = nil
  227. r.mtx.Unlock()
  228. err = r.stateStore.Bootstrap(state)
  229. if err != nil {
  230. return sm.State{}, fmt.Errorf("failed to bootstrap node with new state: %w", err)
  231. }
  232. err = r.blockStore.SaveSeenCommit(state.LastBlockHeight, commit)
  233. if err != nil {
  234. return sm.State{}, fmt.Errorf("failed to store last seen commit: %w", err)
  235. }
  236. return state, nil
  237. }
  238. // Backfill sequentially fetches, verifies and stores light blocks in reverse
  239. // order. It does not stop verifying blocks until reaching a block with a height
  240. // and time that is less or equal to the stopHeight and stopTime. The
  241. // trustedBlockID should be of the header at startHeight.
  242. func (r *Reactor) Backfill(state sm.State) error {
  243. params := state.ConsensusParams.Evidence
  244. stopHeight := state.LastBlockHeight - params.MaxAgeNumBlocks
  245. stopTime := state.LastBlockTime.Add(-params.MaxAgeDuration)
  246. // ensure that stop height doesn't go below the initial height
  247. if stopHeight < state.InitialHeight {
  248. stopHeight = state.InitialHeight
  249. // this essentially makes stop time a void criteria for termination
  250. stopTime = state.LastBlockTime
  251. }
  252. return r.backfill(
  253. context.Background(),
  254. state.ChainID,
  255. state.LastBlockHeight,
  256. stopHeight,
  257. state.InitialHeight,
  258. state.LastBlockID,
  259. stopTime,
  260. )
  261. }
  262. func (r *Reactor) backfill(
  263. ctx context.Context,
  264. chainID string,
  265. startHeight, stopHeight, initialHeight int64,
  266. trustedBlockID types.BlockID,
  267. stopTime time.Time,
  268. ) error {
  269. r.Logger.Info("starting backfill process...", "startHeight", startHeight,
  270. "stopHeight", stopHeight, "trustedBlockID", trustedBlockID)
  271. const sleepTime = 1 * time.Second
  272. var (
  273. lastValidatorSet *types.ValidatorSet
  274. lastChangeHeight int64 = startHeight
  275. )
  276. queue := newBlockQueue(startHeight, stopHeight, initialHeight, stopTime, maxLightBlockRequestRetries)
  277. // fetch light blocks across four workers. The aim with deploying concurrent
  278. // workers is to equate the network messaging time with the verification
  279. // time. Ideally we want the verification process to never have to be
  280. // waiting on blocks. If it takes 4s to retrieve a block and 1s to verify
  281. // it, then steady state involves four workers.
  282. for i := 0; i < int(r.cfg.Fetchers); i++ {
  283. ctxWithCancel, cancel := context.WithCancel(ctx)
  284. defer cancel()
  285. go func() {
  286. for {
  287. select {
  288. case height := <-queue.nextHeight():
  289. r.Logger.Debug("fetching next block", "height", height)
  290. lb, peer, err := r.dispatcher.LightBlock(ctxWithCancel, height)
  291. if errors.Is(err, context.Canceled) {
  292. return
  293. }
  294. if err != nil {
  295. queue.retry(height)
  296. if errors.Is(err, errNoConnectedPeers) {
  297. r.Logger.Info("backfill: no connected peers to fetch light blocks from; sleeping...",
  298. "sleepTime", sleepTime)
  299. time.Sleep(sleepTime)
  300. } else {
  301. // we don't punish the peer as it might just have not responded in time
  302. r.Logger.Info("backfill: error with fetching light block",
  303. "height", height, "err", err)
  304. }
  305. continue
  306. }
  307. if lb == nil {
  308. r.Logger.Info("backfill: peer didn't have block, fetching from another peer", "height", height)
  309. queue.retry(height)
  310. // As we are fetching blocks backwards, if this node doesn't have the block it likely doesn't
  311. // have any prior ones, thus we remove it from the peer list.
  312. r.dispatcher.removePeer(peer)
  313. continue
  314. }
  315. // run a validate basic. This checks the validator set and commit
  316. // hashes line up
  317. err = lb.ValidateBasic(chainID)
  318. if err != nil || lb.Height != height {
  319. r.Logger.Info("backfill: fetched light block failed validate basic, removing peer...",
  320. "err", err, "height", height)
  321. queue.retry(height)
  322. r.blockCh.Error <- p2p.PeerError{
  323. NodeID: peer,
  324. Err: fmt.Errorf("received invalid light block: %w", err),
  325. }
  326. continue
  327. }
  328. // add block to queue to be verified
  329. queue.add(lightBlockResponse{
  330. block: lb,
  331. peer: peer,
  332. })
  333. r.Logger.Debug("backfill: added light block to processing queue", "height", height)
  334. case <-queue.done():
  335. return
  336. }
  337. }
  338. }()
  339. }
  340. // verify all light blocks
  341. for {
  342. select {
  343. case <-r.closeCh:
  344. queue.close()
  345. return nil
  346. case <-ctx.Done():
  347. queue.close()
  348. return nil
  349. case resp := <-queue.verifyNext():
  350. // validate the header hash. We take the last block id of the
  351. // previous header (i.e. one height above) as the trusted hash which
  352. // we equate to. ValidatorsHash and CommitHash have already been
  353. // checked in the `ValidateBasic`
  354. if w, g := trustedBlockID.Hash, resp.block.Hash(); !bytes.Equal(w, g) {
  355. r.Logger.Info("received invalid light block. header hash doesn't match trusted LastBlockID",
  356. "trustedHash", w, "receivedHash", g, "height", resp.block.Height)
  357. r.blockCh.Error <- p2p.PeerError{
  358. NodeID: resp.peer,
  359. Err: fmt.Errorf("received invalid light block. Expected hash %v, got: %v", w, g),
  360. }
  361. queue.retry(resp.block.Height)
  362. continue
  363. }
  364. // save the signed headers
  365. err := r.blockStore.SaveSignedHeader(resp.block.SignedHeader, trustedBlockID)
  366. if err != nil {
  367. return err
  368. }
  369. // check if there has been a change in the validator set
  370. if lastValidatorSet != nil && !bytes.Equal(resp.block.Header.ValidatorsHash, resp.block.Header.NextValidatorsHash) {
  371. // save all the heights that the last validator set was the same
  372. err = r.stateStore.SaveValidatorSets(resp.block.Height+1, lastChangeHeight, lastValidatorSet)
  373. if err != nil {
  374. return err
  375. }
  376. // update the lastChangeHeight
  377. lastChangeHeight = resp.block.Height
  378. }
  379. trustedBlockID = resp.block.LastBlockID
  380. queue.success(resp.block.Height)
  381. r.Logger.Info("backfill: verified and stored light block", "height", resp.block.Height)
  382. lastValidatorSet = resp.block.ValidatorSet
  383. case <-queue.done():
  384. if err := queue.error(); err != nil {
  385. return err
  386. }
  387. // save the final batch of validators
  388. if err := r.stateStore.SaveValidatorSets(queue.terminal.Height, lastChangeHeight, lastValidatorSet); err != nil {
  389. return err
  390. }
  391. r.Logger.Info("successfully completed backfill process", "endHeight", queue.terminal.Height)
  392. return nil
  393. }
  394. }
  395. }
  396. // Dispatcher exposes the dispatcher so that a state provider can use it for
  397. // light client verification
  398. func (r *Reactor) Dispatcher() *dispatcher { //nolint:golint
  399. return r.dispatcher
  400. }
  401. // handleSnapshotMessage handles envelopes sent from peers on the
  402. // SnapshotChannel. It returns an error only if the Envelope.Message is unknown
  403. // for this channel. This should never be called outside of handleMessage.
  404. func (r *Reactor) handleSnapshotMessage(envelope p2p.Envelope) error {
  405. logger := r.Logger.With("peer", envelope.From)
  406. switch msg := envelope.Message.(type) {
  407. case *ssproto.SnapshotsRequest:
  408. snapshots, err := r.recentSnapshots(recentSnapshots)
  409. if err != nil {
  410. logger.Error("failed to fetch snapshots", "err", err)
  411. return nil
  412. }
  413. for _, snapshot := range snapshots {
  414. logger.Info(
  415. "advertising snapshot",
  416. "height", snapshot.Height,
  417. "format", snapshot.Format,
  418. "peer", envelope.From,
  419. )
  420. r.snapshotCh.Out <- p2p.Envelope{
  421. To: envelope.From,
  422. Message: &ssproto.SnapshotsResponse{
  423. Height: snapshot.Height,
  424. Format: snapshot.Format,
  425. Chunks: snapshot.Chunks,
  426. Hash: snapshot.Hash,
  427. Metadata: snapshot.Metadata,
  428. },
  429. }
  430. }
  431. case *ssproto.SnapshotsResponse:
  432. r.mtx.RLock()
  433. defer r.mtx.RUnlock()
  434. if r.syncer == nil {
  435. logger.Debug("received unexpected snapshot; no state sync in progress")
  436. return nil
  437. }
  438. logger.Debug("received snapshot", "height", msg.Height, "format", msg.Format)
  439. _, err := r.syncer.AddSnapshot(envelope.From, &snapshot{
  440. Height: msg.Height,
  441. Format: msg.Format,
  442. Chunks: msg.Chunks,
  443. Hash: msg.Hash,
  444. Metadata: msg.Metadata,
  445. })
  446. if err != nil {
  447. logger.Error(
  448. "failed to add snapshot",
  449. "height", msg.Height,
  450. "format", msg.Format,
  451. "err", err,
  452. "channel", r.snapshotCh.ID,
  453. )
  454. return nil
  455. }
  456. default:
  457. return fmt.Errorf("received unknown message: %T", msg)
  458. }
  459. return nil
  460. }
  461. // handleChunkMessage handles envelopes sent from peers on the ChunkChannel.
  462. // It returns an error only if the Envelope.Message is unknown for this channel.
  463. // This should never be called outside of handleMessage.
  464. func (r *Reactor) handleChunkMessage(envelope p2p.Envelope) error {
  465. switch msg := envelope.Message.(type) {
  466. case *ssproto.ChunkRequest:
  467. r.Logger.Debug(
  468. "received chunk request",
  469. "height", msg.Height,
  470. "format", msg.Format,
  471. "chunk", msg.Index,
  472. "peer", envelope.From,
  473. )
  474. resp, err := r.conn.LoadSnapshotChunkSync(context.Background(), abci.RequestLoadSnapshotChunk{
  475. Height: msg.Height,
  476. Format: msg.Format,
  477. Chunk: msg.Index,
  478. })
  479. if err != nil {
  480. r.Logger.Error(
  481. "failed to load chunk",
  482. "height", msg.Height,
  483. "format", msg.Format,
  484. "chunk", msg.Index,
  485. "err", err,
  486. "peer", envelope.From,
  487. )
  488. return nil
  489. }
  490. r.Logger.Debug(
  491. "sending chunk",
  492. "height", msg.Height,
  493. "format", msg.Format,
  494. "chunk", msg.Index,
  495. "peer", envelope.From,
  496. )
  497. r.chunkCh.Out <- p2p.Envelope{
  498. To: envelope.From,
  499. Message: &ssproto.ChunkResponse{
  500. Height: msg.Height,
  501. Format: msg.Format,
  502. Index: msg.Index,
  503. Chunk: resp.Chunk,
  504. Missing: resp.Chunk == nil,
  505. },
  506. }
  507. case *ssproto.ChunkResponse:
  508. r.mtx.RLock()
  509. defer r.mtx.RUnlock()
  510. if r.syncer == nil {
  511. r.Logger.Debug("received unexpected chunk; no state sync in progress", "peer", envelope.From)
  512. return nil
  513. }
  514. r.Logger.Debug(
  515. "received chunk; adding to sync",
  516. "height", msg.Height,
  517. "format", msg.Format,
  518. "chunk", msg.Index,
  519. "peer", envelope.From,
  520. )
  521. _, err := r.syncer.AddChunk(&chunk{
  522. Height: msg.Height,
  523. Format: msg.Format,
  524. Index: msg.Index,
  525. Chunk: msg.Chunk,
  526. Sender: envelope.From,
  527. })
  528. if err != nil {
  529. r.Logger.Error(
  530. "failed to add chunk",
  531. "height", msg.Height,
  532. "format", msg.Format,
  533. "chunk", msg.Index,
  534. "err", err,
  535. "peer", envelope.From,
  536. )
  537. return nil
  538. }
  539. default:
  540. return fmt.Errorf("received unknown message: %T", msg)
  541. }
  542. return nil
  543. }
  544. func (r *Reactor) handleLightBlockMessage(envelope p2p.Envelope) error {
  545. switch msg := envelope.Message.(type) {
  546. case *ssproto.LightBlockRequest:
  547. r.Logger.Info("received light block request", "height", msg.Height)
  548. lb, err := r.fetchLightBlock(msg.Height)
  549. if err != nil {
  550. r.Logger.Error("failed to retrieve light block", "err", err, "height", msg.Height)
  551. return err
  552. }
  553. lbproto, err := lb.ToProto()
  554. if err != nil {
  555. r.Logger.Error("marshaling light block to proto", "err", err)
  556. return nil
  557. }
  558. // NOTE: If we don't have the light block we will send a nil light block
  559. // back to the requested node, indicating that we don't have it.
  560. r.blockCh.Out <- p2p.Envelope{
  561. To: envelope.From,
  562. Message: &ssproto.LightBlockResponse{
  563. LightBlock: lbproto,
  564. },
  565. }
  566. case *ssproto.LightBlockResponse:
  567. if err := r.dispatcher.respond(msg.LightBlock, envelope.From); err != nil {
  568. r.Logger.Error("error processing light block response", "err", err)
  569. }
  570. default:
  571. return fmt.Errorf("received unknown message: %T", msg)
  572. }
  573. return nil
  574. }
  575. // handleMessage handles an Envelope sent from a peer on a specific p2p Channel.
  576. // It will handle errors and any possible panics gracefully. A caller can handle
  577. // any error returned by sending a PeerError on the respective channel.
  578. func (r *Reactor) handleMessage(chID p2p.ChannelID, envelope p2p.Envelope) (err error) {
  579. defer func() {
  580. if e := recover(); e != nil {
  581. err = fmt.Errorf("panic in processing message: %v", e)
  582. r.Logger.Error(
  583. "recovering from processing message panic",
  584. "err", err,
  585. "stack", string(debug.Stack()),
  586. )
  587. }
  588. }()
  589. r.Logger.Debug("received message", "message", reflect.TypeOf(envelope.Message), "peer", envelope.From)
  590. switch chID {
  591. case SnapshotChannel:
  592. err = r.handleSnapshotMessage(envelope)
  593. case ChunkChannel:
  594. err = r.handleChunkMessage(envelope)
  595. case LightBlockChannel:
  596. err = r.handleLightBlockMessage(envelope)
  597. default:
  598. err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope)
  599. }
  600. return err
  601. }
  602. // processSnapshotCh initiates a blocking process where we listen for and handle
  603. // envelopes on the SnapshotChannel.
  604. func (r *Reactor) processSnapshotCh() {
  605. r.processCh(r.snapshotCh, "snapshot")
  606. }
  607. // processChunkCh initiates a blocking process where we listen for and handle
  608. // envelopes on the ChunkChannel.
  609. func (r *Reactor) processChunkCh() {
  610. r.processCh(r.chunkCh, "chunk")
  611. }
  612. // processBlockCh initiates a blocking process where we listen for and handle
  613. // envelopes on the LightBlockChannel.
  614. func (r *Reactor) processBlockCh() {
  615. r.processCh(r.blockCh, "light block")
  616. }
  617. // processCh routes state sync messages to their respective handlers. Any error
  618. // encountered during message execution will result in a PeerError being sent on
  619. // the respective channel. When the reactor is stopped, we will catch the signal
  620. // and close the p2p Channel gracefully.
  621. func (r *Reactor) processCh(ch *p2p.Channel, chName string) {
  622. defer ch.Close()
  623. for {
  624. select {
  625. case envelope := <-ch.In:
  626. if err := r.handleMessage(ch.ID, envelope); err != nil {
  627. r.Logger.Error(fmt.Sprintf("failed to process %s message", chName),
  628. "ch_id", ch.ID, "envelope", envelope, "err", err)
  629. ch.Error <- p2p.PeerError{
  630. NodeID: envelope.From,
  631. Err: err,
  632. }
  633. }
  634. case <-r.closeCh:
  635. r.Logger.Debug(fmt.Sprintf("stopped listening on %s channel; closing...", chName))
  636. return
  637. }
  638. }
  639. }
  640. // processPeerUpdate processes a PeerUpdate, returning an error upon failing to
  641. // handle the PeerUpdate or if a panic is recovered.
  642. func (r *Reactor) processPeerUpdate(peerUpdate p2p.PeerUpdate) {
  643. r.Logger.Debug("received peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  644. r.mtx.RLock()
  645. defer r.mtx.RUnlock()
  646. switch peerUpdate.Status {
  647. case p2p.PeerStatusUp:
  648. if r.syncer != nil {
  649. r.syncer.AddPeer(peerUpdate.NodeID)
  650. }
  651. r.dispatcher.addPeer(peerUpdate.NodeID)
  652. case p2p.PeerStatusDown:
  653. if r.syncer != nil {
  654. r.syncer.RemovePeer(peerUpdate.NodeID)
  655. }
  656. r.dispatcher.removePeer(peerUpdate.NodeID)
  657. }
  658. }
  659. // processPeerUpdates initiates a blocking process where we listen for and handle
  660. // PeerUpdate messages. When the reactor is stopped, we will catch the signal and
  661. // close the p2p PeerUpdatesCh gracefully.
  662. func (r *Reactor) processPeerUpdates() {
  663. defer r.peerUpdates.Close()
  664. for {
  665. select {
  666. case peerUpdate := <-r.peerUpdates.Updates():
  667. r.processPeerUpdate(peerUpdate)
  668. case <-r.closeCh:
  669. r.Logger.Debug("stopped listening on peer updates channel; closing...")
  670. return
  671. }
  672. }
  673. }
  674. // recentSnapshots fetches the n most recent snapshots from the app
  675. func (r *Reactor) recentSnapshots(n uint32) ([]*snapshot, error) {
  676. resp, err := r.conn.ListSnapshotsSync(context.Background(), abci.RequestListSnapshots{})
  677. if err != nil {
  678. return nil, err
  679. }
  680. sort.Slice(resp.Snapshots, func(i, j int) bool {
  681. a := resp.Snapshots[i]
  682. b := resp.Snapshots[j]
  683. switch {
  684. case a.Height > b.Height:
  685. return true
  686. case a.Height == b.Height && a.Format > b.Format:
  687. return true
  688. default:
  689. return false
  690. }
  691. })
  692. snapshots := make([]*snapshot, 0, n)
  693. for i, s := range resp.Snapshots {
  694. if i >= recentSnapshots {
  695. break
  696. }
  697. snapshots = append(snapshots, &snapshot{
  698. Height: s.Height,
  699. Format: s.Format,
  700. Chunks: s.Chunks,
  701. Hash: s.Hash,
  702. Metadata: s.Metadata,
  703. })
  704. }
  705. return snapshots, nil
  706. }
  707. // fetchLightBlock works out whether the node has a light block at a particular
  708. // height and if so returns it so it can be gossiped to peers
  709. func (r *Reactor) fetchLightBlock(height uint64) (*types.LightBlock, error) {
  710. h := int64(height)
  711. blockMeta := r.blockStore.LoadBlockMeta(h)
  712. if blockMeta == nil {
  713. return nil, nil
  714. }
  715. commit := r.blockStore.LoadBlockCommit(h)
  716. if commit == nil {
  717. return nil, nil
  718. }
  719. vals, err := r.stateStore.LoadValidators(h)
  720. if err != nil {
  721. return nil, err
  722. }
  723. if vals == nil {
  724. return nil, nil
  725. }
  726. return &types.LightBlock{
  727. SignedHeader: &types.SignedHeader{
  728. Header: &blockMeta.Header,
  729. Commit: commit,
  730. },
  731. ValidatorSet: vals,
  732. }, nil
  733. }