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