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.

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