19088: Export collection/project properties as x-amz-meta tags.
[arvados.git] / sdk / go / arvados / fs_collection.go
index 7ce37aa24e7b35bfbabec9508af3b2e308d4cc76..d3af92f9e1649957050f4f62fffae6a4c36e31a9 100644 (file)
@@ -5,10 +5,11 @@
 package arvados
 
 import (
+       "bytes"
+       "context"
        "encoding/json"
        "fmt"
        "io"
-       "log"
        "os"
        "path"
        "regexp"
@@ -16,10 +17,14 @@ import (
        "strconv"
        "strings"
        "sync"
+       "sync/atomic"
        "time"
 )
 
-var maxBlockSize = 1 << 26
+var (
+       maxBlockSize      = 1 << 26
+       concurrentWriters = 4 // max goroutines writing to Keep in background and during flush()
+)
 
 // A CollectionFileSystem is a FileSystem that can be serialized as a
 // manifest and stored as a collection.
@@ -31,27 +36,48 @@ type CollectionFileSystem interface {
        // Prefix (normally ".") is a top level directory, effectively
        // prepended to all paths in the returned manifest.
        MarshalManifest(prefix string) (string, error)
+
+       // Total data bytes in all files.
+       Size() int64
 }
 
 type collectionFileSystem struct {
        fileSystem
-       uuid string
+       uuid           string
+       savedPDH       atomic.Value
+       replicas       int
+       storageClasses []string
+       // guessSignatureTTL tracks a lower bound for the server's
+       // configured BlobSigningTTL. The guess is initially zero, and
+       // increases when we come across a signature with an expiry
+       // time further in the future than the previous guess.
+       //
+       // When the guessed TTL is much smaller than the real TTL,
+       // preemptive signature refresh is delayed or missed entirely,
+       // which is OK.
+       guessSignatureTTL time.Duration
+       holdCheckChanges  time.Time
+       lockCheckChanges  sync.Mutex
 }
 
 // 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,
+               uuid:           c.UUID,
+               storageClasses: c.StorageClassesDesired,
                fileSystem: fileSystem{
                        fsBackend: keepBackend{apiClient: client, keepClient: kc},
+                       thr:       newThrottle(concurrentWriters),
                },
        }
