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.

809 lines
23 KiB

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