16727: Refresh block permission signatures on Sync and Read.
[arvados.git] / sdk / go / arvados / fs_collection.go
1 // Copyright (C) The Arvados Authors. All rights reserved.
2 //
3 // SPDX-License-Identifier: Apache-2.0
4
5 package arvados
6
7 import (
8         "bytes"
9         "context"
10         "encoding/json"
11         "fmt"
12         "io"
13         "os"
14         "path"
15         "regexp"
16         "sort"
17         "strconv"
18         "strings"
19         "sync"
20         "sync/atomic"
21         "time"
22 )
23
24 var (
25         maxBlockSize      = 1 << 26
26         concurrentWriters = 4 // max goroutines writing to Keep in background and during flush()
27 )
28
29 // A CollectionFileSystem is a FileSystem that can be serialized as a
30 // manifest and stored as a collection.
31 type CollectionFileSystem interface {
32         FileSystem
33
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)
39
40         // Total data bytes in all files.
41         Size() int64
42 }
43
44 type collectionFileSystem struct {
45         fileSystem
46         uuid              string
47         savedPDH          atomic.Value
48         replicas          int
49         storageClasses    []string
50         guessSignatureTTL time.Duration
51         holdCheckChanges  time.Time
52         lockCheckChanges  sync.Mutex
53 }
54
55 // FileSystem returns a CollectionFileSystem for the collection.
56 func (c *Collection) FileSystem(client apiClient, kc keepClient) (CollectionFileSystem, error) {
57         modTime := c.ModifiedAt
58         if modTime.IsZero() {
59                 modTime = time.Now()
60         }
61         fs := &collectionFileSystem{
62                 uuid:           c.UUID,
63                 storageClasses: c.StorageClassesDesired,
64                 fileSystem: fileSystem{
65                         fsBackend: keepBackend{apiClient: client, keepClient: kc},
66                         thr:       newThrottle(concurrentWriters),
67                 },
68         }
69         fs.savedPDH.Store(c.PortableDataHash)
70         if r := c.ReplicationDesired; r != nil {
71                 fs.replicas = *r
72         }
73         root := &dirnode{
74                 fs: fs,
75                 treenode: treenode{
76                         fileinfo: fileinfo{
77                                 name:    ".",
78                                 mode:    os.ModeDir | 0755,
79                                 modTime: modTime,
80                         },
81                         inodes: make(map[string]inode),
82                 },
83         }
84         root.SetParent(root, ".")
85         if err := root.loadManifest(c.ManifestText); err != nil {
86                 return nil, err
87         }
88         backdateTree(root, modTime)
89         fs.root = root
90         return fs, nil
91 }
92
93 // caller must have lock (or guarantee no concurrent accesses somehow)
94 func eachNode(n inode, ffunc func(*filenode), dfunc func(*dirnode)) {
95         switch n := n.(type) {
96         case *filenode:
97                 if ffunc != nil {
98                         ffunc(n)
99                 }
100         case *dirnode:
101                 if dfunc != nil {
102                         dfunc(n)
103                 }
104                 for _, n := range n.inodes {
105                         eachNode(n, ffunc, dfunc)
106                 }
107         }
108 }
109
110 // caller must have lock (or guarantee no concurrent accesses somehow)
111 func backdateTree(n inode, modTime time.Time) {
112         eachNode(n, func(fn *filenode) {
113                 fn.fileinfo.modTime = modTime
114         }, func(dn *dirnode) {
115                 dn.fileinfo.modTime = modTime
116         })
117 }
118
119 // Approximate portion of signature TTL remaining, usually between 0
120 // and 1, or negative if some signatures have expired.
121 func (fs *collectionFileSystem) signatureTimeLeft() (float64, time.Duration) {
122         var (
123                 now      = time.Now()
124                 earliest = now.Add(time.Hour * 24 * 7 * 365)
125                 latest   time.Time
126         )
127         fs.fileSystem.root.RLock()
128         eachNode(fs.root, func(fn *filenode) {
129                 fn.Lock()
130                 defer fn.Unlock()
131                 for _, seg := range fn.segments {
132                         seg, ok := seg.(storedSegment)
133                         if !ok {
134                                 continue
135                         }
136                         expiryTime, err := signatureExpiryTime(seg.locator)
137                         if err != nil {
138                                 continue
139                         }
140                         if expiryTime.Before(earliest) {
141                                 earliest = expiryTime
142                         }
143                         if expiryTime.After(latest) {
144                                 latest = expiryTime
145                         }
146                 }
147         }, nil)
148         fs.fileSystem.root.RUnlock()
149
150         if latest.IsZero() {
151                 // No signatures == 100% of TTL remaining.
152                 return 1, 1
153         }
154
155         ttl := latest.Sub(now)
156         fs.fileSystem.root.Lock()
157         {
158                 if ttl > fs.guessSignatureTTL {
159                         fs.guessSignatureTTL = ttl
160                 } else {
161                         ttl = fs.guessSignatureTTL
162                 }
163         }
164         fs.fileSystem.root.Unlock()
165
166         return earliest.Sub(now).Seconds() / ttl.Seconds(), ttl
167 }
168
169 func (fs *collectionFileSystem) updateSignatures(newmanifest string) {
170         newLoc := map[string]string{}
171         for _, tok := range regexp.MustCompile(`\S+`).FindAllString(newmanifest, -1) {
172                 if mBlkRe.MatchString(tok) {
173                         newLoc[stripAllHints(tok)] = tok
174                 }
175         }
176         fs.fileSystem.root.Lock()
177         defer fs.fileSystem.root.Unlock()
178         eachNode(fs.root, func(fn *filenode) {
179                 fn.Lock()
180                 defer fn.Unlock()
181                 for idx, seg := range fn.segments {
182                         seg, ok := seg.(storedSegment)
183                         if !ok {
184                                 continue
185                         }
186                         loc, ok := newLoc[stripAllHints(seg.locator)]
187                         if !ok {
188                                 continue
189                         }
190                         seg.locator = loc
191                         fn.segments[idx] = seg
192                 }
193         }, nil)
194 }
195
196 func (fs *collectionFileSystem) newNode(name string, perm os.FileMode, modTime time.Time) (node inode, err error) {
197         if name == "" || name == "." || name == ".." {
198                 return nil, ErrInvalidArgument
199         }
200         if perm.IsDir() {
201                 return &dirnode{
202                         fs: fs,
203                         treenode: treenode{
204                                 fileinfo: fileinfo{
205                                         name:    name,
206                                         mode:    perm | os.ModeDir,
207                                         modTime: modTime,
208                                 },
209                                 inodes: make(map[string]inode),
210                         },
211                 }, nil
212         }
213         return &filenode{
214                 fs: fs,
215                 fileinfo: fileinfo{
216                         name:    name,
217                         mode:    perm & ^os.ModeDir,
218                         modTime: modTime,
219                 },
220         }, nil
221 }
222
223 func (fs *collectionFileSystem) Child(name string, replace func(inode) (inode, error)) (inode, error) {
224         return fs.rootnode().Child(name, replace)
225 }
226
227 func (fs *collectionFileSystem) FS() FileSystem {
228         return fs
229 }
230
231 func (fs *collectionFileSystem) FileInfo() os.FileInfo {
232         return fs.rootnode().FileInfo()
233 }
234
235 func (fs *collectionFileSystem) IsDir() bool {
236         return true
237 }
238
239 func (fs *collectionFileSystem) Lock() {
240         fs.rootnode().Lock()
241 }
242
243 func (fs *collectionFileSystem) Unlock() {
244         fs.rootnode().Unlock()
245 }
246
247 func (fs *collectionFileSystem) RLock() {
248         fs.rootnode().RLock()
249 }
250
251 func (fs *collectionFileSystem) RUnlock() {
252         fs.rootnode().RUnlock()
253 }
254
255 func (fs *collectionFileSystem) Parent() inode {
256         return fs.rootnode().Parent()
257 }
258
259 func (fs *collectionFileSystem) Read(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
260         return 0, ptr, ErrInvalidOperation
261 }
262
263 func (fs *collectionFileSystem) Write(_ []byte, ptr filenodePtr) (int, filenodePtr, error) {
264         return 0, ptr, ErrInvalidOperation
265 }
266
267 func (fs *collectionFileSystem) Readdir() ([]os.FileInfo, error) {
268         return fs.rootnode().Readdir()
269 }
270
271 func (fs *collectionFileSystem) SetParent(parent inode, name string) {
272         fs.rootnode().SetParent(parent, name)
273 }
274
275 func (fs *collectionFileSystem) Truncate(int64) error {
276         return ErrInvalidOperation
277 }
278
279 // Check for and incorporate upstream changes -- unless that has
280 // already been done recently, in which case this func is a no-op.
281 func (fs *collectionFileSystem) checkChangesOnServer() error {
282         if fs.uuid == "" && fs.savedPDH.Load() == "" {
283                 return nil
284         }
285
286         // First try UUID if any, then last known PDH. Stop if all
287         // signatures are new enough.
288         checkingAll := false
289         for _, id := range []string{fs.uuid, fs.savedPDH.Load().(string)} {
290                 if id == "" {
291                         continue
292                 }
293
294                 fs.lockCheckChanges.Lock()
295                 if !checkingAll && fs.holdCheckChanges.After(time.Now()) {
296                         fs.lockCheckChanges.Unlock()
297                         return nil
298                 }
299                 remain, ttl := fs.signatureTimeLeft()
300                 if remain > 0.01 && !checkingAll {
301                         fs.holdCheckChanges = time.Now().Add(ttl / 100)
302                 }
303                 fs.lockCheckChanges.Unlock()
304
305                 if remain >= 0.5 {
306                         break
307                 }
308                 checkingAll = true
309                 var coll Collection
310                 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+id, nil, map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}})
311                 if err != nil {
312                         continue
313                 }
314                 fs.updateSignatures(coll.ManifestText)
315         }
316         return nil
317 }
318
319 // Refresh signature on a single locator, if necessary. Assume caller
320 // has lock. If an update is needed, and there are any storedSegments
321 // whose signatures can be updated, start a background task to update
322 // them asynchronously when the caller releases locks.
323 func (fs *collectionFileSystem) refreshSignature(locator string) string {
324         exp, err := signatureExpiryTime(locator)
325         if err != nil || exp.Sub(time.Now()) > time.Minute {
326                 // Synchronous update is not needed. Start an
327                 // asynchronous update if needed.
328                 go fs.checkChangesOnServer()
329                 return locator
330         }
331         var manifests string
332         for _, id := range []string{fs.uuid, fs.savedPDH.Load().(string)} {
333                 if id == "" {
334                         continue
335                 }
336                 var coll Collection
337                 err := fs.RequestAndDecode(&coll, "GET", "arvados/v1/collections/"+id, nil, map[string]interface{}{"select": []string{"portable_data_hash", "manifest_text"}})
338                 if err != nil {
339                         continue
340                 }
341                 manifests += coll.ManifestText
342         }
343         hash := stripAllHints(locator)
344         for _, tok := range regexp.MustCompile(`\S+`).FindAllString(manifests, -1) {
345                 if mBlkRe.MatchString(tok) {
346                         if stripAllHints(tok) == hash {
347                                 locator = tok
348                                 break
349                         }
350                 }
351         }
352         go fs.updateSignatures(manifests)
353         return locator
354 }
355
356 func (fs *collectionFileSystem) Sync() error {
357         err := fs.checkChangesOnServer()
358         if err != nil {
359                 return err
360         }
361         if fs.uuid == "" {
362                 return nil
363         }
364         txt, err := fs.MarshalManifest(".")
365         if err != nil {
366                 return fmt.Errorf("sync failed: %s", err)
367         }
368         if PortableDataHash(txt) == fs.savedPDH.Load() {
369                 // No local changes since last save or initial load.
370                 return nil
371         }
372         coll := Collection{
373                 UUID:         fs.uuid,
374                 ManifestText: txt,
375         }
376
377         selectFields := []string{"uuid", "portable_data_hash"}
378         fs.lockCheckChanges.Lock()
379         remain, _ := fs.signatureTimeLeft()
380         fs.lockCheckChanges.Unlock()
381         if remain < 0.5 {
382                 selectFields = append(selectFields, "manifest_text")
383         }
384
385         err = fs.RequestAndDecode(&coll, "PUT", "arvados/v1/collections/"+fs.uuid, nil, map[string]interface{}{
386                 "collection": map[string]string{
387                         "manifest_text": coll.ManifestText,
388                 },
389                 "select": selectFields,
390         })
391         if err != nil {
392                 return fmt.Errorf("sync failed: update %s: %s", fs.uuid, err)
393         }
394         fs.updateSignatures(coll.ManifestText)
395         fs.savedPDH.Store(coll.PortableDataHash)
396         return nil
397 }
398
399 func (fs *collectionFileSystem) Flush(path string, shortBlocks bool) error {
400         node, err := rlookup(fs.fileSystem.root, path)
401         if err != nil {
402                 return err
403         }
404         dn, ok := node.(*dirnode)
405         if !ok {
406                 return ErrNotADirectory
407         }
408         dn.Lock()
409         defer dn.Unlock()
410         names := dn.sortedNames()
411         if path != "" {
412                 // Caller only wants to flush the specified dir,
413                 // non-recursively.  Drop subdirs from the list of
414                 // names.
415                 var filenames []string
416                 for _, name := range names {
417                         if _, ok := dn.inodes[name].(*filenode); ok {
418                                 filenames = append(filenames, name)
419                         }
420                 }
421                 names = filenames
422         }
423         for _, name := range names {
424                 child := dn.inodes[name]
425                 child.Lock()
426                 defer child.Unlock()
427         }
428         return dn.flush(context.TODO(), names, flushOpts{sync: false, shortBlocks: shortBlocks})
429 }
430
431 func (fs *collectionFileSystem) MemorySize() int64 {
432         fs.fileSystem.root.Lock()
433         defer fs.fileSystem.root.Unlock()
434         return fs.fileSystem.root.(*dirnode).MemorySize()
435 }
436
437 func (fs *collectionFileSystem) MarshalManifest(prefix string) (string, error) {
438         fs.fileSystem.root.Lock()
439         defer fs.fileSystem.root.Unlock()
440         return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix)
441 }
442
443 func (fs *collectionFileSystem) Size() int64 {
444         return fs.fileSystem.root.(*dirnode).TreeSize()
445 }
446
447 // filenodePtr is an offset into a file that is (usually) efficient to
448 // seek to. Specifically, if filenode.repacked==filenodePtr.repacked
449 // then
450 // filenode.segments[filenodePtr.segmentIdx][filenodePtr.segmentOff]
451 // corresponds to file offset filenodePtr.off. Otherwise, it is
452 // necessary to reexamine len(filenode.segments[0]) etc. to find the
453 // correct segment and offset.
454 type filenodePtr struct {
455         off        int64
456         segmentIdx int
457         segmentOff int
458         repacked   int64
459 }
460
461 // seek returns a ptr that is consistent with both startPtr.off and
462 // the current state of fn. The caller must already hold fn.RLock() or
463 // fn.Lock().
464 //
465 // If startPtr is beyond EOF, ptr.segment* will indicate precisely
466 // EOF.
467 //
468 // After seeking:
469 //
470 //     ptr.segmentIdx == len(filenode.segments) // i.e., at EOF
471 //     ||
472 //     filenode.segments[ptr.segmentIdx].Len() > ptr.segmentOff
473 func (fn *filenode) seek(startPtr filenodePtr) (ptr filenodePtr) {
474         ptr = startPtr
475         if ptr.off < 0 {
476                 // meaningless anyway
477                 return
478         } else if ptr.off >= fn.fileinfo.size {
479                 ptr.segmentIdx = len(fn.segments)
480                 ptr.segmentOff = 0
481                 ptr.repacked = fn.repacked
482                 return
483         } else if ptr.repacked == fn.repacked {
484                 // segmentIdx and segmentOff accurately reflect
485                 // ptr.off, but might have fallen off the end of a
486                 // segment
487                 if ptr.segmentOff >= fn.segments[ptr.segmentIdx].Len() {
488                         ptr.segmentIdx++
489                         ptr.segmentOff = 0
490                 }
491                 return
492         }
493         defer func() {
494                 ptr.repacked = fn.repacked
495         }()
496         if ptr.off >= fn.fileinfo.size {
497                 ptr.segmentIdx, ptr.segmentOff = len(fn.segments), 0
498                 return
499         }
500         // Recompute segmentIdx and segmentOff.  We have already
501         // established fn.fileinfo.size > ptr.off >= 0, so we don't
502         // have to deal with edge cases here.
503         var off int64
504         for ptr.segmentIdx, ptr.segmentOff = 0, 0; off < ptr.off; ptr.segmentIdx++ {
505                 // This would panic (index out of range) if
506                 // fn.fileinfo.size were larger than
507                 // sum(fn.segments[i].Len()) -- but that can't happen
508                 // because we have ensured fn.fileinfo.size is always
509                 // accurate.
510                 segLen := int64(fn.segments[ptr.segmentIdx].Len())
511                 if off+segLen > ptr.off {
512                         ptr.segmentOff = int(ptr.off - off)
513                         break
514                 }
515                 off += segLen
516         }
517         return
518 }
519
520 // filenode implements inode.
521 type filenode struct {
522         parent   inode
523         fs       *collectionFileSystem
524         fileinfo fileinfo
525         segments []segment
526         // number of times `segments` has changed in a
527         // way that might invalidate a filenodePtr
528         repacked int64
529         memsize  int64 // bytes in memSegments
530         sync.RWMutex
531         nullnode
532 }
533
534 // caller must have lock
535 func (fn *filenode) appendSegment(e segment) {
536         fn.segments = append(fn.segments, e)
537         fn.fileinfo.size += int64(e.Len())
538 }
539
540 func (fn *filenode) SetParent(p inode, name string) {
541         fn.Lock()
542         defer fn.Unlock()
543         fn.parent = p
544         fn.fileinfo.name = name
545 }
546
547 func (fn *filenode) Parent() inode {
548         fn.RLock()
549         defer fn.RUnlock()
550         return fn.parent
551 }
552
553 func (fn *filenode) FS() FileSystem {
554         return fn.fs
555 }
556
557 // Read reads file data from a single segment, starting at startPtr,
558 // into p. startPtr is assumed not to be up-to-date. Caller must have
559 // RLock or Lock.
560 func (fn *filenode) Read(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr, err error) {
561         ptr = fn.seek(startPtr)
562         if ptr.off < 0 {
563                 err = ErrNegativeOffset
564                 return
565         }
566         if ptr.segmentIdx >= len(fn.segments) {
567                 err = io.EOF
568                 return
569         }
570         if ss, ok := fn.segments[ptr.segmentIdx].(storedSegment); ok {
571                 ss.locator = fn.fs.refreshSignature(ss.locator)
572                 fn.segments[ptr.segmentIdx] = ss
573         }
574         n, err = fn.segments[ptr.segmentIdx].ReadAt(p, int64(ptr.segmentOff))
575         if n > 0 {
576                 ptr.off += int64(n)
577                 ptr.segmentOff += n
578                 if ptr.segmentOff == fn.segments[ptr.segmentIdx].Len() {
579                         ptr.segmentIdx++
580                         ptr.segmentOff = 0
581                         if ptr.segmentIdx < len(fn.segments) && err == io.EOF {
582                                 err = nil
583                         }
584                 }
585         }
586         return
587 }
588
589 func (fn *filenode) Size() int64 {
590         fn.RLock()
591         defer fn.RUnlock()
592         return fn.fileinfo.Size()
593 }
594
595 func (fn *filenode) FileInfo() os.FileInfo {
596         fn.RLock()
597         defer fn.RUnlock()
598         return fn.fileinfo
599 }
600
601 func (fn *filenode) Truncate(size int64) error {
602         fn.Lock()
603         defer fn.Unlock()
604         return fn.truncate(size)
605 }
606
607 func (fn *filenode) truncate(size int64) error {
608         if size == fn.fileinfo.size {
609                 return nil
610         }
611         fn.repacked++
612         if size < fn.fileinfo.size {
613                 ptr := fn.seek(filenodePtr{off: size})
614                 for i := ptr.segmentIdx; i < len(fn.segments); i++ {
615                         if seg, ok := fn.segments[i].(*memSegment); ok {
616                                 fn.memsize -= int64(seg.Len())
617                         }
618                 }
619                 if ptr.segmentOff == 0 {
620                         fn.segments = fn.segments[:ptr.segmentIdx]
621                 } else {
622                         fn.segments = fn.segments[:ptr.segmentIdx+1]
623                         switch seg := fn.segments[ptr.segmentIdx].(type) {
624                         case *memSegment:
625                                 seg.Truncate(ptr.segmentOff)
626                                 fn.memsize += int64(seg.Len())
627                         default:
628                                 fn.segments[ptr.segmentIdx] = seg.Slice(0, ptr.segmentOff)
629                         }
630                 }
631                 fn.fileinfo.size = size
632                 return nil
633         }
634         for size > fn.fileinfo.size {
635                 grow := size - fn.fileinfo.size
636                 var seg *memSegment
637                 var ok bool
638                 if len(fn.segments) == 0 {
639                         seg = &memSegment{}
640                         fn.segments = append(fn.segments, seg)
641                 } else if seg, ok = fn.segments[len(fn.segments)-1].(*memSegment); !ok || seg.Len() >= maxBlockSize {
642                         seg = &memSegment{}
643                         fn.segments = append(fn.segments, seg)
644                 }
645                 if maxgrow := int64(maxBlockSize - seg.Len()); maxgrow < grow {
646                         grow = maxgrow
647                 }
648                 seg.Truncate(seg.Len() + int(grow))
649                 fn.fileinfo.size += grow
650                 fn.memsize += grow
651         }
652         return nil
653 }
654
655 // Write writes data from p to the file, starting at startPtr,
656 // extending the file size if necessary. Caller must have Lock.
657 func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePtr, err error) {
658         if startPtr.off > fn.fileinfo.size {
659                 if err = fn.truncate(startPtr.off); err != nil {
660                         return 0, startPtr, err
661                 }
662         }
663         ptr = fn.seek(startPtr)
664         if ptr.off < 0 {
665                 err = ErrNegativeOffset
666                 return
667         }
668         for len(p) > 0 && err == nil {
669                 cando := p
670                 if len(cando) > maxBlockSize {
671                         cando = cando[:maxBlockSize]
672                 }
673                 // Rearrange/grow fn.segments (and shrink cando if
674                 // needed) such that cando can be copied to
675                 // fn.segments[ptr.segmentIdx] at offset
676                 // ptr.segmentOff.
677                 cur := ptr.segmentIdx
678                 prev := ptr.segmentIdx - 1
679                 var curWritable bool
680                 if cur < len(fn.segments) {
681                         _, curWritable = fn.segments[cur].(*memSegment)
682                 }
683                 var prevAppendable bool
684                 if prev >= 0 && fn.segments[prev].Len() < maxBlockSize {
685                         _, prevAppendable = fn.segments[prev].(*memSegment)
686                 }
687                 if ptr.segmentOff > 0 && !curWritable {
688                         // Split a non-writable block.
689                         if max := fn.segments[cur].Len() - ptr.segmentOff; max <= len(cando) {
690                                 // Truncate cur, and insert a new
691                                 // segment after it.
692                                 cando = cando[:max]
693                                 fn.segments = append(fn.segments, nil)
694                                 copy(fn.segments[cur+1:], fn.segments[cur:])
695                         } else {
696                                 // Split cur into two copies, truncate
697                                 // the one on the left, shift the one
698                                 // on the right, and insert a new
699                                 // segment between them.
700                                 fn.segments = append(fn.segments, nil, nil)
701                                 copy(fn.segments[cur+2:], fn.segments[cur:])
702                                 fn.segments[cur+2] = fn.segments[cur+2].Slice(ptr.segmentOff+len(cando), -1)
703                         }
704                         cur++
705                         prev++
706                         seg := &memSegment{}
707                         seg.Truncate(len(cando))
708                         fn.memsize += int64(len(cando))
709                         fn.segments[cur] = seg
710                         fn.segments[prev] = fn.segments[prev].Slice(0, ptr.segmentOff)
711                         ptr.segmentIdx++
712                         ptr.segmentOff = 0
713                         fn.repacked++
714                         ptr.repacked++
715                 } else if curWritable {
716                         if fit := int(fn.segments[cur].Len()) - ptr.segmentOff; fit < len(cando) {
717                                 cando = cando[:fit]
718                         }
719                 } else {
720                         if prevAppendable {
721                                 // Shrink cando if needed to fit in
722                                 // prev segment.
723                                 if cangrow := maxBlockSize - fn.segments[prev].Len(); cangrow < len(cando) {
724                                         cando = cando[:cangrow]
725                                 }
726                         }
727
728                         if cur == len(fn.segments) {
729                                 // ptr is at EOF, filesize is changing.
730                                 fn.fileinfo.size += int64(len(cando))
731                         } else if el := fn.segments[cur].Len(); el <= len(cando) {
732                                 // cando is long enough that we won't
733                                 // need cur any more. shrink cando to
734                                 // be exactly as long as cur
735                                 // (otherwise we'd accidentally shift
736                                 // the effective position of all
737                                 // segments after cur).
738                                 cando = cando[:el]
739                                 copy(fn.segments[cur:], fn.segments[cur+1:])
740                                 fn.segments = fn.segments[:len(fn.segments)-1]
741                         } else {
742                                 // shrink cur by the same #bytes we're growing prev
743                                 fn.segments[cur] = fn.segments[cur].Slice(len(cando), -1)
744                         }
745
746                         if prevAppendable {
747                                 // Grow prev.
748                                 ptr.segmentIdx--
749                                 ptr.segmentOff = fn.segments[prev].Len()
750                                 fn.segments[prev].(*memSegment).Truncate(ptr.segmentOff + len(cando))
751                                 fn.memsize += int64(len(cando))
752                                 ptr.repacked++
753                                 fn.repacked++
754                         } else {
755                                 // Insert a segment between prev and
756                                 // cur, and advance prev/cur.
757                                 fn.segments = append(fn.segments, nil)
758                                 if cur < len(fn.segments) {
759                                         copy(fn.segments[cur+1:], fn.segments[cur:])
760                                         ptr.repacked++
761                                         fn.repacked++
762                                 } else {
763                                         // appending a new segment does
764                                         // not invalidate any ptrs
765                                 }
766                                 seg := &memSegment{}
767                                 seg.Truncate(len(cando))
768                                 fn.memsize += int64(len(cando))
769                                 fn.segments[cur] = seg
770                         }
771                 }
772
773                 // Finally we can copy bytes from cando to the current segment.
774                 fn.segments[ptr.segmentIdx].(*memSegment).WriteAt(cando, ptr.segmentOff)
775                 n += len(cando)
776                 p = p[len(cando):]
777
778                 ptr.off += int64(len(cando))
779                 ptr.segmentOff += len(cando)
780                 if ptr.segmentOff >= maxBlockSize {
781                         fn.pruneMemSegments()
782                 }
783                 if fn.segments[ptr.segmentIdx].Len() == ptr.segmentOff {
784                         ptr.segmentOff = 0
785                         ptr.segmentIdx++
786                 }
787
788                 fn.fileinfo.modTime = time.Now()
789         }
790         return
791 }
792
793 // Write some data out to disk to reduce memory use. Caller must have
794 // write lock.
795 func (fn *filenode) pruneMemSegments() {
796         // TODO: share code with (*dirnode)flush()
797         // TODO: pack/flush small blocks too, when fragmented
798         for idx, seg := range fn.segments {
799                 seg, ok := seg.(*memSegment)
800                 if !ok || seg.Len() < maxBlockSize || seg.flushing != nil {
801                         continue
802                 }
803                 // Setting seg.flushing guarantees seg.buf will not be
804                 // modified in place: WriteAt and Truncate will
805                 // allocate a new buf instead, if necessary.
806                 idx, buf := idx, seg.buf
807                 done := make(chan struct{})
808                 seg.flushing = done
809                 // If lots of background writes are already in
810                 // progress, block here until one finishes, rather
811                 // than pile up an unlimited number of buffered writes
812                 // and network flush operations.
813                 fn.fs.throttle().Acquire()
814                 go func() {
815                         defer close(done)
816                         resp, err := fn.FS().BlockWrite(context.Background(), BlockWriteOptions{
817                                 Data:           buf,
818                                 Replicas:       fn.fs.replicas,
819                                 StorageClasses: fn.fs.storageClasses,
820                         })
821                         fn.fs.throttle().Release()
822                         fn.Lock()
823                         defer fn.Unlock()
824                         if seg.flushing != done {
825                                 // A new seg.buf has been allocated.
826                                 return
827                         }
828                         if err != nil {
829                                 // TODO: stall (or return errors from)
830                                 // subsequent writes until flushing
831                                 // starts to succeed.
832                                 return
833                         }
834                         if len(fn.segments) <= idx || fn.segments[idx] != seg || len(seg.buf) != len(buf) {
835                                 // Segment has been dropped/moved/resized.
836                                 return
837                         }
838                         fn.memsize -= int64(len(buf))
839                         fn.segments[idx] = storedSegment{
840                                 kc:      fn.FS(),
841                                 locator: resp.Locator,
842                                 size:    len(buf),
843                                 offset:  0,
844                                 length:  len(buf),
845                         }
846                 }()
847         }
848 }
849
850 // Block until all pending pruneMemSegments/flush work is
851 // finished. Caller must NOT have lock.
852 func (fn *filenode) waitPrune() {
853         var pending []<-chan struct{}
854         fn.Lock()
855         for _, seg := range fn.segments {
856                 if seg, ok := seg.(*memSegment); ok && seg.flushing != nil {
857                         pending = append(pending, seg.flushing)
858                 }
859         }
860         fn.Unlock()
861         for _, p := range pending {
862                 <-p
863         }
864 }
865
866 type dirnode struct {
867         fs *collectionFileSystem
868         treenode
869 }
870
871 func (dn *dirnode) FS() FileSystem {
872         return dn.fs
873 }
874
875 func (dn *dirnode) Child(name string, replace func(inode) (inode, error)) (inode, error) {
876         if dn == dn.fs.rootnode() && name == ".arvados#collection" {
877                 gn := &getternode{Getter: func() ([]byte, error) {
878                         var coll Collection
879                         var err error
880                         coll.ManifestText, err = dn.fs.MarshalManifest(".")
881                         if err != nil {
882                                 return nil, err
883                         }
884                         coll.UUID = dn.fs.uuid
885                         data, err := json.Marshal(&coll)
886                         if err == nil {
887                                 data = append(data, '\n')
888                         }
889                         return data, err
890                 }}
891                 gn.SetParent(dn, name)
892                 return gn, nil
893         }
894         return dn.treenode.Child(name, replace)
895 }
896
897 type fnSegmentRef struct {
898         fn  *filenode
899         idx int
900 }
901
902 // commitBlock concatenates the data from the given filenode segments
903 // (which must be *memSegments), writes the data out to Keep as a
904 // single block, and replaces the filenodes' *memSegments with
905 // storedSegments that reference the relevant portions of the new
906 // block.
907 //
908 // bufsize is the total data size in refs. It is used to preallocate
909 // the correct amount of memory when len(refs)>1.
910 //
911 // If sync is false, commitBlock returns right away, after starting a
912 // goroutine to do the writes, reacquire the filenodes' locks, and
913 // swap out the *memSegments. Some filenodes' segments might get
914 // modified/rearranged in the meantime, in which case commitBlock
915 // won't replace them.
916 //
917 // Caller must have write lock.
918 func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, bufsize int, sync bool) error {
919         if len(refs) == 0 {
920                 return nil
921         }
922         if err := ctx.Err(); err != nil {
923                 return err
924         }
925         done := make(chan struct{})
926         var block []byte
927         segs := make([]*memSegment, 0, len(refs))
928         offsets := make([]int, 0, len(refs)) // location of segment's data within block
929         for _, ref := range refs {
930                 seg := ref.fn.segments[ref.idx].(*memSegment)
931                 if !sync && seg.flushingUnfinished() {
932                         // Let the other flushing goroutine finish. If
933                         // it fails, we'll try again next time.
934                         close(done)
935                         return nil
936                 }
937                 // In sync mode, we proceed regardless of
938                 // whether another flush is in progress: It
939                 // can't finish before we do, because we hold
940                 // fn's lock until we finish our own writes.
941                 seg.flushing = done
942                 offsets = append(offsets, len(block))
943                 if len(refs) == 1 {
944                         block = seg.buf
945                 } else if block == nil {
946                         block = append(make([]byte, 0, bufsize), seg.buf...)
947                 } else {
948                         block = append(block, seg.buf...)
949                 }
950                 segs = append(segs, seg)
951         }
952         blocksize := len(block)
953         dn.fs.throttle().Acquire()
954         errs := make(chan error, 1)
955         go func() {
956                 defer close(done)
957                 defer close(errs)
958                 resp, err := dn.fs.BlockWrite(context.Background(), BlockWriteOptions{
959                         Data:           block,
960                         Replicas:       dn.fs.replicas,
961                         StorageClasses: dn.fs.storageClasses,
962                 })
963                 dn.fs.throttle().Release()
964                 if err != nil {
965                         errs <- err
966                         return
967                 }
968                 for idx, ref := range refs {
969                         if !sync {
970                                 ref.fn.Lock()
971                                 // In async mode, fn's lock was
972                                 // released while we were waiting for
973                                 // PutB(); lots of things might have
974                                 // changed.
975                                 if len(ref.fn.segments) <= ref.idx {
976                                         // file segments have
977                                         // rearranged or changed in
978                                         // some way
979                                         ref.fn.Unlock()
980                                         continue
981                                 } else if seg, ok := ref.fn.segments[ref.idx].(*memSegment); !ok || seg != segs[idx] {
982                                         // segment has been replaced
983                                         ref.fn.Unlock()
984                                         continue
985                                 } else if seg.flushing != done {
986                                         // seg.buf has been replaced
987                                         ref.fn.Unlock()
988                                         continue
989                                 }
990                         }
991                         data := ref.fn.segments[ref.idx].(*memSegment).buf
992                         ref.fn.segments[ref.idx] = storedSegment{
993                                 kc:      dn.fs,
994                                 locator: resp.Locator,
995                                 size:    blocksize,
996                                 offset:  offsets[idx],
997                                 length:  len(data),
998                         }
999                         // atomic is needed here despite caller having
1000                         // lock: caller might be running concurrent
1001                         // commitBlock() goroutines using the same
1002                         // lock, writing different segments from the
1003                         // same file.
1004                         atomic.AddInt64(&ref.fn.memsize, -int64(len(data)))
1005                         if !sync {
1006                                 ref.fn.Unlock()
1007                         }
1008                 }
1009         }()
1010         if sync {
1011                 return <-errs
1012         }
1013         return nil
1014 }
1015
1016 type flushOpts struct {
1017         sync        bool
1018         shortBlocks bool
1019 }
1020
1021 // flush in-memory data and remote-cluster block references (for the
1022 // children with the given names, which must be children of dn) to
1023 // local-cluster persistent storage.
1024 //
1025 // Caller must have write lock on dn and the named children.
1026 //
1027 // If any children are dirs, they will be flushed recursively.
1028 func (dn *dirnode) flush(ctx context.Context, names []string, opts flushOpts) error {
1029         cg := newContextGroup(ctx)
1030         defer cg.Cancel()
1031
1032         goCommit := func(refs []fnSegmentRef, bufsize int) {
1033                 cg.Go(func() error {
1034                         return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync)
1035                 })
1036         }
1037
1038         var pending []fnSegmentRef
1039         var pendingLen int = 0
1040         localLocator := map[string]string{}
1041         for _, name := range names {
1042                 switch node := dn.inodes[name].(type) {
1043                 case *dirnode:
1044                         grandchildNames := node.sortedNames()
1045                         for _, grandchildName := range grandchildNames {
1046                                 grandchild := node.inodes[grandchildName]
1047                                 grandchild.Lock()
1048                                 defer grandchild.Unlock()
1049                         }
1050                         cg.Go(func() error { return node.flush(cg.Context(), grandchildNames, opts) })
1051                 case *filenode:
1052                         for idx, seg := range node.segments {
1053                                 switch seg := seg.(type) {
1054                                 case storedSegment:
1055                                         loc, ok := localLocator[seg.locator]
1056                                         if !ok {
1057                                                 var err error
1058                                                 loc, err = dn.fs.LocalLocator(seg.locator)
1059                                                 if err != nil {
1060                                                         return err
1061                                                 }
1062                                                 localLocator[seg.locator] = loc
1063                                         }
1064                                         seg.locator = loc
1065                                         node.segments[idx] = seg
1066                                 case *memSegment:
1067                                         if seg.Len() > maxBlockSize/2 {
1068                                                 goCommit([]fnSegmentRef{{node, idx}}, seg.Len())
1069                                                 continue
1070                                         }
1071                                         if pendingLen+seg.Len() > maxBlockSize {
1072                                                 goCommit(pending, pendingLen)
1073                                                 pending = nil
1074                                                 pendingLen = 0
1075                                         }
1076                                         pending = append(pending, fnSegmentRef{node, idx})
1077                                         pendingLen += seg.Len()
1078                                 default:
1079                                         panic(fmt.Sprintf("can't sync segment type %T", seg))
1080                                 }
1081                         }
1082                 }
1083         }
1084         if opts.shortBlocks {
1085                 goCommit(pending, pendingLen)
1086         }
1087         return cg.Wait()
1088 }
1089
1090 // caller must have write lock.
1091 func (dn *dirnode) MemorySize() (size int64) {
1092         for _, name := range dn.sortedNames() {
1093                 node := dn.inodes[name]
1094                 node.Lock()
1095                 defer node.Unlock()
1096                 switch node := node.(type) {
1097                 case *dirnode:
1098                         size += node.MemorySize()
1099                 case *filenode:
1100                         for _, seg := range node.segments {
1101                                 switch seg := seg.(type) {
1102                                 case *memSegment:
1103                                         size += int64(seg.Len())
1104                                 }
1105                         }
1106                 }
1107         }
1108         return
1109 }
1110
1111 // caller must have write lock.
1112 func (dn *dirnode) sortedNames() []string {
1113         names := make([]string, 0, len(dn.inodes))
1114         for name := range dn.inodes {
1115                 names = append(names, name)
1116         }
1117         sort.Strings(names)
1118         return names
1119 }
1120
1121 // caller must have write lock.
1122 func (dn *dirnode) marshalManifest(ctx context.Context, prefix string) (string, error) {
1123         cg := newContextGroup(ctx)
1124         defer cg.Cancel()
1125
1126         if len(dn.inodes) == 0 {
1127                 if prefix == "." {
1128                         return "", nil
1129                 }
1130                 // Express the existence of an empty directory by
1131                 // adding an empty file named `\056`, which (unlike
1132                 // the more obvious spelling `.`) is accepted by the
1133                 // API's manifest validator.
1134                 return manifestEscape(prefix) + " d41d8cd98f00b204e9800998ecf8427e+0 0:0:\\056\n", nil
1135         }
1136
1137         names := dn.sortedNames()
1138
1139         // Wait for children to finish any pending write operations
1140         // before locking them.
1141         for _, name := range names {
1142                 node := dn.inodes[name]
1143                 if fn, ok := node.(*filenode); ok {
1144                         fn.waitPrune()
1145                 }
1146         }
1147
1148         var dirnames []string
1149         var filenames []string
1150         for _, name := range names {
1151                 node := dn.inodes[name]
1152                 node.Lock()
1153                 defer node.Unlock()
1154                 switch node := node.(type) {
1155                 case *dirnode:
1156                         dirnames = append(dirnames, name)
1157                 case *filenode:
1158                         filenames = append(filenames, name)
1159                 default:
1160                         panic(fmt.Sprintf("can't marshal inode type %T", node))
1161                 }
1162         }
1163
1164         subdirs := make([]string, len(dirnames))
1165         rootdir := ""
1166         for i, name := range dirnames {
1167                 i, name := i, name
1168                 cg.Go(func() error {
1169                         txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name)
1170                         subdirs[i] = txt
1171                         return err
1172                 })
1173         }
1174
1175         cg.Go(func() error {
1176                 var streamLen int64
1177                 type filepart struct {
1178                         name   string
1179                         offset int64
1180                         length int64
1181                 }
1182
1183                 var fileparts []filepart
1184                 var blocks []string
1185                 if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil {
1186                         return err
1187                 }
1188                 for _, name := range filenames {
1189                         node := dn.inodes[name].(*filenode)
1190                         if len(node.segments) == 0 {
1191                                 fileparts = append(fileparts, filepart{name: name})
1192                                 continue
1193                         }
1194                         for _, seg := range node.segments {
1195                                 switch seg := seg.(type) {
1196                                 case storedSegment:
1197                                         if len(blocks) > 0 && blocks[len(blocks)-1] == seg.locator {
1198                                                 streamLen -= int64(seg.size)
1199                                         } else {
1200                                                 blocks = append(blocks, seg.locator)
1201                                         }
1202                                         next := filepart{
1203                                                 name:   name,
1204                                                 offset: streamLen + int64(seg.offset),
1205                                                 length: int64(seg.length),
1206                                         }
1207                                         if prev := len(fileparts) - 1; prev >= 0 &&
1208                                                 fileparts[prev].name == name &&
1209                                                 fileparts[prev].offset+fileparts[prev].length == next.offset {
1210                                                 fileparts[prev].length += next.length
1211                                         } else {
1212                                                 fileparts = append(fileparts, next)
1213                                         }
1214                                         streamLen += int64(seg.size)
1215                                 default:
1216                                         // This can't happen: we
1217                                         // haven't unlocked since
1218                                         // calling flush(sync=true).
1219                                         panic(fmt.Sprintf("can't marshal segment type %T", seg))
1220                                 }
1221                         }
1222                 }
1223                 var filetokens []string
1224                 for _, s := range fileparts {
1225                         filetokens = append(filetokens, fmt.Sprintf("%d:%d:%s", s.offset, s.length, manifestEscape(s.name)))
1226                 }
1227                 if len(filetokens) == 0 {
1228                         return nil
1229                 } else if len(blocks) == 0 {
1230                         blocks = []string{"d41d8cd98f00b204e9800998ecf8427e+0"}
1231                 }
1232                 rootdir = manifestEscape(prefix) + " " + strings.Join(blocks, " ") + " " + strings.Join(filetokens, " ") + "\n"
1233                 return nil
1234         })
1235         err := cg.Wait()
1236         return rootdir + strings.Join(subdirs, ""), err
1237 }
1238
1239 func (dn *dirnode) loadManifest(txt string) error {
1240         streams := bytes.Split([]byte(txt), []byte{'\n'})
1241         if len(streams[len(streams)-1]) != 0 {
1242                 return fmt.Errorf("line %d: no trailing newline", len(streams))
1243         }
1244         streams = streams[:len(streams)-1]
1245         segments := []storedSegment{}
1246         // To reduce allocs, we reuse a single "pathparts" slice
1247         // (pre-split on "/" separators) for the duration of this
1248         // func.
1249         var pathparts []string
1250         // To reduce allocs, we reuse a single "toks" slice of 3 byte
1251         // slices.
1252         var toks = make([][]byte, 3)
1253         // Similar to bytes.SplitN(token, []byte{c}, 3), but splits
1254         // into the toks slice rather than allocating a new one, and
1255         // returns the number of toks (1, 2, or 3).
1256         splitToToks := func(src []byte, c rune) int {
1257                 c1 := bytes.IndexRune(src, c)
1258                 if c1 < 0 {
1259                         toks[0] = src
1260                         return 1
1261                 }
1262                 toks[0], src = src[:c1], src[c1+1:]
1263                 c2 := bytes.IndexRune(src, c)
1264                 if c2 < 0 {
1265                         toks[1] = src
1266                         return 2
1267                 }
1268                 toks[1], toks[2] = src[:c2], src[c2+1:]
1269                 return 3
1270         }
1271         for i, stream := range streams {
1272                 lineno := i + 1
1273                 var anyFileTokens bool
1274                 var pos int64
1275                 var segIdx int
1276                 segments = segments[:0]
1277                 pathparts = nil
1278                 streamparts := 0
1279                 for i, token := range bytes.Split(stream, []byte{' '}) {
1280                         if i == 0 {
1281                                 pathparts = strings.Split(manifestUnescape(string(token)), "/")
1282                                 streamparts = len(pathparts)
1283                                 continue
1284                         }
1285                         if !bytes.ContainsRune(token, ':') {
1286                                 if anyFileTokens {
1287                                         return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1288                                 }
1289                                 if splitToToks(token, '+') < 2 {
1290                                         return fmt.Errorf("line %d: bad locator %q", lineno, token)
1291                                 }
1292                                 length, err := strconv.ParseInt(string(toks[1]), 10, 32)
1293                                 if err != nil || length < 0 {
1294                                         return fmt.Errorf("line %d: bad locator %q", lineno, token)
1295                                 }
1296                                 segments = append(segments, storedSegment{
1297                                         locator: string(token),
1298                                         size:    int(length),
1299                                         offset:  0,
1300                                         length:  int(length),
1301                                 })
1302                                 continue
1303                         } else if len(segments) == 0 {
1304                                 return fmt.Errorf("line %d: bad locator %q", lineno, token)
1305                         }
1306                         if splitToToks(token, ':') != 3 {
1307                                 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1308                         }
1309                         anyFileTokens = true
1310
1311                         offset, err := strconv.ParseInt(string(toks[0]), 10, 64)
1312                         if err != nil || offset < 0 {
1313                                 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1314                         }
1315                         length, err := strconv.ParseInt(string(toks[1]), 10, 64)
1316                         if err != nil || length < 0 {
1317                                 return fmt.Errorf("line %d: bad file segment %q", lineno, token)
1318                         }
1319                         if !bytes.ContainsAny(toks[2], `\/`) {
1320                                 // optimization for a common case
1321                                 pathparts = append(pathparts[:streamparts], string(toks[2]))
1322                         } else {
1323                                 pathparts = append(pathparts[:streamparts], strings.Split(manifestUnescape(string(toks[2])), "/")...)
1324                         }
1325                         fnode, err := dn.createFileAndParents(pathparts)
1326                         if fnode == nil && err == nil && length == 0 {
1327                                 // Special case: an empty file used as
1328                                 // a marker to preserve an otherwise
1329                                 // empty directory in a manifest.
1330                                 continue
1331                         }
1332                         if err != nil || (fnode == nil && length != 0) {
1333                                 return fmt.Errorf("line %d: cannot use name %q with length %d: %s", lineno, toks[2], length, err)
1334                         }
1335                         // Map the stream offset/range coordinates to
1336                         // block/offset/range coordinates and add
1337                         // corresponding storedSegments to the filenode
1338                         if pos > offset {
1339                                 // Can't continue where we left off.
1340                                 // TODO: binary search instead of
1341                                 // rewinding all the way (but this
1342                                 // situation might be rare anyway)
1343                                 segIdx, pos = 0, 0
1344                         }
1345                         for ; segIdx < len(segments); segIdx++ {
1346                                 seg := segments[segIdx]
1347                                 next := pos + int64(seg.Len())
1348                                 if next <= offset || seg.Len() == 0 {
1349                                         pos = next
1350                                         continue
1351                                 }
1352                                 if pos >= offset+length {
1353                                         break
1354                                 }
1355                                 var blkOff int
1356                                 if pos < offset {
1357                                         blkOff = int(offset - pos)
1358                                 }
1359                                 blkLen := seg.Len() - blkOff
1360                                 if pos+int64(blkOff+blkLen) > offset+length {
1361                                         blkLen = int(offset + length - pos - int64(blkOff))
1362                                 }
1363                                 fnode.appendSegment(storedSegment{
1364                                         kc:      dn.fs,
1365                                         locator: seg.locator,
1366                                         size:    seg.size,
1367                                         offset:  blkOff,
1368                                         length:  blkLen,
1369                                 })
1370                                 if next > offset+length {
1371                                         break
1372                                 } else {
1373                                         pos = next
1374                                 }
1375                         }
1376                         if segIdx == len(segments) && pos < offset+length {
1377                                 return fmt.Errorf("line %d: invalid segment in %d-byte stream: %q", lineno, pos, token)
1378                         }
1379                 }
1380                 if !anyFileTokens {
1381                         return fmt.Errorf("line %d: no file segments", lineno)
1382                 } else if len(segments) == 0 {
1383                         return fmt.Errorf("line %d: no locators", lineno)
1384                 } else if streamparts == 0 {
1385                         return fmt.Errorf("line %d: no stream name", lineno)
1386                 }
1387         }
1388         return nil
1389 }
1390
1391 // only safe to call from loadManifest -- no locking.
1392 //
1393 // If path is a "parent directory exists" marker (the last path
1394 // component is "."), the returned values are both nil.
1395 //
1396 // Newly added nodes have modtime==0. Caller is responsible for fixing
1397 // them with backdateTree.
1398 func (dn *dirnode) createFileAndParents(names []string) (fn *filenode, err error) {
1399         var node inode = dn
1400         basename := names[len(names)-1]
1401         for _, name := range names[:len(names)-1] {
1402                 switch name {
1403                 case "", ".":
1404                         continue
1405                 case "..":
1406                         if node == dn {
1407                                 // can't be sure parent will be a *dirnode
1408                                 return nil, ErrInvalidArgument
1409                         }
1410                         node = node.Parent()
1411                         continue
1412                 }
1413                 node.Lock()
1414                 unlock := node.Unlock
1415                 node, err = node.Child(name, func(child inode) (inode, error) {
1416                         if child == nil {
1417                                 // note modtime will be fixed later in backdateTree()
1418                                 child, err := node.FS().newNode(name, 0755|os.ModeDir, time.Time{})
1419                                 if err != nil {
1420                                         return nil, err
1421                                 }
1422                                 child.SetParent(node, name)
1423                                 return child, nil
1424                         } else if !child.IsDir() {
1425                                 return child, ErrFileExists
1426                         } else {
1427                                 return child, nil
1428                         }
1429                 })
1430                 unlock()
1431                 if err != nil {
1432                         return
1433                 }
1434         }
1435         if basename == "." {
1436                 return
1437         } else if !permittedName(basename) {
1438                 err = fmt.Errorf("invalid file part %q in path %q", basename, names)
1439                 return
1440         }
1441         node.Lock()
1442         defer node.Unlock()
1443         _, err = node.Child(basename, func(child inode) (inode, error) {
1444                 switch child := child.(type) {
1445                 case nil:
1446                         child, err = node.FS().newNode(basename, 0755, time.Time{})
1447                         if err != nil {
1448                                 return nil, err
1449                         }
1450                         child.SetParent(node, basename)
1451                         fn = child.(*filenode)
1452                         return child, nil
1453                 case *filenode:
1454                         fn = child
1455                         return child, nil
1456                 case *dirnode:
1457                         return child, ErrIsDirectory
1458                 default:
1459                         return child, ErrInvalidArgument
1460                 }
1461         })
1462         return
1463 }
1464
1465 func (dn *dirnode) TreeSize() (bytes int64) {
1466         dn.RLock()
1467         defer dn.RUnlock()
1468         for _, i := range dn.inodes {
1469                 switch i := i.(type) {
1470                 case *filenode:
1471                         bytes += i.Size()
1472                 case *dirnode:
1473                         bytes += i.TreeSize()
1474                 }
1475         }
1476         return
1477 }
1478
1479 type segment interface {
1480         io.ReaderAt
1481         Len() int
1482         // Return a new segment with a subsection of the data from this
1483         // one. length<0 means length=Len()-off.
1484         Slice(off int, length int) segment
1485 }
1486
1487 type memSegment struct {
1488         buf []byte
1489         // If flushing is not nil and not ready/closed, then a) buf is
1490         // being shared by a pruneMemSegments goroutine, and must be
1491         // copied on write; and b) the flushing channel will close
1492         // when the goroutine finishes, whether it succeeds or not.
1493         flushing <-chan struct{}
1494 }
1495
1496 func (me *memSegment) flushingUnfinished() bool {
1497         if me.flushing == nil {
1498                 return false
1499         }
1500         select {
1501         case <-me.flushing:
1502                 me.flushing = nil
1503                 return false
1504         default:
1505                 return true
1506         }
1507 }
1508
1509 func (me *memSegment) Len() int {
1510         return len(me.buf)
1511 }
1512
1513 func (me *memSegment) Slice(off, length int) segment {
1514         if length < 0 {
1515                 length = len(me.buf) - off
1516         }
1517         buf := make([]byte, length)
1518         copy(buf, me.buf[off:])
1519         return &memSegment{buf: buf}
1520 }
1521
1522 func (me *memSegment) Truncate(n int) {
1523         if n > cap(me.buf) || (me.flushing != nil && n > len(me.buf)) {
1524                 newsize := 1024
1525                 for newsize < n {
1526                         newsize = newsize << 2
1527                 }
1528                 newbuf := make([]byte, n, newsize)
1529                 copy(newbuf, me.buf)
1530                 me.buf, me.flushing = newbuf, nil
1531         } else {
1532                 // reclaim existing capacity, and zero reclaimed part
1533                 oldlen := len(me.buf)
1534                 me.buf = me.buf[:n]
1535                 for i := oldlen; i < n; i++ {
1536                         me.buf[i] = 0
1537                 }
1538         }
1539 }
1540
1541 func (me *memSegment) WriteAt(p []byte, off int) {
1542         if off+len(p) > len(me.buf) {
1543                 panic("overflowed segment")
1544         }
1545         if me.flushing != nil {
1546                 me.buf, me.flushing = append([]byte(nil), me.buf...), nil
1547         }
1548         copy(me.buf[off:], p)
1549 }
1550
1551 func (me *memSegment) ReadAt(p []byte, off int64) (n int, err error) {
1552         if off > int64(me.Len()) {
1553                 err = io.EOF
1554                 return
1555         }
1556         n = copy(p, me.buf[int(off):])
1557         if n < len(p) {
1558                 err = io.EOF
1559         }
1560         return
1561 }
1562
1563 type storedSegment struct {
1564         kc      fsBackend
1565         locator string
1566         size    int // size of stored block (also encoded in locator)
1567         offset  int // position of segment within the stored block
1568         length  int // bytes in this segment (offset + length <= size)
1569 }
1570
1571 func (se storedSegment) Len() int {
1572         return se.length
1573 }
1574
1575 func (se storedSegment) Slice(n, size int) segment {
1576         se.offset += n
1577         se.length -= n
1578         if size >= 0 && se.length > size {
1579                 se.length = size
1580         }
1581         return se
1582 }
1583
1584 func (se storedSegment) ReadAt(p []byte, off int64) (n int, err error) {
1585         if off > int64(se.length) {
1586                 return 0, io.EOF
1587         }
1588         maxlen := se.length - int(off)
1589         if len(p) > maxlen {
1590                 p = p[:maxlen]
1591                 n, err = se.kc.ReadAt(se.locator, p, int(off)+se.offset)
1592                 if err == nil {
1593                         err = io.EOF
1594                 }
1595                 return
1596         }
1597         return se.kc.ReadAt(se.locator, p, int(off)+se.offset)
1598 }
1599
1600 func canonicalName(name string) string {
1601         name = path.Clean("/" + name)
1602         if name == "/" || name == "./" {
1603                 name = "."
1604         } else if strings.HasPrefix(name, "/") {
1605                 name = "." + name
1606         }
1607         return name
1608 }
1609
1610 var manifestEscapeSeq = regexp.MustCompile(`\\([0-7]{3}|\\)`)
1611
1612 func manifestUnescapeFunc(seq string) string {
1613         if seq == `\\` {
1614                 return `\`
1615         }
1616         i, err := strconv.ParseUint(seq[1:], 8, 8)
1617         if err != nil {
1618                 // Invalid escape sequence: can't unescape.
1619                 return seq
1620         }
1621         return string([]byte{byte(i)})
1622 }
1623
1624 func manifestUnescape(s string) string {
1625         return manifestEscapeSeq.ReplaceAllStringFunc(s, manifestUnescapeFunc)
1626 }
1627
1628 var manifestEscapedChar = regexp.MustCompile(`[\000-\040:\s\\]`)
1629
1630 func manifestEscapeFunc(seq string) string {
1631         return fmt.Sprintf("\\%03o", byte(seq[0]))
1632 }
1633
1634 func manifestEscape(s string) string {
1635         return manifestEscapedChar.ReplaceAllStringFunc(s, manifestEscapeFunc)
1636 }