1
0
Fork 0
mirror of https://github.com/chrislusf/seaweedfs synced 2024-05-21 19:01:26 +02:00
seaweedfs/weed/server/filer_server_handlers_write_upload.go
Chris Lu d4566d4aaa
more solid weed mount (#4089)
* compare chunks by timestamp

* fix slab clearing error

* fix test compilation

* move oldest chunk to sealed, instead of by fullness

* lock on fh.entryViewCache

* remove verbose logs

* revert slat clearing

* less logs

* less logs

* track write and read by timestamp

* remove useless logic

* add entry lock on file handle release

* use mem chunk only, swap file chunk has problems

* comment out code that maybe used later

* add debug mode to compare data read and write

* more efficient readResolvedChunks with linked list

* small optimization

* fix test compilation

* minor fix on writer

* add SeparateGarbageChunks

* group chunks into sections

* turn off debug mode

* fix tests

* fix tests

* tmp enable swap file chunk

* Revert "tmp enable swap file chunk"

This reverts commit 985137ec47.

* simple refactoring

* simple refactoring

* do not re-use swap file chunk. Sealed chunks should not be re-used.

* comment out debugging facilities

* either mem chunk or swap file chunk is fine now

* remove orderedMutex  as *semaphore.Weighted

not found impactful

* optimize size calculation for changing large files

* optimize performance to avoid going through the long list of chunks

* still problems with swap file chunk

* rename

* tiny optimization

* swap file chunk save only successfully read data

* fix

* enable both mem and swap file chunk

* resolve chunks with range

* rename

* fix chunk interval list

* also change file handle chunk group when adding chunks

* pick in-active chunk with time-decayed counter

* fix compilation

* avoid nil with empty fh.entry

* refactoring

* rename

* rename

* refactor visible intervals to *list.List

* refactor chunkViews to *list.List

* add IntervalList for generic interval list

* change visible interval to use IntervalList in generics

* cahnge chunkViews to *IntervalList[*ChunkView]

* use NewFileChunkSection to create

* rename variables

* refactor

* fix renaming leftover

* renaming

* renaming

* add insert interval

* interval list adds lock

* incrementally add chunks to readers

Fixes:
1. set start and stop offset for the value object
2. clone the value object
3. use pointer instead of copy-by-value when passing to interval.Value
4. use insert interval since adding chunk could be out of order

* fix tests compilation

* fix tests compilation
2023-01-02 23:20:45 -08:00

219 lines
6.6 KiB
Go

package weed_server
import (
"bytes"
"crypto/md5"
"fmt"
"golang.org/x/exp/slices"
"hash"
"io"
"net/http"
"strconv"
"sync"
"sync/atomic"
"time"
"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/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) uploadReaderToChunks(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
}
md5Hash = md5.New()
var partReader = io.NopCloser(io.TeeReader(reader, md5Hash))
var wg sync.WaitGroup
var bytesBufferCounter int64
bytesBufferLimitCond := sync.NewCond(new(sync.Mutex))
var fileChunksLock sync.Mutex
var uploadErrLock sync.Mutex
for {
// need to throttle used byte buffer
bytesBufferLimitCond.L.Lock()
for atomic.LoadInt64(&bytesBufferCounter) >= 4 {
glog.V(4).Infof("waiting for byte buffer %d", atomic.LoadInt64(&bytesBufferCounter))
bytesBufferLimitCond.Wait()
}
atomic.AddInt64(&bytesBufferCounter, 1)
bytesBufferLimitCond.L.Unlock()
bytesBuffer := bufPool.Get().(*bytes.Buffer)
glog.V(4).Infof("received byte buffer %d", atomic.LoadInt64(&bytesBufferCounter))
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)
atomic.AddInt64(&bytesBufferCounter, -1)
bytesBufferLimitCond.Signal()
uploadErrLock.Lock()
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)
atomic.AddInt64(&bytesBufferCounter, -1)
bytesBufferLimitCond.Signal()
stats.FilerRequestCounter.WithLabelValues(stats.ContentSaveToFiler).Inc()
break
}
} else {
stats.FilerRequestCounter.WithLabelValues(stats.AutoChunk).Inc()
}
wg.Add(1)
go func(offset int64) {
defer func() {
bufPool.Put(bytesBuffer)
atomic.AddInt64(&bytesBufferCounter, -1)
bytesBufferLimitCond.Signal()
wg.Done()
}()
chunks, toChunkErr := fs.dataToChunk(fileName, contentType, bytesBuffer.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).Infof("uploaded %s chunk %d to %s [%d,%d)", fileName, fileChunksSize, chunk.FileId, offset, offset+int64(chunk.Size))
}
fileChunksLock.Unlock()
}
}(chunkOffset)
// 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 {
fs.filer.DeleteChunks(fileChunks)
return nil, md5Hash, 0, uploadErr, nil
}
slices.SortFunc(fileChunks, func(a, b *filer_pb.FileChunk) bool {
return a.Offset < b.Offset
})
return fileChunks, md5Hash, chunkOffset, nil, smallContent
}
func (fs *FilerServer) doUpload(urlLocation string, limitedReader io.Reader, fileName string, contentType string, pairMap map[string]string, auth security.EncodedJwt) (*operation.UploadResult, error, []byte) {
stats.FilerRequestCounter.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,
}
uploadResult, err, data := operation.Upload(limitedReader, uploadOption)
if uploadResult != nil && uploadResult.RetryCount > 0 {
stats.FilerRequestCounter.WithLabelValues(stats.ChunkUploadRetry).Add(float64(uploadResult.RetryCount))
}
return uploadResult, err, data
}
func (fs *FilerServer) dataToChunk(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(so)
if uploadErr != nil {
glog.V(4).Infof("retry later due to assign error: %v", uploadErr)
stats.FilerRequestCounter.WithLabelValues(stats.ChunkAssignRetry).Inc()
return uploadErr
}
// upload the chunk to the volume server
uploadResult, uploadErr, _ = fs.doUpload(urlLocation, dataReader, fileName, contentType, nil, auth)
if uploadErr != nil {
glog.V(4).Infof("retry later due to upload error: %v", uploadErr)
stats.FilerRequestCounter.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.Errorf("upload error: %v", err)
return failedFileChunks, err
}
// if last chunk exhausted the reader exactly at the border
if uploadResult.Size == 0 {
return nil, nil
}
return []*filer_pb.FileChunk{uploadResult.ToPbFileChunk(fileId, chunkOffset, time.Now().UnixNano())}, nil
}