1 // Copyright (C) The Arvados Authors. All rights reserved.
3 // SPDX-License-Identifier: Apache-2.0
27 maxBlockSize = 1 << 26
28 concurrentWriters = 4 // max goroutines writing to Keep in background and during flush()
31 // A CollectionFileSystem is a FileSystem that can be serialized as a
32 // manifest and stored as a collection.
33 type CollectionFileSystem interface {
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)
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)
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
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
59 Repack(context.Context, RepackOptions) (int, error)
61 // Total data bytes in all files.
65 type collectionFileSystem struct {
69 storageClasses []string
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.
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.
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.
88 // When the guessed TTL is much smaller than the real TTL,
89 // preemptive signature refresh is delayed or missed entirely,
91 guessSignatureTTL time.Duration
92 holdCheckChanges time.Time
93 lockCheckChanges sync.Mutex
95 // Pending updates to send via replace_segments. See
97 repacked map[BlockSegment]BlockSegment
98 repackedMtx sync.Mutex
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() {
107 fs := &collectionFileSystem{
109 storageClasses: c.StorageClassesDesired,
110 fileSystem: fileSystem{
111 fsBackend: keepBackend{apiClient: client, keepClient: kc},
112 thr: newThrottle(concurrentWriters),
115 fs.loadedPDH.Store(c.PortableDataHash)
116 if r := c.ReplicationDesired; r != nil {
124 mode: os.ModeDir | 0755,
126 sys: func() interface{} {
129 PortableDataHash: fs.loadedPDH.Load().(string),
130 Properties: c.Properties,
134 inodes: make(map[string]inode),
137 root.SetParent(root, ".")
138 if err := root.loadManifest(c.ManifestText); err != nil {
142 txt, err := root.marshalManifest(context.Background(), ".", false)
146 fs.savedPDH.Store(PortableDataHash(txt))
148 backdateTree(root, modTime)
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) {
164 for _, n := range n.inodes {
165 eachNode(n, ffunc, dfunc)
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
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) {
184 earliest = now.Add(time.Hour * 24 * 7 * 365)
187 fs.fileSystem.root.RLock()
188 eachNode(fs.root, func(fn *filenode) {
191 for _, seg := range fn.segments {
192 seg, ok := seg.(storedSegment)
196 expiryTime, err := signatureExpiryTime(seg.locator)
200 if expiryTime.Before(earliest) {
201 earliest = expiryTime
203 if expiryTime.After(latest) {
208 fs.fileSystem.root.RUnlock()
211 // No signatures == 100% of TTL remaining.
215 ttl := latest.Sub(now)
216 fs.fileSystem.root.Lock()
218 if ttl > fs.guessSignatureTTL {
219 // ttl is closer to the real TTL than
220 // guessSignatureTTL.
221 fs.guessSignatureTTL = ttl
223 // Use the previous best guess to compute the
224 // portion remaining (below, after unlocking
226 ttl = fs.guessSignatureTTL
229 fs.fileSystem.root.Unlock()
231 return earliest.Sub(now).Seconds() / ttl.Seconds(), ttl
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
241 fs.fileSystem.root.Lock()
242 defer fs.fileSystem.root.Unlock()
243 eachNode(fs.root, func(fn *filenode) {
246 for idx, seg := range fn.segments {
247 seg, ok := seg.(storedSegment)
251 loc, ok := newLoc[stripAllHints(seg.locator)]
256 fn.segments[idx] = seg
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
271 mode: perm | os.ModeDir,
274 inodes: make(map[string]inode),
282 mode: perm & ^os.ModeDir,
288 func (fs *collectionFileSystem) Child(name string, replace func(inode) (inode, error)) (inode, error) {
289 return fs.rootnode().Child(name, replace)
292 func (fs *collectionFileSystem) FS() FileSystem {
296 func (fs *collectionFileSystem) FileInfo() os.FileInfo {
297 return fs.rootnode().FileInfo()
300 func (fs *collectionFileSystem) IsDir() bool {
304 func (fs *collectionFileSystem) Lock() {
308 func (fs *collectionFileSystem) Unlock() {
309 fs.rootnode().Unlock()
312 func (fs *collectionFileSystem) RLock() {
313 fs.rootnode().RLock()
316 func (fs *collectionFileSystem) RUnlock() {
317 fs.rootnode().RUnlock()
320 func (fs *collectionFileSystem) Parent() inode {
321 return fs.rootnode().Parent()
324 func (fs *collectionFileSystem) Read(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
325 return 0, ptr, ErrInvalidOperation
328 func (fs *collectionFileSystem) Write(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
329 return 0, ptr, ErrInvalidOperation
332 func (fs *collectionFileSystem) Readdir() ([]os.FileInfo, error) {
333 return fs.rootnode().Readdir()
336 func (fs *collectionFileSystem) SetParent(parent inode, name string) {
337 fs.rootnode().SetParent(parent, name)
340 func (fs *collectionFileSystem) Truncate(int64) error {
341 return ErrInvalidOperation
344 // Check for and incorporate upstream changes. If force==false, this
345 // is a no-op except once every ttl/100 or so.
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() == "" {
354 fs.lockCheckChanges.Lock()
355 if !force && fs.holdCheckChanges.After(time.Now()) {
356 fs.lockCheckChanges.Unlock()
359 remain, ttl := fs.signatureTimeLeft()
361 fs.holdCheckChanges = time.Now().Add(ttl / 100)
363 fs.lockCheckChanges.Unlock()
365 if !force && remain >= 0.5 {
366 // plenty of time left on current signatures
370 loadedPDH, _ := fs.loadedPDH.Load().(string)
371 getparams := map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}}
373 reloadTime := time.Now()
375 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+fs.uuid, nil, getparams)
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.
398 newfs, err := coll.FileSystem(fs.fileSystem.fsBackend, fs.fileSystem.fsBackend)
402 snap, err := Snapshot(newfs, "/")
406 err = Splice(fs, "/", snap)
410 fs.loadedPDH.Store(coll.PortableDataHash)
411 fs.savedPDH.Store(newfs.(*collectionFileSystem).savedPDH.Load())
412 fs.reloadTime = reloadTime
415 fs.updateSignatures(coll.ManifestText)
420 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+loadedPDH, nil, getparams)
424 fs.updateSignatures(coll.ManifestText)
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)
441 loadedPDH, _ := fs.loadedPDH.Load().(string)
443 for _, id := range []string{fs.uuid, loadedPDH} {
448 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+id, nil, map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}})
452 manifests += coll.ManifestText
454 hash := stripAllHints(locator)
455 for _, tok := range regexp.MustCompile(`\S+`).FindAllString(manifests, -1) {
456 if mBlkRe.MatchString(tok) {
457 if stripAllHints(tok) == hash {
463 go fs.updateSignatures(manifests)
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,
475 fs.repackedMtx.Unlock()
476 return fmt.Errorf("sync failed: replace_segments %s: %w", fs.uuid, err)
480 fs.repackedMtx.Unlock()
482 refreshed, err := fs.checkChangesOnServer(true)
486 if refreshed || fs.uuid == "" {
489 txt, err := fs.MarshalManifest(".")
491 return fmt.Errorf("sync failed: %s", err)
493 savingPDH := PortableDataHash(txt)
494 if savingPDH == fs.savedPDH.Load() {
495 // No local changes since last save or initial load.
503 selectFields := []string{"uuid", "portable_data_hash"}
504 fs.lockCheckChanges.Lock()
505 remain, _ := fs.signatureTimeLeft()
506 fs.lockCheckChanges.Unlock()
508 selectFields = append(selectFields, "manifest_text")
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,
515 "select": selectFields,
518 return fmt.Errorf("sync failed: update %s: %w", fs.uuid, err)
520 fs.updateSignatures(coll.ManifestText)
521 fs.loadedPDH.Store(coll.PortableDataHash)
522 fs.savedPDH.Store(savingPDH)
526 func (fs *collectionFileSystem) Flush(path string, shortBlocks bool) error {
527 node, err := rlookup(fs.fileSystem.root, path, nil)
531 dn, ok := node.(*dirnode)
533 return ErrNotADirectory
537 names := dn.sortedNames()
539 // Caller only wants to flush the specified dir,
540 // non-recursively. Drop subdirs from the list of
542 var filenames []string
543 for _, name := range names {
544 if _, ok := dn.inodes[name].(*filenode); ok {
545 filenames = append(filenames, name)
550 for _, name := range names {
551 child := dn.inodes[name]
555 return dn.flush(context.TODO(), names, flushOpts{sync: false, shortBlocks: shortBlocks})
558 func (fs *collectionFileSystem) MemorySize() int64 {
559 return fs.fileSystem.root.(*dirnode).MemorySize()
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)
568 func (fs *collectionFileSystem) Size() int64 {
569 return fs.fileSystem.root.(*dirnode).TreeSize()
572 func (fs *collectionFileSystem) Snapshot() (inode, error) {
573 return fs.fileSystem.root.Snapshot()
576 func (fs *collectionFileSystem) Splice(r inode) error {
577 return fs.fileSystem.root.Splice(r)
580 func (fs *collectionFileSystem) Repack(ctx context.Context, opts RepackOptions) (int, error) {
581 return fs.repackTree(ctx, opts, fs.root.(*dirnode))
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()
592 return len(plan), nil
594 repacked, err := fs.repackData(ctx, plan)
598 replaced, err := fs.replaceSegments(repacked)
602 nReplaced := len(replaced)
604 fs.repackedMtx.Lock()
605 if len(repacked) == 0 {
607 } else if len(fs.repacked) == 0 {
608 fs.repacked = repacked
610 // Merge new repacking results with existing unsaved
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
621 repacked[orig] = repl
624 fs.repacked = repacked
626 fs.repackedMtx.Unlock()
628 return nReplaced, nil
631 func (fs *collectionFileSystem) ReplaceSegments(m map[BlockSegment]BlockSegment) (bool, error) {
632 changed, err := fs.replaceSegments(m)
633 return len(changed) > 0, err
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)
644 fs.fileSystem.root.(*dirnode).walkSegments(func(seg segment) segment {
645 if seg, ok := seg.(storedSegment); ok {
646 delete(missing, seg.blockSegment().StripAllHints())
650 skip := make(map[string]bool)
651 for orig, repl := range m {
652 orig.Locator = stripAllHints(orig.Locator)
654 skip[repl.Locator] = true
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)
665 if splitToToks([]byte(repl.Locator), '+', toks) < 2 {
666 return nil, errors.New("invalid replacement locator")
668 blksize, err := strconv.ParseInt(string(toks[1]), 10, 64)
670 return nil, fmt.Errorf("invalid size hint in replacement locator: %w", err)
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)
675 todo[orig] = storedSegment{
676 locator: repl.Locator,
682 changed := make(map[BlockSegment]BlockSegment, len(todo))
683 fs.fileSystem.root.(*dirnode).walkSegments(func(s segment) segment {
684 seg, ok := s.(storedSegment)
688 orig := seg.blockSegment().StripAllHints()
689 repl, ok := todo[orig]
693 seg.locator = repl.locator
694 seg.offset = repl.offset
696 // (leave seg.kc and seg.length unchanged)
697 changed[orig] = seg.blockSegment()
703 // See (*collectionFileSystem)planRepack.
704 type repackBucketThreshold struct {
709 var fullRepackBucketThresholds = []repackBucketThreshold{
710 {maxIn: 1 << 25, minOut: 1 << 25},
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},
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.
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
735 thresholds = fullRepackBucketThresholds
737 thresholds = repackBucketThresholds
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 {
746 if _, err := ss.kc.BlockRead(ctx, BlockReadOptions{
748 CheckCacheOnly: true,
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 {
761 bucketBlocks[bucket] = append(bucketBlocks[bucket], hash)
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]
774 for _, hash := range bucketBlocks[bucket] {
775 if _, planned := blockPlan[hash]; planned || slices.Contains(pending, hash) {
776 // already planned to merge this block
779 size := blockSize[hash]
780 if pendingSize+size > maxBlockSize {
781 for _, hash := range pending {
782 blockPlan[hash] = len(plan)
784 plan = append(plan, nil)
785 pending = pending[:0]
789 pending = append(pending, hash)
791 if pendingSize >= thresholds[bucket].minOut {
792 for _, hash := range pending {
793 blockPlan[hash] = len(plan)
795 plan = append(plan, nil)
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)
807 hash := stripAllHints(ss.locator)
808 if idx, planning := blockPlan[hash]; planning && !done[ss] {
809 plan[idx] = append(plan[idx], ss)
817 // Given a plan returned by planRepack, write new blocks with the
818 // merged segment data, and return a replacement mapping suitable for
820 func (fs *collectionFileSystem) repackData(ctx context.Context, plan [][]storedSegment) (repl map[BlockSegment]BlockSegment, err error) {
824 repl = make(map[BlockSegment]BlockSegment)
825 for _, insegments := range plan {
826 // TODO: concurrency > 1
828 for _, insegment := range insegments {
829 outsize += insegment.length
831 if outsize > maxBlockSize {
832 return nil, fmt.Errorf("combined length %d would exceed maximum block size %d", outsize, maxBlockSize)
834 piper, pipew := io.Pipe()
836 for _, insegment := range insegments {
837 n, err := io.Copy(pipew, io.NewSectionReader(insegment, 0, int64(insegment.length)))
839 pipew.CloseWithError(err)
842 if n != int64(insegment.length) {
843 pipew.CloseWithError(fmt.Errorf("internal error: copied %d bytes, expected %d", n, insegment.length))
846 if ctx.Err() != nil {
847 pipew.CloseWithError(ctx.Err())
853 wrote, err := fs.BlockWrite(ctx, BlockWriteOptions{Reader: piper, DataSize: outsize})
858 for _, insegment := range insegments {
859 repl[insegment.blockSegment().StripAllHints()] = BlockSegment{
860 Locator: wrote.Locator,
862 Length: insegment.length,
864 offset += insegment.length
870 // filenodePtr is an offset into a file that is (usually) efficient to
871 // seek to. Specifically, if filenode.repacked==filenodePtr.repacked
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 {
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
888 // If startPtr is beyond EOF, ptr.segment* will indicate precisely
893 // ptr.segmentIdx == len(filenode.segments) // i.e., at EOF
895 // filenode.segments[ptr.segmentIdx].Len() > ptr.segmentOff
896 func (fn *filenode) seek(startPtr filenodePtr) (ptr filenodePtr) {
899 // meaningless anyway
901 } else if ptr.off >= fn.fileinfo.size {
902 ptr.segmentIdx = len(fn.segments)
904 ptr.repacked = fn.repacked
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
910 if ptr.segmentOff >= fn.segments[ptr.segmentIdx].Len() {
917 ptr.repacked = fn.repacked
919 if ptr.off >= fn.fileinfo.size {
920 ptr.segmentIdx, ptr.segmentOff = len(fn.segments), 0
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.
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
933 segLen := int64(fn.segments[ptr.segmentIdx].Len())
934 if off+segLen > ptr.off {
935 ptr.segmentOff = int(ptr.off - off)
943 // filenode implements inode.
944 type filenode struct {
946 fs *collectionFileSystem
949 // number of times `segments` has changed in a
950 // way that might invalidate a filenodePtr
952 memsize int64 // bytes in memSegments
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())
963 func (fn *filenode) SetParent(p inode, name string) {
967 fn.fileinfo.name = name
970 func (fn *filenode) Parent() inode {
976 func (fn *filenode) FS() FileSystem {
980 func (fn *filenode) MemorySize() (size int64) {
984 for _, seg := range fn.segments {
985 size += seg.memorySize()
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
993 func (fn *filenode) Read(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr, err error) {
994 ptr = fn.seek(startPtr)
996 err = ErrNegativeOffset
999 if ptr.segmentIdx >= len(fn.segments) {
1003 if ss, ok := fn.segments[ptr.segmentIdx].(storedSegment); ok {
1004 ss.locator = fn.fs.refreshSignature(ss.locator)
1005 fn.segments[ptr.segmentIdx] = ss
1007 n, err = fn.segments[ptr.segmentIdx].ReadAt(p, int64(ptr.segmentOff))
1011 if ptr.segmentOff == fn.segments[ptr.segmentIdx].Len() {
1014 if ptr.segmentIdx < len(fn.segments) && err == io.EOF {
1022 func (fn *filenode) Size() int64 {
1025 return fn.fileinfo.Size()
1028 func (fn *filenode) FileInfo() os.FileInfo {
1034 func (fn *filenode) Truncate(size int64) error {
1037 return fn.truncate(size)
1040 func (fn *filenode) truncate(size int64) error {
1041 if size == fn.fileinfo.size {
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())
1052 if ptr.segmentOff == 0 {
1053 fn.segments = fn.segments[:ptr.segmentIdx]
1055 fn.segments = fn.segments[:ptr.segmentIdx+1]
1056 switch seg := fn.segments[ptr.segmentIdx].(type) {
1058 seg.Truncate(ptr.segmentOff)
1059 fn.memsize += int64(seg.Len())
1061 fn.segments[ptr.segmentIdx] = seg.Slice(0, ptr.segmentOff)
1064 fn.fileinfo.size = size
1067 for size > fn.fileinfo.size {
1068 grow := size - fn.fileinfo.size
1071 if len(fn.segments) == 0 {
1073 fn.segments = append(fn.segments, seg)
1074 } else if seg, ok = fn.segments[len(fn.segments)-1].(*memSegment); !ok || seg.Len() >= maxBlockSize {
1076 fn.segments = append(fn.segments, seg)
1078 if maxgrow := int64(maxBlockSize - seg.Len()); maxgrow < grow {
1081 seg.Truncate(seg.Len() + int(grow))
1082 fn.fileinfo.size += grow
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
1096 ptr = fn.seek(startPtr)
1098 err = ErrNegativeOffset
1101 for len(p) > 0 && err == nil {
1103 if len(cando) > maxBlockSize {
1104 cando = cando[:maxBlockSize]
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
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)
1116 var prevAppendable bool
1117 if prev >= 0 && fn.segments[prev].Len() < maxBlockSize {
1118 _, prevAppendable = fn.segments[prev].(*memSegment)
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.
1126 fn.segments = append(fn.segments, nil)
1127 copy(fn.segments[cur+1:], fn.segments[cur:])
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)
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)
1148 } else if curWritable {
1149 if fit := int(fn.segments[cur].Len()) - ptr.segmentOff; fit < len(cando) {
1154 // Shrink cando if needed to fit in
1156 if cangrow := maxBlockSize - fn.segments[prev].Len(); cangrow < len(cando) {
1157 cando = cando[:cangrow]
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).
1172 copy(fn.segments[cur:], fn.segments[cur+1:])
1173 fn.segments = fn.segments[:len(fn.segments)-1]
1175 // shrink cur by the same #bytes we're growing prev
1176 fn.segments[cur] = fn.segments[cur].Slice(len(cando), -1)
1182 ptr.segmentOff = fn.segments[prev].Len()
1183 fn.segments[prev].(*memSegment).Truncate(ptr.segmentOff + len(cando))
1184 fn.memsize += int64(len(cando))
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:])
1196 // appending a new segment does
1197 // not invalidate any ptrs
1199 seg := &memSegment{}
1200 seg.Truncate(len(cando))
1201 fn.memsize += int64(len(cando))
1202 fn.segments[cur] = seg
1206 // Finally we can copy bytes from cando to the current segment.
1207 fn.segments[ptr.segmentIdx].(*memSegment).WriteAt(cando, ptr.segmentOff)
1211 ptr.off += int64(len(cando))
1212 ptr.segmentOff += len(cando)
1213 if ptr.segmentOff >= maxBlockSize {
1214 fn.pruneMemSegments()
1216 if fn.segments[ptr.segmentIdx].Len() == ptr.segmentOff {
1221 fn.fileinfo.modTime = time.Now()
1226 // Write some data out to disk to reduce memory use. Caller must have
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 {
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{})
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()
1249 resp, err := fn.FS().BlockWrite(context.Background(), BlockWriteOptions{
1251 Replicas: fn.fs.replicas,
1252 StorageClasses: fn.fs.storageClasses,
1254 fn.fs.throttle().Release()
1257 if seg.flushing != done {
1258 // A new seg.buf has been allocated.
1262 // TODO: stall (or return errors from)
1263 // subsequent writes until flushing
1264 // starts to succeed.
1267 if len(fn.segments) <= idx || fn.segments[idx] != seg || len(seg.buf) != len(buf) {
1268 // Segment has been dropped/moved/resized.
1271 fn.memsize -= int64(len(buf))
1272 fn.segments[idx] = storedSegment{
1274 locator: resp.Locator,
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{}
1288 for _, seg := range fn.segments {
1289 if seg, ok := seg.(*memSegment); ok && seg.flushing != nil {
1290 pending = append(pending, seg.flushing)
1294 for _, p := range pending {
1299 func (fn *filenode) Snapshot() (inode, error) {
1302 segments := make([]segment, 0, len(fn.segments))
1303 for _, seg := range fn.segments {
1304 segments = append(segments, seg.Slice(0, seg.Len()))
1307 fileinfo: fn.fileinfo,
1310 // Clear references to the original filesystem, otherwise the
1311 // snapshot will prevent the old filesystem from being garbage
1317 func (fn *filenode) Splice(repl inode) error {
1318 repl, err := repl.Snapshot()
1323 defer fn.parent.Unlock()
1326 _, err = fn.parent.Child(fn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
1330 switch repl := repl.(type) {
1332 repl.parent = fn.parent
1333 repl.fileinfo.name = fn.fileinfo.name
1334 repl.setTreeFS(fn.fs)
1336 repl.parent = fn.parent
1337 repl.fileinfo.name = fn.fileinfo.name
1340 return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
1345 // Caller must have lock.
1346 func (fn *filenode) setFS(fs *collectionFileSystem) {
1348 for i, seg := range fn.segments {
1349 if ss, ok := seg.(storedSegment); ok {
1356 type dirnode struct {
1357 fs *collectionFileSystem
1361 func (dn *dirnode) FS() FileSystem {
1365 func (dn *dirnode) Child(name string, replace func(inode) (inode, error)) (inode, error) {
1366 if dn == dn.fs.rootnode() && name == ".arvados#collection" {
1367 gn := &getternode{Getter: func() ([]byte, error) {
1370 coll.ManifestText, err = dn.fs.MarshalManifest(".")
1374 coll.UUID = dn.fs.uuid
1375 data, err := json.Marshal(&coll)
1377 data = append(data, '\n')
1381 gn.SetParent(dn, name)
1384 return dn.treenode.Child(name, replace)
1387 type fnSegmentRef struct {
1392 // commitBlock concatenates the data from the given filenode segments
1393 // (which must be *memSegments), writes the data out to Keep as a
1394 // single block, and replaces the filenodes' *memSegments with
1395 // storedSegments that reference the relevant portions of the new
1398 // bufsize is the total data size in refs. It is used to preallocate
1399 // the correct amount of memory when len(refs)>1.
1401 // If sync is false, commitBlock returns right away, after starting a
1402 // goroutine to do the writes, reacquire the filenodes' locks, and
1403 // swap out the *memSegments. Some filenodes' segments might get
1404 // modified/rearranged in the meantime, in which case commitBlock
1405 // won't replace them.
1407 // Caller must have write lock.
1408 func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, bufsize int, sync bool) error {
1412 if err := ctx.Err(); err != nil {
1415 done := make(chan struct{})
1417 segs := make([]*memSegment, 0, len(refs))
1418 offsets := make([]int, 0, len(refs)) // location of segment's data within block
1419 for _, ref := range refs {
1420 seg := ref.fn.segments[ref.idx].(*memSegment)
1421 if !sync && seg.flushingUnfinished() {
1422 // Let the other flushing goroutine finish. If
1423 // it fails, we'll try again next time.
1427 // In sync mode, we proceed regardless of
1428 // whether another flush is in progress: It
1429 // can't finish before we do, because we hold
1430 // fn's lock until we finish our own writes.
1432 offsets = append(offsets, len(block))
1435 } else if block == nil {
1436 block = append(make([]byte, 0, bufsize), seg.buf...)
1438 block = append(block, seg.buf...)
1440 segs = append(segs, seg)
1442 blocksize := len(block)
1443 dn.fs.throttle().Acquire()
1444 errs := make(chan error, 1)
1448 resp, err := dn.fs.BlockWrite(context.Background(), BlockWriteOptions{
1450 Replicas: dn.fs.replicas,
1451 StorageClasses: dn.fs.storageClasses,
1453 dn.fs.throttle().Release()
1458 for idx, ref := range refs {
1461 // In async mode, fn's lock was
1462 // released while we were waiting for
1463 // PutB(); lots of things might have
1465 if len(ref.fn.segments) <= ref.idx {
1466 // file segments have
1467 // rearranged or changed in
1471 } else if seg, ok := ref.fn.segments[ref.idx].(*memSegment); !ok || seg != segs[idx] {
1472 // segment has been replaced
1475 } else if seg.flushing != done {
1476 // seg.buf has been replaced
1481 data := ref.fn.segments[ref.idx].(*memSegment).buf
1482 ref.fn.segments[ref.idx] = storedSegment{
1484 locator: resp.Locator,
1486 offset: offsets[idx],
1489 // atomic is needed here despite caller having
1490 // lock: caller might be running concurrent
1491 // commitBlock() goroutines using the same
1492 // lock, writing different segments from the
1494 atomic.AddInt64(&ref.fn.memsize, -int64(len(data)))
1506 type flushOpts struct {
1511 // flush in-memory data and remote-cluster block references (for the
1512 // children with the given names, which must be children of dn) to
1513 // local-cluster persistent storage.
1515 // Caller must have write lock on dn and the named children.
1517 // If any children are dirs, they will be flushed recursively.
1518 func (dn *dirnode) flush(ctx context.Context, names []string, opts flushOpts) error {
1519 cg := newContextGroup(ctx)
1522 goCommit := func(refs []fnSegmentRef, bufsize int) {
1523 cg.Go(func() error {
1524 return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync)
1528 var pending []fnSegmentRef
1529 var pendingLen int = 0
1530 localLocator := map[string]string{}
1531 for _, name := range names {
1532 switch node := dn.inodes[name].(type) {
1534 grandchildNames := node.sortedNames()
1535 for _, grandchildName := range grandchildNames {
1536 grandchild := node.inodes[grandchildName]
1538 defer grandchild.Unlock()
1540 cg.Go(func() error { return node.flush(cg.Context(), grandchildNames, opts) })
1542 for idx, seg := range node.segments {
1543 switch seg := seg.(type) {
1545 loc, ok := localLocator[seg.locator]
1548 loc, err = dn.fs.LocalLocator(seg.locator)
1552 localLocator[seg.locator] = loc
1555 node.segments[idx] = seg
1557 if seg.Len() > maxBlockSize/2 {
1558 goCommit([]fnSegmentRef{{node, idx}}, seg.Len())
1561 if pendingLen+seg.Len() > maxBlockSize {
1562 goCommit(pending, pendingLen)
1566 pending = append(pending, fnSegmentRef{node, idx})
1567 pendingLen += seg.Len()
1569 panic(fmt.Sprintf("can't sync segment type %T", seg))
1574 if opts.shortBlocks {
1575 goCommit(pending, pendingLen)
1580 func (dn *dirnode) MemorySize() (size int64) {
1582 todo := make([]inode, 0, len(dn.inodes))
1583 for _, node := range dn.inodes {
1584 todo = append(todo, node)
1588 for _, node := range todo {
1589 size += node.MemorySize()
1594 // caller must have write lock.
1595 func (dn *dirnode) sortedNames() []string {
1596 names := make([]string, 0, len(dn.inodes))
1597 for name := range dn.inodes {
1598 names = append(names, name)
1604 // caller must have write lock.
1605 func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, flush bool) (string, error) {
1606 cg := newContextGroup(ctx)
1609 if len(dn.inodes) == 0 {
1613 // Express the existence of an empty directory by
1614 // adding an empty file named `\056`, which (unlike
1615 // the more obvious spelling `.`) is accepted by the
1616 // API's manifest validator.
1617 return manifestEscape(prefix) + " d41d8cd98f00b204e9800998ecf8427e+0 0:0:\\056\n", nil
1620 names := dn.sortedNames()
1622 // Wait for children to finish any pending write operations
1623 // before locking them.
1624 for _, name := range names {
1625 node := dn.inodes[name]
1626 if fn, ok := node.(*filenode); ok {
1631 var dirnames []string
1632 var filenames []string
1633 for _, name := range names {
1634 node := dn.inodes[name]
1637 switch node := node.(type) {
1639 dirnames = append(dirnames, name)
1641 filenames = append(filenames, name)
1643 panic(fmt.Sprintf("can't marshal inode type %T", node))
1647 subdirs := make([]string, len(dirnames))
1649 for i, name := range dirnames {
1651 cg.Go(func() error {
1652 txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name, flush)
1658 cg.Go(func() error {
1660 type filepart struct {
1666 var fileparts []filepart
1669 // skip flush -- will fail below if anything
1671 } else if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil {
1674 for _, name := range filenames {
1675 node := dn.inodes[name].(*filenode)
1676 if len(node.segments) == 0 {
1677 fileparts = append(fileparts, filepart{name: name})
1680 for _, seg := range node.segments {
1681 switch seg := seg.(type) {
1683 if len(blocks) > 0 && blocks[len(blocks)-1] == seg.locator {
1684 streamLen -= int64(seg.size)
1686 blocks = append(blocks, seg.locator)
1690 offset: streamLen + int64(seg.offset),
1691 length: int64(seg.length),
1693 if prev := len(fileparts) - 1; prev >= 0 &&
1694 fileparts[prev].name == name &&
1695 fileparts[prev].offset+fileparts[prev].length == next.offset {
1696 fileparts[prev].length += next.length
1698 fileparts = append(fileparts, next)
1700 streamLen += int64(seg.size)
1702 // We haven't unlocked since
1703 // calling flush(sync=true).
1704 // Evidently the caller passed
1705 // flush==false but there were
1707 return fmt.Errorf("can't marshal segment type %T", seg)
1711 var filetokens []string
1712 for _, s := range fileparts {
1713 filetokens = append(filetokens, fmt.Sprintf("%d:%d:%s", s.offset, s.length, manifestEscape(s.name)))
1715 if len(filetokens) == 0 {
1717 } else if len(blocks) == 0 {
1718 blocks = []string{"d41d8cd98f00b204e9800998ecf8427e+0"}
1720 rootdir = manifestEscape(prefix) + " " + strings.Join(blocks, " ") + " " + strings.Join(filetokens, " ") + "\n"
1724 return rootdir + strings.Join(subdirs, ""), err
1727 // splitToToks is similar to bytes.SplitN(token, []byte{c}, 3), but
1728 // splits into the toks slice rather than allocating a new one, and
1729 // returns the number of toks (1, 2, or 3).
1730 func splitToToks(src []byte, c rune, toks [][]byte) int {
1731 c1 := bytes.IndexRune(src, c)
1736 toks[0], src = src[:c1], src[c1+1:]
1737 c2 := bytes.IndexRune(src, c)
1742 toks[1], toks[2] = src[:c2], src[c2+1:]
1746 func (dn *dirnode) loadManifest(txt string) error {
1747 streams := bytes.Split([]byte(txt), []byte{'\n'})
1748 if len(streams[len(streams)-1]) != 0 {
1749 return fmt.Errorf("line %d: no trailing newline", len(streams))
1751 streams = streams[:len(streams)-1]
1752 segments := []storedSegment{}
1753 // streamoffset[n] is the position in the stream of the nth
1754 // block, i.e., ∑ segments[j].size ∀ 0≤j<n. We ensure
1755 // len(streamoffset) == len(segments) + 1.
1756 streamoffset := []int64{0}
1757 // To reduce allocs, we reuse a single "pathparts" slice
1758 // (pre-split on "/" separators) for the duration of this
1760 var pathparts []string
1761 // To reduce allocs, we reuse a single "toks" slice of 3 byte
1763 var toks = make([][]byte, 3)
1764 for i, stream := range streams {
1766 fnodeCache := make(map[string]*filenode)
1767 var anyFileTokens bool
1769 segments = segments[:0]
1770 streamoffset = streamoffset[:1]
1773 for i, token := range bytes.Split(stream, []byte{' '}) {
1775 pathparts = strings.Split(manifestUnescape(string(token)), "/")
1776 streamparts = len(pathparts)
1779 if !bytes.ContainsRune(token, ':') {
1781 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1783 if splitToToks(token, '+', toks) < 2 {
1784 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1786 length, err := strconv.ParseInt(string(toks[1]), 10, 32)
1787 if err != nil || length < 0 {
1788 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1790 streamoffset = append(streamoffset, streamoffset[len(segments)]+int64(length))
1791 segments = append(segments, storedSegment{
1792 locator: string(token),
1795 length: int(length),
1798 } else if len(segments) == 0 {
1799 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1801 if splitToToks(token, ':', toks) != 3 {
1802 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1804 anyFileTokens = true
1806 offset, err := strconv.ParseInt(string(toks[0]), 10, 64)
1807 if err != nil || offset < 0 {
1808 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1810 length, err := strconv.ParseInt(string(toks[1]), 10, 64)
1811 if err != nil || length < 0 {
1812 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1814 fnode, cached := fnodeCache[string(toks[2])]
1816 if !bytes.ContainsAny(toks[2], `\/`) {
1817 // optimization for a common case
1818 pathparts = append(pathparts[:streamparts], string(toks[2]))
1820 pathparts = append(pathparts[:streamparts], strings.Split(manifestUnescape(string(toks[2])), "/")...)
1822 fnode, err = dn.createFileAndParents(pathparts)
1824 return fmt.Errorf("line %d: cannot use name %q with length %d: %s", lineno, toks[2], length, err)
1826 fnodeCache[string(toks[2])] = fnode
1829 // name matches an existing directory
1831 return fmt.Errorf("line %d: cannot use name %q with length %d: is a directory", lineno, toks[2], length)
1833 // Special case: an empty file used as
1834 // a marker to preserve an otherwise
1835 // empty directory in a manifest.
1838 // Map the stream offset/range coordinates to
1839 // block/offset/range coordinates and add
1840 // corresponding storedSegments to the filenode
1841 if segIdx < len(segments) && streamoffset[segIdx] <= offset && streamoffset[segIdx+1] > offset {
1842 // common case with an easy
1843 // optimization: start where the
1844 // previous segment ended
1845 } else if guess := int(offset >> 26); guess >= 0 && guess < len(segments) && streamoffset[guess] <= offset && streamoffset[guess+1] > offset {
1846 // another common case with an easy
1847 // optimization: all blocks are 64 MiB
1848 // (or close enough)
1852 segIdx = sort.Search(len(segments), func(i int) bool {
1853 return streamoffset[i+1] > offset
1856 for ; segIdx < len(segments); segIdx++ {
1857 blkStart := streamoffset[segIdx]
1858 if blkStart >= offset+length {
1861 seg := &segments[segIdx]
1866 if blkStart < offset {
1867 blkOff = int(offset - blkStart)
1869 blkLen := seg.size - blkOff
1870 if blkStart+int64(seg.size) > offset+length {
1871 blkLen = int(offset + length - blkStart - int64(blkOff))
1873 fnode.appendSegment(storedSegment{
1875 locator: seg.locator,
1881 if segIdx == len(segments) && streamoffset[segIdx] < offset+length {
1882 return fmt.Errorf("line %d: invalid segment in %d-byte stream: %q", lineno, streamoffset[segIdx], token)
1886 return fmt.Errorf("line %d: no file segments", lineno)
1887 } else if len(segments) == 0 {
1888 return fmt.Errorf("line %d: no locators", lineno)
1889 } else if streamparts == 0 {
1890 return fmt.Errorf("line %d: no stream name", lineno)
1896 // only safe to call from loadManifest -- no locking.
1898 // If path is a "parent directory exists" marker (the last path
1899 // component is "."), the returned values are both nil.
1901 // Newly added nodes have modtime==0. Caller is responsible for fixing
1902 // them with backdateTree.
1903 func (dn *dirnode) createFileAndParents(names []string) (fn *filenode, err error) {
1905 basename := names[len(names)-1]
1906 for _, name := range names[:len(names)-1] {
1912 // can't be sure parent will be a *dirnode
1913 return nil, ErrInvalidArgument
1915 node = node.Parent()
1919 unlock := node.Unlock
1920 node, err = node.Child(name, func(child inode) (inode, error) {
1922 // note modtime will be fixed later in backdateTree()
1923 child, err := node.FS().newNode(name, 0755|os.ModeDir, time.Time{})
1927 child.SetParent(node, name)
1929 } else if !child.IsDir() {
1930 return child, os.ErrExist
1940 if basename == "." {
1942 } else if !permittedName(basename) {
1943 err = fmt.Errorf("invalid file part %q in path %q", basename, names)
1948 _, err = node.Child(basename, func(child inode) (inode, error) {
1949 switch child := child.(type) {
1951 child, err = node.FS().newNode(basename, 0755, time.Time{})
1955 child.SetParent(node, basename)
1956 fn = child.(*filenode)
1962 return child, ErrIsDirectory
1964 return child, ErrInvalidArgument
1970 func (dn *dirnode) TreeSize() (bytes int64) {
1973 for _, i := range dn.inodes {
1974 switch i := i.(type) {
1978 bytes += i.TreeSize()
1984 func (dn *dirnode) Snapshot() (inode, error) {
1985 return dn.snapshot()
1988 func (dn *dirnode) snapshot() (*dirnode, error) {
1993 inodes: make(map[string]inode, len(dn.inodes)),
1994 fileinfo: dn.fileinfo,
1997 for name, child := range dn.inodes {
1998 dupchild, err := child.Snapshot()
2002 snap.inodes[name] = dupchild
2003 dupchild.SetParent(snap, name)
2008 func (dn *dirnode) Splice(repl inode) error {
2009 repl, err := repl.Snapshot()
2011 return fmt.Errorf("cannot copy snapshot: %w", err)
2013 switch repl := repl.(type) {
2015 return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
2019 dn.inodes = repl.inodes
2020 for name, child := range dn.inodes {
2021 child.SetParent(dn, name)
2026 defer dn.parent.Unlock()
2027 removing, err := dn.parent.Child(dn.fileinfo.name, nil)
2029 return fmt.Errorf("cannot use Splice to replace a top-level directory with a file: %w", ErrInvalidOperation)
2030 } else if removing != dn {
2031 // If ../thisdirname is not this dirnode, it
2032 // must be an inode that wraps a dirnode, like
2033 // a collectionFileSystem or deferrednode.
2034 if deferred, ok := removing.(*deferrednode); ok {
2035 // More useful to report the type of
2036 // the wrapped node rather than just
2037 // *deferrednode. (We know the real
2038 // inode is already loaded because dn
2040 removing = deferred.realinode()
2042 return fmt.Errorf("cannot use Splice to attach a file at top level of %T: %w", removing, ErrInvalidOperation)
2046 _, err = dn.parent.Child(dn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
2048 return fmt.Errorf("error replacing filenode: dn.parent.Child(): %w", err)
2055 func (dn *dirnode) setTreeFS(fs *collectionFileSystem) {
2057 for _, child := range dn.inodes {
2058 switch child := child.(type) {
2067 // walkSegments visits all file data in the tree beneath dn, calling
2068 // fn on each segment and replacing it with fn's return value.
2070 // caller must have lock.
2071 func (dn *dirnode) walkSegments(fn func(segment) segment) {
2072 // Visit all segments in files, then traverse subdirectories.
2073 // This way planRepack will tend to repack siblings together.
2074 names := dn.sortedNames()
2075 for _, name := range names {
2076 child := dn.inodes[name]
2078 if child, ok := child.(*filenode); ok {
2079 for i, seg := range child.segments {
2080 child.segments[i] = fn(seg)
2085 for _, name := range names {
2086 child := dn.inodes[name]
2088 if child, ok := child.(*dirnode); ok {
2089 child.walkSegments(fn)
2095 type segment interface {
2098 // Return a new segment with a subsection of the data from this
2099 // one. length<0 means length=Len()-off.
2100 Slice(off int, length int) segment
2104 type memSegment struct {
2106 // If flushing is not nil and not ready/closed, then a) buf is
2107 // being shared by a pruneMemSegments goroutine, and must be
2108 // copied on write; and b) the flushing channel will close
2109 // when the goroutine finishes, whether it succeeds or not.
2110 flushing <-chan struct{}
2113 func (me *memSegment) flushingUnfinished() bool {
2114 if me.flushing == nil {
2126 func (me *memSegment) Len() int {
2130 func (me *memSegment) Slice(off, length int) segment {
2132 length = len(me.buf) - off
2134 buf := make([]byte, length)
2135 copy(buf, me.buf[off:])
2136 return &memSegment{buf: buf}
2139 func (me *memSegment) Truncate(n int) {
2140 if n > cap(me.buf) || (me.flushing != nil && n > len(me.buf)) {
2143 newsize = newsize << 2
2145 newbuf := make([]byte, n, newsize)
2146 copy(newbuf, me.buf)
2147 me.buf, me.flushing = newbuf, nil
2149 // reclaim existing capacity, and zero reclaimed part
2150 oldlen := len(me.buf)
2152 for i := oldlen; i < n; i++ {
2158 func (me *memSegment) WriteAt(p []byte, off int) {
2159 if off+len(p) > len(me.buf) {
2160 panic("overflowed segment")
2162 if me.flushing != nil {
2163 me.buf, me.flushing = append([]byte(nil), me.buf...), nil
2165 copy(me.buf[off:], p)
2168 func (me *memSegment) ReadAt(p []byte, off int64) (n int, err error) {
2169 if off > int64(me.Len()) {
2173 n = copy(p, me.buf[int(off):])
2180 func (me *memSegment) memorySize() int64 {
2181 return 64 + int64(len(me.buf))
2184 type storedSegment struct {
2187 size int // size of stored block (also encoded in locator)
2188 offset int // position of segment within the stored block
2189 length int // bytes in this segment (offset + length <= size)
2192 func (se storedSegment) Len() int {
2196 func (se storedSegment) Slice(n, size int) segment {
2199 if size >= 0 && se.length > size {
2205 func (se storedSegment) ReadAt(p []byte, off int64) (n int, err error) {
2206 if off > int64(se.length) {
2209 maxlen := se.length - int(off)
2210 if len(p) > maxlen {
2212 n, err = se.kc.ReadAt(se.locator, p, int(off)+se.offset)
2218 return se.kc.ReadAt(se.locator, p, int(off)+se.offset)
2221 func (se storedSegment) memorySize() int64 {
2222 return 64 + int64(len(se.locator))
2225 func (se storedSegment) blockSegment() BlockSegment {
2226 return BlockSegment{
2227 Locator: se.locator,
2233 func canonicalName(name string) string {
2234 name = path.Clean("/" + name)
2235 if name == "/" || name == "./" {
2237 } else if strings.HasPrefix(name, "/") {
2243 var manifestEscapeSeq = regexp.MustCompile(`\\([0-7]{3}|\\)`)
2245 func manifestUnescapeFunc(seq string) string {
2249 i, err := strconv.ParseUint(seq[1:], 8, 8)
2251 // Invalid escape sequence: can't unescape.
2254 return string([]byte{byte(i)})
2257 func manifestUnescape(s string) string {
2258 return manifestEscapeSeq.ReplaceAllStringFunc(s, manifestUnescapeFunc)
2261 var manifestEscapedChar = regexp.MustCompile(`[\000-\040:\s\\]`)
2263 func manifestEscapeFunc(seq string) string {
2264 return fmt.Sprintf("\\%03o", byte(seq[0]))
2267 func manifestEscape(s string) string {
2268 return manifestEscapedChar.ReplaceAllStringFunc(s, manifestEscapeFunc)