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
-
210weed/filer/filechunks.go
-
90weed/filer/filechunks_read.go
-
86weed/filer/filechunks_read_test.go
-
174weed/filer/filechunks_test.go
-
2weed/filer/filer_notify_append.go
-
259weed/filer/interval_list.go
-
327weed/filer/interval_list_test.go
-
63weed/filer/reader_at.go
-
62weed/filer/reader_at_test.go
-
7weed/filer/reader_cache.go
-
88weed/filer/stream.go
-
16weed/mount/dirty_pages_chunked.go
-
79weed/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
-
77weed/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
-
27weed/mount/page_writer/page_chunk_mem.go
-
91weed/mount/page_writer/page_chunk_swapfile.go
-
40weed/mount/page_writer/upload_pipeline.go
-
4weed/mount/page_writer/upload_pipeline_test.go
-
18weed/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
-
16weed/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