]> git.arvados.org - arvados.git/blob - sdk/go/arvados/fs_collection.go
22320: Fix test panic.
[arvados.git] / sdk / go / arvados / fs_collection.go
1 // Copyright (C) The Arvados Authors. All rights reserved.
2 //
3 // SPDX-License-Identifier: Apache-2.0
4
5 package arvados
6
7 import (
8         "bytes"
9         "context"
10         "encoding/json"
11         "errors"
12         "fmt"
13         "io"
14         "os"
15         "path"
16         "regexp"
17         "sort"
18         "strconv"
19         "strings"
20         "sync"
21         "sync/atomic"
22         "time"
23 )
24
25 var (
26         maxBlockSize      = 1 << 26
27         concurrentWriters = 4 // max goroutines writing to Keep in background and during flush()
28 )
29
30 // A CollectionFileSystem is a FileSystem that can be serialized as a
31 // manifest and stored as a collection.
32 type CollectionFileSystem interface {
33         FileSystem
34
35         // Flush all file data to Keep and return a snapshot of the
36         // filesystem suitable for saving as (Collection)ManifestText.
37         // Prefix (normally ".") is a top level directory, effectively
38         // prepended to all paths in the returned manifest.
39         MarshalManifest(prefix string) (string, error)
40
41         // Given map {x->y}, replace each occurrence of x with y.
42         // Except: If segment x is not referenced anywhere in the
43         // collection, do not make any replacements that reference the
44         // same locator as y. The first return value is true if any
45         // substitutions were made.
46         ReplaceSegments(map[BlockSegment]BlockSegment) (bool, error)
47
48         // Total data bytes in all files.
49         Size() int64
50 }
51
52 type collectionFileSystem struct {
53         fileSystem
54         uuid           string
55         replicas       int
56         storageClasses []string
57
58         // PDH returned by the server as of last sync/load.
59         loadedPDH atomic.Value
60         // Time when the most recently synced version was retrieved
61         // from the server, if any. See checkChangesOnServer.
62         reloadTime    time.Time
63         reloadTimeMtx sync.Mutex
64         // PDH of the locally generated manifest as of last
65         // sync/load. This can differ from loadedPDH after loading a
66         // version that was generated with different code and sorts
67         // filenames differently than we do, for example.
68         savedPDH atomic.Value
69
70         // guessSignatureTTL tracks a lower bound for the server's
71         // configured BlobSigningTTL. The guess is initially zero, and
72         // increases when we come across a signature with an expiry
73         // time further in the future than the previous guess.
74         //
75         // When the guessed TTL is much smaller than the real TTL,
76         // preemptive signature refresh is delayed or missed entirely,
77         // which is OK.
78         guessSignatureTTL time.Duration
79         holdCheckChanges  time.Time
80         lockCheckChanges  sync.Mutex
81
82         // Pending updates to send via replace_segments.  See
83         // repackTree().
84         repacked    map[BlockSegment]BlockSegment
85         repackedMtx sync.Mutex
86 }
87
88 // FileSystem returns a CollectionFileSystem for the collection.
89 func (c *Collection) FileSystem(client apiClient, kc keepClient) (CollectionFileSystem, error) {
90         modTime := c.ModifiedAt
91         if modTime.IsZero() {
92                 modTime = time.Now()
93         }
94         fs := &collectionFileSystem{
95                 uuid:           c.UUID,
96                 storageClasses: c.StorageClassesDesired,
97                 fileSystem: fileSystem{
98                         fsBackend: keepBackend{apiClient: client, keepClient: kc},
99                         thr:       newThrottle(concurrentWriters),
100                 },
101         }
102         fs.loadedPDH.Store(c.PortableDataHash)
103         if r := c.ReplicationDesired; r != nil {
104                 fs.replicas = *r
105         }
106         root := &dirnode{
107                 fs: fs,
108                 treenode: treenode{
109                         fileinfo: fileinfo{
110                                 name:    ".",
111                                 mode:    os.ModeDir | 0755,
112                                 modTime: modTime,
113                                 sys: func() interface{} {
114                                         return &Collection{
115                                                 UUID:             fs.uuid,
116                                                 PortableDataHash: fs.loadedPDH.Load().(string),
117                                                 Properties:       c.Properties,
118                                         }
119                                 },
120                         },
121                         inodes: make(map[string]inode),
122                 },
123         }
124         root.SetParent(root, ".")
125         if err := root.loadManifest(c.ManifestText); err != nil {
126                 return nil, err
127         }
128
129         txt, err := root.marshalManifest(context.Background(), ".", false)
130         if err != nil {
131                 return nil, err
132         }
133         fs.savedPDH.Store(PortableDataHash(txt))
134
135         backdateTree(root, modTime)
136         fs.root = root
137         return fs, nil
138 }
139
140 // caller must have lock (or guarantee no concurrent accesses somehow)
141 func eachNode(n inode, ffunc func(*filenode), dfunc func(*dirnode)) {
142         switch n := n.(type) {
143         case *filenode:
144                 if ffunc != nil {
145                         ffunc(n)
146                 }
147         case *dirnode:
148                 if dfunc != nil {
149                         dfunc(n)
150                 }
151                 for _, n := range n.inodes {
152                         eachNode(n, ffunc, dfunc)
153                 }
154         }
155 }
156
157 // caller must have lock (or guarantee no concurrent accesses somehow)
158 func backdateTree(n inode, modTime time.Time) {
159         eachNode(n, func(fn *filenode) {
160                 fn.fileinfo.modTime = modTime
161         }, func(dn *dirnode) {
162                 dn.fileinfo.modTime = modTime
163         })
164 }
165
166 // Approximate portion of signature TTL remaining, usually between 0
167 // and 1, or negative if some signatures have expired.
168 func (fs *collectionFileSystem) signatureTimeLeft() (float64, time.Duration) {
169         var (
170                 now      = time.Now()
171                 earliest = now.Add(time.Hour * 24 * 7 * 365)
172                 latest   time.Time
173         )
174         fs.fileSystem.root.RLock()
175         eachNode(fs.root, func(fn *filenode) {
176                 fn.Lock()
177                 defer fn.Unlock()
178                 for _, seg := range fn.segments {
179                         seg, ok := seg.(storedSegment)
180                         if !ok {
181                                 continue
182                         }
183                         expiryTime, err := signatureExpiryTime(seg.locator)
184                         if err != nil {
185                                 continue
186                         }
187                         if expiryTime.Before(earliest) {
188                                 earliest = expiryTime
189                         }
190                         if expiryTime.After(latest) {
191                                 latest = expiryTime
192                         }
193                 }
194         }, nil)
195         fs.fileSystem.root.RUnlock()
196
197         if latest.IsZero() {
198                 // No signatures == 100% of TTL remaining.
199                 return 1, 1
200         }
201
202         ttl := latest.Sub(now)
203         fs.fileSystem.root.Lock()
204         {
205                 if ttl > fs.guessSignatureTTL {
206                         // ttl is closer to the real TTL than
207                         // guessSignatureTTL.
208                         fs.guessSignatureTTL = ttl
209                 } else {
210                         // Use the previous best guess to compute the
211                         // portion remaining (below, after unlocking
212                         // mutex).
213                         ttl = fs.guessSignatureTTL
214                 }
215         }
216         fs.fileSystem.root.Unlock()
217
218         return earliest.Sub(now).Seconds() / ttl.Seconds(), ttl
219 }
220
221 func (fs *collectionFileSystem) updateSignatures(newmanifest string) {
222         newLoc := map[string]string{}
223         for _, tok := range regexp.MustCompile(`\S+`).FindAllString(newmanifest, -1) {
224                 if mBlkRe.MatchString(tok) {
225                         newLoc[stripAllHints(tok)] = tok
226                 }
227         }
228         fs.fileSystem.root.Lock()
229         defer fs.fileSystem.root.Unlock()
230         eachNode(fs.root, func(fn *filenode) {
231                 fn.Lock()
232                 defer fn.Unlock()
233                 for idx, seg := range fn.segments {
234                         seg, ok := seg.(storedSegment)
235                         if !ok {
236                                 continue
237                         }
238                         loc, ok := newLoc[stripAllHints(seg.locator)]
239                         if !ok {
240                                 continue
241                         }
242                         seg.locator = loc
243                         fn.segments[idx] = seg
244                 }
245         }, nil)
246 }
247
248 func (fs *collectionFileSystem) newNode(name string, perm os.FileMode, modTime time.Time) (node inode, err error) {
249         if name == "" || name == "." || name == ".." {
250                 return nil, ErrInvalidArgument
251         }
252         if perm.IsDir() {
253                 return &dirnode{
254                         fs: fs,
255                         treenode: treenode{
256                                 fileinfo: fileinfo{
257                                         name:    name,
258                                         mode:    perm | os.ModeDir,
259                                         modTime: modTime,
260                                 },
261                                 inodes: make(map[string]inode),
262                         },
263                 }, nil
264         }
265         return &filenode{
266                 fs: fs,
267                 fileinfo: fileinfo{
268                         name:    name,
269                         mode:    perm & ^os.ModeDir,
270                         modTime: modTime,
271                 },
272         }, nil
273 }
274
275 func (fs *collectionFileSystem) Child(name string, replace func(inode) (inode, error)) (inode, error) {
276         return fs.rootnode().Child(name, replace)
277 }
278
279 func (fs *collectionFileSystem) FS() FileSystem {
280         return fs
281 }
282
283 func (fs *collectionFileSystem) FileInfo() os.FileInfo {
284         return fs.rootnode().FileInfo()
285 }
286
287 func (fs *collectionFileSystem) IsDir() bool {
288         return true
289 }
290
291 func (fs *collectionFileSystem) Lock() {
292         fs.rootnode().Lock()
293 }
294
295 func (fs *collectionFileSystem) Unlock() {
296         fs.rootnode().Unlock()
297 }
298
299 func (fs *collectionFileSystem) RLock() {
300         fs.rootnode().RLock()
301 }
302
303 func (fs *collectionFileSystem) RUnlock() {
304         fs.rootnode().RUnlock()
305 }
306
307 func (fs *collectionFileSystem) Parent() inode {
308         return fs.rootnode().Parent()
309 }
310
311 func (fs *collectionFileSystem) Read(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
312         return 0, ptr, ErrInvalidOperation
313 }
314
315 func (fs *collectionFileSystem) Write(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
316         return 0, ptr, ErrInvalidOperation
317 }
318
319 func (fs *collectionFileSystem) Readdir() ([]os.FileInfo, error) {
320         return fs.rootnode().Readdir()
321 }
322
323 func (fs *collectionFileSystem) SetParent(parent inode, name string) {
324         fs.rootnode().SetParent(parent, name)
325 }
326
327 func (fs *collectionFileSystem) Truncate(int64) error {
328         return ErrInvalidOperation
329 }
330
331 // Check for and incorporate upstream changes. If force==false, this
332 // is a no-op except once every ttl/100 or so.
333 //
334 // Return value is true if new content was loaded from upstream and
335 // any unsaved local changes have been discarded.
336 func (fs *collectionFileSystem) checkChangesOnServer(force bool) (bool, error) {
337         if fs.uuid == "" && fs.loadedPDH.Load() == "" {
338                 return false, nil
339         }
340
341         fs.lockCheckChanges.Lock()
342         if !force && fs.holdCheckChanges.After(time.Now()) {
343                 fs.lockCheckChanges.Unlock()
344                 return false, nil
345         }
346         remain, ttl := fs.signatureTimeLeft()
347         if remain > 0.01 {
348                 fs.holdCheckChanges = time.Now().Add(ttl / 100)
349         }
350         fs.lockCheckChanges.Unlock()
351
352         if !force && remain >= 0.5 {
353                 // plenty of time left on current signatures
354                 return false, nil
355         }
356
357         loadedPDH, _ := fs.loadedPDH.Load().(string)
358         getparams := map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}}
359         if fs.uuid != "" {
360                 reloadTime := time.Now()
361                 var coll Collection
362                 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+fs.uuid, nil, getparams)
363                 if err != nil {
364                         return false, err
365                 }
366                 if coll.PortableDataHash != loadedPDH {
367                         // collection has changed upstream since we
368                         // last loaded or saved. Refresh local data,
369                         // losing any unsaved local changes.
370                         fs.reloadTimeMtx.Lock()
371                         defer fs.reloadTimeMtx.Unlock()
372                         if fs.reloadTime.After(reloadTime) {
373                                 // Another goroutine called
374                                 // checkChangesOnServer after we
375                                 // started, and already updated the
376                                 // collection.  This means their GET
377                                 // request started after our caller
378                                 // called Sync, so their response is
379                                 // new enough to be consistent with
380                                 // our semantics.  The converse is not
381                                 // true, so the only safe option is to
382                                 // leave their update in place.
383                                 return true, nil
384                         }
385                         newfs, err := coll.FileSystem(fs.fileSystem.fsBackend, fs.fileSystem.fsBackend)
386                         if err != nil {
387                                 return false, err
388                         }
389                         snap, err := Snapshot(newfs, "/")
390                         if err != nil {
391                                 return false, err
392                         }
393                         err = Splice(fs, "/", snap)
394                         if err != nil {
395                                 return false, err
396                         }
397                         fs.loadedPDH.Store(coll.PortableDataHash)
398                         fs.savedPDH.Store(newfs.(*collectionFileSystem).savedPDH.Load())
399                         fs.reloadTime = reloadTime
400                         return true, nil
401                 }
402                 fs.updateSignatures(coll.ManifestText)
403                 return false, nil
404         }
405         if loadedPDH != "" {
406                 var coll Collection
407                 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+loadedPDH, nil, getparams)
408                 if err != nil {
409                         return false, err
410                 }
411                 fs.updateSignatures(coll.ManifestText)
412         }
413         return false, nil
414 }
415
416 // Refresh signature on a single locator, if necessary. Assume caller
417 // has lock. If an update is needed, and there are any storedSegments
418 // whose signatures can be updated, start a background task to update
419 // them asynchronously when the caller releases locks.
420 func (fs *collectionFileSystem) refreshSignature(locator string) string {
421         exp, err := signatureExpiryTime(locator)
422         if err != nil || exp.Sub(time.Now()) > time.Minute {
423                 // Synchronous update is not needed. Start an
424                 // asynchronous update if needed.
425                 go fs.checkChangesOnServer(false)
426                 return locator
427         }
428         loadedPDH, _ := fs.loadedPDH.Load().(string)
429         var manifests string
430         for _, id := range []string{fs.uuid, loadedPDH} {
431                 if id == "" {
432                         continue
433                 }
434                 var coll Collection
435                 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+id, nil, map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}})
436                 if err != nil {
437                         continue
438                 }
439                 manifests += coll.ManifestText
440         }
441         hash := stripAllHints(locator)
442         for _, tok := range regexp.MustCompile(`\S+`).FindAllString(manifests, -1) {
443                 if mBlkRe.MatchString(tok) {
444                         if stripAllHints(tok) == hash {
445                                 locator = tok
446                                 break
447                         }
448                 }
449         }
450         go fs.updateSignatures(manifests)
451         return locator
452 }
453
454 func (fs *collectionFileSystem) Sync() error {
455         refreshed, err := fs.checkChangesOnServer(true)
456         if err != nil {
457                 return err
458         }
459         if refreshed || fs.uuid == "" {
460                 return nil
461         }
462         txt, err := fs.MarshalManifest(".")
463         if err != nil {
464                 return fmt.Errorf("sync failed: %s", err)
465         }
466         savingPDH := PortableDataHash(txt)
467         if savingPDH == fs.savedPDH.Load() {
468                 // No local changes since last save or initial load.
469                 return nil
470         }
471         coll := Collection{
472                 UUID:         fs.uuid,
473                 ManifestText: txt,
474         }
475
476         selectFields := []string{"uuid", "portable_data_hash"}
477         fs.lockCheckChanges.Lock()
478         remain, _ := fs.signatureTimeLeft()
479         fs.lockCheckChanges.Unlock()
480         if remain < 0.5 {
481                 selectFields = append(selectFields, "manifest_text")
482         }
483
484         err = fs.RequestAndDecode(&coll, "PUT", "arvados/v1/collections/"+fs.uuid, nil, map[string]interface{}{
485                 "collection": map[string]string{
486                         "manifest_text": coll.ManifestText,
487                 },
488                 "select": selectFields,
489         })
490         if err != nil {
491                 return fmt.Errorf("sync failed: update %s: %w", fs.uuid, err)
492         }
493         fs.updateSignatures(coll.ManifestText)
494         fs.loadedPDH.Store(coll.PortableDataHash)
495         fs.savedPDH.Store(savingPDH)
496         return nil
497 }
498
499 func (fs *collectionFileSystem) Flush(path string, shortBlocks bool) error {
500         node, err := rlookup(fs.fileSystem.root, path, nil)
501         if err != nil {
502                 return err
503         }
504         dn, ok := node.(*dirnode)
505         if !ok {
506                 return ErrNotADirectory
507         }
508         dn.Lock()
509         defer dn.Unlock()
510         names := dn.sortedNames()
511         if path != "" {
512                 // Caller only wants to flush the specified dir,
513                 // non-recursively.  Drop subdirs from the list of
514                 // names.
515                 var filenames []string
516                 for _, name := range names {
517                         if _, ok := dn.inodes[name].(*filenode); ok {
518                                 filenames = append(filenames, name)
519                         }
520                 }
521                 names = filenames
522         }
523         for _, name := range names {
524                 child := dn.inodes[name]
525                 child.Lock()
526                 defer child.Unlock()
527         }
528         return dn.flush(context.TODO(), names, flushOpts{sync: false, shortBlocks: shortBlocks})
529 }
530
531 func (fs *collectionFileSystem) MemorySize() int64 {
532         return fs.fileSystem.root.(*dirnode).MemorySize()
533 }
534
535 func (fs *collectionFileSystem) MarshalManifest(prefix string) (string, error) {
536         fs.fileSystem.root.Lock()
537         defer fs.fileSystem.root.Unlock()
538         return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix, true)
539 }
540
541 func (fs *collectionFileSystem) Size() int64 {
542         return fs.fileSystem.root.(*dirnode).TreeSize()
543 }
544
545 func (fs *collectionFileSystem) Snapshot() (inode, error) {
546         return fs.fileSystem.root.Snapshot()
547 }
548
549 func (fs *collectionFileSystem) Splice(r inode) error {
550         return fs.fileSystem.root.Splice(r)
551 }
552
553 func (fs *collectionFileSystem) Repack(ctx context.Context, opts RepackOptions) (int, error) {
554         return fs.repackTree(ctx, opts, fs.root.(*dirnode))
555 }
556
557 func (fs *collectionFileSystem) repackTree(ctx context.Context, opts RepackOptions, root *dirnode) (int, error) {
558         fs.fileSystem.root.Lock()
559         plan, err := fs.planRepack(ctx, opts, fs.root.(*dirnode))
560         fs.fileSystem.root.Unlock()
561         if err != nil {
562                 return 0, err
563         }
564         if opts.DryRun {
565                 return len(plan), nil
566         }
567         repacked, err := fs.repackData(ctx, plan)
568         if err != nil {
569                 return 0, err
570         }
571         replaced, err := fs.replaceSegments(repacked)
572         if err != nil {
573                 return 0, err
574         }
575         nReplaced := len(replaced)
576
577         fs.repackedMtx.Lock()
578         if len(repacked) == 0 {
579                 // nothing to save
580         } else if len(fs.repacked) == 0 {
581                 fs.repacked = repacked
582         } else {
583                 // Merge new repacking results with existing unsaved
584                 // fs.repacked map.
585                 for orig, repl := range fs.repacked {
586                         // If a previous repack saved
587                         // fs.repacked[A]==B, and now we have
588                         // repacked[B]==C, then next time we sync to
589                         // the server, we should replace A with C
590                         // instead of B. So we set repacked[A]=C.
591                         if newrepl, ok := repacked[repl.StripAllHints()]; ok {
592                                 repacked[orig] = newrepl
593                         } else {
594                                 repacked[orig] = repl
595                         }
596                 }
597                 fs.repacked = repacked
598         }
599         fs.repackedMtx.Unlock()
600
601         return nReplaced, nil
602 }
603
604 func (fs *collectionFileSystem) ReplaceSegments(m map[BlockSegment]BlockSegment) (bool, error) {
605         changed, err := fs.replaceSegments(m)
606         return len(changed) > 0, err
607 }
608
609 func (fs *collectionFileSystem) replaceSegments(m map[BlockSegment]BlockSegment) (map[BlockSegment]BlockSegment, error) {
610         fs.fileSystem.root.Lock()
611         defer fs.fileSystem.root.Unlock()
612         missing := make(map[BlockSegment]bool, len(m))
613         for orig := range m {
614                 orig.Locator = stripAllHints(orig.Locator)
615                 missing[orig] = true
616         }
617         fs.fileSystem.root.(*dirnode).walkSegments(func(seg segment) segment {
618                 if seg, ok := seg.(storedSegment); ok {
619                         delete(missing, seg.blockSegment().StripAllHints())
620                 }
621                 return seg
622         })
623         skip := make(map[string]bool)
624         for orig, repl := range m {
625                 orig.Locator = stripAllHints(orig.Locator)
626                 if missing[orig] {
627                         skip[repl.Locator] = true
628                 }
629         }
630         todo := make(map[BlockSegment]storedSegment, len(m))
631         toks := make([][]byte, 3)
632         for orig, repl := range m {
633                 if !skip[repl.Locator] {
634                         orig.Locator = stripAllHints(orig.Locator)
635                         if orig.Length != repl.Length {
636                                 return nil, fmt.Errorf("mismatched length: replacing segment length %d with segment length %d", orig.Length, repl.Length)
637                         }
638                         if splitToToks([]byte(repl.Locator), '+', toks) < 2 {
639                                 return nil, errors.New("invalid replacement locator")
640                         }
641                         blksize, err := strconv.ParseInt(string(toks[1]), 10, 64)
642                         if err != nil {
643                                 return nil, fmt.Errorf("invalid size hint in replacement locator: %w", err)
644                         }
645                         if repl.Offset+repl.Length > int(blksize) {
646                                 return nil, fmt.Errorf("invalid replacement: offset %d + length %d > block size %d", repl.Offset, repl.Length, blksize)
647                         }
648                         todo[orig] = storedSegment{
649                                 locator: repl.Locator,
650                                 offset:  repl.Offset,
651                                 size:    int(blksize),
652                         }
653                 }
654         }
655         changed := make(map[BlockSegment]BlockSegment, len(todo))
656         fs.fileSystem.root.(*dirnode).walkSegments(func(s segment) segment {
657                 seg, ok := s.(storedSegment)
658                 if !ok {
659                         return s
660                 }
661                 orig := seg.blockSegment().StripAllHints()
662                 repl, ok := todo[orig]
663                 if !ok {
664                         return s
665                 }
666                 seg.locator = repl.locator
667                 seg.offset = repl.offset
668                 seg.size = repl.size
669                 // (leave seg.kc and seg.length unchanged)
670                 changed[orig] = seg.blockSegment()
671                 return seg
672         })
673         return changed, nil
674 }
675
676 // See (*collectionFileSystem)planRepack.
677 var repackBucketThreshold = []struct {
678         maxIn  int
679         minOut int
680 }{
681         {maxIn: 1 << 23, minOut: 1 << 25},
682         {maxIn: 1 << 21, minOut: 1 << 24},
683         {maxIn: 1 << 19, minOut: 1 << 22},
684         {maxIn: 1 << 17, minOut: 1 << 20},
685         {maxIn: 1 << 15, minOut: 1 << 18},
686         {maxIn: 1 << 13, minOut: 1 << 16},
687         {maxIn: 1 << 11, minOut: 1 << 14},
688         {maxIn: 1 << 9, minOut: 1 << 12},
689         {maxIn: 1 << 7, minOut: 1 << 10},
690         {maxIn: 1 << 5, minOut: 1 << 8},
691         {maxIn: 1 << 3, minOut: 1 << 6},
692 }
693
694 // Produce a list of segment merges that would result in a more
695 // efficient packing.  Each element in the returned plan is a slice of
696 // 2+ segments with a combined length no greater than maxBlockSize.
697 //
698 // Caller must have lock on given root node.
699 func (fs *collectionFileSystem) planRepack(ctx context.Context, opts RepackOptions, root *dirnode) (plan [][]storedSegment, err error) {
700         if opts.CachedOnly {
701                 // TODO: use diskCacheProber
702                 return nil, errors.New("CacheOnly not yet implemented")
703         }
704         // TODO: depending on opts, plan as if large but underutilized
705         // blocks are short blocks.
706         blockSize := make(map[string]int)
707         bucketBlocks := make([][]string, len(repackBucketThreshold))
708         root.walkSegments(func(seg segment) segment {
709                 if ss, ok := seg.(storedSegment); ok {
710                         hash := stripAllHints(ss.locator)
711                         if blockSize[hash] == 0 {
712                                 blockSize[hash] = ss.size
713                                 for bucket, threshold := range repackBucketThreshold {
714                                         if ss.size >= threshold.maxIn {
715                                                 break
716                                         }
717                                         bucketBlocks[bucket] = append(bucketBlocks[bucket], hash)
718                                 }
719                         }
720                 }
721                 return seg
722         })
723         // blockPlan[oldhash] == idx means plan[idx] will merge all
724         // segments in <oldhash> into a new block.
725         blockPlan := make(map[string]int)
726         pending := []string{}
727         for bucket := range bucketBlocks {
728                 pending = pending[:0]
729                 pendingSize := 0
730                 for _, hash := range bucketBlocks[bucket] {
731                         if _, planned := blockPlan[hash]; planned {
732                                 // already planned to merge this block
733                                 // in a different bucket
734                                 continue
735                         }
736                         size := blockSize[hash]
737                         if pendingSize+size > maxBlockSize {
738                                 for _, hash := range pending {
739                                         blockPlan[hash] = len(plan)
740                                 }
741                                 plan = append(plan, nil)
742                                 pending = pending[:0]
743                                 pendingSize = 0
744                         }
745                         pendingSize += size
746                         pending = append(pending, hash)
747                 }
748                 if pendingSize >= repackBucketThreshold[bucket].minOut {
749                         for _, hash := range pending {
750                                 blockPlan[hash] = len(plan)
751                         }
752                         plan = append(plan, nil)
753                 }
754         }
755         // We have decided which blocks to merge.  Now we collect all
756         // of the segments that reference those blocks, and return
757         // that as the final plan.
758         root.walkSegments(func(seg segment) segment {
759                 ss, ok := seg.(storedSegment)
760                 if !ok {
761                         return seg
762                 }
763                 hash := stripAllHints(ss.locator)
764                 idx, planning := blockPlan[hash]
765                 if !planning {
766                         return seg
767                 }
768                 plan[idx] = append(plan[idx], ss)
769                 return seg
770         })
771         return plan, nil
772 }
773
774 // Given a plan returned by planRepack, write new blocks with the
775 // merged segment data, and return a replacement mapping suitable for
776 // ReplaceSegments.
777 func (fs *collectionFileSystem) repackData(ctx context.Context, plan [][]storedSegment) (repl map[BlockSegment]BlockSegment, err error) {
778         if len(plan) == 0 {
779                 return
780         }
781         repl = make(map[BlockSegment]BlockSegment)
782         for _, insegments := range plan {
783                 // TODO: concurrency > 1
784                 outsize := 0
785                 for _, insegment := range insegments {
786                         outsize += insegment.length
787                 }
788                 if outsize > maxBlockSize {
789                         return nil, fmt.Errorf("combined length %d would exceed maximum block size %d", outsize, maxBlockSize)
790                 }
791                 piper, pipew := io.Pipe()
792                 go func() {
793                         for _, insegment := range insegments {
794                                 n, err := io.Copy(pipew, io.NewSectionReader(insegment, 0, int64(insegment.length)))
795                                 if err != nil {
796                                         pipew.CloseWithError(err)
797                                         return
798                                 }
799                                 if n != int64(insegment.length) {
800                                         pipew.CloseWithError(fmt.Errorf("internal error: copied %d bytes, expected %d", n, insegment.length))
801                                         return
802                                 }
803                                 if ctx.Err() != nil {
804                                         pipew.CloseWithError(ctx.Err())
805                                         return
806                                 }
807                         }
808                         pipew.Close()
809                 }()
810                 wrote, err := fs.BlockWrite(ctx, BlockWriteOptions{Reader: piper, DataSize: outsize})
811                 if err != nil {
812                         return nil, err
813                 }
814                 offset := 0
815                 for _, insegment := range insegments {
816                         repl[insegment.blockSegment().StripAllHints()] = BlockSegment{
817                                 Locator: wrote.Locator,
818                                 Offset:  offset,
819                                 Length:  insegment.length,
820                         }
821                         offset += insegment.length
822                 }
823         }
824         return
825 }
826
827 // filenodePtr is an offset into a file that is (usually) efficient to
828 // seek to. Specifically, if filenode.repacked==filenodePtr.repacked
829 // then
830 // filenode.segments[filenodePtr.segmentIdx][filenodePtr.segmentOff]
831 // corresponds to file offset filenodePtr.off. Otherwise, it is
832 // necessary to reexamine len(filenode.segments[0]) etc. to find the
833 // correct segment and offset.
834 type filenodePtr struct {
835         off        int64
836         segmentIdx int
837         segmentOff int
838         repacked   int64
839 }
840
841 // seek returns a ptr that is consistent with both startPtr.off and
842 // the current state of fn. The caller must already hold fn.RLock() or
843 // fn.Lock().
844 //
845 // If startPtr is beyond EOF, ptr.segment* will indicate precisely
846 // EOF.
847 //
848 // After seeking:
849 //
850 //      ptr.segmentIdx == len(filenode.segments) // i.e., at EOF
851 //      ||
852 //      filenode.segments[ptr.segmentIdx].Len() > ptr.segmentOff
853 func (fn *filenode) seek(startPtr filenodePtr) (ptr filenodePtr) {
854         ptr = startPtr
855         if ptr.off < 0 {
856                 // meaningless anyway
857                 return
858         } else if ptr.off >= fn.fileinfo.size {
859                 ptr.segmentIdx = len(fn.segments)
860                 ptr.segmentOff = 0
861                 ptr.repacked = fn.repacked
862                 return
863         } else if ptr.repacked == fn.repacked {
864                 // segmentIdx and segmentOff accurately reflect
865                 // ptr.off, but might have fallen off the end of a
866                 // segment
867                 if ptr.segmentOff >= fn.segments[ptr.segmentIdx].Len() {
868                         ptr.segmentIdx++
869                         ptr.segmentOff = 0
870                 }
871                 return
872         }
873         defer func() {
874                 ptr.repacked = fn.repacked
875         }()
876         if ptr.off >= fn.fileinfo.size {
877                 ptr.segmentIdx, ptr.segmentOff = len(fn.segments), 0
878                 return
879         }
880         // Recompute segmentIdx and segmentOff.  We have already
881         // established fn.fileinfo.size > ptr.off >= 0, so we don't
882         // have to deal with edge cases here.
883         var off int64
884         for ptr.segmentIdx, ptr.segmentOff = 0, 0; off < ptr.off; ptr.segmentIdx++ {
885                 // This would panic (index out of range) if
886                 // fn.fileinfo.size were larger than
887                 // sum(fn.segments[i].Len()) -- but that can't happen
888                 // because we have ensured fn.fileinfo.size is always
889                 // accurate.
890                 segLen := int64(fn.segments[ptr.segmentIdx].Len())
891                 if off+segLen > ptr.off {
892                         ptr.segmentOff = int(ptr.off - off)
893                         break
894                 }
895                 off += segLen
896         }
897         return
898 }
899
900 // filenode implements inode.
901 type filenode struct {
902         parent   inode
903         fs       *collectionFileSystem
904         fileinfo fileinfo
905         segments []segment
906         // number of times `segments` has changed in a
907         // way that might invalidate a filenodePtr
908         repacked int64
909         memsize  int64 // bytes in memSegments
910         sync.RWMutex
911         nullnode
912 }
913
914 // caller must have lock
915 func (fn *filenode) appendSegment(e segment) {
916         fn.segments = append(fn.segments, e)
917         fn.fileinfo.size += int64(e.Len())
918 }
919
920 func (fn *filenode) SetParent(p inode, name string) {
921         fn.Lock()
922         defer fn.Unlock()
923         fn.parent = p
924         fn.fileinfo.name = name
925 }
926
927 func (fn *filenode) Parent() inode {
928         fn.RLock()
929         defer fn.RUnlock()
930         return fn.parent
931 }
932
933 func (fn *filenode) FS() FileSystem {
934         return fn.fs
935 }
936
937 func (fn *filenode) MemorySize() (size int64) {
938         fn.RLock()
939         defer fn.RUnlock()
940         size = 64
941         for _, seg := range fn.segments {
942                 size += seg.memorySize()
943         }
944         return
945 }
946
947 // Read reads file data from a single segment, starting at startPtr,
948 // into p. startPtr is assumed not to be up-to-date. Caller must have
949 // RLock or Lock.
950 func (fn *filenode) Read(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr, err error) {
951         ptr = fn.seek(startPtr)
952         if ptr.off < 0 {
953                 err = ErrNegativeOffset
954                 return
955         }
956         if ptr.segmentIdx >= len(fn.segments) {
957                 err = io.EOF
958                 return
959         }
960         if ss, ok := fn.segments[ptr.segmentIdx].(storedSegment); ok {
961                 ss.locator = fn.fs.refreshSignature(ss.locator)
962                 fn.segments[ptr.segmentIdx] = ss
963         }
964         n, err = fn.segments[ptr.segmentIdx].ReadAt(p, int64(ptr.segmentOff))
965         if n > 0 {
966                 ptr.off += int64(n)
967                 ptr.segmentOff += n
968                 if ptr.segmentOff == fn.segments[ptr.segmentIdx].Len() {
969                         ptr.segmentIdx++
970                         ptr.segmentOff = 0
971                         if ptr.segmentIdx < len(fn.segments) && err == io.EOF {
972                                 err = nil
973                         }
974                 }
975         }
976         return
977 }
978
979 func (fn *filenode) Size() int64 {
980         fn.RLock()
981         defer fn.RUnlock()
982         return fn.fileinfo.Size()
983 }
984
985 func (fn *filenode) FileInfo() os.FileInfo {
986         fn.RLock()
987         defer fn.RUnlock()
988         return fn.fileinfo
989 }
990
991 func (fn *filenode) Truncate(size int64) error {
992         fn.Lock()
993         defer fn.Unlock()
994         return fn.truncate(size)
995 }
996
997 func (fn *filenode) truncate(size int64) error {
998         if size == fn.fileinfo.size {
999                 return nil
1000         }
1001         fn.repacked++
1002         if size < fn.fileinfo.size {
1003                 ptr := fn.seek(filenodePtr{off: size})
1004                 for i := ptr.segmentIdx; i < len(fn.segments); i++ {
1005                         if seg, ok := fn.segments[i].(*memSegment); ok {
1006                                 fn.memsize -= int64(seg.Len())
1007                         }
1008                 }
1009                 if ptr.segmentOff == 0 {
1010                         fn.segments = fn.segments[:ptr.segmentIdx]
1011                 } else {
1012                         fn.segments = fn.segments[:ptr.segmentIdx+1]
1013                         switch seg := fn.segments[ptr.segmentIdx].(type) {
1014                         case *memSegment:
1015                                 seg.Truncate(ptr.segmentOff)
1016                                 fn.memsize += int64(seg.Len())
1017                         default:
1018                                 fn.segments[ptr.segmentIdx] = seg.Slice(0, ptr.segmentOff)
1019                         }
1020                 }
1021                 fn.fileinfo.size = size
1022                 return nil
1023         }
1024         for size > fn.fileinfo.size {
1025                 grow := size - fn.fileinfo.size
1026                 var seg *memSegment
1027                 var ok bool
1028                 if len(fn.segments) == 0 {
1029                         seg = &memSegment{}
1030                         fn.segments = append(fn.segments, seg)
1031                 } else if seg, ok = fn.segments[len(fn.segments)-1].(*memSegment); !ok || seg.Len() >= maxBlockSize {
1032                         seg = &memSegment{}
1033                         fn.segments = append(fn.segments, seg)
1034                 }
1035                 if maxgrow := int64(maxBlockSize - seg.Len()); maxgrow < grow {
1036                         grow = maxgrow
1037                 }
1038                 seg.Truncate(seg.Len() + int(grow))
1039                 fn.fileinfo.size += grow
1040                 fn.memsize += grow
1041         }
1042         return nil
1043 }
1044
1045 // Write writes data from p to the file, starting at startPtr,
1046 // extending the file size if necessary. Caller must have Lock.
1047 func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr, err error) {
1048         if startPtr.off > fn.fileinfo.size {
1049                 if err = fn.truncate(startPtr.off); err != nil {
1050                         return 0, startPtr, err
1051                 }
1052         }
1053         ptr = fn.seek(startPtr)
1054         if ptr.off < 0 {
1055                 err = ErrNegativeOffset
1056                 return
1057         }
1058         for len(p) > 0 && err == nil {
1059                 cando := p
1060                 if len(cando) > maxBlockSize {
1061                         cando = cando[:maxBlockSize]
1062                 }
1063                 // Rearrange/grow fn.segments (and shrink cando if
1064                 // needed) such that cando can be copied to
1065                 // fn.segments[ptr.segmentIdx] at offset
1066                 // ptr.segmentOff.
1067                 cur := ptr.segmentIdx
1068                 prev := ptr.segmentIdx - 1
1069                 var curWritable bool
1070                 if cur < len(fn.segments) {
1071                         _, curWritable = fn.segments[cur].(*memSegment)
1072                 }
1073                 var prevAppendable bool
1074                 if prev >= 0 && fn.segments[prev].Len() < maxBlockSize {
1075                         _, prevAppendable = fn.segments[prev].(*memSegment)
1076                 }
1077                 if ptr.segmentOff > 0 && !curWritable {
1078                         // Split a non-writable block.
1079                         if max := fn.segments[cur].Len() - ptr.segmentOff; max <= len(cando) {
1080                                 // Truncate cur, and insert a new
1081                                 // segment after it.
1082                                 cando = cando[:max]
1083                                 fn.segments = append(fn.segments, nil)
1084                                 copy(fn.segments[cur+1:], fn.segments[cur:])
1085                         } else {
1086                                 // Split cur into two copies, truncate
1087                                 // the one on the left, shift the one
1088                                 // on the right, and insert a new
1089                                 // segment between them.
1090                                 fn.segments = append(fn.segments, nil, nil)
1091                                 copy(fn.segments[cur+2:], fn.segments[cur:])
1092                                 fn.segments[cur+2] = fn.segments[cur+2].Slice(ptr.segmentOff+len(cando), -1)
1093                         }
1094                         cur++
1095                         prev++
1096                         seg := &memSegment{}
1097                         seg.Truncate(len(cando))
1098                         fn.memsize += int64(len(cando))
1099                         fn.segments[cur] = seg
1100                         fn.segments[prev] = fn.segments[prev].Slice(0, ptr.segmentOff)
1101                         ptr.segmentIdx++
1102                         ptr.segmentOff = 0
1103                         fn.repacked++
1104                         ptr.repacked++
1105                 } else if curWritable {
1106                         if fit := int(fn.segments[cur].Len()) - ptr.segmentOff; fit < len(cando) {
1107                                 cando = cando[:fit]
1108                         }
1109                 } else {
1110                         if prevAppendable {
1111                                 // Shrink cando if needed to fit in
1112                                 // prev segment.
1113                                 if cangrow := maxBlockSize - fn.segments[prev].Len(); cangrow < len(cando) {
1114                                         cando = cando[:cangrow]
1115                                 }
1116                         }
1117
1118                         if cur == len(fn.segments) {
1119                                 // ptr is at EOF, filesize is changing.
1120                                 fn.fileinfo.size += int64(len(cando))
1121                         } else if el := fn.segments[cur].Len(); el <= len(cando) {
1122                                 // cando is long enough that we won't
1123                                 // need cur any more. shrink cando to
1124                                 // be exactly as long as cur
1125                                 // (otherwise we'd accidentally shift
1126                                 // the effective position of all
1127                                 // segments after cur).
1128                                 cando = cando[:el]
1129                                 copy(fn.segments[cur:], fn.segments[cur+1:])
1130                                 fn.segments = fn.segments[:len(fn.segments)-1]
1131                         } else {
1132                                 // shrink cur by the same #bytes we're growing prev
1133                                 fn.segments[cur] = fn.segments[cur].Slice(len(cando), -1)
1134                         }
1135
1136                         if prevAppendable {
1137                                 // Grow prev.
1138                                 ptr.segmentIdx--
1139                                 ptr.segmentOff = fn.segments[prev].Len()
1140                                 fn.segments[prev].(*memSegment).Truncate(ptr.segmentOff + len(cando))
1141                                 fn.memsize += int64(len(cando))
1142                                 ptr.repacked++
1143                                 fn.repacked++
1144                         } else {
1145                                 // Insert a segment between prev and
1146                                 // cur, and advance prev/cur.
1147                                 fn.segments = append(fn.segments, nil)
1148                                 if cur < len(fn.segments) {
1149                                         copy(fn.segments[cur+1:], fn.segments[cur:])
1150                                         ptr.repacked++
1151                                         fn.repacked++
1152                                 } else {
1153                                         // appending a new segment does
1154                                         // not invalidate any ptrs
1155                                 }
1156                                 seg := &memSegment{}
1157                                 seg.Truncate(len(cando))
1158                                 fn.memsize += int64(len(cando))
1159                                 fn.segments[cur] = seg
1160                         }
1161                 }
1162
1163                 // Finally we can copy bytes from cando to the current segment.
1164                 fn.segments[ptr.segmentIdx].(*memSegment).WriteAt(cando, ptr.segmentOff)
1165                 n += len(cando)
1166                 p = p[len(cando):]
1167
1168                 ptr.off += int64(len(cando))
1169                 ptr.segmentOff += len(cando)
1170                 if ptr.segmentOff >= maxBlockSize {
1171                         fn.pruneMemSegments()
1172                 }
1173                 if fn.segments[ptr.segmentIdx].Len() == ptr.segmentOff {
1174                         ptr.segmentOff = 0
1175                         ptr.segmentIdx++
1176                 }
1177
1178                 fn.fileinfo.modTime = time.Now()
1179         }
1180         return
1181 }
1182
1183 // Write some data out to disk to reduce memory use. Caller must have
1184 // write lock.
1185 func (fn *filenode) pruneMemSegments() {
1186         // TODO: share code with (*dirnode)flush()
1187         // TODO: pack/flush small blocks too, when fragmented
1188         for idx, seg := range fn.segments {
1189                 seg, ok := seg.(*memSegment)
1190                 if !ok || seg.Len() < maxBlockSize || seg.flushing != nil {
1191                         continue
1192                 }
1193                 // Setting seg.flushing guarantees seg.buf will not be
1194                 // modified in place: WriteAt and Truncate will
1195                 // allocate a new buf instead, if necessary.
1196                 idx, buf := idx, seg.buf
1197                 done := make(chan struct{})
1198                 seg.flushing = done
1199                 // If lots of background writes are already in
1200                 // progress, block here until one finishes, rather
1201                 // than pile up an unlimited number of buffered writes
1202                 // and network flush operations.
1203                 fn.fs.throttle().Acquire()
1204                 go func() {
1205                         defer close(done)
1206                         resp, err := fn.FS().BlockWrite(context.Background(), BlockWriteOptions{
1207                                 Data:           buf,
1208                                 Replicas:       fn.fs.replicas,
1209                                 StorageClasses: fn.fs.storageClasses,
1210                         })
1211                         fn.fs.throttle().Release()
1212                         fn.Lock()
1213                         defer fn.Unlock()
1214                         if seg.flushing != done {
1215                                 // A new seg.buf has been allocated.
1216                                 return
1217                         }
1218                         if err != nil {
1219                                 // TODO: stall (or return errors from)
1220                                 // subsequent writes until flushing
1221                                 // starts to succeed.
1222                                 return
1223                         }
1224                         if len(fn.segments) <= idx || fn.segments[idx] != seg || len(seg.buf) != len(buf) {
1225                                 // Segment has been dropped/moved/resized.
1226                                 return
1227                         }
1228                         fn.memsize -= int64(len(buf))
1229                         fn.segments[idx] = storedSegment{
1230                                 kc:      fn.FS(),
1231                                 locator: resp.Locator,
1232                                 size:    len(buf),
1233                                 offset:  0,
1234                                 length:  len(buf),
1235                         }
1236                 }()
1237         }
1238 }
1239
1240 // Block until all pending pruneMemSegments/flush work is
1241 // finished. Caller must NOT have lock.
1242 func (fn *filenode) waitPrune() {
1243         var pending []<-chan struct{}
1244         fn.Lock()
1245         for _, seg := range fn.segments {
1246                 if seg, ok := seg.(*memSegment); ok && seg.flushing != nil {
1247                         pending = append(pending, seg.flushing)
1248                 }
1249         }
1250         fn.Unlock()
1251         for _, p := range pending {
1252                 <-p
1253         }
1254 }
1255
1256 func (fn *filenode) Snapshot() (inode, error) {
1257         fn.RLock()
1258         defer fn.RUnlock()
1259         segments := make([]segment, 0, len(fn.segments))
1260         for _, seg := range fn.segments {
1261                 segments = append(segments, seg.Slice(0, seg.Len()))
1262         }
1263         return &filenode{
1264                 fileinfo: fn.fileinfo,
1265                 segments: segments,
1266         }, nil
1267 }
1268
1269 func (fn *filenode) Splice(repl inode) error {
1270         repl, err := repl.Snapshot()
1271         if err != nil {
1272                 return err
1273         }
1274         fn.parent.Lock()
1275         defer fn.parent.Unlock()
1276         fn.Lock()
1277         defer fn.Unlock()
1278         _, err = fn.parent.Child(fn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
1279         if err != nil {
1280                 return err
1281         }
1282         switch repl := repl.(type) {
1283         case *dirnode:
1284                 repl.parent = fn.parent
1285                 repl.fileinfo.name = fn.fileinfo.name
1286                 repl.setTreeFS(fn.fs)
1287         case *filenode:
1288                 repl.parent = fn.parent
1289                 repl.fileinfo.name = fn.fileinfo.name
1290                 repl.fs = fn.fs
1291         default:
1292                 return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
1293         }
1294         return nil
1295 }
1296
1297 type dirnode struct {
1298         fs *collectionFileSystem
1299         treenode
1300 }
1301
1302 func (dn *dirnode) FS() FileSystem {
1303         return dn.fs
1304 }
1305
1306 func (dn *dirnode) Child(name string, replace func(inode) (inode, error)) (inode, error) {
1307         if dn == dn.fs.rootnode() && name == ".arvados#collection" {
1308                 gn := &getternode{Getter: func() ([]byte, error) {
1309                         var coll Collection
1310                         var err error
1311                         coll.ManifestText, err = dn.fs.MarshalManifest(".")
1312                         if err != nil {
1313                                 return nil, err
1314                         }
1315                         coll.UUID = dn.fs.uuid
1316                         data, err := json.Marshal(&coll)
1317                         if err == nil {
1318                                 data = append(data, '\n')
1319                         }
1320                         return data, err
1321                 }}
1322                 gn.SetParent(dn, name)
1323                 return gn, nil
1324         }
1325         return dn.treenode.Child(name, replace)
1326 }
1327
1328 type fnSegmentRef struct {
1329         fn  *filenode
1330         idx int
1331 }
1332
1333 // commitBlock concatenates the data from the given filenode segments
1334 // (which must be *memSegments), writes the data out to Keep as a
1335 // single block, and replaces the filenodes' *memSegments with
1336 // storedSegments that reference the relevant portions of the new
1337 // block.
1338 //
1339 // bufsize is the total data size in refs. It is used to preallocate
1340 // the correct amount of memory when len(refs)>1.
1341 //
1342 // If sync is false, commitBlock returns right away, after starting a
1343 // goroutine to do the writes, reacquire the filenodes' locks, and
1344 // swap out the *memSegments. Some filenodes' segments might get
1345 // modified/rearranged in the meantime, in which case commitBlock
1346 // won't replace them.
1347 //
1348 // Caller must have write lock.
1349 func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, bufsize int, sync bool) error {
1350         if len(refs) == 0 {
1351                 return nil
1352         }
1353         if err := ctx.Err(); err != nil {
1354                 return err
1355         }
1356         done := make(chan struct{})
1357         var block []byte
1358         segs := make([]*memSegment, 0, len(refs))
1359         offsets := make([]int, 0, len(refs)) // location of segment's data within block
1360         for _, ref := range refs {
1361                 seg := ref.fn.segments[ref.idx].(*memSegment)
1362                 if !sync && seg.flushingUnfinished() {
1363                         // Let the other flushing goroutine finish. If
1364                         // it fails, we'll try again next time.
1365                         close(done)
1366                         return nil
1367                 }
1368                 // In sync mode, we proceed regardless of
1369                 // whether another flush is in progress: It
1370                 // can't finish before we do, because we hold
1371                 // fn's lock until we finish our own writes.
1372                 seg.flushing = done
1373                 offsets = append(offsets, len(block))
1374                 if len(refs) == 1 {
1375                         block = seg.buf
1376                 } else if block == nil {
1377                         block = append(make([]byte, 0, bufsize), seg.buf...)
1378                 } else {
1379                         block = append(block, seg.buf...)
1380                 }
1381                 segs = append(segs, seg)
1382         }
1383         blocksize := len(block)
1384         dn.fs.throttle().Acquire()
1385         errs := make(chan error, 1)
1386         go func() {
1387                 defer close(done)
1388                 defer close(errs)
1389                 resp, err := dn.fs.BlockWrite(context.Background(), BlockWriteOptions{
1390                         Data:           block,
1391                         Replicas:       dn.fs.replicas,
1392                         StorageClasses: dn.fs.storageClasses,
1393                 })
1394                 dn.fs.throttle().Release()
1395                 if err != nil {
1396                         errs <- err
1397                         return
1398                 }
1399                 for idx, ref := range refs {
1400                         if !sync {
1401                                 ref.fn.Lock()
1402                                 // In async mode, fn's lock was
1403                                 // released while we were waiting for
1404                                 // PutB(); lots of things might have
1405                                 // changed.
1406                                 if len(ref.fn.segments) <= ref.idx {
1407                                         // file segments have
1408                                         // rearranged or changed in
1409                                         // some way
1410                                         ref.fn.Unlock()
1411                                         continue
1412                                 } else if seg, ok := ref.fn.segments[ref.idx].(*memSegment); !ok || seg != segs[idx] {
1413                                         // segment has been replaced
1414                                         ref.fn.Unlock()
1415                                         continue
1416                                 } else if seg.flushing != done {
1417                                         // seg.buf has been replaced
1418                                         ref.fn.Unlock()
1419                                         continue
1420                                 }
1421                         }
1422                         data := ref.fn.segments[ref.idx].(*memSegment).buf
1423                         ref.fn.segments[ref.idx] = storedSegment{
1424                                 kc:      dn.fs,
1425                                 locator: resp.Locator,
1426                                 size:    blocksize,
1427                                 offset:  offsets[idx],
1428                                 length:  len(data),
1429                         }
1430                         // atomic is needed here despite caller having
1431                         // lock: caller might be running concurrent
1432                         // commitBlock() goroutines using the same
1433                         // lock, writing different segments from the
1434                         // same file.
1435                         atomic.AddInt64(&ref.fn.memsize, -int64(len(data)))
1436                         if !sync {
1437                                 ref.fn.Unlock()
1438                         }
1439                 }
1440         }()
1441         if sync {
1442                 return <-errs
1443         }
1444         return nil
1445 }
1446
1447 type flushOpts struct {
1448         sync        bool
1449         shortBlocks bool
1450 }
1451
1452 // flush in-memory data and remote-cluster block references (for the
1453 // children with the given names, which must be children of dn) to
1454 // local-cluster persistent storage.
1455 //
1456 // Caller must have write lock on dn and the named children.
1457 //
1458 // If any children are dirs, they will be flushed recursively.
1459 func (dn *dirnode) flush(ctx context.Context, names []string, opts flushOpts) error {
1460         cg := newContextGroup(ctx)
1461         defer cg.Cancel()
1462
1463         goCommit := func(refs []fnSegmentRef, bufsize int) {
1464                 cg.Go(func() error {
1465                         return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync)
1466                 })
1467         }
1468
1469         var pending []fnSegmentRef
1470         var pendingLen int = 0
1471         localLocator := map[string]string{}
1472         for _, name := range names {
1473                 switch node := dn.inodes[name].(type) {
1474                 case *dirnode:
1475                         grandchildNames := node.sortedNames()
1476                         for _, grandchildName := range grandchildNames {
1477                                 grandchild := node.inodes[grandchildName]
1478                                 grandchild.Lock()
1479                                 defer grandchild.Unlock()
1480                         }
1481                         cg.Go(func() error { return node.flush(cg.Context(), grandchildNames, opts) })
1482                 case *filenode:
1483                         for idx, seg := range node.segments {
1484                                 switch seg := seg.(type) {
1485                                 case storedSegment:
1486                                         loc, ok := localLocator[seg.locator]
1487                                         if !ok {
1488                                                 var err error
1489                                                 loc, err = dn.fs.LocalLocator(seg.locator)
1490                                                 if err != nil {
1491                                                         return err
1492                                                 }
1493                                                 localLocator[seg.locator] = loc
1494                                         }
1495                                         seg.locator = loc
1496                                         node.segments[idx] = seg
1497                                 case *memSegment:
1498                                         if seg.Len() > maxBlockSize/2 {
1499                                                 goCommit([]fnSegmentRef{{node, idx}}, seg.Len())
1500                                                 continue
1501                                         }
1502                                         if pendingLen+seg.Len() > maxBlockSize {
1503                                                 goCommit(pending, pendingLen)
1504                                                 pending = nil
1505                                                 pendingLen = 0
1506                                         }
1507                                         pending = append(pending, fnSegmentRef{node, idx})
1508                                         pendingLen += seg.Len()
1509                                 default:
1510                                         panic(fmt.Sprintf("can't sync segment type %T", seg))
1511                                 }
1512                         }
1513                 }
1514         }
1515         if opts.shortBlocks {
1516                 goCommit(pending, pendingLen)
1517         }
1518         return cg.Wait()
1519 }
1520
1521 func (dn *dirnode) MemorySize() (size int64) {
1522         dn.RLock()
1523         todo := make([]inode, 0, len(dn.inodes))
1524         for _, node := range dn.inodes {
1525                 todo = append(todo, node)
1526         }
1527         dn.RUnlock()
1528         size = 64
1529         for _, node := range todo {
1530                 size += node.MemorySize()
1531         }
1532         return
1533 }
1534
1535 // caller must have write lock.
1536 func (dn *dirnode) sortedNames() []string {
1537         names := make([]string, 0, len(dn.inodes))
1538         for name := range dn.inodes {
1539                 names = append(names, name)
1540         }
1541         sort.Strings(names)
1542         return names
1543 }
1544
1545 // caller must have write lock.
1546 func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, flush bool) (string, error) {
1547         cg := newContextGroup(ctx)
1548         defer cg.Cancel()
1549
1550         if len(dn.inodes) == 0 {
1551                 if prefix == "." {
1552                         return "", nil
1553                 }
1554                 // Express the existence of an empty directory by
1555                 // adding an empty file named `\056`, which (unlike
1556                 // the more obvious spelling `.`) is accepted by the
1557                 // API's manifest validator.
1558                 return manifestEscape(prefix) + " d41d8cd98f00b204e9800998ecf8427e+0 0:0:\\056\n", nil
1559         }
1560
1561         names := dn.sortedNames()
1562
1563         // Wait for children to finish any pending write operations
1564         // before locking them.
1565         for _, name := range names {
1566                 node := dn.inodes[name]
1567                 if fn, ok := node.(*filenode); ok {
1568                         fn.waitPrune()
1569                 }
1570         }
1571
1572         var dirnames []string
1573         var filenames []string
1574         for _, name := range names {
1575                 node := dn.inodes[name]
1576                 node.Lock()
1577                 defer node.Unlock()
1578                 switch node := node.(type) {
1579                 case *dirnode:
1580                         dirnames = append(dirnames, name)
1581                 case *filenode:
1582                         filenames = append(filenames, name)
1583                 default:
1584                         panic(fmt.Sprintf("can't marshal inode type %T", node))
1585                 }
1586         }
1587
1588         subdirs := make([]string, len(dirnames))
1589         rootdir := ""
1590         for i, name := range dirnames {
1591                 i, name := i, name
1592                 cg.Go(func() error {
1593                         txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name, flush)
1594                         subdirs[i] = txt
1595                         return err
1596                 })
1597         }
1598
1599         cg.Go(func() error {
1600                 var streamLen int64
1601                 type filepart struct {
1602                         name   string
1603                         offset int64
1604                         length int64
1605                 }
1606
1607                 var fileparts []filepart
1608                 var blocks []string
1609                 if !flush {
1610                         // skip flush -- will fail below if anything
1611                         // needed flushing
1612                 } else if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil {
1613                         return err
1614                 }
1615                 for _, name := range filenames {
1616                         node := dn.inodes[name].(*filenode)
1617                         if len(node.segments) == 0 {
1618                                 fileparts = append(fileparts, filepart{name: name})
1619                                 continue
1620                         }
1621                         for _, seg := range node.segments {
1622                                 switch seg := seg.(type) {
1623                                 case storedSegment:
1624                                         if len(blocks) > 0 && blocks[len(blocks)-1] == seg.locator {
1625                                                 streamLen -= int64(seg.size)
1626                                         } else {
1627                                                 blocks = append(blocks, seg.locator)
1628                                         }
1629                                         next := filepart{
1630                                                 name:   name,
1631                                                 offset: streamLen + int64(seg.offset),
1632                                                 length: int64(seg.length),
1633                                         }
1634                                         if prev := len(fileparts) - 1; prev >= 0 &&
1635                                                 fileparts[prev].name == name &&
1636                                                 fileparts[prev].offset+fileparts[prev].length == next.offset {
1637                                                 fileparts[prev].length += next.length
1638                                         } else {
1639                                                 fileparts = append(fileparts, next)
1640                                         }
1641                                         streamLen += int64(seg.size)
1642                                 default:
1643                                         // We haven't unlocked since
1644                                         // calling flush(sync=true).
1645                                         // Evidently the caller passed
1646                                         // flush==false but there were
1647                                         // local changes.
1648                                         return fmt.Errorf("can't marshal segment type %T", seg)
1649                                 }
1650                         }
1651                 }
1652                 var filetokens []string
1653                 for _, s := range fileparts {
1654                         filetokens = append(filetokens, fmt.Sprintf("%d:%d:%s", s.offset, s.length, manifestEscape(s.name)))
1655                 }
1656                 if len(filetokens) == 0 {
1657                         return nil
1658                 } else if len(blocks) == 0 {
1659                         blocks = []string{"d41d8cd98f00b204e9800998ecf8427e+0"}
1660                 }
1661                 rootdir = manifestEscape(prefix) + " " + strings.Join(blocks, " ") + " " + strings.Join(filetokens, " ") + "\n"
1662                 return nil
1663         })
1664         err := cg.Wait()
1665         return rootdir + strings.Join(subdirs, ""), err
1666 }
1667
1668 // splitToToks is similar to bytes.SplitN(token, []byte{c}, 3), but
1669 // splits into the toks slice rather than allocating a new one, and
1670 // returns the number of toks (1, 2, or 3).
1671 func splitToToks(src []byte, c rune, toks [][]byte) int {
1672         c1 := bytes.IndexRune(src, c)
1673         if c1 < 0 {
1674                 toks[0] = src
1675                 return 1
1676         }
1677         toks[0], src = src[:c1], src[c1+1:]
1678         c2 := bytes.IndexRune(src, c)
1679         if c2 < 0 {
1680                 toks[1] = src
1681                 return 2
1682         }
1683         toks[1], toks[2] = src[:c2], src[c2+1:]
1684         return 3
1685 }
1686
1687 func (dn *dirnode) loadManifest(txt string) error {
1688         streams := bytes.Split([]byte(txt), []byte{'\n'})
1689         if len(streams[len(streams)-1]) != 0 {
1690                 return fmt.Errorf("line %d: no trailing newline", len(streams))
1691         }
1692         streams = streams[:len(streams)-1]
1693         segments := []storedSegment{}
1694         // streamoffset[n] is the position in the stream of the nth
1695         // block, i.e., âˆ‘ segments[j].size âˆ€ 0≤j<n. We ensure
1696         // len(streamoffset) == len(segments) + 1.
1697         streamoffset := []int64{0}
1698         // To reduce allocs, we reuse a single "pathparts" slice
1699         // (pre-split on "/" separators) for the duration of this
1700         // func.
1701         var pathparts []string
1702         // To reduce allocs, we reuse a single "toks" slice of 3 byte
1703         // slices.
1704         var toks = make([][]byte, 3)
1705         for i, stream := range streams {
1706                 lineno := i + 1
1707                 fnodeCache := make(map[string]*filenode)
1708                 var anyFileTokens bool
1709                 var segIdx int
1710                 segments = segments[:0]
1711                 streamoffset = streamoffset[:1]
1712                 pathparts = nil
1713                 streamparts := 0
1714                 for i, token := range bytes.Split(stream, []byte{' '}) {
1715                         if i == 0 {
1716                                 pathparts = strings.Split(manifestUnescape(string(token)), "/")
1717                                 streamparts = len(pathparts)
1718                                 continue
1719                         }
1720                         if !bytes.ContainsRune(token, ':') {
1721                                 if anyFileTokens {
1722                                         return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1723                                 }
1724                                 if splitToToks(token, '+', toks) < 2 {
1725                                         return fmt.Errorf("line %d: bad locator %q", lineno, token)
1726                                 }
1727                                 length, err := strconv.ParseInt(string(toks[1]), 10, 32)
1728                                 if err != nil || length < 0 {
1729                                         return fmt.Errorf("line %d: bad locator %q", lineno, token)
1730                                 }
1731                                 streamoffset = append(streamoffset, streamoffset[len(segments)]+int64(length))
1732                                 segments = append(segments, storedSegment{
1733                                         locator: string(token),
1734                                         size:    int(length),
1735                                         offset:  0,
1736                                         length:  int(length),
1737                                 })
1738                                 continue
1739                         } else if len(segments) == 0 {
1740                                 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1741                         }
1742                         if splitToToks(token, ':', toks) != 3 {
1743                                 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1744                         }
1745                         anyFileTokens = true
1746
1747                         offset, err := strconv.ParseInt(string(toks[0]), 10, 64)
1748                         if err != nil || offset < 0 {
1749                                 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1750                         }
1751                         length, err := strconv.ParseInt(string(toks[1]), 10, 64)
1752                         if err != nil || length < 0 {
1753                                 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1754                         }
1755                         fnode, cached := fnodeCache[string(toks[2])]
1756                         if !cached {
1757                                 if !bytes.ContainsAny(toks[2], `\/`) {
1758                                         // optimization for a common case
1759                                         pathparts = append(pathparts[:streamparts], string(toks[2]))
1760                                 } else {
1761                                         pathparts = append(pathparts[:streamparts], strings.Split(manifestUnescape(string(toks[2])), "/")...)
1762                                 }
1763                                 fnode, err = dn.createFileAndParents(pathparts)
1764                                 if err != nil {
1765                                         return fmt.Errorf("line %d: cannot use name %q with length %d: %s", lineno, toks[2], length, err)
1766                                 }
1767                                 fnodeCache[string(toks[2])] = fnode
1768                         }
1769                         if fnode == nil {
1770                                 // name matches an existing directory
1771                                 if length != 0 {
1772                                         return fmt.Errorf("line %d: cannot use name %q with length %d: is a directory", lineno, toks[2], length)
1773                                 }
1774                                 // Special case: an empty file used as
1775                                 // a marker to preserve an otherwise
1776                                 // empty directory in a manifest.
1777                                 continue
1778                         }
1779                         // Map the stream offset/range coordinates to
1780                         // block/offset/range coordinates and add
1781                         // corresponding storedSegments to the filenode
1782                         if segIdx < len(segments) && streamoffset[segIdx] <= offset && streamoffset[segIdx+1] > offset {
1783                                 // common case with an easy
1784                                 // optimization: start where the
1785                                 // previous segment ended
1786                         } else if guess := int(offset >> 26); guess >= 0 && guess < len(segments) && streamoffset[guess] <= offset && streamoffset[guess+1] > offset {
1787                                 // another common case with an easy
1788                                 // optimization: all blocks are 64 MiB
1789                                 // (or close enough)
1790                                 segIdx = guess
1791                         } else {
1792                                 // general case
1793                                 segIdx = sort.Search(len(segments), func(i int) bool {
1794                                         return streamoffset[i+1] > offset
1795                                 })
1796                         }
1797                         for ; segIdx < len(segments); segIdx++ {
1798                                 blkStart := streamoffset[segIdx]
1799                                 if blkStart >= offset+length {
1800                                         break
1801                                 }
1802                                 seg := &segments[segIdx]
1803                                 if seg.size == 0 {
1804                                         continue
1805                                 }
1806                                 var blkOff int
1807                                 if blkStart < offset {
1808                                         blkOff = int(offset - blkStart)
1809                                 }
1810                                 blkLen := seg.size - blkOff
1811                                 if blkStart+int64(seg.size) > offset+length {
1812                                         blkLen = int(offset + length - blkStart - int64(blkOff))
1813                                 }
1814                                 fnode.appendSegment(storedSegment{
1815                                         kc:      dn.fs,
1816                                         locator: seg.locator,
1817                                         size:    seg.size,
1818                                         offset:  blkOff,
1819                                         length:  blkLen,
1820                                 })
1821                         }
1822                         if segIdx == len(segments) && streamoffset[segIdx] < offset+length {
1823                                 return fmt.Errorf("line %d: invalid segment in %d-byte stream: %q", lineno, streamoffset[segIdx], token)
1824                         }
1825                 }
1826                 if !anyFileTokens {
1827                         return fmt.Errorf("line %d: no file segments", lineno)
1828                 } else if len(segments) == 0 {
1829                         return fmt.Errorf("line %d: no locators", lineno)
1830                 } else if streamparts == 0 {
1831                         return fmt.Errorf("line %d: no stream name", lineno)
1832                 }
1833         }
1834         return nil
1835 }
1836
1837 // only safe to call from loadManifest -- no locking.
1838 //
1839 // If path is a "parent directory exists" marker (the last path
1840 // component is "."), the returned values are both nil.
1841 //
1842 // Newly added nodes have modtime==0. Caller is responsible for fixing
1843 // them with backdateTree.
1844 func (dn *dirnode) createFileAndParents(names []string) (fn *filenode, err error) {
1845         var node inode = dn
1846         basename := names[len(names)-1]
1847         for _, name := range names[:len(names)-1] {
1848                 switch name {
1849                 case "", ".":
1850                         continue
1851                 case "..":
1852                         if node == dn {
1853                                 // can't be sure parent will be a *dirnode
1854                                 return nil, ErrInvalidArgument
1855                         }
1856                         node = node.Parent()
1857                         continue
1858                 }
1859                 node.Lock()
1860                 unlock := node.Unlock
1861                 node, err = node.Child(name, func(child inode) (inode, error) {
1862                         if child == nil {
1863                                 // note modtime will be fixed later in backdateTree()
1864                                 child, err := node.FS().newNode(name, 0755|os.ModeDir, time.Time{})
1865                                 if err != nil {
1866                                         return nil, err
1867                                 }
1868                                 child.SetParent(node, name)
1869                                 return child, nil
1870                         } else if !child.IsDir() {
1871                                 return child, os.ErrExist
1872                         } else {
1873                                 return child, nil
1874                         }
1875                 })
1876                 unlock()
1877                 if err != nil {
1878                         return
1879                 }
1880         }
1881         if basename == "." {
1882                 return
1883         } else if !permittedName(basename) {
1884                 err = fmt.Errorf("invalid file part %q in path %q", basename, names)
1885                 return
1886         }
1887         node.Lock()
1888         defer node.Unlock()
1889         _, err = node.Child(basename, func(child inode) (inode, error) {
1890                 switch child := child.(type) {
1891                 case nil:
1892                         child, err = node.FS().newNode(basename, 0755, time.Time{})
1893                         if err != nil {
1894                                 return nil, err
1895                         }
1896                         child.SetParent(node, basename)
1897                         fn = child.(*filenode)
1898                         return child, nil
1899                 case *filenode:
1900                         fn = child
1901                         return child, nil
1902                 case *dirnode:
1903                         return child, ErrIsDirectory
1904                 default:
1905                         return child, ErrInvalidArgument
1906                 }
1907         })
1908         return
1909 }
1910
1911 func (dn *dirnode) TreeSize() (bytes int64) {
1912         dn.RLock()
1913         defer dn.RUnlock()
1914         for _, i := range dn.inodes {
1915                 switch i := i.(type) {
1916                 case *filenode:
1917                         bytes += i.Size()
1918                 case *dirnode:
1919                         bytes += i.TreeSize()
1920                 }
1921         }
1922         return
1923 }
1924
1925 func (dn *dirnode) Snapshot() (inode, error) {
1926         return dn.snapshot()
1927 }
1928
1929 func (dn *dirnode) snapshot() (*dirnode, error) {
1930         dn.RLock()
1931         defer dn.RUnlock()
1932         snap := &dirnode{
1933                 treenode: treenode{
1934                         inodes:   make(map[string]inode, len(dn.inodes)),
1935                         fileinfo: dn.fileinfo,
1936                 },
1937         }
1938         for name, child := range dn.inodes {
1939                 dupchild, err := child.Snapshot()
1940                 if err != nil {
1941                         return nil, err
1942                 }
1943                 snap.inodes[name] = dupchild
1944                 dupchild.SetParent(snap, name)
1945         }
1946         return snap, nil
1947 }
1948
1949 func (dn *dirnode) Splice(repl inode) error {
1950         repl, err := repl.Snapshot()
1951         if err != nil {
1952                 return fmt.Errorf("cannot copy snapshot: %w", err)
1953         }
1954         switch repl := repl.(type) {
1955         default:
1956                 return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
1957         case *dirnode:
1958                 dn.Lock()
1959                 defer dn.Unlock()
1960                 dn.inodes = repl.inodes
1961                 for name, child := range dn.inodes {
1962                         child.SetParent(dn, name)
1963                 }
1964                 dn.setTreeFS(dn.fs)
1965         case *filenode:
1966                 dn.parent.Lock()
1967                 defer dn.parent.Unlock()
1968                 removing, err := dn.parent.Child(dn.fileinfo.name, nil)
1969                 if err != nil {
1970                         return fmt.Errorf("cannot use Splice to replace a top-level directory with a file: %w", ErrInvalidOperation)
1971                 } else if removing != dn {
1972                         // If ../thisdirname is not this dirnode, it
1973                         // must be an inode that wraps a dirnode, like
1974                         // a collectionFileSystem or deferrednode.
1975                         if deferred, ok := removing.(*deferrednode); ok {
1976                                 // More useful to report the type of
1977                                 // the wrapped node rather than just
1978                                 // *deferrednode. (We know the real
1979                                 // inode is already loaded because dn
1980                                 // is inside it.)
1981                                 removing = deferred.realinode()
1982                         }
1983                         return fmt.Errorf("cannot use Splice to attach a file at top level of %T: %w", removing, ErrInvalidOperation)
1984                 }
1985                 dn.Lock()
1986                 defer dn.Unlock()
1987                 _, err = dn.parent.Child(dn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
1988                 if err != nil {
1989                         return fmt.Errorf("error replacing filenode: dn.parent.Child(): %w", err)
1990                 }
1991                 repl.fs = dn.fs
1992         }
1993         return nil
1994 }
1995
1996 func (dn *dirnode) setTreeFS(fs *collectionFileSystem) {
1997         dn.fs = fs
1998         for _, child := range dn.inodes {
1999                 switch child := child.(type) {
2000                 case *dirnode:
2001                         child.setTreeFS(fs)
2002                 case *filenode:
2003                         child.fs = fs
2004                 }
2005         }
2006 }
2007
2008 // walkSegments visits all file data in the tree beneath dn, calling
2009 // fn on each segment and replacing it with fn's return value.
2010 //
2011 // caller must have lock.
2012 func (dn *dirnode) walkSegments(fn func(segment) segment) {
2013         // Visit all segments in files, then traverse subdirectories.
2014         // This way planRepack will tend to repack siblings together.
2015         names := dn.sortedNames()
2016         for _, name := range names {
2017                 child := dn.inodes[name]
2018                 child.Lock()
2019                 if child, ok := child.(*filenode); ok {
2020                         for i, seg := range child.segments {
2021                                 child.segments[i] = fn(seg)
2022                         }
2023                 }
2024                 child.Unlock()
2025         }
2026         for _, name := range names {
2027                 child := dn.inodes[name]
2028                 child.Lock()
2029                 if child, ok := child.(*dirnode); ok {
2030                         child.walkSegments(fn)
2031                 }
2032                 child.Unlock()
2033         }
2034 }
2035
2036 type segment interface {
2037         io.ReaderAt
2038         Len() int
2039         // Return a new segment with a subsection of the data from this
2040         // one. length<0 means length=Len()-off.
2041         Slice(off int, length int) segment
2042         memorySize() int64
2043 }
2044
2045 type memSegment struct {
2046         buf []byte
2047         // If flushing is not nil and not ready/closed, then a) buf is
2048         // being shared by a pruneMemSegments goroutine, and must be
2049         // copied on write; and b) the flushing channel will close
2050         // when the goroutine finishes, whether it succeeds or not.
2051         flushing <-chan struct{}
2052 }
2053
2054 func (me *memSegment) flushingUnfinished() bool {
2055         if me.flushing == nil {
2056                 return false
2057         }
2058         select {
2059         case <-me.flushing:
2060                 me.flushing = nil
2061                 return false
2062         default:
2063                 return true
2064         }
2065 }
2066
2067 func (me *memSegment) Len() int {
2068         return len(me.buf)
2069 }
2070
2071 func (me *memSegment) Slice(off, length int) segment {
2072         if length < 0 {
2073                 length = len(me.buf) - off
2074         }
2075         buf := make([]byte, length)
2076         copy(buf, me.buf[off:])
2077         return &memSegment{buf: buf}
2078 }
2079
2080 func (me *memSegment) Truncate(n int) {
2081         if n > cap(me.buf) || (me.flushing != nil && n > len(me.buf)) {
2082                 newsize := 1024
2083                 for newsize < n {
2084                         newsize = newsize << 2
2085                 }
2086                 newbuf := make([]byte, n, newsize)
2087                 copy(newbuf, me.buf)
2088                 me.buf, me.flushing = newbuf, nil
2089         } else {
2090                 // reclaim existing capacity, and zero reclaimed part
2091                 oldlen := len(me.buf)
2092                 me.buf = me.buf[:n]
2093                 for i := oldlen; i < n; i++ {
2094                         me.buf[i] = 0
2095                 }
2096         }
2097 }
2098
2099 func (me *memSegment) WriteAt(p []byte, off int) {
2100         if off+len(p) > len(me.buf) {
2101                 panic("overflowed segment")
2102         }
2103         if me.flushing != nil {
2104                 me.buf, me.flushing = append([]byte(nil), me.buf...), nil
2105         }
2106         copy(me.buf[off:], p)
2107 }
2108
2109 func (me *memSegment) ReadAt(p []byte, off int64) (n int, err error) {
2110         if off > int64(me.Len()) {
2111                 err = io.EOF
2112                 return
2113         }
2114         n = copy(p, me.buf[int(off):])
2115         if n < len(p) {
2116                 err = io.EOF
2117         }
2118         return
2119 }
2120
2121 func (me *memSegment) memorySize() int64 {
2122         return 64 + int64(len(me.buf))
2123 }
2124
2125 type storedSegment struct {
2126         kc      fsBackend
2127         locator string
2128         size    int // size of stored block (also encoded in locator)
2129         offset  int // position of segment within the stored block
2130         length  int // bytes in this segment (offset + length <= size)
2131 }
2132
2133 func (se storedSegment) Len() int {
2134         return se.length
2135 }
2136
2137 func (se storedSegment) Slice(n, size int) segment {
2138         se.offset += n
2139         se.length -= n
2140         if size >= 0 && se.length > size {
2141                 se.length = size
2142         }
2143         return se
2144 }
2145
2146 func (se storedSegment) ReadAt(p []byte, off int64) (n int, err error) {
2147         if off > int64(se.length) {
2148                 return 0, io.EOF
2149         }
2150         maxlen := se.length - int(off)
2151         if len(p) > maxlen {
2152                 p = p[:maxlen]
2153                 n, err = se.kc.ReadAt(se.locator, p, int(off)+se.offset)
2154                 if err == nil {
2155                         err = io.EOF
2156                 }
2157                 return
2158         }
2159         return se.kc.ReadAt(se.locator, p, int(off)+se.offset)
2160 }
2161
2162 func (se storedSegment) memorySize() int64 {
2163         return 64 + int64(len(se.locator))
2164 }
2165
2166 func (se storedSegment) blockSegment() BlockSegment {
2167         return BlockSegment{
2168                 Locator: se.locator,
2169                 Offset:  se.offset,
2170                 Length:  se.length,
2171         }
2172 }
2173
2174 func canonicalName(name string) string {
2175         name = path.Clean("/" + name)
2176         if name == "/" || name == "./" {
2177                 name = "."
2178         } else if strings.HasPrefix(name, "/") {
2179                 name = "." + name
2180         }
2181         return name
2182 }
2183
2184 var manifestEscapeSeq = regexp.MustCompile(`\\([0-7]{3}|\\)`)
2185
2186 func manifestUnescapeFunc(seq string) string {
2187         if seq == `\\` {
2188                 return `\`
2189         }
2190         i, err := strconv.ParseUint(seq[1:], 8, 8)
2191         if err != nil {
2192                 // Invalid escape sequence: can't unescape.
2193                 return seq
2194         }
2195         return string([]byte{byte(i)})
2196 }
2197
2198 func manifestUnescape(s string) string {
2199         return manifestEscapeSeq.ReplaceAllStringFunc(s, manifestUnescapeFunc)
2200 }
2201
2202 var manifestEscapedChar = regexp.MustCompile(`[\000-\040:\s\\]`)
2203
2204 func manifestEscapeFunc(seq string) string {
2205         return fmt.Sprintf("\\%03o", byte(seq[0]))
2206 }
2207
2208 func manifestEscape(s string) string {
2209         return manifestEscapedChar.ReplaceAllStringFunc(s, manifestEscapeFunc)
2210 }