16328: Merge branch 'master' into 16328-keep-proxy-uses-config.yaml-to-find-keepstores
[arvados.git] / sdk / go / arvados / fs_collection.go
index 543d89385b01f1df43b4576107a6e733cf33a0a0..37bd494914df507dc9fc193576dc9e0afcc98ea9 100644 (file)
@@ -16,13 +16,13 @@ import (
        "strconv"
        "strings"
        "sync"
+       "sync/atomic"
        "time"
 )
 
 var (
        maxBlockSize      = 1 << 26
-       concurrentWriters = 4 // max goroutines writing to Keep during flush()
-       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
@@ -50,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{
@@ -146,11 +145,36 @@ func (fs *collectionFileSystem) Sync() error {
        return nil
 }
 
-func (fs *collectionFileSystem) Flush(shortBlocks bool) error {
-       fs.fileSystem.root.Lock()
-       defer fs.fileSystem.root.Unlock()
-       dn := fs.fileSystem.root.(*dirnode)
-       return dn.flush(context.TODO(), newThrottle(concurrentWriters), dn.sortedNames(), flushOpts{sync: false, shortBlocks: shortBlocks})
+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 {
@@ -162,7 +186,7 @@ func (fs *collectionFileSystem) memorySize() int64 {
 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 {
@@ -254,7 +278,6 @@ type filenode struct {
        memsize  int64 // bytes in memSegments
        sync.RWMutex
        nullnode
-       throttle *throttle
 }
 
 // caller must have lock
@@ -519,10 +542,6 @@ func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePt
 func (fn *filenode) pruneMemSegments() {
        // 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 {
@@ -538,18 +557,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 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
@@ -629,6 +647,9 @@ 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
@@ -636,19 +657,23 @@ type fnSegmentRef struct {
 // won't replace them.
 //
 // Caller must have write lock.
-func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, sync bool) error {
+func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, bufsize int, sync bool) error {
+       if len(refs) == 0 {
+               return nil
+       }
        if err := ctx.Err(); err != nil {
                return err
        }
        done := make(chan struct{})
-       block := make([]byte, 0, maxBlockSize)
+       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 {
                seg := ref.fn.segments[ref.idx].(*memSegment)
-               if seg.flushing != nil && !sync {
+               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
@@ -658,39 +683,30 @@ func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, sync bo
                }
                seg.flushing = done
                offsets = append(offsets, len(block))
-               block = append(block, seg.buf...)
+               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)
        }
+       blocksize := len(block)
+       dn.fs.throttle().Acquire()
        errs := make(chan error, 1)
        go func() {
                defer close(done)
                defer close(errs)
-               locked := map[*filenode]bool{}
                locator, _, err := dn.fs.PutB(block)
-               {
-                       if !sync {
-                               for _, name := range dn.sortedNames() {
-                                       if fn, ok := dn.inodes[name].(*filenode); ok {
-                                               fn.Lock()
-                                               defer fn.Unlock()
-                                               locked[fn] = true
-                                       }
-                               }
-                       }
-                       defer func() {
-                               for _, seg := range segs {
-                                       if seg.flushing == done {
-                                               seg.flushing = nil
-                                       }
-                               }
-                       }()
-               }
+               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
@@ -699,17 +715,15 @@ func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, sync bo
                                        // 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
-                                       continue
-                               } else if !locked[ref.fn] {
-                                       // file was renamed, moved, or
-                                       // deleted since we called
-                                       // PutB
+                                       ref.fn.Unlock()
                                        continue
                                }
                        }
@@ -717,11 +731,19 @@ func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, sync bo
                        ref.fn.segments[ref.idx] = storedSegment{
                                kc:      dn.fs,
                                locator: locator,
-                               size:    len(block),
+                               size:    blocksize,
                                offset:  offsets[idx],
                                length:  len(data),
                        }
-                       ref.fn.memsize -= int64(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()
+                       }
                }
        }()
        if sync {
@@ -743,18 +765,13 @@ type flushOpts struct {
 // 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, throttle *throttle, names []string, opts flushOpts) error {
+func (dn *dirnode) flush(ctx context.Context, names []string, opts flushOpts) error {
        cg := newContextGroup(ctx)
        defer cg.Cancel()
 
-       goCommit := func(refs []fnSegmentRef) {
-               if len(refs) == 0 {
-                       return
-               }
+       goCommit := func(refs []fnSegmentRef, bufsize int) {
                cg.Go(func() error {
-                       throttle.Acquire()
-                       defer throttle.Release()
-                       return dn.commitBlock(cg.Context(), refs, opts.sync)
+                       return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync)
                })
        }
 
@@ -770,7 +787,7 @@ func (dn *dirnode) flush(ctx context.Context, throttle *throttle, names []string
                                grandchild.Lock()
                                defer grandchild.Unlock()
                        }
-                       cg.Go(func() error { return node.flush(cg.Context(), throttle, grandchildNames, opts) })
+                       cg.Go(func() error { return node.flush(cg.Context(), grandchildNames, opts) })
                case *filenode:
                        for idx, seg := range node.segments {
                                switch seg := seg.(type) {
@@ -788,11 +805,11 @@ func (dn *dirnode) flush(ctx context.Context, throttle *throttle, names []string
                                        node.segments[idx] = seg
                                case *memSegment:
                                        if seg.Len() > maxBlockSize/2 {
-                                               goCommit([]fnSegmentRef{{node, idx}})
+                                               goCommit([]fnSegmentRef{{node, idx}}, seg.Len())
                                                continue
                                        }
                                        if pendingLen+seg.Len() > maxBlockSize {
-                                               goCommit(pending)
+                                               goCommit(pending, pendingLen)
                                                pending = nil
                                                pendingLen = 0
                                        }
@@ -805,7 +822,7 @@ func (dn *dirnode) flush(ctx context.Context, throttle *throttle, names []string
                }
        }
        if opts.shortBlocks {
-               goCommit(pending)
+               goCommit(pending, pendingLen)
        }
        return cg.Wait()
 }
@@ -842,7 +859,7 @@ func (dn *dirnode) sortedNames() []string {
 }
 
 // caller must have write lock.
-func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle *throttle) (string, error) {
+func (dn *dirnode) marshalManifest(ctx context.Context, prefix string) (string, error) {
        cg := newContextGroup(ctx)
        defer cg.Cancel()
 
@@ -889,7 +906,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
                })
@@ -905,7 +922,7 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle
 
                var fileparts []filepart
                var blocks []string
-               if err := dn.flush(cg.Context(), throttle, filenames, flushOpts{sync: true, shortBlocks: true}); err != nil {
+               if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil {
                        return err
                }
                for _, name := range filenames {
@@ -1172,13 +1189,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)
 }