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.

580 lines
18 KiB

  1. package statesync
  2. import (
  3. "bytes"
  4. "context"
  5. "errors"
  6. "fmt"
  7. "sync"
  8. "time"
  9. abciclient "github.com/tendermint/tendermint/abci/client"
  10. abci "github.com/tendermint/tendermint/abci/types"
  11. "github.com/tendermint/tendermint/config"
  12. "github.com/tendermint/tendermint/internal/p2p"
  13. "github.com/tendermint/tendermint/internal/proxy"
  14. sm "github.com/tendermint/tendermint/internal/state"
  15. "github.com/tendermint/tendermint/libs/log"
  16. "github.com/tendermint/tendermint/light"
  17. ssproto "github.com/tendermint/tendermint/proto/tendermint/statesync"
  18. "github.com/tendermint/tendermint/types"
  19. )
  20. const (
  21. // chunkTimeout is the timeout while waiting for the next chunk from the chunk queue.
  22. chunkTimeout = 2 * time.Minute
  23. // minimumDiscoveryTime is the lowest allowable time for a
  24. // SyncAny discovery time.
  25. minimumDiscoveryTime = 5 * time.Second
  26. )
  27. var (
  28. // errAbort is returned by Sync() when snapshot restoration is aborted.
  29. errAbort = errors.New("state sync aborted")
  30. // errRetrySnapshot is returned by Sync() when the snapshot should be retried.
  31. errRetrySnapshot = errors.New("retry snapshot")
  32. // errRejectSnapshot is returned by Sync() when the snapshot is rejected.
  33. errRejectSnapshot = errors.New("snapshot was rejected")
  34. // errRejectFormat is returned by Sync() when the snapshot format is rejected.
  35. errRejectFormat = errors.New("snapshot format was rejected")
  36. // errRejectSender is returned by Sync() when the snapshot sender is rejected.
  37. errRejectSender = errors.New("snapshot sender was rejected")
  38. // errVerifyFailed is returned by Sync() when app hash or last height
  39. // verification fails.
  40. errVerifyFailed = errors.New("verification with app 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 abciclient.Client
  53. snapshots *snapshotPool
  54. snapshotCh *p2p.Channel
  55. chunkCh *p2p.Channel
  56. tempDir string
  57. fetchers int32
  58. retryTimeout time.Duration
  59. mtx sync.RWMutex
  60. chunks *chunkQueue
  61. metrics *Metrics
  62. avgChunkTime int64
  63. lastSyncedSnapshotHeight int64
  64. processingSnapshot *snapshot
  65. }
  66. // newSyncer creates a new syncer.
  67. func newSyncer(
  68. cfg config.StateSyncConfig,
  69. logger log.Logger,
  70. conn abciclient.Client,
  71. stateProvider StateProvider,
  72. snapshotCh *p2p.Channel,
  73. chunkCh *p2p.Channel,
  74. tempDir string,
  75. metrics *Metrics,
  76. ) *syncer {
  77. return &syncer{
  78. logger: logger,
  79. stateProvider: stateProvider,
  80. conn: conn,
  81. snapshots: newSnapshotPool(),
  82. snapshotCh: snapshotCh,
  83. chunkCh: chunkCh,
  84. tempDir: tempDir,
  85. fetchers: cfg.Fetchers,
  86. retryTimeout: cfg.ChunkRequestTimeout,
  87. metrics: metrics,
  88. }
  89. }
  90. // AddChunk adds a chunk to the chunk queue, if any. It returns false if the chunk has already
  91. // been added to the queue, or an error if there's no sync in progress.
  92. func (s *syncer) AddChunk(chunk *chunk) (bool, error) {
  93. s.mtx.RLock()
  94. defer s.mtx.RUnlock()
  95. if s.chunks == nil {
  96. return false, errors.New("no state sync in progress")
  97. }
  98. added, err := s.chunks.Add(chunk)
  99. if err != nil {
  100. return false, err
  101. }
  102. if added {
  103. s.logger.Debug("Added chunk to queue", "height", chunk.Height, "format", chunk.Format,
  104. "chunk", chunk.Index)
  105. } else {
  106. s.logger.Debug("Ignoring duplicate chunk in queue", "height", chunk.Height, "format", chunk.Format,
  107. "chunk", chunk.Index)
  108. }
  109. return added, nil
  110. }
  111. // AddSnapshot adds a snapshot to the snapshot pool. It returns true if a new, previously unseen
  112. // snapshot was accepted and added.
  113. func (s *syncer) AddSnapshot(peerID types.NodeID, snapshot *snapshot) (bool, error) {
  114. added, err := s.snapshots.Add(peerID, snapshot)
  115. if err != nil {
  116. return false, err
  117. }
  118. if added {
  119. s.metrics.TotalSnapshots.Add(1)
  120. s.logger.Info("Discovered new snapshot", "height", snapshot.Height, "format", snapshot.Format,
  121. "hash", snapshot.Hash)
  122. }
  123. return added, nil
  124. }
  125. // AddPeer adds a peer to the pool. For now we just keep it simple and send a
  126. // single request to discover snapshots, later we may want to do retries and stuff.
  127. func (s *syncer) AddPeer(ctx context.Context, peerID types.NodeID) error {
  128. s.logger.Debug("Requesting snapshots from peer", "peer", peerID)
  129. return s.snapshotCh.Send(ctx, p2p.Envelope{
  130. To: peerID,
  131. Message: &ssproto.SnapshotsRequest{},
  132. })
  133. }
  134. // RemovePeer removes a peer from the pool.
  135. func (s *syncer) RemovePeer(peerID types.NodeID) {
  136. s.logger.Debug("Removing peer from sync", "peer", peerID)
  137. s.snapshots.RemovePeer(peerID)
  138. }
  139. // SyncAny tries to sync any of the snapshots in the snapshot pool, waiting to discover further
  140. // snapshots if none were found and discoveryTime > 0. It returns the latest state and block commit
  141. // which the caller must use to bootstrap the node.
  142. func (s *syncer) SyncAny(
  143. ctx context.Context,
  144. discoveryTime time.Duration,
  145. requestSnapshots func() error,
  146. ) (sm.State, *types.Commit, error) {
  147. if discoveryTime != 0 && discoveryTime < minimumDiscoveryTime {
  148. discoveryTime = minimumDiscoveryTime
  149. }
  150. if discoveryTime > 0 {
  151. if err := requestSnapshots(); err != nil {
  152. return sm.State{}, nil, err
  153. }
  154. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  155. time.Sleep(discoveryTime)
  156. }
  157. // The app may ask us to retry a snapshot restoration, in which case we need to reuse
  158. // the snapshot and chunk queue from the previous loop iteration.
  159. var (
  160. snapshot *snapshot
  161. chunks *chunkQueue
  162. err error
  163. )
  164. for {
  165. // If not nil, we're going to retry restoration of the same snapshot.
  166. if snapshot == nil {
  167. snapshot = s.snapshots.Best()
  168. chunks = nil
  169. }
  170. if snapshot == nil {
  171. if discoveryTime == 0 {
  172. return sm.State{}, nil, errNoSnapshots
  173. }
  174. s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
  175. time.Sleep(discoveryTime)
  176. continue
  177. }
  178. if chunks == nil {
  179. chunks, err = newChunkQueue(snapshot, s.tempDir)
  180. if err != nil {
  181. return sm.State{}, nil, fmt.Errorf("failed to create chunk queue: %w", err)
  182. }
  183. defer chunks.Close() // in case we forget to close it elsewhere
  184. }
  185. s.processingSnapshot = snapshot
  186. s.metrics.SnapshotChunkTotal.Set(float64(snapshot.Chunks))
  187. newState, commit, err := s.Sync(ctx, snapshot, chunks)
  188. switch {
  189. case err == nil:
  190. s.metrics.SnapshotHeight.Set(float64(snapshot.Height))
  191. s.lastSyncedSnapshotHeight = int64(snapshot.Height)
  192. return newState, commit, nil
  193. case errors.Is(err, errAbort):
  194. return sm.State{}, nil, err
  195. case errors.Is(err, errRetrySnapshot):
  196. chunks.RetryAll()
  197. s.logger.Info("Retrying snapshot", "height", snapshot.Height, "format", snapshot.Format,
  198. "hash", snapshot.Hash)
  199. continue
  200. case errors.Is(err, errTimeout):
  201. s.snapshots.Reject(snapshot)
  202. s.logger.Error("Timed out waiting for snapshot chunks, rejected snapshot",
  203. "height", snapshot.Height, "format", snapshot.Format, "hash", snapshot.Hash)
  204. case errors.Is(err, errRejectSnapshot):
  205. s.snapshots.Reject(snapshot)
  206. s.logger.Info("Snapshot rejected", "height", snapshot.Height, "format", snapshot.Format,
  207. "hash", snapshot.Hash)
  208. case errors.Is(err, errRejectFormat):
  209. s.snapshots.RejectFormat(snapshot.Format)
  210. s.logger.Info("Snapshot format rejected", "format", snapshot.Format)
  211. case errors.Is(err, errRejectSender):
  212. s.logger.Info("Snapshot senders rejected", "height", snapshot.Height, "format", snapshot.Format,
  213. "hash", snapshot.Hash)
  214. for _, peer := range s.snapshots.GetPeers(snapshot) {
  215. s.snapshots.RejectPeer(peer)
  216. s.logger.Info("Snapshot sender rejected", "peer", peer)
  217. }
  218. default:
  219. return sm.State{}, nil, fmt.Errorf("snapshot restoration failed: %w", err)
  220. }
  221. // Discard snapshot and chunks for next iteration
  222. err = chunks.Close()
  223. if err != nil {
  224. s.logger.Error("Failed to clean up chunk queue", "err", err)
  225. }
  226. snapshot = nil
  227. chunks = nil
  228. s.processingSnapshot = nil
  229. }
  230. }
  231. // Sync executes a sync for a specific snapshot, returning the latest state and block commit which
  232. // the caller must use to bootstrap the node.
  233. func (s *syncer) Sync(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) (sm.State, *types.Commit, error) {
  234. s.mtx.Lock()
  235. if s.chunks != nil {
  236. s.mtx.Unlock()
  237. return sm.State{}, nil, errors.New("a state sync is already in progress")
  238. }
  239. s.chunks = chunks
  240. s.mtx.Unlock()
  241. defer func() {
  242. s.mtx.Lock()
  243. s.chunks = nil
  244. s.mtx.Unlock()
  245. }()
  246. hctx, hcancel := context.WithTimeout(ctx, 30*time.Second)
  247. defer hcancel()
  248. // Fetch the app hash corresponding to the snapshot
  249. appHash, err := s.stateProvider.AppHash(hctx, snapshot.Height)
  250. if err != nil {
  251. // check if the main context was triggered
  252. if ctx.Err() != nil {
  253. return sm.State{}, nil, ctx.Err()
  254. }
  255. // catch the case where all the light client providers have been exhausted
  256. if err == light.ErrNoWitnesses {
  257. return sm.State{}, nil,
  258. fmt.Errorf("failed to get app hash at height %d. No witnesses remaining", snapshot.Height)
  259. }
  260. s.logger.Info("failed to get and verify tendermint state. Dropping snapshot and trying again",
  261. "err", err, "height", snapshot.Height)
  262. return sm.State{}, nil, errRejectSnapshot
  263. }
  264. snapshot.trustedAppHash = appHash
  265. // Offer snapshot to ABCI app.
  266. err = s.offerSnapshot(ctx, snapshot)
  267. if err != nil {
  268. return sm.State{}, nil, err
  269. }
  270. // Spawn chunk fetchers. They will terminate when the chunk queue is closed or context canceled.
  271. fetchCtx, cancel := context.WithCancel(ctx)
  272. defer cancel()
  273. fetchStartTime := time.Now()
  274. for i := int32(0); i < s.fetchers; i++ {
  275. go s.fetchChunks(fetchCtx, snapshot, chunks)
  276. }
  277. pctx, pcancel := context.WithTimeout(ctx, 1*time.Minute)
  278. defer pcancel()
  279. // Optimistically build new state, so we don't discover any light client failures at the end.
  280. state, err := s.stateProvider.State(pctx, snapshot.Height)
  281. if err != nil {
  282. // check if the main context was triggered
  283. if ctx.Err() != nil {
  284. return sm.State{}, nil, ctx.Err()
  285. }
  286. if err == light.ErrNoWitnesses {
  287. return sm.State{}, nil,
  288. fmt.Errorf("failed to get tendermint state at height %d. No witnesses remaining", snapshot.Height)
  289. }
  290. s.logger.Info("failed to get and verify tendermint state. Dropping snapshot and trying again",
  291. "err", err, "height", snapshot.Height)
  292. return sm.State{}, nil, errRejectSnapshot
  293. }
  294. commit, err := s.stateProvider.Commit(pctx, snapshot.Height)
  295. if err != nil {
  296. // check if the provider context exceeded the 10 second deadline
  297. if ctx.Err() != nil {
  298. return sm.State{}, nil, ctx.Err()
  299. }
  300. if err == light.ErrNoWitnesses {
  301. return sm.State{}, nil,
  302. fmt.Errorf("failed to get commit at height %d. No witnesses remaining", snapshot.Height)
  303. }
  304. s.logger.Info("failed to get and verify commit. Dropping snapshot and trying again",
  305. "err", err, "height", snapshot.Height)
  306. return sm.State{}, nil, errRejectSnapshot
  307. }
  308. // Restore snapshot
  309. err = s.applyChunks(ctx, chunks, fetchStartTime)
  310. if err != nil {
  311. return sm.State{}, nil, err
  312. }
  313. // Verify app and app version
  314. if err := s.verifyApp(ctx, snapshot, state.Version.Consensus.App); err != nil {
  315. return sm.State{}, nil, err
  316. }
  317. // Done! 🎉
  318. s.logger.Info("Snapshot restored", "height", snapshot.Height, "format", snapshot.Format,
  319. "hash", snapshot.Hash)
  320. return state, commit, nil
  321. }
  322. // offerSnapshot offers a snapshot to the app. It returns various errors depending on the app's
  323. // response, or nil if the snapshot was accepted.
  324. func (s *syncer) offerSnapshot(ctx context.Context, snapshot *snapshot) error {
  325. s.logger.Info("Offering snapshot to ABCI app", "height", snapshot.Height,
  326. "format", snapshot.Format, "hash", snapshot.Hash)
  327. resp, err := s.conn.OfferSnapshot(ctx, abci.RequestOfferSnapshot{
  328. Snapshot: &abci.Snapshot{
  329. Height: snapshot.Height,
  330. Format: snapshot.Format,
  331. Chunks: snapshot.Chunks,
  332. Hash: snapshot.Hash,
  333. Metadata: snapshot.Metadata,
  334. },
  335. AppHash: snapshot.trustedAppHash,
  336. })
  337. if err != nil {
  338. return fmt.Errorf("failed to offer snapshot: %w", err)
  339. }
  340. switch resp.Result {
  341. case abci.ResponseOfferSnapshot_ACCEPT:
  342. s.logger.Info("Snapshot accepted, restoring", "height", snapshot.Height,
  343. "format", snapshot.Format, "hash", snapshot.Hash)
  344. return nil
  345. case abci.ResponseOfferSnapshot_ABORT:
  346. return errAbort
  347. case abci.ResponseOfferSnapshot_REJECT:
  348. return errRejectSnapshot
  349. case abci.ResponseOfferSnapshot_REJECT_FORMAT:
  350. return errRejectFormat
  351. case abci.ResponseOfferSnapshot_REJECT_SENDER:
  352. return errRejectSender
  353. default:
  354. return fmt.Errorf("unknown ResponseOfferSnapshot result %v", resp.Result)
  355. }
  356. }
  357. // applyChunks applies chunks to the app. It returns various errors depending on the app's
  358. // response, or nil once the snapshot is fully restored.
  359. func (s *syncer) applyChunks(ctx context.Context, chunks *chunkQueue, start time.Time) error {
  360. for {
  361. chunk, err := chunks.Next()
  362. if err == errDone {
  363. return nil
  364. } else if err != nil {
  365. return fmt.Errorf("failed to fetch chunk: %w", err)
  366. }
  367. resp, err := s.conn.ApplySnapshotChunk(ctx, abci.RequestApplySnapshotChunk{
  368. Index: chunk.Index,
  369. Chunk: chunk.Chunk,
  370. Sender: string(chunk.Sender),
  371. })
  372. if err != nil {
  373. return fmt.Errorf("failed to apply chunk %v: %w", chunk.Index, err)
  374. }
  375. s.logger.Info("Applied snapshot chunk to ABCI app", "height", chunk.Height,
  376. "format", chunk.Format, "chunk", chunk.Index, "total", chunks.Size())
  377. // Discard and refetch any chunks as requested by the app
  378. for _, index := range resp.RefetchChunks {
  379. err := chunks.Discard(index)
  380. if err != nil {
  381. return fmt.Errorf("failed to discard chunk %v: %w", index, err)
  382. }
  383. }
  384. // Reject any senders as requested by the app
  385. for _, sender := range resp.RejectSenders {
  386. if sender != "" {
  387. peerID := types.NodeID(sender)
  388. s.snapshots.RejectPeer(peerID)
  389. if err := chunks.DiscardSender(peerID); err != nil {
  390. return fmt.Errorf("failed to reject sender: %w", err)
  391. }
  392. }
  393. }
  394. switch resp.Result {
  395. case abci.ResponseApplySnapshotChunk_ACCEPT:
  396. s.metrics.SnapshotChunk.Add(1)
  397. s.avgChunkTime = time.Since(start).Nanoseconds() / int64(chunks.numChunksReturned())
  398. s.metrics.ChunkProcessAvgTime.Set(float64(s.avgChunkTime))
  399. case abci.ResponseApplySnapshotChunk_ABORT:
  400. return errAbort
  401. case abci.ResponseApplySnapshotChunk_RETRY:
  402. chunks.Retry(chunk.Index)
  403. case abci.ResponseApplySnapshotChunk_RETRY_SNAPSHOT:
  404. return errRetrySnapshot
  405. case abci.ResponseApplySnapshotChunk_REJECT_SNAPSHOT:
  406. return errRejectSnapshot
  407. default:
  408. return fmt.Errorf("unknown ResponseApplySnapshotChunk result %v", resp.Result)
  409. }
  410. }
  411. }
  412. // fetchChunks requests chunks from peers, receiving allocations from the chunk queue. Chunks
  413. // will be received from the reactor via syncer.AddChunks() to chunkQueue.Add().
  414. func (s *syncer) fetchChunks(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) {
  415. var (
  416. next = true
  417. index uint32
  418. err error
  419. )
  420. for {
  421. if next {
  422. index, err = chunks.Allocate()
  423. if errors.Is(err, errDone) {
  424. // Keep checking until the context is canceled (restore is done), in case any
  425. // chunks need to be refetched.
  426. select {
  427. case <-ctx.Done():
  428. return
  429. case <-time.After(2 * time.Second):
  430. continue
  431. }
  432. }
  433. if err != nil {
  434. s.logger.Error("Failed to allocate chunk from queue", "err", err)
  435. return
  436. }
  437. }
  438. s.logger.Info("Fetching snapshot chunk", "height", snapshot.Height,
  439. "format", snapshot.Format, "chunk", index, "total", chunks.Size())
  440. ticker := time.NewTicker(s.retryTimeout)
  441. defer ticker.Stop()
  442. if err := s.requestChunk(ctx, snapshot, index); err != nil {
  443. return
  444. }
  445. select {
  446. case <-chunks.WaitFor(index):
  447. next = true
  448. case <-ticker.C:
  449. next = false
  450. case <-ctx.Done():
  451. return
  452. }
  453. ticker.Stop()
  454. }
  455. }
  456. // requestChunk requests a chunk from a peer.
  457. //
  458. // returns nil if there are no peers for the given snapshot or the
  459. // request is successfully made and an error if the request cannot be
  460. // completed
  461. func (s *syncer) requestChunk(ctx context.Context, snapshot *snapshot, chunk uint32) error {
  462. peer := s.snapshots.GetPeer(snapshot)
  463. if peer == "" {
  464. s.logger.Error("No valid peers found for snapshot", "height", snapshot.Height,
  465. "format", snapshot.Format, "hash", snapshot.Hash)
  466. return nil
  467. }
  468. s.logger.Debug(
  469. "Requesting snapshot chunk",
  470. "height", snapshot.Height,
  471. "format", snapshot.Format,
  472. "chunk", chunk,
  473. "peer", peer,
  474. )
  475. msg := p2p.Envelope{
  476. To: peer,
  477. Message: &ssproto.ChunkRequest{
  478. Height: snapshot.Height,
  479. Format: snapshot.Format,
  480. Index: chunk,
  481. },
  482. }
  483. if err := s.chunkCh.Send(ctx, msg); err != nil {
  484. return err
  485. }
  486. return nil
  487. }
  488. // verifyApp verifies the sync, checking the app hash, last block height and app version
  489. func (s *syncer) verifyApp(ctx context.Context, snapshot *snapshot, appVersion uint64) error {
  490. resp, err := s.conn.Info(ctx, proxy.RequestInfo)
  491. if err != nil {
  492. return fmt.Errorf("failed to query ABCI app for appHash: %w", err)
  493. }
  494. // sanity check that the app version in the block matches the application's own record
  495. // of its version
  496. if resp.AppVersion != appVersion {
  497. // An error here most likely means that the app hasn't inplemented state sync
  498. // or the Info call correctly
  499. return fmt.Errorf("app version mismatch. Expected: %d, got: %d",
  500. appVersion, resp.AppVersion)
  501. }
  502. if !bytes.Equal(snapshot.trustedAppHash, resp.LastBlockAppHash) {
  503. s.logger.Error("appHash verification failed",
  504. "expected", snapshot.trustedAppHash,
  505. "actual", resp.LastBlockAppHash)
  506. return errVerifyFailed
  507. }
  508. if uint64(resp.LastBlockHeight) != snapshot.Height {
  509. s.logger.Error(
  510. "ABCI app reported unexpected last block height",
  511. "expected", snapshot.Height,
  512. "actual", resp.LastBlockHeight,
  513. )
  514. return errVerifyFailed
  515. }
  516. s.logger.Info("Verified ABCI app", "height", snapshot.Height, "appHash", snapshot.trustedAppHash)
  517. return nil
  518. }