X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/a45b256a3261fff8f168321c02e61b94ae9b4a64..bd8bdd90055d61263eff5bdb9a953c57319aa83d:/sdk/go/arvados/fs_collection.go?ds=sidebyside diff --git a/sdk/go/arvados/fs_collection.go b/sdk/go/arvados/fs_collection.go index 6644f4cfb8..060b57b493 100644 --- a/sdk/go/arvados/fs_collection.go +++ b/sdk/go/arvados/fs_collection.go @@ -16,13 +16,13 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "time" ) var ( maxBlockSize = 1 << 26 - concurrentWriters = 4 // max goroutines writing to Keep during sync() - writeAheadBlocks = 1 // max background jobs flushing to Keep before blocking writes + concurrentWriters = 4 // max goroutines writing to Keep in background and during flush() ) // A CollectionFileSystem is a FileSystem that can be serialized as a @@ -38,6 +38,9 @@ type CollectionFileSystem interface { // Total data bytes in all files. Size() int64 + + // Memory consumed by buffered file data. + memorySize() int64 } type collectionFileSystem struct { @@ -47,16 +50,15 @@ type collectionFileSystem struct { // FileSystem returns a CollectionFileSystem for the collection. func (c *Collection) FileSystem(client apiClient, kc keepClient) (CollectionFileSystem, error) { - var modTime time.Time - if c.ModifiedAt == nil { + modTime := c.ModifiedAt + if modTime.IsZero() { modTime = time.Now() - } else { - modTime = *c.ModifiedAt } fs := &collectionFileSystem{ uuid: c.UUID, fileSystem: fileSystem{ fsBackend: keepBackend{apiClient: client, keepClient: kc}, + thr: newThrottle(concurrentWriters), }, } root := &dirnode{ @@ -119,6 +121,62 @@ func (fs *collectionFileSystem) newNode(name string, perm os.FileMode, modTime t } } +func (fs *collectionFileSystem) Child(name string, replace func(inode) (inode, error)) (inode, error) { + return fs.rootnode().Child(name, replace) +} + +func (fs *collectionFileSystem) FS() FileSystem { + return fs +} + +func (fs *collectionFileSystem) FileInfo() os.FileInfo { + return fs.rootnode().FileInfo() +} + +func (fs *collectionFileSystem) IsDir() bool { + return true +} + +func (fs *collectionFileSystem) Lock() { + fs.rootnode().Lock() +} + +func (fs *collectionFileSystem) Unlock() { + fs.rootnode().Unlock() +} + +func (fs *collectionFileSystem) RLock() { + fs.rootnode().RLock() +} + +func (fs *collectionFileSystem) RUnlock() { + fs.rootnode().RUnlock() +} + +func (fs *collectionFileSystem) Parent() inode { + return fs.rootnode().Parent() +} + +func (fs *collectionFileSystem) Read(_ []byte, ptr filenodePtr) (int, filenodePtr, error) { + return 0, ptr, ErrInvalidOperation +} + +func (fs *collectionFileSystem) Write(_ []byte, ptr filenodePtr) (int, filenodePtr, error) { + return 0, ptr, ErrInvalidOperation +} + +func (fs *collectionFileSystem) Readdir() ([]os.FileInfo, error) { + return fs.rootnode().Readdir() +} + +func (fs *collectionFileSystem) SetParent(parent inode, name string) { + fs.rootnode().SetParent(parent, name) +} + +func (fs *collectionFileSystem) Truncate(int64) error { + return ErrInvalidOperation +} + func (fs *collectionFileSystem) Sync() error { if fs.uuid == "" { return nil @@ -131,17 +189,60 @@ func (fs *collectionFileSystem) Sync() error { UUID: fs.uuid, ManifestText: txt, } - err = fs.RequestAndDecode(nil, "PUT", "arvados/v1/collections/"+fs.uuid, fs.UpdateBody(coll), map[string]interface{}{"select": []string{"uuid"}}) + err = fs.RequestAndDecode(nil, "PUT", "arvados/v1/collections/"+fs.uuid, nil, map[string]interface{}{ + "collection": map[string]string{ + "manifest_text": coll.ManifestText, + }, + "select": []string{"uuid"}, + }) if err != nil { return fmt.Errorf("sync failed: update %s: %s", fs.uuid, err) } return nil } +func (fs *collectionFileSystem) Flush(path string, shortBlocks bool) error { + node, err := rlookup(fs.fileSystem.root, path) + if err != nil { + return err + } + dn, ok := node.(*dirnode) + if !ok { + return ErrNotADirectory + } + dn.Lock() + defer dn.Unlock() + names := dn.sortedNames() + if path != "" { + // Caller only wants to flush the specified dir, + // non-recursively. Drop subdirs from the list of + // names. + var filenames []string + for _, name := range names { + if _, ok := dn.inodes[name].(*filenode); ok { + filenames = append(filenames, name) + } + } + names = filenames + } + for _, name := range names { + child := dn.inodes[name] + child.Lock() + defer child.Unlock() + } + return dn.flush(context.TODO(), names, flushOpts{sync: false, shortBlocks: shortBlocks}) +} + +func (fs *collectionFileSystem) memorySize() int64 { + fs.fileSystem.root.Lock() + defer fs.fileSystem.root.Unlock() + return fs.fileSystem.root.(*dirnode).memorySize() +} + func (fs *collectionFileSystem) MarshalManifest(prefix string) (string, error) { fs.fileSystem.root.Lock() defer fs.fileSystem.root.Unlock() - return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix, newThrottle(concurrentWriters)) + return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix) } func (fs *collectionFileSystem) Size() int64 { @@ -233,7 +334,6 @@ type filenode struct { memsize int64 // bytes in memSegments sync.RWMutex nullnode - throttle *throttle } // caller must have lock @@ -468,8 +568,6 @@ func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePt seg.Truncate(len(cando)) fn.memsize += int64(len(cando)) fn.segments[cur] = seg - cur++ - prev++ } } @@ -496,12 +594,8 @@ func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePt // Write some data out to disk to reduce memory use. Caller must have // write lock. func (fn *filenode) pruneMemSegments() { - // TODO: share code with (*dirnode)sync() + // TODO: share code with (*dirnode)flush() // TODO: pack/flush small blocks too, when fragmented - if fn.throttle == nil { - // TODO: share a throttle with filesystem - fn.throttle = newThrottle(writeAheadBlocks) - } for idx, seg := range fn.segments { seg, ok := seg.(*memSegment) if !ok || seg.Len() < maxBlockSize || seg.flushing != nil { @@ -517,18 +611,17 @@ func (fn *filenode) pruneMemSegments() { // progress, block here until one finishes, rather // than pile up an unlimited number of buffered writes // and network flush operations. - fn.throttle.Acquire() + fn.fs.throttle().Acquire() go func() { defer close(done) locator, _, err := fn.FS().PutB(buf) - fn.throttle.Release() + fn.fs.throttle().Release() fn.Lock() defer fn.Unlock() - if curbuf := seg.buf[:1]; &curbuf[0] != &buf[0] { + if seg.flushing != done { // A new seg.buf has been allocated. return } - seg.flushing = nil if err != nil { // TODO: stall (or return errors from) // subsequent writes until flushing @@ -551,8 +644,8 @@ func (fn *filenode) pruneMemSegments() { } } -// Block until all pending pruneMemSegments work is finished. Caller -// must NOT have lock. +// Block until all pending pruneMemSegments/flush work is +// finished. Caller must NOT have lock. func (fn *filenode) waitPrune() { var pending []<-chan struct{} fn.Lock() @@ -608,51 +701,131 @@ type fnSegmentRef struct { // storedSegments that reference the relevant portions of the new // block. // +// bufsize is the total data size in refs. It is used to preallocate +// the correct amount of memory when len(refs)>1. +// +// If sync is false, commitBlock returns right away, after starting a +// goroutine to do the writes, reacquire the filenodes' locks, and +// swap out the *memSegments. Some filenodes' segments might get +// modified/rearranged in the meantime, in which case commitBlock +// won't replace them. +// // Caller must have write lock. -func (dn *dirnode) commitBlock(ctx context.Context, throttle *throttle, refs []fnSegmentRef) error { +func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, bufsize int, sync bool) error { if len(refs) == 0 { return nil } - throttle.Acquire() - defer throttle.Release() if err := ctx.Err(); err != nil { return err } - block := make([]byte, 0, maxBlockSize) + done := make(chan struct{}) + var block []byte + segs := make([]*memSegment, 0, len(refs)) + offsets := make([]int, 0, len(refs)) // location of segment's data within block for _, ref := range refs { - block = append(block, ref.fn.segments[ref.idx].(*memSegment).buf...) - } - locator, _, err := dn.fs.PutB(block) - if err != nil { - return err + seg := ref.fn.segments[ref.idx].(*memSegment) + if !sync && seg.flushingUnfinished() { + // Let the other flushing goroutine finish. If + // it fails, we'll try again next time. + close(done) + return nil + } else { + // In sync mode, we proceed regardless of + // whether another flush is in progress: It + // can't finish before we do, because we hold + // fn's lock until we finish our own writes. + } + seg.flushing = done + offsets = append(offsets, len(block)) + if len(refs) == 1 { + block = seg.buf + } else if block == nil { + block = append(make([]byte, 0, bufsize), seg.buf...) + } else { + block = append(block, seg.buf...) + } + segs = append(segs, seg) } - off := 0 - for _, ref := range refs { - data := ref.fn.segments[ref.idx].(*memSegment).buf - ref.fn.segments[ref.idx] = storedSegment{ - kc: dn.fs, - locator: locator, - size: len(block), - offset: off, - length: len(data), + blocksize := len(block) + dn.fs.throttle().Acquire() + errs := make(chan error, 1) + go func() { + defer close(done) + defer close(errs) + locator, _, err := dn.fs.PutB(block) + dn.fs.throttle().Release() + if err != nil { + errs <- err + return + } + for idx, ref := range refs { + if !sync { + ref.fn.Lock() + // In async mode, fn's lock was + // released while we were waiting for + // PutB(); lots of things might have + // changed. + if len(ref.fn.segments) <= ref.idx { + // file segments have + // rearranged or changed in + // some way + ref.fn.Unlock() + continue + } else if seg, ok := ref.fn.segments[ref.idx].(*memSegment); !ok || seg != segs[idx] { + // segment has been replaced + ref.fn.Unlock() + continue + } else if seg.flushing != done { + // seg.buf has been replaced + ref.fn.Unlock() + continue + } + } + data := ref.fn.segments[ref.idx].(*memSegment).buf + ref.fn.segments[ref.idx] = storedSegment{ + kc: dn.fs, + locator: locator, + size: blocksize, + offset: offsets[idx], + length: len(data), + } + // atomic is needed here despite caller having + // lock: caller might be running concurrent + // commitBlock() goroutines using the same + // lock, writing different segments from the + // same file. + atomic.AddInt64(&ref.fn.memsize, -int64(len(data))) + if !sync { + ref.fn.Unlock() + } } - off += len(data) - ref.fn.memsize -= int64(len(data)) + }() + if sync { + return <-errs + } else { + return nil } - return nil } -// sync flushes in-memory data and remote block references (for the +type flushOpts struct { + sync bool + shortBlocks bool +} + +// flush in-memory data and remote-cluster block references (for the // children with the given names, which must be children of dn) to -// local persistent storage. Caller must have write lock on dn and the -// named children. -func (dn *dirnode) sync(ctx context.Context, throttle *throttle, names []string) error { +// local-cluster persistent storage. +// +// Caller must have write lock on dn and the named children. +// +// If any children are dirs, they will be flushed recursively. +func (dn *dirnode) flush(ctx context.Context, names []string, opts flushOpts) error { cg := newContextGroup(ctx) defer cg.Cancel() - goCommit := func(refs []fnSegmentRef) { + goCommit := func(refs []fnSegmentRef, bufsize int) { cg.Go(func() error { - return dn.commitBlock(cg.Context(), throttle, refs) + return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync) }) } @@ -660,47 +833,87 @@ func (dn *dirnode) sync(ctx context.Context, throttle *throttle, names []string) var pendingLen int = 0 localLocator := map[string]string{} for _, name := range names { - fn, ok := dn.inodes[name].(*filenode) - if !ok { - continue - } - for idx, seg := range fn.segments { - switch seg := seg.(type) { - case storedSegment: - loc, ok := localLocator[seg.locator] - if !ok { - var err error - loc, err = dn.fs.LocalLocator(seg.locator) - if err != nil { - return err + switch node := dn.inodes[name].(type) { + case *dirnode: + grandchildNames := node.sortedNames() + for _, grandchildName := range grandchildNames { + grandchild := node.inodes[grandchildName] + grandchild.Lock() + defer grandchild.Unlock() + } + cg.Go(func() error { return node.flush(cg.Context(), grandchildNames, opts) }) + case *filenode: + for idx, seg := range node.segments { + switch seg := seg.(type) { + case storedSegment: + loc, ok := localLocator[seg.locator] + if !ok { + var err error + loc, err = dn.fs.LocalLocator(seg.locator) + if err != nil { + return err + } + localLocator[seg.locator] = loc } - localLocator[seg.locator] = loc - } - seg.locator = loc - fn.segments[idx] = seg - case *memSegment: - if seg.Len() > maxBlockSize/2 { - goCommit([]fnSegmentRef{{fn, idx}}) - continue - } - if pendingLen+seg.Len() > maxBlockSize { - goCommit(pending) - pending = nil - pendingLen = 0 + seg.locator = loc + node.segments[idx] = seg + case *memSegment: + if seg.Len() > maxBlockSize/2 { + goCommit([]fnSegmentRef{{node, idx}}, seg.Len()) + continue + } + if pendingLen+seg.Len() > maxBlockSize { + goCommit(pending, pendingLen) + pending = nil + pendingLen = 0 + } + pending = append(pending, fnSegmentRef{node, idx}) + pendingLen += seg.Len() + default: + panic(fmt.Sprintf("can't sync segment type %T", seg)) } - pending = append(pending, fnSegmentRef{fn, idx}) - pendingLen += seg.Len() - default: - panic(fmt.Sprintf("can't sync segment type %T", seg)) } } } - goCommit(pending) + if opts.shortBlocks { + goCommit(pending, pendingLen) + } return cg.Wait() } // caller must have write lock. -func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle *throttle) (string, error) { +func (dn *dirnode) memorySize() (size int64) { + for _, name := range dn.sortedNames() { + node := dn.inodes[name] + node.Lock() + defer node.Unlock() + switch node := node.(type) { + case *dirnode: + size += node.memorySize() + case *filenode: + for _, seg := range node.segments { + switch seg := seg.(type) { + case *memSegment: + size += int64(seg.Len()) + } + } + } + } + return +} + +// caller must have write lock. +func (dn *dirnode) sortedNames() []string { + names := make([]string, 0, len(dn.inodes)) + for name := range dn.inodes { + names = append(names, name) + } + sort.Strings(names) + return names +} + +// caller must have write lock. +func (dn *dirnode) marshalManifest(ctx context.Context, prefix string) (string, error) { cg := newContextGroup(ctx) defer cg.Cancel() @@ -715,11 +928,7 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle return manifestEscape(prefix) + " d41d8cd98f00b204e9800998ecf8427e+0 0:0:\\056\n", nil } - names := make([]string, 0, len(dn.inodes)) - for name := range dn.inodes { - names = append(names, name) - } - sort.Strings(names) + names := dn.sortedNames() // Wait for children to finish any pending write operations // before locking them. @@ -751,7 +960,7 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle for i, name := range dirnames { i, name := i, name cg.Go(func() error { - txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name, throttle) + txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name) subdirs[i] = txt return err }) @@ -767,7 +976,7 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle var fileparts []filepart var blocks []string - if err := dn.sync(cg.Context(), throttle, names); err != nil { + if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil { return err } for _, name := range filenames { @@ -800,7 +1009,7 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle default: // This can't happen: we // haven't unlocked since - // calling sync(). + // calling flush(sync=true). panic(fmt.Sprintf("can't marshal segment type %T", seg)) } } @@ -898,9 +1107,9 @@ func (dn *dirnode) loadManifest(txt string) error { // situation might be rare anyway) segIdx, pos = 0, 0 } - for next := int64(0); segIdx < len(segments); segIdx++ { + for ; segIdx < len(segments); segIdx++ { seg := segments[segIdx] - next = pos + int64(seg.Len()) + next := pos + int64(seg.Len()) if next <= offset || seg.Len() == 0 { pos = next continue @@ -1034,13 +1243,26 @@ type segment interface { type memSegment struct { buf []byte - // If flushing is not nil, then a) buf is being shared by a - // pruneMemSegments goroutine, and must be copied on write; - // and b) the flushing channel will close when the goroutine - // finishes, whether it succeeds or not. + // If flushing is not nil and not ready/closed, then a) buf is + // being shared by a pruneMemSegments goroutine, and must be + // copied on write; and b) the flushing channel will close + // when the goroutine finishes, whether it succeeds or not. flushing <-chan struct{} } +func (me *memSegment) flushingUnfinished() bool { + if me.flushing == nil { + return false + } + select { + case <-me.flushing: + me.flushing = nil + return false + default: + return true + } +} + func (me *memSegment) Len() int { return len(me.buf) }