Browse Source

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
pull/4109/head
Chris Lu 2 years ago
committed by GitHub
parent
commit
d4566d4aaa
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
  1. 8
      weed/command/filer_copy.go
  2. 148
      weed/filer/filechunk_group.go
  3. 36
      weed/filer/filechunk_group_test.go
  4. 4
      weed/filer/filechunk_manifest.go
  5. 119
      weed/filer/filechunk_section.go
  6. 248
      weed/filer/filechunks.go
  7. 106
      weed/filer/filechunks_read.go
  8. 86
      weed/filer/filechunks_read_test.go
  9. 214
      weed/filer/filechunks_test.go
  10. 2
      weed/filer/filer_notify_append.go
  11. 259
      weed/filer/interval_list.go
  12. 327
      weed/filer/interval_list_test.go
  13. 65
      weed/filer/reader_at.go
  14. 142
      weed/filer/reader_at_test.go
  15. 7
      weed/filer/reader_cache.go
  16. 90
      weed/filer/stream.go
  17. 16
      weed/mount/dirty_pages_chunked.go
  18. 109
      weed/mount/filehandle.go
  19. 4
      weed/mount/filehandle_map.go
  20. 43
      weed/mount/filehandle_read.go
  21. 12
      weed/mount/page_writer.go
  22. 39
      weed/mount/page_writer/activity_score.go
  23. 83
      weed/mount/page_writer/chunk_interval_list.go
  24. 72
      weed/mount/page_writer/chunk_interval_list_test.go
  25. 4
      weed/mount/page_writer/dirty_pages.go
  26. 8
      weed/mount/page_writer/page_chunk.go
  27. 31
      weed/mount/page_writer/page_chunk_mem.go
  28. 125
      weed/mount/page_writer/page_chunk_swapfile.go
  29. 40
      weed/mount/page_writer/upload_pipeline.go
  30. 4
      weed/mount/page_writer/upload_pipeline_test.go
  31. 20
      weed/mount/weedfs_attr.go
  32. 12
      weed/mount/weedfs_file_copy_range.go
  33. 43
      weed/mount/weedfs_file_lseek.go
  34. 28
      weed/mount/weedfs_file_read.go
  35. 15
      weed/mount/weedfs_file_sync.go
  36. 15
      weed/mount/weedfs_file_write.go
  37. 4
      weed/mount/weedfs_write.go
  38. 4
      weed/operation/upload_content.go
  39. 7
      weed/replication/repl_util/replication_util.go
  40. 4
      weed/server/filer_server_handlers_write_autochunk.go
  41. 2
      weed/server/filer_server_handlers_write_cipher.go
  42. 2
      weed/server/filer_server_handlers_write_upload.go
  43. 30
      weed/server/webdav_server.go
  44. 2
      weed/shell/command_fs_verify.go
  45. 2
      weed/shell/command_volume_fsck.go

8
weed/command/filer_copy.go

@ -365,7 +365,7 @@ func (worker *FileCopyWorker) uploadFileAsOne(task FileCopyTask, f *os.File) err
if flushErr != nil { if flushErr != nil {
return flushErr return flushErr
} }
chunks = append(chunks, uploadResult.ToPbFileChunk(finalFileId, 0))
chunks = append(chunks, uploadResult.ToPbFileChunk(finalFileId, 0, time.Now().UnixNano()))
} }
if err := pb.WithGrpcFilerClient(false, worker.filerAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error { if err := pb.WithGrpcFilerClient(false, worker.filerAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
@ -450,7 +450,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(task FileCopyTask, f *os.File,
uploadError = fmt.Errorf("upload %v result: %v\n", fileName, uploadResult.Error) uploadError = fmt.Errorf("upload %v result: %v\n", fileName, uploadResult.Error)
return return
} }
chunksChan <- uploadResult.ToPbFileChunk(fileId, i*chunkSize)
chunksChan <- uploadResult.ToPbFileChunk(fileId, i*chunkSize, time.Now().UnixNano())
fmt.Printf("uploaded %s-%d [%d,%d)\n", fileName, i+1, i*chunkSize, i*chunkSize+int64(uploadResult.Size)) fmt.Printf("uploaded %s-%d [%d,%d)\n", fileName, i+1, i*chunkSize, i*chunkSize+int64(uploadResult.Size))
}(i) }(i)
@ -530,7 +530,7 @@ func detectMimeType(f *os.File) string {
return mimeType return mimeType
} }
func (worker *FileCopyWorker) saveDataAsChunk(reader io.Reader, name string, offset int64) (chunk *filer_pb.FileChunk, err error) {
func (worker *FileCopyWorker) saveDataAsChunk(reader io.Reader, name string, offset int64, tsNs int64) (chunk *filer_pb.FileChunk, err error) {
finalFileId, uploadResult, flushErr, _ := operation.UploadWithRetry( finalFileId, uploadResult, flushErr, _ := operation.UploadWithRetry(
worker, worker,
@ -561,7 +561,7 @@ func (worker *FileCopyWorker) saveDataAsChunk(reader io.Reader, name string, off
if uploadResult.Error != "" { if uploadResult.Error != "" {
return nil, fmt.Errorf("upload result: %v", uploadResult.Error) return nil, fmt.Errorf("upload result: %v", uploadResult.Error)
} }
return uploadResult.ToPbFileChunk(finalFileId, offset), nil
return uploadResult.ToPbFileChunk(finalFileId, offset, tsNs), nil
} }
var _ = filer_pb.FilerClient(&FileCopyWorker{}) var _ = filer_pb.FilerClient(&FileCopyWorker{})

148
weed/filer/filechunk_group.go

@ -0,0 +1,148 @@
package filer
import (
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util/chunk_cache"
"github.com/seaweedfs/seaweedfs/weed/wdclient"
"sync"
)
type ChunkGroup struct {
lookupFn wdclient.LookupFileIdFunctionType
chunkCache chunk_cache.ChunkCache
manifestChunks []*filer_pb.FileChunk
sections map[SectionIndex]*FileChunkSection
sectionsLock sync.RWMutex
}
func NewChunkGroup(lookupFn wdclient.LookupFileIdFunctionType, chunkCache chunk_cache.ChunkCache, chunks []*filer_pb.FileChunk) (*ChunkGroup, error) {
group := &ChunkGroup{
lookupFn: lookupFn,
chunkCache: chunkCache,
sections: make(map[SectionIndex]*FileChunkSection),
}
err := group.SetChunks(chunks)
return group, err
}
func (group *ChunkGroup) AddChunk(chunk *filer_pb.FileChunk) error {
group.sectionsLock.Lock()
defer group.sectionsLock.Unlock()
sectionIndexStart, sectionIndexStop := SectionIndex(chunk.Offset/SectionSize), SectionIndex((chunk.Offset+int64(chunk.Size))/SectionSize)
for si := sectionIndexStart; si < sectionIndexStop+1; si++ {
section, found := group.sections[si]
if !found {
section = NewFileChunkSection(si)
group.sections[si] = section
}
section.addChunk(chunk)
}
return nil
}
func (group *ChunkGroup) ReadDataAt(fileSize int64, buff []byte, offset int64) (n int, tsNs int64, err error) {
group.sectionsLock.RLock()
defer group.sectionsLock.RUnlock()
sectionIndexStart, sectionIndexStop := SectionIndex(offset/SectionSize), SectionIndex((offset+int64(len(buff)))/SectionSize)
for si := sectionIndexStart; si < sectionIndexStop+1; si++ {
section, found := group.sections[si]
rangeStart, rangeStop := max(offset, int64(si*SectionSize)), min(offset+int64(len(buff)), int64((si+1)*SectionSize))
if !found {
for i := rangeStart; i < rangeStop; i++ {
buff[i-offset] = 0
}
continue
}
xn, xTsNs, xErr := section.readDataAt(group, fileSize, buff[rangeStart-offset:rangeStop-offset], rangeStart)
if xErr != nil {
err = xErr
}
n += xn
tsNs = max(tsNs, xTsNs)
}
return
}
func (group *ChunkGroup) SetChunks(chunks []*filer_pb.FileChunk) error {
var dataChunks []*filer_pb.FileChunk
for _, chunk := range chunks {
if !chunk.IsChunkManifest {
dataChunks = append(dataChunks, chunk)
continue
}
resolvedChunks, err := ResolveOneChunkManifest(group.lookupFn, chunk)
if err != nil {
return err
}
group.manifestChunks = append(group.manifestChunks, chunk)
dataChunks = append(dataChunks, resolvedChunks...)
}
for _, chunk := range dataChunks {
sectionIndexStart, sectionIndexStop := SectionIndex(chunk.Offset/SectionSize), SectionIndex((chunk.Offset+int64(chunk.Size))/SectionSize)
for si := sectionIndexStart; si < sectionIndexStop+1; si++ {
section, found := group.sections[si]
if !found {
section = NewFileChunkSection(si)
group.sections[si] = section
}
section.chunks = append(section.chunks, chunk)
}
}
return nil
}
const (
// see weedfs_file_lseek.go
SEEK_DATA uint32 = 3 // seek to next data after the offset
// SEEK_HOLE uint32 = 4 // seek to next hole after the offset
)
// FIXME: needa tests
func (group *ChunkGroup) SearchChunks(offset, fileSize int64, whence uint32) (found bool, out int64) {
group.sectionsLock.RLock()
defer group.sectionsLock.RUnlock()
return group.doSearchChunks(offset, fileSize, whence)
}
func (group *ChunkGroup) doSearchChunks(offset, fileSize int64, whence uint32) (found bool, out int64) {
sectionIndex, maxSectionIndex := SectionIndex(offset/SectionSize), SectionIndex(fileSize/SectionSize)
if whence == SEEK_DATA {
for si := sectionIndex; si < maxSectionIndex+1; si++ {
section, foundSection := group.sections[si]
if !foundSection {
continue
}
sectionStart := section.DataStartOffset(group, offset, fileSize)
if sectionStart == -1 {
continue
}
return true, sectionStart
}
return false, 0
} else {
// whence == SEEK_HOLE
for si := sectionIndex; si < maxSectionIndex; si++ {
section, foundSection := group.sections[si]
if !foundSection {
return true, offset
}
holeStart := section.NextStopOffset(group, offset, fileSize)
if holeStart%SectionSize == 0 {
continue
}
return true, holeStart
}
return true, fileSize
}
}

36
weed/filer/filechunk_group_test.go

@ -0,0 +1,36 @@
package filer
import (
"github.com/stretchr/testify/assert"
"testing"
)
func TestChunkGroup_doSearchChunks(t *testing.T) {
type fields struct {
sections map[SectionIndex]*FileChunkSection
}
type args struct {
offset int64
fileSize int64
whence uint32
}
tests := []struct {
name string
fields fields
args args
wantFound bool
wantOut int64
}{
// TODO: Add test cases.
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
group := &ChunkGroup{
sections: tt.fields.sections,
}
gotFound, gotOut := group.doSearchChunks(tt.args.offset, tt.args.fileSize, tt.args.whence)
assert.Equalf(t, tt.wantFound, gotFound, "doSearchChunks(%v, %v, %v)", tt.args.offset, tt.args.fileSize, tt.args.whence)
assert.Equalf(t, tt.wantOut, gotOut, "doSearchChunks(%v, %v, %v)", tt.args.offset, tt.args.fileSize, tt.args.whence)
})
}
}

4
weed/filer/filechunk_manifest.go

@ -264,7 +264,7 @@ func mergeIntoManifest(saveFunc SaveDataAsChunkFunctionType, dataChunks []*filer
} }
} }
manifestChunk, err = saveFunc(bytes.NewReader(data), "", 0)
manifestChunk, err = saveFunc(bytes.NewReader(data), "", 0, 0)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -275,4 +275,4 @@ func mergeIntoManifest(saveFunc SaveDataAsChunkFunctionType, dataChunks []*filer
return return
} }
type SaveDataAsChunkFunctionType func(reader io.Reader, name string, offset int64) (chunk *filer_pb.FileChunk, err error)
type SaveDataAsChunkFunctionType func(reader io.Reader, name string, offset int64, tsNs int64) (chunk *filer_pb.FileChunk, err error)

119
weed/filer/filechunk_section.go

@ -0,0 +1,119 @@
package filer
import (
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"sync"
)
const SectionSize = 2 * 1024 * 1024 * 128 // 256MiB
type SectionIndex int64
type FileChunkSection struct {
sectionIndex SectionIndex
chunks []*filer_pb.FileChunk
visibleIntervals *IntervalList[*VisibleInterval]
chunkViews *IntervalList[*ChunkView]
reader *ChunkReadAt
lock sync.Mutex
}
func NewFileChunkSection(si SectionIndex) *FileChunkSection {
return &FileChunkSection{
sectionIndex: si,
}
}
func (section *FileChunkSection) addChunk(chunk *filer_pb.FileChunk) error {
section.lock.Lock()
defer section.lock.Unlock()
start, stop := max(int64(section.sectionIndex)*SectionSize, chunk.Offset), min(((int64(section.sectionIndex)+1)*SectionSize), chunk.Offset+int64(chunk.Size))
section.chunks = append(section.chunks, chunk)
if section.visibleIntervals != nil {
MergeIntoVisibles(section.visibleIntervals, start, stop, chunk)
}
if section.visibleIntervals != nil {
section.chunks, _ = SeparateGarbageChunks(section.visibleIntervals, section.chunks)
}
if section.chunkViews != nil {
MergeIntoChunkViews(section.chunkViews, start, stop, chunk)
}
return nil
}
func (section *FileChunkSection) setupForRead(group *ChunkGroup, fileSize int64) {
if section.visibleIntervals == nil {
section.visibleIntervals = readResolvedChunks(section.chunks, int64(section.sectionIndex)*SectionSize, (int64(section.sectionIndex)+1)*SectionSize)
section.chunks, _ = SeparateGarbageChunks(section.visibleIntervals, section.chunks)
if section.reader != nil {
_ = section.reader.Close()
section.reader = nil
}
}
if section.chunkViews == nil {
section.chunkViews = ViewFromVisibleIntervals(section.visibleIntervals, int64(section.sectionIndex)*SectionSize, (int64(section.sectionIndex)+1)*SectionSize)
}
if section.reader == nil {
section.reader = NewChunkReaderAtFromClient(group.lookupFn, section.chunkViews, group.chunkCache, min(int64(section.sectionIndex+1)*SectionSize, fileSize))
}
section.reader.fileSize = fileSize
}
func (section *FileChunkSection) readDataAt(group *ChunkGroup, fileSize int64, buff []byte, offset int64) (n int, tsNs int64, err error) {
section.lock.Lock()
defer section.lock.Unlock()
section.setupForRead(group, fileSize)
return section.reader.ReadAtWithTime(buff, offset)
}
func (section *FileChunkSection) DataStartOffset(group *ChunkGroup, offset int64, fileSize int64) int64 {
section.lock.Lock()
defer section.lock.Unlock()
section.setupForRead(group, fileSize)
for x := section.visibleIntervals.Front(); x != nil; x = x.Next {
visible := x.Value
if visible.stop <= offset {
continue
}
if offset < visible.start {
return offset
}
return offset
}
return -1
}
func (section *FileChunkSection) NextStopOffset(group *ChunkGroup, offset int64, fileSize int64) int64 {
section.lock.Lock()
defer section.lock.Unlock()
section.setupForRead(group, fileSize)
isAfterOffset := false
for x := section.visibleIntervals.Front(); x != nil; x = x.Next {
visible := x.Value
if !isAfterOffset {
if visible.stop <= offset {
continue
}
isAfterOffset = true
}
if offset < visible.start {
return offset
}
// now visible.start <= offset
if offset < visible.stop {
offset = visible.stop
}
}
return offset
}

248
weed/filer/filechunks.go

