stream.go 11 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390
  1. package filer
  2. import (
  3. "bytes"
  4. "context"
  5. "fmt"
  6. "io"
  7. "math"
  8. "strings"
  9. "sync"
  10. "time"
  11. "slices"
  12. "github.com/seaweedfs/seaweedfs/weed/glog"
  13. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  14. "github.com/seaweedfs/seaweedfs/weed/stats"
  15. "github.com/seaweedfs/seaweedfs/weed/util"
  16. util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
  17. "github.com/seaweedfs/seaweedfs/weed/wdclient"
  18. )
  19. var getLookupFileIdBackoffSchedule = []time.Duration{
  20. 150 * time.Millisecond,
  21. 600 * time.Millisecond,
  22. 1800 * time.Millisecond,
  23. }
  24. func HasData(entry *filer_pb.Entry) bool {
  25. if len(entry.Content) > 0 {
  26. return true
  27. }
  28. return len(entry.GetChunks()) > 0
  29. }
  30. func IsSameData(a, b *filer_pb.Entry) bool {
  31. if len(a.Content) > 0 || len(b.Content) > 0 {
  32. return bytes.Equal(a.Content, b.Content)
  33. }
  34. return isSameChunks(a.Chunks, b.Chunks)
  35. }
  36. func isSameChunks(a, b []*filer_pb.FileChunk) bool {
  37. if len(a) != len(b) {
  38. return false
  39. }
  40. slices.SortFunc(a, func(i, j *filer_pb.FileChunk) int {
  41. return strings.Compare(i.ETag, j.ETag)
  42. })
  43. slices.SortFunc(b, func(i, j *filer_pb.FileChunk) int {
  44. return strings.Compare(i.ETag, j.ETag)
  45. })
  46. for i := 0; i < len(a); i++ {
  47. if a[i].ETag != b[i].ETag {
  48. return false
  49. }
  50. }
  51. return true
  52. }
  53. func NewFileReader(filerClient filer_pb.FilerClient, entry *filer_pb.Entry) io.Reader {
  54. if len(entry.Content) > 0 {
  55. return bytes.NewReader(entry.Content)
  56. }
  57. return NewChunkStreamReader(filerClient, entry.GetChunks())
  58. }
  59. type DoStreamContent func(writer io.Writer) error
  60. func PrepareStreamContent(masterClient wdclient.HasLookupFileIdFunction, jwtFunc VolumeServerJwtFunction, chunks []*filer_pb.FileChunk, offset int64, size int64) (DoStreamContent, error) {
  61. return PrepareStreamContentWithThrottler(context.Background(), masterClient, jwtFunc, chunks, offset, size, 0)
  62. }
  63. type VolumeServerJwtFunction func(fileId string) string
  64. func noJwtFunc(string) string {
  65. return ""
  66. }
  67. func PrepareStreamContentWithThrottler(ctx context.Context, masterClient wdclient.HasLookupFileIdFunction, jwtFunc VolumeServerJwtFunction, chunks []*filer_pb.FileChunk, offset int64, size int64, downloadMaxBytesPs int64) (DoStreamContent, error) {
  68. glog.V(4).InfofCtx(ctx, "prepare to stream content for chunks: %d", len(chunks))
  69. chunkViews := ViewFromChunks(ctx, masterClient.GetLookupFileIdFunction(), chunks, offset, size)
  70. fileId2Url := make(map[string][]string)
  71. for x := chunkViews.Front(); x != nil; x = x.Next {
  72. chunkView := x.Value
  73. var urlStrings []string
  74. var err error
  75. for _, backoff := range getLookupFileIdBackoffSchedule {
  76. urlStrings, err = masterClient.GetLookupFileIdFunction()(ctx, chunkView.FileId)
  77. if err == nil && len(urlStrings) > 0 {
  78. break
  79. }
  80. glog.V(4).InfofCtx(ctx, "waiting for chunk: %s", chunkView.FileId)
  81. time.Sleep(backoff)
  82. }
  83. if err != nil {
  84. glog.V(1).InfofCtx(ctx, "operation LookupFileId %s failed, err: %v", chunkView.FileId, err)
  85. return nil, err
  86. } else if len(urlStrings) == 0 {
  87. errUrlNotFound := fmt.Errorf("operation LookupFileId %s failed, err: urls not found", chunkView.FileId)
  88. glog.ErrorCtx(ctx, errUrlNotFound)
  89. return nil, errUrlNotFound
  90. }
  91. fileId2Url[chunkView.FileId] = urlStrings
  92. }
  93. return func(writer io.Writer) error {
  94. downloadThrottler := util.NewWriteThrottler(downloadMaxBytesPs)
  95. remaining := size
  96. for x := chunkViews.Front(); x != nil; x = x.Next {
  97. chunkView := x.Value
  98. if offset < chunkView.ViewOffset {
  99. gap := chunkView.ViewOffset - offset
  100. remaining -= gap
  101. glog.V(4).InfofCtx(ctx, "zero [%d,%d)", offset, chunkView.ViewOffset)
  102. err := writeZero(writer, gap)
  103. if err != nil {
  104. return fmt.Errorf("write zero [%d,%d)", offset, chunkView.ViewOffset)
  105. }
  106. offset = chunkView.ViewOffset
  107. }
  108. urlStrings := fileId2Url[chunkView.FileId]
  109. start := time.Now()
  110. jwt := jwtFunc(chunkView.FileId)
  111. err := retriedStreamFetchChunkData(ctx, writer, urlStrings, jwt, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk, int(chunkView.ViewSize))
  112. offset += int64(chunkView.ViewSize)
  113. remaining -= int64(chunkView.ViewSize)
  114. stats.FilerRequestHistogram.WithLabelValues("chunkDownload").Observe(time.Since(start).Seconds())
  115. if err != nil {
  116. stats.FilerHandlerCounter.WithLabelValues("chunkDownloadError").Inc()
  117. return fmt.Errorf("read chunk: %w", err)
  118. }
  119. stats.FilerHandlerCounter.WithLabelValues("chunkDownload").Inc()
  120. downloadThrottler.MaybeSlowdown(int64(chunkView.ViewSize))
  121. }
  122. if remaining > 0 {
  123. glog.V(4).InfofCtx(ctx, "zero [%d,%d)", offset, offset+remaining)
  124. err := writeZero(writer, remaining)
  125. if err != nil {
  126. return fmt.Errorf("write zero [%d,%d)", offset, offset+remaining)
  127. }
  128. }
  129. return nil
  130. }, nil
  131. }
  132. func StreamContent(masterClient wdclient.HasLookupFileIdFunction, writer io.Writer, chunks []*filer_pb.FileChunk, offset int64, size int64) error {
  133. streamFn, err := PrepareStreamContent(masterClient, noJwtFunc, chunks, offset, size)
  134. if err != nil {
  135. return err
  136. }
  137. return streamFn(writer)
  138. }
  139. // ---------------- ReadAllReader ----------------------------------
  140. func writeZero(w io.Writer, size int64) (err error) {
  141. zeroPadding := make([]byte, 1024)
  142. var written int
  143. for size > 0 {
  144. if size > 1024 {
  145. written, err = w.Write(zeroPadding)
  146. } else {
  147. written, err = w.Write(zeroPadding[:size])
  148. }
  149. size -= int64(written)
  150. if err != nil {
  151. return
  152. }
  153. }
  154. return
  155. }
  156. func ReadAll(ctx context.Context, buffer []byte, masterClient *wdclient.MasterClient, chunks []*filer_pb.FileChunk) error {
  157. lookupFileIdFn := func(ctx context.Context, fileId string) (targetUrls []string, err error) {
  158. return masterClient.LookupFileId(ctx, fileId)
  159. }
  160. chunkViews := ViewFromChunks(ctx, lookupFileIdFn, chunks, 0, int64(len(buffer)))
  161. idx := 0
  162. for x := chunkViews.Front(); x != nil; x = x.Next {
  163. chunkView := x.Value
  164. urlStrings, err := lookupFileIdFn(ctx, chunkView.FileId)
  165. if err != nil {
  166. glog.V(1).InfofCtx(ctx, "operation LookupFileId %s failed, err: %v", chunkView.FileId, err)
  167. return err
  168. }
  169. n, err := util_http.RetriedFetchChunkData(ctx, buffer[idx:idx+int(chunkView.ViewSize)], urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk, chunkView.FileId)
  170. if err != nil {
  171. return err
  172. }
  173. idx += n
  174. }
  175. return nil
  176. }
  177. // ---------------- ChunkStreamReader ----------------------------------
  178. type ChunkStreamReader struct {
  179. head *Interval[*ChunkView]
  180. chunkView *Interval[*ChunkView]
  181. totalSize int64
  182. logicOffset int64
  183. buffer []byte
  184. bufferOffset int64
  185. bufferLock sync.Mutex
  186. chunk string
  187. lookupFileId wdclient.LookupFileIdFunctionType
  188. }
  189. var _ = io.ReadSeeker(&ChunkStreamReader{})
  190. var _ = io.ReaderAt(&ChunkStreamReader{})
  191. func doNewChunkStreamReader(ctx context.Context, lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk) *ChunkStreamReader {
  192. chunkViews := ViewFromChunks(ctx, lookupFileIdFn, chunks, 0, math.MaxInt64)
  193. var totalSize int64
  194. for x := chunkViews.Front(); x != nil; x = x.Next {
  195. chunk := x.Value
  196. totalSize += int64(chunk.ViewSize)
  197. }
  198. return &ChunkStreamReader{
  199. head: chunkViews.Front(),
  200. chunkView: chunkViews.Front(),
  201. lookupFileId: lookupFileIdFn,
  202. totalSize: totalSize,
  203. }
  204. }
  205. func NewChunkStreamReaderFromFiler(ctx context.Context, masterClient *wdclient.MasterClient, chunks []*filer_pb.FileChunk) *ChunkStreamReader {
  206. lookupFileIdFn := func(ctx context.Context, fileId string) (targetUrl []string, err error) {
  207. return masterClient.LookupFileId(ctx, fileId)
  208. }
  209. return doNewChunkStreamReader(ctx, lookupFileIdFn, chunks)
  210. }
  211. func NewChunkStreamReader(filerClient filer_pb.FilerClient, chunks []*filer_pb.FileChunk) *ChunkStreamReader {
  212. lookupFileIdFn := LookupFn(filerClient)
  213. return doNewChunkStreamReader(context.Background(), lookupFileIdFn, chunks)
  214. }
  215. func (c *ChunkStreamReader) ReadAt(p []byte, off int64) (n int, err error) {
  216. c.bufferLock.Lock()
  217. defer c.bufferLock.Unlock()
  218. if err = c.prepareBufferFor(off); err != nil {
  219. return
  220. }
  221. c.logicOffset = off
  222. return c.doRead(p)
  223. }
  224. func (c *ChunkStreamReader) Read(p []byte) (n int, err error) {
  225. c.bufferLock.Lock()
  226. defer c.bufferLock.Unlock()
  227. return c.doRead(p)
  228. }
  229. func (c *ChunkStreamReader) doRead(p []byte) (n int, err error) {
  230. // fmt.Printf("do read [%d,%d) at %s[%d,%d)\n", c.logicOffset, c.logicOffset+int64(len(p)), c.chunk, c.bufferOffset, c.bufferOffset+int64(len(c.buffer)))
  231. for n < len(p) {
  232. // println("read", c.logicOffset)
  233. if err = c.prepareBufferFor(c.logicOffset); err != nil {
  234. return
  235. }
  236. t := copy(p[n:], c.buffer[c.logicOffset-c.bufferOffset:])
  237. n += t
  238. c.logicOffset += int64(t)
  239. }
  240. return
  241. }
  242. func (c *ChunkStreamReader) isBufferEmpty() bool {
  243. return len(c.buffer) <= int(c.logicOffset-c.bufferOffset)
  244. }
  245. func (c *ChunkStreamReader) Seek(offset int64, whence int) (int64, error) {
  246. c.bufferLock.Lock()
  247. defer c.bufferLock.Unlock()
  248. var err error
  249. switch whence {
  250. case io.SeekStart:
  251. case io.SeekCurrent:
  252. offset += c.logicOffset
  253. case io.SeekEnd:
  254. offset = c.totalSize + offset
  255. }
  256. if offset > c.totalSize {
  257. err = io.ErrUnexpectedEOF
  258. } else {
  259. c.logicOffset = offset
  260. }
  261. return offset, err
  262. }
  263. func insideChunk(offset int64, chunk *ChunkView) bool {
  264. return chunk.ViewOffset <= offset && offset < chunk.ViewOffset+int64(chunk.ViewSize)
  265. }
  266. func (c *ChunkStreamReader) prepareBufferFor(offset int64) (err error) {
  267. // stay in the same chunk
  268. if c.bufferOffset <= offset && offset < c.bufferOffset+int64(len(c.buffer)) {
  269. return nil
  270. }
  271. // glog.V(2).Infof("c.chunkView: %v buffer:[%d,%d) offset:%d totalSize:%d", c.chunkView, c.bufferOffset, c.bufferOffset+int64(len(c.buffer)), offset, c.totalSize)
  272. // find a possible chunk view
  273. p := c.chunkView
  274. for p != nil {
  275. chunk := p.Value
  276. // glog.V(2).Infof("prepareBufferFor check chunk:[%d,%d)", chunk.ViewOffset, chunk.ViewOffset+int64(chunk.ViewSize))
  277. if insideChunk(offset, chunk) {
  278. if c.isBufferEmpty() || c.bufferOffset != chunk.ViewOffset {
  279. c.chunkView = p
  280. return c.fetchChunkToBuffer(chunk)
  281. }
  282. }
  283. if offset < c.bufferOffset {
  284. p = p.Prev
  285. } else {
  286. p = p.Next
  287. }
  288. }
  289. return io.EOF
  290. }
  291. func (c *ChunkStreamReader) fetchChunkToBuffer(chunkView *ChunkView) error {
  292. urlStrings, err := c.lookupFileId(context.Background(), chunkView.FileId)
  293. if err != nil {
  294. glog.V(1).Infof("operation LookupFileId %s failed, err: %v", chunkView.FileId, err)
  295. return err
  296. }
  297. var buffer bytes.Buffer
  298. var shouldRetry bool
  299. for _, urlString := range urlStrings {
  300. shouldRetry, err = util_http.ReadUrlAsStream(context.Background(), urlString+"?readDeleted=true", chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk, int(chunkView.ViewSize), func(data []byte) {
  301. buffer.Write(data)
  302. })
  303. if !shouldRetry {
  304. break
  305. }
  306. if err != nil {
  307. glog.V(1).Infof("read %s failed, err: %v", chunkView.FileId, err)
  308. buffer.Reset()
  309. } else {
  310. break
  311. }
  312. }
  313. if err != nil {
  314. return err
  315. }
  316. c.buffer = buffer.Bytes()
  317. c.bufferOffset = chunkView.ViewOffset
  318. c.chunk = chunkView.FileId
  319. // glog.V(0).Infof("fetched %s [%d,%d)", chunkView.FileId, chunkView.ViewOffset, chunkView.ViewOffset+int64(chunkView.ViewSize))
  320. return nil
  321. }
  322. func (c *ChunkStreamReader) Close() {
  323. // TODO try to release and reuse buffer
  324. }
  325. func VolumeId(fileId string) string {
  326. lastCommaIndex := strings.LastIndex(fileId, ",")
  327. if lastCommaIndex > 0 {
  328. return fileId[:lastCommaIndex]
  329. }
  330. return fileId
  331. }