+       fs.savedPDH.Store(c.PortableDataHash)
+       if r := c.ReplicationDesired; r != nil {
+               fs.replicas = *r
+       }
        root := &dirnode{
                fs: fs,
                treenode: treenode{
@@ -59,6 +85,7 @@ func (c *Collection) FileSystem(client apiClient, kc keepClient) (CollectionFile
                                name:    ".",
                                mode:    os.ModeDir | 0755,
                                modTime: modTime,
+                               sys:     c,
                        },
                        inodes: make(map[string]inode),
                },
@@ -72,16 +99,112 @@ func (c *Collection) FileSystem(client apiClient, kc keepClient) (CollectionFile
        return fs, nil
 }
 
-func backdateTree(n inode, modTime time.Time) {
+// caller must have lock (or guarantee no concurrent accesses somehow)
+func eachNode(n inode, ffunc func(*filenode), dfunc func(*dirnode)) {
        switch n := n.(type) {
        case *filenode:
-               n.fileinfo.modTime = modTime
+               if ffunc != nil {
+                       ffunc(n)
+               }
        case *dirnode:
-               n.fileinfo.modTime = modTime
+               if dfunc != nil {
+                       dfunc(n)
+               }
                for _, n := range n.inodes {
-                       backdateTree(n, modTime)
+                       eachNode(n, ffunc, dfunc)
+               }
+       }
+}
+
+// caller must have lock (or guarantee no concurrent accesses somehow)
+func backdateTree(n inode, modTime time.Time) {
+       eachNode(n, func(fn *filenode) {
+               fn.fileinfo.modTime = modTime
+       }, func(dn *dirnode) {
+               dn.fileinfo.modTime = modTime
+       })
+}
+
+// Approximate portion of signature TTL remaining, usually between 0
+// and 1, or negative if some signatures have expired.
+func (fs *collectionFileSystem) signatureTimeLeft() (float64, time.Duration) {
+       var (
+               now      = time.Now()
+               earliest = now.Add(time.Hour * 24 * 7 * 365)
+               latest   time.Time
+       )
+       fs.fileSystem.root.RLock()
+       eachNode(fs.root, func(fn *filenode) {
+               fn.Lock()
+               defer fn.Unlock()
+               for _, seg := range fn.segments {
+                       seg, ok := seg.(storedSegment)
+                       if !ok {
+                               continue
+                       }
+                       expiryTime, err := signatureExpiryTime(seg.locator)
+                       if err != nil {
+                               continue
+                       }
+                       if expiryTime.Before(earliest) {
+                               earliest = expiryTime
+                       }
+                       if expiryTime.After(latest) {
+                               latest = expiryTime
+                       }
                }
+       }, nil)
+       fs.fileSystem.root.RUnlock()
+
+       if latest.IsZero() {
+               // No signatures == 100% of TTL remaining.
+               return 1, 1
        }
+
+       ttl := latest.Sub(now)
+       fs.fileSystem.root.Lock()
+       {
+               if ttl > fs.guessSignatureTTL {
+                       // ttl is closer to the real TTL than
+                       // guessSignatureTTL.
+                       fs.guessSignatureTTL = ttl
+               } else {
+                       // Use the previous best guess to compute the
+                       // portion remaining (below, after unlocking
+                       // mutex).
+                       ttl = fs.guessSignatureTTL
+               }
+       }
+       fs.fileSystem.root.Unlock()
+
+       return earliest.Sub(now).Seconds() / ttl.Seconds(), ttl
+}
+
+func (fs *collectionFileSystem) updateSignatures(newmanifest string) {
+       newLoc := map[string]string{}
+       for _, tok := range regexp.MustCompile(`\S+`).FindAllString(newmanifest, -1) {
+               if mBlkRe.MatchString(tok) {
+                       newLoc[stripAllHints(tok)] = tok
+               }
+       }
+       fs.fileSystem.root.Lock()
+       defer fs.fileSystem.root.Unlock()
+       eachNode(fs.root, func(fn *filenode) {
+               fn.Lock()
+               defer fn.Unlock()
+               for idx, seg := range fn.segments {
+                       seg, ok := seg.(storedSegment)
+                       if !ok {
+                               continue
+                       }
+                       loc, ok := newLoc[stripAllHints(seg.locator)]
+                       if !ok {
+                               continue
+                       }
+                       seg.locator = loc
+                       fn.segments[idx] = seg
+               }
+       }, nil)
 }
 
 func (fs *collectionFileSystem) newNode(name string, perm os.FileMode, modTime time.Time) (node inode, err error) {
@@ -100,43 +223,247 @@ func (fs *collectionFileSystem) newNode(name string, perm os.FileMode, modTime t
                                inodes: make(map[string]inode),
                        },
                }, nil
-       } else {
-               return &filenode{
-                       fs: fs,
-                       fileinfo: fileinfo{
-                               name:    name,
-                               mode:    perm & ^os.ModeDir,
-                               modTime: modTime,
-                       },
-               }, nil
        }
+       return &filenode{
+               fs: fs,
+               fileinfo: fileinfo{
+                       name:    name,
+                       mode:    perm & ^os.ModeDir,
+                       modTime: modTime,
+               },
+       }, nil
+}
+
+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
+}
+
+// Check for and incorporate upstream changes -- unless that has
+// already been done recently, in which case this func is a no-op.
+func (fs *collectionFileSystem) checkChangesOnServer() error {
+       if fs.uuid == "" && fs.savedPDH.Load() == "" {
+               return nil
+       }
+
+       // First try UUID if any, then last known PDH. Stop if all
+       // signatures are new enough.
+       checkingAll := false
+       for _, id := range []string{fs.uuid, fs.savedPDH.Load().(string)} {
+               if id == "" {
+                       continue
+               }
+
+               fs.lockCheckChanges.Lock()
+               if !checkingAll && fs.holdCheckChanges.After(time.Now()) {
+                       fs.lockCheckChanges.Unlock()
+                       return nil
+               }
+               remain, ttl := fs.signatureTimeLeft()
+               if remain > 0.01 && !checkingAll {
+                       fs.holdCheckChanges = time.Now().Add(ttl / 100)
+               }
+               fs.lockCheckChanges.Unlock()
+
+               if remain >= 0.5 {
+                       break
+               }
+               checkingAll = true
+               var coll Collection
+               err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+id, nil, map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}})
+               if err != nil {
+                       continue
+               }
+               fs.updateSignatures(coll.ManifestText)
+       }
+       return nil
+}
+
+// Refresh signature on a single locator, if necessary. Assume caller
+// has lock. If an update is needed, and there are any storedSegments
+// whose signatures can be updated, start a background task to update
+// them asynchronously when the caller releases locks.
+func (fs *collectionFileSystem) refreshSignature(locator string) string {
+       exp, err := signatureExpiryTime(locator)
+       if err != nil || exp.Sub(time.Now()) > time.Minute {
+               // Synchronous update is not needed. Start an
+               // asynchronous update if needed.
+               go fs.checkChangesOnServer()
+               return locator
+       }
+       var manifests string
+       for _, id := range []string{fs.uuid, fs.savedPDH.Load().(string)} {
+               if id == "" {
+                       continue
+               }
+               var coll Collection
+               err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+id, nil, map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}})
+               if err != nil {
+                       continue
+               }
+               manifests += coll.ManifestText
+       }
+       hash := stripAllHints(locator)
+       for _, tok := range regexp.MustCompile(`\S+`).FindAllString(manifests, -1) {
+               if mBlkRe.MatchString(tok) {
+                       if stripAllHints(tok) == hash {
+                               locator = tok
+                               break
+                       }
+               }
+       }
+       go fs.updateSignatures(manifests)
+       return locator
 }
 
 func (fs *collectionFileSystem) Sync() error {
-       log.Printf("cfs.Sync()")
+       err := fs.checkChangesOnServer()
+       if err != nil {
+               return err
+       }
        if fs.uuid == "" {
                return nil
        }
        txt, err := fs.MarshalManifest(".")
        if err != nil {
-               log.Printf("WARNING: (collectionFileSystem)Sync() failed: %s", err)
-               return err
+               return fmt.Errorf("sync failed: %s", err)
        }
-       coll := &Collection{
+       if PortableDataHash(txt) == fs.savedPDH.Load() {
+               // No local changes since last save or initial load.
+               return nil
+       }
+       coll := Collection{
                UUID:         fs.uuid,
                ManifestText: txt,
        }
-       err = fs.RequestAndDecode(nil, "PUT", "arvados/v1/collections/"+fs.uuid, fs.UpdateBody(coll), map[string]interface{}{"select": []string{"uuid"}})
+
+       selectFields := []string{"uuid", "portable_data_hash"}
+       fs.lockCheckChanges.Lock()
+       remain, _ := fs.signatureTimeLeft()
+       fs.lockCheckChanges.Unlock()
+       if remain < 0.5 {
+               selectFields = append(selectFields, "manifest_text")
+       }
+
+       err = fs.RequestAndDecode(&coll, "PUT", "arvados/v1/collections/"+fs.uuid, nil, map[string]interface{}{
+               "collection": map[string]string{
+                       "manifest_text": coll.ManifestText,
+               },
+               "select": selectFields,
+       })
        if err != nil {
-               log.Printf("WARNING: (collectionFileSystem)Sync() failed: %s", err)
+               return fmt.Errorf("sync failed: update %s: %s", fs.uuid, err)
        }
-       return err
+       fs.updateSignatures(coll.ManifestText)
+       fs.savedPDH.Store(coll.PortableDataHash)
+       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(prefix)
+       return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix)
+}
+
+func (fs *collectionFileSystem) Size() int64 {
+       return fs.fileSystem.root.(*dirnode).TreeSize()
+}
+
+func (fs *collectionFileSystem) Snapshot() (inode, error) {
+       return fs.fileSystem.root.Snapshot()
+}
+
+func (fs *collectionFileSystem) Splice(r inode) error {
+       return fs.fileSystem.root.Splice(r)
 }
 
 // filenodePtr is an offset into a file that is (usually) efficient to
