X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/78724c7c38805402c442629d4d678f8145e31d76..15f6e58f3b7c158aa735ce78597cf22c6ef3543b:/sdk/go/arvados/fs_collection.go diff --git a/sdk/go/arvados/fs_collection.go b/sdk/go/arvados/fs_collection.go index 9aaa3747b4..2b5df76ad6 100644 --- a/sdk/go/arvados/fs_collection.go +++ b/sdk/go/arvados/fs_collection.go @@ -5,6 +5,7 @@ package arvados import ( + "bytes" "context" "encoding/json" "fmt" @@ -16,13 +17,13 @@ import ( "strconv" "strings" "sync" + "sync/atomic" "time" ) var ( maxBlockSize = 1 << 26 - concurrentWriters = 4 // max goroutines writing to Keep during flush() - writeAheadBlocks = 1 // max background jobs flushing to Keep before blocking writes + concurrentWriters = 4 // max goroutines writing to Keep in background and during flush() ) // A CollectionFileSystem is a FileSystem that can be serialized as a @@ -38,30 +39,32 @@ type CollectionFileSystem interface { // Total data bytes in all files. Size() int64 - - // Memory consumed by buffered file data. - memorySize() int64 } type collectionFileSystem struct { fileSystem - uuid string + uuid string + replicas int + storageClasses []string } // FileSystem returns a CollectionFileSystem for the collection. func (c *Collection) FileSystem(client apiClient, kc keepClient) (CollectionFileSystem, error) { - var modTime time.Time - if c.ModifiedAt == nil { + modTime := c.ModifiedAt + if modTime.IsZero() { modTime = time.Now() - } else { - modTime = *c.ModifiedAt } fs := &collectionFileSystem{ - uuid: c.UUID, + uuid: c.UUID, + storageClasses: c.StorageClassesDesired, fileSystem: fileSystem{ fsBackend: keepBackend{apiClient: client, keepClient: kc}, + thr: newThrottle(concurrentWriters), }, } + if r := c.ReplicationDesired; r != nil { + fs.replicas = *r + } root := &dirnode{ fs: fs, treenode: treenode{ @@ -110,16 +113,71 @@ func (fs *collectionFileSystem) newNode(name string, perm os.FileMode, modTime t inodes: make(map[string]inode), }, }, nil - } else { - return &filenode{ - fs: fs, - fileinfo: fileinfo{ - name: name, - mode: perm & ^os.ModeDir, - modTime: modTime, - }, - }, nil } + return &filenode{ + fs: fs, + fileinfo: fileinfo{ + name: name, + mode: perm & ^os.ModeDir, + modTime: modTime, + }, + }, nil +} + +func (fs *collectionFileSystem) Child(name string, replace func(inode) (inode, error)) (inode, error) { + return fs.rootnode().Child(name, replace) +} + +func (fs *collectionFileSystem) FS() FileSystem { + return fs +} + +func (fs *collectionFileSystem) FileInfo() os.FileInfo { + return fs.rootnode().FileInfo() +} + +func (fs *collectionFileSystem) IsDir() bool { + return true +} + +func (fs *collectionFileSystem) Lock() { + fs.rootnode().Lock() +} + +func (fs *collectionFileSystem) Unlock() { + fs.rootnode().Unlock() +} + +func (fs *collectionFileSystem) RLock() { + fs.rootnode().RLock() +} + +func (fs *collectionFileSystem) RUnlock() { + fs.rootnode().RUnlock() +} + +func (fs *collectionFileSystem) Parent() inode { + return fs.rootnode().Parent() +} + +func (fs *collectionFileSystem) Read(_ []byte, ptr filenodePtr) (int, filenodePtr, error) { + return 0, ptr, ErrInvalidOperation +} + +func (fs *collectionFileSystem) Write(_ []byte, ptr filenodePtr) (int, filenodePtr, error) { + return 0, ptr, ErrInvalidOperation +} + +func (fs *collectionFileSystem) Readdir() ([]os.FileInfo, error) { + return fs.rootnode().Readdir() +} + +func (fs *collectionFileSystem) SetParent(parent inode, name string) { + fs.rootnode().SetParent(parent, name) +} + +func (fs *collectionFileSystem) Truncate(int64) error { + return ErrInvalidOperation } func (fs *collectionFileSystem) Sync() error { @@ -146,23 +204,48 @@ func (fs *collectionFileSystem) Sync() error { return nil } -func (fs *collectionFileSystem) Flush(shortBlocks bool) error { - fs.fileSystem.root.Lock() - defer fs.fileSystem.root.Unlock() - dn := fs.fileSystem.root.(*dirnode) - return dn.flush(context.TODO(), newThrottle(concurrentWriters), dn.sortedNames(), flushOpts{sync: false, shortBlocks: shortBlocks}) +func (fs *collectionFileSystem) Flush(path string, shortBlocks bool) error { + node, err := rlookup(fs.fileSystem.root, path) + if err != nil { + return err + } + dn, ok := node.(*dirnode) + if !ok { + return ErrNotADirectory + } + dn.Lock() + defer dn.Unlock() + names := dn.sortedNames() + if path != "" { + // Caller only wants to flush the specified dir, + // non-recursively. Drop subdirs from the list of + // names. + var filenames []string + for _, name := range names { + if _, ok := dn.inodes[name].(*filenode); ok { + filenames = append(filenames, name) + } + } + names = filenames + } + for _, name := range names { + child := dn.inodes[name] + child.Lock() + defer child.Unlock() + } + return dn.flush(context.TODO(), names, flushOpts{sync: false, shortBlocks: shortBlocks}) } -func (fs *collectionFileSystem) memorySize() int64 { +func (fs *collectionFileSystem) MemorySize() int64 { fs.fileSystem.root.Lock() defer fs.fileSystem.root.Unlock() - return fs.fileSystem.root.(*dirnode).memorySize() + return fs.fileSystem.root.(*dirnode).MemorySize() } func (fs *collectionFileSystem) MarshalManifest(prefix string) (string, error) { fs.fileSystem.root.Lock() defer fs.fileSystem.root.Unlock() - return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix, newThrottle(concurrentWriters)) + return fs.fileSystem.root.(*dirnode).marshalManifest(context.TODO(), prefix) } func (fs *collectionFileSystem) Size() int64 { @@ -245,7 +328,7 @@ func (fn *filenode) seek(startPtr filenodePtr) (ptr filenodePtr) { // filenode implements inode. type filenode struct { parent inode - fs FileSystem + fs *collectionFileSystem fileinfo fileinfo segments []segment // number of times `segments` has changed in a @@ -254,7 +337,6 @@ type filenode struct { memsize int64 // bytes in memSegments sync.RWMutex nullnode - throttle *throttle } // caller must have lock @@ -489,8 +571,6 @@ func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePt seg.Truncate(len(cando)) fn.memsize += int64(len(cando)) fn.segments[cur] = seg - cur++ - prev++ } } @@ -519,10 +599,6 @@ func (fn *filenode) Write(p []byte, startPtr filenodePtr) (n int, ptr filenodePt func (fn *filenode) pruneMemSegments() { // TODO: share code with (*dirnode)flush() // TODO: pack/flush small blocks too, when fragmented - if fn.throttle == nil { - // TODO: share a throttle with filesystem - fn.throttle = newThrottle(writeAheadBlocks) - } for idx, seg := range fn.segments { seg, ok := seg.(*memSegment) if !ok || seg.Len() < maxBlockSize || seg.flushing != nil { @@ -538,18 +614,21 @@ func (fn *filenode) pruneMemSegments() { // progress, block here until one finishes, rather // than pile up an unlimited number of buffered writes // and network flush operations. - fn.throttle.Acquire() + fn.fs.throttle().Acquire() go func() { defer close(done) - locator, _, err := fn.FS().PutB(buf) - fn.throttle.Release() + resp, err := fn.FS().BlockWrite(context.Background(), BlockWriteOptions{ + Data: buf, + Replicas: fn.fs.replicas, + StorageClasses: fn.fs.storageClasses, + }) + fn.fs.throttle().Release() fn.Lock() defer fn.Unlock() if seg.flushing != done { // A new seg.buf has been allocated. return } - seg.flushing = nil if err != nil { // TODO: stall (or return errors from) // subsequent writes until flushing @@ -563,7 +642,7 @@ func (fn *filenode) pruneMemSegments() { fn.memsize -= int64(len(buf)) fn.segments[idx] = storedSegment{ kc: fn.FS(), - locator: locator, + locator: resp.Locator, size: len(buf), offset: 0, length: len(buf), @@ -606,6 +685,7 @@ func (dn *dirnode) Child(name string, replace func(inode) (inode, error)) (inode if err != nil { return nil, err } + coll.UUID = dn.fs.uuid data, err := json.Marshal(&coll) if err == nil { data = append(data, '\n') @@ -629,6 +709,9 @@ type fnSegmentRef struct { // storedSegments that reference the relevant portions of the new // block. // +// bufsize is the total data size in refs. It is used to preallocate +// the correct amount of memory when len(refs)>1. +// // If sync is false, commitBlock returns right away, after starting a // goroutine to do the writes, reacquire the filenodes' locks, and // swap out the *memSegments. Some filenodes' segments might get @@ -636,61 +719,59 @@ type fnSegmentRef struct { // won't replace them. // // Caller must have write lock. -func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, sync bool) error { +func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, bufsize int, sync bool) error { + if len(refs) == 0 { + return nil + } if err := ctx.Err(); err != nil { return err } done := make(chan struct{}) - block := make([]byte, 0, maxBlockSize) + var block []byte segs := make([]*memSegment, 0, len(refs)) offsets := make([]int, 0, len(refs)) // location of segment's data within block for _, ref := range refs { seg := ref.fn.segments[ref.idx].(*memSegment) - if seg.flushing != nil && !sync { + if !sync && seg.flushingUnfinished() { // Let the other flushing goroutine finish. If // it fails, we'll try again next time. + close(done) return nil - } else { - // In sync mode, we proceed regardless of - // whether another flush is in progress: It - // can't finish before we do, because we hold - // fn's lock until we finish our own writes. } + // In sync mode, we proceed regardless of + // whether another flush is in progress: It + // can't finish before we do, because we hold + // fn's lock until we finish our own writes. seg.flushing = done offsets = append(offsets, len(block)) - block = append(block, seg.buf...) + if len(refs) == 1 { + block = seg.buf + } else if block == nil { + block = append(make([]byte, 0, bufsize), seg.buf...) + } else { + block = append(block, seg.buf...) + } segs = append(segs, seg) } + blocksize := len(block) + dn.fs.throttle().Acquire() errs := make(chan error, 1) go func() { defer close(done) defer close(errs) - locked := map[*filenode]bool{} - locator, _, err := dn.fs.PutB(block) - { - if !sync { - for _, name := range dn.sortedNames() { - if fn, ok := dn.inodes[name].(*filenode); ok { - fn.Lock() - defer fn.Unlock() - locked[fn] = true - } - } - } - defer func() { - for _, seg := range segs { - if seg.flushing == done { - seg.flushing = nil - } - } - }() - } + resp, err := dn.fs.BlockWrite(context.Background(), BlockWriteOptions{ + Data: block, + Replicas: dn.fs.replicas, + StorageClasses: dn.fs.storageClasses, + }) + dn.fs.throttle().Release() if err != nil { errs <- err return } for idx, ref := range refs { if !sync { + ref.fn.Lock() // In async mode, fn's lock was // released while we were waiting for // PutB(); lots of things might have @@ -699,36 +780,41 @@ func (dn *dirnode) commitBlock(ctx context.Context, refs []fnSegmentRef, sync bo // file segments have // rearranged or changed in // some way + ref.fn.Unlock() continue } else if seg, ok := ref.fn.segments[ref.idx].(*memSegment); !ok || seg != segs[idx] { // segment has been replaced + ref.fn.Unlock() continue } else if seg.flushing != done { // seg.buf has been replaced - continue - } else if !locked[ref.fn] { - // file was renamed, moved, or - // deleted since we called - // PutB + ref.fn.Unlock() continue } } data := ref.fn.segments[ref.idx].(*memSegment).buf ref.fn.segments[ref.idx] = storedSegment{ kc: dn.fs, - locator: locator, - size: len(block), + locator: resp.Locator, + size: blocksize, offset: offsets[idx], length: len(data), } - ref.fn.memsize -= int64(len(data)) + // atomic is needed here despite caller having + // lock: caller might be running concurrent + // commitBlock() goroutines using the same + // lock, writing different segments from the + // same file. + atomic.AddInt64(&ref.fn.memsize, -int64(len(data))) + if !sync { + ref.fn.Unlock() + } } }() if sync { return <-errs - } else { - return nil } + return nil } type flushOpts struct { @@ -743,18 +829,13 @@ type flushOpts struct { // Caller must have write lock on dn and the named children. // // If any children are dirs, they will be flushed recursively. -func (dn *dirnode) flush(ctx context.Context, throttle *throttle, names []string, opts flushOpts) error { +func (dn *dirnode) flush(ctx context.Context, names []string, opts flushOpts) error { cg := newContextGroup(ctx) defer cg.Cancel() - goCommit := func(refs []fnSegmentRef) { - if len(refs) == 0 { - return - } + goCommit := func(refs []fnSegmentRef, bufsize int) { cg.Go(func() error { - throttle.Acquire() - defer throttle.Release() - return dn.commitBlock(cg.Context(), refs, opts.sync) + return dn.commitBlock(cg.Context(), refs, bufsize, opts.sync) }) } @@ -764,13 +845,13 @@ func (dn *dirnode) flush(ctx context.Context, throttle *throttle, names []string for _, name := range names { switch node := dn.inodes[name].(type) { case *dirnode: - names := node.sortedNames() - for _, name := range names { - child := node.inodes[name] - child.Lock() - defer child.Unlock() + grandchildNames := node.sortedNames() + for _, grandchildName := range grandchildNames { + grandchild := node.inodes[grandchildName] + grandchild.Lock() + defer grandchild.Unlock() } - cg.Go(func() error { return node.flush(cg.Context(), throttle, node.sortedNames(), opts) }) + cg.Go(func() error { return node.flush(cg.Context(), grandchildNames, opts) }) case *filenode: for idx, seg := range node.segments { switch seg := seg.(type) { @@ -788,11 +869,11 @@ func (dn *dirnode) flush(ctx context.Context, throttle *throttle, names []string node.segments[idx] = seg case *memSegment: if seg.Len() > maxBlockSize/2 { - goCommit([]fnSegmentRef{{node, idx}}) + goCommit([]fnSegmentRef{{node, idx}}, seg.Len()) continue } if pendingLen+seg.Len() > maxBlockSize { - goCommit(pending) + goCommit(pending, pendingLen) pending = nil pendingLen = 0 } @@ -805,20 +886,20 @@ func (dn *dirnode) flush(ctx context.Context, throttle *throttle, names []string } } if opts.shortBlocks { - goCommit(pending) + goCommit(pending, pendingLen) } return cg.Wait() } // caller must have write lock. -func (dn *dirnode) memorySize() (size int64) { +func (dn *dirnode) MemorySize() (size int64) { for _, name := range dn.sortedNames() { node := dn.inodes[name] node.Lock() defer node.Unlock() switch node := node.(type) { case *dirnode: - size += node.memorySize() + size += node.MemorySize() case *filenode: for _, seg := range node.segments { switch seg := seg.(type) { @@ -842,7 +923,7 @@ func (dn *dirnode) sortedNames() []string { } // caller must have write lock. -func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle *throttle) (string, error) { +func (dn *dirnode) marshalManifest(ctx context.Context, prefix string) (string, error) { cg := newContextGroup(ctx) defer cg.Cancel() @@ -889,7 +970,7 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle for i, name := range dirnames { i, name := i, name cg.Go(func() error { - txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name, throttle) + txt, err := dn.inodes[name].(*dirnode).marshalManifest(cg.Context(), prefix+"/"+name) subdirs[i] = txt return err }) @@ -905,7 +986,7 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle var fileparts []filepart var blocks []string - if err := dn.flush(cg.Context(), throttle, filenames, flushOpts{sync: true, shortBlocks: true}); err != nil { + if err := dn.flush(cg.Context(), filenames, flushOpts{sync: true, shortBlocks: true}); err != nil { return err } for _, name := range filenames { @@ -960,38 +1041,64 @@ func (dn *dirnode) marshalManifest(ctx context.Context, prefix string, throttle } func (dn *dirnode) loadManifest(txt string) error { - var dirname string - streams := strings.Split(txt, "\n") - if streams[len(streams)-1] != "" { + streams := bytes.Split([]byte(txt), []byte{'\n'}) + if len(streams[len(streams)-1]) != 0 { return fmt.Errorf("line %d: no trailing newline", len(streams)) } streams = streams[:len(streams)-1] segments := []storedSegment{} + // To reduce allocs, we reuse a single "pathparts" slice + // (pre-split on "/" separators) for the duration of this + // func. + var pathparts []string + // To reduce allocs, we reuse a single "toks" slice of 3 byte + // slices. + var toks = make([][]byte, 3) + // Similar to bytes.SplitN(token, []byte{c}, 3), but splits + // into the toks slice rather than allocating a new one, and + // returns the number of toks (1, 2, or 3). + splitToToks := func(src []byte, c rune) int { + c1 := bytes.IndexRune(src, c) + if c1 < 0 { + toks[0] = src + return 1 + } + toks[0], src = src[:c1], src[c1+1:] + c2 := bytes.IndexRune(src, c) + if c2 < 0 { + toks[1] = src + return 2 + } + toks[1], toks[2] = src[:c2], src[c2+1:] + return 3 + } for i, stream := range streams { lineno := i + 1 var anyFileTokens bool var pos int64 var segIdx int segments = segments[:0] - for i, token := range strings.Split(stream, " ") { + pathparts = nil + streamparts := 0 + for i, token := range bytes.Split(stream, []byte{' '}) { if i == 0 { - dirname = manifestUnescape(token) + pathparts = strings.Split(manifestUnescape(string(token)), "/") + streamparts = len(pathparts) continue } - if !strings.Contains(token, ":") { + if !bytes.ContainsRune(token, ':') { if anyFileTokens { return fmt.Errorf("line %d: bad file segment %q", lineno, token) } - toks := strings.SplitN(token, "+", 3) - if len(toks) < 2 { + if splitToToks(token, '+') < 2 { return fmt.Errorf("line %d: bad locator %q", lineno, token) } - length, err := strconv.ParseInt(toks[1], 10, 32) + length, err := strconv.ParseInt(string(toks[1]), 10, 32) if err != nil || length < 0 { return fmt.Errorf("line %d: bad locator %q", lineno, token) } segments = append(segments, storedSegment{ - locator: token, + locator: string(token), size: int(length), offset: 0, length: int(length), @@ -1000,23 +1107,26 @@ func (dn *dirnode) loadManifest(txt string) error { } else if len(segments) == 0 { return fmt.Errorf("line %d: bad locator %q", lineno, token) } - - toks := strings.SplitN(token, ":", 3) - if len(toks) != 3 { + if splitToToks(token, ':') != 3 { return fmt.Errorf("line %d: bad file segment %q", lineno, token) } anyFileTokens = true - offset, err := strconv.ParseInt(toks[0], 10, 64) + offset, err := strconv.ParseInt(string(toks[0]), 10, 64) if err != nil || offset < 0 { return fmt.Errorf("line %d: bad file segment %q", lineno, token) } - length, err := strconv.ParseInt(toks[1], 10, 64) + length, err := strconv.ParseInt(string(toks[1]), 10, 64) if err != nil || length < 0 { return fmt.Errorf("line %d: bad file segment %q", lineno, token) } - name := dirname + "/" + manifestUnescape(toks[2]) - fnode, err := dn.createFileAndParents(name) + if !bytes.ContainsAny(toks[2], `\/`) { + // optimization for a common case + pathparts = append(pathparts[:streamparts], string(toks[2])) + } else { + pathparts = append(pathparts[:streamparts], strings.Split(manifestUnescape(string(toks[2])), "/")...) + } + fnode, err := dn.createFileAndParents(pathparts) if fnode == nil && err == nil && length == 0 { // Special case: an empty file used as // a marker to preserve an otherwise @@ -1024,7 +1134,7 @@ func (dn *dirnode) loadManifest(txt string) error { continue } if err != nil || (fnode == nil && length != 0) { - return fmt.Errorf("line %d: cannot use path %q with length %d: %s", lineno, name, length, err) + return fmt.Errorf("line %d: cannot use name %q with length %d: %s", lineno, toks[2], length, err) } // Map the stream offset/range coordinates to // block/offset/range coordinates and add @@ -1036,9 +1146,9 @@ func (dn *dirnode) loadManifest(txt string) error { // situation might be rare anyway) segIdx, pos = 0, 0 } - for next := int64(0); segIdx < len(segments); segIdx++ { + for ; segIdx < len(segments); segIdx++ { seg := segments[segIdx] - next = pos + int64(seg.Len()) + next := pos + int64(seg.Len()) if next <= offset || seg.Len() == 0 { pos = next continue @@ -1075,7 +1185,7 @@ func (dn *dirnode) loadManifest(txt string) error { return fmt.Errorf("line %d: no file segments", lineno) } else if len(segments) == 0 { return fmt.Errorf("line %d: no locators", lineno) - } else if dirname == "" { + } else if streamparts == 0 { return fmt.Errorf("line %d: no stream name", lineno) } } @@ -1086,9 +1196,11 @@ func (dn *dirnode) loadManifest(txt string) error { // // If path is a "parent directory exists" marker (the last path // component is "."), the returned values are both nil. -func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) { +// +// Newly added nodes have modtime==0. Caller is responsible for fixing +// them with backdateTree. +func (dn *dirnode) createFileAndParents(names []string) (fn *filenode, err error) { var node inode = dn - names := strings.Split(path, "/") basename := names[len(names)-1] for _, name := range names[:len(names)-1] { switch name { @@ -1102,9 +1214,12 @@ func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) { node = node.Parent() continue } + node.Lock() + unlock := node.Unlock node, err = node.Child(name, func(child inode) (inode, error) { if child == nil { - child, err := node.FS().newNode(name, 0755|os.ModeDir, node.Parent().FileInfo().ModTime()) + // note modtime will be fixed later in backdateTree() + child, err := node.FS().newNode(name, 0755|os.ModeDir, time.Time{}) if err != nil { return nil, err } @@ -1116,6 +1231,7 @@ func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) { return child, nil } }) + unlock() if err != nil { return } @@ -1123,13 +1239,15 @@ func (dn *dirnode) createFileAndParents(path string) (fn *filenode, err error) { if basename == "." { return } else if !permittedName(basename) { - err = fmt.Errorf("invalid file part %q in path %q", basename, path) + err = fmt.Errorf("invalid file part %q in path %q", basename, names) return } + node.Lock() + defer node.Unlock() _, err = node.Child(basename, func(child inode) (inode, error) { switch child := child.(type) { case nil: - child, err = node.FS().newNode(basename, 0755, node.FileInfo().ModTime()) + child, err = node.FS().newNode(basename, 0755, time.Time{}) if err != nil { return nil, err } @@ -1172,13 +1290,26 @@ type segment interface { type memSegment struct { buf []byte - // If flushing is not nil, then a) buf is being shared by a - // pruneMemSegments goroutine, and must be copied on write; - // and b) the flushing channel will close when the goroutine - // finishes, whether it succeeds or not. + // If flushing is not nil and not ready/closed, then a) buf is + // being shared by a pruneMemSegments goroutine, and must be + // copied on write; and b) the flushing channel will close + // when the goroutine finishes, whether it succeeds or not. flushing <-chan struct{} } +func (me *memSegment) flushingUnfinished() bool { + if me.flushing == nil { + return false + } + select { + case <-me.flushing: + me.flushing = nil + return false + default: + return true + } +} + func (me *memSegment) Len() int { return len(me.buf) }