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 | |
| }
 |