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.

480 lines
15 KiB

  1. package statesync
  2. import (
  3. "bytes"
  4. "context"
  5. "errors"
  6. "fmt"
  7. "time"
  8. abci "github.com/tendermint/tendermint/abci/types"
  9. "github.com/tendermint/tendermint/libs/log"
  10. tmsync "github.com/tendermint/tendermint/libs/sync"
  11. "github.com/tendermint/tendermint/p2p"
  12. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  13. "github.com/tendermint/tendermint/proxy"
  14. sm "github.com/tendermint/tendermint/state"
  15. "github.com/tendermint/tendermint/types"
  16. )
  17. const (
  18. // chunkFetchers is the number of concurrent chunk fetchers to run.
  19. chunkFetchers = 4
  20. // chunkTimeout is the timeout while waiting for the next chunk from the chunk queue.
  21. chunkTimeout = 2 * time.Minute
  22. // requestTimeout is the timeout before rerequesting a chunk, possibly from a different peer.
  23. chunkRequestTimeout = 10 * time.Second
  24. // minimumDiscoveryTime is the lowest allowable time for a
  25. // SyncAny discovery time.
  26. minimumDiscoveryTime = 5 * time.Second
  27. )
  28. var (
  29. // errAbort is returned by Sync() when snapshot restoration is aborted.
  30. errAbort = errors.New("state sync aborted")
  31. // errRetrySnapshot is returned by Sync() when the snapshot should be retried.
  32. errRetrySnapshot = errors.New("retry snapshot")
  33. // errRejectSnapshot is returned by Sync() when the snapshot is rejected.
  34. errRejectSnapshot = errors.New("snapshot was rejected")
  35. // errRejectFormat is returned by Sync() when the snapshot format is rejected.
  36. errRejectFormat = errors.New("snapshot format was rejected")
  37. // errRejectSender is returned by Sync() when the snapshot sender is rejected.
  38. errRejectSender = errors.New("snapshot sender was rejected")
  39. // errVerifyFailed is returned by Sync() when app hash or last height verification fails.
  40. errVerifyFailed = errors.New("verification failed")
  41. // errTimeout is returned by Sync() when we've waited too long to receive a chunk.
  42. errTimeout = errors.New("timed out waiting for chunk")
  43. // errNoSnapshots is returned by SyncAny() if no snapshots are found and discovery is disabled.
  44. errNoSnapshots = errors.New("no suitable snapshots found")
  45. )
  46. // syncer runs a state sync against an ABCI app. Use either SyncAny() to automatically attempt to
  47. // sync all snapshots in the pool (pausing to discover new ones), or Sync() to sync a specific
  48. // snapshot. Snapshots and chunks are fed via AddSnapshot() and AddChunk() as appropriate.
  49. type syncer struct {
  50. logger log.Logger
  51. stateProvider StateProvider
  52. conn proxy.AppConnSnapshot
  53. connQuery proxy.AppConnQuery
  54. snapshots *snapshotPool
  55. snapshotCh chan<- p2p.Envelope
  56. chunkCh chan<- p2p.Envelope
  57. tempDir string
  58. mtx tmsync.RWMutex
  59. chunks *chunkQueue
  60. }
  61. // newSyncer creates a new syncer.
  62. func newSyncer(
  63. logger log.Logger,
  64. conn proxy.AppConnSnapshot,
  65. connQuery proxy.AppConnQuery,
  66. stateProvider StateProvider,
  67. snapshotCh, chunkCh chan<- p2p.Envelope,
  68. tempDir string,
  69. ) *syncer {
  70. return &syncer{
  71. logger: logger,
  72. stateProvider: stateProvider,
  73. conn: conn,
  74. connQuery: connQuery,
  75. snapshots: newSnapshotPool(stateProvider),
  76. snapshotCh: snapshotCh,
  77. chunkCh: chunkCh,
  78. tempDir: tempDir,
  79. }
  80. }
  81. // AddChunk adds a chunk to the chunk queue, if any. It returns false if the chunk has already
  82. // been added to the queue, or an error if there's no sync in progress.
  83. func (s *syncer) AddChunk(chunk *chunk) (bool, error) {
  84. s.mtx.RLock()
  85. defer s.mtx.RUnlock()
  86. if s.chunks == nil {
  87. return false, errors.New("no state sync in progress")
  88. }
  89. added, err := s.chunks.Add(chunk)
  90. if err != nil {
  91. return false, err
  92. }
  93. if added {
  94. s.logger.Debug("Added chunk to queue", "height", chunk.Height, "format", chunk.Format,
  95. "chunk", chunk.Index)
  96. } else {
  97. s.logger.Debug("Ignoring duplicate chunk in queue", "height", chunk.Height, "format", chunk.Format,
  98. "chunk", chunk.Index)
  99. }
  100. return added, nil
  101. }
  102. // AddSnapshot adds a snapshot to the snapshot pool. It returns true if a new, previously unseen
  103. // snapshot was accepted and added.
  104. func (s *syncer) AddSnapshot(peerID p2p.NodeID, snapshot *snapshot) (bool, error) {
  105. added, err := s.snapshots.Add(peerID, snapshot)
  106. if err != nil {
  107. return false, err
  108. }
  109. if added {
  110. s.logger.Info("Discovered new snapshot", "height", snapshot.Height, "format", snapshot.Format,
  111. "hash", snapshot.Hash)
  112. }
  113. return added, nil
  114. }
  115. // AddPeer adds a peer to the pool. For now we just keep it simple and send a
  116. // single request to discover snapshots, later we may want to do retries and stuff.
  117. func (s *syncer) AddPeer(peerID p2p.NodeID) {
  118. s.logger.Debug("Requesting snapshots from peer", "peer", peerID)
  119. s.snapshotCh <- p2p.Envelope{
  120. To: peerID,
  121. Message: &ssproto.SnapshotsRequest{},
  122. }
  123. }
  124. // RemovePeer removes a peer from the pool.
  125. func (s *syncer) RemovePeer(peerID p2p.NodeID) {
  126. s.logger.Debug("Removing peer from sync", "peer", peerID)
  127. s.snapshots.RemovePeer(peerID)
  128. }
  129. // SyncAny tries to sync any of the snapshots in the snapshot pool, waiting to discover further
  130. // snapshots if none were found and discoveryTime > 0. It returns the latest state and block commit
  131. // which the caller must use to bootstrap the node.
  132. func (s *syncer) SyncAny(discoveryTime time.Duration, retryHook func()) (sm.State, *types.Commit, error) {
  133. if discoveryTime != 0 && discoveryTime < minimumDiscoveryTime {
  134. discoveryTime = 5 * minimumDiscoveryTime
  135. }
  136. if discoveryTime > 0 {
  137. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  138. time.Sleep(discoveryTime)
  139. }
  140. // The app may ask us to retry a snapshot restoration, in which case we need to reuse
  141. // the snapshot and chunk queue from the previous loop iteration.
  142. var (
  143. snapshot *snapshot
  144. chunks *chunkQueue
  145. err error
  146. )
  147. for {
  148. // If not nil, we're going to retry restoration of the same snapshot.
  149. if snapshot == nil {
  150. snapshot = s.snapshots.Best()
  151. chunks = nil
  152. }
  153. if snapshot == nil {
  154. if discoveryTime == 0 {
  155. return sm.State{}, nil, errNoSnapshots
  156. }
  157. retryHook()
  158. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  159. time.Sleep(discoveryTime)
  160. continue
  161. }
  162. if chunks == nil {
  163. chunks, err = newChunkQueue(snapshot, s.tempDir)
  164. if err != nil {
  165. return sm.State{}, nil, fmt.Errorf("failed to create chunk queue: %w", err)
  166. }
  167. defer chunks.Close() // in case we forget to close it elsewhere
  168. }
  169. newState, commit, err := s.Sync(snapshot, chunks)
  170. switch {
  171. case err == nil:
  172. return newState, commit, nil
  173. case errors.Is(err, errAbort):
  174. return sm.State{}, nil, err
  175. case errors.Is(err, errRetrySnapshot):
  176. chunks.RetryAll()
  177. s.logger.Info("Retrying snapshot", "height", snapshot.Height, "format", snapshot.Format,
  178. "hash", snapshot.Hash)
  179. continue
  180. case errors.Is(err, errTimeout):
  181. s.snapshots.Reject(snapshot)
  182. s.logger.Error("Timed out waiting for snapshot chunks, rejected snapshot",
  183. "height", snapshot.Height, "format", snapshot.Format, "hash", snapshot.Hash)
  184. case errors.Is(err, errRejectSnapshot):
  185. s.snapshots.Reject(snapshot)
  186. s.logger.Info("Snapshot rejected", "height", snapshot.Height, "format", snapshot.Format,
  187. "hash", snapshot.Hash)
  188. case errors.Is(err, errRejectFormat):
  189. s.snapshots.RejectFormat(snapshot.Format)
  190. s.logger.Info("Snapshot format rejected", "format", snapshot.Format)
  191. case errors.Is(err, errRejectSender):
  192. s.logger.Info("Snapshot senders rejected", "height", snapshot.Height, "format", snapshot.Format,
  193. "hash", snapshot.Hash)
  194. for _, peer := range s.snapshots.GetPeers(snapshot) {
  195. s.snapshots.RejectPeer(peer)
  196. s.logger.Info("Snapshot sender rejected", "peer", peer)
  197. }
  198. default:
  199. return sm.State{}, nil, fmt.Errorf("snapshot restoration failed: %w", err)
  200. }
  201. // Discard snapshot and chunks for next iteration
  202. err = chunks.Close()
  203. if err != nil {
  204. s.logger.Error("Failed to clean up chunk queue", "err", err)
  205. }
  206. snapshot = nil
  207. chunks = nil
  208. }
  209. }
  210. // Sync executes a sync for a specific snapshot, returning the latest state and block commit which
  211. // the caller must use to bootstrap the node.
  212. func (s *syncer) Sync(snapshot *snapshot, chunks *chunkQueue) (sm.State, *types.Commit, error) {
  213. s.mtx.Lock()
  214. if s.chunks != nil {
  215. s.mtx.Unlock()
  216. return sm.State{}, nil, errors.New("a state sync is already in progress")
  217. }
  218. s.chunks = chunks
  219. s.mtx.Unlock()
  220. defer func() {
  221. s.mtx.Lock()
  222. s.chunks = nil
  223. s.mtx.Unlock()
  224. }()
  225. // Offer snapshot to ABCI app.
  226. err := s.offerSnapshot(snapshot)
  227. if err != nil {
  228. return sm.State{}, nil, err
  229. }
  230. // Spawn chunk fetchers. They will terminate when the chunk queue is closed or context canceled.
  231. ctx, cancel := context.WithCancel(context.Background())
  232. defer cancel()
  233. for i := int32(0); i < chunkFetchers; i++ {
  234. go s.fetchChunks(ctx, snapshot, chunks)
  235. }
  236. pctx, pcancel := context.WithTimeout(context.Background(), 10*time.Second)
  237. defer pcancel()
  238. // Optimistically build new state, so we don't discover any light client failures at the end.
  239. state, err := s.stateProvider.State(pctx, snapshot.Height)
  240. if err != nil {
  241. return sm.State{}, nil, fmt.Errorf("failed to build new state: %w", err)
  242. }
  243. commit, err := s.stateProvider.Commit(pctx, snapshot.Height)
  244. if err != nil {
  245. return sm.State{}, nil, fmt.Errorf("failed to fetch commit: %w", err)
  246. }
  247. // Restore snapshot
  248. err = s.applyChunks(chunks)
  249. if err != nil {
  250. return sm.State{}, nil, err
  251. }
  252. // Verify app and update app version
  253. appVersion, err := s.verifyApp(snapshot)
  254. if err != nil {
  255. return sm.State{}, nil, err
  256. }
  257. state.Version.Consensus.App = appVersion
  258. // Done! 🎉
  259. s.logger.Info("Snapshot restored", "height", snapshot.Height, "format", snapshot.Format,
  260. "hash", snapshot.Hash)
  261. return state, commit, nil
  262. }
  263. // offerSnapshot offers a snapshot to the app. It returns various errors depending on the app's
  264. // response, or nil if the snapshot was accepted.
  265. func (s *syncer) offerSnapshot(snapshot *snapshot) error {
  266. s.logger.Info("Offering snapshot to ABCI app", "height", snapshot.Height,
  267. "format", snapshot.Format, "hash", snapshot.Hash)
  268. resp, err := s.conn.OfferSnapshotSync(context.Background(), abci.RequestOfferSnapshot{
  269. Snapshot: &abci.Snapshot{
  270. Height: snapshot.Height,
  271. Format: snapshot.Format,
  272. Chunks: snapshot.Chunks,
  273. Hash: snapshot.Hash,
  274. Metadata: snapshot.Metadata,
  275. },
  276. AppHash: snapshot.trustedAppHash,
  277. })
  278. if err != nil {
  279. return fmt.Errorf("failed to offer snapshot: %w", err)
  280. }
  281. switch resp.Result {
  282. case abci.ResponseOfferSnapshot_ACCEPT:
  283. s.logger.Info("Snapshot accepted, restoring", "height", snapshot.Height,
  284. "format", snapshot.Format, "hash", snapshot.Hash)
  285. return nil
  286. case abci.ResponseOfferSnapshot_ABORT:
  287. return errAbort
  288. case abci.ResponseOfferSnapshot_REJECT:
  289. return errRejectSnapshot
  290. case abci.ResponseOfferSnapshot_REJECT_FORMAT:
  291. return errRejectFormat
  292. case abci.ResponseOfferSnapshot_REJECT_SENDER:
  293. return errRejectSender
  294. default:
  295. return fmt.Errorf("unknown ResponseOfferSnapshot result %v", resp.Result)
  296. }
  297. }
  298. // applyChunks applies chunks to the app. It returns various errors depending on the app's
  299. // response, or nil once the snapshot is fully restored.
  300. func (s *syncer) applyChunks(chunks *chunkQueue) error {
  301. for {
  302. chunk, err := chunks.Next()
  303. if err == errDone {
  304. return nil
  305. } else if err != nil {
  306. return fmt.Errorf("failed to fetch chunk: %w", err)
  307. }
  308. resp, err := s.conn.ApplySnapshotChunkSync(context.Background(), abci.RequestApplySnapshotChunk{
  309. Index: chunk.Index,
  310. Chunk: chunk.Chunk,
  311. Sender: string(chunk.Sender),
  312. })
  313. if err != nil {
  314. return fmt.Errorf("failed to apply chunk %v: %w", chunk.Index, err)
  315. }
  316. s.logger.Info("Applied snapshot chunk to ABCI app", "height", chunk.Height,
  317. "format", chunk.Format, "chunk", chunk.Index, "total", chunks.Size())
  318. // Discard and refetch any chunks as requested by the app
  319. for _, index := range resp.RefetchChunks {
  320. err := chunks.Discard(index)
  321. if err != nil {
  322. return fmt.Errorf("failed to discard chunk %v: %w", index, err)
  323. }
  324. }
  325. // Reject any senders as requested by the app
  326. for _, sender := range resp.RejectSenders {
  327. if sender != "" {
  328. peerID := p2p.NodeID(sender)
  329. s.snapshots.RejectPeer(peerID)
  330. if err := chunks.DiscardSender(peerID); err != nil {
  331. return fmt.Errorf("failed to reject sender: %w", err)
  332. }
  333. }
  334. }
  335. switch resp.Result {
  336. case abci.ResponseApplySnapshotChunk_ACCEPT:
  337. case abci.ResponseApplySnapshotChunk_ABORT:
  338. return errAbort
  339. case abci.ResponseApplySnapshotChunk_RETRY:
  340. chunks.Retry(chunk.Index)
  341. case abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT:
  342. return errRetrySnapshot
  343. case abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT:
  344. return errRejectSnapshot
  345. default:
  346. return fmt.Errorf("unknown ResponseApplySnapshotChunk result %v", resp.Result)
  347. }
  348. }
  349. }
  350. // fetchChunks requests chunks from peers, receiving allocations from the chunk queue. Chunks
  351. // will be received from the reactor via syncer.AddChunks() to chunkQueue.Add().
  352. func (s *syncer) fetchChunks(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) {
  353. for {
  354. index, err := chunks.Allocate()
  355. if err == errDone {
  356. // Keep checking until the context is canceled (restore is done), in case any
  357. // chunks need to be refetched.
  358. select {
  359. case <-ctx.Done():
  360. return
  361. default:
  362. }
  363. time.Sleep(2 * time.Second)
  364. continue
  365. }
  366. if err != nil {
  367. s.logger.Error("Failed to allocate chunk from queue", "err", err)
  368. return
  369. }
  370. s.logger.Info("Fetching snapshot chunk", "height", snapshot.Height,
  371. "format", snapshot.Format, "chunk", index, "total", chunks.Size())
  372. ticker := time.NewTicker(chunkRequestTimeout)
  373. defer ticker.Stop()
  374. s.requestChunk(snapshot, index)
  375. select {
  376. case <-chunks.WaitFor(index):
  377. case <-ticker.C:
  378. s.requestChunk(snapshot, index)
  379. case <-ctx.Done():
  380. return
  381. }
  382. ticker.Stop()
  383. }
  384. }
  385. // requestChunk requests a chunk from a peer.
  386. func (s *syncer) requestChunk(snapshot *snapshot, chunk uint32) {
  387. peer := s.snapshots.GetPeer(snapshot)
  388. if peer == "" {
  389. s.logger.Error("No valid peers found for snapshot", "height", snapshot.Height,
  390. "format", snapshot.Format, "hash", snapshot.Hash)
  391. return
  392. }
  393. s.logger.Debug(
  394. "Requesting snapshot chunk",
  395. "height", snapshot.Height,
  396. "format", snapshot.Format,
  397. "chunk", chunk,
  398. "peer", peer,
  399. )
  400. s.chunkCh <- p2p.Envelope{
  401. To: peer,
  402. Message: &ssproto.ChunkRequest{
  403. Height: snapshot.Height,
  404. Format: snapshot.Format,
  405. Index: chunk,
  406. },
  407. }
  408. }
  409. // verifyApp verifies the sync, checking the app hash and last block height. It returns the
  410. // app version, which should be returned as part of the initial state.
  411. func (s *syncer) verifyApp(snapshot *snapshot) (uint64, error) {
  412. resp, err := s.connQuery.InfoSync(context.Background(), proxy.RequestInfo)
  413. if err != nil {
  414. return 0, fmt.Errorf("failed to query ABCI app for appHash: %w", err)
  415. }
  416. if !bytes.Equal(snapshot.trustedAppHash, resp.LastBlockAppHash) {
  417. s.logger.Error("appHash verification failed",
  418. "expected", snapshot.trustedAppHash,
  419. "actual", resp.LastBlockAppHash)
  420. return 0, errVerifyFailed
  421. }
  422. if uint64(resp.LastBlockHeight) != snapshot.Height {
  423. s.logger.Error(
  424. "ABCI app reported unexpected last block height",
  425. "expected", snapshot.Height,
  426. "actual", resp.LastBlockHeight,
  427. )
  428. return 0, errVerifyFailed
  429. }
  430. s.logger.Info("Verified ABCI app", "height", snapshot.Height, "appHash", snapshot.trustedAppHash)
  431. return resp.AppVersion, nil
  432. }