@ -4,7 +4,6 @@ import (
"bytes" "bytes"
"fmt" "fmt"
"github.com/seaweedfs/seaweedfs/weed/wdclient" "github.com/seaweedfs/seaweedfs/weed/wdclient"
"golang.org/x/exp/slices"
"math" "math"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
@ -42,7 +41,7 @@ func ETag(entry *filer_pb.Entry) (etag string) {
} }
func ETagEntry(entry *Entry) (etag string) { func ETagEntry(entry *Entry) (etag string) {
if entry.IsInRemoteOnly() {
if entry.IsInRemoteOnly() {
return entry.Remote.RemoteETag return entry.Remote.RemoteETag
} }
if entry.Attr.Md5 == nil { if entry.Attr.Md5 == nil {
@ -66,8 +65,15 @@ func CompactFileChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks
visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks, 0, math.MaxInt64) visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks, 0, math.MaxInt64)
compacted, garbage = SeparateGarbageChunks(visibles, chunks)
return
}
func SeparateGarbageChunks(visibles *IntervalList[*VisibleInterval], chunks []*filer_pb.FileChunk) (compacted []*filer_pb.FileChunk, garbage []*filer_pb.FileChunk) {
fileIds := make(map[string]bool) fileIds := make(map[string]bool)
for _, interval := range visibles {
for x := visibles.Front(); x != nil; x = x.Next {
interval := x.Value
fileIds[interval.fileId] = true fileIds[interval.fileId] = true
} }
for _, chunk := range chunks { for _, chunk := range chunks {
@ -77,8 +83,7 @@ func CompactFileChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks
garbage = append(garbage, chunk) garbage = append(garbage, chunk)
} }
} }
return
return compacted, garbage
} }
func MinusChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, as, bs []*filer_pb.FileChunk) (delta []*filer_pb.FileChunk, err error) { func MinusChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, as, bs []*filer_pb.FileChunk) (delta []*filer_pb.FileChunk, err error) {
@ -131,20 +136,39 @@ func DoMinusChunksBySourceFileId(as, bs []*filer_pb.FileChunk) (delta []*filer_p
} }
type ChunkView struct { type ChunkView struct {
FileId string
Offset int64
Size uint64
LogicOffset int64 // actual offset in the file, for the data specified via [offset, offset+size) in current chunk
ChunkSize uint64
CipherKey []byte
IsGzipped bool
FileId string
OffsetInChunk int64 // offset within the chunk
ViewSize uint64
ViewOffset int64 // actual offset in the file, for the data specified via [offset, offset+size) in current chunk
ChunkSize uint64
CipherKey []byte
IsGzipped bool
ModifiedTsNs int64
}
func (cv *ChunkView) SetStartStop(start, stop int64) {
cv.OffsetInChunk += start - cv.ViewOffset
cv.ViewOffset = start
cv.ViewSize = uint64(stop - start)
}
func (cv *ChunkView) Clone() IntervalValue {
return &ChunkView{
FileId: cv.FileId,
OffsetInChunk: cv.OffsetInChunk,
ViewSize: cv.ViewSize,
ViewOffset: cv.ViewOffset,
ChunkSize: cv.ChunkSize,
CipherKey: cv.CipherKey,
IsGzipped: cv.IsGzipped,
ModifiedTsNs: cv.ModifiedTsNs,
}
} }
func (cv *ChunkView) IsFullChunk() bool { func (cv *ChunkView) IsFullChunk() bool {
return cv.Size == cv.ChunkSize
return cv.ViewSize == cv.ChunkSize
} }
func ViewFromChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk, offset int64, size int64) (views []*ChunkView) {
func ViewFromChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk, offset int64, size int64) (chunkViews *IntervalList[*ChunkView]) {
visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks, offset, offset+size) visibles, _ := NonOverlappingVisibleIntervals(lookupFileIdFn, chunks, offset, offset+size)
@ -152,7 +176,7 @@ func ViewFromChunks(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*
} }
func ViewFromVisibleIntervals(visibles []VisibleInterval, offset int64, size int64) (views []*ChunkView) {
func ViewFromVisibleIntervals(visibles *IntervalList[*VisibleInterval], offset int64, size int64) (chunkViews *IntervalList[*ChunkView]) {
stop := offset + size stop := offset + size
if size == math.MaxInt64 { if size == math.MaxInt64 {
@ -162,164 +186,112 @@ func ViewFromVisibleIntervals(visibles []VisibleInterval, offset int64, size int
stop = math.MaxInt64 stop = math.MaxInt64
} }
for _, chunk := range visibles {
chunkViews = NewIntervalList[*ChunkView]()
for x := visibles.Front(); x != nil; x = x.Next {
chunk := x.Value
chunkStart, chunkStop := max(offset, chunk.start), min(stop, chunk.stop) chunkStart, chunkStop := max(offset, chunk.start), min(stop, chunk.stop)
if chunkStart < chunkStop { if chunkStart < chunkStop {
views = append(views, &ChunkView{
FileId: chunk.fileId,
Offset: chunkStart - chunk.start + chunk.chunkOffset,
Size: uint64(chunkStop - chunkStart),
LogicOffset: chunkStart,
ChunkSize: chunk.chunkSize,
CipherKey: chunk.cipherKey,
IsGzipped: chunk.isGzipped,
chunkView := &ChunkView{
FileId: chunk.fileId,
OffsetInChunk: chunkStart - chunk.start + chunk.offsetInChunk,
ViewSize: uint64(chunkStop - chunkStart),
ViewOffset: chunkStart,
ChunkSize: chunk.chunkSize,
CipherKey: chunk.cipherKey,
IsGzipped: chunk.isGzipped,
ModifiedTsNs: chunk.modifiedTsNs,
}
chunkViews.AppendInterval(&Interval[*ChunkView]{
StartOffset: chunkStart,
StopOffset: chunkStop,
TsNs: chunk.modifiedTsNs,
Value: chunkView,
Prev: nil,
Next: nil,
}) })
} }
} }
return views
return chunkViews
} }
func logPrintf(name string, visibles []VisibleInterval) {
/*
glog.V(0).Infof("%s len %d", name, len(visibles))
for _, v := range visibles {
glog.V(0).Infof("%s: [%d,%d) %s %d", name, v.start, v.stop, v.fileId, v.chunkOffset)
}
*/
}
func MergeIntoVisibles(visibles []VisibleInterval, chunk *filer_pb.FileChunk) (newVisibles []VisibleInterval) {
newV := newVisibleInterval(chunk.Offset, chunk.Offset+int64(chunk.Size), chunk.GetFileIdString(), chunk.ModifiedTsNs, 0, chunk.Size, chunk.CipherKey, chunk.IsCompressed)
length := len(visibles)
if length == 0 {
return append(visibles, newV)
}
last := visibles[length-1]
if last.stop <= chunk.Offset {
return append(visibles, newV)
func MergeIntoVisibles(visibles *IntervalList[*VisibleInterval], start int64, stop int64, chunk *filer_pb.FileChunk) {
newV := &VisibleInterval{
start: start,
stop: stop,
fileId: chunk.GetFileIdString(),
modifiedTsNs: chunk.ModifiedTsNs,
offsetInChunk: start - chunk.Offset, // the starting position in the chunk
chunkSize: chunk.Size, // size of the chunk
cipherKey: chunk.CipherKey,
isGzipped: chunk.IsCompressed,
} }
logPrintf(" before", visibles)
// glog.V(0).Infof("newVisibles %d adding chunk [%d,%d) %s size:%d", len(newVisibles), chunk.Offset, chunk.Offset+int64(chunk.Size), chunk.GetFileIdString(), chunk.Size)
chunkStop := chunk.Offset + int64(chunk.Size)
for _, v := range visibles {
if v.start < chunk.Offset && chunk.Offset < v.stop {
t := newVisibleInterval(v.start, chunk.Offset, v.fileId, v.modifiedTsNs, v.chunkOffset, v.chunkSize, v.cipherKey, v.isGzipped)
newVisibles = append(newVisibles, t)
// glog.V(0).Infof("visible %d [%d,%d) =1> [%d,%d)", i, v.start, v.stop, t.start, t.stop)
}
if v.start < chunkStop && chunkStop < v.stop {
t := newVisibleInterval(chunkStop, v.stop, v.fileId, v.modifiedTsNs, v.chunkOffset+(chunkStop-v.start), v.chunkSize, v.cipherKey, v.isGzipped)
newVisibles = append(newVisibles, t)
// glog.V(0).Infof("visible %d [%d,%d) =2> [%d,%d)", i, v.start, v.stop, t.start, t.stop)
}
if chunkStop <= v.start || v.stop <= chunk.Offset {
newVisibles = append(newVisibles, v)
// glog.V(0).Infof("visible %d [%d,%d) =3> [%d,%d)", i, v.start, v.stop, v.start, v.stop)
}
}
newVisibles = append(newVisibles, newV)
logPrintf(" append", newVisibles)
visibles.InsertInterval(start, stop, chunk.ModifiedTsNs, newV)
}
for i := len(newVisibles) - 1; i >= 0; i-- {
if i > 0 && newV.start < newVisibles[i-1].start {
newVisibles[i] = newVisibles[i-1]
} else {
newVisibles[i] = newV
break
}
func MergeIntoChunkViews(chunkViews *IntervalList[*ChunkView], start int64, stop int64, chunk *filer_pb.FileChunk) {
chunkView := &ChunkView{
FileId: chunk.GetFileIdString(),
OffsetInChunk: start - chunk.Offset,
ViewSize: uint64(stop - start),
ViewOffset: start,
ChunkSize: chunk.Size,
CipherKey: chunk.CipherKey,
IsGzipped: chunk.IsCompressed,
ModifiedTsNs: chunk.ModifiedTsNs,
} }
logPrintf(" sorted", newVisibles)
return newVisibles
chunkViews.InsertInterval(start, stop, chunk.ModifiedTsNs, chunkView)
} }
// NonOverlappingVisibleIntervals translates the file chunk into VisibleInterval in memory // NonOverlappingVisibleIntervals translates the file chunk into VisibleInterval in memory
// If the file chunk content is a chunk manifest // If the file chunk content is a chunk manifest
func NonOverlappingVisibleIntervals(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk, startOffset int64, stopOffset int64) (visibles []VisibleInterval, err error) {
func NonOverlappingVisibleIntervals(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk, startOffset int64, stopOffset int64) (visibles *IntervalList[*VisibleInterval], err error) {
chunks, _, err = ResolveChunkManifest(lookupFileIdFn, chunks, startOffset, stopOffset) chunks, _, err = ResolveChunkManifest(lookupFileIdFn, chunks, startOffset, stopOffset)
if err != nil { if err != nil {
return return
} }
visibles2 := readResolvedChunks(chunks)
if true {
return visibles2, err
}
slices.SortFunc(chunks, func(a, b *filer_pb.FileChunk) bool {
if a.ModifiedTsNs == b.ModifiedTsNs {
filer_pb.EnsureFid(a)
filer_pb.EnsureFid(b)
if a.Fid == nil || b.Fid == nil {
return true
}
return a.Fid.FileKey < b.Fid.FileKey
}
return a.ModifiedTsNs < b.ModifiedTsNs
})
for _, chunk := range chunks {
// glog.V(0).Infof("merge [%d,%d)", chunk.Offset, chunk.Offset+int64(chunk.Size))
visibles = MergeIntoVisibles(visibles, chunk)
logPrintf("add", visibles)
visibles2 := readResolvedChunks(chunks, 0, math.MaxInt64)
}
if len(visibles) != len(visibles2) {
fmt.Printf("different visibles size %d : %d\n", len(visibles), len(visibles2))
} else {
for i := 0; i < len(visibles); i++ {
checkDifference(visibles[i], visibles2[i])
}
}
return
}
func checkDifference(x, y VisibleInterval) {
if x.start != y.start ||
x.stop != y.stop ||
x.fileId != y.fileId ||
x.modifiedTsNs != y.modifiedTsNs {
fmt.Printf("different visible %+v : %+v\n", x, y)
}
return visibles2, err
} }
// find non-overlapping visible intervals // find non-overlapping visible intervals
// visible interval map to one file chunk // visible interval map to one file chunk
type VisibleInterval struct { type VisibleInterval struct {
start int64
stop int64
modifiedTsNs int64
fileId string
chunkOffset int64
chunkSize uint64
cipherKey []byte
isGzipped bool
start int64
stop int64
modifiedTsNs int64
fileId string
offsetInChunk int64
chunkSize uint64
cipherKey []byte
isGzipped bool
} }
func newVisibleInterval(start, stop int64, fileId string, modifiedTime int64, chunkOffset int64, chunkSize uint64, cipherKey []byte, isGzipped bool) VisibleInterval {
return VisibleInterval{
start: start,
stop: stop,
fileId: fileId,
modifiedTsNs: modifiedTime,
chunkOffset: chunkOffset, // the starting position in the chunk
chunkSize: chunkSize,
cipherKey: cipherKey,
isGzipped: isGzipped,
func (v *VisibleInterval) SetStartStop(start, stop int64) {
v.offsetInChunk += start - v.start
v.start, v.stop = start, stop
}
func (v *VisibleInterval) Clone() IntervalValue {
return &VisibleInterval{
start: v.start,
stop: v.stop,
modifiedTsNs: v.modifiedTsNs,
fileId: v.fileId,
offsetInChunk: v.offsetInChunk,
chunkSize: v.chunkSize,
cipherKey: v.cipherKey,
isGzipped: v.isGzipped,
} }
} }

106
weed/filer/filechunks_read.go

@ -1,14 +1,22 @@
package filer package filer
import ( import (
"container/list"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"golang.org/x/exp/slices" "golang.org/x/exp/slices"
) )
func readResolvedChunks(chunks []*filer_pb.FileChunk) (visibles []VisibleInterval) {
func readResolvedChunks(chunks []*filer_pb.FileChunk, startOffset int64, stopOffset int64) (visibles *IntervalList[*VisibleInterval]) {
var points []*Point var points []*Point
for _, chunk := range chunks { for _, chunk := range chunks {
if chunk.IsChunkManifest {
println("This should not happen! A manifest chunk found:", chunk.GetFileIdString())
}
start, stop := max(chunk.Offset, startOffset), min(chunk.Offset+int64(chunk.Size), stopOffset)
if start >= stop {
continue
}
points = append(points, &Point{ points = append(points, &Point{
x: chunk.Offset, x: chunk.Offset,
ts: chunk.ModifiedTsNs, ts: chunk.ModifiedTsNs,
@ -33,40 +41,45 @@ func readResolvedChunks(chunks []*filer_pb.FileChunk) (visibles []VisibleInterva
}) })
var prevX int64 var prevX int64
var queue []*Point
queue := list.New() // points with higher ts are at the tail
visibles = NewIntervalList[*VisibleInterval]()
var prevPoint *Point
for _, point := range points { for _, point := range points {
if queue.Len() > 0 {
prevPoint = queue.Back().Value.(*Point)
} else {
prevPoint = nil
}
if point.isStart { if point.isStart {
if len(queue) > 0 {
lastIndex := len(queue) - 1
lastPoint := queue[lastIndex]
if point.x != prevX && lastPoint.ts < point.ts {
visibles = addToVisibles(visibles, prevX, lastPoint, point)
if prevPoint != nil {
if point.x != prevX && prevPoint.ts < point.ts {
addToVisibles(visibles, prevX, prevPoint, point)
prevX = point.x prevX = point.x
} }
} }
// insert into queue // insert into queue
for i := len(queue); i >= 0; i-- {
if i == 0 || queue[i-1].ts <= point.ts {
if i == len(queue) {
prevX = point.x
if prevPoint == nil || prevPoint.ts < point.ts {
queue.PushBack(point)
prevX = point.x
} else {
for e := queue.Front(); e != nil; e = e.Next() {
if e.Value.(*Point).ts > point.ts {
queue.InsertBefore(point, e)
break
} }
queue = addToQueue(queue, i, point)
break
} }
} }
} else { } else {
lastIndex := len(queue) - 1
index := lastIndex
var startPoint *Point
for ; index >= 0; index-- {
startPoint = queue[index]
if startPoint.ts == point.ts {
queue = removeFromQueue(queue, index)
isLast := true
for e := queue.Back(); e != nil; e = e.Prev() {
if e.Value.(*Point).ts == point.ts {
queue.Remove(e)
break break
} }
isLast = false
} }
if index == lastIndex && startPoint != nil {
visibles = addToVisibles(visibles, prevX, startPoint, point)
if isLast && prevPoint != nil {
addToVisibles(visibles, prevX, prevPoint, point)
prevX = point.x prevX = point.x
} }
} }
@ -75,37 +88,30 @@ func readResolvedChunks(chunks []*filer_pb.FileChunk) (visibles []VisibleInterva
return return
} }
func removeFromQueue(queue []*Point, index int) []*Point {
for i := index; i < len(queue)-1; i++ {
queue[i] = queue[i+1]
}
queue = queue[:len(queue)-1]
return queue
}
func addToQueue(queue []*Point, index int, point *Point) []*Point {
queue = append(queue, point)
for i := len(queue) - 1; i > index; i-- {
queue[i], queue[i-1] = queue[i-1], queue[i]
}
return queue
}
func addToVisibles(visibles []VisibleInterval, prevX int64, startPoint *Point, point *Point) []VisibleInterval {
func addToVisibles(visibles *IntervalList[*VisibleInterval], prevX int64, startPoint *Point, point *Point) {
if prevX < point.x { if prevX < point.x {
chunk := startPoint.chunk chunk := startPoint.chunk
visibles = append(visibles, VisibleInterval{
start: prevX,
stop: point.x,
fileId: chunk.GetFileIdString(),
modifiedTsNs: chunk.ModifiedTsNs,
chunkOffset: prevX - chunk.Offset,
chunkSize: chunk.Size,
cipherKey: chunk.CipherKey,
isGzipped: chunk.IsCompressed,
})
visible := &VisibleInterval{
start: prevX,
stop: point.x,
fileId: chunk.GetFileIdString(),
modifiedTsNs: chunk.ModifiedTsNs,
offsetInChunk: prevX - chunk.Offset,
chunkSize: chunk.Size,
cipherKey: chunk.CipherKey,
isGzipped: chunk.IsCompressed,
}
appendVisibleInterfal(visibles, visible)
} }
return visibles
}
func appendVisibleInterfal(visibles *IntervalList[*VisibleInterval], visible *VisibleInterval) {
visibles.AppendInterval(&Interval[*VisibleInterval]{
StartOffset: visible.start,
StopOffset: visible.stop,
TsNs: visible.modifiedTsNs,
Value: visible,
})
} }
type Point struct { type Point struct {

86
weed/filer/filechunks_read_test.go

@ -3,6 +3,7 @@ package filer
import ( import (
"fmt" "fmt"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"math"
"math/rand" "math/rand"
"testing" "testing"
) )
@ -42,9 +43,38 @@ func TestReadResolvedChunks(t *testing.T) {
}, },
} }
visibles := readResolvedChunks(chunks)
visibles := readResolvedChunks(chunks, 0, math.MaxInt64)
for _, visible := range visibles {
fmt.Printf("resolved to %d visible intervales\n", visibles.Len())
for x := visibles.Front(); x != nil; x = x.Next {
visible := x.Value
fmt.Printf("[%d,%d) %s %d\n", visible.start, visible.stop, visible.fileId, visible.modifiedTsNs)
}
}
func TestReadResolvedChunks2(t *testing.T) {
chunks := []*filer_pb.FileChunk{
{
FileId: "c",
Offset: 200,
Size: 50,
ModifiedTsNs: 3,
},
{
FileId: "e",
Offset: 200,
Size: 25,
ModifiedTsNs: 5,
},
}
visibles := readResolvedChunks(chunks, 0, math.MaxInt64)
fmt.Printf("resolved to %d visible intervales\n", visibles.Len())
for x := visibles.Front(); x != nil; x = x.Next {
visible := x.Value
fmt.Printf("[%d,%d) %s %d\n", visible.start, visible.stop, visible.fileId, visible.modifiedTsNs) fmt.Printf("[%d,%d) %s %d\n", visible.start, visible.stop, visible.fileId, visible.modifiedTsNs)
} }
@ -72,9 +102,10 @@ func TestRandomizedReadResolvedChunks(t *testing.T) {
chunks = append(chunks, randomWrite(array, start, size, ts)) chunks = append(chunks, randomWrite(array, start, size, ts))
} }
visibles := readResolvedChunks(chunks)
visibles := readResolvedChunks(chunks, 0, math.MaxInt64)
for _, visible := range visibles {
for x := visibles.Front(); x != nil; x = x.Next {
visible := x.Value
for i := visible.start; i < visible.stop; i++ { for i := visible.start; i < visible.stop; i++ {
if array[i] != visible.modifiedTsNs { if array[i] != visible.modifiedTsNs {
t.Errorf("position %d expected ts %d actual ts %d", i, array[i], visible.modifiedTsNs) t.Errorf("position %d expected ts %d actual ts %d", i, array[i], visible.modifiedTsNs)
@ -112,9 +143,9 @@ func TestSequentialReadResolvedChunks(t *testing.T) {
}) })
} }
visibles := readResolvedChunks(chunks)
visibles := readResolvedChunks(chunks, 0, math.MaxInt64)
fmt.Printf("visibles %d", len(visibles))
fmt.Printf("visibles %d", visibles.Len())
} }
@ -201,9 +232,48 @@ func TestActualReadResolvedChunks(t *testing.T) {
}, },
} }
visibles := readResolvedChunks(chunks)
visibles := readResolvedChunks(chunks, 0, math.MaxInt64)
for x := visibles.Front(); x != nil; x = x.Next {
visible := x.Value
fmt.Printf("[%d,%d) %s %d\n", visible.start, visible.stop, visible.fileId, visible.modifiedTsNs)
}
}
func TestActualReadResolvedChunks2(t *testing.T) {
chunks := []*filer_pb.FileChunk{
{
FileId: "1,e7b96fef48",
Offset: 0,
Size: 184320,
ModifiedTsNs: 1,
},
{
FileId: "2,22562640b9",
Offset: 184320,
Size: 4096,
ModifiedTsNs: 2,
},
{
FileId: "2,33562640b9",
Offset: 184320,
Size: 4096,
ModifiedTsNs: 4,
},
{
FileId: "4,df033e0fe4",
Offset: 188416,
Size: 2097152,
ModifiedTsNs: 3,
},
}
visibles := readResolvedChunks(chunks, 0, math.MaxInt64)
for _, visible := range visibles {
for x := visibles.Front(); x != nil; x = x.Next {
visible := x.Value
fmt.Printf("[%d,%d) %s %d\n", visible.start, visible.stop, visible.fileId, visible.modifiedTsNs) fmt.Printf("[%d,%d) %s %d\n", visible.start, visible.stop, visible.fileId, visible.modifiedTsNs)
} }

214
weed/filer/filechunks_test.go

@ -92,7 +92,8 @@ func TestRandomFileChunksCompact(t *testing.T) {
visibles, _ := NonOverlappingVisibleIntervals(nil, chunks, 0, math.MaxInt64) visibles, _ := NonOverlappingVisibleIntervals(nil, chunks, 0, math.MaxInt64)
for _, v := range visibles {
for visible := visibles.Front(); visible != nil; visible = visible.Next {
v := visible.Value
for x := v.start; x < v.stop; x++ { for x := v.start; x < v.stop; x++ {
assert.Equal(t, strconv.Itoa(int(data[x])), v.fileId) assert.Equal(t, strconv.Itoa(int(data[x])), v.fileId)
} }
@ -137,7 +138,7 @@ func TestIntervalMerging(t *testing.T) {
}, },
Expected: []*VisibleInterval{ Expected: []*VisibleInterval{
{start: 0, stop: 70, fileId: "b"}, {start: 0, stop: 70, fileId: "b"},
{start: 70, stop: 100, fileId: "a", chunkOffset: 70},
{start: 70, stop: 100, fileId: "a", offsetInChunk: 70},
}, },
}, },
// case 3: updates overwrite full chunks // case 3: updates overwrite full chunks
@ -174,15 +175,15 @@ func TestIntervalMerging(t *testing.T) {
}, },
Expected: []*VisibleInterval{ Expected: []*VisibleInterval{
{start: 0, stop: 200, fileId: "d"}, {start: 0, stop: 200, fileId: "d"},
{start: 200, stop: 220, fileId: "c", chunkOffset: 130},
{start: 200, stop: 220, fileId: "c", offsetInChunk: 130},
}, },
}, },
// case 6: same updates // case 6: same updates
{ {
Chunks: []*filer_pb.FileChunk{ Chunks: []*filer_pb.FileChunk{
{Offset: 0, Size: 100, FileId: "abc", Fid: &filer_pb.FileId{FileKey: 1}, ModifiedTsNs: 123}, {Offset: 0, Size: 100, FileId: "abc", Fid: &filer_pb.FileId{FileKey: 1}, ModifiedTsNs: 123},
{Offset: 0, Size: 100, FileId: "axf", Fid: &filer_pb.FileId{FileKey: 2}, ModifiedTsNs: 123},
{Offset: 0, Size: 100, FileId: "xyz", Fid: &filer_pb.FileId{FileKey: 3}, ModifiedTsNs: 123},
{Offset: 0, Size: 100, FileId: "axf", Fid: &filer_pb.FileId{FileKey: 2}, ModifiedTsNs: 124},
{Offset: 0, Size: 100, FileId: "xyz", Fid: &filer_pb.FileId{FileKey: 3}, ModifiedTsNs: 125},
}, },
Expected: []*VisibleInterval{ Expected: []*VisibleInterval{
{start: 0, stop: 100, fileId: "xyz"}, {start: 0, stop: 100, fileId: "xyz"},
@ -228,11 +229,17 @@ func TestIntervalMerging(t *testing.T) {
for i, testcase := range testcases { for i, testcase := range testcases {
log.Printf("++++++++++ merged test case %d ++++++++++++++++++++", i) log.Printf("++++++++++ merged test case %d ++++++++++++++++++++", i)
intervals, _ := NonOverlappingVisibleIntervals(nil, testcase.Chunks, 0, math.MaxInt64) intervals, _ := NonOverlappingVisibleIntervals(nil, testcase.Chunks, 0, math.MaxInt64)
for x, interval := range intervals {
log.Printf("test case %d, interval %d, start=%d, stop=%d, fileId=%s",
i, x, interval.start, interval.stop, interval.fileId)
x := -1
for visible := intervals.Front(); visible != nil; visible = visible.Next {
x++
interval := visible.Value
log.Printf("test case %d, interval start=%d, stop=%d, fileId=%s",
i, interval.start, interval.stop, interval.fileId)
} }
for x, interval := range intervals {
x = -1
for visible := intervals.Front(); visible != nil; visible = visible.Next {
x++
interval := visible.Value
if interval.start != testcase.Expected[x].start { if interval.start != testcase.Expected[x].start {
t.Fatalf("failed on test case %d, interval %d, start %d, expect %d", t.Fatalf("failed on test case %d, interval %d, start %d, expect %d",
i, x, interval.start, testcase.Expected[x].start) i, x, interval.start, testcase.Expected[x].start)
@ -245,13 +252,13 @@ func TestIntervalMerging(t *testing.T) {
t.Fatalf("failed on test case %d, interval %d, chunkId %s, expect %s", t.Fatalf("failed on test case %d, interval %d, chunkId %s, expect %s",
i, x, interval.fileId, testcase.Expected[x].fileId) i, x, interval.fileId, testcase.Expected[x].fileId)
} }
if interval.chunkOffset != testcase.Expected[x].chunkOffset {
t.Fatalf("failed on test case %d, interval %d, chunkOffset %d, expect %d",
i, x, interval.chunkOffset, testcase.Expected[x].chunkOffset)
if interval.offsetInChunk != testcase.Expected[x].offsetInChunk {
t.Fatalf("failed on test case %d, interval %d, offsetInChunk %d, expect %d",
i, x, interval.offsetInChunk, testcase.Expected[x].offsetInChunk)
} }
} }
if len(intervals) != len(testcase.Expected) {
t.Fatalf("failed to compact test case %d, len %d expected %d", i, len(intervals), len(testcase.Expected))
if intervals.Len() != len(testcase.Expected) {
t.Fatalf("failed to compact test case %d, len %d expected %d", i, intervals.Len(), len(testcase.Expected))
} }
} }
@ -276,9 +283,9 @@ func TestChunksReading(t *testing.T) {
Offset: 0, Offset: 0,
Size: 250, Size: 250,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 100, FileId: "abc", LogicOffset: 0},
{Offset: 0, Size: 100, FileId: "asdf", LogicOffset: 100},
{Offset: 0, Size: 50, FileId: "fsad", LogicOffset: 200},
{OffsetInChunk: 0, ViewSize: 100, FileId: "abc", ViewOffset: 0},
{OffsetInChunk: 0, ViewSize: 100, FileId: "asdf", ViewOffset: 100},
{OffsetInChunk: 0, ViewSize: 50, FileId: "fsad", ViewOffset: 200},
}, },
}, },
// case 1: updates overwrite full chunks // case 1: updates overwrite full chunks
@ -290,7 +297,7 @@ func TestChunksReading(t *testing.T) {
Offset: 50, Offset: 50,
Size: 100, Size: 100,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 50, Size: 100, FileId: "asdf", LogicOffset: 50},
{OffsetInChunk: 50, ViewSize: 100, FileId: "asdf", ViewOffset: 50},
}, },
}, },
// case 2: updates overwrite part of previous chunks // case 2: updates overwrite part of previous chunks
@ -302,8 +309,8 @@ func TestChunksReading(t *testing.T) {
Offset: 30, Offset: 30,
Size: 40, Size: 40,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 20, Size: 30, FileId: "b", LogicOffset: 30},
{Offset: 57, Size: 10, FileId: "a", LogicOffset: 60},
{OffsetInChunk: 20, ViewSize: 30, FileId: "b", ViewOffset: 30},
{OffsetInChunk: 57, ViewSize: 10, FileId: "a", ViewOffset: 60},
}, },
}, },
// case 3: updates overwrite full chunks // case 3: updates overwrite full chunks
@ -316,8 +323,8 @@ func TestChunksReading(t *testing.T) {
Offset: 0, Offset: 0,
Size: 200, Size: 200,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 50, FileId: "asdf", LogicOffset: 0},
{Offset: 0, Size: 150, FileId: "xxxx", LogicOffset: 50},
{OffsetInChunk: 0, ViewSize: 50, FileId: "asdf", ViewOffset: 0},
{OffsetInChunk: 0, ViewSize: 150, FileId: "xxxx", ViewOffset: 50},
}, },
}, },
// case 4: updates far away from prev chunks // case 4: updates far away from prev chunks
@ -330,8 +337,8 @@ func TestChunksReading(t *testing.T) {
Offset: 0, Offset: 0,
Size: 400, Size: 400,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 200, FileId: "asdf", LogicOffset: 0},
{Offset: 0, Size: 150, FileId: "xxxx", LogicOffset: 250},
{OffsetInChunk: 0, ViewSize: 200, FileId: "asdf", ViewOffset: 0},
{OffsetInChunk: 0, ViewSize: 150, FileId: "xxxx", ViewOffset: 250},
}, },
}, },
// case 5: updates overwrite full chunks // case 5: updates overwrite full chunks
@ -345,21 +352,21 @@ func TestChunksReading(t *testing.T) {
Offset: 0, Offset: 0,
Size: 220, Size: 220,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 200, FileId: "c", LogicOffset: 0},
{Offset: 130, Size: 20, FileId: "b", LogicOffset: 200},
{OffsetInChunk: 0, ViewSize: 200, FileId: "c", ViewOffset: 0},
{OffsetInChunk: 130, ViewSize: 20, FileId: "b", ViewOffset: 200},
}, },
}, },
// case 6: same updates // case 6: same updates
{ {
Chunks: []*filer_pb.FileChunk{ Chunks: []*filer_pb.FileChunk{
{Offset: 0, Size: 100, FileId: "abc", Fid: &filer_pb.FileId{FileKey: 1}, ModifiedTsNs: 123}, {Offset: 0, Size: 100, FileId: "abc", Fid: &filer_pb.FileId{FileKey: 1}, ModifiedTsNs: 123},
{Offset: 0, Size: 100, FileId: "def", Fid: &filer_pb.FileId{FileKey: 2}, ModifiedTsNs: 123},
{Offset: 0, Size: 100, FileId: "xyz", Fid: &filer_pb.FileId{FileKey: 3}, ModifiedTsNs: 123},
{Offset: 0, Size: 100, FileId: "def", Fid: &filer_pb.FileId{FileKey: 2}, ModifiedTsNs: 124},
{Offset: 0, Size: 100, FileId: "xyz", Fid: &filer_pb.FileId{FileKey: 3}, ModifiedTsNs: 125},
}, },
Offset: 0, Offset: 0,
Size: 100, Size: 100,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 100, FileId: "xyz", LogicOffset: 0},
{OffsetInChunk: 0, ViewSize: 100, FileId: "xyz", ViewOffset: 0},
}, },
}, },
// case 7: edge cases // case 7: edge cases
@ -372,8 +379,8 @@ func TestChunksReading(t *testing.T) {
Offset: 0, Offset: 0,
Size: 200, Size: 200,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 100, FileId: "abc", LogicOffset: 0},
{Offset: 0, Size: 100, FileId: "asdf", LogicOffset: 100},
{OffsetInChunk: 0, ViewSize: 100, FileId: "abc", ViewOffset: 0},
{OffsetInChunk: 0, ViewSize: 100, FileId: "asdf", ViewOffset: 100},
}, },
}, },
// case 8: edge cases // case 8: edge cases
@ -386,9 +393,9 @@ func TestChunksReading(t *testing.T) {
Offset: 0, Offset: 0,
Size: 300, Size: 300,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 90, FileId: "abc", LogicOffset: 0},
{Offset: 0, Size: 100, FileId: "asdf", LogicOffset: 90},
{Offset: 0, Size: 110, FileId: "fsad", LogicOffset: 190},
{OffsetInChunk: 0, ViewSize: 90, FileId: "abc", ViewOffset: 0},
{OffsetInChunk: 0, ViewSize: 100, FileId: "asdf", ViewOffset: 90},
{OffsetInChunk: 0, ViewSize: 110, FileId: "fsad", ViewOffset: 190},
}, },
}, },
// case 9: edge cases // case 9: edge cases
@ -404,12 +411,12 @@ func TestChunksReading(t *testing.T) {
Offset: 0, Offset: 0,
Size: 153578836, Size: 153578836,
Expected: []*ChunkView{ Expected: []*ChunkView{
{Offset: 0, Size: 43175936, FileId: "2,111fc2cbfac1", LogicOffset: 0},
{Offset: 0, Size: 52981760 - 43175936, FileId: "2,112a36ea7f85", LogicOffset: 43175936},
{Offset: 0, Size: 72564736 - 52981760, FileId: "4,112d5f31c5e7", LogicOffset: 52981760},
{Offset: 0, Size: 133255168 - 72564736, FileId: "1,113245f0cdb6", LogicOffset: 72564736},
{Offset: 0, Size: 137269248 - 133255168, FileId: "3,1141a70733b5", LogicOffset: 133255168},
{Offset: 0, Size: 153578836 - 137269248, FileId: "1,114201d5bbdb", LogicOffset: 137269248},
{OffsetInChunk: 0, ViewSize: 43175936, FileId: "2,111fc2cbfac1", ViewOffset: 0},
{OffsetInChunk: 0, ViewSize: 52981760 - 43175936, FileId: "2,112a36ea7f85", ViewOffset: 43175936},
{OffsetInChunk: 0, ViewSize: 72564736 - 52981760, FileId: "4,112d5f31c5e7", ViewOffset: 52981760},
{OffsetInChunk: 0, ViewSize: 133255168 - 72564736, FileId: "1,113245f0cdb6", ViewOffset: 72564736},
{OffsetInChunk: 0, ViewSize: 137269248 - 133255168, FileId: "3,1141a70733b5", ViewOffset: 133255168},
{OffsetInChunk: 0, ViewSize: 153578836 - 137269248, FileId: "1,114201d5bbdb", ViewOffset: 137269248},
}, },
}, },
} }
@ -420,28 +427,31 @@ func TestChunksReading(t *testing.T) {
} }
log.Printf("++++++++++ read test case %d ++++++++++++++++++++", i) log.Printf("++++++++++ read test case %d ++++++++++++++++++++", i)
chunks := ViewFromChunks(nil, testcase.Chunks, testcase.Offset, testcase.Size) chunks := ViewFromChunks(nil, testcase.Chunks, testcase.Offset, testcase.Size)
for x, chunk := range chunks {
x := -1
for c := chunks.Front(); c != nil; c = c.Next {
x++
chunk := c.Value
log.Printf("read case %d, chunk %d, offset=%d, size=%d, fileId=%s", log.Printf("read case %d, chunk %d, offset=%d, size=%d, fileId=%s",
i, x, chunk.Offset, chunk.Size, chunk.FileId)
if chunk.Offset != testcase.Expected[x].Offset {
i, x, chunk.OffsetInChunk, chunk.ViewSize, chunk.FileId)
if chunk.OffsetInChunk != testcase.Expected[x].OffsetInChunk {
t.Fatalf("failed on read case %d, chunk %s, Offset %d, expect %d", t.Fatalf("failed on read case %d, chunk %s, Offset %d, expect %d",
i, chunk.FileId, chunk.Offset, testcase.Expected[x].Offset)
i, chunk.FileId, chunk.OffsetInChunk, testcase.Expected[x].OffsetInChunk)
} }
if chunk.Size != testcase.Expected[x].Size {
t.Fatalf("failed on read case %d, chunk %s, Size %d, expect %d",
i, chunk.FileId, chunk.Size, testcase.Expected[x].Size)
if chunk.ViewSize != testcase.Expected[x].ViewSize {
t.Fatalf("failed on read case %d, chunk %s, ViewSize %d, expect %d",
i, chunk.FileId, chunk.ViewSize, testcase.Expected[x].ViewSize)
} }
if chunk.FileId != testcase.Expected[x].FileId { if chunk.FileId != testcase.Expected[x].FileId {
t.Fatalf("failed on read case %d, chunk %d, FileId %s, expect %s", t.Fatalf("failed on read case %d, chunk %d, FileId %s, expect %s",
i, x, chunk.FileId, testcase.Expected[x].FileId) i, x, chunk.FileId, testcase.Expected[x].FileId)
} }
if chunk.LogicOffset != testcase.Expected[x].LogicOffset {
t.Fatalf("failed on read case %d, chunk %d, LogicOffset %d, expect %d",
i, x, chunk.LogicOffset, testcase.Expected[x].LogicOffset)
if chunk.ViewOffset != testcase.Expected[x].ViewOffset {
t.Fatalf("failed on read case %d, chunk %d, ViewOffset %d, expect %d",
i, x, chunk.ViewOffset, testcase.Expected[x].ViewOffset)
} }
} }
if len(chunks) != len(testcase.Expected) {
t.Fatalf("failed to read test case %d, len %d expected %d", i, len(chunks), len(testcase.Expected))
if chunks.Len() != len(testcase.Expected) {
t.Fatalf("failed to read test case %d, len %d expected %d", i, chunks.Len(), len(testcase.Expected))
} }
} }
@ -467,73 +477,79 @@ func BenchmarkCompactFileChunks(b *testing.B) {
} }
} }
func addVisibleInterval(visibles *IntervalList[*VisibleInterval], x *VisibleInterval) {
visibles.AppendInterval(&Interval[*VisibleInterval]{
StartOffset: x.start,
StopOffset: x.stop,
TsNs: x.modifiedTsNs,
Value: x,
})
}
func TestViewFromVisibleIntervals(t *testing.T) { func TestViewFromVisibleIntervals(t *testing.T) {
visibles := []VisibleInterval{
{
start: 0,
stop: 25,
fileId: "fid1",
},
{
start: 4096,
stop: 8192,
fileId: "fid2",
},
{
start: 16384,
stop: 18551,
fileId: "fid3",
},
}
visibles := NewIntervalList[*VisibleInterval]()
addVisibleInterval(visibles, &VisibleInterval{
start: 0,
stop: 25,
fileId: "fid1",
})
addVisibleInterval(visibles, &VisibleInterval{
start: 4096,
stop: 8192,
fileId: "fid2",
})
addVisibleInterval(visibles, &VisibleInterval{
start: 16384,
stop: 18551,
fileId: "fid3",
})
views := ViewFromVisibleIntervals(visibles, 0, math.MaxInt32) views := ViewFromVisibleIntervals(visibles, 0, math.MaxInt32)
if len(views) != len(visibles) {
assert.Equal(t, len(visibles), len(views), "ViewFromVisibleIntervals error")
if views.Len() != visibles.Len() {
assert.Equal(t, visibles.Len(), views.Len(), "ViewFromVisibleIntervals error")
} }
} }
func TestViewFromVisibleIntervals2(t *testing.T) { func TestViewFromVisibleIntervals2(t *testing.T) {
visibles := []VisibleInterval{
{
start: 344064,
stop: 348160,
fileId: "fid1",
},
{
start: 348160,
stop: 356352,
fileId: "fid2",
},
}
visibles := NewIntervalList[*VisibleInterval]()
addVisibleInterval(visibles, &VisibleInterval{
start: 344064,
stop: 348160,
fileId: "fid1",
})
addVisibleInterval(visibles, &VisibleInterval{
start: 348160,
stop: 356352,
fileId: "fid2",
})
views := ViewFromVisibleIntervals(visibles, 0, math.MaxInt32) views := ViewFromVisibleIntervals(visibles, 0, math.MaxInt32)
if len(views) != len(visibles) {
assert.Equal(t, len(visibles), len(views), "ViewFromVisibleIntervals error")
if views.Len() != visibles.Len() {
assert.Equal(t, visibles.Len(), views.Len(), "ViewFromVisibleIntervals error")
} }
} }
func TestViewFromVisibleIntervals3(t *testing.T) { func TestViewFromVisibleIntervals3(t *testing.T) {
visibles := []VisibleInterval{
{
start: 1000,
stop: 2000,
fileId: "fid1",
},
{
start: 3000,
stop: 4000,
fileId: "fid2",
},
}
visibles := NewIntervalList[*VisibleInterval]()
addVisibleInterval(visibles, &VisibleInterval{
start: 1000,
stop: 2000,
fileId: "fid1",
})
addVisibleInterval(visibles, &VisibleInterval{
start: 3000,
stop: 4000,
fileId: "fid2",
})
views := ViewFromVisibleIntervals(visibles, 1700, 1500) views := ViewFromVisibleIntervals(visibles, 1700, 1500)
if len(views) != len(visibles) {
assert.Equal(t, len(visibles), len(views), "ViewFromVisibleIntervals error")
if views.Len() != visibles.Len() {
assert.Equal(t, visibles.Len(), views.Len(), "ViewFromVisibleIntervals error")
} }
} }

