1 // Copyright (C) The Arvados Authors. All rights reserved.
3 // SPDX-License-Identifier: Apache-2.0
25 maxBlockSize = 1 << 26
26 concurrentWriters = 4 // max goroutines writing to Keep in background and during flush()
29 // A CollectionFileSystem is a FileSystem that can be serialized as a
30 // manifest and stored as a collection.
31 type CollectionFileSystem interface {
34 // Flush all file data to Keep and return a snapshot of the
35 // filesystem suitable for saving as (Collection)ManifestText.
36 // Prefix (normally ".") is a top level directory, effectively
37 // prepended to all paths in the returned manifest.
38 MarshalManifest(prefix string) (string, error)
40 // Total data bytes in all files.
44 type collectionFileSystem struct {
48 storageClasses []string
50 // PDH returned by the server as of last sync/load.
51 loadedPDH atomic.Value
52 // PDH of the locally generated manifest as of last
53 // sync/load. This can differ from loadedPDH after loading a
54 // version that was generated with different code and sorts
55 // filenames differently than we do, for example.
58 // guessSignatureTTL tracks a lower bound for the server's
59 // configured BlobSigningTTL. The guess is initially zero, and
60 // increases when we come across a signature with an expiry
61 // time further in the future than the previous guess.
63 // When the guessed TTL is much smaller than the real TTL,
64 // preemptive signature refresh is delayed or missed entirely,
66 guessSignatureTTL time.Duration
67 holdCheckChanges time.Time
68 lockCheckChanges sync.Mutex
71 // FileSystem returns a CollectionFileSystem for the collection.
72 func (c *Collection) FileSystem(client apiClient, kc keepClient) (CollectionFileSystem, error) {
73 modTime := c.ModifiedAt
77 fs := &collectionFileSystem{
79 storageClasses: c.StorageClassesDesired,
80 fileSystem: fileSystem{
81 fsBackend: keepBackend{apiClient: client, keepClient: kc},
82 thr: newThrottle(concurrentWriters),
85 fs.loadedPDH.Store(c.PortableDataHash)
86 if r := c.ReplicationDesired; r != nil {
94 mode: os.ModeDir | 0755,
96 sys: func() interface{} { return c },
98 inodes: make(map[string]inode),
101 root.SetParent(root, ".")
102 if err := root.loadManifest(c.ManifestText); err != nil {
106 txt, err := root.marshalManifest(context.Background(), ".", false)
110 fs.savedPDH.Store(PortableDataHash(txt))
112 backdateTree(root, modTime)
117 // caller must have lock (or guarantee no concurrent accesses somehow)
118 func eachNode(n inode, ffunc func(*filenode), dfunc func(*dirnode)) {
119 switch n := n.(type) {
128 for _, n := range n.inodes {
129 eachNode(n, ffunc, dfunc)
134 // caller must have lock (or guarantee no concurrent accesses somehow)
135 func backdateTree(n inode, modTime time.Time) {
136 eachNode(n, func(fn *filenode) {
137 fn.fileinfo.modTime = modTime
138 }, func(dn *dirnode) {
139 dn.fileinfo.modTime = modTime
143 // Approximate portion of signature TTL remaining, usually between 0
144 // and 1, or negative if some signatures have expired.
145 func (fs *collectionFileSystem) signatureTimeLeft() (float64, time.Duration) {
148 earliest = now.Add(time.Hour * 24 * 7 * 365)
151 fs.fileSystem.root.RLock()
152 eachNode(fs.root, func(fn *filenode) {
155 for _, seg := range fn.segments {
156 seg, ok := seg.(storedSegment)
160 expiryTime, err := signatureExpiryTime(seg.locator)
164 if expiryTime.Before(earliest) {
165 earliest = expiryTime
167 if expiryTime.After(latest) {
172 fs.fileSystem.root.RUnlock()
175 // No signatures == 100% of TTL remaining.
179 ttl := latest.Sub(now)
180 fs.fileSystem.root.Lock()
182 if ttl > fs.guessSignatureTTL {
183 // ttl is closer to the real TTL than
184 // guessSignatureTTL.
185 fs.guessSignatureTTL = ttl
187 // Use the previous best guess to compute the
188 // portion remaining (below, after unlocking
190 ttl = fs.guessSignatureTTL
193 fs.fileSystem.root.Unlock()
195 return earliest.Sub(now).Seconds() / ttl.Seconds(), ttl
198 func (fs *collectionFileSystem) updateSignatures(newmanifest string) {
199 newLoc := map[string]string{}
200 for _, tok := range regexp.MustCompile(`\S+`).FindAllString(newmanifest, -1) {
201 if mBlkRe.MatchString(tok) {
202 newLoc[stripAllHints(tok)] = tok
205 fs.fileSystem.root.Lock()
206 defer fs.fileSystem.root.Unlock()
207 eachNode(fs.root, func(fn *filenode) {
210 for idx, seg := range fn.segments {
211 seg, ok := seg.(storedSegment)
215 loc, ok := newLoc[stripAllHints(seg.locator)]
220 fn.segments[idx] = seg
225 func (fs *collectionFileSystem) newNode(name string, perm os.FileMode, modTime time.Time) (node inode, err error) {
226 if name == "" || name == "." || name == ".." {
227 return nil, ErrInvalidArgument
235 mode: perm | os.ModeDir,
238 inodes: make(map[string]inode),
246 mode: perm & ^os.ModeDir,
252 func (fs *collectionFileSystem) Child(name string, replace func(inode) (inode, error)) (inode, error) {
253 return fs.rootnode().Child(name, replace)
256 func (fs *collectionFileSystem) FS() FileSystem {
260 func (fs *collectionFileSystem) FileInfo() os.FileInfo {
261 return fs.rootnode().FileInfo()
264 func (fs *collectionFileSystem) IsDir() bool {
268 func (fs *collectionFileSystem) Lock() {
272 func (fs *collectionFileSystem) Unlock() {
273 fs.rootnode().Unlock()
276 func (fs *collectionFileSystem) RLock() {
277 fs.rootnode().RLock()
280 func (fs *collectionFileSystem) RUnlock() {
281 fs.rootnode().RUnlock()
284 func (fs *collectionFileSystem) Parent() inode {
285 return fs.rootnode().Parent()
288 func (fs *collectionFileSystem) Read(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
289 return 0, ptr, ErrInvalidOperation
292 func (fs *collectionFileSystem) Write(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
293 return 0, ptr, ErrInvalidOperation
296 func (fs *collectionFileSystem) Readdir() ([]os.FileInfo, error) {
297 return fs.rootnode().Readdir()
300 func (fs *collectionFileSystem) SetParent(parent inode, name string) {
301 fs.rootnode().SetParent(parent, name)
304 func (fs *collectionFileSystem) Truncate(int64) error {
305 return ErrInvalidOperation
308 // Check for and incorporate upstream changes. If force==false, this
309 // is a no-op except once every ttl/100 or so.
311 // Return value is true if new content was loaded from upstream and
312 // any unsaved local changes have been discarded.
313 func (fs *collectionFileSystem) checkChangesOnServer(force bool) (bool, error) {
314 if fs.uuid == "" && fs.loadedPDH.Load() == "" {
318 fs.lockCheckChanges.Lock()
319 if !force && fs.holdCheckChanges.After(time.Now()) {
320 fs.lockCheckChanges.Unlock()
323 remain, ttl := fs.signatureTimeLeft()
325 fs.holdCheckChanges = time.Now().Add(ttl / 100)
327 fs.lockCheckChanges.Unlock()
329 if !force && remain >= 0.5 {
330 // plenty of time left on current signatures
334 loadedPDH, _ := fs.loadedPDH.Load().(string)
335 getparams := map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}}
338 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+fs.uuid, nil, getparams)
342 if coll.PortableDataHash != loadedPDH {
343 // collection has changed upstream since we
344 // last loaded or saved. Refresh local data,
345 // losing any unsaved local changes.
346 newfs, err := coll.FileSystem(fs.fileSystem.fsBackend, fs.fileSystem.fsBackend)
350 snap, err := Snapshot(newfs, "/")
354 err = Splice(fs, "/", snap)
358 fs.loadedPDH.Store(coll.PortableDataHash)
359 fs.savedPDH.Store(newfs.(*collectionFileSystem).savedPDH.Load())
362 fs.updateSignatures(coll.ManifestText)
367 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+loadedPDH, nil, getparams)
371 fs.updateSignatures(coll.ManifestText)
376 // Refresh signature on a single locator, if necessary. Assume caller
377 // has lock. If an update is needed, and there are any storedSegments
378 // whose signatures can be updated, start a background task to update
379 // them asynchronously when the caller releases locks.
380 func (fs *collectionFileSystem) refreshSignature(locator string) string {
381 exp, err := signatureExpiryTime(locator)
382 if err != nil || exp.Sub(time.Now()) > time.Minute {
383 // Synchronous update is not needed. Start an
384 // asynchronous update if needed.
385 go fs.checkChangesOnServer(false)
388 loadedPDH, _ := fs.loadedPDH.Load().(string)
390 for _, id := range []string{fs.uuid, loadedPDH} {
395 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+id, nil, map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}})
399 manifests += coll.ManifestText
401 hash := stripAllHints(locator)
402 for _, tok := range regexp.MustCompile(`\S+`).FindAllString(manifests, -1) {
403 if mBlkRe.MatchString(tok) {
404 if stripAllHints(tok) == hash {
410 go fs.updateSignatures(manifests)
414 func (fs *collectionFileSystem) Sync() error {
415 refreshed, err := fs.checkChangesOnServer(true)
419 if refreshed || fs.uuid == "" {
422 txt, err := fs.MarshalManifest(".")
424 return fmt.Errorf("sync failed: %s", err)
426 savingPDH := PortableDataHash(txt)
427 if savingPDH == fs.savedPDH.Load() {
428 // No local changes since last save or initial load.
436 selectFields := []string{"uuid", "portable_data_hash"}
437 fs.lockCheckChanges.Lock()
438 remain, _ := fs.signatureTimeLeft()
439 fs.lockCheckChanges.Unlock()
441 selectFields = append(selectFields, "manifest_text")
444 err = fs.RequestAndDecode(&coll, "PUT", "arvados/v1/collections/"+fs.uuid, nil, map[string]interface{}{
445 "collection": map[string]string{
446 "manifest_text": coll.ManifestText,
448 "select": selectFields,
451 return fmt.Errorf("sync failed: update %s: %w", fs.uuid, err)
453 fs.updateSignatures(coll.ManifestText)
454 fs.loadedPDH.Store(coll.PortableDataHash)
455 fs.savedPDH.Store(savingPDH)
459 func (fs *collectionFileSystem) Flush(path string, shortBlocks bool) error {
460 node, err := rlookup(fs.fileSystem.root, path)
464 dn, ok := node.(*dirnode)
466 return ErrNotADirectory
470 names := dn.sortedNames()
472 // Caller only wants to flush the specified dir,
473 // non-recursively. Drop subdirs from the list of
475 var filenames []string
476 for _, name := range names {
477 if _, ok := dn.inodes[name].(*filenode); ok {
478 filenames = append(filenames, name)
483 for _, name := range names {
484 child := dn.inodes[name]
488 return dn.flush(context.TODO(), names, flushOpts{sync: false, shortBlocks: shortBlocks})
491 func (fs *collectionFileSystem) MemorySize() int64 {
492 return fs.fileSystem.root.(*dirnode).MemorySize()
495 func (fs *collectionFileSystem) MarshalManifest(prefix string) (string, error) {
496 fs.fileSystem.root.Lock()
497 defer fs.fileSystem.root.Unlock()
498 return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix, true)
501 func (fs *collectionFileSystem) Size() int64 {
502 return fs.fileSystem.root.(*dirnode).TreeSize()
505 func (fs *collectionFileSystem) Snapshot() (inode, error) {
506 return fs.fileSystem.root.Snapshot()
509 func (fs *collectionFileSystem) Splice(r inode) error {
510 return fs.fileSystem.root.Splice(r)
513 // filenodePtr is an offset into a file that is (usually) efficient to
514 // seek to. Specifically, if filenode.repacked==filenodePtr.repacked
516 // filenode.segments[filenodePtr.segmentIdx][filenodePtr.segmentOff]
517 // corresponds to file offset filenodePtr.off. Otherwise, it is
518 // necessary to reexamine len(filenode.segments[0]) etc. to find the
519 // correct segment and offset.
520 type filenodePtr struct {
527 // seek returns a ptr that is consistent with both startPtr.off and
528 // the current state of fn. The caller must already hold fn.RLock() or
531 // If startPtr is beyond EOF, ptr.segment* will indicate precisely
536 // ptr.segmentIdx == len(filenode.segments) // i.e., at EOF
538 // filenode.segments[ptr.segmentIdx].Len() > ptr.segmentOff
539 func (fn *filenode) seek(startPtr filenodePtr) (ptr filenodePtr) {
542 // meaningless anyway
544 } else if ptr.off >= fn.fileinfo.size {
545 ptr.segmentIdx = len(fn.segments)
547 ptr.repacked = fn.repacked
549 } else if ptr.repacked == fn.repacked {
550 // segmentIdx and segmentOff accurately reflect
551 // ptr.off, but might have fallen off the end of a
553 if ptr.segmentOff >= fn.segments[ptr.segmentIdx].Len() {
560 ptr.repacked = fn.repacked
562 if ptr.off >= fn.fileinfo.size {
563 ptr.segmentIdx, ptr.segmentOff = len(fn.segments), 0
566 // Recompute segmentIdx and segmentOff. We have already
567 // established fn.fileinfo.size > ptr.off >= 0, so we don't
568 // have to deal with edge cases here.
570 for ptr.segmentIdx, ptr.segmentOff = 0, 0; off < ptr.off; ptr.segmentIdx++ {
571 // This would panic (index out of range) if
572 // fn.fileinfo.size were larger than
573 // sum(fn.segments[i].Len()) -- but that can't happen
574 // because we have ensured fn.fileinfo.size is always
576 segLen := int64(fn.segments[ptr.segmentIdx].Len())
577 if off+segLen > ptr.off {
578 ptr.segmentOff = int(ptr.off - off)
586 // filenode implements inode.
587 type filenode struct {
589 fs *collectionFileSystem
592 // number of times `segments` has changed in a
593 // way that might invalidate a filenodePtr
595 memsize int64 // bytes in memSegments
600 // caller must have lock
601 func (fn *filenode) appendSegment(e segment) {
602 fn.segments = append(fn.segments, e)
603 fn.fileinfo.size += int64(e.Len())
606 func (fn *filenode) SetParent(p inode, name string) {
610 fn.fileinfo.name = name
613 func (fn *filenode) Parent() inode {
619 func (fn *filenode) FS() FileSystem {
623 func (fn *filenode) MemorySize() (size int64) {
627 for _, seg := range fn.segments {
628 size += seg.memorySize()
633 // Read reads file data from a single segment, starting at startPtr,
634 // into p. startPtr is assumed not to be up-to-date. Caller must have
636 func (fn *filenode) Read(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr, err error) {
637 ptr = fn.seek(startPtr)
639 err = ErrNegativeOffset
642 if ptr.segmentIdx >= len(fn.segments) {
646 if ss, ok := fn.segments[ptr.segmentIdx].(storedSegment); ok {
647 ss.locator = fn.fs.refreshSignature(ss.locator)
648 fn.segments[ptr.segmentIdx] = ss
650 n, err = fn.segments[ptr.segmentIdx].ReadAt(p, int64(ptr.segmentOff))
654 if ptr.segmentOff == fn.segments[ptr.segmentIdx].Len() {
657 if ptr.segmentIdx < len(fn.segments) && err == io.EOF {
665 func (fn *filenode) Size() int64 {
668 return fn.fileinfo.Size()
671 func (fn *filenode) FileInfo() os.FileInfo {
677 func (fn *filenode) Truncate(size int64) error {
680 return fn.truncate(size)
683 func (fn *filenode) truncate(size int64) error {
684 if size == fn.fileinfo.size {
688 if size < fn.fileinfo.size {
689 ptr := fn.seek(filenodePtr{off: size})
690 for i := ptr.segmentIdx; i < len(fn.segments); i++ {
691 if seg, ok := fn.segments[i].(*memSegment); ok {
692 fn.memsize -= int64(seg.Len())
695 if ptr.segmentOff == 0 {
696 fn.segments = fn.segments[:ptr.segmentIdx]
698 fn.segments = fn.segments[:ptr.segmentIdx+1]
699 switch seg := fn.segments[ptr.segmentIdx].(type) {
701 seg.Truncate(ptr.segmentOff)
702 fn.memsize += int64(seg.Len())
704 fn.segments[ptr.segmentIdx] = seg.Slice(0, ptr.segmentOff)
707 fn.fileinfo.size = size
710 for size > fn.fileinfo.size {
711 grow := size - fn.fileinfo.size
714 if len(fn.segments) == 0 {
716 fn.segments = append(fn.segments, seg)
717 } else if seg, ok = fn.segments[len(fn.segments)-1].(*memSegment); !ok || seg.Len() >= maxBlockSize {
719 fn.segments = append(fn.segments, seg)
721 if maxgrow := int64(maxBlockSize - seg.Len()); maxgrow < grow {
724 seg.Truncate(seg.Len() + int(grow))
725 fn.fileinfo.size += grow
731 // Write writes data from p to the file, starting at startPtr,
732 // extending the file size if necessary. Caller must have Lock.
733 func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr, err error) {
734 if startPtr.off > fn.fileinfo.size {
735 if err = fn.truncate(startPtr.off); err != nil {
736 return 0, startPtr, err
739 ptr = fn.seek(startPtr)
741 err = ErrNegativeOffset
744 for len(p) > 0 && err == nil {
746 if len(cando) > maxBlockSize {
747 cando = cando[:maxBlockSize]
749 // Rearrange/grow fn.segments (and shrink cando if
750 // needed) such that cando can be copied to
751 // fn.segments[ptr.segmentIdx] at offset
753 cur := ptr.segmentIdx
754 prev := ptr.segmentIdx - 1
756 if cur < len(fn.segments) {
757 _, curWritable = fn.segments[cur].(*memSegment)
759 var prevAppendable bool
760 if prev >= 0 && fn.segments[prev].Len() < maxBlockSize {
761 _, prevAppendable = fn.segments[prev].(*memSegment)
763 if ptr.segmentOff > 0 && !curWritable {
764 // Split a non-writable block.
765 if max := fn.segments[cur].Len() - ptr.segmentOff; max <= len(cando) {
766 // Truncate cur, and insert a new
769 fn.segments = append(fn.segments, nil)
770 copy(fn.segments[cur+1:], fn.segments[cur:])
772 // Split cur into two copies, truncate
773 // the one on the left, shift the one
774 // on the right, and insert a new
775 // segment between them.
776 fn.segments = append(fn.segments, nil, nil)
777 copy(fn.segments[cur+2:], fn.segments[cur:])
778 fn.segments[cur+2] = fn.segments[cur+2].Slice(ptr.segmentOff+len(cando), -1)
783 seg.Truncate(len(cando))
784 fn.memsize += int64(len(cando))
785 fn.segments[cur] = seg
786 fn.segments[prev] = fn.segments[prev].Slice(0, ptr.segmentOff)
791 } else if curWritable {
792 if fit := int(fn.segments[cur].Len()) - ptr.segmentOff; fit < len(cando) {
797 // Shrink cando if needed to fit in
799 if cangrow := maxBlockSize - fn.segments[prev].Len(); cangrow < len(cando) {
800 cando = cando[:cangrow]
804 if cur == len(fn.segments) {
805 // ptr is at EOF, filesize is changing.
806 fn.fileinfo.size += int64(len(cando))
807 } else if el := fn.segments[cur].Len(); el <= len(cando) {
808 // cando is long enough that we won't
809 // need cur any more. shrink cando to
810 // be exactly as long as cur
811 // (otherwise we'd accidentally shift
812 // the effective position of all
813 // segments after cur).
815 copy(fn.segments[cur:], fn.segments[cur+1:])
816 fn.segments = fn.segments[:len(fn.segments)-1]
818 // shrink cur by the same #bytes we're growing prev
819 fn.segments[cur] = fn.segments[cur].Slice(len(cando), -1)
825 ptr.segmentOff = fn.segments[prev].Len()
826 fn.segments[prev].(*memSegment).Truncate(ptr.segmentOff + len(cando))
827 fn.memsize += int64(len(cando))
831 // Insert a segment between prev and
832 // cur, and advance prev/cur.
833 fn.segments = append(fn.segments, nil)
834 if cur < len(fn.segments) {
835 copy(fn.segments[cur+1:], fn.segments[cur:])
839 // appending a new segment does
840 // not invalidate any ptrs
843 seg.Truncate(len(cando))
844 fn.memsize += int64(len(cando))
845 fn.segments[cur] = seg
849 // Finally we can copy bytes from cando to the current segment.
850 fn.segments[ptr.segmentIdx].(*memSegment).WriteAt(cando, ptr.segmentOff)
854 ptr.off += int64(len(cando))
855 ptr.segmentOff += len(cando)
856 if ptr.segmentOff >= maxBlockSize {
857 fn.pruneMemSegments()
859 if fn.segments[ptr.segmentIdx].Len() == ptr.segmentOff {
864 fn.fileinfo.modTime = time.Now()
869 // Write some data out to disk to reduce memory use. Caller must have
871 func (fn *filenode) pruneMemSegments() {
872 // TODO: share code with (*dirnode)flush()
873 // TODO: pack/flush small blocks too, when fragmented
874 for idx, seg := range fn.segments {
875 seg, ok := seg.(*memSegment)
876 if !ok || seg.Len() < maxBlockSize || seg.flushing != nil {
879 // Setting seg.flushing guarantees seg.buf will not be
880 // modified in place: WriteAt and Truncate will
881 // allocate a new buf instead, if necessary.
882 idx, buf := idx, seg.buf
883 done := make(chan struct{})
885 // If lots of background writes are already in
886 // progress, block here until one finishes, rather
887 // than pile up an unlimited number of buffered writes
888 // and network flush operations.
889 fn.fs.throttle().Acquire()
892 resp, err := fn.FS().BlockWrite(context.Background(), BlockWriteOptions{
894 Replicas: fn.fs.replicas,
895 StorageClasses: fn.fs.storageClasses,
897 fn.fs.throttle().Release()
900 if seg.flushing != done {
901 // A new seg.buf has been allocated.
905 // TODO: stall (or return errors from)
906 // subsequent writes until flushing
907 // starts to succeed.
910 if len(fn.segments) <= idx || fn.segments[idx] != seg || len(seg.buf) != len(buf) {
911 // Segment has been dropped/moved/resized.
914 fn.memsize -= int64(len(buf))
915 fn.segments[idx] = storedSegment{
917 locator: resp.Locator,
926 // Block until all pending pruneMemSegments/flush work is
927 // finished. Caller must NOT have lock.
928 func (fn *filenode) waitPrune() {
929 var pending []<-chan struct{}
931 for _, seg := range fn.segments {
932 if seg, ok := seg.(*memSegment); ok && seg.flushing != nil {
933 pending = append(pending, seg.flushing)
937 for _, p := range pending {
942 func (fn *filenode) Snapshot() (inode, error) {
945 segments := make([]segment, 0, len(fn.segments))
946 for _, seg := range fn.segments {
947 segments = append(segments, seg.Slice(0, seg.Len()))
950 fileinfo: fn.fileinfo,
955 func (fn *filenode) Splice(repl inode) error {
956 repl, err := repl.Snapshot()
961 defer fn.parent.Unlock()
964 _, err = fn.parent.Child(fn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
968 switch repl := repl.(type) {
970 repl.parent = fn.parent
971 repl.fileinfo.name = fn.fileinfo.name
972 repl.setTreeFS(fn.fs)
974 repl.parent = fn.parent
975 repl.fileinfo.name = fn.fileinfo.name
978 return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
983 type dirnode struct {
984 fs *collectionFileSystem
988 func (dn *dirnode) FS() FileSystem {
992 func (dn *dirnode) Child(name string, replace func(inode) (inode, error)) (inode, error) {
993 if dn == dn.fs.rootnode() && name == ".arvados#collection" {
994 gn := &getternode{Getter: func() ([]byte, error) {
997 coll.ManifestText, err = dn.fs.MarshalManifest(".")
1001 coll.UUID = dn.fs.uuid
1002 data, err := json.Marshal(&coll)
1004 data = append(data, '\n')
1008 gn.SetParent(dn, name)
1011 return dn.treenode.Child(name, replace)
1014 type fnSegmentRef struct {
1019 // commitBlock concatenates the data from the given filenode segments
1020 // (which must be *memSegments), writes the data out to Keep as a
1021 // single block, and replaces the filenodes' *memSegments with
1022 // storedSegments that reference the relevant portions of the new
1025 // bufsize is the total data size in refs. It is used to preallocate
1026 // the correct amount of memory when len(refs)>1.
1028 // If sync is false, commitBlock returns right away, after starting a
1029 // goroutine to do the writes, reacquire the filenodes' locks, and
1030 // swap out the *memSegments. Some filenodes' segments might get
1031 // modified/rearranged in the meantime, in which case commitBlock
1032 // won't replace them.
1034 // Caller must have write lock.
1035 func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, bufsize int, sync bool) error {
1039 if err := ctx.Err(); err != nil {
1042 done := make(chan struct{})
1044 segs := make([]*memSegment, 0, len(refs))
1045 offsets := make([]int, 0, len(refs)) // location of segment's data within block
1046 for _, ref := range refs {
1047 seg := ref.fn.segments[ref.idx].(*memSegment)
1048 if !sync && seg.flushingUnfinished() {
1049 // Let the other flushing goroutine finish. If
1050 // it fails, we'll try again next time.
1054 // In sync mode, we proceed regardless of
1055 // whether another flush is in progress: It
1056 // can't finish before we do, because we hold
1057 // fn's lock until we finish our own writes.
1059 offsets = append(offsets, len(block))
1062 } else if block == nil {
1063 block = append(make([]byte, 0, bufsize), seg.buf...)
1065 block = append(block, seg.buf...)
1067 segs = append(segs, seg)
1069 blocksize := len(block)
1070 dn.fs.throttle().Acquire()
1071 errs := make(chan error, 1)
1075 resp, err := dn.fs.BlockWrite(context.Background(), BlockWriteOptions{
1077 Replicas: dn.fs.replicas,
1078 StorageClasses: dn.fs.storageClasses,
1080 dn.fs.throttle().Release()
1085 for idx, ref := range refs {
1088 // In async mode, fn's lock was
1089 // released while we were waiting for
1090 // PutB(); lots of things might have
1092 if len(ref.fn.segments) <= ref.idx {
1093 // file segments have
1094 // rearranged or changed in
1098 } else if seg, ok := ref.fn.segments[ref.idx].(*memSegment); !ok || seg != segs[idx] {
1099 // segment has been replaced
1102 } else if seg.flushing != done {
1103 // seg.buf has been replaced
1108 data := ref.fn.segments[ref.idx].(*memSegment).buf
1109 ref.fn.segments[ref.idx] = storedSegment{
1111 locator: resp.Locator,
1113 offset: offsets[idx],
1116 // atomic is needed here despite caller having
1117 // lock: caller might be running concurrent
1118 // commitBlock() goroutines using the same
1119 // lock, writing different segments from the
1121 atomic.AddInt64(&ref.fn.memsize, -int64(len(data)))
1133 type flushOpts struct {
1138 // flush in-memory data and remote-cluster block references (for the
1139 // children with the given names, which must be children of dn) to
1140 // local-cluster persistent storage.
1142 // Caller must have write lock on dn and the named children.
1144 // If any children are dirs, they will be flushed recursively.
1145 func (dn *dirnode) flush(ctx context.Context, names []string, opts flushOpts) error {
1146 cg := newContextGroup(ctx)
1149 goCommit := func(refs []fnSegmentRef, bufsize int) {
1150 cg.Go(func() error {
1151 return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync)
1155 var pending []fnSegmentRef
1156 var pendingLen int = 0
1157 localLocator := map[string]string{}
1158 for _, name := range names {
1159 switch node := dn.inodes[name].(type) {
1161 grandchildNames := node.sortedNames()
1162 for _, grandchildName := range grandchildNames {
1163 grandchild := node.inodes[grandchildName]
1165 defer grandchild.Unlock()
1167 cg.Go(func() error { return node.flush(cg.Context(), grandchildNames, opts) })
1169 for idx, seg := range node.segments {
1170 switch seg := seg.(type) {
1172 loc, ok := localLocator[seg.locator]
1175 loc, err = dn.fs.LocalLocator(seg.locator)
1179 localLocator[seg.locator] = loc
1182 node.segments[idx] = seg
1184 if seg.Len() > maxBlockSize/2 {
1185 goCommit([]fnSegmentRef{{node, idx}}, seg.Len())
1188 if pendingLen+seg.Len() > maxBlockSize {
1189 goCommit(pending, pendingLen)
1193 pending = append(pending, fnSegmentRef{node, idx})
1194 pendingLen += seg.Len()
1196 panic(fmt.Sprintf("can't sync segment type %T", seg))
1201 if opts.shortBlocks {
1202 goCommit(pending, pendingLen)
1207 func (dn *dirnode) MemorySize() (size int64) {
1209 todo := make([]inode, 0, len(dn.inodes))
1210 for _, node := range dn.inodes {
1211 todo = append(todo, node)
1215 for _, node := range todo {
1216 size += node.MemorySize()
1221 // caller must have write lock.
1222 func (dn *dirnode) sortedNames() []string {
1223 names := make([]string, 0, len(dn.inodes))
1224 for name := range dn.inodes {
1225 names = append(names, name)
1231 // caller must have write lock.
1232 func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, flush bool) (string, error) {
1233 cg := newContextGroup(ctx)
1236 if len(dn.inodes) == 0 {
1240 // Express the existence of an empty directory by
1241 // adding an empty file named `\056`, which (unlike
1242 // the more obvious spelling `.`) is accepted by the
1243 // API's manifest validator.
1244 return manifestEscape(prefix) + " d41d8cd98f00b204e9800998ecf8427e+0 0:0:\\056\n", nil
1247 names := dn.sortedNames()
1249 // Wait for children to finish any pending write operations
1250 // before locking them.
1251 for _, name := range names {
1252 node := dn.inodes[name]
1253 if fn, ok := node.(*filenode); ok {
1258 var dirnames []string
1259 var filenames []string
1260 for _, name := range names {
1261 node := dn.inodes[name]
1264 switch node := node.(type) {
1266 dirnames = append(dirnames, name)
1268 filenames = append(filenames, name)
1270 panic(fmt.Sprintf("can't marshal inode type %T", node))
1274 subdirs := make([]string, len(dirnames))
1276 for i, name := range dirnames {
1278 cg.Go(func() error {
1279 txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name, flush)
1285 cg.Go(func() error {
1287 type filepart struct {
1293 var fileparts []filepart
1296 // skip flush -- will fail below if anything
1298 } else if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil {
1301 for _, name := range filenames {
1302 node := dn.inodes[name].(*filenode)
1303 if len(node.segments) == 0 {
1304 fileparts = append(fileparts, filepart{name: name})
1307 for _, seg := range node.segments {
1308 switch seg := seg.(type) {
1310 if len(blocks) > 0 && blocks[len(blocks)-1] == seg.locator {
1311 streamLen -= int64(seg.size)
1313 blocks = append(blocks, seg.locator)
1317 offset: streamLen + int64(seg.offset),
1318 length: int64(seg.length),
1320 if prev := len(fileparts) - 1; prev >= 0 &&
1321 fileparts[prev].name == name &&
1322 fileparts[prev].offset+fileparts[prev].length == next.offset {
1323 fileparts[prev].length += next.length
1325 fileparts = append(fileparts, next)
1327 streamLen += int64(seg.size)
1329 // We haven't unlocked since
1330 // calling flush(sync=true).
1331 // Evidently the caller passed
1332 // flush==false but there were
1334 return fmt.Errorf("can't marshal segment type %T", seg)
1338 var filetokens []string
1339 for _, s := range fileparts {
1340 filetokens = append(filetokens, fmt.Sprintf("%d:%d:%s", s.offset, s.length, manifestEscape(s.name)))
1342 if len(filetokens) == 0 {
1344 } else if len(blocks) == 0 {
1345 blocks = []string{"d41d8cd98f00b204e9800998ecf8427e+0"}
1347 rootdir = manifestEscape(prefix) + " " + strings.Join(blocks, " ") + " " + strings.Join(filetokens, " ") + "\n"
1351 return rootdir + strings.Join(subdirs, ""), err
1354 func (dn *dirnode) loadManifest(txt string) error {
1355 streams := bytes.Split([]byte(txt), []byte{'\n'})
1356 if len(streams[len(streams)-1]) != 0 {
1357 return fmt.Errorf("line %d: no trailing newline", len(streams))
1359 streams = streams[:len(streams)-1]
1360 segments := []storedSegment{}
1361 // To reduce allocs, we reuse a single "pathparts" slice
1362 // (pre-split on "/" separators) for the duration of this
1364 var pathparts []string
1365 // To reduce allocs, we reuse a single "toks" slice of 3 byte
1367 var toks = make([][]byte, 3)
1368 // Similar to bytes.SplitN(token, []byte{c}, 3), but splits
1369 // into the toks slice rather than allocating a new one, and
1370 // returns the number of toks (1, 2, or 3).
1371 splitToToks := func(src []byte, c rune) int {
1372 c1 := bytes.IndexRune(src, c)
1377 toks[0], src = src[:c1], src[c1+1:]
1378 c2 := bytes.IndexRune(src, c)
1383 toks[1], toks[2] = src[:c2], src[c2+1:]
1386 for i, stream := range streams {
1388 var anyFileTokens bool
1391 segments = segments[:0]
1394 for i, token := range bytes.Split(stream, []byte{' '}) {
1396 pathparts = strings.Split(manifestUnescape(string(token)), "/")
1397 streamparts = len(pathparts)
1400 if !bytes.ContainsRune(token, ':') {
1402 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1404 if splitToToks(token, '+') < 2 {
1405 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1407 length, err := strconv.ParseInt(string(toks[1]), 10, 32)
1408 if err != nil || length < 0 {
1409 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1411 segments = append(segments, storedSegment{
1412 locator: string(token),
1415 length: int(length),
1418 } else if len(segments) == 0 {
1419 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1421 if splitToToks(token, ':') != 3 {
1422 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1424 anyFileTokens = true
1426 offset, err := strconv.ParseInt(string(toks[0]), 10, 64)
1427 if err != nil || offset < 0 {
1428 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1430 length, err := strconv.ParseInt(string(toks[1]), 10, 64)
1431 if err != nil || length < 0 {
1432 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1434 if !bytes.ContainsAny(toks[2], `\/`) {
1435 // optimization for a common case
1436 pathparts = append(pathparts[:streamparts], string(toks[2]))
1438 pathparts = append(pathparts[:streamparts], strings.Split(manifestUnescape(string(toks[2])), "/")...)
1440 fnode, err := dn.createFileAndParents(pathparts)
1441 if fnode == nil && err == nil && length == 0 {
1442 // Special case: an empty file used as
1443 // a marker to preserve an otherwise
1444 // empty directory in a manifest.
1447 if err != nil || (fnode == nil && length != 0) {
1448 return fmt.Errorf("line %d: cannot use name %q with length %d: %s", lineno, toks[2], length, err)
1450 // Map the stream offset/range coordinates to
1451 // block/offset/range coordinates and add
1452 // corresponding storedSegments to the filenode
1454 // Can't continue where we left off.
1455 // TODO: binary search instead of
1456 // rewinding all the way (but this
1457 // situation might be rare anyway)
1460 for ; segIdx < len(segments); segIdx++ {
1461 seg := segments[segIdx]
1462 next := pos + int64(seg.Len())
1463 if next <= offset || seg.Len() == 0 {
1467 if pos >= offset+length {
1472 blkOff = int(offset - pos)
1474 blkLen := seg.Len() - blkOff
1475 if pos+int64(blkOff+blkLen) > offset+length {
1476 blkLen = int(offset + length - pos - int64(blkOff))
1478 fnode.appendSegment(storedSegment{
1480 locator: seg.locator,
1485 if next > offset+length {
1491 if segIdx == len(segments) && pos < offset+length {
1492 return fmt.Errorf("line %d: invalid segment in %d-byte stream: %q", lineno, pos, token)
1496 return fmt.Errorf("line %d: no file segments", lineno)
1497 } else if len(segments) == 0 {
1498 return fmt.Errorf("line %d: no locators", lineno)
1499 } else if streamparts == 0 {
1500 return fmt.Errorf("line %d: no stream name", lineno)
1506 // only safe to call from loadManifest -- no locking.
1508 // If path is a "parent directory exists" marker (the last path
1509 // component is "."), the returned values are both nil.
1511 // Newly added nodes have modtime==0. Caller is responsible for fixing
1512 // them with backdateTree.
1513 func (dn *dirnode) createFileAndParents(names []string) (fn *filenode, err error) {
1515 basename := names[len(names)-1]
1516 for _, name := range names[:len(names)-1] {
1522 // can't be sure parent will be a *dirnode
1523 return nil, ErrInvalidArgument
1525 node = node.Parent()
1529 unlock := node.Unlock
1530 node, err = node.Child(name, func(child inode) (inode, error) {
1532 // note modtime will be fixed later in backdateTree()
1533 child, err := node.FS().newNode(name, 0755|os.ModeDir, time.Time{})
1537 child.SetParent(node, name)
1539 } else if !child.IsDir() {
1540 return child, ErrFileExists
1550 if basename == "." {
1552 } else if !permittedName(basename) {
1553 err = fmt.Errorf("invalid file part %q in path %q", basename, names)
1558 _, err = node.Child(basename, func(child inode) (inode, error) {
1559 switch child := child.(type) {
1561 child, err = node.FS().newNode(basename, 0755, time.Time{})
1565 child.SetParent(node, basename)
1566 fn = child.(*filenode)
1572 return child, ErrIsDirectory
1574 return child, ErrInvalidArgument
1580 func (dn *dirnode) TreeSize() (bytes int64) {
1583 for _, i := range dn.inodes {
1584 switch i := i.(type) {
1588 bytes += i.TreeSize()
1594 func (dn *dirnode) Snapshot() (inode, error) {
1595 return dn.snapshot()
1598 func (dn *dirnode) snapshot() (*dirnode, error) {
1603 inodes: make(map[string]inode, len(dn.inodes)),
1604 fileinfo: dn.fileinfo,
1607 for name, child := range dn.inodes {
1608 dupchild, err := child.Snapshot()
1612 snap.inodes[name] = dupchild
1613 dupchild.SetParent(snap, name)
1618 func (dn *dirnode) Splice(repl inode) error {
1619 repl, err := repl.Snapshot()
1621 return fmt.Errorf("cannot copy snapshot: %w", err)
1623 switch repl := repl.(type) {
1625 return fmt.Errorf("cannot splice snapshot containing %T: %w", repl, ErrInvalidArgument)
1629 dn.inodes = repl.inodes
1633 defer dn.parent.Unlock()
1634 removing, err := dn.parent.Child(dn.fileinfo.name, nil)
1636 return fmt.Errorf("cannot use Splice to replace a top-level directory with a file: %w", ErrInvalidOperation)
1637 } else if removing != dn {
1638 // If ../thisdirname is not this dirnode, it
1639 // must be an inode that wraps a dirnode, like
1640 // a collectionFileSystem or deferrednode.
1641 if deferred, ok := removing.(*deferrednode); ok {
1642 // More useful to report the type of
1643 // the wrapped node rather than just
1644 // *deferrednode. (We know the real
1645 // inode is already loaded because dn
1647 removing = deferred.realinode()
1649 return fmt.Errorf("cannot use Splice to attach a file at top level of %T: %w", removing, ErrInvalidOperation)
1653 _, err = dn.parent.Child(dn.fileinfo.name, func(inode) (inode, error) { return repl, nil })
1655 return fmt.Errorf("error replacing filenode: dn.parent.Child(): %w", err)
1662 func (dn *dirnode) setTreeFS(fs *collectionFileSystem) {
1664 for _, child := range dn.inodes {
1665 switch child := child.(type) {
1674 type segment interface {
1677 // Return a new segment with a subsection of the data from this
1678 // one. length<0 means length=Len()-off.
1679 Slice(off int, length int) segment
1683 type memSegment struct {
1685 // If flushing is not nil and not ready/closed, then a) buf is
1686 // being shared by a pruneMemSegments goroutine, and must be
1687 // copied on write; and b) the flushing channel will close
1688 // when the goroutine finishes, whether it succeeds or not.
1689 flushing <-chan struct{}
1692 func (me *memSegment) flushingUnfinished() bool {
1693 if me.flushing == nil {
1705 func (me *memSegment) Len() int {
1709 func (me *memSegment) Slice(off, length int) segment {
1711 length = len(me.buf) - off
1713 buf := make([]byte, length)
1714 copy(buf, me.buf[off:])
1715 return &memSegment{buf: buf}
1718 func (me *memSegment) Truncate(n int) {
1719 if n > cap(me.buf) || (me.flushing != nil && n > len(me.buf)) {
1722 newsize = newsize << 2
1724 newbuf := make([]byte, n, newsize)
1725 copy(newbuf, me.buf)
1726 me.buf, me.flushing = newbuf, nil
1728 // reclaim existing capacity, and zero reclaimed part
1729 oldlen := len(me.buf)
1731 for i := oldlen; i < n; i++ {
1737 func (me *memSegment) WriteAt(p []byte, off int) {
1738 if off+len(p) > len(me.buf) {
1739 panic("overflowed segment")
1741 if me.flushing != nil {
1742 me.buf, me.flushing = append([]byte(nil), me.buf...), nil
1744 copy(me.buf[off:], p)
1747 func (me *memSegment) ReadAt(p []byte, off int64) (n int, err error) {
1748 if off > int64(me.Len()) {
1752 n = copy(p, me.buf[int(off):])
1759 func (me *memSegment) memorySize() int64 {
1760 return 64 + int64(len(me.buf))
1763 type storedSegment struct {
1766 size int // size of stored block (also encoded in locator)
1767 offset int // position of segment within the stored block
1768 length int // bytes in this segment (offset + length <= size)
1771 func (se storedSegment) Len() int {
1775 func (se storedSegment) Slice(n, size int) segment {
1778 if size >= 0 && se.length > size {
1784 func (se storedSegment) ReadAt(p []byte, off int64) (n int, err error) {
1785 if off > int64(se.length) {
1788 maxlen := se.length - int(off)
1789 if len(p) > maxlen {
1791 n, err = se.kc.ReadAt(se.locator, p, int(off)+se.offset)
1797 return se.kc.ReadAt(se.locator, p, int(off)+se.offset)
1800 func (se storedSegment) memorySize() int64 {
1801 return 64 + int64(len(se.locator))
1804 func canonicalName(name string) string {
1805 name = path.Clean("/" + name)
1806 if name == "/" || name == "./" {
1808 } else if strings.HasPrefix(name, "/") {
1814 var manifestEscapeSeq = regexp.MustCompile(`\\([0-7]{3}|\\)`)
1816 func manifestUnescapeFunc(seq string) string {
1820 i, err := strconv.ParseUint(seq[1:], 8, 8)
1822 // Invalid escape sequence: can't unescape.
1825 return string([]byte{byte(i)})
1828 func manifestUnescape(s string) string {
1829 return manifestEscapeSeq.ReplaceAllStringFunc(s, manifestUnescapeFunc)
1832 var manifestEscapedChar = regexp.MustCompile(`[\000-\040:\s\\]`)
1834 func manifestEscapeFunc(seq string) string {
1835 return fmt.Sprintf("\\%03o", byte(seq[0]))
1838 func manifestEscape(s string) string {
1839 return manifestEscapedChar.ReplaceAllStringFunc(s, manifestEscapeFunc)