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.

582 lines
18 KiB

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