2
weed/filer/filer_notify_append.go

@ -40,7 +40,7 @@ func (f *Filer) appendToFile(targetFile string, data []byte) error {
} }
// append to existing chunks // append to existing chunks
entry.Chunks = append(entry.GetChunks(), uploadResult.ToPbFileChunk(assignResult.Fid, offset))
entry.Chunks = append(entry.GetChunks(), uploadResult.ToPbFileChunk(assignResult.Fid, offset, time.Now().UnixNano()))
// update the entry // update the entry
err = f.CreateEntry(context.Background(), entry, false, false, nil, false) err = f.CreateEntry(context.Background(), entry, false, false, nil, false)

259
weed/filer/interval_list.go

@ -0,0 +1,259 @@
package filer
import (
"math"
"sync"
)
type IntervalValue interface {
SetStartStop(start, stop int64)
Clone() IntervalValue
}
type Interval[T IntervalValue] struct {
StartOffset int64
StopOffset int64
TsNs int64
Value T
Prev *Interval[T]
Next *Interval[T]
}
func (interval *Interval[T]) Size() int64 {
return interval.StopOffset - interval.StartOffset
}
// IntervalList mark written intervals within one page chunk
type IntervalList[T IntervalValue] struct {
head *Interval[T]
tail *Interval[T]
Lock sync.Mutex
}
func NewIntervalList[T IntervalValue]() *IntervalList[T] {
list := &IntervalList[T]{
head: &Interval[T]{
StartOffset: -1,
StopOffset: -1,
},
tail: &Interval[T]{
StartOffset: math.MaxInt64,
StopOffset: math.MaxInt64,
},
}
return list
}
func (list *IntervalList[T]) Front() (interval *Interval[T]) {
return list.head.Next
}
func (list *IntervalList[T]) AppendInterval(interval *Interval[T]) {
list.Lock.Lock()
defer list.Lock.Unlock()
if list.head.Next == nil {
list.head.Next = interval
}
interval.Prev = list.tail.Prev
if list.tail.Prev != nil {
list.tail.Prev.Next = interval
}
list.tail.Prev = interval
}
func (list *IntervalList[T]) Overlay(startOffset, stopOffset, tsNs int64, value T) {
if startOffset >= stopOffset {
return
}
interval := &Interval[T]{
StartOffset: startOffset,
StopOffset: stopOffset,
TsNs: tsNs,
Value: value,
}
list.Lock.Lock()
defer list.Lock.Unlock()
list.overlayInterval(interval)
}
func (list *IntervalList[T]) InsertInterval(startOffset, stopOffset, tsNs int64, value T) {
interval := &Interval[T]{
StartOffset: startOffset,
StopOffset: stopOffset,
TsNs: tsNs,
Value: value,
}
list.Lock.Lock()
defer list.Lock.Unlock()
value.SetStartStop(startOffset, stopOffset)
list.insertInterval(interval)
}
func (list *IntervalList[T]) insertInterval(interval *Interval[T]) {
prev := list.head
next := prev.Next
for interval.StartOffset < interval.StopOffset {
if next == nil {
// add to the end
list.insertBetween(prev, interval, list.tail)
break
}
// interval is ahead of the next
if interval.StopOffset <= next.StartOffset {
list.insertBetween(prev, interval, next)
break
}
// interval is after the next
if next.StopOffset <= interval.StartOffset {
prev = next
next = next.Next
continue
}
// intersecting next and interval
if interval.TsNs >= next.TsNs {
// interval is newer
if next.StartOffset < interval.StartOffset {
// left side of next is ahead of interval
t := &Interval[T]{
StartOffset: next.StartOffset,
StopOffset: interval.StartOffset,
TsNs: next.TsNs,
Value: next.Value.Clone().(T),
}
t.Value.SetStartStop(t.StartOffset, t.StopOffset)
list.insertBetween(prev, t, interval)
next.StartOffset = interval.StartOffset
next.Value.SetStartStop(next.StartOffset, next.StopOffset)
prev = t
}
if interval.StopOffset < next.StopOffset {
// right side of next is after interval
next.StartOffset = interval.StopOffset
next.Value.SetStartStop(next.StartOffset, next.StopOffset)
list.insertBetween(prev, interval, next)
break
} else {
// next is covered
prev.Next = interval
next = next.Next
}
} else {
// next is newer
if interval.StartOffset < next.StartOffset {
// left side of interval is ahead of next
t := &Interval[T]{
StartOffset: interval.StartOffset,
StopOffset: next.StartOffset,
TsNs: interval.TsNs,
Value: interval.Value.Clone().(T),
}
t.Value.SetStartStop(t.StartOffset, t.StopOffset)
list.insertBetween(prev, t, next)
interval.StartOffset = next.StartOffset
interval.Value.SetStartStop(interval.StartOffset, interval.StopOffset)
}
if next.StopOffset < interval.StopOffset {
// right side of interval is after next
interval.StartOffset = next.StopOffset
interval.Value.SetStartStop(interval.StartOffset, interval.StopOffset)
} else {
// interval is covered
break
}
}
}
}
func (list *IntervalList[T]) insertBetween(a, interval, b *Interval[T]) {
a.Next = interval
b.Prev = interval
if a != list.head {
interval.Prev = a
}
if b != list.tail {
interval.Next = b
}
}
func (list *IntervalList[T]) overlayInterval(interval *Interval[T]) {
//t := list.head
//for ; t.Next != nil; t = t.Next {
// if t.TsNs > interval.TsNs {
// println("writes is out of order", t.TsNs-interval.TsNs, "ns")
// }
//}
p := list.head
for ; p.Next != nil && p.Next.StopOffset <= interval.StartOffset; p = p.Next {
}
q := list.tail
for ; q.Prev != nil && q.Prev.StartOffset >= interval.StopOffset; q = q.Prev {
}
// left side
// interval after p.Next start
if p.Next != nil && p.Next.StartOffset < interval.StartOffset {
t := &Interval[T]{
StartOffset: p.Next.StartOffset,
StopOffset: interval.StartOffset,
TsNs: p.Next.TsNs,
Value: p.Next.Value,
}
p.Next = t
if p != list.head {
t.Prev = p
}
t.Next = interval
interval.Prev = t
} else {
p.Next = interval
if p != list.head {
interval.Prev = p
}
}
// right side
// interval ends before p.Prev
if q.Prev != nil && interval.StopOffset < q.Prev.StopOffset {
t := &Interval[T]{
StartOffset: interval.StopOffset,
StopOffset: q.Prev.StopOffset,
TsNs: q.Prev.TsNs,
Value: q.Prev.Value,
}
q.Prev = t
if q != list.tail {
t.Next = q
}
interval.Next = t
t.Prev = interval
} else {
q.Prev = interval
if q != list.tail {
interval.Next = q
}
}
}
func (list *IntervalList[T]) Len() int {
list.Lock.Lock()
defer list.Lock.Unlock()
var count int
for t := list.head; t != nil; t = t.Next {
count++
}
return count - 1
}

327
weed/filer/interval_list_test.go

