Merge pull request #2596 from chrislusf/memory_chunked_write

fix concurrency issues found in tests
This commit is contained in:
Chris Lu 2022-01-17 23:04:30 -08:00 committed by GitHub
commit e271721d4e
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
18 changed files with 460 additions and 1089 deletions

View file

@ -26,6 +26,8 @@ type MountOptions struct {
uidMap *string
gidMap *string
readOnly *bool
debug *bool
debugPort *int
}
var (
@ -57,6 +59,8 @@ func init() {
mountOptions.uidMap = cmdMount.Flag.String("map.uid", "", "map local uid to uid on filer, comma-separated <local_uid>:<filer_uid>")
mountOptions.gidMap = cmdMount.Flag.String("map.gid", "", "map local gid to gid on filer, comma-separated <local_gid>:<filer_gid>")
mountOptions.readOnly = cmdMount.Flag.Bool("readOnly", false, "read only")
mountOptions.debug = cmdMount.Flag.Bool("debug", false, "serves runtime profiling data, e.g., http://localhost:<debug.port>/debug/pprof/goroutine?debug=2")
mountOptions.debugPort = cmdMount.Flag.Int("debug.port", 6061, "http port for debugging")
mountCpuProfile = cmdMount.Flag.String("cpuprofile", "", "cpu profile output file")
mountMemProfile = cmdMount.Flag.String("memprofile", "", "memory profile output file")

View file

@ -6,6 +6,7 @@ package command
import (
"context"
"fmt"
"net/http"
"os"
"os/user"
"path"
@ -34,6 +35,10 @@ import (
func runMount(cmd *Command, args []string) bool {
if *mountOptions.debug {
go http.ListenAndServe(fmt.Sprintf(":%d", *mountOptions.debugPort), nil)
}
grace.SetupProfiling(*mountCpuProfile, *mountMemProfile)
if *mountReadRetryTime < time.Second {
*mountReadRetryTime = time.Second

View file

@ -1,138 +0,0 @@
package filesys
import (
"bytes"
"fmt"
"github.com/chrislusf/seaweedfs/weed/filesys/page_writer"
"io"
"sync"
"time"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
)
type ContinuousDirtyPages struct {
intervals *page_writer.ContinuousIntervals
f *File
writeWaitGroup sync.WaitGroup
chunkAddLock sync.Mutex
lastErr error
collection string
replication string
}
func newContinuousDirtyPages(file *File) *ContinuousDirtyPages {
dirtyPages := &ContinuousDirtyPages{
intervals: &page_writer.ContinuousIntervals{},
f: file,
}
return dirtyPages
}
func (pages *ContinuousDirtyPages) AddPage(offset int64, data []byte) {
glog.V(4).Infof("%s AddPage [%d,%d)", pages.f.fullpath(), offset, offset+int64(len(data)))
if len(data) > int(pages.f.wfs.option.ChunkSizeLimit) {
// this is more than what buffer can hold.
pages.flushAndSave(offset, data)
}
pages.intervals.AddInterval(data, offset)
if pages.intervals.TotalSize() >= pages.f.wfs.option.ChunkSizeLimit {
pages.saveExistingLargestPageToStorage()
}
return
}
func (pages *ContinuousDirtyPages) flushAndSave(offset int64, data []byte) {
// flush existing
pages.saveExistingPagesToStorage()
// flush the new page
pages.saveToStorage(bytes.NewReader(data), offset, int64(len(data)))
return
}
func (pages *ContinuousDirtyPages) FlushData() error {
pages.saveExistingPagesToStorage()
pages.writeWaitGroup.Wait()
if pages.lastErr != nil {
return fmt.Errorf("flush data: %v", pages.lastErr)
}
return nil
}
func (pages *ContinuousDirtyPages) saveExistingPagesToStorage() {
for pages.saveExistingLargestPageToStorage() {
}
}
func (pages *ContinuousDirtyPages) saveExistingLargestPageToStorage() (hasSavedData bool) {
maxList := pages.intervals.RemoveLargestIntervalLinkedList()
if maxList == nil {
return false
}
entry := pages.f.getEntry()
if entry == nil {
return false
}
fileSize := int64(entry.Attributes.FileSize)
chunkSize := min(maxList.Size(), fileSize-maxList.Offset())
if chunkSize == 0 {
return false
}
pages.saveToStorage(maxList.ToReader(), maxList.Offset(), chunkSize)
return true
}
func (pages *ContinuousDirtyPages) saveToStorage(reader io.Reader, offset int64, size int64) {
mtime := time.Now().UnixNano()
pages.writeWaitGroup.Add(1)
writer := func() {
defer pages.writeWaitGroup.Done()
reader = io.LimitReader(reader, size)
chunk, collection, replication, err := pages.f.wfs.saveDataAsChunk(pages.f.fullpath())(reader, pages.f.Name, offset)
if err != nil {
glog.V(0).Infof("%s saveToStorage [%d,%d): %v", pages.f.fullpath(), offset, offset+size, err)
pages.lastErr = err
return
}
chunk.Mtime = mtime
pages.collection, pages.replication = collection, replication
pages.chunkAddLock.Lock()
defer pages.chunkAddLock.Unlock()
pages.f.addChunks([]*filer_pb.FileChunk{chunk})
glog.V(3).Infof("%s saveToStorage [%d,%d)", pages.f.fullpath(), offset, offset+size)
}
if pages.f.wfs.concurrentWriters != nil {
pages.f.wfs.concurrentWriters.Execute(writer)
} else {
go writer()
}
}
func (pages *ContinuousDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) {
return pages.intervals.ReadDataAt(data, startOffset)
}
func (pages *ContinuousDirtyPages) GetStorageOptions() (collection, replication string) {
return pages.collection, pages.replication
}
func (pages ContinuousDirtyPages) Destroy() {
}

View file

@ -0,0 +1,101 @@
package filesys
import (
"fmt"
"github.com/chrislusf/seaweedfs/weed/filesys/page_writer"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"io"
"sync"
"time"
)
type MemoryChunkPages struct {
fh *FileHandle
writeWaitGroup sync.WaitGroup
chunkAddLock sync.Mutex
lastErr error
collection string
replication string
uploadPipeline *page_writer.UploadPipeline
hasWrites bool
}
var (
_ = page_writer.DirtyPages(&MemoryChunkPages{})
)
func newMemoryChunkPages(fh *FileHandle, chunkSize int64) *MemoryChunkPages {
dirtyPages := &MemoryChunkPages{
fh: fh,
}
dirtyPages.uploadPipeline = page_writer.NewUploadPipeline(fh.f.fullpath(),
fh.f.wfs.concurrentWriters, chunkSize, dirtyPages.saveChunkedFileIntevalToStorage)
return dirtyPages
}
func (pages *MemoryChunkPages) AddPage(offset int64, data []byte) {
pages.hasWrites = true
glog.V(4).Infof("%v memory AddPage [%d, %d)", pages.fh.f.fullpath(), offset, offset+int64(len(data)))
pages.uploadPipeline.SaveDataAt(data, offset)
return
}
func (pages *MemoryChunkPages) FlushData() error {
if !pages.hasWrites {
return nil
}
pages.uploadPipeline.FlushAll()
if pages.lastErr != nil {
return fmt.Errorf("flush data: %v", pages.lastErr)
}
return nil
}
func (pages *MemoryChunkPages) ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) {
if !pages.hasWrites {
return
}
return pages.uploadPipeline.MaybeReadDataAt(data, startOffset)
}
func (pages *MemoryChunkPages) GetStorageOptions() (collection, replication string) {
return pages.collection, pages.replication
}
func (pages *MemoryChunkPages) saveChunkedFileIntevalToStorage(reader io.Reader, offset int64, size int64, cleanupFn func()) {
mtime := time.Now().UnixNano()
defer cleanupFn()
chunk, collection, replication, err := pages.fh.f.wfs.saveDataAsChunk(pages.fh.f.fullpath())(reader, pages.fh.f.Name, offset)
if err != nil {
glog.V(0).Infof("%s saveToStorage [%d,%d): %v", pages.fh.f.fullpath(), offset, offset+size, err)
pages.lastErr = err
return
}
chunk.Mtime = mtime
pages.collection, pages.replication = collection, replication
pages.chunkAddLock.Lock()
pages.fh.f.addChunks([]*filer_pb.FileChunk{chunk})
pages.fh.entryViewCache = nil
glog.V(3).Infof("%s saveToStorage %s [%d,%d)", pages.fh.f.fullpath(), chunk.FileId, offset, offset+size)
pages.chunkAddLock.Unlock()
}
func (pages MemoryChunkPages) Destroy() {
pages.uploadPipeline.Shutdown()
}
func (pages *MemoryChunkPages) LockForRead(startOffset, stopOffset int64) {
pages.uploadPipeline.LockForRead(startOffset, stopOffset)
}
func (pages *MemoryChunkPages) UnlockForRead(startOffset, stopOffset int64) {
pages.uploadPipeline.UnlockForRead(startOffset, stopOffset)
}

View file

@ -1,106 +0,0 @@
package filesys
import (
"fmt"
"github.com/chrislusf/seaweedfs/weed/filesys/page_writer"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"io"
"sync"
"time"
)
type StreamDirtyPages struct {
f *File
writeWaitGroup sync.WaitGroup
pageAddLock sync.Mutex
chunkAddLock sync.Mutex
lastErr error
collection string
replication string
chunkedStream *page_writer.ChunkedStreamWriter
}
func newStreamDirtyPages(file *File, chunkSize int64) *StreamDirtyPages {
dirtyPages := &StreamDirtyPages{
f: file,
chunkedStream: page_writer.NewChunkedStreamWriter(chunkSize),
}
dirtyPages.chunkedStream.SetSaveToStorageFunction(dirtyPages.saveChunkedFileIntevalToStorage)
return dirtyPages
}
func (pages *StreamDirtyPages) AddPage(offset int64, data []byte) {
pages.pageAddLock.Lock()
defer pages.pageAddLock.Unlock()
glog.V(4).Infof("%v stream AddPage [%d, %d)", pages.f.fullpath(), offset, offset+int64(len(data)))
if _, err := pages.chunkedStream.WriteAt(data, offset); err != nil {
pages.lastErr = err
}
return
}
func (pages *StreamDirtyPages) FlushData() error {
pages.saveChunkedFileToStorage()
pages.writeWaitGroup.Wait()
if pages.lastErr != nil {
return fmt.Errorf("flush data: %v", pages.lastErr)
}
pages.chunkedStream.Reset()
return nil
}
func (pages *StreamDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) {
return pages.chunkedStream.ReadDataAt(data, startOffset)
}
func (pages *StreamDirtyPages) GetStorageOptions() (collection, replication string) {
return pages.collection, pages.replication
}
func (pages *StreamDirtyPages) saveChunkedFileToStorage() {
pages.chunkedStream.FlushAll()
}
func (pages *StreamDirtyPages) saveChunkedFileIntevalToStorage(reader io.Reader, offset int64, size int64, cleanupFn func()) {
mtime := time.Now().UnixNano()
pages.writeWaitGroup.Add(1)
writer := func() {
defer pages.writeWaitGroup.Done()
defer cleanupFn()
chunk, collection, replication, err := pages.f.wfs.saveDataAsChunk(pages.f.fullpath())(reader, pages.f.Name, offset)
if err != nil {
glog.V(0).Infof("%s saveToStorage [%d,%d): %v", pages.f.fullpath(), offset, offset+size, err)
pages.lastErr = err
return
}
chunk.Mtime = mtime
pages.collection, pages.replication = collection, replication
pages.chunkAddLock.Lock()
pages.f.addChunks([]*filer_pb.FileChunk{chunk})
glog.V(3).Infof("%s saveToStorage %s [%d,%d)", pages.f.fullpath(), chunk.FileId, offset, offset+size)
pages.chunkAddLock.Unlock()
}
if pages.f.wfs.concurrentWriters != nil {
pages.f.wfs.concurrentWriters.Execute(writer)
} else {
go writer()
}
}
func (pages StreamDirtyPages) Destroy() {
pages.chunkedStream.Reset()
}

View file

@ -1,106 +0,0 @@
package filesys
import (
"fmt"
"github.com/chrislusf/seaweedfs/weed/filesys/page_writer"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"io"
"os"
"sync"
"time"
)
type TempFileDirtyPages struct {
f *File
writeWaitGroup sync.WaitGroup
pageAddLock sync.Mutex
chunkAddLock sync.Mutex
lastErr error
collection string
replication string
chunkedFile *page_writer.ChunkedFileWriter
}
func newTempFileDirtyPages(file *File, chunkSize int64) *TempFileDirtyPages {
tempFile := &TempFileDirtyPages{
f: file,
chunkedFile: page_writer.NewChunkedFileWriter(file.wfs.option.getTempFilePageDir(), chunkSize),
}
return tempFile
}
func (pages *TempFileDirtyPages) AddPage(offset int64, data []byte) {
pages.pageAddLock.Lock()
defer pages.pageAddLock.Unlock()
glog.V(4).Infof("%v tempfile AddPage [%d, %d)", pages.f.fullpath(), offset, offset+int64(len(data)))
if _, err := pages.chunkedFile.WriteAt(data, offset); err != nil {
pages.lastErr = err
}
return
}
func (pages *TempFileDirtyPages) FlushData() error {
pages.saveChunkedFileToStorage()
pages.writeWaitGroup.Wait()
if pages.lastErr != nil {
return fmt.Errorf("flush data: %v", pages.lastErr)
}
pages.chunkedFile.Reset()
return nil
}
func (pages *TempFileDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) {
return pages.chunkedFile.ReadDataAt(data, startOffset)
}
func (pages *TempFileDirtyPages) GetStorageOptions() (collection, replication string) {
return pages.collection, pages.replication
}
func (pages *TempFileDirtyPages) saveChunkedFileToStorage() {
pages.chunkedFile.ProcessEachInterval(func(file *os.File, logicChunkIndex page_writer.LogicChunkIndex, interval *page_writer.ChunkWrittenInterval) {
reader := page_writer.NewFileIntervalReader(pages.chunkedFile, logicChunkIndex, interval)
pages.saveChunkedFileIntevalToStorage(reader, int64(logicChunkIndex)*pages.chunkedFile.ChunkSize+interval.StartOffset, interval.Size())
})
}
func (pages *TempFileDirtyPages) saveChunkedFileIntevalToStorage(reader io.Reader, offset int64, size int64) {
mtime := time.Now().UnixNano()
pages.writeWaitGroup.Add(1)
writer := func() {
defer pages.writeWaitGroup.Done()
chunk, collection, replication, err := pages.f.wfs.saveDataAsChunk(pages.f.fullpath())(reader, pages.f.Name, offset)
if err != nil {
glog.V(0).Infof("%s saveToStorage [%d,%d): %v", pages.f.fullpath(), offset, offset+size, err)
pages.lastErr = err
return
}
chunk.Mtime = mtime
pages.collection, pages.replication = collection, replication
pages.chunkAddLock.Lock()
defer pages.chunkAddLock.Unlock()
pages.f.addChunks([]*filer_pb.FileChunk{chunk})
glog.V(3).Infof("%s saveToStorage %s [%d,%d)", pages.f.fullpath(), chunk.FileId, offset, offset+size)
}
if pages.f.wfs.concurrentWriters != nil {
pages.f.wfs.concurrentWriters.Execute(writer)
} else {
go writer()
}
}
func (pages TempFileDirtyPages) Destroy() {
pages.chunkedFile.Reset()
}

View file

@ -140,7 +140,15 @@ func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *f
}
}
}
// set the new chunks and reset entry cache
entry.Chunks = chunks
file.wfs.handlesLock.Lock()
existingHandle, found := file.wfs.handles[file.Id()]
file.wfs.handlesLock.Unlock()
if found {
existingHandle.entryViewCache = nil
}
}
entry.Attributes.Mtime = time.Now().Unix()
entry.Attributes.FileSize = req.Size

View file

@ -28,22 +28,20 @@ type FileHandle struct {
sync.Mutex
f *File
RequestId fuse.RequestID // unique ID for request
NodeId fuse.NodeID // file or directory the request is about
Uid uint32 // user ID of process making request
Gid uint32 // group ID of process making request
writeOnly bool
NodeId fuse.NodeID // file or directory the request is about
Uid uint32 // user ID of process making request
Gid uint32 // group ID of process making request
isDeleted bool
}
func newFileHandle(file *File, uid, gid uint32) *FileHandle {
fh := &FileHandle{
f: file,
// dirtyPages: newContinuousDirtyPages(file, writeOnly),
dirtyPages: newPageWriter(file, file.wfs.option.ChunkSizeLimit),
Uid: uid,
Gid: gid,
f: file,
Uid: uid,
Gid: gid,
}
// dirtyPages: newContinuousDirtyPages(file, writeOnly),
fh.dirtyPages = newPageWriter(fh, file.wfs.option.ChunkSizeLimit)
entry := fh.f.getEntry()
if entry != nil {
entry.Attributes.FileSize = filer.FileSize(entry)
@ -77,6 +75,8 @@ func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fus
buff = make([]byte, req.Size)
}
fh.lockForRead(req.Offset, len(buff))
defer fh.unlockForRead(req.Offset, len(buff))
totalRead, err := fh.readFromChunks(buff, req.Offset)
if err == nil || err == io.EOF {
maxStop := fh.readFromDirtyPages(buff, req.Offset)
@ -103,6 +103,13 @@ func (fh *FileHandle) Read(ctx context.Context, req *fuse.ReadRequest, resp *fus
return err
}
func (fh *FileHandle) lockForRead(startOffset int64, size int) {
fh.dirtyPages.LockForRead(startOffset, startOffset+int64(size))
}
func (fh *FileHandle) unlockForRead(startOffset int64, size int) {
fh.dirtyPages.UnlockForRead(startOffset, startOffset+int64(size))
}
func (fh *FileHandle) readFromDirtyPages(buff []byte, startOffset int64) (maxStop int64) {
maxStop = fh.dirtyPages.ReadDirtyDataAt(buff, startOffset)
return
@ -151,6 +158,10 @@ func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
reader := fh.reader
if reader == nil {
chunkViews := filer.ViewFromVisibleIntervals(fh.entryViewCache, 0, math.MaxInt64)
glog.V(4).Infof("file handle read %s [%d,%d) from %+v", fileFullPath, offset, offset+int64(len(buff)), chunkViews)
for _, chunkView := range chunkViews {
glog.V(4).Infof(" read %s [%d,%d) from chunk %+v", fileFullPath, chunkView.LogicOffset, chunkView.LogicOffset+int64(chunkView.Size), chunkView.FileId)
}
reader = filer.NewChunkReaderAtFromClient(fh.f.wfs.LookupFn(), chunkViews, fh.f.wfs.chunkCache, fileSize)
}
fh.reader = reader
@ -161,7 +172,7 @@ func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
glog.Errorf("file handle read %s: %v", fileFullPath, err)
}
// glog.V(4).Infof("file handle read %s [%d,%d] %d : %v", fileFullPath, offset, offset+int64(totalRead), totalRead, err)
glog.V(4).Infof("file handle read %s [%d,%d] %d : %v", fileFullPath, offset, offset+int64(totalRead), totalRead, err)
return int64(totalRead), err
}

View file

@ -6,36 +6,32 @@ import (
)
type PageWriter struct {
f *File
fh *FileHandle
collection string
replication string
chunkSize int64
writerPattern *WriterPattern
randomWriter page_writer.DirtyPages
streamWriter page_writer.DirtyPages
}
var (
_ = page_writer.DirtyPages(&PageWriter{})
)
func newPageWriter(file *File, chunkSize int64) *PageWriter {
func newPageWriter(fh *FileHandle, chunkSize int64) *PageWriter {
pw := &PageWriter{
f: file,
fh: fh,
chunkSize: chunkSize,
writerPattern: NewWriterPattern(chunkSize),
randomWriter: newTempFileDirtyPages(file, chunkSize),
streamWriter: newStreamDirtyPages(file, chunkSize),
//streamWriter: newContinuousDirtyPages(file),
//streamWriter: nil,
randomWriter: newMemoryChunkPages(fh, chunkSize),
}
return pw
}
func (pw *PageWriter) AddPage(offset int64, data []byte) {
glog.V(4).Infof("%v AddPage [%d, %d) streaming:%v", pw.f.fullpath(), offset, offset+int64(len(data)), pw.writerPattern.IsStreamingMode())
glog.V(4).Infof("%v AddPage [%d, %d) streaming:%v", pw.fh.f.fullpath(), offset, offset+int64(len(data)), pw.writerPattern.IsStreamingMode())
chunkIndex := offset / pw.chunkSize
for i := chunkIndex; len(data) > 0; i++ {
@ -47,38 +43,22 @@ func (pw *PageWriter) AddPage(offset int64, data []byte) {
}
func (pw *PageWriter) addToOneChunk(chunkIndex, offset int64, data []byte) {
if chunkIndex > 0 {
if pw.writerPattern.IsStreamingMode() && pw.streamWriter != nil {
pw.streamWriter.AddPage(offset, data)
return
}
}
pw.randomWriter.AddPage(offset, data)
}
func (pw *PageWriter) FlushData() error {
pw.writerPattern.Reset()
if pw.streamWriter != nil {
if err := pw.streamWriter.FlushData(); err != nil {
return err
}
}
return pw.randomWriter.FlushData()
}
func (pw *PageWriter) ReadDirtyDataAt(data []byte, offset int64) (maxStop int64) {
glog.V(4).Infof("ReadDirtyDataAt %v [%d, %d)", pw.f.fullpath(), offset, offset+int64(len(data)))
glog.V(4).Infof("ReadDirtyDataAt %v [%d, %d)", pw.fh.f.fullpath(), offset, offset+int64(len(data)))
chunkIndex := offset / pw.chunkSize
for i := chunkIndex; len(data) > 0; i++ {
readSize := min(int64(len(data)), (i+1)*pw.chunkSize-offset)
if pw.streamWriter != nil {
m1 := pw.streamWriter.ReadDirtyDataAt(data[:readSize], offset)
maxStop = max(maxStop, m1)
}
m2 := pw.randomWriter.ReadDirtyDataAt(data[:readSize], offset)
maxStop = max(maxStop, m2)
maxStop = pw.randomWriter.ReadDirtyDataAt(data[:readSize], offset)
offset += readSize
data = data[readSize:]
@ -88,16 +68,18 @@ func (pw *PageWriter) ReadDirtyDataAt(data []byte, offset int64) (maxStop int64)
}
func (pw *PageWriter) GetStorageOptions() (collection, replication string) {
if pw.writerPattern.IsStreamingMode() && pw.streamWriter != nil {
return pw.streamWriter.GetStorageOptions()
}
return pw.randomWriter.GetStorageOptions()
}
func (pw *PageWriter) LockForRead(startOffset, stopOffset int64) {
pw.randomWriter.LockForRead(startOffset, stopOffset)
}
func (pw *PageWriter) UnlockForRead(startOffset, stopOffset int64) {
pw.randomWriter.UnlockForRead(startOffset, stopOffset)
}
func (pw *PageWriter) Destroy() {
if pw.streamWriter != nil {
pw.streamWriter.Destroy()
}
pw.randomWriter.Destroy()
}

View file

@ -1,160 +0,0 @@
package page_writer
import (
"github.com/chrislusf/seaweedfs/weed/glog"
"io"
"os"
"sync"
)
type LogicChunkIndex int
type ActualChunkIndex int
// ChunkedFileWriter assumes the write requests will come in within chunks
type ChunkedFileWriter struct {
dir string
file *os.File
logicToActualChunkIndex map[LogicChunkIndex]ActualChunkIndex
chunkUsages []*ChunkWrittenIntervalList
ChunkSize int64
sync.Mutex
}
var _ = io.WriterAt(&ChunkedFileWriter{})
func NewChunkedFileWriter(dir string, chunkSize int64) *ChunkedFileWriter {
return &ChunkedFileWriter{
dir: dir,
file: nil,
logicToActualChunkIndex: make(map[LogicChunkIndex]ActualChunkIndex),
ChunkSize: chunkSize,
}
}
func (cw *ChunkedFileWriter) WriteAt(p []byte, off int64) (n int, err error) {
cw.Lock()
defer cw.Unlock()
if cw.file == nil {
cw.file, err = os.CreateTemp(cw.dir, "")
if err != nil {
glog.Errorf("create temp file: %v", err)
return
}
}
actualOffset, chunkUsage := cw.toActualWriteOffset(off)
n, err = cw.file.WriteAt(p, actualOffset)
if err == nil {
startOffset := off % cw.ChunkSize
chunkUsage.MarkWritten(startOffset, startOffset+int64(n))
}
return
}
func (cw *ChunkedFileWriter) ReadDataAt(p []byte, off int64) (maxStop int64) {
cw.Lock()
defer cw.Unlock()
if cw.file == nil {
return
}
logicChunkIndex := off / cw.ChunkSize
actualChunkIndex, chunkUsage := cw.toActualReadOffset(off)
if chunkUsage != nil {
for t := chunkUsage.head.next; t != chunkUsage.tail; t = t.next {
logicStart := max(off, logicChunkIndex*cw.ChunkSize+t.StartOffset)
logicStop := min(off+int64(len(p)), logicChunkIndex*cw.ChunkSize+t.stopOffset)
if logicStart < logicStop {
actualStart := logicStart - logicChunkIndex*cw.ChunkSize + int64(actualChunkIndex)*cw.ChunkSize
_, err := cw.file.ReadAt(p[logicStart-off:logicStop-off], actualStart)
if err != nil {
glog.Errorf("reading temp file: %v", err)
break
}
maxStop = max(maxStop, logicStop)
}
}
}
return
}
func (cw *ChunkedFileWriter) toActualWriteOffset(logicOffset int64) (actualOffset int64, chunkUsage *ChunkWrittenIntervalList) {
logicChunkIndex := LogicChunkIndex(logicOffset / cw.ChunkSize)
offsetRemainder := logicOffset % cw.ChunkSize
existingActualChunkIndex, found := cw.logicToActualChunkIndex[logicChunkIndex]
if found {
return int64(existingActualChunkIndex)*cw.ChunkSize + offsetRemainder, cw.chunkUsages[existingActualChunkIndex]
}
cw.logicToActualChunkIndex[logicChunkIndex] = ActualChunkIndex(len(cw.chunkUsages))
chunkUsage = newChunkWrittenIntervalList()
cw.chunkUsages = append(cw.chunkUsages, chunkUsage)
return int64(len(cw.chunkUsages)-1)*cw.ChunkSize + offsetRemainder, chunkUsage
}
func (cw *ChunkedFileWriter) toActualReadOffset(logicOffset int64) (actualChunkIndex ActualChunkIndex, chunkUsage *ChunkWrittenIntervalList) {
logicChunkIndex := LogicChunkIndex(logicOffset / cw.ChunkSize)
existingActualChunkIndex, found := cw.logicToActualChunkIndex[logicChunkIndex]
if found {
return existingActualChunkIndex, cw.chunkUsages[existingActualChunkIndex]
}
return 0, nil
}
func (cw *ChunkedFileWriter) ProcessEachInterval(process func(file *os.File, logicChunkIndex LogicChunkIndex, interval *ChunkWrittenInterval)) {
for logicChunkIndex, actualChunkIndex := range cw.logicToActualChunkIndex {
chunkUsage := cw.chunkUsages[actualChunkIndex]
for t := chunkUsage.head.next; t != chunkUsage.tail; t = t.next {
process(cw.file, logicChunkIndex, t)
}
}
}
// Reset releases used resources
func (cw *ChunkedFileWriter) Reset() {
if cw.file != nil {
cw.file.Close()
os.Remove(cw.file.Name())
cw.file = nil
}
cw.logicToActualChunkIndex = make(map[LogicChunkIndex]ActualChunkIndex)
cw.chunkUsages = cw.chunkUsages[:0]
}
type FileIntervalReader struct {
f *os.File
startOffset int64
stopOffset int64
position int64
}
var _ = io.Reader(&FileIntervalReader{})
func NewFileIntervalReader(cw *ChunkedFileWriter, logicChunkIndex LogicChunkIndex, interval *ChunkWrittenInterval) *FileIntervalReader {
actualChunkIndex, found := cw.logicToActualChunkIndex[logicChunkIndex]
if !found {
// this should never happen
return nil
}
return &FileIntervalReader{
f: cw.file,
startOffset: int64(actualChunkIndex)*cw.ChunkSize + interval.StartOffset,
stopOffset: int64(actualChunkIndex)*cw.ChunkSize + interval.stopOffset,
position: 0,
}
}
func (fr *FileIntervalReader) Read(p []byte) (n int, err error) {
readSize := minInt(len(p), int(fr.stopOffset-fr.startOffset-fr.position))
n, err = fr.f.ReadAt(p[:readSize], fr.startOffset+fr.position)
if err == nil || err == io.EOF {
fr.position += int64(n)
if fr.stopOffset-fr.startOffset-fr.position == 0 {
// return a tiny bit faster
err = io.EOF
return
}
}
return
}

View file

@ -1,60 +0,0 @@
package page_writer
import (
"github.com/stretchr/testify/assert"
"os"
"testing"
)
func TestChunkedFileWriter_toActualOffset(t *testing.T) {
cw := NewChunkedFileWriter("", 16)
writeToFile(cw, 50, 60)
writeToFile(cw, 60, 64)
writeToFile(cw, 32, 40)
writeToFile(cw, 42, 48)
writeToFile(cw, 48, 50)
assert.Equal(t, 1, cw.chunkUsages[0].size(), "fully covered")
assert.Equal(t, 2, cw.chunkUsages[1].size(), "2 intervals")
}
func writeToFile(cw *ChunkedFileWriter, startOffset int64, stopOffset int64) {
_, chunkUsage := cw.toActualWriteOffset(startOffset)
// skip doing actual writing
innerOffset := startOffset % cw.ChunkSize
chunkUsage.MarkWritten(innerOffset, innerOffset+stopOffset-startOffset)
}
func TestWriteChunkedFile(t *testing.T) {
x := NewChunkedFileWriter(os.TempDir(), 20)
defer x.Reset()
y := NewChunkedFileWriter(os.TempDir(), 12)
defer y.Reset()
batchSize := 4
buf := make([]byte, batchSize)
for i := 0; i < 256; i++ {
for x := 0; x < batchSize; x++ {
buf[x] = byte(i)
}
x.WriteAt(buf, int64(i*batchSize))
y.WriteAt(buf, int64((255-i)*batchSize))
}
a := make([]byte, 1)
b := make([]byte, 1)
for i := 0; i < 256*batchSize; i++ {
x.ReadDataAt(a, int64(i))
y.ReadDataAt(b, int64(256*batchSize-1-i))
assert.Equal(t, a[0], b[0], "same read")
}
}

View file

@ -1,119 +1,12 @@
package page_writer
import (
"github.com/chrislusf/seaweedfs/weed/util"
"github.com/chrislusf/seaweedfs/weed/util/mem"
"io"
"sync"
"sync/atomic"
)
type SaveToStorageFunc func(reader io.Reader, offset int64, size int64, cleanupFn func())
// ChunkedStreamWriter assumes the write requests will come in within chunks and in streaming mode
type ChunkedStreamWriter struct {
activeChunks map[LogicChunkIndex]*MemChunk
activeChunksLock sync.Mutex
ChunkSize int64
saveToStorageFn SaveToStorageFunc
sync.Mutex
}
type MemChunk struct {
buf []byte
usage *ChunkWrittenIntervalList
}
var _ = io.WriterAt(&ChunkedStreamWriter{})
func NewChunkedStreamWriter(chunkSize int64) *ChunkedStreamWriter {
return &ChunkedStreamWriter{
ChunkSize: chunkSize,
activeChunks: make(map[LogicChunkIndex]*MemChunk),
}
}
func (cw *ChunkedStreamWriter) SetSaveToStorageFunction(saveToStorageFn SaveToStorageFunc) {
cw.saveToStorageFn = saveToStorageFn
}
func (cw *ChunkedStreamWriter) WriteAt(p []byte, off int64) (n int, err error) {
cw.Lock()
defer cw.Unlock()
logicChunkIndex := LogicChunkIndex(off / cw.ChunkSize)
offsetRemainder := off % cw.ChunkSize
memChunk, found := cw.activeChunks[logicChunkIndex]
if !found {
memChunk = &MemChunk{
buf: mem.Allocate(int(cw.ChunkSize)),
usage: newChunkWrittenIntervalList(),
}
cw.activeChunks[logicChunkIndex] = memChunk
}
n = copy(memChunk.buf[offsetRemainder:], p)
memChunk.usage.MarkWritten(offsetRemainder, offsetRemainder+int64(n))
if memChunk.usage.IsComplete(cw.ChunkSize) {
if cw.saveToStorageFn != nil {
cw.saveOneChunk(memChunk, logicChunkIndex)
delete(cw.activeChunks, logicChunkIndex)
}
}
return
}
func (cw *ChunkedStreamWriter) ReadDataAt(p []byte, off int64) (maxStop int64) {
cw.Lock()
defer cw.Unlock()
logicChunkIndex := LogicChunkIndex(off / cw.ChunkSize)
memChunkBaseOffset := int64(logicChunkIndex) * cw.ChunkSize
memChunk, found := cw.activeChunks[logicChunkIndex]
if !found {
return
}
for t := memChunk.usage.head.next; t != memChunk.usage.tail; t = t.next {
logicStart := max(off, int64(logicChunkIndex)*cw.ChunkSize+t.StartOffset)
logicStop := min(off+int64(len(p)), memChunkBaseOffset+t.stopOffset)
if logicStart < logicStop {
copy(p[logicStart-off:logicStop-off], memChunk.buf[logicStart-memChunkBaseOffset:logicStop-memChunkBaseOffset])
maxStop = max(maxStop, logicStop)
}
}
return
}
func (cw *ChunkedStreamWriter) FlushAll() {
cw.Lock()
defer cw.Unlock()
for logicChunkIndex, memChunk := range cw.activeChunks {
if cw.saveToStorageFn != nil {
cw.saveOneChunk(memChunk, logicChunkIndex)
delete(cw.activeChunks, logicChunkIndex)
}
}
}
func (cw *ChunkedStreamWriter) saveOneChunk(memChunk *MemChunk, logicChunkIndex LogicChunkIndex) {
var referenceCounter = int32(memChunk.usage.size())
for t := memChunk.usage.head.next; t != memChunk.usage.tail; t = t.next {
reader := util.NewBytesReader(memChunk.buf[t.StartOffset:t.stopOffset])
cw.saveToStorageFn(reader, int64(logicChunkIndex)*cw.ChunkSize+t.StartOffset, t.Size(), func() {
atomic.AddInt32(&referenceCounter, -1)
if atomic.LoadInt32(&referenceCounter) == 0 {
mem.Free(memChunk.buf)
}
})
}
}
// Reset releases used resources
func (cw *ChunkedStreamWriter) Reset() {
for t, memChunk := range cw.activeChunks {
mem.Free(memChunk.buf)
delete(cw.activeChunks, t)
}
}

View file

@ -1,33 +0,0 @@
package page_writer
import (
"github.com/stretchr/testify/assert"
"os"
"testing"
)
func TestWriteChunkedStream(t *testing.T) {
x := NewChunkedStreamWriter(20)
defer x.Reset()
y := NewChunkedFileWriter(os.TempDir(), 12)
defer y.Reset()
batchSize := 4
buf := make([]byte, batchSize)
for i := 0; i < 256; i++ {
for x := 0; x < batchSize; x++ {
buf[x] = byte(i)
}
x.WriteAt(buf, int64(i*batchSize))
y.WriteAt(buf, int64((255-i)*batchSize))
}
a := make([]byte, 1)
b := make([]byte, 1)
for i := 0; i < 256*batchSize; i++ {
x.ReadDataAt(a, int64(i))
y.ReadDataAt(b, int64(256*batchSize-1-i))
assert.Equal(t, a[0], b[0], "same read")
}
}

View file

@ -1,222 +0,0 @@
package page_writer
import (
"io"
"github.com/chrislusf/seaweedfs/weed/util"
)
type IntervalNode struct {
Data []byte
Offset int64
Size int64
Next *IntervalNode
}
type IntervalLinkedList struct {
Head *IntervalNode
Tail *IntervalNode
}
type ContinuousIntervals struct {
lists []*IntervalLinkedList
}
func (list *IntervalLinkedList) Offset() int64 {
return list.Head.Offset
}
func (list *IntervalLinkedList) Size() int64 {
return list.Tail.Offset + list.Tail.Size - list.Head.Offset
}
func (list *IntervalLinkedList) addNodeToTail(node *IntervalNode) {
// glog.V(4).Infof("add to tail [%d,%d) + [%d,%d) => [%d,%d)", list.Head.Offset, list.Tail.Offset+list.Tail.Size, node.Offset, node.Offset+node.Size, list.Head.Offset, node.Offset+node.Size)
list.Tail.Next = node
list.Tail = node
}
func (list *IntervalLinkedList) addNodeToHead(node *IntervalNode) {
// glog.V(4).Infof("add to head [%d,%d) + [%d,%d) => [%d,%d)", node.Offset, node.Offset+node.Size, list.Head.Offset, list.Tail.Offset+list.Tail.Size, node.Offset, list.Tail.Offset+list.Tail.Size)
node.Next = list.Head
list.Head = node
}
func (list *IntervalLinkedList) ReadData(buf []byte, start, stop int64) {
t := list.Head
for {
nodeStart, nodeStop := max(start, t.Offset), min(stop, t.Offset+t.Size)
if nodeStart < nodeStop {
// glog.V(0).Infof("copying start=%d stop=%d t=[%d,%d) t.data=%d => bufSize=%d nodeStart=%d, nodeStop=%d", start, stop, t.Offset, t.Offset+t.Size, len(t.Data), len(buf), nodeStart, nodeStop)
copy(buf[nodeStart-start:], t.Data[nodeStart-t.Offset:nodeStop-t.Offset])
}
if t.Next == nil {
break
}
t = t.Next
}
}
func (c *ContinuousIntervals) TotalSize() (total int64) {
for _, list := range c.lists {
total += list.Size()
}
return
}
func subList(list *IntervalLinkedList, start, stop int64) *IntervalLinkedList {
var nodes []*IntervalNode
for t := list.Head; t != nil; t = t.Next {
nodeStart, nodeStop := max(start, t.Offset), min(stop, t.Offset+t.Size)
if nodeStart >= nodeStop {
// skip non overlapping IntervalNode
continue
}
nodes = append(nodes, &IntervalNode{
Data: t.Data[nodeStart-t.Offset : nodeStop-t.Offset],
Offset: nodeStart,
Size: nodeStop - nodeStart,
Next: nil,
})
}
for i := 1; i < len(nodes); i++ {
nodes[i-1].Next = nodes[i]
}
return &IntervalLinkedList{
Head: nodes[0],
Tail: nodes[len(nodes)-1],
}
}
func (c *ContinuousIntervals) AddInterval(data []byte, offset int64) {
interval := &IntervalNode{Data: data, Offset: offset, Size: int64(len(data))}
// append to the tail and return
if len(c.lists) == 1 {
lastSpan := c.lists[0]
if lastSpan.Tail.Offset+lastSpan.Tail.Size == offset {
lastSpan.addNodeToTail(interval)
return
}
}
var newLists []*IntervalLinkedList
for _, list := range c.lists {
// if list is to the left of new interval, add to the new list
if list.Tail.Offset+list.Tail.Size <= interval.Offset {
newLists = append(newLists, list)
}
// if list is to the right of new interval, add to the new list
if interval.Offset+interval.Size <= list.Head.Offset {
newLists = append(newLists, list)
}
// if new interval overwrite the right part of the list
if list.Head.Offset < interval.Offset && interval.Offset < list.Tail.Offset+list.Tail.Size {
// create a new list of the left part of existing list
newLists = append(newLists, subList(list, list.Offset(), interval.Offset))
}
// if new interval overwrite the left part of the list
if list.Head.Offset < interval.Offset+interval.Size && interval.Offset+interval.Size < list.Tail.Offset+list.Tail.Size {
// create a new list of the right part of existing list
newLists = append(newLists, subList(list, interval.Offset+interval.Size, list.Tail.Offset+list.Tail.Size))
}
// skip anything that is fully overwritten by the new interval
}
c.lists = newLists
// add the new interval to the lists, connecting neighbor lists
var prevList, nextList *IntervalLinkedList
for _, list := range c.lists {
if list.Head.Offset == interval.Offset+interval.Size {
nextList = list
break
}
}
for _, list := range c.lists {
if list.Head.Offset+list.Size() == offset {
list.addNodeToTail(interval)
prevList = list
break
}
}
if prevList != nil && nextList != nil {
// glog.V(4).Infof("connecting [%d,%d) + [%d,%d) => [%d,%d)", prevList.Head.Offset, prevList.Tail.Offset+prevList.Tail.Size, nextList.Head.Offset, nextList.Tail.Offset+nextList.Tail.Size, prevList.Head.Offset, nextList.Tail.Offset+nextList.Tail.Size)
prevList.Tail.Next = nextList.Head
prevList.Tail = nextList.Tail
c.removeList(nextList)
} else if nextList != nil {
// add to head was not done when checking
nextList.addNodeToHead(interval)
}
if prevList == nil && nextList == nil {
c.lists = append(c.lists, &IntervalLinkedList{
Head: interval,
Tail: interval,
})
}
return
}
func (c *ContinuousIntervals) RemoveLargestIntervalLinkedList() *IntervalLinkedList {
var maxSize int64
maxIndex := -1
for k, list := range c.lists {
if maxSize <= list.Size() {
maxSize = list.Size()
maxIndex = k
}
}
if maxSize <= 0 {
return nil
}
t := c.lists[maxIndex]
c.lists = append(c.lists[0:maxIndex], c.lists[maxIndex+1:]...)
return t
}
func (c *ContinuousIntervals) removeList(target *IntervalLinkedList) {
index := -1
for k, list := range c.lists {
if list.Offset() == target.Offset() {
index = k
}
}
if index < 0 {
return
}
c.lists = append(c.lists[0:index], c.lists[index+1:]...)
}
func (c *ContinuousIntervals) ReadDataAt(data []byte, startOffset int64) (maxStop int64) {
for _, list := range c.lists {
start := max(startOffset, list.Offset())
stop := min(startOffset+int64(len(data)), list.Offset()+list.Size())
if start < stop {
list.ReadData(data[start-startOffset:], start, stop)
maxStop = max(maxStop, stop)
}
}
return
}
func (l *IntervalLinkedList) ToReader() io.Reader {
var readers []io.Reader
t := l.Head
readers = append(readers, util.NewBytesReader(t.Data))
for t.Next != nil {
t = t.Next
readers = append(readers, util.NewBytesReader(t.Data))
}
if len(readers) == 1 {
return readers[0]
}
return io.MultiReader(readers...)
}

View file

@ -1,113 +0,0 @@
package page_writer
import (
"bytes"
"math/rand"
"testing"
)
func TestContinuousIntervals_AddIntervalAppend(t *testing.T) {
c := &ContinuousIntervals{}
// 25, 25, 25
c.AddInterval(getBytes(25, 3), 0)
// _, _, 23, 23, 23, 23
c.AddInterval(getBytes(23, 4), 2)
expectedData(t, c, 0, 25, 25, 23, 23, 23, 23)
}
func TestContinuousIntervals_AddIntervalInnerOverwrite(t *testing.T) {
c := &ContinuousIntervals{}
// 25, 25, 25, 25, 25
c.AddInterval(getBytes(25, 5), 0)
// _, _, 23, 23
c.AddInterval(getBytes(23, 2), 2)
expectedData(t, c, 0, 25, 25, 23, 23, 25)
}
func TestContinuousIntervals_AddIntervalFullOverwrite(t *testing.T) {
c := &ContinuousIntervals{}
// 1,
c.AddInterval(getBytes(1, 1), 0)
// _, 2,
c.AddInterval(getBytes(2, 1), 1)
// _, _, 3, 3, 3
c.AddInterval(getBytes(3, 3), 2)
// _, _, _, 4, 4, 4
c.AddInterval(getBytes(4, 3), 3)
expectedData(t, c, 0, 1, 2, 3, 4, 4, 4)
}
func TestContinuousIntervals_RealCase1(t *testing.T) {
c := &ContinuousIntervals{}
// 25,
c.AddInterval(getBytes(25, 1), 0)
// _, _, _, _, 23, 23
c.AddInterval(getBytes(23, 2), 4)
// _, _, _, 24, 24, 24, 24
c.AddInterval(getBytes(24, 4), 3)
// _, 22, 22
c.AddInterval(getBytes(22, 2), 1)
expectedData(t, c, 0, 25, 22, 22, 24, 24, 24, 24)
}
func TestRandomWrites(t *testing.T) {
c := &ContinuousIntervals{}
data := make([]byte, 1024)
for i := 0; i < 1024; i++ {
start, stop := rand.Intn(len(data)), rand.Intn(len(data))
if start > stop {
start, stop = stop, start
}
rand.Read(data[start : stop+1])
c.AddInterval(data[start:stop+1], int64(start))
expectedData(t, c, 0, data...)
}
}
func expectedData(t *testing.T, c *ContinuousIntervals, offset int, data ...byte) {
start, stop := int64(offset), int64(offset+len(data))
for _, list := range c.lists {
nodeStart, nodeStop := max(start, list.Head.Offset), min(stop, list.Head.Offset+list.Size())
if nodeStart < nodeStop {
buf := make([]byte, nodeStop-nodeStart)
list.ReadData(buf, nodeStart, nodeStop)
if bytes.Compare(buf, data[nodeStart-start:nodeStop-start]) != 0 {
t.Errorf("expected %v actual %v", data[nodeStart-start:nodeStop-start], buf)
}
}
}
}
func getBytes(content byte, length int) []byte {
data := make([]byte, length)
for i := 0; i < length; i++ {
data[i] = content
}
return data
}

View file

@ -6,6 +6,8 @@ type DirtyPages interface {
ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64)
GetStorageOptions() (collection, replication string)
Destroy()
LockForRead(startOffset, stopOffset int64)
UnlockForRead(startOffset, stopOffset int64)
}
func max(x, y int64) int64 {

View file

@ -0,0 +1,256 @@
package page_writer
import (
"fmt"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/util"
"github.com/chrislusf/seaweedfs/weed/util/mem"
"sync"
"sync/atomic"
"time"
)
type LogicChunkIndex int
type UploadPipeline struct {
filepath util.FullPath
ChunkSize int64
writers *util.LimitedConcurrentExecutor
writableChunks map[LogicChunkIndex]*MemChunk
writableChunksLock sync.Mutex
sealedChunks map[LogicChunkIndex]*SealedChunk
sealedChunksLock sync.Mutex
activeWriterCond *sync.Cond
activeWriterCount int32
activeReadChunks map[LogicChunkIndex]int
activeReadChunksLock sync.Mutex
saveToStorageFn SaveToStorageFunc
}
type SealedChunk struct {
chunk *MemChunk
referenceCounter int // track uploading or reading processes
}
func (sc *SealedChunk) FreeReference(messageOnFree string) {
sc.referenceCounter--
if sc.referenceCounter == 0 {
glog.V(4).Infof("Free sealed chunk: %s", messageOnFree)
mem.Free(sc.chunk.buf)
}
}
func NewUploadPipeline(filepath util.FullPath, writers *util.LimitedConcurrentExecutor, chunkSize int64, saveToStorageFn SaveToStorageFunc) *UploadPipeline {
return &UploadPipeline{
ChunkSize: chunkSize,
writableChunks: make(map[LogicChunkIndex]*MemChunk),
sealedChunks: make(map[LogicChunkIndex]*SealedChunk),
writers: writers,
activeWriterCond: sync.NewCond(&sync.Mutex{}),
saveToStorageFn: saveToStorageFn,
filepath: filepath,
activeReadChunks: make(map[LogicChunkIndex]int),
}
}
func (cw *UploadPipeline) SaveDataAt(p []byte, off int64) (n int) {
cw.writableChunksLock.Lock()
defer cw.writableChunksLock.Unlock()
logicChunkIndex := LogicChunkIndex(off / cw.ChunkSize)
offsetRemainder := off % cw.ChunkSize
memChunk, found := cw.writableChunks[logicChunkIndex]
if !found {
memChunk = &MemChunk{
buf: mem.Allocate(int(cw.ChunkSize)),
usage: newChunkWrittenIntervalList(),
}
cw.writableChunks[logicChunkIndex] = memChunk
}
n = copy(memChunk.buf[offsetRemainder:], p)
memChunk.usage.MarkWritten(offsetRemainder, offsetRemainder+int64(n))
cw.maybeMoveToSealed(memChunk, logicChunkIndex)
return
}
func (cw *UploadPipeline) MaybeReadDataAt(p []byte, off int64) (maxStop int64) {
logicChunkIndex := LogicChunkIndex(off / cw.ChunkSize)
// read from sealed chunks first
cw.sealedChunksLock.Lock()
sealedChunk, found := cw.sealedChunks[logicChunkIndex]
if found {
sealedChunk.referenceCounter++
}
cw.sealedChunksLock.Unlock()
if found {
maxStop = readMemChunk(sealedChunk.chunk, p, off, logicChunkIndex, cw.ChunkSize)
glog.V(4).Infof("%s read sealed memchunk [%d,%d)", cw.filepath, off, maxStop)
sealedChunk.FreeReference(fmt.Sprintf("%s finish reading chunk %d", cw.filepath, logicChunkIndex))
}
// read from writable chunks last
cw.writableChunksLock.Lock()
defer cw.writableChunksLock.Unlock()
writableChunk, found := cw.writableChunks[logicChunkIndex]
if !found {
return
}
writableMaxStop := readMemChunk(writableChunk, p, off, logicChunkIndex, cw.ChunkSize)
glog.V(4).Infof("%s read writable memchunk [%d,%d)", cw.filepath, off, writableMaxStop)
maxStop = max(maxStop, writableMaxStop)
return
}
func (cw *UploadPipeline) FlushAll() {
cw.writableChunksLock.Lock()
defer cw.writableChunksLock.Unlock()
for logicChunkIndex, memChunk := range cw.writableChunks {
cw.moveToSealed(memChunk, logicChunkIndex)
}
cw.waitForCurrentWritersToComplete()
}
func (cw *UploadPipeline) LockForRead(startOffset, stopOffset int64) {
startLogicChunkIndex := LogicChunkIndex(startOffset / cw.ChunkSize)
stopLogicChunkIndex := LogicChunkIndex(stopOffset / cw.ChunkSize)
if stopOffset%cw.ChunkSize > 0 {
stopLogicChunkIndex += 1
}
cw.activeReadChunksLock.Lock()
defer cw.activeReadChunksLock.Unlock()
for i := startLogicChunkIndex; i < stopLogicChunkIndex; i++ {
if count, found := cw.activeReadChunks[i]; found {
cw.activeReadChunks[i] = count + 1
} else {
cw.activeReadChunks[i] = 1
}
}
}
func (cw *UploadPipeline) UnlockForRead(startOffset, stopOffset int64) {
startLogicChunkIndex := LogicChunkIndex(startOffset / cw.ChunkSize)
stopLogicChunkIndex := LogicChunkIndex(stopOffset / cw.ChunkSize)
if stopOffset%cw.ChunkSize > 0 {
stopLogicChunkIndex += 1
}
cw.activeReadChunksLock.Lock()
defer cw.activeReadChunksLock.Unlock()
for i := startLogicChunkIndex; i < stopLogicChunkIndex; i++ {
if count, found := cw.activeReadChunks[i]; found {
if count == 1 {
delete(cw.activeReadChunks, i)
} else {
cw.activeReadChunks[i] = count - 1
}
}
}
}
func (cw *UploadPipeline) IsLocked(logicChunkIndex LogicChunkIndex) bool {
cw.activeReadChunksLock.Lock()
defer cw.activeReadChunksLock.Unlock()
if count, found := cw.activeReadChunks[logicChunkIndex]; found {
return count > 0
}
return false
}
func (cw *UploadPipeline) waitForCurrentWritersToComplete() {
cw.activeWriterCond.L.Lock()
t := int32(100)
for {
t = atomic.LoadInt32(&cw.activeWriterCount)
if t <= 0 {
break
}
glog.V(4).Infof("activeWriterCond is %d", t)
cw.activeWriterCond.Wait()
}
cw.activeWriterCond.L.Unlock()
}
func (cw *UploadPipeline) maybeMoveToSealed(memChunk *MemChunk, logicChunkIndex LogicChunkIndex) {
if memChunk.usage.IsComplete(cw.ChunkSize) {
cw.moveToSealed(memChunk, logicChunkIndex)
}
}
func (cw *UploadPipeline) moveToSealed(memChunk *MemChunk, logicChunkIndex LogicChunkIndex) {
atomic.AddInt32(&cw.activeWriterCount, 1)
glog.V(4).Infof("%s activeWriterCount %d ++> %d", cw.filepath, cw.activeWriterCount-1, cw.activeWriterCount)
cw.sealedChunksLock.Lock()
if oldMemChunk, found := cw.sealedChunks[logicChunkIndex]; found {
oldMemChunk.FreeReference(fmt.Sprintf("%s replace chunk %d", cw.filepath, logicChunkIndex))
}
sealedChunk := &SealedChunk{
chunk: memChunk,
referenceCounter: 1, // default 1 is for uploading process
}
cw.sealedChunks[logicChunkIndex] = sealedChunk
delete(cw.writableChunks, logicChunkIndex)
cw.sealedChunksLock.Unlock()
cw.writers.Execute(func() {
// first add to the file chunks
cw.saveOneChunk(sealedChunk.chunk, logicChunkIndex)
// notify waiting process
atomic.AddInt32(&cw.activeWriterCount, -1)
glog.V(4).Infof("%s activeWriterCount %d --> %d", cw.filepath, cw.activeWriterCount+1, cw.activeWriterCount)
// Lock and Unlock are not required,
// but it may signal multiple times during one wakeup,
// and the waiting goroutine may miss some of them!
cw.activeWriterCond.L.Lock()
cw.activeWriterCond.Broadcast()
cw.activeWriterCond.L.Unlock()
// wait for readers
for cw.IsLocked(logicChunkIndex) {
time.Sleep(59 * time.Millisecond)
}
// then remove from sealed chunks
cw.sealedChunksLock.Lock()
defer cw.sealedChunksLock.Unlock()
delete(cw.sealedChunks, logicChunkIndex)
sealedChunk.FreeReference(fmt.Sprintf("%s finished uploading chunk %d", cw.filepath, logicChunkIndex))
})
}
func (cw *UploadPipeline) saveOneChunk(memChunk *MemChunk, logicChunkIndex LogicChunkIndex) {
if cw.saveToStorageFn == nil {
return
}
for t := memChunk.usage.head.next; t != memChunk.usage.tail; t = t.next {
reader := util.NewBytesReader(memChunk.buf[t.StartOffset:t.stopOffset])
cw.saveToStorageFn(reader, int64(logicChunkIndex)*cw.ChunkSize+t.StartOffset, t.Size(), func() {
})
}
}
func readMemChunk(memChunk *MemChunk, p []byte, off int64, logicChunkIndex LogicChunkIndex, chunkSize int64) (maxStop int64) {
memChunkBaseOffset := int64(logicChunkIndex) * chunkSize
for t := memChunk.usage.head.next; t != memChunk.usage.tail; t = t.next {
logicStart := max(off, int64(logicChunkIndex)*chunkSize+t.StartOffset)
logicStop := min(off+int64(len(p)), memChunkBaseOffset+t.stopOffset)
if logicStart < logicStop {
copy(p[logicStart-off:logicStop-off], memChunk.buf[logicStart-memChunkBaseOffset:logicStop-memChunkBaseOffset])
maxStop = max(maxStop, logicStop)
}
}
return
}
func (p2 *UploadPipeline) Shutdown() {
}

View file

@ -0,0 +1,47 @@
package page_writer
import (
"github.com/chrislusf/seaweedfs/weed/util"
"testing"
)
func TestUploadPipeline(t *testing.T) {
uploadPipeline := NewUploadPipeline("", nil, 2*1024*1024, nil)
writeRange(uploadPipeline, 0, 131072)
writeRange(uploadPipeline, 131072, 262144)
writeRange(uploadPipeline, 262144, 1025536)
confirmRange(t, uploadPipeline, 0, 1025536)
writeRange(uploadPipeline, 1025536, 1296896)
confirmRange(t, uploadPipeline, 1025536, 1296896)
writeRange(uploadPipeline, 1296896, 2162688)
confirmRange(t, uploadPipeline, 1296896, 2162688)
confirmRange(t, uploadPipeline, 1296896, 2162688)
}
// startOff and stopOff must be divided by 4
func writeRange(uploadPipeline *UploadPipeline, startOff, stopOff int64) {
p := make([]byte, 4)
for i := startOff / 4; i < stopOff/4; i += 4 {
util.Uint32toBytes(p, uint32(i))
uploadPipeline.SaveDataAt(p, i)
}
}
func confirmRange(t *testing.T, uploadPipeline *UploadPipeline, startOff, stopOff int64) {
p := make([]byte, 4)
for i := startOff; i < stopOff/4; i += 4 {
uploadPipeline.MaybeReadDataAt(p, i)
x := util.BytesToUint32(p)
if x != uint32(i) {
t.Errorf("expecting %d found %d at offset [%d,%d)", i, x, i, i+4)
}
}
}