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.
		
		
		
		
		
			
		
			
				
					
					
						
							325 lines
						
					
					
						
							11 KiB
						
					
					
				
			
		
		
		
			
			
			
		
		
	
	
							325 lines
						
					
					
						
							11 KiB
						
					
					
				
								package weed_server
							 | 
						|
								
							 | 
						|
								import (
							 | 
						|
									"bytes"
							 | 
						|
									"context"
							 | 
						|
									"crypto/md5"
							 | 
						|
									"encoding/base64"
							 | 
						|
									"fmt"
							 | 
						|
									"hash"
							 | 
						|
									"io"
							 | 
						|
									"net/http"
							 | 
						|
									"strconv"
							 | 
						|
									"sync"
							 | 
						|
									"time"
							 | 
						|
								
							 | 
						|
									"slices"
							 | 
						|
								
							 | 
						|
									"encoding/json"
							 | 
						|
								
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/glog"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/operation"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/security"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/stats"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/util"
							 | 
						|
								)
							 | 
						|
								
							 | 
						|
								var bufPool = sync.Pool{
							 | 
						|
									New: func() interface{} {
							 | 
						|
										return new(bytes.Buffer)
							 | 
						|
									},
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								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) {
							 | 
						|
									query := r.URL.Query()
							 | 
						|
								
							 | 
						|
									isAppend := isAppend(r)
							 | 
						|
									if query.Has("offset") {
							 | 
						|
										offset := query.Get("offset")
							 | 
						|
										offsetInt, err := strconv.ParseInt(offset, 10, 64)
							 | 
						|
										if err != nil || offsetInt < 0 {
							 | 
						|
											err = fmt.Errorf("invalid 'offset': '%s'", offset)
							 | 
						|
											return nil, nil, 0, err, nil
							 | 
						|
										}
							 | 
						|
										if isAppend && offsetInt > 0 {
							 | 
						|
											err = fmt.Errorf("cannot set offset when op=append")
							 | 
						|
											return nil, nil, 0, err, nil
							 | 
						|
										}
							 | 
						|
										chunkOffset = offsetInt
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									return fs.uploadReaderToChunks(ctx, r, reader, chunkOffset, chunkSize, fileName, contentType, isAppend, so)
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								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) {
							 | 
						|
								
							 | 
						|
									md5Hash = md5.New()
							 | 
						|
									chunkOffset = startOffset
							 | 
						|
									var partReader = io.NopCloser(io.TeeReader(reader, md5Hash))
							 | 
						|
								
							 | 
						|
									var wg sync.WaitGroup
							 | 
						|
									var bytesBufferCounter int64 = 4
							 | 
						|
									bytesBufferLimitChan := make(chan struct{}, bytesBufferCounter)
							 | 
						|
									var fileChunksLock sync.Mutex
							 | 
						|
									var uploadErrLock sync.Mutex
							 | 
						|
									for {
							 | 
						|
								
							 | 
						|
										// need to throttle used byte buffer
							 | 
						|
										bytesBufferLimitChan <- struct{}{}
							 | 
						|
								
							 | 
						|
										// As long as there is an error in the upload of one chunk, it can be terminated early
							 | 
						|
										// uploadErr may be modified in other go routines, lock is needed to avoid race condition
							 | 
						|
										uploadErrLock.Lock()
							 | 
						|
										if uploadErr != nil {
							 | 
						|
											<-bytesBufferLimitChan
							 | 
						|
											uploadErrLock.Unlock()
							 | 
						|
											break
							 | 
						|
										}
							 | 
						|
										uploadErrLock.Unlock()
							 | 
						|
								
							 | 
						|
										bytesBuffer := bufPool.Get().(*bytes.Buffer)
							 | 
						|
								
							 | 
						|
										limitedReader := io.LimitReader(partReader, int64(chunkSize))
							 | 
						|
								
							 | 
						|
										bytesBuffer.Reset()
							 | 
						|
								
							 | 
						|
										dataSize, err := bytesBuffer.ReadFrom(limitedReader)
							 | 
						|
								
							 | 
						|
										// data, err := io.ReadAll(limitedReader)
							 | 
						|
										if err != nil || dataSize == 0 {
							 | 
						|
											bufPool.Put(bytesBuffer)
							 | 
						|
											<-bytesBufferLimitChan
							 | 
						|
											if err != nil {
							 | 
						|
												uploadErrLock.Lock()
							 | 
						|
												if uploadErr == nil {
							 | 
						|
													uploadErr = err
							 | 
						|
												}
							 | 
						|
												uploadErrLock.Unlock()
							 | 
						|
											}
							 | 
						|
											break
							 | 
						|
										}
							 | 
						|
										if chunkOffset == 0 && !isAppend {
							 | 
						|
											if dataSize < fs.option.SaveToFilerLimit {
							 | 
						|
												chunkOffset += dataSize
							 | 
						|
												smallContent = make([]byte, dataSize)
							 | 
						|
												bytesBuffer.Read(smallContent)
							 | 
						|
												bufPool.Put(bytesBuffer)
							 | 
						|
												<-bytesBufferLimitChan
							 | 
						|
												stats.FilerHandlerCounter.WithLabelValues(stats.ContentSaveToFiler).Inc()
							 | 
						|
												break
							 | 
						|
											}
							 | 
						|
										} else {
							 | 
						|
											stats.FilerHandlerCounter.WithLabelValues(stats.AutoChunk).Inc()
							 | 
						|
										}
							 | 
						|
								
							 | 
						|
										wg.Add(1)
							 | 
						|
										go func(offset int64, buf *bytes.Buffer) {
							 | 
						|
											defer func() {
							 | 
						|
												bufPool.Put(buf)
							 | 
						|
												<-bytesBufferLimitChan
							 | 
						|
												wg.Done()
							 | 
						|
											}()
							 | 
						|
								
							 | 
						|
											chunks, toChunkErr := fs.dataToChunkWithSSE(ctx, r, fileName, contentType, buf.Bytes(), offset, so)
							 | 
						|
											if toChunkErr != nil {
							 | 
						|
												uploadErrLock.Lock()
							 | 
						|
												if uploadErr == nil {
							 | 
						|
													uploadErr = toChunkErr
							 | 
						|
												}
							 | 
						|
												uploadErrLock.Unlock()
							 | 
						|
											}
							 | 
						|
											if chunks != nil {
							 | 
						|
												fileChunksLock.Lock()
							 | 
						|
												fileChunksSize := len(fileChunks) + len(chunks)
							 | 
						|
												for _, chunk := range chunks {
							 | 
						|
													fileChunks = append(fileChunks, chunk)
							 | 
						|
													glog.V(4).InfofCtx(ctx, "uploaded %s chunk %d to %s [%d,%d)", fileName, fileChunksSize, chunk.FileId, offset, offset+int64(chunk.Size))
							 | 
						|
												}
							 | 
						|
												fileChunksLock.Unlock()
							 | 
						|
											}
							 | 
						|
										}(chunkOffset, bytesBuffer)
							 | 
						|
								
							 | 
						|
										// reset variables for the next chunk
							 | 
						|
										chunkOffset = chunkOffset + dataSize
							 | 
						|
								
							 | 
						|
										// if last chunk was not at full chunk size, but already exhausted the reader
							 | 
						|
										if dataSize < int64(chunkSize) {
							 | 
						|
											break
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									wg.Wait()
							 | 
						|
								
							 | 
						|
									if uploadErr != nil {
							 | 
						|
										glog.V(0).InfofCtx(ctx, "upload file %s error: %v", fileName, uploadErr)
							 | 
						|
										for _, chunk := range fileChunks {
							 | 
						|
											glog.V(4).InfofCtx(ctx, "purging failed uploaded %s chunk %s [%d,%d)", fileName, chunk.FileId, chunk.Offset, chunk.Offset+int64(chunk.Size))
							 | 
						|
										}
							 | 
						|
										fs.filer.DeleteUncommittedChunks(ctx, fileChunks)
							 | 
						|
										return nil, md5Hash, 0, uploadErr, nil
							 | 
						|
									}
							 | 
						|
									slices.SortFunc(fileChunks, func(a, b *filer_pb.FileChunk) int {
							 | 
						|
										return int(a.Offset - b.Offset)
							 | 
						|
									})
							 | 
						|
									return fileChunks, md5Hash, chunkOffset, nil, smallContent
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								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) {
							 | 
						|
								
							 | 
						|
									stats.FilerHandlerCounter.WithLabelValues(stats.ChunkUpload).Inc()
							 | 
						|
									start := time.Now()
							 | 
						|
									defer func() {
							 | 
						|
										stats.FilerRequestHistogram.WithLabelValues(stats.ChunkUpload).Observe(time.Since(start).Seconds())
							 | 
						|
									}()
							 | 
						|
								
							 | 
						|
									uploadOption := &operation.UploadOption{
							 | 
						|
										UploadUrl:         urlLocation,
							 | 
						|
										Filename:          fileName,
							 | 
						|
										Cipher:            fs.option.Cipher,
							 | 
						|
										IsInputCompressed: false,
							 | 
						|
										MimeType:          contentType,
							 | 
						|
										PairMap:           pairMap,
							 | 
						|
										Jwt:               auth,
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									uploader, err := operation.NewUploader()
							 | 
						|
									if err != nil {
							 | 
						|
										return nil, err, []byte{}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									uploadResult, err, data := uploader.Upload(ctx, limitedReader, uploadOption)
							 | 
						|
									if uploadResult != nil && uploadResult.RetryCount > 0 {
							 | 
						|
										stats.FilerHandlerCounter.WithLabelValues(stats.ChunkUploadRetry).Add(float64(uploadResult.RetryCount))
							 | 
						|
									}
							 | 
						|
									return uploadResult, err, data
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func (fs *FilerServer) dataToChunk(ctx context.Context, fileName, contentType string, data []byte, chunkOffset int64, so *operation.StorageOption) ([]*filer_pb.FileChunk, error) {
							 | 
						|
									return fs.dataToChunkWithSSE(ctx, nil, fileName, contentType, data, chunkOffset, so)
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func (fs *FilerServer) dataToChunkWithSSE(ctx context.Context, r *http.Request, fileName, contentType string, data []byte, chunkOffset int64, so *operation.StorageOption) ([]*filer_pb.FileChunk, error) {
							 | 
						|
									dataReader := util.NewBytesReader(data)
							 | 
						|
								
							 | 
						|
									// retry to assign a different file id
							 | 
						|
									var fileId, urlLocation string
							 | 
						|
									var auth security.EncodedJwt
							 | 
						|
									var uploadErr error
							 | 
						|
									var uploadResult *operation.UploadResult
							 | 
						|
									var failedFileChunks []*filer_pb.FileChunk
							 | 
						|
								
							 | 
						|
									err := util.Retry("filerDataToChunk", func() error {
							 | 
						|
										// assign one file id for one chunk
							 | 
						|
										fileId, urlLocation, auth, uploadErr = fs.assignNewFileInfo(ctx, so)
							 | 
						|
										if uploadErr != nil {
							 | 
						|
											glog.V(4).InfofCtx(ctx, "retry later due to assign error: %v", uploadErr)
							 | 
						|
											stats.FilerHandlerCounter.WithLabelValues(stats.ChunkAssignRetry).Inc()
							 | 
						|
											return uploadErr
							 | 
						|
										}
							 | 
						|
										// upload the chunk to the volume server
							 | 
						|
										uploadResult, uploadErr, _ = fs.doUpload(ctx, urlLocation, dataReader, fileName, contentType, nil, auth)
							 | 
						|
										if uploadErr != nil {
							 | 
						|
											glog.V(4).InfofCtx(ctx, "retry later due to upload error: %v", uploadErr)
							 | 
						|
											stats.FilerHandlerCounter.WithLabelValues(stats.ChunkDoUploadRetry).Inc()
							 | 
						|
											fid, _ := filer_pb.ToFileIdObject(fileId)
							 | 
						|
											fileChunk := filer_pb.FileChunk{
							 | 
						|
												FileId: fileId,
							 | 
						|
												Offset: chunkOffset,
							 | 
						|
												Fid:    fid,
							 | 
						|
											}
							 | 
						|
											failedFileChunks = append(failedFileChunks, &fileChunk)
							 | 
						|
											return uploadErr
							 | 
						|
										}
							 | 
						|
										return nil
							 | 
						|
									})
							 | 
						|
									if err != nil {
							 | 
						|
										glog.ErrorfCtx(ctx, "upload error: %v", err)
							 | 
						|
										return failedFileChunks, err
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// if last chunk exhausted the reader exactly at the border
							 | 
						|
									if uploadResult.Size == 0 {
							 | 
						|
										return nil, nil
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Extract SSE metadata from request headers if available
							 | 
						|
									var sseType filer_pb.SSEType = filer_pb.SSEType_NONE
							 | 
						|
									var sseMetadata []byte
							 | 
						|
								
							 | 
						|
									if r != nil {
							 | 
						|
								
							 | 
						|
										// Check for SSE-KMS
							 | 
						|
										sseKMSHeaderValue := r.Header.Get(s3_constants.SeaweedFSSSEKMSKeyHeader)
							 | 
						|
										if sseKMSHeaderValue != "" {
							 | 
						|
											sseType = filer_pb.SSEType_SSE_KMS
							 | 
						|
											if kmsData, err := base64.StdEncoding.DecodeString(sseKMSHeaderValue); err == nil {
							 | 
						|
												sseMetadata = kmsData
							 | 
						|
												glog.V(4).InfofCtx(ctx, "Storing SSE-KMS metadata for chunk %s at offset %d", fileId, chunkOffset)
							 | 
						|
											} else {
							 | 
						|
												glog.V(1).InfofCtx(ctx, "Failed to decode SSE-KMS metadata for chunk %s: %v", fileId, err)
							 | 
						|
											}
							 | 
						|
										} else if r.Header.Get(s3_constants.AmzServerSideEncryptionCustomerAlgorithm) != "" {
							 | 
						|
											// SSE-C: Create per-chunk metadata for unified handling
							 | 
						|
											sseType = filer_pb.SSEType_SSE_C
							 | 
						|
								
							 | 
						|
											// Get SSE-C metadata from headers to create unified per-chunk metadata
							 | 
						|
											sseIVHeader := r.Header.Get(s3_constants.SeaweedFSSSEIVHeader)
							 | 
						|
											keyMD5Header := r.Header.Get(s3_constants.AmzServerSideEncryptionCustomerKeyMD5)
							 | 
						|
								
							 | 
						|
											if sseIVHeader != "" && keyMD5Header != "" {
							 | 
						|
												// Decode IV from header
							 | 
						|
												if ivData, err := base64.StdEncoding.DecodeString(sseIVHeader); err == nil {
							 | 
						|
													// Create SSE-C metadata with chunk offset = chunkOffset for proper IV calculation
							 | 
						|
													ssecMetadataStruct := struct {
							 | 
						|
														Algorithm  string `json:"algorithm"`
							 | 
						|
														IV         string `json:"iv"`
							 | 
						|
														KeyMD5     string `json:"keyMD5"`
							 | 
						|
														PartOffset int64  `json:"partOffset"`
							 | 
						|
													}{
							 | 
						|
														Algorithm:  "AES256",
							 | 
						|
														IV:         base64.StdEncoding.EncodeToString(ivData),
							 | 
						|
														KeyMD5:     keyMD5Header,
							 | 
						|
														PartOffset: chunkOffset,
							 | 
						|
													}
							 | 
						|
													if ssecMetadata, serErr := json.Marshal(ssecMetadataStruct); serErr == nil {
							 | 
						|
														sseMetadata = ssecMetadata
							 | 
						|
													} else {
							 | 
						|
														glog.V(1).InfofCtx(ctx, "Failed to serialize SSE-C metadata for chunk %s: %v", fileId, serErr)
							 | 
						|
													}
							 | 
						|
												} else {
							 | 
						|
													glog.V(1).InfofCtx(ctx, "Failed to decode SSE-C IV for chunk %s: %v", fileId, err)
							 | 
						|
												}
							 | 
						|
											} else {
							 | 
						|
												glog.V(4).InfofCtx(ctx, "SSE-C chunk %s missing IV or KeyMD5 header", fileId)
							 | 
						|
											}
							 | 
						|
										} else if r.Header.Get(s3_constants.SeaweedFSSSES3Key) != "" {
							 | 
						|
											// SSE-S3: Server-side encryption with server-managed keys
							 | 
						|
											// Set the correct SSE type for SSE-S3 chunks to maintain proper tracking
							 | 
						|
											sseType = filer_pb.SSEType_SSE_S3
							 | 
						|
								
							 | 
						|
											// Get SSE-S3 metadata from headers
							 | 
						|
											sseS3Header := r.Header.Get(s3_constants.SeaweedFSSSES3Key)
							 | 
						|
											if sseS3Header != "" {
							 | 
						|
												if s3Data, err := base64.StdEncoding.DecodeString(sseS3Header); err == nil {
							 | 
						|
													// For SSE-S3, store metadata at chunk level for consistency with SSE-KMS/SSE-C
							 | 
						|
													glog.V(4).InfofCtx(ctx, "Storing SSE-S3 metadata for chunk %s at offset %d", fileId, chunkOffset)
							 | 
						|
													sseMetadata = s3Data
							 | 
						|
												} else {
							 | 
						|
													glog.V(1).InfofCtx(ctx, "Failed to decode SSE-S3 metadata for chunk %s: %v", fileId, err)
							 | 
						|
												}
							 | 
						|
											}
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create chunk with SSE metadata if available
							 | 
						|
									var chunk *filer_pb.FileChunk
							 | 
						|
									if sseType != filer_pb.SSEType_NONE {
							 | 
						|
										chunk = uploadResult.ToPbFileChunkWithSSE(fileId, chunkOffset, time.Now().UnixNano(), sseType, sseMetadata)
							 | 
						|
									} else {
							 | 
						|
										chunk = uploadResult.ToPbFileChunk(fileId, chunkOffset, time.Now().UnixNano())
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									return []*filer_pb.FileChunk{chunk}, nil
							 | 
						|
								}
							 |