filer_server_handlers_write_upload.go 11 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325
  1. package weed_server
  2. import (
  3. "bytes"
  4. "context"
  5. "crypto/md5"
  6. "encoding/base64"
  7. "fmt"
  8. "hash"
  9. "io"
  10. "net/http"
  11. "strconv"
  12. "sync"
  13. "time"
  14. "slices"
  15. "encoding/json"
  16. "github.com/seaweedfs/seaweedfs/weed/glog"
  17. "github.com/seaweedfs/seaweedfs/weed/operation"
  18. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  19. "github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
  20. "github.com/seaweedfs/seaweedfs/weed/security"
  21. "github.com/seaweedfs/seaweedfs/weed/stats"
  22. "github.com/seaweedfs/seaweedfs/weed/util"
  23. )
  24. var bufPool = sync.Pool{
  25. New: func() interface{} {
  26. return new(bytes.Buffer)
  27. },
  28. }
  29. func (fs *FilerServer) uploadRequestToChunks(ctx context.Context, w http.ResponseWriter, r *http.Request, reader io.Reader, chunkSize int32, fileName, contentType string, contentLength int64, so *operation.StorageOption) (fileChunks []*filer_pb.FileChunk, md5Hash hash.Hash, chunkOffset int64, uploadErr error, smallContent []byte) {
  30. query := r.URL.Query()
  31. isAppend := isAppend(r)
  32. if query.Has("offset") {
  33. offset := query.Get("offset")
  34. offsetInt, err := strconv.ParseInt(offset, 10, 64)
  35. if err != nil || offsetInt < 0 {
  36. err = fmt.Errorf("invalid 'offset': '%s'", offset)
  37. return nil, nil, 0, err, nil
  38. }
  39. if isAppend && offsetInt > 0 {
  40. err = fmt.Errorf("cannot set offset when op=append")
  41. return nil, nil, 0, err, nil
  42. }
  43. chunkOffset = offsetInt
  44. }
  45. return fs.uploadReaderToChunks(ctx, r, reader, chunkOffset, chunkSize, fileName, contentType, isAppend, so)
  46. }
  47. func (fs *FilerServer) uploadReaderToChunks(ctx context.Context, r *http.Request, reader io.Reader, startOffset int64, chunkSize int32, fileName, contentType string, isAppend bool, so *operation.StorageOption) (fileChunks []*filer_pb.FileChunk, md5Hash hash.Hash, chunkOffset int64, uploadErr error, smallContent []byte) {
  48. md5Hash = md5.New()
  49. chunkOffset = startOffset
  50. var partReader = io.NopCloser(io.TeeReader(reader, md5Hash))
  51. var wg sync.WaitGroup
  52. var bytesBufferCounter int64 = 4
  53. bytesBufferLimitChan := make(chan struct{}, bytesBufferCounter)
  54. var fileChunksLock sync.Mutex
  55. var uploadErrLock sync.Mutex
  56. for {
  57. // need to throttle used byte buffer
  58. bytesBufferLimitChan <- struct{}{}
  59. // As long as there is an error in the upload of one chunk, it can be terminated early
  60. // uploadErr may be modified in other go routines, lock is needed to avoid race condition
  61. uploadErrLock.Lock()
  62. if uploadErr != nil {
  63. <-bytesBufferLimitChan
  64. uploadErrLock.Unlock()
  65. break
  66. }
  67. uploadErrLock.Unlock()
  68. bytesBuffer := bufPool.Get().(*bytes.Buffer)
  69. limitedReader := io.LimitReader(partReader, int64(chunkSize))
  70. bytesBuffer.Reset()
  71. dataSize, err := bytesBuffer.ReadFrom(limitedReader)
  72. // data, err := io.ReadAll(limitedReader)
  73. if err != nil || dataSize == 0 {
  74. bufPool.Put(bytesBuffer)
  75. <-bytesBufferLimitChan
  76. if err != nil {
  77. uploadErrLock.Lock()
  78. if uploadErr == nil {
  79. uploadErr = err
  80. }
  81. uploadErrLock.Unlock()
  82. }
  83. break
  84. }
  85. if chunkOffset == 0 && !isAppend {
  86. if dataSize < fs.option.SaveToFilerLimit {
  87. chunkOffset += dataSize
  88. smallContent = make([]byte, dataSize)
  89. bytesBuffer.Read(smallContent)
  90. bufPool.Put(bytesBuffer)
  91. <-bytesBufferLimitChan
  92. stats.FilerHandlerCounter.WithLabelValues(stats.ContentSaveToFiler).Inc()
  93. break
  94. }
  95. } else {
  96. stats.FilerHandlerCounter.WithLabelValues(stats.AutoChunk).Inc()
  97. }
  98. wg.Add(1)
  99. go func(offset int64, buf *bytes.Buffer) {
  100. defer func() {
  101. bufPool.Put(buf)
  102. <-bytesBufferLimitChan
  103. wg.Done()
  104. }()
  105. chunks, toChunkErr := fs.dataToChunkWithSSE(ctx, r, fileName, contentType, buf.Bytes(), offset, so)
  106. if toChunkErr != nil {
  107. uploadErrLock.Lock()
  108. if uploadErr == nil {
  109. uploadErr = toChunkErr
  110. }
  111. uploadErrLock.Unlock()
  112. }
  113. if chunks != nil {
  114. fileChunksLock.Lock()
  115. fileChunksSize := len(fileChunks) + len(chunks)
  116. for _, chunk := range chunks {
  117. fileChunks = append(fileChunks, chunk)
  118. glog.V(4).InfofCtx(ctx, "uploaded %s chunk %d to %s [%d,%d)", fileName, fileChunksSize, chunk.FileId, offset, offset+int64(chunk.Size))
  119. }
  120. fileChunksLock.Unlock()
  121. }
  122. }(chunkOffset, bytesBuffer)
  123. // reset variables for the next chunk
  124. chunkOffset = chunkOffset + dataSize
  125. // if last chunk was not at full chunk size, but already exhausted the reader
  126. if dataSize < int64(chunkSize) {
  127. break
  128. }
  129. }
  130. wg.Wait()
  131. if uploadErr != nil {
  132. glog.V(0).InfofCtx(ctx, "upload file %s error: %v", fileName, uploadErr)
  133. for _, chunk := range fileChunks {
  134. glog.V(4).InfofCtx(ctx, "purging failed uploaded %s chunk %s [%d,%d)", fileName, chunk.FileId, chunk.Offset, chunk.Offset+int64(chunk.Size))
  135. }
  136. fs.filer.DeleteUncommittedChunks(ctx, fileChunks)
  137. return nil, md5Hash, 0, uploadErr, nil
  138. }
  139. slices.SortFunc(fileChunks, func(a, b *filer_pb.FileChunk) int {
  140. return int(a.Offset - b.Offset)
  141. })
  142. return fileChunks, md5Hash, chunkOffset, nil, smallContent
  143. }
  144. func (fs *FilerServer) doUpload(ctx context.Context, urlLocation string, limitedReader io.Reader, fileName string, contentType string, pairMap map[string]string, auth security.EncodedJwt) (*operation.UploadResult, error, []byte) {
  145. stats.FilerHandlerCounter.WithLabelValues(stats.ChunkUpload).Inc()
  146. start := time.Now()
  147. defer func() {
  148. stats.FilerRequestHistogram.WithLabelValues(stats.ChunkUpload).Observe(time.Since(start).Seconds())
  149. }()
  150. uploadOption := &operation.UploadOption{
  151. UploadUrl: urlLocation,
  152. Filename: fileName,
  153. Cipher: fs.option.Cipher,
  154. IsInputCompressed: false,
  155. MimeType: contentType,
  156. PairMap: pairMap,
  157. Jwt: auth,
  158. }
  159. uploader, err := operation.NewUploader()
  160. if err != nil {
  161. return nil, err, []byte{}
  162. }
  163. uploadResult, err, data := uploader.Upload(ctx, limitedReader, uploadOption)
  164. if uploadResult != nil && uploadResult.RetryCount > 0 {
  165. stats.FilerHandlerCounter.WithLabelValues(stats.ChunkUploadRetry).Add(float64(uploadResult.RetryCount))
  166. }
  167. return uploadResult, err, data
  168. }
  169. func (fs *FilerServer) dataToChunk(ctx context.Context, fileName, contentType string, data []byte, chunkOffset int64, so *operation.StorageOption) ([]*filer_pb.FileChunk, error) {
  170. return fs.dataToChunkWithSSE(ctx, nil, fileName, contentType, data, chunkOffset, so)
  171. }
  172. func (fs *FilerServer) dataToChunkWithSSE(ctx context.Context, r *http.Request, fileName, contentType string, data []byte, chunkOffset int64, so *operation.StorageOption) ([]*filer_pb.FileChunk, error) {
  173. dataReader := util.NewBytesReader(data)
  174. // retry to assign a different file id
  175. var fileId, urlLocation string
  176. var auth security.EncodedJwt
  177. var uploadErr error
  178. var uploadResult *operation.UploadResult
  179. var failedFileChunks []*filer_pb.FileChunk
  180. err := util.Retry("filerDataToChunk", func() error {
  181. // assign one file id for one chunk
  182. fileId, urlLocation, auth, uploadErr = fs.assignNewFileInfo(ctx, so)
  183. if uploadErr != nil {
  184. glog.V(4).InfofCtx(ctx, "retry later due to assign error: %v", uploadErr)
  185. stats.FilerHandlerCounter.WithLabelValues(stats.ChunkAssignRetry).Inc()
  186. return uploadErr
  187. }
  188. // upload the chunk to the volume server
  189. uploadResult, uploadErr, _ = fs.doUpload(ctx, urlLocation, dataReader, fileName, contentType, nil, auth)
  190. if uploadErr != nil {
  191. glog.V(4).InfofCtx(ctx, "retry later due to upload error: %v", uploadErr)
  192. stats.FilerHandlerCounter.WithLabelValues(stats.ChunkDoUploadRetry).Inc()
  193. fid, _ := filer_pb.ToFileIdObject(fileId)
  194. fileChunk := filer_pb.FileChunk{
  195. FileId: fileId,
  196. Offset: chunkOffset,
  197. Fid: fid,
  198. }
  199. failedFileChunks = append(failedFileChunks, &fileChunk)
  200. return uploadErr
  201. }
  202. return nil
  203. })
  204. if err != nil {
  205. glog.ErrorfCtx(ctx, "upload error: %v", err)
  206. return failedFileChunks, err
  207. }
  208. // if last chunk exhausted the reader exactly at the border
  209. if uploadResult.Size == 0 {
  210. return nil, nil
  211. }
  212. // Extract SSE metadata from request headers if available
  213. var sseType filer_pb.SSEType = filer_pb.SSEType_NONE
  214. var sseMetadata []byte
  215. if r != nil {
  216. // Check for SSE-KMS
  217. sseKMSHeaderValue := r.Header.Get(s3_constants.SeaweedFSSSEKMSKeyHeader)
  218. if sseKMSHeaderValue != "" {
  219. sseType = filer_pb.SSEType_SSE_KMS
  220. if kmsData, err := base64.StdEncoding.DecodeString(sseKMSHeaderValue); err == nil {
  221. sseMetadata = kmsData
  222. glog.V(4).InfofCtx(ctx, "Storing SSE-KMS metadata for chunk %s at offset %d", fileId, chunkOffset)
  223. } else {
  224. glog.V(1).InfofCtx(ctx, "Failed to decode SSE-KMS metadata for chunk %s: %v", fileId, err)
  225. }
  226. } else if r.Header.Get(s3_constants.AmzServerSideEncryptionCustomerAlgorithm) != "" {
  227. // SSE-C: Create per-chunk metadata for unified handling
  228. sseType = filer_pb.SSEType_SSE_C
  229. // Get SSE-C metadata from headers to create unified per-chunk metadata
  230. sseIVHeader := r.Header.Get(s3_constants.SeaweedFSSSEIVHeader)
  231. keyMD5Header := r.Header.Get(s3_constants.AmzServerSideEncryptionCustomerKeyMD5)
  232. if sseIVHeader != "" && keyMD5Header != "" {
  233. // Decode IV from header
  234. if ivData, err := base64.StdEncoding.DecodeString(sseIVHeader); err == nil {
  235. // Create SSE-C metadata with chunk offset = chunkOffset for proper IV calculation
  236. ssecMetadataStruct := struct {
  237. Algorithm string `json:"algorithm"`
  238. IV string `json:"iv"`
  239. KeyMD5 string `json:"keyMD5"`
  240. PartOffset int64 `json:"partOffset"`
  241. }{
  242. Algorithm: "AES256",
  243. IV: base64.StdEncoding.EncodeToString(ivData),
  244. KeyMD5: keyMD5Header,
  245. PartOffset: chunkOffset,
  246. }
  247. if ssecMetadata, serErr := json.Marshal(ssecMetadataStruct); serErr == nil {
  248. sseMetadata = ssecMetadata
  249. } else {
  250. glog.V(1).InfofCtx(ctx, "Failed to serialize SSE-C metadata for chunk %s: %v", fileId, serErr)
  251. }
  252. } else {
  253. glog.V(1).InfofCtx(ctx, "Failed to decode SSE-C IV for chunk %s: %v", fileId, err)
  254. }
  255. } else {
  256. glog.V(4).InfofCtx(ctx, "SSE-C chunk %s missing IV or KeyMD5 header", fileId)
  257. }
  258. } else if r.Header.Get(s3_constants.SeaweedFSSSES3Key) != "" {
  259. // SSE-S3: Server-side encryption with server-managed keys
  260. // Set the correct SSE type for SSE-S3 chunks to maintain proper tracking
  261. sseType = filer_pb.SSEType_SSE_S3
  262. // Get SSE-S3 metadata from headers
  263. sseS3Header := r.Header.Get(s3_constants.SeaweedFSSSES3Key)
  264. if sseS3Header != "" {
  265. if s3Data, err := base64.StdEncoding.DecodeString(sseS3Header); err == nil {
  266. // For SSE-S3, store metadata at chunk level for consistency with SSE-KMS/SSE-C
  267. glog.V(4).InfofCtx(ctx, "Storing SSE-S3 metadata for chunk %s at offset %d", fileId, chunkOffset)
  268. sseMetadata = s3Data
  269. } else {
  270. glog.V(1).InfofCtx(ctx, "Failed to decode SSE-S3 metadata for chunk %s: %v", fileId, err)
  271. }
  272. }
  273. }
  274. }
  275. // Create chunk with SSE metadata if available
  276. var chunk *filer_pb.FileChunk
  277. if sseType != filer_pb.SSEType_NONE {
  278. chunk = uploadResult.ToPbFileChunkWithSSE(fileId, chunkOffset, time.Now().UnixNano(), sseType, sseMetadata)
  279. } else {
  280. chunk = uploadResult.ToPbFileChunk(fileId, chunkOffset, time.Now().UnixNano())
  281. }
  282. return []*filer_pb.FileChunk{chunk}, nil
  283. }