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.

369 lines
8.9 KiB

  1. package statesync
  2. import (
  3. "errors"
  4. "fmt"
  5. "os"
  6. "path/filepath"
  7. "strconv"
  8. "time"
  9. tmsync "github.com/tendermint/tendermint/internal/libs/sync"
  10. "github.com/tendermint/tendermint/types"
  11. )
  12. // errDone is returned by chunkQueue.Next() when all chunks have been returned.
  13. var errDone = errors.New("chunk queue has completed")
  14. // chunk contains data for a chunk.
  15. type chunk struct {
  16. Height uint64
  17. Format uint32
  18. Index uint32
  19. Chunk []byte
  20. Sender types.NodeID
  21. }
  22. // chunkQueue manages chunks for a state sync process, ordering them if requested. It acts as an
  23. // iterator over all chunks, but callers can request chunks to be retried, optionally after
  24. // refetching.
  25. type chunkQueue struct {
  26. tmsync.Mutex
  27. snapshot *snapshot // if this is nil, the queue has been closed
  28. dir string // temp dir for on-disk chunk storage
  29. chunkFiles map[uint32]string // path to temporary chunk file
  30. chunkSenders map[uint32]types.NodeID // the peer who sent the given chunk
  31. chunkAllocated map[uint32]bool // chunks that have been allocated via Allocate()
  32. chunkReturned map[uint32]bool // chunks returned via Next()
  33. waiters map[uint32][]chan<- uint32 // signals WaitFor() waiters about chunk arrival
  34. }
  35. // newChunkQueue creates a new chunk queue for a snapshot, using a temp dir for storage.
  36. // Callers must call Close() when done.
  37. func newChunkQueue(snapshot *snapshot, tempDir string) (*chunkQueue, error) {
  38. dir, err := os.MkdirTemp(tempDir, "tm-statesync")
  39. if err != nil {
  40. return nil, fmt.Errorf("unable to create temp dir for state sync chunks: %w", err)
  41. }
  42. if snapshot.Chunks == 0 {
  43. return nil, errors.New("snapshot has no chunks")
  44. }
  45. return &chunkQueue{
  46. snapshot: snapshot,
  47. dir: dir,
  48. chunkFiles: make(map[uint32]string, snapshot.Chunks),
  49. chunkSenders: make(map[uint32]types.NodeID, snapshot.Chunks),
  50. chunkAllocated: make(map[uint32]bool, snapshot.Chunks),
  51. chunkReturned: make(map[uint32]bool, snapshot.Chunks),
  52. waiters: make(map[uint32][]chan<- uint32),
  53. }, nil
  54. }
  55. // Add adds a chunk to the queue. It ignores chunks that already exist, returning false.
  56. func (q *chunkQueue) Add(chunk *chunk) (bool, error) {
  57. if chunk == nil || chunk.Chunk == nil {
  58. return false, errors.New("cannot add nil chunk")
  59. }
  60. q.Lock()
  61. defer q.Unlock()
  62. if q.snapshot == nil {
  63. return false, nil // queue is closed
  64. }
  65. if chunk.Height != q.snapshot.Height {
  66. return false, fmt.Errorf("invalid chunk height %v, expected %v", chunk.Height, q.snapshot.Height)
  67. }
  68. if chunk.Format != q.snapshot.Format {
  69. return false, fmt.Errorf("invalid chunk format %v, expected %v", chunk.Format, q.snapshot.Format)
  70. }
  71. if chunk.Index >= q.snapshot.Chunks {
  72. return false, fmt.Errorf("received unexpected chunk %v", chunk.Index)
  73. }
  74. if q.chunkFiles[chunk.Index] != "" {
  75. return false, nil
  76. }
  77. path := filepath.Join(q.dir, strconv.FormatUint(uint64(chunk.Index), 10))
  78. err := os.WriteFile(path, chunk.Chunk, 0600)
  79. if err != nil {
  80. return false, fmt.Errorf("failed to save chunk %v to file %v: %w", chunk.Index, path, err)
  81. }
  82. q.chunkFiles[chunk.Index] = path
  83. q.chunkSenders[chunk.Index] = chunk.Sender
  84. // Signal any waiters that the chunk has arrived.
  85. for _, waiter := range q.waiters[chunk.Index] {
  86. waiter <- chunk.Index
  87. close(waiter)
  88. }
  89. delete(q.waiters, chunk.Index)
  90. return true, nil
  91. }
  92. // Allocate allocates a chunk to the caller, making it responsible for fetching it. Returns
  93. // errDone once no chunks are left or the queue is closed.
  94. func (q *chunkQueue) Allocate() (uint32, error) {
  95. q.Lock()
  96. defer q.Unlock()
  97. if q.snapshot == nil {
  98. return 0, errDone
  99. }
  100. if uint32(len(q.chunkAllocated)) >= q.snapshot.Chunks {
  101. return 0, errDone
  102. }
  103. for i := uint32(0); i < q.snapshot.Chunks; i++ {
  104. if !q.chunkAllocated[i] {
  105. q.chunkAllocated[i] = true
  106. return i, nil
  107. }
  108. }
  109. return 0, errDone
  110. }
  111. // Close closes the chunk queue, cleaning up all temporary files.
  112. func (q *chunkQueue) Close() error {
  113. q.Lock()
  114. defer q.Unlock()
  115. if q.snapshot == nil {
  116. return nil
  117. }
  118. for _, waiters := range q.waiters {
  119. for _, waiter := range waiters {
  120. close(waiter)
  121. }
  122. }
  123. q.waiters = nil
  124. q.snapshot = nil
  125. if err := os.RemoveAll(q.dir); err != nil {
  126. return fmt.Errorf("failed to clean up state sync tempdir %v: %w", q.dir, err)
  127. }
  128. return nil
  129. }
  130. // Discard discards a chunk. It will be removed from the queue, available for allocation, and can
  131. // be added and returned via Next() again. If the chunk is not already in the queue this does
  132. // nothing, to avoid it being allocated to multiple fetchers.
  133. func (q *chunkQueue) Discard(index uint32) error {
  134. q.Lock()
  135. defer q.Unlock()
  136. return q.discard(index)
  137. }
  138. // discard discards a chunk, scheduling it for refetching. The caller must hold the mutex lock.
  139. func (q *chunkQueue) discard(index uint32) error {
  140. if q.snapshot == nil {
  141. return nil
  142. }
  143. path := q.chunkFiles[index]
  144. if path == "" {
  145. return nil
  146. }
  147. if err := os.Remove(path); err != nil {
  148. return fmt.Errorf("failed to remove chunk %v: %w", index, err)
  149. }
  150. delete(q.chunkFiles, index)
  151. delete(q.chunkReturned, index)
  152. delete(q.chunkAllocated, index)
  153. return nil
  154. }
  155. // DiscardSender discards all *unreturned* chunks from a given sender. If the caller wants to
  156. // discard already returned chunks, this can be done via Discard().
  157. func (q *chunkQueue) DiscardSender(peerID types.NodeID) error {
  158. q.Lock()
  159. defer q.Unlock()
  160. for index, sender := range q.chunkSenders {
  161. if sender == peerID && !q.chunkReturned[index] {
  162. err := q.discard(index)
  163. if err != nil {
  164. return err
  165. }
  166. delete(q.chunkSenders, index)
  167. }
  168. }
  169. return nil
  170. }
  171. // GetSender returns the sender of the chunk with the given index, or empty if
  172. // not found.
  173. func (q *chunkQueue) GetSender(index uint32) types.NodeID {
  174. q.Lock()
  175. defer q.Unlock()
  176. return q.chunkSenders[index]
  177. }
  178. // Has checks whether a chunk exists in the queue.
  179. func (q *chunkQueue) Has(index uint32) bool {
  180. q.Lock()
  181. defer q.Unlock()
  182. return q.chunkFiles[index] != ""
  183. }
  184. // load loads a chunk from disk, or nil if the chunk is not in the queue. The caller must hold the
  185. // mutex lock.
  186. func (q *chunkQueue) load(index uint32) (*chunk, error) {
  187. path, ok := q.chunkFiles[index]
  188. if !ok {
  189. return nil, nil
  190. }
  191. body, err := os.ReadFile(path)
  192. if err != nil {
  193. return nil, fmt.Errorf("failed to load chunk %v: %w", index, err)
  194. }
  195. return &chunk{
  196. Height: q.snapshot.Height,
  197. Format: q.snapshot.Format,
  198. Index: index,
  199. Chunk: body,
  200. Sender: q.chunkSenders[index],
  201. }, nil
  202. }
  203. // Next returns the next chunk from the queue, or errDone if all chunks have been returned. It
  204. // blocks until the chunk is available. Concurrent Next() calls may return the same chunk.
  205. func (q *chunkQueue) Next() (*chunk, error) {
  206. q.Lock()
  207. var chunk *chunk
  208. index, err := q.nextUp()
  209. if err == nil {
  210. chunk, err = q.load(index)
  211. if err == nil {
  212. q.chunkReturned[index] = true
  213. }
  214. }
  215. q.Unlock()
  216. if chunk != nil || err != nil {
  217. return chunk, err
  218. }
  219. select {
  220. case _, ok := <-q.WaitFor(index):
  221. if !ok {
  222. return nil, errDone // queue closed
  223. }
  224. case <-time.After(chunkTimeout):
  225. return nil, errTimeout
  226. }
  227. q.Lock()
  228. defer q.Unlock()
  229. chunk, err = q.load(index)
  230. if err != nil {
  231. return nil, err
  232. }
  233. q.chunkReturned[index] = true
  234. return chunk, nil
  235. }
  236. // nextUp returns the next chunk to be returned, or errDone if all chunks have been returned. The
  237. // caller must hold the mutex lock.
  238. func (q *chunkQueue) nextUp() (uint32, error) {
  239. if q.snapshot == nil {
  240. return 0, errDone
  241. }
  242. for i := uint32(0); i < q.snapshot.Chunks; i++ {
  243. if !q.chunkReturned[i] {
  244. return i, nil
  245. }
  246. }
  247. return 0, errDone
  248. }
  249. // Retry schedules a chunk to be retried, without refetching it.
  250. func (q *chunkQueue) Retry(index uint32) {
  251. q.Lock()
  252. defer q.Unlock()
  253. delete(q.chunkReturned, index)
  254. }
  255. // RetryAll schedules all chunks to be retried, without refetching them.
  256. func (q *chunkQueue) RetryAll() {
  257. q.Lock()
  258. defer q.Unlock()
  259. q.chunkReturned = make(map[uint32]bool)
  260. }
  261. // Size returns the total number of chunks for the snapshot and queue, or 0 when closed.
  262. func (q *chunkQueue) Size() uint32 {
  263. q.Lock()
  264. defer q.Unlock()
  265. if q.snapshot == nil {
  266. return 0
  267. }
  268. return q.snapshot.Chunks
  269. }
  270. // WaitFor returns a channel that receives a chunk index when it arrives in the queue, or
  271. // immediately if it has already arrived. The channel is closed without a value if the queue is
  272. // closed or if the chunk index is not valid.
  273. func (q *chunkQueue) WaitFor(index uint32) <-chan uint32 {
  274. q.Lock()
  275. defer q.Unlock()
  276. ch := make(chan uint32, 1)
  277. switch {
  278. case q.snapshot == nil:
  279. close(ch)
  280. case index >= q.snapshot.Chunks:
  281. close(ch)
  282. case q.chunkFiles[index] != "":
  283. ch <- index
  284. close(ch)
  285. default:
  286. if q.waiters[index] == nil {
  287. q.waiters[index] = make([]chan<- uint32, 0)
  288. }
  289. q.waiters[index] = append(q.waiters[index], ch)
  290. }
  291. return ch
  292. }
  293. func (q *chunkQueue) numChunksReturned() int {
  294. q.Lock()
  295. defer q.Unlock()
  296. cnt := 0
  297. for _, b := range q.chunkReturned {
  298. if b {
  299. cnt++
  300. }
  301. }
  302. return cnt
  303. }