@@ -215,7 +542,7 @@ func (fn *filenode) seek(startPtr filenodePtr) (ptr filenodePtr) {
 // filenode implements inode.
 type filenode struct {
        parent   inode
-       fs       FileSystem
+       fs       *collectionFileSystem
        fileinfo fileinfo
        segments []segment
        // number of times `segments` has changed in a
@@ -262,6 +589,10 @@ func (fn *filenode) Read(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr
                err = io.EOF
                return
        }
+       if ss, ok := fn.segments[ptr.segmentIdx].(storedSegment); ok {
+               ss.locator = fn.fs.refreshSignature(ss.locator)
+               fn.segments[ptr.segmentIdx] = ss
+       }
        n, err = fn.segments[ptr.segmentIdx].ReadAt(p, int64(ptr.segmentOff))
        if n > 0 {
                ptr.off += int64(n)
@@ -458,8 +789,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++
                        }
                }
 
@@ -486,30 +815,115 @@ 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: async (don't hold Lock() while waiting for Keep)
-       // TODO: share code with (*dirnode)sync()
+       // TODO: share code with (*dirnode)flush()
        // TODO: pack/flush small blocks too, when fragmented
        for idx, seg := range fn.segments {
                seg, ok := seg.(*memSegment)
-               if !ok || seg.Len() < maxBlockSize {
-                       continue
-               }
-               locator, _, err := fn.FS().PutB(seg.buf)
-               if err != nil {
-                       // TODO: stall (or return errors from)
-                       // subsequent writes until flushing
-                       // starts to succeed
+               if !ok || seg.Len() < maxBlockSize || seg.flushing != nil {
                        continue
                }
-               fn.memsize -= int64(seg.Len())
-               fn.segments[idx] = storedSegment{
-                       kc:      fn.FS(),
-                       locator: locator,
-                       size:    seg.Len(),
-                       offset:  0,
-                       length:  seg.Len(),
+               // Setting seg.flushing guarantees seg.buf will not be
+               // modified in place: WriteAt and Truncate will
+               // allocate a new buf instead, if necessary.
+               idx, buf := idx, seg.buf
+               done := make(chan struct{})
+               seg.flushing = done
+               // If lots of background writes are already in
+               // progress, block here until one finishes, rather
+               // than pile up an unlimited number of buffered writes
+               // and network flush operations.
+               fn.fs.throttle().Acquire()
+               go func() {
+                       defer close(done)
+                       resp, err := fn.FS().BlockWrite(context.Background(), BlockWriteOptions{
+                               Data:           buf,
+                               Replicas:       fn.fs.replicas,
+                               StorageClasses: fn.fs.storageClasses,
+                       })
+                       fn.fs.throttle().Release()
+                       fn.Lock()
+                       defer fn.Unlock()
+                       if seg.flushing != done {
+                               // A new seg.buf has been allocated.
+                               return
+                       }
+                       if err != nil {
+                               // TODO: stall (or return errors from)
+                               // subsequent writes until flushing
+                               // starts to succeed.
+                               return
+                       }
+                       if len(fn.segments) <= idx || fn.segments[idx] != seg || len(seg.buf) != len(buf) {
+                               // Segment has been dropped/moved/resized.
+                               return
+                       }
+                       fn.memsize -= int64(len(buf))
+                       fn.segments[idx] = storedSegment{
+                               kc:      fn.FS(),
+                               locator: resp.Locator,
+                               size:    len(buf),
+                               offset:  0,
+                               length:  len(buf),
+                       }
+               }()
+       }
+}
+
+// Block until all pending pruneMemSegments/flush work is
+// finished. Caller must NOT have lock.
+func (fn *filenode) waitPrune() {
+       var pending []<-chan struct{}
+       fn.Lock()
+       for _, seg := range fn.segments {
+               if seg, ok := seg.(*memSegment); ok && seg.flushing != nil {
+                       pending = append(pending, seg.flushing)
                }
        }
+       fn.Unlock()
+       for _, p := range pending {
+               <-p
+       }
+}
+
+func (fn *filenode) Snapshot() (inode, error) {
+       fn.RLock()
+       defer fn.RUnlock()
+       segments := make([]segment, 0, len(fn.segments))
+       for _, seg := range fn.segments {
+               segments = append(segments, seg.Slice(0, seg.Len()))
+       }
+       return &filenode{
+               fileinfo: fn.fileinfo,
+               segments: segments,
+       }, nil
+}
+
+func (fn *filenode) Splice(repl inode) error {
+       repl, err := repl.Snapshot()
+       if err != nil {
+               return err
+       }
+       fn.parent.Lock()
+       defer fn.parent.Unlock()
+       fn.Lock()
+       defer fn.Unlock()
+       _, err = fn.parent.Child(fn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
+       if err != nil {
+               return err
+       }
+       switch repl := repl.(type) {
+       case *dirnode:
+               repl.parent = fn.parent
+               repl.fileinfo.name = fn.fileinfo.name
+               repl.setTreeFS(fn.fs)
+       case *filenode:
+               repl.parent = fn.parent
+               repl.fileinfo.name = fn.fileinfo.name
+               repl.fs = fn.fs
+       default:
+               return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
+       }
+       return nil
 }
 
 type dirnode struct {
@@ -530,6 +944,7 @@ func (dn *dirnode) Child(name string, replace func(inode) (inode, error)) (inode
                        if err != nil {
                                return nil, err
                        }
+                       coll.UUID = dn.fs.uuid
                        data, err := json.Marshal(&coll)
                        if err == nil {
                                data = append(data, '\n')
@@ -542,118 +957,304 @@ func (dn *dirnode) Child(name string, replace func(inode) (inode, error)) (inode
        return dn.treenode.Child(name, replace)
 }
 
-// sync flushes in-memory data (for all files in the tree rooted at
-// dn) to persistent storage. Caller must hold dn.Lock().
-func (dn *dirnode) sync() error {
-       type shortBlock struct {
-               fn  *filenode
-               idx int
-       }
-       var pending []shortBlock
-       var pendingLen int
+type fnSegmentRef struct {
+       fn  *filenode
+       idx int
+}
 
-       flush := func(sbs []shortBlock) error {
-               if len(sbs) == 0 {
+// commitBlock concatenates the data from the given filenode segments
+// (which must be *memSegments), writes the data out to Keep as a
+// single block, and replaces the filenodes' *memSegments with
+// 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, 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{})
+       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 !sync && seg.flushingUnfinished() {
+                       // Let the other flushing goroutine finish. If
+                       // it fails, we'll try again next time.
+                       close(done)
                        return nil
                }
-               block := make([]byte, 0, maxBlockSize)
-               for _, sb := range sbs {
-                       block = append(block, sb.fn.segments[sb.idx].(*memSegment).buf...)
+               // 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...)
                }
-               locator, _, err := dn.fs.PutB(block)
+               segs = append(segs, seg)
+       }
+       blocksize := len(block)
+       dn.fs.throttle().Acquire()
+       errs := make(chan error, 1)
+       go func() {
+               defer close(done)
+               defer close(errs)
+               resp, err := dn.fs.BlockWrite(context.Background(), BlockWriteOptions{
+                       Data:           block,
+                       Replicas:       dn.fs.replicas,
+                       StorageClasses: dn.fs.storageClasses,
+               })
+               dn.fs.throttle().Release()
                if err != nil {
-                       return err
+                       errs <- err
+                       return
                }
-               off := 0
-               for _, sb := range sbs {
-                       data := sb.fn.segments[sb.idx].(*memSegment).buf
-                       sb.fn.segments[sb.idx] = storedSegment{
+               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:    len(block),
-                               offset:  off,
+                               locator: resp.Locator,
+                               size:    blocksize,
+                               offset:  offsets[idx],
                                length:  len(data),
                        }
-                       off += len(data)
-                       sb.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()
+                       }
                }
-               return nil
+       }()
+       if sync {
+               return <-errs
        }
+       return nil
+}
 
-       names := make([]string, 0, len(dn.inodes))
-       for name := range dn.inodes {
-               names = append(names, name)
+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-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, bufsize int) {
+               cg.Go(func() error {
+                       return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync)
+               })
        }
-       sort.Strings(names)
 
+       var pending []fnSegmentRef
+       var pendingLen int = 0
+       localLocator := map[string]string{}
        for _, name := range names {
-               fn, ok := dn.inodes[name].(*filenode)
-               if !ok {
-                       continue
-               }
-               fn.Lock()
-               defer fn.Unlock()
-               for idx, seg := range fn.segments {
-                       seg, ok := seg.(*memSegment)
-                       if !ok {
-                               continue
+               switch node := dn.inodes[name].(type) {
+               case *dirnode:
+                       grandchildNames := node.sortedNames()
+                       for _, grandchildName := range grandchildNames {
+                               grandchild := node.inodes[grandchildName]
+                               grandchild.Lock()
+                               defer grandchild.Unlock()
                        }
-                       if seg.Len() > maxBlockSize/2 {
-                               if err := flush([]shortBlock{{fn, idx}}); err != nil {
-                                       return err
+                       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
+                                       }
+                                       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))
                                }
-                               continue
                        }
-                       if pendingLen+seg.Len() > maxBlockSize {
-                               if err := flush(pending); err != nil {
-                                       return err
+               }
+       }
+       if opts.shortBlocks {
+               goCommit(pending, pendingLen)
+       }
+       return cg.Wait()
+}
+
+// caller must have write lock.
+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:
+                       size += 64
+                       for _, seg := range node.segments {
+                               switch seg := seg.(type) {
+                               case *memSegment:
+                                       size += int64(seg.Len())
                                }
-                               pending = nil
-                               pendingLen = 0
+                               size += 64
                        }
-                       pending = append(pending, shortBlock{fn, idx})
-                       pendingLen += seg.Len()
                }
        }
-       return flush(pending)
+       return 64 + size
 }
 
-// caller must have read lock.
-func (dn *dirnode) marshalManifest(prefix string) (string, error) {
-       var streamLen int64
-       type filepart struct {
-               name   string
-               offset int64
-               length int64
+// 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)
        }
-       var fileparts []filepart
-       var subdirs string
-       var blocks []string
+       sort.Strings(names)
+       return names
+}
 
-       if err := dn.sync(); err != nil {
-               return "", err
+// caller must have write lock.
+func (dn *dirnode) marshalManifest(ctx context.Context, prefix string) (string, error) {
+       cg := newContextGroup(ctx)
+       defer cg.Cancel()
+
+       if len(dn.inodes) == 0 {
+               if prefix == "." {
+                       return "", nil
+               }
+               // Express the existence of an empty directory by
+               // adding an empty file named `\056`, which (unlike
+               // the more obvious spelling `.`) is accepted by the
+               // API's manifest validator.
+               return manifestEscape(prefix) + " d41d8cd98f00b204e9800998ecf8427e+0 0:0:\\056\n", nil
        }
 
-       names := make([]string, 0, len(dn.inodes))
-       for name, node := range dn.inodes {
-               names = append(names, name)
-               node.Lock()
-               defer node.Unlock()
+       names := dn.sortedNames()
+
+       // Wait for children to finish any pending write operations
+       // before locking them.
+       for _, name := range names {
+               node := dn.inodes[name]
+               if fn, ok := node.(*filenode); ok {
+                       fn.waitPrune()
+               }
        }
-       sort.Strings(names)
 
+       var dirnames []string
+       var filenames []string
        for _, name := range names {
-               switch node := dn.inodes[name].(type) {
+               node := dn.inodes[name]
+               node.Lock()
+               defer node.Unlock()
+               switch node := node.(type) {
                case *dirnode:
-                       subdir, err := node.marshalManifest(prefix + "/" + name)
-                       if err != nil {
-                               return "", err
-                       }
-                       subdirs = subdirs + subdir
+                       dirnames = append(dirnames, name)
                case *filenode:
+                       filenames = append(filenames, name)
+               default:
+                       panic(fmt.Sprintf("can't marshal inode type %T", node))
+               }
+       }
+
+       subdirs := make([]string, len(dirnames))
+       rootdir := ""
+       for i, name := range dirnames {
+               i, name := i, name
+               cg.Go(func() error {
+                       txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name)
+                       subdirs[i] = txt
+                       return err
+               })
+       }
+
+       cg.Go(func() error {
+               var streamLen int64
+               type filepart struct {
+                       name   string
+                       offset int64
+                       length int64
+               }
+
+               var fileparts []filepart
+               var blocks []string
+               if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil {
+                       return err
+               }
+               for _, name := range filenames {
+                       node := dn.inodes[name].(*filenode)
                        if len(node.segments) == 0 {
                                fileparts = append(fileparts, filepart{name: name})
-                               break
+                               continue
                        }
                        for _, seg := range node.segments {
                                switch seg := seg.(type) {
@@ -679,59 +1280,86 @@ func (dn *dirnode) marshalManifest(prefix string) (string, error) {
                                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))
                                }
                        }
-               default:
-                       panic(fmt.Sprintf("can't marshal inode type %T", node))
                }
-       }
-       var filetokens []string
-       for _, s := range fileparts {
-               filetokens = append(filetokens, fmt.Sprintf("%d:%d:%s", s.offset, s.length, manifestEscape(s.name)))
-       }
-       if len(filetokens) == 0 {
-               return subdirs, nil
-       } else if len(blocks) == 0 {
-               blocks = []string{"d41d8cd98f00b204e9800998ecf8427e+0"}
-       }
-       return manifestEscape(prefix) + " " + strings.Join(blocks, " ") + " " + strings.Join(filetokens, " ") + "\n" + subdirs, nil
+               var filetokens []string
+               for _, s := range fileparts {
+                       filetokens = append(filetokens, fmt.Sprintf("%d:%d:%s", s.offset, s.length, manifestEscape(s.name)))
+               }
+               if len(filetokens) == 0 {
+                       return nil
+               } else if len(blocks) == 0 {
+                       blocks = []string{"d41d8cd98f00b204e9800998ecf8427e+0"}
+               }
+               rootdir = manifestEscape(prefix) + " " + strings.Join(blocks, " ") + " " + strings.Join(filetokens, " ") + "\n"
+               return nil
+       })
+       err := cg.Wait()
+       return rootdir + strings.Join(subdirs, ""), err
 }
 
 func (dn *dirnode) loadManifest(txt string) error {
-       var dirname string
-       streams := strings.Split(txt, "\n")
-       if streams[len(streams)-1] != "" {
+       streams := bytes.Split([]byte(txt), []byte{'\n'})
+       if len(streams[len(streams)-1]) != 0 {
                return fmt.Errorf("line %d: no trailing newline", len(streams))
        }
        streams = streams[:len(streams)-1]
        segments := []storedSegment{}
+       // To reduce allocs, we reuse a single "pathparts" slice
+       // (pre-split on "/" separators) for the duration of this
+       // func.
+       var pathparts []string
+       // To reduce allocs, we reuse a single "toks" slice of 3 byte
+       // slices.
+       var toks = make([][]byte, 3)
+       // Similar to bytes.SplitN(token, []byte{c}, 3), but splits
+       // into the toks slice rather than allocating a new one, and
+       // returns the number of toks (1, 2, or 3).
+       splitToToks := func(src []byte, c rune) int {
+               c1 := bytes.IndexRune(src, c)
+               if c1 < 0 {
+                       toks[0] = src
+                       return 1
+               }
+               toks[0], src = src[:c1], src[c1+1:]
+               c2 := bytes.IndexRune(src, c)
+               if c2 < 0 {
+                       toks[1] = src
+                       return 2
+               }
+               toks[1], toks[2] = src[:c2], src[c2+1:]
+               return 3
+       }
        for i, stream := range streams {
                lineno := i + 1
                var anyFileTokens bool
                var pos int64
                var segIdx int
                segments = segments[:0]
-               for i, token := range strings.Split(stream, " ") {
+               pathparts = nil
+               streamparts := 0
+               for i, token := range bytes.Split(stream, []byte{' '}) {
                        if i == 0 {
-                               dirname = manifestUnescape(token)
+                               pathparts = strings.Split(manifestUnescape(string(token)), "/")
+                               streamparts = len(pathparts)
                                continue
                        }
-                       if !strings.Contains(token, ":") {
+                       if !bytes.ContainsRune(token, ':') {
                                if anyFileTokens {
                                        return fmt.Errorf("line %d: bad file segment %q", lineno, token)
                                }
-                               toks := strings.SplitN(token, "+", 3)
-                               if len(toks) < 2 {
+                               if splitToToks(token, '+') < 2 {
                                        return fmt.Errorf("line %d: bad locator %q", lineno, token)
                                }
-                               length, err := strconv.ParseInt(toks[1], 10, 32)
+                               length, err := strconv.ParseInt(string(toks[1]), 10, 32)
                                if err != nil || length < 0 {
                                        return fmt.Errorf("line %d: bad locator %q", lineno, token)
                                }
                                segments = append(segments, storedSegment{
-                                       locator: token,
+                                       locator: string(token),
                                        size:    int(length),
                                        offset:  0,
                                        length:  int(length),
@@ -740,25 +1368,34 @@ func (dn *dirnode) loadManifest(txt string) error {
                        } else if len(segments) == 0 {
                                return fmt.Errorf("line %d: bad locator %q", lineno, token)
                        }
-
-                       toks := strings.SplitN(token, ":", 3)
-                       if len(toks) != 3 {
+                       if splitToToks(token, ':') != 3 {
                                return fmt.Errorf("line %d: bad file segment %q", lineno, token)
                        }
                        anyFileTokens = true
 
-                       offset, err := strconv.ParseInt(toks[0], 10, 64)
+                       offset, err := strconv.ParseInt(string(toks[0]), 10, 64)
                        if err != nil || offset < 0 {
                                return fmt.Errorf("line %d: bad file segment %q", lineno, token)
                        }
-                       length, err := strconv.ParseInt(toks[1], 10, 64)
+                       length, err := strconv.ParseInt(string(toks[1]), 10, 64)
                        if err != nil || length < 0 {
                                return fmt.Errorf("line %d: bad file segment %q", lineno, token)
                        }
-                       name := dirname + "/" + manifestUnescape(toks[2])
-                       fnode, err := dn.createFileAndParents(name)
-                       if err != nil {
-                               return fmt.Errorf("line %d: cannot use path %q: %s", lineno, name, err)
+                       if !bytes.ContainsAny(toks[2], `\/`) {
+                               // optimization for a common case
+                               pathparts = append(pathparts[:streamparts], string(toks[2]))
+                       } else {
+                               pathparts = append(pathparts[:streamparts], strings.Split(manifestUnescape(string(toks[2])), "/")...)
+                       }
+                       fnode, err := dn.createFileAndParents(pathparts)
+                       if fnode == nil && err == nil && length == 0 {
+                               // Special case: an empty file used as
+                               // a marker to preserve an otherwise
+                               // empty directory in a manifest.
+                               continue
+                       }
+                       if err != nil || (fnode == nil && length != 0) {
+                               return fmt.Errorf("line %d: cannot use name %q with length %d: %s", lineno, toks[2], length, err)
                        }
                        // Map the stream offset/range coordinates to
                        // block/offset/range coordinates and add
@@ -770,9 +1407,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
@@ -809,22 +1446,23 @@ func (dn *dirnode) loadManifest(txt string) error {
                        return fmt.Errorf("line %d: no file segments", lineno)
                } else if len(segments) == 0 {
                        return fmt.Errorf("line %d: no locators", lineno)
-               } else if dirname == "" {
+               } else if streamparts == 0 {
                        return fmt.Errorf("line %d: no stream name", lineno)
                }
        }
        return nil
 }
 
-// only safe to call from loadManifest -- no locking
-func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) {
+// only safe to call from loadManifest -- no locking.
+//
+// If path is a "parent directory exists" marker (the last path
+// component is "."), the returned values are both nil.
+//
+// Newly added nodes have modtime==0. Caller is responsible for fixing
+// them with backdateTree.
+func (dn *dirnode) createFileAndParents(names []string) (fn *filenode, err error) {
        var node inode = dn
-       names := strings.Split(path, "/")
        basename := names[len(names)-1]
-       if !permittedName(basename) {
-               err = fmt.Errorf("invalid file part %q in path %q", basename, path)
-               return
-       }
        for _, name := range names[:len(names)-1] {
                switch name {
                case "", ".":
@@ -837,9 +1475,12 @@ func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) {
                        node = node.Parent()
                        continue
                }
+               node.Lock()
+               unlock := node.Unlock
                node, err = node.Child(name, func(child inode) (inode, error) {
                        if child == nil {
-                               child, err := node.FS().newNode(name, 0755|os.ModeDir, node.Parent().FileInfo().ModTime())
+                               // note modtime will be fixed later in backdateTree()
+                               child, err := node.FS().newNode(name, 0755|os.ModeDir, time.Time{})
                                if err != nil {
                                        return nil, err
                                }
@@ -851,14 +1492,23 @@ func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) {
                                return child, nil
                        }
                })
+               unlock()
                if err != nil {
                        return
                }
        }
+       if basename == "." {
+               return
+       } else if !permittedName(basename) {
+               err = fmt.Errorf("invalid file part %q in path %q", basename, names)
+               return
+       }
+       node.Lock()
+       defer node.Unlock()
        _, err = node.Child(basename, func(child inode) (inode, error) {
                switch child := child.(type) {
                case nil:
-                       child, err = node.FS().newNode(basename, 0755, node.FileInfo().ModTime())
+                       child, err = node.FS().newNode(basename, 0755, time.Time{})
                        if err != nil {
                                return nil, err
                        }
@@ -877,6 +1527,100 @@ func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) {
        return
 }
 
+func (dn *dirnode) TreeSize() (bytes int64) {
+       dn.RLock()
+       defer dn.RUnlock()
+       for _, i := range dn.inodes {
+               switch i := i.(type) {
+               case *filenode:
+                       bytes += i.Size()
+               case *dirnode:
+                       bytes += i.TreeSize()
+               }
+       }
+       return
+}
+
+func (dn *dirnode) Snapshot() (inode, error) {
+       return dn.snapshot()
+}
+
+func (dn *dirnode) snapshot() (*dirnode, error) {
+       dn.RLock()
+       defer dn.RUnlock()
+       snap := &dirnode{
+               treenode: treenode{
+                       inodes:   make(map[string]inode, len(dn.inodes)),
+                       fileinfo: dn.fileinfo,
+               },
+       }
+       for name, child := range dn.inodes {
+               dupchild, err := child.Snapshot()
+               if err != nil {
+                       return nil, err
+               }
+               snap.inodes[name] = dupchild
+               dupchild.SetParent(snap, name)
+       }
+       return snap, nil
+}
+
+func (dn *dirnode) Splice(repl inode) error {
+       repl, err := repl.Snapshot()
+       if err != nil {
+               return fmt.Errorf("cannot copy snapshot: %w", err)
+       }
+       switch repl := repl.(type) {
+       default:
+               return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
+       case *dirnode:
+               dn.Lock()
+               defer dn.Unlock()
+               dn.inodes = repl.inodes
+               dn.setTreeFS(dn.fs)
+       case *filenode:
+               dn.parent.Lock()
+               defer dn.parent.Unlock()
+               removing, err := dn.parent.Child(dn.fileinfo.name, nil)
+               if err != nil {
+                       return fmt.Errorf("cannot use Splice to replace a top-level directory with a file: %w", ErrInvalidOperation)
+               } else if removing != dn {
+                       // If ../thisdirname is not this dirnode, it
+                       // must be an inode that wraps a dirnode, like
+                       // a collectionFileSystem or deferrednode.
+                       if deferred, ok := removing.(*deferrednode); ok {
+                               // More useful to report the type of
+                               // the wrapped node rather than just
+                               // *deferrednode. (We know the real
+                               // inode is already loaded because dn
+                               // is inside it.)
+                               removing = deferred.realinode()
+                       }
+                       return fmt.Errorf("cannot use Splice to attach a file at top level of %T: %w", removing, ErrInvalidOperation)
+               }
+               dn.Lock()
+               defer dn.Unlock()
+               _, err = dn.parent.Child(dn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
+               if err != nil {
+                       return fmt.Errorf("error replacing filenode: dn.parent.Child(): %w", err)
+               }
+               repl.fs = dn.fs
+       }
+       return nil
+}
+
+func (dn *dirnode) setTreeFS(fs *collectionFileSystem) {
+       dn.fs = fs
+       for _, child := range dn.inodes {
+               switch child := child.(type) {
+               case *dirnode:
+                       child.setTreeFS(fs)
+               case *filenode:
+                       child.fs = fs
+               }
+       }
+}
+
 type segment interface {
        io.ReaderAt
        Len() int
@@ -887,6 +1631,24 @@ type segment interface {
 
 type memSegment struct {
        buf []byte
+       // 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 {
@@ -903,28 +1665,31 @@ func (me *memSegment) Slice(off, length int) segment {
 }
 
 func (me *memSegment) Truncate(n int) {
-       if n > cap(me.buf) {
+       if n > cap(me.buf) || (me.flushing != nil && n > len(me.buf)) {
                newsize := 1024
                for newsize < n {
                        newsize = newsize << 2
                }
                newbuf := make([]byte, n, newsize)
                copy(newbuf, me.buf)
-               me.buf = newbuf
+               me.buf, me.flushing = newbuf, nil
        } else {
-               // Zero unused part when shrinking, in case we grow
-               // and start using it again later.
-               for i := n; i < len(me.buf); i++ {
+               // reclaim existing capacity, and zero reclaimed part
+               oldlen := len(me.buf)
+               me.buf = me.buf[:n]
+               for i := oldlen; i < n; i++ {
                        me.buf[i] = 0
                }
        }
-       me.buf = me.buf[:n]
 }
 
 func (me *memSegment) WriteAt(p []byte, off int) {
        if off+len(p) > len(me.buf) {
                panic("overflowed segment")
        }
+       if me.flushing != nil {
+               me.buf, me.flushing = append([]byte(nil), me.buf...), nil
+       }
        copy(me.buf[off:], p)
 }