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.

498 lines
14 KiB

  1. package statesync
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "sort"
  7. "time"
  8. abci "github.com/tendermint/tendermint/abci/types"
  9. "github.com/tendermint/tendermint/libs/log"
  10. "github.com/tendermint/tendermint/libs/service"
  11. tmsync "github.com/tendermint/tendermint/libs/sync"
  12. "github.com/tendermint/tendermint/p2p"
  13. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  14. "github.com/tendermint/tendermint/proxy"
  15. sm "github.com/tendermint/tendermint/state"
  16. "github.com/tendermint/tendermint/types"
  17. )
  18. var (
  19. _ service.Service = (*Reactor)(nil)
  20. _ p2p.Wrapper = (*ssproto.Message)(nil)
  21. // ChannelShims contains a map of ChannelDescriptorShim objects, where each
  22. // object wraps a reference to a legacy p2p ChannelDescriptor and the corresponding
  23. // p2p proto.Message the new p2p Channel is responsible for handling.
  24. //
  25. //
  26. // TODO: Remove once p2p refactor is complete.
  27. // ref: https://github.com/tendermint/tendermint/issues/5670
  28. ChannelShims = map[p2p.ChannelID]*p2p.ChannelDescriptorShim{
  29. SnapshotChannel: {
  30. MsgType: new(ssproto.Message),
  31. Descriptor: &p2p.ChannelDescriptor{
  32. ID: byte(SnapshotChannel),
  33. Priority: 5,
  34. SendQueueCapacity: 10,
  35. RecvMessageCapacity: snapshotMsgSize,
  36. MaxSendBytes: 400,
  37. },
  38. },
  39. ChunkChannel: {
  40. MsgType: new(ssproto.Message),
  41. Descriptor: &p2p.ChannelDescriptor{
  42. ID: byte(ChunkChannel),
  43. Priority: 1,
  44. SendQueueCapacity: 4,
  45. RecvMessageCapacity: chunkMsgSize,
  46. MaxSendBytes: 400,
  47. },
  48. },
  49. }
  50. )
  51. const (
  52. // SnapshotChannel exchanges snapshot metadata
  53. SnapshotChannel = p2p.ChannelID(0x60)
  54. // ChunkChannel exchanges chunk contents
  55. ChunkChannel = p2p.ChannelID(0x61)
  56. // recentSnapshots is the number of recent snapshots to send and receive per peer.
  57. recentSnapshots = 10
  58. // snapshotMsgSize is the maximum size of a snapshotResponseMessage
  59. snapshotMsgSize = int(4e6)
  60. // chunkMsgSize is the maximum size of a chunkResponseMessage
  61. chunkMsgSize = int(16e6)
  62. )
  63. // Reactor handles state sync, both restoring snapshots for the local node and
  64. // serving snapshots for other nodes.
  65. type Reactor struct {
  66. service.BaseService
  67. conn proxy.AppConnSnapshot
  68. connQuery proxy.AppConnQuery
  69. tempDir string
  70. snapshotCh *p2p.Channel
  71. chunkCh *p2p.Channel
  72. peerUpdates *p2p.PeerUpdates
  73. closeCh chan struct{}
  74. // This will only be set when a state sync is in progress. It is used to feed
  75. // received snapshots and chunks into the sync.
  76. mtx tmsync.RWMutex
  77. syncer *syncer
  78. }
  79. // NewReactor returns a reference to a new state sync reactor, which implements
  80. // the service.Service interface. It accepts a logger, connections for snapshots
  81. // and querying, references to p2p Channels and a channel to listen for peer
  82. // updates on. Note, the reactor will close all p2p Channels when stopping.
  83. func NewReactor(
  84. logger log.Logger,
  85. conn proxy.AppConnSnapshot,
  86. connQuery proxy.AppConnQuery,
  87. snapshotCh, chunkCh *p2p.Channel,
  88. peerUpdates *p2p.PeerUpdates,
  89. tempDir string,
  90. ) *Reactor {
  91. r := &Reactor{
  92. conn: conn,
  93. connQuery: connQuery,
  94. snapshotCh: snapshotCh,
  95. chunkCh: chunkCh,
  96. peerUpdates: peerUpdates,
  97. closeCh: make(chan struct{}),
  98. tempDir: tempDir,
  99. }
  100. r.BaseService = *service.NewBaseService(logger, "StateSync", r)
  101. return r
  102. }
  103. // OnStart starts separate go routines for each p2p Channel and listens for
  104. // envelopes on each. In addition, it also listens for peer updates and handles
  105. // messages on that p2p channel accordingly. The caller must be sure to execute
  106. // OnStop to ensure the outbound p2p Channels are closed. No error is returned.
  107. func (r *Reactor) OnStart() error {
  108. // Listen for envelopes on the snapshot p2p Channel in a separate go-routine
  109. // as to not block or cause IO contention with the chunk p2p Channel. Note,
  110. // we do not launch a go-routine to handle individual envelopes as to not
  111. // have to deal with bounding workers or pools.
  112. go r.processSnapshotCh()
  113. // Listen for envelopes on the chunk p2p Channel in a separate go-routine
  114. // as to not block or cause IO contention with the snapshot p2p Channel. Note,
  115. // we do not launch a go-routine to handle individual envelopes as to not
  116. // have to deal with bounding workers or pools.
  117. go r.processChunkCh()
  118. go r.processPeerUpdates()
  119. return nil
  120. }
  121. // OnStop stops the reactor by signaling to all spawned goroutines to exit and
  122. // blocking until they all exit.
  123. func (r *Reactor) OnStop() {
  124. // Close closeCh to signal to all spawned goroutines to gracefully exit. All
  125. // p2p Channels should execute Close().
  126. close(r.closeCh)
  127. // Wait for all p2p Channels to be closed before returning. This ensures we
  128. // can easily reason about synchronization of all p2p Channels and ensure no
  129. // panics will occur.
  130. <-r.snapshotCh.Done()
  131. <-r.chunkCh.Done()
  132. <-r.peerUpdates.Done()
  133. }
  134. // handleSnapshotMessage handles envelopes sent from peers on the
  135. // SnapshotChannel. It returns an error only if the Envelope.Message is unknown
  136. // for this channel. This should never be called outside of handleMessage.
  137. func (r *Reactor) handleSnapshotMessage(envelope p2p.Envelope) error {
  138. logger := r.Logger.With("peer", envelope.From)
  139. switch msg := envelope.Message.(type) {
  140. case *ssproto.SnapshotsRequest:
  141. snapshots, err := r.recentSnapshots(recentSnapshots)
  142. if err != nil {
  143. logger.Error("failed to fetch snapshots", "err", err)
  144. return nil
  145. }
  146. for _, snapshot := range snapshots {
  147. logger.Debug(
  148. "advertising snapshot",
  149. "height", snapshot.Height,
  150. "format", snapshot.Format,
  151. )
  152. r.snapshotCh.Out <- p2p.Envelope{
  153. To: envelope.From,
  154. Message: &ssproto.SnapshotsResponse{
  155. Height: snapshot.Height,
  156. Format: snapshot.Format,
  157. Chunks: snapshot.Chunks,
  158. Hash: snapshot.Hash,
  159. Metadata: snapshot.Metadata,
  160. },
  161. }
  162. }
  163. case *ssproto.SnapshotsResponse:
  164. r.mtx.RLock()
  165. defer r.mtx.RUnlock()
  166. if r.syncer == nil {
  167. logger.Debug("received unexpected snapshot; no state sync in progress")
  168. return nil
  169. }
  170. logger.Debug("received snapshot", "height", msg.Height, "format", msg.Format)
  171. _, err := r.syncer.AddSnapshot(envelope.From, &snapshot{
  172. Height: msg.Height,
  173. Format: msg.Format,
  174. Chunks: msg.Chunks,
  175. Hash: msg.Hash,
  176. Metadata: msg.Metadata,
  177. })
  178. if err != nil {
  179. logger.Error(
  180. "failed to add snapshot",
  181. "height", msg.Height,
  182. "format", msg.Format,
  183. "err", err,
  184. "channel", r.snapshotCh.ID,
  185. )
  186. return nil
  187. }
  188. default:
  189. return fmt.Errorf("received unknown message: %T", msg)
  190. }
  191. return nil
  192. }
  193. // handleChunkMessage handles envelopes sent from peers on the ChunkChannel.
  194. // It returns an error only if the Envelope.Message is unknown for this channel.
  195. // This should never be called outside of handleMessage.
  196. func (r *Reactor) handleChunkMessage(envelope p2p.Envelope) error {
  197. switch msg := envelope.Message.(type) {
  198. case *ssproto.ChunkRequest:
  199. r.Logger.Debug(
  200. "received chunk request",
  201. "height", msg.Height,
  202. "format", msg.Format,
  203. "chunk", msg.Index,
  204. "peer", envelope.From,
  205. )
  206. resp, err := r.conn.LoadSnapshotChunkSync(context.Background(), abci.RequestLoadSnapshotChunk{
  207. Height: msg.Height,
  208. Format: msg.Format,
  209. Chunk: msg.Index,
  210. })
  211. if err != nil {
  212. r.Logger.Error(
  213. "failed to load chunk",
  214. "height", msg.Height,
  215. "format", msg.Format,
  216. "chunk", msg.Index,
  217. "err", err,
  218. "peer", envelope.From,
  219. )
  220. return nil
  221. }
  222. r.Logger.Debug(
  223. "sending chunk",
  224. "height", msg.Height,
  225. "format", msg.Format,
  226. "chunk", msg.Index,
  227. "peer", envelope.From,
  228. )
  229. r.chunkCh.Out <- p2p.Envelope{
  230. To: envelope.From,
  231. Message: &ssproto.ChunkResponse{
  232. Height: msg.Height,
  233. Format: msg.Format,
  234. Index: msg.Index,
  235. Chunk: resp.Chunk,
  236. Missing: resp.Chunk == nil,
  237. },
  238. }
  239. case *ssproto.ChunkResponse:
  240. r.mtx.RLock()
  241. defer r.mtx.RUnlock()
  242. if r.syncer == nil {
  243. r.Logger.Debug("received unexpected chunk; no state sync in progress", "peer", envelope.From)
  244. return nil
  245. }
  246. r.Logger.Debug(
  247. "received chunk; adding to sync",
  248. "height", msg.Height,
  249. "format", msg.Format,
  250. "chunk", msg.Index,
  251. "peer", envelope.From,
  252. )
  253. _, err := r.syncer.AddChunk(&chunk{
  254. Height: msg.Height,
  255. Format: msg.Format,
  256. Index: msg.Index,
  257. Chunk: msg.Chunk,
  258. Sender: envelope.From,
  259. })
  260. if err != nil {
  261. r.Logger.Error(
  262. "failed to add chunk",
  263. "height", msg.Height,
  264. "format", msg.Format,
  265. "chunk", msg.Index,
  266. "err", err,
  267. "peer", envelope.From,
  268. )
  269. return nil
  270. }
  271. default:
  272. return fmt.Errorf("received unknown message: %T", msg)
  273. }
  274. return nil
  275. }
  276. // handleMessage handles an Envelope sent from a peer on a specific p2p Channel.
  277. // It will handle errors and any possible panics gracefully. A caller can handle
  278. // any error returned by sending a PeerError on the respective channel.
  279. func (r *Reactor) handleMessage(chID p2p.ChannelID, envelope p2p.Envelope) (err error) {
  280. defer func() {
  281. if e := recover(); e != nil {
  282. err = fmt.Errorf("panic in processing message: %v", e)
  283. }
  284. }()
  285. r.Logger.Debug("received message", "message", envelope.Message, "peer", envelope.From)
  286. switch chID {
  287. case SnapshotChannel:
  288. err = r.handleSnapshotMessage(envelope)
  289. case ChunkChannel:
  290. err = r.handleChunkMessage(envelope)
  291. default:
  292. err = fmt.Errorf("unknown channel ID (%d) for envelope (%v)", chID, envelope)
  293. }
  294. return err
  295. }
  296. // processSnapshotCh initiates a blocking process where we listen for and handle
  297. // envelopes on the SnapshotChannel. Any error encountered during message
  298. // execution will result in a PeerError being sent on the SnapshotChannel. When
  299. // the reactor is stopped, we will catch the signal and close the p2p Channel
  300. // gracefully.
  301. func (r *Reactor) processSnapshotCh() {
  302. defer r.snapshotCh.Close()
  303. for {
  304. select {
  305. case envelope := <-r.snapshotCh.In:
  306. if err := r.handleMessage(r.snapshotCh.ID, envelope); err != nil {
  307. r.Logger.Error("failed to process message", "ch_id", r.snapshotCh.ID, "envelope", envelope, "err", err)
  308. r.snapshotCh.Error <- p2p.PeerError{
  309. NodeID: envelope.From,
  310. Err: err,
  311. }
  312. }
  313. case <-r.closeCh:
  314. r.Logger.Debug("stopped listening on snapshot channel; closing...")
  315. return
  316. }
  317. }
  318. }
  319. // processChunkCh initiates a blocking process where we listen for and handle
  320. // envelopes on the ChunkChannel. Any error encountered during message
  321. // execution will result in a PeerError being sent on the ChunkChannel. When
  322. // the reactor is stopped, we will catch the signal and close the p2p Channel
  323. // gracefully.
  324. func (r *Reactor) processChunkCh() {
  325. defer r.chunkCh.Close()
  326. for {
  327. select {
  328. case envelope := <-r.chunkCh.In:
  329. if err := r.handleMessage(r.chunkCh.ID, envelope); err != nil {
  330. r.Logger.Error("failed to process message", "ch_id", r.chunkCh.ID, "envelope", envelope, "err", err)
  331. r.chunkCh.Error <- p2p.PeerError{
  332. NodeID: envelope.From,
  333. Err: err,
  334. }
  335. }
  336. case <-r.closeCh:
  337. r.Logger.Debug("stopped listening on chunk channel; closing...")
  338. return
  339. }
  340. }
  341. }
  342. // processPeerUpdate processes a PeerUpdate, returning an error upon failing to
  343. // handle the PeerUpdate or if a panic is recovered.
  344. func (r *Reactor) processPeerUpdate(peerUpdate p2p.PeerUpdate) {
  345. r.Logger.Debug("received peer update", "peer", peerUpdate.NodeID, "status", peerUpdate.Status)
  346. r.mtx.RLock()
  347. defer r.mtx.RUnlock()
  348. if r.syncer != nil {
  349. switch peerUpdate.Status {
  350. case p2p.PeerStatusUp:
  351. r.syncer.AddPeer(peerUpdate.NodeID)
  352. case p2p.PeerStatusDown:
  353. r.syncer.RemovePeer(peerUpdate.NodeID)
  354. }
  355. }
  356. }
  357. // processPeerUpdates initiates a blocking process where we listen for and handle
  358. // PeerUpdate messages. When the reactor is stopped, we will catch the signal and
  359. // close the p2p PeerUpdatesCh gracefully.
  360. func (r *Reactor) processPeerUpdates() {
  361. defer r.peerUpdates.Close()
  362. for {
  363. select {
  364. case peerUpdate := <-r.peerUpdates.Updates():
  365. r.processPeerUpdate(peerUpdate)
  366. case <-r.closeCh:
  367. r.Logger.Debug("stopped listening on peer updates channel; closing...")
  368. return
  369. }
  370. }
  371. }
  372. // recentSnapshots fetches the n most recent snapshots from the app
  373. func (r *Reactor) recentSnapshots(n uint32) ([]*snapshot, error) {
  374. resp, err := r.conn.ListSnapshotsSync(context.Background(), abci.RequestListSnapshots{})
  375. if err != nil {
  376. return nil, err
  377. }
  378. sort.Slice(resp.Snapshots, func(i, j int) bool {
  379. a := resp.Snapshots[i]
  380. b := resp.Snapshots[j]
  381. switch {
  382. case a.Height > b.Height:
  383. return true
  384. case a.Height == b.Height && a.Format > b.Format:
  385. return true
  386. default:
  387. return false
  388. }
  389. })
  390. snapshots := make([]*snapshot, 0, n)
  391. for i, s := range resp.Snapshots {
  392. if i >= recentSnapshots {
  393. break
  394. }
  395. snapshots = append(snapshots, &snapshot{
  396. Height: s.Height,
  397. Format: s.Format,
  398. Chunks: s.Chunks,
  399. Hash: s.Hash,
  400. Metadata: s.Metadata,
  401. })
  402. }
  403. return snapshots, nil
  404. }
  405. // Sync runs a state sync, returning the new state and last commit at the snapshot height.
  406. // The caller must store the state and commit in the state database and block store.
  407. func (r *Reactor) Sync(stateProvider StateProvider, discoveryTime time.Duration) (sm.State, *types.Commit, error) {
  408. r.mtx.Lock()
  409. if r.syncer != nil {
  410. r.mtx.Unlock()
  411. return sm.State{}, nil, errors.New("a state sync is already in progress")
  412. }
  413. r.syncer = newSyncer(r.Logger, r.conn, r.connQuery, stateProvider, r.snapshotCh.Out, r.chunkCh.Out, r.tempDir)
  414. r.mtx.Unlock()
  415. hook := func() {
  416. // request snapshots from all currently connected peers
  417. r.Logger.Debug("requesting snapshots from known peers")
  418. r.snapshotCh.Out <- p2p.Envelope{
  419. Broadcast: true,
  420. Message: &ssproto.SnapshotsRequest{},
  421. }
  422. }
  423. hook()
  424. state, commit, err := r.syncer.SyncAny(discoveryTime, hook)
  425. r.mtx.Lock()
  426. r.syncer = nil
  427. r.mtx.Unlock()
  428. return state, commit, err
  429. }