@ -0,0 +1,327 @@
package filer
import (
"fmt"
"github.com/stretchr/testify/assert"
"testing"
)
type IntervalInt int
func (i IntervalInt) SetStartStop(start, stop int64) {
}
func (i IntervalInt) Clone() IntervalValue {
return i
}
func TestIntervalList_Overlay(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(0, 100, 1, 1)
list.Overlay(50, 150, 2, 2)
list.Overlay(200, 250, 3, 3)
list.Overlay(225, 250, 4, 4)
list.Overlay(175, 210, 5, 5)
list.Overlay(0, 25, 6, 6)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 6, list.Len())
println()
list.Overlay(50, 150, 7, 7)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 6, list.Len())
}
func TestIntervalList_Overlay2(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(0, 50, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
}
func TestIntervalList_Overlay3(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
assert.Equal(t, 1, list.Len())
list.Overlay(0, 60, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_Overlay4(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(0, 100, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 1, list.Len())
}
func TestIntervalList_Overlay5(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(0, 110, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 1, list.Len())
}
func TestIntervalList_Overlay6(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(50, 110, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 1, list.Len())
}
func TestIntervalList_Overlay7(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(50, 90, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_Overlay8(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(60, 90, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 3, list.Len())
}
func TestIntervalList_Overlay9(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(60, 100, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_Overlay10(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(50, 100, 1, 1)
list.Overlay(60, 110, 2, 2)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_Overlay11(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.Overlay(0, 100, 1, 1)
list.Overlay(100, 110, 2, 2)
list.Overlay(0, 90, 3, 3)
list.Overlay(0, 80, 4, 4)
list.Overlay(0, 90, 5, 5)
list.Overlay(90, 90, 6, 6)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 3, list.Len())
}
func TestIntervalList_insertInterval1(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 150, 2, 2)
list.InsertInterval(200, 250, 3, 3)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_insertInterval2(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 150, 2, 2)
list.InsertInterval(0, 25, 3, 3)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_insertInterval3(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 150, 2, 2)
list.InsertInterval(200, 250, 4, 4)
list.InsertInterval(0, 75, 3, 3)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 3, list.Len())
}
func TestIntervalList_insertInterval4(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(200, 250, 4, 4)
list.InsertInterval(0, 225, 3, 3)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_insertInterval5(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(200, 250, 4, 4)
list.InsertInterval(0, 225, 5, 5)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_insertInterval6(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 150, 2, 2)
list.InsertInterval(200, 250, 4, 4)
list.InsertInterval(0, 275, 1, 1)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 5, list.Len())
}
func TestIntervalList_insertInterval7(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 150, 2, 2)
list.InsertInterval(200, 250, 4, 4)
list.InsertInterval(75, 275, 1, 1)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 4, list.Len())
}
func TestIntervalList_insertInterval8(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 150, 2, 2)
list.InsertInterval(200, 250, 4, 4)
list.InsertInterval(75, 275, 3, 3)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 4, list.Len())
}
func TestIntervalList_insertInterval9(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 150, 2, 2)
list.InsertInterval(200, 250, 4, 4)
list.InsertInterval(50, 150, 3, 3)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 2, list.Len())
}
func TestIntervalList_insertInterval10(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(50, 100, 2, 2)
list.InsertInterval(200, 300, 4, 4)
list.InsertInterval(100, 200, 5, 5)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 3, list.Len())
}
func TestIntervalList_insertInterval11(t *testing.T) {
list := NewIntervalList[IntervalInt]()
list.InsertInterval(0, 64, 1, 1)
list.InsertInterval(72, 136, 3, 3)
list.InsertInterval(64, 128, 2, 2)
list.InsertInterval(68, 72, 4, 4)
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 4, list.Len())
}
type IntervalStruct struct {
x int
start int64
stop int64
}
func newIntervalStruct(i int) IntervalStruct {
return IntervalStruct{
x: i,
}
}
func (i IntervalStruct) SetStartStop(start, stop int64) {
i.start, i.stop = start, stop
}
func (i IntervalStruct) Clone() IntervalValue {
return &IntervalStruct{
x: i.x,
start: i.start,
stop: i.stop,
}
}
func TestIntervalList_insertIntervalStruct(t *testing.T) {
list := NewIntervalList[IntervalStruct]()
list.InsertInterval(0, 64, 1, newIntervalStruct(1))
list.InsertInterval(64, 72, 2, newIntervalStruct(2))
list.InsertInterval(72, 136, 3, newIntervalStruct(3))
list.InsertInterval(64, 68, 4, newIntervalStruct(4))
for p := list.Front(); p != nil; p = p.Next {
fmt.Printf("[%d,%d) %d %d\n", p.StartOffset, p.StopOffset, p.TsNs, p.Value)
}
assert.Equal(t, 4, list.Len())
}

65
weed/filer/reader_at.go

