Browse Source
more solid weed mount (#4089)
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
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
45 changed files with 1835 additions and 806 deletions
-
8weed/command/filer_copy.go
-
148weed/filer/filechunk_group.go
-
36weed/filer/filechunk_group_test.go
-
4weed/filer/filechunk_manifest.go
-
119weed/filer/filechunk_section.go
-
248weed/filer/filechunks.go
-
106weed/filer/filechunks_read.go
-
86weed/filer/filechunks_read_test.go
-
214weed/filer/filechunks_test.go
-
2weed/filer/filer_notify_append.go
-
259weed/filer/interval_list.go
-
327weed/filer/interval_list_test.go
-
65weed/filer/reader_at.go
-
142weed/filer/reader_at_test.go
-
7weed/filer/reader_cache.go
-
90weed/filer/stream.go
-
16weed/mount/dirty_pages_chunked.go
-
109weed/mount/filehandle.go
-
4weed/mount/filehandle_map.go
-
43weed/mount/filehandle_read.go
-
12weed/mount/page_writer.go
-
39weed/mount/page_writer/activity_score.go
-
83weed/mount/page_writer/chunk_interval_list.go
-
72weed/mount/page_writer/chunk_interval_list_test.go
-
4weed/mount/page_writer/dirty_pages.go
-
8weed/mount/page_writer/page_chunk.go
-
31weed/mount/page_writer/page_chunk_mem.go
-
125weed/mount/page_writer/page_chunk_swapfile.go
-
40weed/mount/page_writer/upload_pipeline.go
-
4weed/mount/page_writer/upload_pipeline_test.go
-
20weed/mount/weedfs_attr.go
-
12weed/mount/weedfs_file_copy_range.go
-
43weed/mount/weedfs_file_lseek.go
-
28weed/mount/weedfs_file_read.go
-
15weed/mount/weedfs_file_sync.go
-
15weed/mount/weedfs_file_write.go
-
4weed/mount/weedfs_write.go
-
4weed/operation/upload_content.go
-
7weed/replication/repl_util/replication_util.go
-
4weed/server/filer_server_handlers_write_autochunk.go
-
2weed/server/filer_server_handlers_write_cipher.go
-
2weed/server/filer_server_handlers_write_upload.go
-
30weed/server/webdav_server.go
-
2weed/shell/command_fs_verify.go
-
2weed/shell/command_volume_fsck.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 |
||||
|
} |
||||
|
} |
@ -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) |
||||
|
}) |
||||
|
} |
||||
|
} |
@ -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 |
||||
|
} |
@ -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 |
||||
|
} |
@ -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()) |
||||
|
} |
@ -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 |
||||
|
} |
Write
Preview
Loading…
Cancel
Save
Reference in new issue