@ -16,8 +16,7 @@ import (
type ChunkReadAt struct { type ChunkReadAt struct {
masterClient *wdclient.MasterClient masterClient *wdclient.MasterClient
chunkViews []*ChunkView
readerLock sync.Mutex
chunkViews *IntervalList[*ChunkView]
fileSize int64 fileSize int64
readerCache *ReaderCache readerCache *ReaderCache
readerPattern *ReaderPattern readerPattern *ReaderPattern
@ -89,7 +88,7 @@ func LookupFn(filerClient filer_pb.FilerClient) wdclient.LookupFileIdFunctionTyp
} }
} }
func NewChunkReaderAtFromClient(lookupFn wdclient.LookupFileIdFunctionType, chunkViews []*ChunkView, chunkCache chunk_cache.ChunkCache, fileSize int64) *ChunkReadAt {
func NewChunkReaderAtFromClient(lookupFn wdclient.LookupFileIdFunctionType, chunkViews *IntervalList[*ChunkView], chunkCache chunk_cache.ChunkCache, fileSize int64) *ChunkReadAt {
return &ChunkReadAt{ return &ChunkReadAt{
chunkViews: chunkViews, chunkViews: chunkViews,
@ -108,44 +107,58 @@ func (c *ChunkReadAt) ReadAt(p []byte, offset int64) (n int, err error) {
c.readerPattern.MonitorReadAt(offset, len(p)) c.readerPattern.MonitorReadAt(offset, len(p))
c.readerLock.Lock()
defer c.readerLock.Unlock()
c.chunkViews.Lock.Lock()
defer c.chunkViews.Lock.Unlock()
// glog.V(4).Infof("ReadAt [%d,%d) of total file size %d bytes %d chunk views", offset, offset+int64(len(p)), c.fileSize, len(c.chunkViews))
n, _, err = c.doReadAt(p, offset)
return
}
func (c *ChunkReadAt) ReadAtWithTime(p []byte, offset int64) (n int, ts int64, err error) {
c.readerPattern.MonitorReadAt(offset, len(p))
c.chunkViews.Lock.Lock()
defer c.chunkViews.Lock.Unlock()
// glog.V(4).Infof("ReadAt [%d,%d) of total file size %d bytes %d chunk views", offset, offset+int64(len(p)), c.fileSize, len(c.chunkViews)) // glog.V(4).Infof("ReadAt [%d,%d) of total file size %d bytes %d chunk views", offset, offset+int64(len(p)), c.fileSize, len(c.chunkViews))
return c.doReadAt(p, offset) return c.doReadAt(p, offset)
} }
func (c *ChunkReadAt) doReadAt(p []byte, offset int64) (n int, err error) {
func (c *ChunkReadAt) doReadAt(p []byte, offset int64) (n int, ts int64, err error) {
startOffset, remaining := offset, int64(len(p)) startOffset, remaining := offset, int64(len(p))
var nextChunks []*ChunkView
for i, chunk := range c.chunkViews {
var nextChunks *Interval[*ChunkView]
for x := c.chunkViews.Front(); x != nil; x = x.Next {
chunk := x.Value
if remaining <= 0 { if remaining <= 0 {
break break
} }
if i+1 < len(c.chunkViews) {
nextChunks = c.chunkViews[i+1:]
if x.Next != nil {
nextChunks = x.Next
} }
if startOffset < chunk.LogicOffset {
gap := chunk.LogicOffset - startOffset
glog.V(4).Infof("zero [%d,%d)", startOffset, chunk.LogicOffset)
if startOffset < chunk.ViewOffset {
gap := chunk.ViewOffset - startOffset
glog.V(4).Infof("zero [%d,%d)", startOffset, chunk.ViewOffset)
n += zero(p, startOffset-offset, gap) n += zero(p, startOffset-offset, gap)
startOffset, remaining = chunk.LogicOffset, remaining-gap
startOffset, remaining = chunk.ViewOffset, remaining-gap
if remaining <= 0 { if remaining <= 0 {
break break
} }
} }
// fmt.Printf(">>> doReadAt [%d,%d), chunk[%d,%d)\n", offset, offset+int64(len(p)), chunk.LogicOffset, chunk.LogicOffset+int64(chunk.Size))
chunkStart, chunkStop := max(chunk.LogicOffset, startOffset), min(chunk.LogicOffset+int64(chunk.Size), startOffset+remaining)
// fmt.Printf(">>> doReadAt [%d,%d), chunk[%d,%d)\n", offset, offset+int64(len(p)), chunk.ViewOffset, chunk.ViewOffset+int64(chunk.ViewSize))
chunkStart, chunkStop := max(chunk.ViewOffset, startOffset), min(chunk.ViewOffset+int64(chunk.ViewSize), startOffset+remaining)
if chunkStart >= chunkStop { if chunkStart >= chunkStop {
continue continue
} }
// glog.V(4).Infof("read [%d,%d), %d/%d chunk %s [%d,%d)", chunkStart, chunkStop, i, len(c.chunkViews), chunk.FileId, chunk.LogicOffset-chunk.Offset, chunk.LogicOffset-chunk.Offset+int64(chunk.Size))
bufferOffset := chunkStart - chunk.LogicOffset + chunk.Offset
// glog.V(4).Infof("read [%d,%d), %d/%d chunk %s [%d,%d)", chunkStart, chunkStop, i, len(c.chunkViews), chunk.FileId, chunk.ViewOffset-chunk.Offset, chunk.ViewOffset-chunk.Offset+int64(chunk.ViewSize))
bufferOffset := chunkStart - chunk.ViewOffset + chunk.OffsetInChunk
ts = chunk.ModifiedTsNs
copied, err := c.readChunkSliceAt(p[startOffset-offset:chunkStop-chunkStart+startOffset-offset], chunk, nextChunks, uint64(bufferOffset)) copied, err := c.readChunkSliceAt(p[startOffset-offset:chunkStop-chunkStart+startOffset-offset], chunk, nextChunks, uint64(bufferOffset))
if err != nil { if err != nil {
glog.Errorf("fetching chunk %+v: %v\n", chunk, err) glog.Errorf("fetching chunk %+v: %v\n", chunk, err)
return copied, err
return copied, ts, err
} }
n += copied n += copied
@ -177,7 +190,7 @@ func (c *ChunkReadAt) doReadAt(p []byte, offset int64) (n int, err error) {
} }
func (c *ChunkReadAt) readChunkSliceAt(buffer []byte, chunkView *ChunkView, nextChunkViews []*ChunkView, offset uint64) (n int, err error) {
func (c *ChunkReadAt) readChunkSliceAt(buffer []byte, chunkView *ChunkView, nextChunkViews *Interval[*ChunkView], offset uint64) (n int, err error) {
if c.readerPattern.IsRandomMode() { if c.readerPattern.IsRandomMode() {
n, err := c.readerCache.chunkCache.ReadChunkAt(buffer, chunkView.FileId, offset) n, err := c.readerCache.chunkCache.ReadChunkAt(buffer, chunkView.FileId, offset)
@ -187,16 +200,14 @@ func (c *ChunkReadAt) readChunkSliceAt(buffer []byte, chunkView *ChunkView, next
return fetchChunkRange(buffer, c.readerCache.lookupFileIdFn, chunkView.FileId, chunkView.CipherKey, chunkView.IsGzipped, int64(offset)) return fetchChunkRange(buffer, c.readerCache.lookupFileIdFn, chunkView.FileId, chunkView.CipherKey, chunkView.IsGzipped, int64(offset))
} }
n, err = c.readerCache.ReadChunkAt(buffer, chunkView.FileId, chunkView.CipherKey, chunkView.IsGzipped, int64(offset), int(chunkView.ChunkSize), chunkView.LogicOffset == 0)
n, err = c.readerCache.ReadChunkAt(buffer, chunkView.FileId, chunkView.CipherKey, chunkView.IsGzipped, int64(offset), int(chunkView.ChunkSize), chunkView.ViewOffset == 0)
if c.lastChunkFid != chunkView.FileId { if c.lastChunkFid != chunkView.FileId {
if chunkView.Offset == 0 { // start of a new chunk
if chunkView.OffsetInChunk == 0 { // start of a new chunk
if c.lastChunkFid != "" { if c.lastChunkFid != "" {
c.readerCache.UnCache(c.lastChunkFid) c.readerCache.UnCache(c.lastChunkFid)
c.readerCache.MaybeCache(nextChunkViews)
} else {
if len(nextChunkViews) >= 1 {
c.readerCache.MaybeCache(nextChunkViews[:1]) // just read the next chunk if at the very beginning
}
}
if nextChunkViews != nil {
c.readerCache.MaybeCache(nextChunkViews) // just read the next chunk if at the very beginning
} }
} }
} }

142
weed/filer/reader_at_test.go

@ -5,7 +5,6 @@ import (
"io" "io"
"math" "math"
"strconv" "strconv"
"sync"
"testing" "testing"
) )
@ -34,42 +33,40 @@ func (m *mockChunkCache) SetChunk(fileId string, data []byte) {
func TestReaderAt(t *testing.T) { func TestReaderAt(t *testing.T) {
visibles := []VisibleInterval{
{
start: 1,
stop: 2,
fileId: "1",
chunkSize: 9,
},
{
start: 3,
stop: 4,
fileId: "3",
chunkSize: 1,
},
{
start: 5,
stop: 6,
fileId: "5",
chunkSize: 2,
},
{
start: 7,
stop: 9,
fileId: "7",
chunkSize: 2,
},
{
start: 9,
stop: 10,
fileId: "9",
chunkSize: 2,
},
}
visibles := NewIntervalList[*VisibleInterval]()
addVisibleInterval(visibles, &VisibleInterval{
start: 1,
stop: 2,
fileId: "1",
chunkSize: 9,
})
addVisibleInterval(visibles, &VisibleInterval{
start: 3,
stop: 4,
fileId: "3",
chunkSize: 1,
})
addVisibleInterval(visibles, &VisibleInterval{
start: 5,
stop: 6,
fileId: "5",
chunkSize: 2,
})
addVisibleInterval(visibles, &VisibleInterval{
start: 7,
stop: 9,
fileId: "7",
chunkSize: 2,
})
addVisibleInterval(visibles, &VisibleInterval{
start: 9,
stop: 10,
fileId: "9",
chunkSize: 2,
})
readerAt := &ChunkReadAt{ readerAt := &ChunkReadAt{
chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64), chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64),
readerLock: sync.Mutex{},
fileSize: 10, fileSize: 10,
readerCache: newReaderCache(3, &mockChunkCache{}, nil), readerCache: newReaderCache(3, &mockChunkCache{}, nil),
readerPattern: NewReaderPattern(), readerPattern: NewReaderPattern(),
@ -86,7 +83,7 @@ func testReadAt(t *testing.T, readerAt *ChunkReadAt, offset int64, size int, exp
if data == nil { if data == nil {
data = make([]byte, size) data = make([]byte, size)
} }
n, err := readerAt.doReadAt(data, offset)
n, _, err := readerAt.doReadAt(data, offset)
if expectedN != n { if expectedN != n {
t.Errorf("unexpected read size: %d, expect: %d", n, expectedN) t.Errorf("unexpected read size: %d, expect: %d", n, expectedN)
@ -101,24 +98,22 @@ func testReadAt(t *testing.T, readerAt *ChunkReadAt, offset int64, size int, exp
func TestReaderAt0(t *testing.T) { func TestReaderAt0(t *testing.T) {
visibles := []VisibleInterval{
{
start: 2,
stop: 5,
fileId: "1",
chunkSize: 9,
},
{
start: 7,
stop: 9,
fileId: "2",
chunkSize: 9,
},
}
visibles := NewIntervalList[*VisibleInterval]()
addVisibleInterval(visibles, &VisibleInterval{
start: 2,
stop: 5,
fileId: "1",
chunkSize: 9,
})
addVisibleInterval(visibles, &VisibleInterval{
start: 7,
stop: 9,
fileId: "2",
chunkSize: 9,
})
readerAt := &ChunkReadAt{ readerAt := &ChunkReadAt{
chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64), chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64),
readerLock: sync.Mutex{},
fileSize: 10, fileSize: 10,
readerCache: newReaderCache(3, &mockChunkCache{}, nil), readerCache: newReaderCache(3, &mockChunkCache{}, nil),
readerPattern: NewReaderPattern(), readerPattern: NewReaderPattern(),
@ -135,18 +130,16 @@ func TestReaderAt0(t *testing.T) {
func TestReaderAt1(t *testing.T) { func TestReaderAt1(t *testing.T) {
visibles := []VisibleInterval{
{
start: 2,
stop: 5,
fileId: "1",
chunkSize: 9,
},
}
visibles := NewIntervalList[*VisibleInterval]()
addVisibleInterval(visibles, &VisibleInterval{
start: 2,
stop: 5,
fileId: "1",
chunkSize: 9,
})
readerAt := &ChunkReadAt{ readerAt := &ChunkReadAt{
chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64), chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64),
readerLock: sync.Mutex{},
fileSize: 20, fileSize: 20,
readerCache: newReaderCache(3, &mockChunkCache{}, nil), readerCache: newReaderCache(3, &mockChunkCache{}, nil),
readerPattern: NewReaderPattern(), readerPattern: NewReaderPattern(),
@ -164,24 +157,22 @@ func TestReaderAt1(t *testing.T) {
} }
func TestReaderAtGappedChunksDoNotLeak(t *testing.T) { func TestReaderAtGappedChunksDoNotLeak(t *testing.T) {
visibles := []VisibleInterval{
{
start: 2,
stop: 3,
fileId: "1",
chunkSize: 5,
},
{
start: 7,
stop: 9,
fileId: "1",
chunkSize: 4,
},
}
visibles := NewIntervalList[*VisibleInterval]()
addVisibleInterval(visibles, &VisibleInterval{
start: 2,
stop: 3,
fileId: "1",
chunkSize: 5,
})
addVisibleInterval(visibles, &VisibleInterval{
start: 7,
stop: 9,
fileId: "1",
chunkSize: 4,
})
readerAt := &ChunkReadAt{ readerAt := &ChunkReadAt{
chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64), chunkViews: ViewFromVisibleIntervals(visibles, 0, math.MaxInt64),
readerLock: sync.Mutex{},
fileSize: 9, fileSize: 9,
readerCache: newReaderCache(3, &mockChunkCache{}, nil), readerCache: newReaderCache(3, &mockChunkCache{}, nil),
readerPattern: NewReaderPattern(), readerPattern: NewReaderPattern(),
@ -193,8 +184,7 @@ func TestReaderAtGappedChunksDoNotLeak(t *testing.T) {
func TestReaderAtSparseFileDoesNotLeak(t *testing.T) { func TestReaderAtSparseFileDoesNotLeak(t *testing.T) {
readerAt := &ChunkReadAt{ readerAt := &ChunkReadAt{
chunkViews: ViewFromVisibleIntervals([]VisibleInterval{}, 0, math.MaxInt64),
readerLock: sync.Mutex{},
chunkViews: ViewFromVisibleIntervals(NewIntervalList[*VisibleInterval](), 0, math.MaxInt64),
fileSize: 3, fileSize: 3,
readerCache: newReaderCache(3, &mockChunkCache{}, nil), readerCache: newReaderCache(3, &mockChunkCache{}, nil),
readerPattern: NewReaderPattern(), readerPattern: NewReaderPattern(),

7
weed/filer/reader_cache.go

@ -43,7 +43,7 @@ func newReaderCache(limit int, chunkCache chunk_cache.ChunkCache, lookupFileIdFn
} }
} }
func (rc *ReaderCache) MaybeCache(chunkViews []*ChunkView) {
func (rc *ReaderCache) MaybeCache(chunkViews *Interval[*ChunkView]) {
if rc.lookupFileIdFn == nil { if rc.lookupFileIdFn == nil {
return return
} }
@ -55,7 +55,8 @@ func (rc *ReaderCache) MaybeCache(chunkViews []*ChunkView) {
return return
} }
for _, chunkView := range chunkViews {
for x := chunkViews; x != nil; x = x.Next {
chunkView := x.Value
if _, found := rc.downloaders[chunkView.FileId]; found { if _, found := rc.downloaders[chunkView.FileId]; found {
continue continue
} }
@ -65,7 +66,7 @@ func (rc *ReaderCache) MaybeCache(chunkViews []*ChunkView) {
return return
} }
// glog.V(4).Infof("prefetch %s offset %d", chunkView.FileId, chunkView.LogicOffset)
// glog.V(4).Infof("prefetch %s offset %d", chunkView.FileId, chunkView.ViewOffset)
// cache this chunk if not yet // cache this chunk if not yet
cacher := newSingleChunkCacher(rc, chunkView.FileId, chunkView.CipherKey, chunkView.IsGzipped, int(chunkView.ChunkSize), false) cacher := newSingleChunkCacher(rc, chunkView.FileId, chunkView.CipherKey, chunkView.IsGzipped, int(chunkView.ChunkSize), false)
go cacher.startCaching() go cacher.startCaching()

90
weed/filer/stream.go

@ -6,7 +6,6 @@ import (
"golang.org/x/exp/slices" "golang.org/x/exp/slices"
"io" "io"
"math" "math"
"sort"
"strings" "strings"
"sync" "sync"
"time" "time"
@ -78,7 +77,8 @@ func StreamContentWithThrottler(masterClient wdclient.HasLookupFileIdFunction, w
fileId2Url := make(map[string][]string) fileId2Url := make(map[string][]string)
for _, chunkView := range chunkViews {
for x := chunkViews.Front(); x != nil; x = x.Next {
chunkView := x.Value
var urlStrings []string var urlStrings []string
var err error var err error
for _, backoff := range getLookupFileIdBackoffSchedule { for _, backoff := range getLookupFileIdBackoffSchedule {
@ -102,29 +102,30 @@ func StreamContentWithThrottler(masterClient wdclient.HasLookupFileIdFunction, w
downloadThrottler := util.NewWriteThrottler(downloadMaxBytesPs) downloadThrottler := util.NewWriteThrottler(downloadMaxBytesPs)
remaining := size remaining := size
for _, chunkView := range chunkViews {
if offset < chunkView.LogicOffset {
gap := chunkView.LogicOffset - offset
for x := chunkViews.Front(); x != nil; x = x.Next {
chunkView := x.Value
if offset < chunkView.ViewOffset {
gap := chunkView.ViewOffset - offset
remaining -= gap remaining -= gap
glog.V(4).Infof("zero [%d,%d)", offset, chunkView.LogicOffset)
glog.V(4).Infof("zero [%d,%d)", offset, chunkView.ViewOffset)
err := writeZero(writer, gap) err := writeZero(writer, gap)
if err != nil { if err != nil {
return fmt.Errorf("write zero [%d,%d)", offset, chunkView.LogicOffset)
return fmt.Errorf("write zero [%d,%d)", offset, chunkView.ViewOffset)
} }
offset = chunkView.LogicOffset
offset = chunkView.ViewOffset
} }
urlStrings := fileId2Url[chunkView.FileId] urlStrings := fileId2Url[chunkView.FileId]
start := time.Now() start := time.Now()
err := retriedStreamFetchChunkData(writer, urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.Offset, int(chunkView.Size))
offset += int64(chunkView.Size)
remaining -= int64(chunkView.Size)
err := retriedStreamFetchChunkData(writer, urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk, int(chunkView.ViewSize))
offset += int64(chunkView.ViewSize)
remaining -= int64(chunkView.ViewSize)
stats.FilerRequestHistogram.WithLabelValues("chunkDownload").Observe(time.Since(start).Seconds()) stats.FilerRequestHistogram.WithLabelValues("chunkDownload").Observe(time.Since(start).Seconds())
if err != nil { if err != nil {
stats.FilerRequestCounter.WithLabelValues("chunkDownloadError").Inc() stats.FilerRequestCounter.WithLabelValues("chunkDownloadError").Inc()
return fmt.Errorf("read chunk: %v", err) return fmt.Errorf("read chunk: %v", err)
} }
stats.FilerRequestCounter.WithLabelValues("chunkDownload").Inc() stats.FilerRequestCounter.WithLabelValues("chunkDownload").Inc()
downloadThrottler.MaybeSlowdown(int64(chunkView.Size))
downloadThrottler.MaybeSlowdown(int64(chunkView.ViewSize))
} }
if remaining > 0 { if remaining > 0 {
glog.V(4).Infof("zero [%d,%d)", offset, offset+remaining) glog.V(4).Infof("zero [%d,%d)", offset, offset+remaining)
@ -167,14 +168,15 @@ func ReadAll(buffer []byte, masterClient *wdclient.MasterClient, chunks []*filer
idx := 0 idx := 0
for _, chunkView := range chunkViews {
for x := chunkViews.Front(); x != nil; x = x.Next {
chunkView := x.Value
urlStrings, err := lookupFileIdFn(chunkView.FileId) urlStrings, err := lookupFileIdFn(chunkView.FileId)
if err != nil { if err != nil {
glog.V(1).Infof("operation LookupFileId %s failed, err: %v", chunkView.FileId, err) glog.V(1).Infof("operation LookupFileId %s failed, err: %v", chunkView.FileId, err)
return err return err
} }
n, err := retriedFetchChunkData(buffer[idx:idx+int(chunkView.Size)], urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.Offset)
n, err := retriedFetchChunkData(buffer[idx:idx+int(chunkView.ViewSize)], urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk)
if err != nil { if err != nil {
return err return err
} }
@ -185,7 +187,7 @@ func ReadAll(buffer []byte, masterClient *wdclient.MasterClient, chunks []*filer
// ---------------- ChunkStreamReader ---------------------------------- // ---------------- ChunkStreamReader ----------------------------------
type ChunkStreamReader struct { type ChunkStreamReader struct {
chunkViews []*ChunkView
chunkView *Interval[*ChunkView]
totalSize int64 totalSize int64
logicOffset int64 logicOffset int64
buffer []byte buffer []byte
@ -201,17 +203,15 @@ var _ = io.ReaderAt(&ChunkStreamReader{})
func doNewChunkStreamReader(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk) *ChunkStreamReader { func doNewChunkStreamReader(lookupFileIdFn wdclient.LookupFileIdFunctionType, chunks []*filer_pb.FileChunk) *ChunkStreamReader {
chunkViews := ViewFromChunks(lookupFileIdFn, chunks, 0, math.MaxInt64) chunkViews := ViewFromChunks(lookupFileIdFn, chunks, 0, math.MaxInt64)
slices.SortFunc(chunkViews, func(a, b *ChunkView) bool {
return a.LogicOffset < b.LogicOffset
})
var totalSize int64 var totalSize int64
for _, chunk := range chunkViews {
totalSize += int64(chunk.Size)
for x := chunkViews.Front(); x != nil; x = x.Next {
chunk := x.Value
totalSize += int64(chunk.ViewSize)
} }
return &ChunkStreamReader{ return &ChunkStreamReader{
chunkViews: chunkViews,
chunkView: chunkViews.Front(),
lookupFileId: lookupFileIdFn, lookupFileId: lookupFileIdFn,
totalSize: totalSize, totalSize: totalSize,
} }
@ -290,7 +290,7 @@ func (c *ChunkStreamReader) Seek(offset int64, whence int) (int64, error) {
} }
func insideChunk(offset int64, chunk *ChunkView) bool { func insideChunk(offset int64, chunk *ChunkView) bool {
return chunk.LogicOffset <= offset && offset < chunk.LogicOffset+int64(chunk.Size)
return chunk.ViewOffset <= offset && offset < chunk.ViewOffset+int64(chunk.ViewSize)
} }
func (c *ChunkStreamReader) prepareBufferFor(offset int64) (err error) { func (c *ChunkStreamReader) prepareBufferFor(offset int64) (err error) {
@ -300,48 +300,22 @@ func (c *ChunkStreamReader) prepareBufferFor(offset int64) (err error) {
} }
// fmt.Printf("fetch for offset %d\n", offset) // fmt.Printf("fetch for offset %d\n", offset)
// need to seek to a different chunk
currentChunkIndex := sort.Search(len(c.chunkViews), func(i int) bool {
return offset < c.chunkViews[i].LogicOffset
})
if currentChunkIndex == len(c.chunkViews) {
// not found
if insideChunk(offset, c.chunkViews[0]) {
// fmt.Printf("select0 chunk %d %s\n", currentChunkIndex, c.chunkViews[currentChunkIndex].FileId)
currentChunkIndex = 0
} else if insideChunk(offset, c.chunkViews[len(c.chunkViews)-1]) {
currentChunkIndex = len(c.chunkViews) - 1
// fmt.Printf("select last chunk %d %s\n", currentChunkIndex, c.chunkViews[currentChunkIndex].FileId)
} else {
return io.EOF
}
} else if currentChunkIndex > 0 {
if insideChunk(offset, c.chunkViews[currentChunkIndex]) {
// good hit
} else if insideChunk(offset, c.chunkViews[currentChunkIndex-1]) {
currentChunkIndex -= 1
// fmt.Printf("select -1 chunk %d %s\n", currentChunkIndex, c.chunkViews[currentChunkIndex].FileId)
} else {
// glog.Fatalf("unexpected1 offset %d", offset)
return fmt.Errorf("unexpected1 offset %d", offset)
}
} else {
// glog.Fatalf("unexpected2 offset %d", offset)
return fmt.Errorf("unexpected2 offset %d", offset)
c.chunkView = c.chunkView.Next
if c.chunkView == nil {
return io.EOF
} }
// positioning within the new chunk // positioning within the new chunk
chunk := c.chunkViews[currentChunkIndex]
chunk := c.chunkView.Value
if insideChunk(offset, chunk) { if insideChunk(offset, chunk) {
if c.isBufferEmpty() || c.bufferOffset != chunk.LogicOffset {
if c.isBufferEmpty() || c.bufferOffset != chunk.ViewOffset {
if err = c.fetchChunkToBuffer(chunk); err != nil { if err = c.fetchChunkToBuffer(chunk); err != nil {
return return
} }
} }
} else { } else {
// glog.Fatalf("unexpected3 offset %d in %s [%d,%d)", offset, chunk.FileId, chunk.LogicOffset, chunk.LogicOffset+int64(chunk.Size))
return fmt.Errorf("unexpected3 offset %d in %s [%d,%d)", offset, chunk.FileId, chunk.LogicOffset, chunk.LogicOffset+int64(chunk.Size))
// glog.Fatalf("unexpected3 offset %d in %s [%d,%d)", offset, chunk.FileId, chunk.ViewOffset, chunk.ViewOffset+int64(chunk.ViewSize))
return fmt.Errorf("unexpected3 offset %d in %s [%d,%d)", offset, chunk.FileId, chunk.ViewOffset, chunk.ViewOffset+int64(chunk.ViewSize))
} }
return return
} }
@ -355,7 +329,7 @@ func (c *ChunkStreamReader) fetchChunkToBuffer(chunkView *ChunkView) error {
var buffer bytes.Buffer var buffer bytes.Buffer
var shouldRetry bool var shouldRetry bool
for _, urlString := range urlStrings { for _, urlString := range urlStrings {
shouldRetry, err = util.ReadUrlAsStream(urlString+"?readDeleted=true", chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.Offset, int(chunkView.Size), func(data []byte) {
shouldRetry, err = util.ReadUrlAsStream(urlString+"?readDeleted=true", chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk, int(chunkView.ViewSize), func(data []byte) {
buffer.Write(data) buffer.Write(data)
}) })
if !shouldRetry { if !shouldRetry {
@ -372,10 +346,10 @@ func (c *ChunkStreamReader) fetchChunkToBuffer(chunkView *ChunkView) error {
return err return err
} }
c.buffer = buffer.Bytes() c.buffer = buffer.Bytes()
c.bufferOffset = chunkView.LogicOffset
c.bufferOffset = chunkView.ViewOffset
c.chunk = chunkView.FileId c.chunk = chunkView.FileId
// glog.V(0).Infof("fetched %s [%d,%d)", chunkView.FileId, chunkView.LogicOffset, chunkView.LogicOffset+int64(chunkView.Size))
// glog.V(0).Infof("fetched %s [%d,%d)", chunkView.FileId, chunkView.ViewOffset, chunkView.ViewOffset+int64(chunkView.ViewSize))
return nil return nil
} }

16
weed/mount/dirty_pages_chunked.go

@ -7,7 +7,6 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"io" "io"
"sync" "sync"
"time"
) )
type ChunkedDirtyPages struct { type ChunkedDirtyPages struct {
@ -38,11 +37,11 @@ func newMemoryChunkPages(fh *FileHandle, chunkSize int64) *ChunkedDirtyPages {
return dirtyPages return dirtyPages
} }
func (pages *ChunkedDirtyPages) AddPage(offset int64, data []byte, isSequential bool) {
func (pages *ChunkedDirtyPages) AddPage(offset int64, data []byte, isSequential bool, tsNs int64) {
pages.hasWrites = true pages.hasWrites = true
glog.V(4).Infof("%v memory AddPage [%d, %d)", pages.fh.fh, offset, offset+int64(len(data))) glog.V(4).Infof("%v memory AddPage [%d, %d)", pages.fh.fh, offset, offset+int64(len(data)))
pages.uploadPipeline.SaveDataAt(data, offset, isSequential)
pages.uploadPipeline.SaveDataAt(data, offset, isSequential, tsNs)
return return
} }
@ -58,28 +57,27 @@ func (pages *ChunkedDirtyPages) FlushData() error {
return nil return nil
} }
func (pages *ChunkedDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64) {
func (pages *ChunkedDirtyPages) ReadDirtyDataAt(data []byte, startOffset int64, tsNs int64) (maxStop int64) {
if !pages.hasWrites { if !pages.hasWrites {
return return
} }
return pages.uploadPipeline.MaybeReadDataAt(data, startOffset)
return pages.uploadPipeline.MaybeReadDataAt(data, startOffset, tsNs)
} }
func (pages *ChunkedDirtyPages) saveChunkedFileIntervalToStorage(reader io.Reader, offset int64, size int64, cleanupFn func()) {
func (pages *ChunkedDirtyPages) saveChunkedFileIntervalToStorage(reader io.Reader, offset int64, size int64, modifiedTsNs int64, cleanupFn func()) {
mtime := time.Now().UnixNano()
defer cleanupFn() defer cleanupFn()
fileFullPath := pages.fh.FullPath() fileFullPath := pages.fh.FullPath()
fileName := fileFullPath.Name() fileName := fileFullPath.Name()
chunk, err := pages.fh.wfs.saveDataAsChunk(fileFullPath)(reader, fileName, offset)
chunk, err := pages.fh.wfs.saveDataAsChunk(fileFullPath)(reader, fileName, offset, modifiedTsNs)
if err != nil { if err != nil {
glog.V(0).Infof("%v saveToStorage [%d,%d): %v", fileFullPath, offset, offset+size, err) glog.V(0).Infof("%v saveToStorage [%d,%d): %v", fileFullPath, offset, offset+size, err)
pages.lastErr = err pages.lastErr = err
return return
} }
chunk.ModifiedTsNs = mtime
pages.fh.AddChunks([]*filer_pb.FileChunk{chunk}) pages.fh.AddChunks([]*filer_pb.FileChunk{chunk})
pages.fh.entryChunkGroup.AddChunk(chunk)
glog.V(3).Infof("%v saveToStorage %s [%d,%d)", fileFullPath, chunk.FileId, offset, offset+size) glog.V(3).Infof("%v saveToStorage %s [%d,%d)", fileFullPath, chunk.FileId, offset, offset+size)
} }

109
weed/mount/filehandle.go

@ -5,50 +5,60 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb" "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util" "github.com/seaweedfs/seaweedfs/weed/util"
"golang.org/x/exp/slices"
"golang.org/x/sync/semaphore"
"math"
"os"
"sync" "sync"
) )
type FileHandleId uint64 type FileHandleId uint64
var IsDebugFileReadWrite = false
type FileHandle struct { type FileHandle struct {
fh FileHandleId
counter int64
entry *LockedEntry
entryLock sync.Mutex
inode uint64
wfs *WFS
fh FileHandleId
counter int64
entry *LockedEntry
entryLock sync.Mutex
entryChunkGroup *filer.ChunkGroup
inode uint64
wfs *WFS
// cache file has been written to // cache file has been written to
dirtyMetadata bool
dirtyPages *PageWriter
entryViewCache []filer.VisibleInterval
reader *filer.ChunkReadAt
contentType string
handle uint64
orderedMutex *semaphore.Weighted
dirtyMetadata bool
dirtyPages *PageWriter
reader *filer.ChunkReadAt
contentType string
handle uint64
sync.Mutex
isDeleted bool isDeleted bool
// for debugging
mirrorFile *os.File
} }
func newFileHandle(wfs *WFS, handleId FileHandleId, inode uint64, entry *filer_pb.Entry) *FileHandle { func newFileHandle(wfs *WFS, handleId FileHandleId, inode uint64, entry *filer_pb.Entry) *FileHandle {
fh := &FileHandle{ fh := &FileHandle{
fh: handleId,
counter: 1,
inode: inode,
wfs: wfs,
orderedMutex: semaphore.NewWeighted(int64(math.MaxInt64)),
fh: handleId,
counter: 1,
inode: inode,
wfs: wfs,
} }
// dirtyPages: newContinuousDirtyPages(file, writeOnly), // dirtyPages: newContinuousDirtyPages(file, writeOnly),
fh.dirtyPages = newPageWriter(fh, wfs.option.ChunkSizeLimit) fh.dirtyPages = newPageWriter(fh, wfs.option.ChunkSizeLimit)
if entry != nil {
entry.Attributes.FileSize = filer.FileSize(entry)
}
fh.entry = &LockedEntry{ fh.entry = &LockedEntry{
Entry: entry, Entry: entry,
} }
if entry != nil {
fh.SetEntry(entry)
}
if IsDebugFileReadWrite {
var err error
fh.mirrorFile, err = os.OpenFile("/tmp/sw/"+entry.Name, os.O_RDWR|os.O_CREATE, 0600)
if err != nil {
println("failed to create mirror:", err.Error())
}
}
return fh return fh
} }
@ -63,6 +73,17 @@ func (fh *FileHandle) GetEntry() *filer_pb.Entry {
} }
func (fh *FileHandle) SetEntry(entry *filer_pb.Entry) { func (fh *FileHandle) SetEntry(entry *filer_pb.Entry) {
if entry != nil {
fileSize := filer.FileSize(entry)
entry.Attributes.FileSize = fileSize
var resolveManifestErr error
fh.entryChunkGroup, resolveManifestErr = filer.NewChunkGroup(fh.wfs.LookupFn(), fh.wfs.chunkCache, entry.Chunks)
if resolveManifestErr != nil {
glog.Warningf("failed to resolve manifest chunks in %+v", entry)
}
} else {
glog.Fatalf("setting file handle entry to nil")
}
fh.entry.SetEntry(entry) fh.entry.SetEntry(entry)
} }
@ -78,43 +99,17 @@ func (fh *FileHandle) AddChunks(chunks []*filer_pb.FileChunk) {
return return
} }
// find the earliest incoming chunk
newChunks := chunks
earliestChunk := newChunks[0]
for i := 1; i < len(newChunks); i++ {
if lessThan(earliestChunk, newChunks[i]) {
earliestChunk = newChunks[i]
}
}
// pick out-of-order chunks from existing chunks
for _, chunk := range fh.entry.GetChunks() {
if lessThan(earliestChunk, chunk) {
chunks = append(chunks, chunk)
}
}
// sort incoming chunks
slices.SortFunc(chunks, func(a, b *filer_pb.FileChunk) bool {
return lessThan(a, b)
})
glog.V(4).Infof("%s existing %d chunks adds %d more", fh.FullPath(), len(fh.entry.GetChunks()), len(chunks))
fh.entry.AppendChunks(newChunks)
fh.entryViewCache = nil
fh.entry.AppendChunks(chunks)
} }
func (fh *FileHandle) CloseReader() {
if fh.reader != nil {
_ = fh.reader.Close()
fh.reader = nil
}
}
func (fh *FileHandle) ReleaseHandle() {
fh.entryLock.Lock()
defer fh.entryLock.Unlock()
func (fh *FileHandle) Release() {
fh.dirtyPages.Destroy() fh.dirtyPages.Destroy()
fh.CloseReader()
if IsDebugFileReadWrite {
fh.mirrorFile.Close()
}
} }
func lessThan(a, b *filer_pb.FileChunk) bool { func lessThan(a, b *filer_pb.FileChunk) bool {

4
weed/mount/filehandle_map.go

@ -65,7 +65,7 @@ func (i *FileHandleToInode) ReleaseByInode(inode uint64) {
if fh.counter <= 0 { if fh.counter <= 0 {
delete(i.inode2fh, inode) delete(i.inode2fh, inode)
delete(i.fh2inode, fh.fh) delete(i.fh2inode, fh.fh)
fh.Release()
fh.ReleaseHandle()
} }
} }
} }
@ -82,7 +82,7 @@ func (i *FileHandleToInode) ReleaseByHandle(fh FileHandleId) {
if fhHandle.counter <= 0 { if fhHandle.counter <= 0 {
delete(i.inode2fh, inode) delete(i.inode2fh, inode)
delete(i.fh2inode, fhHandle.fh) delete(i.fh2inode, fhHandle.fh)
fhHandle.Release()
fhHandle.ReleaseHandle()
} }
} }

43
weed/mount/filehandle_read.go

@ -17,18 +17,20 @@ func (fh *FileHandle) unlockForRead(startOffset int64, size int) {
fh.dirtyPages.UnlockForRead(startOffset, startOffset+int64(size)) fh.dirtyPages.UnlockForRead(startOffset, startOffset+int64(size))
} }
func (fh *FileHandle) readFromDirtyPages(buff []byte, startOffset int64) (maxStop int64) {
maxStop = fh.dirtyPages.ReadDirtyDataAt(buff, startOffset)
func (fh *FileHandle) readFromDirtyPages(buff []byte, startOffset int64, tsNs int64) (maxStop int64) {
maxStop = fh.dirtyPages.ReadDirtyDataAt(buff, startOffset, tsNs)
return return
} }
func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, int64, error) {
fh.entryLock.Lock()
defer fh.entryLock.Unlock()
fileFullPath := fh.FullPath() fileFullPath := fh.FullPath()
entry := fh.GetEntry() entry := fh.GetEntry()
if entry == nil { if entry == nil {
return 0, io.EOF
return 0, 0, io.EOF
} }
if entry.IsInRemoteOnly() { if entry.IsInRemoteOnly() {
@ -36,43 +38,28 @@ func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
newEntry, err := fh.downloadRemoteEntry(entry) newEntry, err := fh.downloadRemoteEntry(entry)
if err != nil { if err != nil {
glog.V(1).Infof("download remote entry %s: %v", fileFullPath, err) glog.V(1).Infof("download remote entry %s: %v", fileFullPath, err)
return 0, err
return 0, 0, err
} }
entry = newEntry entry = newEntry
} }
fileSize := int64(filer.FileSize(entry))
fileSize := int64(entry.Attributes.FileSize)
if fileSize == 0 {
fileSize = int64(filer.FileSize(entry))
}
if fileSize == 0 { if fileSize == 0 {
glog.V(1).Infof("empty fh %v", fileFullPath) glog.V(1).Infof("empty fh %v", fileFullPath)
return 0, io.EOF
return 0, 0, io.EOF
} }
if offset+int64(len(buff)) <= int64(len(entry.Content)) { if offset+int64(len(buff)) <= int64(len(entry.Content)) {
totalRead := copy(buff, entry.Content[offset:]) totalRead := copy(buff, entry.Content[offset:])
glog.V(4).Infof("file handle read cached %s [%d,%d] %d", fileFullPath, offset, offset+int64(totalRead), totalRead) glog.V(4).Infof("file handle read cached %s [%d,%d] %d", fileFullPath, offset, offset+int64(totalRead), totalRead)
return int64(totalRead), nil
}
var chunkResolveErr error
if fh.entryViewCache == nil {
fh.entryViewCache, chunkResolveErr = filer.NonOverlappingVisibleIntervals(fh.wfs.LookupFn(), entry.GetChunks(), 0, fileSize)
if chunkResolveErr != nil {
return 0, fmt.Errorf("fail to resolve chunk manifest: %v", chunkResolveErr)
}
fh.CloseReader()
}
if fh.reader == nil {
chunkViews := filer.ViewFromVisibleIntervals(fh.entryViewCache, 0, fileSize)
glog.V(4).Infof("file handle read %s [%d,%d) from %d views", fileFullPath, offset, offset+int64(len(buff)), len(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)
}
fh.reader = filer.NewChunkReaderAtFromClient(fh.wfs.LookupFn(), chunkViews, fh.wfs.chunkCache, fileSize)
return int64(totalRead), 0, nil
} }
totalRead, err := fh.reader.ReadAt(buff, offset)
totalRead, ts, err := fh.entryChunkGroup.ReadDataAt(fileSize, buff, offset)
if err != nil && err != io.EOF { if err != nil && err != io.EOF {
glog.Errorf("file handle read %s: %v", fileFullPath, err) glog.Errorf("file handle read %s: %v", fileFullPath, err)
@ -80,7 +67,7 @@ func (fh *FileHandle) readFromChunks(buff []byte, offset int64) (int64, error) {
// 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
return int64(totalRead), ts, err
} }
func (fh *FileHandle) downloadRemoteEntry(entry *filer_pb.Entry) (*filer_pb.Entry, error) { func (fh *FileHandle) downloadRemoteEntry(entry *filer_pb.Entry) (*filer_pb.Entry, error) {

12
weed/mount/page_writer.go

@ -29,35 +29,35 @@ func newPageWriter(fh *FileHandle, chunkSize int64) *PageWriter {
return pw return pw
} }
func (pw *PageWriter) AddPage(offset int64, data []byte, isSequential bool) {
func (pw *PageWriter) AddPage(offset int64, data []byte, isSequential bool, tsNs int64) {
glog.V(4).Infof("%v AddPage [%d, %d)", pw.fh.fh, offset, offset+int64(len(data))) glog.V(4).Infof("%v AddPage [%d, %d)", pw.fh.fh, offset, offset+int64(len(data)))
chunkIndex := offset / pw.chunkSize chunkIndex := offset / pw.chunkSize
for i := chunkIndex; len(data) > 0; i++ { for i := chunkIndex; len(data) > 0; i++ {
writeSize := min(int64(len(data)), (i+1)*pw.chunkSize-offset) writeSize := min(int64(len(data)), (i+1)*pw.chunkSize-offset)
pw.addToOneChunk(i, offset, data[:writeSize], isSequential)
pw.addToOneChunk(i, offset, data[:writeSize], isSequential, tsNs)
offset += writeSize offset += writeSize
data = data[writeSize:] data = data[writeSize:]
} }
} }
func (pw *PageWriter) addToOneChunk(chunkIndex, offset int64, data []byte, isSequential bool) {
pw.randomWriter.AddPage(offset, data, isSequential)
func (pw *PageWriter) addToOneChunk(chunkIndex, offset int64, data []byte, isSequential bool, tsNs int64) {
pw.randomWriter.AddPage(offset, data, isSequential, tsNs)
} }
func (pw *PageWriter) FlushData() error { func (pw *PageWriter) FlushData() error {
return pw.randomWriter.FlushData() return pw.randomWriter.FlushData()
} }
func (pw *PageWriter) ReadDirtyDataAt(data []byte, offset int64) (maxStop int64) {
func (pw *PageWriter) ReadDirtyDataAt(data []byte, offset int64, tsNs int64) (maxStop int64) {
glog.V(4).Infof("ReadDirtyDataAt %v [%d, %d)", pw.fh.fh, offset, offset+int64(len(data))) glog.V(4).Infof("ReadDirtyDataAt %v [%d, %d)", pw.fh.fh, offset, offset+int64(len(data)))
chunkIndex := offset / pw.chunkSize chunkIndex := offset / pw.chunkSize
for i := chunkIndex; len(data) > 0; i++ { for i := chunkIndex; len(data) > 0; i++ {
readSize := min(int64(len(data)), (i+1)*pw.chunkSize-offset) readSize := min(int64(len(data)), (i+1)*pw.chunkSize-offset)
maxStop = pw.randomWriter.ReadDirtyDataAt(data[:readSize], offset)
maxStop = pw.randomWriter.ReadDirtyDataAt(data[:readSize], offset, tsNs)
offset += readSize offset += readSize
data = data[readSize:] data = data[readSize:]

39
weed/mount/page_writer/activity_score.go

@ -0,0 +1,39 @@
package page_writer
import "time"
type ActivityScore struct {
lastActiveTsNs int64
decayedActivenessScore int64
}
func NewActivityScore() *ActivityScore {
return &ActivityScore{}
}
func (as ActivityScore) MarkRead() {
now := time.Now().UnixNano()
deltaTime := (now - as.lastActiveTsNs) >> 30 // about number of seconds
as.lastActiveTsNs = now
as.decayedActivenessScore = as.decayedActivenessScore>>deltaTime + 256
if as.decayedActivenessScore < 0 {
as.decayedActivenessScore = 0
}
}
func (as ActivityScore) MarkWrite() {
now := time.Now().UnixNano()
deltaTime := (now - as.lastActiveTsNs) >> 30 // about number of seconds
as.lastActiveTsNs = now
as.decayedActivenessScore = as.decayedActivenessScore>>deltaTime + 1024
if as.decayedActivenessScore < 0 {
as.decayedActivenessScore = 0
}
}
func (as ActivityScore) ActivityScore() int64 {
deltaTime := (time.Now().UnixNano() - as.lastActiveTsNs) >> 30 // about number of seconds
return as.decayedActivenessScore >> deltaTime
}

83
weed/mount/page_writer/chunk_interval_list.go

@ -8,6 +8,7 @@ import (
type ChunkWrittenInterval struct { type ChunkWrittenInterval struct {
StartOffset int64 StartOffset int64
stopOffset int64 stopOffset int64
TsNs int64
prev *ChunkWrittenInterval prev *ChunkWrittenInterval
next *ChunkWrittenInterval next *ChunkWrittenInterval
} }
@ -42,10 +43,14 @@ func newChunkWrittenIntervalList() *ChunkWrittenIntervalList {
return list return list
} }
func (list *ChunkWrittenIntervalList) MarkWritten(startOffset, stopOffset int64) {
func (list *ChunkWrittenIntervalList) MarkWritten(startOffset, stopOffset, tsNs int64) {
if startOffset >= stopOffset {
return
}
interval := &ChunkWrittenInterval{ interval := &ChunkWrittenInterval{
StartOffset: startOffset, StartOffset: startOffset,
stopOffset: stopOffset, stopOffset: stopOffset,
TsNs: tsNs,
} }
list.addInterval(interval) list.addInterval(interval)
} }
@ -62,50 +67,54 @@ func (list *ChunkWrittenIntervalList) WrittenSize() (writtenByteCount int64) {
func (list *ChunkWrittenIntervalList) addInterval(interval *ChunkWrittenInterval) { func (list *ChunkWrittenIntervalList) addInterval(interval *ChunkWrittenInterval) {
//t := list.head
//for ; t.next != nil; t = t.next {
// if t.TsNs > interval.TsNs {
// println("writes is out of order", t.TsNs-interval.TsNs, "ns")
// }
//}
p := list.head p := list.head
for ; p.next != nil && p.next.StartOffset <= interval.StartOffset; p = p.next {
for ; p.next != nil && p.next.stopOffset <= interval.StartOffset; p = p.next {
} }
q := list.tail q := list.tail
for ; q.prev != nil && q.prev.stopOffset >= interval.stopOffset; q = q.prev {
for ; q.prev != nil && q.prev.StartOffset >= interval.stopOffset; q = q.prev {
} }
if interval.StartOffset <= p.stopOffset && q.StartOffset <= interval.stopOffset {
// merge p and q together
p.stopOffset = q.stopOffset
unlinkNodesBetween(p, q.next)
return
// left side
// interval after p.next start
if p.next.StartOffset < interval.StartOffset {
t := &ChunkWrittenInterval{
StartOffset: p.next.StartOffset,
stopOffset: interval.StartOffset,
TsNs: p.next.TsNs,
}
p.next = t
t.prev = p
t.next = interval
interval.prev = t
} else {
p.next = interval
interval.prev = p
} }
if interval.StartOffset <= p.stopOffset {
// merge new interval into p
p.stopOffset = interval.stopOffset
unlinkNodesBetween(p, q)
return
}
if q.StartOffset <= interval.stopOffset {
// merge new interval into q
q.StartOffset = interval.StartOffset
unlinkNodesBetween(p, q)
return
}
// add the new interval between p and q
unlinkNodesBetween(p, q)
p.next = interval
interval.prev = p
q.prev = interval
interval.next = q
}
// unlinkNodesBetween remove all nodes after start and before stop, exclusive
func unlinkNodesBetween(start *ChunkWrittenInterval, stop *ChunkWrittenInterval) {
if start.next == stop {
return
// right side
// interval ends before p.prev
if interval.stopOffset < q.prev.stopOffset {
t := &ChunkWrittenInterval{
StartOffset: interval.stopOffset,
stopOffset: q.prev.stopOffset,
TsNs: q.prev.TsNs,
}
q.prev = t
t.next = q
interval.next = t
t.prev = interval
} else {
q.prev = interval
interval.next = q
} }
start.next.prev = nil
start.next = stop
stop.prev.next = nil
stop.prev = start
} }
func (list *ChunkWrittenIntervalList) size() int { func (list *ChunkWrittenIntervalList) size() int {

72
weed/mount/page_writer/chunk_interval_list_test.go

@ -10,40 +10,72 @@ func Test_PageChunkWrittenIntervalList(t *testing.T) {
assert.Equal(t, 0, list.size(), "empty list") assert.Equal(t, 0, list.size(), "empty list")
list.MarkWritten(0, 5)
list.MarkWritten(0, 5, 1)
assert.Equal(t, 1, list.size(), "one interval") assert.Equal(t, 1, list.size(), "one interval")
list.MarkWritten(0, 5)
list.MarkWritten(0, 5, 2)
assert.Equal(t, 1, list.size(), "duplicated interval2") assert.Equal(t, 1, list.size(), "duplicated interval2")
list.MarkWritten(95, 100)
list.MarkWritten(95, 100, 3)
assert.Equal(t, 2, list.size(), "two intervals") assert.Equal(t, 2, list.size(), "two intervals")
list.MarkWritten(50, 60)
list.MarkWritten(50, 60, 4)
assert.Equal(t, 3, list.size(), "three intervals") assert.Equal(t, 3, list.size(), "three intervals")
list.MarkWritten(50, 55)
assert.Equal(t, 3, list.size(), "three intervals merge")
list.MarkWritten(50, 55, 5)
assert.Equal(t, 4, list.size(), "three intervals merge")
list.MarkWritten(40, 50)
assert.Equal(t, 3, list.size(), "three intervals grow forward")
list.MarkWritten(40, 50, 6)
assert.Equal(t, 5, list.size(), "three intervals grow forward")
list.MarkWritten(50, 65)
assert.Equal(t, 3, list.size(), "three intervals grow backward")
list.MarkWritten(50, 65, 7)
assert.Equal(t, 4, list.size(), "three intervals grow backward")
list.MarkWritten(70, 80)
assert.Equal(t, 4, list.size(), "four intervals")
list.MarkWritten(70, 80, 8)
assert.Equal(t, 5, list.size(), "four intervals")
list.MarkWritten(60, 70)
assert.Equal(t, 3, list.size(), "three intervals merged")
list.MarkWritten(60, 70, 9)
assert.Equal(t, 6, list.size(), "three intervals merged")
list.MarkWritten(59, 71)
assert.Equal(t, 3, list.size(), "covered three intervals")
list.MarkWritten(59, 71, 10)
assert.Equal(t, 6, list.size(), "covered three intervals")
list.MarkWritten(5, 59)
assert.Equal(t, 2, list.size(), "covered two intervals")
list.MarkWritten(5, 59, 11)
assert.Equal(t, 5, list.size(), "covered two intervals")
list.MarkWritten(70, 99)
assert.Equal(t, 1, list.size(), "covered one intervals")
list.MarkWritten(70, 99, 12)
assert.Equal(t, 5, list.size(), "covered one intervals")
} }
type interval struct {
start int64
stop int64
expected bool
}
func Test_PageChunkWrittenIntervalList1(t *testing.T) {
list := newChunkWrittenIntervalList()
inputs := []interval{
{1, 5, true},
{2, 3, true},
}
for i, input := range inputs {
list.MarkWritten(input.start, input.stop, int64(i)+1)
actual := hasData(list, 0, 4)
if actual != input.expected {
t.Errorf("input [%d,%d) expected %v actual %v", input.start, input.stop, input.expected, actual)
}
}
}
func hasData(usage *ChunkWrittenIntervalList, chunkStartOffset, x int64) bool {
for t := usage.head.next; t != usage.tail; t = t.next {
logicStart := chunkStartOffset + t.StartOffset
logicStop := chunkStartOffset + t.stopOffset
if logicStart <= x && x < logicStop {
return true
}
}
return false
}

4
weed/mount/page_writer/dirty_pages.go

@ -1,9 +1,9 @@
package page_writer package page_writer
type DirtyPages interface { type DirtyPages interface {
AddPage(offset int64, data []byte, isSequential bool)
AddPage(offset int64, data []byte, isSequential bool, tsNs int64)
FlushData() error FlushData() error
ReadDirtyDataAt(data []byte, startOffset int64) (maxStop int64)
ReadDirtyDataAt(data []byte, startOffset int64, tsNs int64) (maxStop int64)
Destroy() Destroy()
LockForRead(startOffset, stopOffset int64) LockForRead(startOffset, stopOffset int64)
UnlockForRead(startOffset, stopOffset int64) UnlockForRead(startOffset, stopOffset int64)

8
weed/mount/page_writer/page_chunk.go

@ -4,13 +4,13 @@ import (
"io" "io"
) )
type SaveToStorageFunc func(reader io.Reader, offset int64, size int64, cleanupFn func())
type SaveToStorageFunc func(reader io.Reader, offset int64, size int64, modifiedTsNs int64, cleanupFn func())
type PageChunk interface { type PageChunk interface {
FreeResource() FreeResource()
WriteDataAt(src []byte, offset int64) (n int)
ReadDataAt(p []byte, off int64) (maxStop int64)
WriteDataAt(src []byte, offset int64, tsNs int64) (n int)
ReadDataAt(p []byte, off int64, tsNs int64) (maxStop int64)
IsComplete() bool IsComplete() bool
WrittenSize() int64
ActivityScore() int64
SaveContent(saveFn SaveToStorageFunc) SaveContent(saveFn SaveToStorageFunc)
} }

31
weed/mount/page_writer/page_chunk_mem.go

@ -19,6 +19,7 @@ type MemChunk struct {
usage *ChunkWrittenIntervalList usage *ChunkWrittenIntervalList
chunkSize int64 chunkSize int64
logicChunkIndex LogicChunkIndex logicChunkIndex LogicChunkIndex
activityScore *ActivityScore
} }
func NewMemChunk(logicChunkIndex LogicChunkIndex, chunkSize int64) *MemChunk { func NewMemChunk(logicChunkIndex LogicChunkIndex, chunkSize int64) *MemChunk {
@ -28,6 +29,7 @@ func NewMemChunk(logicChunkIndex LogicChunkIndex, chunkSize int64) *MemChunk {
chunkSize: chunkSize, chunkSize: chunkSize,
buf: mem.Allocate(int(chunkSize)), buf: mem.Allocate(int(chunkSize)),
usage: newChunkWrittenIntervalList(), usage: newChunkWrittenIntervalList(),
activityScore: NewActivityScore(),
} }
} }
@ -39,29 +41,37 @@ func (mc *MemChunk) FreeResource() {
mem.Free(mc.buf) mem.Free(mc.buf)
} }
func (mc *MemChunk) WriteDataAt(src []byte, offset int64) (n int) {
func (mc *MemChunk) WriteDataAt(src []byte, offset int64, tsNs int64) (n int) {
mc.Lock() mc.Lock()
defer mc.Unlock() defer mc.Unlock()
innerOffset := offset % mc.chunkSize innerOffset := offset % mc.chunkSize
n = copy(mc.buf[innerOffset:], src) n = copy(mc.buf[innerOffset:], src)
mc.usage.MarkWritten(innerOffset, innerOffset+int64(n))
mc.usage.MarkWritten(innerOffset, innerOffset+int64(n), tsNs)
mc.activityScore.MarkWrite()
return return
} }
func (mc *MemChunk) ReadDataAt(p []byte, off int64) (maxStop int64) {
func (mc *MemChunk) ReadDataAt(p []byte, off int64, tsNs int64) (maxStop int64) {
mc.RLock() mc.RLock()
defer mc.RUnlock() defer mc.RUnlock()
memChunkBaseOffset := int64(mc.logicChunkIndex) * mc.chunkSize memChunkBaseOffset := int64(mc.logicChunkIndex) * mc.chunkSize
for t := mc.usage.head.next; t != mc.usage.tail; t = t.next { for t := mc.usage.head.next; t != mc.usage.tail; t = t.next {
logicStart := max(off, int64(mc.logicChunkIndex)*mc.chunkSize+t.StartOffset)
logicStart := max(off, memChunkBaseOffset+t.StartOffset)
logicStop := min(off+int64(len(p)), memChunkBaseOffset+t.stopOffset) logicStop := min(off+int64(len(p)), memChunkBaseOffset+t.stopOffset)
if logicStart < logicStop { if logicStart < logicStop {
copy(p[logicStart-off:logicStop-off], mc.buf[logicStart-memChunkBaseOffset:logicStop-memChunkBaseOffset])
maxStop = max(maxStop, logicStop)
if t.TsNs >= tsNs {
copy(p[logicStart-off:logicStop-off], mc.buf[logicStart-memChunkBaseOffset:logicStop-memChunkBaseOffset])
maxStop = max(maxStop, logicStop)
} else {
println("read old data1", tsNs-t.TsNs, "ns")
}
} }
} }
mc.activityScore.MarkRead()
return return
} }
@ -72,11 +82,8 @@ func (mc *MemChunk) IsComplete() bool {
return mc.usage.IsComplete(mc.chunkSize) return mc.usage.IsComplete(mc.chunkSize)
} }
func (mc *MemChunk) WrittenSize() int64 {
mc.RLock()
defer mc.RUnlock()
return mc.usage.WrittenSize()
func (mc *MemChunk) ActivityScore() int64 {
return mc.activityScore.ActivityScore()
} }
func (mc *MemChunk) SaveContent(saveFn SaveToStorageFunc) { func (mc *MemChunk) SaveContent(saveFn SaveToStorageFunc) {
@ -88,7 +95,7 @@ func (mc *MemChunk) SaveContent(saveFn SaveToStorageFunc) {
} }
for t := mc.usage.head.next; t != mc.usage.tail; t = t.next { for t := mc.usage.head.next; t != mc.usage.tail; t = t.next {
reader := util.NewBytesReader(mc.buf[t.StartOffset:t.stopOffset]) reader := util.NewBytesReader(mc.buf[t.StartOffset:t.stopOffset])
saveFn(reader, int64(mc.logicChunkIndex)*mc.chunkSize+t.StartOffset, t.Size(), func() {
saveFn(reader, int64(mc.logicChunkIndex)*mc.chunkSize+t.StartOffset, t.Size(), t.TsNs, func() {
}) })
} }
} }

125
weed/mount/page_writer/page_chunk_swapfile.go

@ -15,12 +15,12 @@ var (
type ActualChunkIndex int type ActualChunkIndex int
type SwapFile struct { type SwapFile struct {
dir string
file *os.File
logicToActualChunkIndex map[LogicChunkIndex]ActualChunkIndex
logicToActualChunkIndexLock sync.Mutex
chunkSize int64
freeActualChunkList []ActualChunkIndex
dir string
file *os.File
chunkSize int64
chunkTrackingLock sync.Mutex
activeChunkCount int
freeActualChunkList []ActualChunkIndex
} }
type SwapFileChunk struct { type SwapFileChunk struct {
@ -29,14 +29,15 @@ type SwapFileChunk struct {
usage *ChunkWrittenIntervalList usage *ChunkWrittenIntervalList
logicChunkIndex LogicChunkIndex logicChunkIndex LogicChunkIndex
actualChunkIndex ActualChunkIndex actualChunkIndex ActualChunkIndex
activityScore *ActivityScore
//memChunk *MemChunk
} }
func NewSwapFile(dir string, chunkSize int64) *SwapFile { func NewSwapFile(dir string, chunkSize int64) *SwapFile {
return &SwapFile{ return &SwapFile{
dir: dir,
file: nil,
logicToActualChunkIndex: make(map[LogicChunkIndex]ActualChunkIndex),
chunkSize: chunkSize,
dir: dir,
file: nil,
chunkSize: chunkSize,
} }
} }
func (sf *SwapFile) FreeResource() { func (sf *SwapFile) FreeResource() {
@ -46,7 +47,7 @@ func (sf *SwapFile) FreeResource() {
} }
} }
func (sf *SwapFile) NewTempFileChunk(logicChunkIndex LogicChunkIndex) (tc *SwapFileChunk) {
func (sf *SwapFile) NewSwapFileChunk(logicChunkIndex LogicChunkIndex) (tc *SwapFileChunk) {
if sf.file == nil { if sf.file == nil {
var err error var err error
sf.file, err = os.CreateTemp(sf.dir, "") sf.file, err = os.CreateTemp(sf.dir, "")
@ -55,70 +56,98 @@ func (sf *SwapFile) NewTempFileChunk(logicChunkIndex LogicChunkIndex) (tc *SwapF
return nil return nil
} }
} }
sf.logicToActualChunkIndexLock.Lock()
defer sf.logicToActualChunkIndexLock.Unlock()
actualChunkIndex, found := sf.logicToActualChunkIndex[logicChunkIndex]
if !found {
if len(sf.freeActualChunkList) > 0 {
actualChunkIndex = sf.freeActualChunkList[0]
sf.freeActualChunkList = sf.freeActualChunkList[1:]
} else {
actualChunkIndex = ActualChunkIndex(len(sf.logicToActualChunkIndex))
}
sf.logicToActualChunkIndex[logicChunkIndex] = actualChunkIndex
sf.chunkTrackingLock.Lock()
defer sf.chunkTrackingLock.Unlock()
sf.activeChunkCount++
// assign a new physical chunk
var actualChunkIndex ActualChunkIndex
if len(sf.freeActualChunkList) > 0 {
actualChunkIndex = sf.freeActualChunkList[0]
sf.freeActualChunkList = sf.freeActualChunkList[1:]
} else {
actualChunkIndex = ActualChunkIndex(sf.activeChunkCount)
} }
return &SwapFileChunk{
swapFileChunk := &SwapFileChunk{
swapfile: sf, swapfile: sf,
usage: newChunkWrittenIntervalList(), usage: newChunkWrittenIntervalList(),
logicChunkIndex: logicChunkIndex, logicChunkIndex: logicChunkIndex,
actualChunkIndex: actualChunkIndex, actualChunkIndex: actualChunkIndex,
activityScore: NewActivityScore(),
// memChunk: NewMemChunk(logicChunkIndex, sf.chunkSize),
} }
// println(logicChunkIndex, "|", "++++", swapFileChunk.actualChunkIndex, swapFileChunk, sf)
return swapFileChunk
} }
func (sc *SwapFileChunk) FreeResource() { func (sc *SwapFileChunk) FreeResource() {
sc.swapfile.logicToActualChunkIndexLock.Lock()
defer sc.swapfile.logicToActualChunkIndexLock.Unlock()
sc.Lock() sc.Lock()
defer sc.Unlock() defer sc.Unlock()
sc.swapfile.chunkTrackingLock.Lock()
defer sc.swapfile.chunkTrackingLock.Unlock()
sc.swapfile.freeActualChunkList = append(sc.swapfile.freeActualChunkList, sc.actualChunkIndex) sc.swapfile.freeActualChunkList = append(sc.swapfile.freeActualChunkList, sc.actualChunkIndex)
delete(sc.swapfile.logicToActualChunkIndex, sc.logicChunkIndex)
sc.swapfile.activeChunkCount--
// println(sc.logicChunkIndex, "|", "----", sc.actualChunkIndex, sc, sc.swapfile)
} }
func (sc *SwapFileChunk) WriteDataAt(src []byte, offset int64) (n int) {
func (sc *SwapFileChunk) WriteDataAt(src []byte, offset int64, tsNs int64) (n int) {
sc.Lock() sc.Lock()
defer sc.Unlock() defer sc.Unlock()
// println(sc.logicChunkIndex, "|", tsNs, "write at", offset, len(src), sc.actualChunkIndex)
innerOffset := offset % sc.swapfile.chunkSize innerOffset := offset % sc.swapfile.chunkSize
var err error var err error
n, err = sc.swapfile.file.WriteAt(src, int64(sc.actualChunkIndex)*sc.swapfile.chunkSize+innerOffset) n, err = sc.swapfile.file.WriteAt(src, int64(sc.actualChunkIndex)*sc.swapfile.chunkSize+innerOffset)
if err == nil {
sc.usage.MarkWritten(innerOffset, innerOffset+int64(n))
} else {
sc.usage.MarkWritten(innerOffset, innerOffset+int64(n), tsNs)
if err != nil {
glog.Errorf("failed to write swap file %s: %v", sc.swapfile.file.Name(), err) glog.Errorf("failed to write swap file %s: %v", sc.swapfile.file.Name(), err)
} }
//sc.memChunk.WriteDataAt(src, offset, tsNs)
sc.activityScore.MarkWrite()
return return
} }
func (sc *SwapFileChunk) ReadDataAt(p []byte, off int64) (maxStop int64) {
func (sc *SwapFileChunk) ReadDataAt(p []byte, off int64, tsNs int64) (maxStop int64) {
sc.RLock() sc.RLock()
defer sc.RUnlock() defer sc.RUnlock()
// println(sc.logicChunkIndex, "|", tsNs, "read at", off, len(p), sc.actualChunkIndex)
//memCopy := make([]byte, len(p))
//copy(memCopy, p)
chunkStartOffset := int64(sc.logicChunkIndex) * sc.swapfile.chunkSize chunkStartOffset := int64(sc.logicChunkIndex) * sc.swapfile.chunkSize
for t := sc.usage.head.next; t != sc.usage.tail; t = t.next { for t := sc.usage.head.next; t != sc.usage.tail; t = t.next {
logicStart := max(off, chunkStartOffset+t.StartOffset) logicStart := max(off, chunkStartOffset+t.StartOffset)
logicStop := min(off+int64(len(p)), chunkStartOffset+t.stopOffset) logicStop := min(off+int64(len(p)), chunkStartOffset+t.stopOffset)
if logicStart < logicStop { if logicStart < logicStop {
actualStart := logicStart - chunkStartOffset + int64(sc.actualChunkIndex)*sc.swapfile.chunkSize
if _, err := sc.swapfile.file.ReadAt(p[logicStart-off:logicStop-off], actualStart); err != nil {
glog.Errorf("failed to reading swap file %s: %v", sc.swapfile.file.Name(), err)
break
if t.TsNs >= tsNs {
actualStart := logicStart - chunkStartOffset + int64(sc.actualChunkIndex)*sc.swapfile.chunkSize
if _, err := sc.swapfile.file.ReadAt(p[logicStart-off:logicStop-off], actualStart); err != nil {
glog.Errorf("failed to reading swap file %s: %v", sc.swapfile.file.Name(), err)
break
}
maxStop = max(maxStop, logicStop)
} else {
println("read old data2", tsNs-t.TsNs, "ns")
} }
maxStop = max(maxStop, logicStop)
} }
} }
//sc.memChunk.ReadDataAt(memCopy, off, tsNs)
//if bytes.Compare(memCopy, p) != 0 {
// println("read wrong data from swap file", off, sc.logicChunkIndex)
//}
sc.activityScore.MarkRead()
return return
} }
@ -128,27 +157,27 @@ func (sc *SwapFileChunk) IsComplete() bool {
return sc.usage.IsComplete(sc.swapfile.chunkSize) return sc.usage.IsComplete(sc.swapfile.chunkSize)
} }
func (sc *SwapFileChunk) WrittenSize() int64 {
sc.RLock()
defer sc.RUnlock()
return sc.usage.WrittenSize()
func (sc *SwapFileChunk) ActivityScore() int64 {
return sc.activityScore.ActivityScore()
} }
func (sc *SwapFileChunk) SaveContent(saveFn SaveToStorageFunc) { func (sc *SwapFileChunk) SaveContent(saveFn SaveToStorageFunc) {
sc.RLock()
defer sc.RUnlock()
if saveFn == nil { if saveFn == nil {
return return
} }
sc.Lock()
defer sc.Unlock()
// println(sc.logicChunkIndex, "|", "save")
for t := sc.usage.head.next; t != sc.usage.tail; t = t.next { for t := sc.usage.head.next; t != sc.usage.tail; t = t.next {
data := mem.Allocate(int(t.Size())) data := mem.Allocate(int(t.Size()))
sc.swapfile.file.ReadAt(data, t.StartOffset+int64(sc.actualChunkIndex)*sc.swapfile.chunkSize)
reader := util.NewBytesReader(data)
saveFn(reader, int64(sc.logicChunkIndex)*sc.swapfile.chunkSize+t.StartOffset, t.Size(), func() {
})
n, _ := sc.swapfile.file.ReadAt(data, t.StartOffset+int64(sc.actualChunkIndex)*sc.swapfile.chunkSize)
if n > 0 {
reader := util.NewBytesReader(data[:n])
saveFn(reader, int64(sc.logicChunkIndex)*sc.swapfile.chunkSize+t.StartOffset, int64(n), t.TsNs, func() {
})
}
mem.Free(data) mem.Free(data)
} }
sc.usage = newChunkWrittenIntervalList()
} }

40
weed/mount/page_writer/upload_pipeline.go

@ -4,6 +4,7 @@ import (
"fmt" "fmt"
"github.com/seaweedfs/seaweedfs/weed/glog" "github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/util" "github.com/seaweedfs/seaweedfs/weed/util"
"math"
"sync" "sync"
"sync/atomic" "sync/atomic"
) )
@ -55,7 +56,8 @@ func NewUploadPipeline(writers *util.LimitedConcurrentExecutor, chunkSize int64,
return t return t
} }
func (up *UploadPipeline) SaveDataAt(p []byte, off int64, isSequential bool) (n int) {
func (up *UploadPipeline) SaveDataAt(p []byte, off int64, isSequential bool, tsNs int64) (n int) {
up.chunksLock.Lock() up.chunksLock.Lock()
defer up.chunksLock.Unlock() defer up.chunksLock.Unlock()
@ -65,33 +67,39 @@ func (up *UploadPipeline) SaveDataAt(p []byte, off int64, isSequential bool) (n
if !found { if !found {
if len(up.writableChunks) > up.writableChunkLimit { if len(up.writableChunks) > up.writableChunkLimit {
// if current file chunks is over the per file buffer count limit // if current file chunks is over the per file buffer count limit
fullestChunkIndex, fullness := LogicChunkIndex(-1), int64(0)
for lci, mc := range up.writableChunks {
chunkFullness := mc.WrittenSize()
if fullness < chunkFullness {
fullestChunkIndex = lci
fullness = chunkFullness
laziestChunkIndex, lowestActivityScore := LogicChunkIndex(-1), int64(math.MaxInt64)
for wci, wc := range up.writableChunks {
activityScore := wc.ActivityScore()
if lowestActivityScore > activityScore {
laziestChunkIndex = wci
lowestActivityScore = activityScore
} }
} }
up.moveToSealed(up.writableChunks[fullestChunkIndex], fullestChunkIndex)
// fmt.Printf("flush chunk %d with %d bytes written\n", logicChunkIndex, fullness)
up.moveToSealed(up.writableChunks[laziestChunkIndex], laziestChunkIndex)
// fmt.Printf("flush chunk %d with %d bytes written\n", logicChunkIndex, oldestTs)
} }
if isSequential && if isSequential &&
len(up.writableChunks) < up.writableChunkLimit && len(up.writableChunks) < up.writableChunkLimit &&
atomic.LoadInt64(&memChunkCounter) < 4*int64(up.writableChunkLimit) { atomic.LoadInt64(&memChunkCounter) < 4*int64(up.writableChunkLimit) {
pageChunk = NewMemChunk(logicChunkIndex, up.ChunkSize) pageChunk = NewMemChunk(logicChunkIndex, up.ChunkSize)
} else { } else {
pageChunk = up.swapFile.NewTempFileChunk(logicChunkIndex)
pageChunk = up.swapFile.NewSwapFileChunk(logicChunkIndex)
} }
up.writableChunks[logicChunkIndex] = pageChunk up.writableChunks[logicChunkIndex] = pageChunk
} }
n = pageChunk.WriteDataAt(p, off)
//if _, foundSealed := up.sealedChunks[logicChunkIndex]; foundSealed {
// println("found already sealed chunk", logicChunkIndex)
//}
//if _, foundReading := up.activeReadChunks[logicChunkIndex]; foundReading {
// println("found active read chunk", logicChunkIndex)
//}
n = pageChunk.WriteDataAt(p, off, tsNs)
up.maybeMoveToSealed(pageChunk, logicChunkIndex) up.maybeMoveToSealed(pageChunk, logicChunkIndex)
return return
} }
func (up *UploadPipeline) MaybeReadDataAt(p []byte, off int64) (maxStop int64) {
func (up *UploadPipeline) MaybeReadDataAt(p []byte, off int64, tsNs int64) (maxStop int64) {
logicChunkIndex := LogicChunkIndex(off / up.ChunkSize) logicChunkIndex := LogicChunkIndex(off / up.ChunkSize)
up.chunksLock.Lock() up.chunksLock.Lock()
@ -103,12 +111,8 @@ func (up *UploadPipeline) MaybeReadDataAt(p []byte, off int64) (maxStop int64) {
// read from sealed chunks first // read from sealed chunks first
sealedChunk, found := up.sealedChunks[logicChunkIndex] sealedChunk, found := up.sealedChunks[logicChunkIndex]
if found { if found {
sealedChunk.referenceCounter++
}
if found {
maxStop = sealedChunk.chunk.ReadDataAt(p, off)
maxStop = sealedChunk.chunk.ReadDataAt(p, off, tsNs)
glog.V(4).Infof("%s read sealed memchunk [%d,%d)", up.filepath, off, maxStop) glog.V(4).Infof("%s read sealed memchunk [%d,%d)", up.filepath, off, maxStop)
sealedChunk.FreeReference(fmt.Sprintf("%s finish reading chunk %d", up.filepath, logicChunkIndex))
} }
// read from writable chunks last // read from writable chunks last
@ -116,7 +120,7 @@ func (up *UploadPipeline) MaybeReadDataAt(p []byte, off int64) (maxStop int64) {
if !found { if !found {
return return
} }
writableMaxStop := writableChunk.ReadDataAt(p, off)
writableMaxStop := writableChunk.ReadDataAt(p, off, tsNs)
glog.V(4).Infof("%s read writable memchunk [%d,%d)", up.filepath, off, writableMaxStop) glog.V(4).Infof("%s read writable memchunk [%d,%d)", up.filepath, off, writableMaxStop)
maxStop = max(maxStop, writableMaxStop) maxStop = max(maxStop, writableMaxStop)

4
weed/mount/page_writer/upload_pipeline_test.go

@ -31,14 +31,14 @@ func writeRange(uploadPipeline *UploadPipeline, startOff, stopOff int64) {
p := make([]byte, 4) p := make([]byte, 4)
for i := startOff / 4; i < stopOff/4; i += 4 { for i := startOff / 4; i < stopOff/4; i += 4 {
util.Uint32toBytes(p, uint32(i)) util.Uint32toBytes(p, uint32(i))
uploadPipeline.SaveDataAt(p, i, false)
uploadPipeline.SaveDataAt(p, i, false, 0)
} }
} }
func confirmRange(t *testing.T, uploadPipeline *UploadPipeline, startOff, stopOff int64) { func confirmRange(t *testing.T, uploadPipeline *UploadPipeline, startOff, stopOff int64) {
p := make([]byte, 4) p := make([]byte, 4)
for i := startOff; i < stopOff/4; i += 4 { for i := startOff; i < stopOff/4; i += 4 {
uploadPipeline.MaybeReadDataAt(p, i)
uploadPipeline.MaybeReadDataAt(p, i, 0)
x := util.BytesToUint32(p) x := util.BytesToUint32(p)
if x != uint32(i) { if x != uint32(i) {
t.Errorf("expecting %d found %d at offset [%d,%d)", i, x, i, i+4) t.Errorf("expecting %d found %d at offset [%d,%d)", i, x, i, i+4)

20
weed/mount/weedfs_attr.go

@ -20,12 +20,12 @@ func (wfs *WFS) GetAttr(cancel <-chan struct{}, input *fuse.GetAttrIn, out *fuse
_, _, entry, status := wfs.maybeReadEntry(inode) _, _, entry, status := wfs.maybeReadEntry(inode)
if status == fuse.OK { if status == fuse.OK {
out.AttrValid = 1 out.AttrValid = 1
wfs.setAttrByPbEntry(&out.Attr, inode, entry)
wfs.setAttrByPbEntry(&out.Attr, inode, entry, true)
return status return status
} else { } else {
if fh, found := wfs.fhmap.FindFileHandle(inode); found { if fh, found := wfs.fhmap.FindFileHandle(inode); found {
out.AttrValid = 1 out.AttrValid = 1
wfs.setAttrByPbEntry(&out.Attr, inode, fh.entry.GetEntry())
wfs.setAttrByPbEntry(&out.Attr, inode, fh.entry.GetEntry(), true)
out.Nlink = 0 out.Nlink = 0
return fuse.OK return fuse.OK
} }
@ -75,7 +75,7 @@ func (wfs *WFS) SetAttr(cancel <-chan struct{}, input *fuse.SetAttrIn, out *fuse
// set the new chunks and reset entry cache // set the new chunks and reset entry cache
entry.Chunks = chunks entry.Chunks = chunks
if fh != nil { if fh != nil {
fh.entryViewCache = nil
fh.entryChunkGroup.SetChunks(chunks)
} }
} }
entry.Attributes.Mtime = time.Now().Unix() entry.Attributes.Mtime = time.Now().Unix()
@ -114,7 +114,11 @@ func (wfs *WFS) SetAttr(cancel <-chan struct{}, input *fuse.SetAttrIn, out *fuse
} }
out.AttrValid = 1 out.AttrValid = 1
wfs.setAttrByPbEntry(&out.Attr, input.NodeId, entry)
size, includeSize := input.GetSize()
if includeSize {
out.Attr.Size = size
}
wfs.setAttrByPbEntry(&out.Attr, input.NodeId, entry, !includeSize)
if fh != nil { if fh != nil {
fh.dirtyMetadata = true fh.dirtyMetadata = true
@ -139,12 +143,14 @@ func (wfs *WFS) setRootAttr(out *fuse.AttrOut) {
out.Nlink = 1 out.Nlink = 1
} }
func (wfs *WFS) setAttrByPbEntry(out *fuse.Attr, inode uint64, entry *filer_pb.Entry) {
func (wfs *WFS) setAttrByPbEntry(out *fuse.Attr, inode uint64, entry *filer_pb.Entry, calculateSize bool) {
out.Ino = inode out.Ino = inode
if entry.Attributes != nil && entry.Attributes.Inode != 0 { if entry.Attributes != nil && entry.Attributes.Inode != 0 {
out.Ino = entry.Attributes.Inode out.Ino = entry.Attributes.Inode
} }
out.Size = filer.FileSize(entry)
if calculateSize {
out.Size = filer.FileSize(entry)
}
if entry.FileMode()&os.ModeSymlink != 0 { if entry.FileMode()&os.ModeSymlink != 0 {
out.Size = uint64(len(entry.Attributes.SymlinkTarget)) out.Size = uint64(len(entry.Attributes.SymlinkTarget))
} }
@ -194,7 +200,7 @@ func (wfs *WFS) outputPbEntry(out *fuse.EntryOut, inode uint64, entry *filer_pb.
out.Generation = 1 out.Generation = 1
out.EntryValid = 1 out.EntryValid = 1
out.AttrValid = 1 out.AttrValid = 1
wfs.setAttrByPbEntry(&out.Attr, inode, entry)
wfs.setAttrByPbEntry(&out.Attr, inode, entry, true)
} }
func (wfs *WFS) outputFilerEntry(out *fuse.EntryOut, inode uint64, entry *filer.Entry) { func (wfs *WFS) outputFilerEntry(out *fuse.EntryOut, inode uint64, entry *filer.Entry) {

12
weed/mount/weedfs_file_copy_range.go

@ -1,8 +1,8 @@
package mount package mount
import ( import (
"context"
"net/http" "net/http"
"time"
"github.com/hanwen/go-fuse/v2/fuse" "github.com/hanwen/go-fuse/v2/fuse"
@ -44,8 +44,8 @@ func (wfs *WFS) CopyFileRange(cancel <-chan struct{}, in *fuse.CopyFileRangeIn)
} }
// lock source and target file handles // lock source and target file handles
fhOut.orderedMutex.Acquire(context.Background(), 1)
defer fhOut.orderedMutex.Release(1)
fhOut.Lock()
defer fhOut.Unlock()
fhOut.entryLock.Lock() fhOut.entryLock.Lock()
defer fhOut.entryLock.Unlock() defer fhOut.entryLock.Unlock()
@ -54,8 +54,8 @@ func (wfs *WFS) CopyFileRange(cancel <-chan struct{}, in *fuse.CopyFileRangeIn)
} }
if fhIn.fh != fhOut.fh { if fhIn.fh != fhOut.fh {
fhIn.orderedMutex.Acquire(context.Background(), 1)
defer fhIn.orderedMutex.Release(1)
fhIn.Lock()
defer fhIn.Unlock()
fhIn.entryLock.Lock() fhIn.entryLock.Lock()
defer fhIn.entryLock.Unlock() defer fhIn.entryLock.Unlock()
} }
@ -88,7 +88,7 @@ func (wfs *WFS) CopyFileRange(cancel <-chan struct{}, in *fuse.CopyFileRangeIn)
// put data at the specified offset in target file // put data at the specified offset in target file
fhOut.dirtyPages.writerPattern.MonitorWriteAt(int64(in.OffOut), int(in.Len)) fhOut.dirtyPages.writerPattern.MonitorWriteAt(int64(in.OffOut), int(in.Len))
fhOut.entry.Content = nil fhOut.entry.Content = nil
fhOut.dirtyPages.AddPage(int64(in.OffOut), data, fhOut.dirtyPages.writerPattern.IsSequentialMode())
fhOut.dirtyPages.AddPage(int64(in.OffOut), data, fhOut.dirtyPages.writerPattern.IsSequentialMode(), time.Now().UnixNano())
fhOut.entry.Attributes.FileSize = uint64(max(int64(in.OffOut)+totalRead, int64(fhOut.entry.Attributes.FileSize))) fhOut.entry.Attributes.FileSize = uint64(max(int64(in.OffOut)+totalRead, int64(fhOut.entry.Attributes.FileSize)))
fhOut.dirtyMetadata = true fhOut.dirtyMetadata = true
written = uint32(totalRead) written = uint32(totalRead)

43
weed/mount/weedfs_file_lseek.go

@ -1,7 +1,6 @@
package mount package mount
import ( import (
"context"
"syscall" "syscall"
"github.com/hanwen/go-fuse/v2/fuse" "github.com/hanwen/go-fuse/v2/fuse"
@ -36,8 +35,8 @@ func (wfs *WFS) Lseek(cancel <-chan struct{}, in *fuse.LseekIn, out *fuse.LseekO
} }
// lock the file until the proper offset was calculated // lock the file until the proper offset was calculated
fh.orderedMutex.Acquire(context.Background(), 1)
defer fh.orderedMutex.Release(1)
fh.Lock()
defer fh.Unlock()
fh.entryLock.Lock() fh.entryLock.Lock()
defer fh.entryLock.Unlock() defer fh.entryLock.Unlock()
@ -56,17 +55,8 @@ func (wfs *WFS) Lseek(cancel <-chan struct{}, in *fuse.LseekIn, out *fuse.LseekO
return ENXIO return ENXIO
} }
// refresh view cache if necessary
if fh.entryViewCache == nil {
var err error
fh.entryViewCache, err = filer.NonOverlappingVisibleIntervals(fh.wfs.LookupFn(), fh.entry.GetChunks(), 0, fileSize)
if err != nil {
return fuse.EIO
}
}
// search chunks for the offset // search chunks for the offset
found, offset := searchChunks(fh, offset, fileSize, in.Whence)
found, offset := fh.entryChunkGroup.SearchChunks(offset, fileSize, in.Whence)
if found { if found {
out.Offset = uint64(offset) out.Offset = uint64(offset)
return fuse.OK return fuse.OK
@ -82,30 +72,3 @@ func (wfs *WFS) Lseek(cancel <-chan struct{}, in *fuse.LseekIn, out *fuse.LseekO
return fuse.OK return fuse.OK
} }
// searchChunks goes through all chunks to find the correct offset
func searchChunks(fh *FileHandle, offset, fileSize int64, whence uint32) (found bool, out int64) {
chunkViews := filer.ViewFromVisibleIntervals(fh.entryViewCache, offset, fileSize)
for _, chunkView := range chunkViews {
if offset < chunkView.LogicOffset {
if whence == SEEK_HOLE {
out = offset
} else {
out = chunkView.LogicOffset
}
return true, out
}
if offset >= chunkView.LogicOffset && offset < chunkView.Offset+int64(chunkView.Size) && whence == SEEK_DATA {
out = offset
return true, out
}
offset += int64(chunkView.Size)
}
return
}

28
weed/mount/weedfs_file_read.go

@ -1,7 +1,8 @@
package mount package mount
import ( import (
"context"
"bytes"
"fmt"
"io" "io"
"github.com/hanwen/go-fuse/v2/fuse" "github.com/hanwen/go-fuse/v2/fuse"
@ -40,8 +41,8 @@ func (wfs *WFS) Read(cancel <-chan struct{}, in *fuse.ReadIn, buff []byte) (fuse
return nil, fuse.ENOENT return nil, fuse.ENOENT
} }
fh.orderedMutex.Acquire(context.Background(), 1)
defer fh.orderedMutex.Release(1)
fh.Lock()
defer fh.Unlock()
offset := int64(in.Offset) offset := int64(in.Offset)
totalRead, err := readDataByFileHandle(buff, fh, offset) totalRead, err := readDataByFileHandle(buff, fh, offset)
@ -50,6 +51,23 @@ func (wfs *WFS) Read(cancel <-chan struct{}, in *fuse.ReadIn, buff []byte) (fuse
return nil, fuse.EIO return nil, fuse.EIO
} }
if IsDebugFileReadWrite {
// print(".")
mirrorData := make([]byte, totalRead)
fh.mirrorFile.ReadAt(mirrorData, offset)
if bytes.Compare(mirrorData, buff[:totalRead]) != 0 {
againBuff := make([]byte, len(buff))
againRead, _ := readDataByFileHandle(buff, fh, offset)
againCorrect := bytes.Compare(mirrorData, againBuff[:againRead]) == 0
againSame := bytes.Compare(buff[:totalRead], againBuff[:againRead]) == 0
fmt.Printf("\ncompare %v [%d,%d) size:%d againSame:%v againCorrect:%v\n", fh.mirrorFile.Name(), offset, offset+totalRead, totalRead, againSame, againCorrect)
//fmt.Printf("read mirrow data: %v\n", mirrorData)
//fmt.Printf("read actual data: %v\n", buff[:totalRead])
}
}
return fuse.ReadResultData(buff[:totalRead]), fuse.OK return fuse.ReadResultData(buff[:totalRead]), fuse.OK
} }
@ -59,9 +77,9 @@ func readDataByFileHandle(buff []byte, fhIn *FileHandle, offset int64) (int64, e
fhIn.lockForRead(offset, size) fhIn.lockForRead(offset, size)
defer fhIn.unlockForRead(offset, size) defer fhIn.unlockForRead(offset, size)
n, err := fhIn.readFromChunks(buff, offset)
n, tsNs, err := fhIn.readFromChunks(buff, offset)
if err == nil || err == io.EOF { if err == nil || err == io.EOF {
maxStop := fhIn.readFromDirtyPages(buff, offset)
maxStop := fhIn.readFromDirtyPages(buff, offset, tsNs)
n = max(maxStop-offset, n) n = max(maxStop-offset, n)
} }
if err == io.EOF { if err == io.EOF {

15
weed/mount/weedfs_file_sync.go

@ -89,8 +89,8 @@ func (wfs *WFS) Fsync(cancel <-chan struct{}, in *fuse.FsyncIn) (code fuse.Statu
} }
func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status { func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
fh.orderedMutex.Acquire(context.Background(), 1)
defer fh.orderedMutex.Release(1)
fh.Lock()
defer fh.Unlock()
// flush works at fh level // flush works at fh level
fileFullPath := fh.FullPath() fileFullPath := fh.FullPath()
@ -145,9 +145,9 @@ func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
} }
glog.V(4).Infof("%s set chunks: %v", fileFullPath, len(entry.GetChunks())) glog.V(4).Infof("%s set chunks: %v", fileFullPath, len(entry.GetChunks()))
for i, chunk := range entry.GetChunks() {
glog.V(4).Infof("%s chunks %d: %v [%d,%d)", fileFullPath, i, chunk.GetFileIdString(), chunk.Offset, chunk.Offset+int64(chunk.Size))
}
//for i, chunk := range entry.GetChunks() {
// glog.V(4).Infof("%s chunks %d: %v [%d,%d)", fileFullPath, i, chunk.GetFileIdString(), chunk.Offset, chunk.Offset+int64(chunk.Size))
//}
manifestChunks, nonManifestChunks := filer.SeparateManifestChunks(entry.GetChunks()) manifestChunks, nonManifestChunks := filer.SeparateManifestChunks(entry.GetChunks())
@ -158,6 +158,7 @@ func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
glog.V(0).Infof("MaybeManifestize: %v", manifestErr) glog.V(0).Infof("MaybeManifestize: %v", manifestErr)
} }
entry.Chunks = append(chunks, manifestChunks...) entry.Chunks = append(chunks, manifestChunks...)
fh.entryChunkGroup.SetChunks(entry.Chunks)
wfs.mapPbIdFromLocalToFiler(request.Entry) wfs.mapPbIdFromLocalToFiler(request.Entry)
defer wfs.mapPbIdFromFilerToLocal(request.Entry) defer wfs.mapPbIdFromFilerToLocal(request.Entry)
@ -181,5 +182,9 @@ func (wfs *WFS) doFlush(fh *FileHandle, uid, gid uint32) fuse.Status {
return fuse.EIO return fuse.EIO
} }
if IsDebugFileReadWrite {
fh.mirrorFile.Sync()
}
return fuse.OK return fuse.OK
} }

15
weed/mount/weedfs_file_write.go

@ -1,10 +1,10 @@
package mount package mount
import ( import (
"context"
"github.com/hanwen/go-fuse/v2/fuse" "github.com/hanwen/go-fuse/v2/fuse"
"net/http" "net/http"
"syscall" "syscall"
"time"
) )
/** /**
@ -46,8 +46,10 @@ func (wfs *WFS) Write(cancel <-chan struct{}, in *fuse.WriteIn, data []byte) (wr
fh.dirtyPages.writerPattern.MonitorWriteAt(int64(in.Offset), int(in.Size)) fh.dirtyPages.writerPattern.MonitorWriteAt(int64(in.Offset), int(in.Size))
fh.orderedMutex.Acquire(context.Background(), 1)
defer fh.orderedMutex.Release(1)
tsNs := time.Now().UnixNano()
fh.Lock()
defer fh.Unlock()
entry := fh.GetEntry() entry := fh.GetEntry()
if entry == nil { if entry == nil {
@ -59,7 +61,7 @@ func (wfs *WFS) Write(cancel <-chan struct{}, in *fuse.WriteIn, data []byte) (wr
entry.Attributes.FileSize = uint64(max(offset+int64(len(data)), int64(entry.Attributes.FileSize))) entry.Attributes.FileSize = uint64(max(offset+int64(len(data)), int64(entry.Attributes.FileSize)))
// glog.V(4).Infof("%v write [%d,%d) %d", fh.f.fullpath(), req.Offset, req.Offset+int64(len(req.Data)), len(req.Data)) // glog.V(4).Infof("%v write [%d,%d) %d", fh.f.fullpath(), req.Offset, req.Offset+int64(len(req.Data)), len(req.Data))
fh.dirtyPages.AddPage(offset, data, fh.dirtyPages.writerPattern.IsSequentialMode())
fh.dirtyPages.AddPage(offset, data, fh.dirtyPages.writerPattern.IsSequentialMode(), tsNs)
written = uint32(len(data)) written = uint32(len(data))
@ -70,5 +72,10 @@ func (wfs *WFS) Write(cancel <-chan struct{}, in *fuse.WriteIn, data []byte) (wr
fh.dirtyMetadata = true fh.dirtyMetadata = true
if IsDebugFileReadWrite {
// print("+")
fh.mirrorFile.WriteAt(data, offset)
}
return written, fuse.OK return written, fuse.OK
} }

4
weed/mount/weedfs_write.go

@ -13,7 +13,7 @@ import (
func (wfs *WFS) saveDataAsChunk(fullPath util.FullPath) filer.SaveDataAsChunkFunctionType { func (wfs *WFS) saveDataAsChunk(fullPath util.FullPath) filer.SaveDataAsChunkFunctionType {
return func(reader io.Reader, filename string, offset int64) (chunk *filer_pb.FileChunk, err error) {
return func(reader io.Reader, filename string, offset int64, tsNs int64) (chunk *filer_pb.FileChunk, err error) {
fileId, uploadResult, err, data := operation.UploadWithRetry( fileId, uploadResult, err, data := operation.UploadWithRetry(
wfs, wfs,
@ -56,7 +56,7 @@ func (wfs *WFS) saveDataAsChunk(fullPath util.FullPath) filer.SaveDataAsChunkFun
wfs.chunkCache.SetChunk(fileId, data) wfs.chunkCache.SetChunk(fileId, data)
} }
chunk = uploadResult.ToPbFileChunk(fileId, offset)
chunk = uploadResult.ToPbFileChunk(fileId, offset, tsNs)
return chunk, nil return chunk, nil
} }
} }

4
weed/operation/upload_content.go

@ -45,13 +45,13 @@ type UploadResult struct {
RetryCount int `json:"-"` RetryCount int `json:"-"`
} }
func (uploadResult *UploadResult) ToPbFileChunk(fileId string, offset int64) *filer_pb.FileChunk {
func (uploadResult *UploadResult) ToPbFileChunk(fileId string, offset int64, tsNs int64) *filer_pb.FileChunk {
fid, _ := filer_pb.ToFileIdObject(fileId) fid, _ := filer_pb.ToFileIdObject(fileId)
return &filer_pb.FileChunk{ return &filer_pb.FileChunk{
FileId: fileId, FileId: fileId,
Offset: offset, Offset: offset,
Size: uint64(uploadResult.Size), Size: uint64(uploadResult.Size),
ModifiedTsNs: time.Now().UnixNano(),
ModifiedTsNs: tsNs,
ETag: uploadResult.ContentMd5, ETag: uploadResult.ContentMd5,
CipherKey: uploadResult.CipherKey, CipherKey: uploadResult.CipherKey,
IsCompressed: uploadResult.Gzip > 0, IsCompressed: uploadResult.Gzip > 0,

7
weed/replication/repl_util/replication_util.go

@ -7,9 +7,10 @@ import (
"github.com/seaweedfs/seaweedfs/weed/util" "github.com/seaweedfs/seaweedfs/weed/util"
) )
func CopyFromChunkViews(chunkViews []*filer.ChunkView, filerSource *source.FilerSource, writeFunc func(data []byte) error) error {
func CopyFromChunkViews(chunkViews *filer.IntervalList[*filer.ChunkView], filerSource *source.FilerSource, writeFunc func(data []byte) error) error {
for _, chunk := range chunkViews {
for x := chunkViews.Front(); x != nil; x = x.Next {
chunk := x.Value
fileUrls, err := filerSource.LookupFileId(chunk.FileId) fileUrls, err := filerSource.LookupFileId(chunk.FileId)
if err != nil { if err != nil {
@ -20,7 +21,7 @@ func CopyFromChunkViews(chunkViews []*filer.ChunkView, filerSource *source.Filer
var shouldRetry bool var shouldRetry bool
for _, fileUrl := range fileUrls { for _, fileUrl := range fileUrls {
shouldRetry, err = util.ReadUrlAsStream(fileUrl, chunk.CipherKey, chunk.IsGzipped, chunk.IsFullChunk(), chunk.Offset, int(chunk.Size), func(data []byte) {
shouldRetry, err = util.ReadUrlAsStream(fileUrl, chunk.CipherKey, chunk.IsGzipped, chunk.IsFullChunk(), chunk.OffsetInChunk, int(chunk.ViewSize), func(data []byte) {
writeErr = writeFunc(data) writeErr = writeFunc(data)
}) })
if err != nil { if err != nil {

4
weed/server/filer_server_handlers_write_autochunk.go

@ -256,7 +256,7 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
func (fs *FilerServer) saveAsChunk(so *operation.StorageOption) filer.SaveDataAsChunkFunctionType { func (fs *FilerServer) saveAsChunk(so *operation.StorageOption) filer.SaveDataAsChunkFunctionType {
return func(reader io.Reader, name string, offset int64) (*filer_pb.FileChunk, error) {
return func(reader io.Reader, name string, offset int64, tsNs int64) (*filer_pb.FileChunk, error) {
var fileId string var fileId string
var uploadResult *operation.UploadResult var uploadResult *operation.UploadResult
@ -290,7 +290,7 @@ func (fs *FilerServer) saveAsChunk(so *operation.StorageOption) filer.SaveDataAs
return nil, err return nil, err
} }
return uploadResult.ToPbFileChunk(fileId, offset), nil
return uploadResult.ToPbFileChunk(fileId, offset, tsNs), nil
} }
} }

2
weed/server/filer_server_handlers_write_cipher.go

@ -59,7 +59,7 @@ func (fs *FilerServer) encrypt(ctx context.Context, w http.ResponseWriter, r *ht
} }
// Save to chunk manifest structure // Save to chunk manifest structure
fileChunks := []*filer_pb.FileChunk{uploadResult.ToPbFileChunk(fileId, 0)}
fileChunks := []*filer_pb.FileChunk{uploadResult.ToPbFileChunk(fileId, 0, time.Now().UnixNano())}
// fmt.Printf("uploaded: %+v\n", uploadResult) // fmt.Printf("uploaded: %+v\n", uploadResult)

2
weed/server/filer_server_handlers_write_upload.go

@ -214,5 +214,5 @@ func (fs *FilerServer) dataToChunk(fileName, contentType string, data []byte, ch
if uploadResult.Size == 0 { if uploadResult.Size == 0 {
return nil, nil return nil, nil
} }
return []*filer_pb.FileChunk{uploadResult.ToPbFileChunk(fileId, chunkOffset)}, nil
return []*filer_pb.FileChunk{uploadResult.ToPbFileChunk(fileId, chunkOffset, time.Now().UnixNano())}, nil
} }

30
weed/server/webdav_server.go

@ -102,14 +102,14 @@ func (fi *FileInfo) IsDir() bool { return fi.isDirectory }
func (fi *FileInfo) Sys() interface{} { return nil } func (fi *FileInfo) Sys() interface{} { return nil }
type WebDavFile struct { type WebDavFile struct {
fs *WebDavFileSystem
name string
isDirectory bool
off int64
entry *filer_pb.Entry
entryViewCache []filer.VisibleInterval
reader io.ReaderAt
bufWriter *buffered_writer.BufferedWriteCloser
fs *WebDavFileSystem
name string
isDirectory bool
off int64
entry *filer_pb.Entry
visibleIntervals *filer.IntervalList[*filer.VisibleInterval]
reader io.ReaderAt
bufWriter *buffered_writer.BufferedWriteCloser
} }
func NewWebDavFileSystem(option *WebDavOption) (webdav.FileSystem, error) { func NewWebDavFileSystem(option *WebDavOption) (webdav.FileSystem, error) {
@ -381,7 +381,7 @@ func (fs *WebDavFileSystem) Stat(ctx context.Context, name string) (os.FileInfo,
return fs.stat(ctx, name) return fs.stat(ctx, name)
} }
func (f *WebDavFile) saveDataAsChunk(reader io.Reader, name string, offset int64) (chunk *filer_pb.FileChunk, err error) {
func (f *WebDavFile) saveDataAsChunk(reader io.Reader, name string, offset int64, tsNs int64) (chunk *filer_pb.FileChunk, err error) {
fileId, uploadResult, flushErr, _ := operation.UploadWithRetry( fileId, uploadResult, flushErr, _ := operation.UploadWithRetry(
f.fs, f.fs,
@ -413,7 +413,7 @@ func (f *WebDavFile) saveDataAsChunk(reader io.Reader, name string, offset int64
glog.V(0).Infof("upload failure %v: %v", f.name, flushErr) glog.V(0).Infof("upload failure %v: %v", f.name, flushErr)
return nil, fmt.Errorf("upload result: %v", uploadResult.Error) return nil, fmt.Errorf("upload result: %v", uploadResult.Error)
} }
return uploadResult.ToPbFileChunk(fileId, offset), nil
return uploadResult.ToPbFileChunk(fileId, offset, tsNs), nil
} }
func (f *WebDavFile) Write(buf []byte) (int, error) { func (f *WebDavFile) Write(buf []byte) (int, error) {
@ -439,7 +439,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
f.bufWriter.FlushFunc = func(data []byte, offset int64) (flushErr error) { f.bufWriter.FlushFunc = func(data []byte, offset int64) (flushErr error) {
var chunk *filer_pb.FileChunk var chunk *filer_pb.FileChunk
chunk, flushErr = f.saveDataAsChunk(util.NewBytesReader(data), f.name, offset)
chunk, flushErr = f.saveDataAsChunk(util.NewBytesReader(data), f.name, offset, time.Now().UnixNano())
if flushErr != nil { if flushErr != nil {
return fmt.Errorf("%s upload result: %v", f.name, flushErr) return fmt.Errorf("%s upload result: %v", f.name, flushErr)
@ -498,7 +498,7 @@ func (f *WebDavFile) Close() error {
if f.entry != nil { if f.entry != nil {
f.entry = nil f.entry = nil
f.entryViewCache = nil
f.visibleIntervals = nil
} }
return err return err
@ -521,12 +521,12 @@ func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
if fileSize == 0 { if fileSize == 0 {
return 0, io.EOF return 0, io.EOF
} }
if f.entryViewCache == nil {
f.entryViewCache, _ = filer.NonOverlappingVisibleIntervals(filer.LookupFn(f.fs), f.entry.GetChunks(), 0, fileSize)
if f.visibleIntervals == nil {
f.visibleIntervals, _ = filer.NonOverlappingVisibleIntervals(filer.LookupFn(f.fs), f.entry.GetChunks(), 0, fileSize)
f.reader = nil f.reader = nil
} }
if f.reader == nil { if f.reader == nil {
chunkViews := filer.ViewFromVisibleIntervals(f.entryViewCache, 0, fileSize)
chunkViews := filer.ViewFromVisibleIntervals(f.visibleIntervals, 0, fileSize)
f.reader = filer.NewChunkReaderAtFromClient(filer.LookupFn(f.fs), chunkViews, f.fs.chunkCache, fileSize) f.reader = filer.NewChunkReaderAtFromClient(filer.LookupFn(f.fs), chunkViews, f.fs.chunkCache, fileSize)
} }

2
weed/shell/command_fs_verify.go

@ -117,7 +117,7 @@ type ItemEntry struct {
func (c *commandFsVerify) verifyTraverseBfs(path string) (fileCount int64, errCount int64, err error) { func (c *commandFsVerify) verifyTraverseBfs(path string) (fileCount int64, errCount int64, err error) {
timeNowAtSec := time.Now().Unix() timeNowAtSec := time.Now().Unix()
return fileCount, errCount, doTraverseBfsAndSaving(c.env, nil, path, false,
return fileCount, errCount, doTraverseBfsAndSaving(c.env, c.writer, path, false,
func(entry *filer_pb.FullEntry, outputChan chan interface{}) (err error) { func(entry *filer_pb.FullEntry, outputChan chan interface{}) (err error) {
if c.modifyTimeAgoAtSec > 0 { if c.modifyTimeAgoAtSec > 0 {
if entry.Entry.Attributes != nil && c.modifyTimeAgoAtSec < timeNowAtSec-entry.Entry.Attributes.Mtime { if entry.Entry.Attributes != nil && c.modifyTimeAgoAtSec < timeNowAtSec-entry.Entry.Attributes.Mtime {

2
weed/shell/command_volume_fsck.go

@ -211,7 +211,7 @@ func (c *commandVolumeFsck) collectFilerFileIdAndPaths(dataNodeVolumeIdToVInfo m
} }
}() }()
return doTraverseBfsAndSaving(c.env, nil, c.getCollectFilerFilePath(), false,
return doTraverseBfsAndSaving(c.env, c.writer, c.getCollectFilerFilePath(), false,
func(entry *filer_pb.FullEntry, outputChan chan interface{}) (err error) { func(entry *filer_pb.FullEntry, outputChan chan interface{}) (err error) {
if *c.verbose && entry.Entry.IsDirectory { if *c.verbose && entry.Entry.IsDirectory {
fmt.Fprintf(c.writer, "checking directory %s\n", util.NewFullPath(entry.Dir, entry.Entry.Name)) fmt.Fprintf(c.writer, "checking directory %s\n", util.NewFullPath(entry.Dir, entry.Entry.Name))

Loading…
Cancel
Save