X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/90198dc6c008ca60889dd4ef5f5def8dd272df82..d0584563200b0fe69e508b2fc5b0ddb223ebccc0:/services/keepstore/volume_unix.go diff --git a/services/keepstore/volume_unix.go b/services/keepstore/volume_unix.go index 84877c0034..23d6753592 100644 --- a/services/keepstore/volume_unix.go +++ b/services/keepstore/volume_unix.go @@ -1,149 +1,349 @@ -// A UnixVolume is a Volume backed by a locally mounted disk. +// Copyright (C) The Arvados Authors. All rights reserved. // +// SPDX-License-Identifier: AGPL-3.0 + package main import ( + "bufio" + "context" + "flag" "fmt" + "io" "io/ioutil" - "log" "os" + "os/exec" "path/filepath" + "regexp" "strconv" "strings" + "sync" + "sync/atomic" "syscall" "time" ) -// IORequests are encapsulated Get or Put requests. They are used to -// implement serialized I/O (i.e. only one read/write operation per -// volume). When running in serialized mode, the Keep front end sends -// IORequests on a channel to an IORunner, which handles them one at a -// time and returns an IOResponse. -// -type IOMethod int +type unixVolumeAdder struct { + *Config +} -const ( - KeepGet IOMethod = iota - KeepPut -) +// String implements flag.Value +func (s *unixVolumeAdder) String() string { + return "-" +} -type IORequest struct { - method IOMethod - loc string - data []byte - reply chan *IOResponse +func (vs *unixVolumeAdder) Set(path string) error { + if dirs := strings.Split(path, ","); len(dirs) > 1 { + log.Print("DEPRECATED: using comma-separated volume list.") + for _, dir := range dirs { + if err := vs.Set(dir); err != nil { + return err + } + } + return nil + } + vs.Config.Volumes = append(vs.Config.Volumes, &UnixVolume{ + Root: path, + ReadOnly: deprecated.flagReadonly, + Serialize: deprecated.flagSerializeIO, + }) + return nil } -type IOResponse struct { - data []byte - err error +func init() { + VolumeTypes = append(VolumeTypes, func() VolumeWithExamples { return &UnixVolume{} }) + + flag.Var(&unixVolumeAdder{theConfig}, "volumes", "see Volumes configuration") + flag.Var(&unixVolumeAdder{theConfig}, "volume", "see Volumes configuration") } -// A UnixVolume has the following properties: -// -// root -// the path to the volume's root directory -// queue -// A channel of IORequests. If non-nil, all I/O requests for -// this volume should be queued on this channel; the result -// will be delivered on the IOResponse channel supplied in the -// request. -// -type UnixVolume struct { - root string // path to this volume - queue chan *IORequest -} - -func (v *UnixVolume) IOHandler() { - for req := range v.queue { - var result IOResponse - switch req.method { - case KeepGet: - result.data, result.err = v.Read(req.loc) - case KeepPut: - result.err = v.Write(req.loc, req.data) +// Discover adds a UnixVolume for every directory named "keep" that is +// located at the top level of a device- or tmpfs-backed mount point +// other than "/". It returns the number of volumes added. +func (vs *unixVolumeAdder) Discover() int { + added := 0 + f, err := os.Open(ProcMounts) + if err != nil { + log.Fatalf("opening %s: %s", ProcMounts, err) + } + scanner := bufio.NewScanner(f) + for scanner.Scan() { + args := strings.Fields(scanner.Text()) + if err := scanner.Err(); err != nil { + log.Fatalf("reading %s: %s", ProcMounts, err) + } + dev, mount := args[0], args[1] + if mount == "/" { + continue } - req.reply <- &result + if dev != "tmpfs" && !strings.HasPrefix(dev, "/dev/") { + continue + } + keepdir := mount + "/keep" + if st, err := os.Stat(keepdir); err != nil || !st.IsDir() { + continue + } + // Set the -readonly flag (but only for this volume) + // if the filesystem is mounted readonly. + flagReadonlyWas := deprecated.flagReadonly + for _, fsopt := range strings.Split(args[3], ",") { + if fsopt == "ro" { + deprecated.flagReadonly = true + break + } + if fsopt == "rw" { + break + } + } + if err := vs.Set(keepdir); err != nil { + log.Printf("adding %q: %s", keepdir, err) + } else { + added++ + } + deprecated.flagReadonly = flagReadonlyWas } + return added } -func MakeUnixVolume(root string, serialize bool) (v UnixVolume) { - if serialize { - v = UnixVolume{root, make(chan *IORequest)} - go v.IOHandler() +// A UnixVolume stores and retrieves blocks in a local directory. +type UnixVolume struct { + Root string // path to the volume's root directory + ReadOnly bool + Serialize bool + DirectoryReplication int + StorageClasses []string + + // something to lock during IO, typically a sync.Mutex (or nil + // to skip locking) + locker sync.Locker + + os osWithStats +} + +// DeviceID returns a globally unique ID for the volume's root +// directory, consisting of the filesystem's UUID and the path from +// filesystem root to storage directory, joined by "/". For example, +// the DeviceID for a local directory "/mnt/xvda1/keep" might be +// "fa0b6166-3b55-4994-bd3f-92f4e00a1bb0/keep". +func (v *UnixVolume) DeviceID() string { + giveup := func(f string, args ...interface{}) string { + log.Printf(f+"; using blank DeviceID for volume %s", append(args, v)...) + return "" + } + buf, err := exec.Command("findmnt", "--noheadings", "--target", v.Root).CombinedOutput() + if err != nil { + return giveup("findmnt: %s (%q)", err, buf) + } + findmnt := strings.Fields(string(buf)) + if len(findmnt) < 2 { + return giveup("could not parse findmnt output: %q", buf) + } + fsRoot, dev := findmnt[0], findmnt[1] + + absRoot, err := filepath.Abs(v.Root) + if err != nil { + return giveup("resolving relative path %q: %s", v.Root, err) + } + realRoot, err := filepath.EvalSymlinks(absRoot) + if err != nil { + return giveup("resolving symlinks in %q: %s", absRoot, err) + } + + // Find path from filesystem root to realRoot + var fsPath string + if strings.HasPrefix(realRoot, fsRoot+"/") { + fsPath = realRoot[len(fsRoot):] + } else if fsRoot == "/" { + fsPath = realRoot + } else if fsRoot == realRoot { + fsPath = "" } else { - v = UnixVolume{root, nil} + return giveup("findmnt reports mount point %q which is not a prefix of volume root %q", fsRoot, realRoot) } - return + + if !strings.HasPrefix(dev, "/") { + return giveup("mount %q device %q is not a path", fsRoot, dev) + } + + fi, err := os.Stat(dev) + if err != nil { + return giveup("stat %q: %s\n", dev, err) + } + ino := fi.Sys().(*syscall.Stat_t).Ino + + // Find a symlink in /dev/disk/by-uuid/ whose target is (i.e., + // has the same inode as) the mounted device + udir := "/dev/disk/by-uuid" + d, err := os.Open(udir) + if err != nil { + return giveup("opening %q: %s", udir, err) + } + uuids, err := d.Readdirnames(0) + if err != nil { + return giveup("reading %q: %s", udir, err) + } + for _, uuid := range uuids { + link := filepath.Join(udir, uuid) + fi, err = os.Stat(link) + if err != nil { + log.Printf("error: stat %q: %s", link, err) + continue + } + if fi.Sys().(*syscall.Stat_t).Ino == ino { + return uuid + fsPath + } + } + return giveup("could not find entry in %q matching %q", udir, dev) } -func (v *UnixVolume) Get(loc string) ([]byte, error) { - if v.queue == nil { - return v.Read(loc) +// Examples implements VolumeWithExamples. +func (*UnixVolume) Examples() []Volume { + return []Volume{ + &UnixVolume{ + Root: "/mnt/local-disk", + Serialize: true, + DirectoryReplication: 1, + }, + &UnixVolume{ + Root: "/mnt/network-disk", + Serialize: false, + DirectoryReplication: 2, + }, } - reply := make(chan *IOResponse) - v.queue <- &IORequest{KeepGet, loc, nil, reply} - response := <-reply - return response.data, response.err } -func (v *UnixVolume) Put(loc string, block []byte) error { - if v.queue == nil { - return v.Write(loc, block) +// Type implements Volume +func (v *UnixVolume) Type() string { + return "Directory" +} + +// Start implements Volume +func (v *UnixVolume) Start() error { + if v.Serialize { + v.locker = &sync.Mutex{} + } + if !strings.HasPrefix(v.Root, "/") { + return fmt.Errorf("volume root does not start with '/': %q", v.Root) } - reply := make(chan *IOResponse) - v.queue <- &IORequest{KeepPut, loc, block, reply} - response := <-reply - return response.err + if v.DirectoryReplication == 0 { + v.DirectoryReplication = 1 + } + _, err := v.os.Stat(v.Root) + return err } +// Touch sets the timestamp for the given locator to the current time func (v *UnixVolume) Touch(loc string) error { + if v.ReadOnly { + return MethodDisabledError + } p := v.blockPath(loc) - f, err := os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644) + f, err := v.os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644) if err != nil { return err } defer f.Close() - if e := lockfile(f); e != nil { + if err := v.lock(context.TODO()); err != nil { + return err + } + defer v.unlock() + if e := v.lockfile(f); e != nil { return e } - defer unlockfile(f) - now := time.Now().Unix() - utime := syscall.Utimbuf{now, now} - return syscall.Utime(p, &utime) + defer v.unlockfile(f) + ts := syscall.NsecToTimespec(time.Now().UnixNano()) + v.os.stats.Tick(&v.os.stats.UtimesOps) + err = syscall.UtimesNano(p, []syscall.Timespec{ts, ts}) + v.os.stats.TickErr(err) + return err } +// Mtime returns the stored timestamp for the given locator. func (v *UnixVolume) Mtime(loc string) (time.Time, error) { p := v.blockPath(loc) - if fi, err := os.Stat(p); err != nil { + fi, err := v.os.Stat(p) + if err != nil { return time.Time{}, err - } else { - return fi.ModTime(), nil } + return fi.ModTime(), nil } -// Read retrieves a block identified by the locator string "loc", and -// returns its contents as a byte slice. -// -// If the block could not be opened or read, Read returns a nil slice -// and the os.Error that was generated. -// -// If the block is present but its content hash does not match loc, -// Read returns the block and a CorruptError. It is the caller's -// responsibility to decide what (if anything) to do with the -// corrupted data block. -// -func (v *UnixVolume) Read(loc string) ([]byte, error) { - buf, err := ioutil.ReadFile(v.blockPath(loc)) - return buf, err +// Lock the locker (if one is in use), open the file for reading, and +// call the given function if and when the file is ready to read. +func (v *UnixVolume) getFunc(ctx context.Context, path string, fn func(io.Reader) error) error { + if err := v.lock(ctx); err != nil { + return err + } + defer v.unlock() + f, err := v.os.Open(path) + if err != nil { + return err + } + defer f.Close() + return fn(NewCountingReader(ioutil.NopCloser(f), v.os.stats.TickInBytes)) +} + +// stat is os.Stat() with some extra sanity checks. +func (v *UnixVolume) stat(path string) (os.FileInfo, error) { + stat, err := v.os.Stat(path) + if err == nil { + if stat.Size() < 0 { + err = os.ErrInvalid + } else if stat.Size() > BlockSize { + err = TooLongError + } + } + return stat, err +} + +// Get retrieves a block, copies it to the given slice, and returns +// the number of bytes copied. +func (v *UnixVolume) Get(ctx context.Context, loc string, buf []byte) (int, error) { + return getWithPipe(ctx, loc, buf, v) } -// Write stores a block of data identified by the locator string +// ReadBlock implements BlockReader. +func (v *UnixVolume) ReadBlock(ctx context.Context, loc string, w io.Writer) error { + path := v.blockPath(loc) + stat, err := v.stat(path) + if err != nil { + return v.translateError(err) + } + return v.getFunc(ctx, path, func(rdr io.Reader) error { + n, err := io.Copy(w, rdr) + if err == nil && n != stat.Size() { + err = io.ErrUnexpectedEOF + } + return err + }) +} + +// Compare returns nil if Get(loc) would return the same content as +// expect. It is functionally equivalent to Get() followed by +// bytes.Compare(), but uses less memory. +func (v *UnixVolume) Compare(ctx context.Context, loc string, expect []byte) error { + path := v.blockPath(loc) + if _, err := v.stat(path); err != nil { + return v.translateError(err) + } + return v.getFunc(ctx, path, func(rdr io.Reader) error { + return compareReaderWithBuf(ctx, rdr, expect, loc[:32]) + }) +} + +// Put stores a block of data identified by the locator string // "loc". It returns nil on success. If the volume is full, it // returns a FullError. If the write fails due to some other error, // that error is returned. -// -func (v *UnixVolume) Write(loc string, block []byte) error { +func (v *UnixVolume) Put(ctx context.Context, loc string, block []byte) error { + return putWithPipe(ctx, loc, block, v) +} + +// ReadBlock implements BlockWriter. +func (v *UnixVolume) WriteBlock(ctx context.Context, loc string, rdr io.Reader) error { + if v.ReadOnly { + return MethodDisabledError + } if v.IsFull() { return FullError } @@ -154,46 +354,51 @@ func (v *UnixVolume) Write(loc string, block []byte) error { return err } - tmpfile, tmperr := ioutil.TempFile(bdir, "tmp"+loc) + tmpfile, tmperr := v.os.TempFile(bdir, "tmp"+loc) if tmperr != nil { log.Printf("ioutil.TempFile(%s, tmp%s): %s", bdir, loc, tmperr) return tmperr } + bpath := v.blockPath(loc) - if _, err := tmpfile.Write(block); err != nil { + if err := v.lock(ctx); err != nil { + return err + } + defer v.unlock() + n, err := io.Copy(tmpfile, rdr) + v.os.stats.TickOutBytes(uint64(n)) + if err != nil { log.Printf("%s: writing to %s: %s\n", v, bpath, err) + tmpfile.Close() + v.os.Remove(tmpfile.Name()) return err } if err := tmpfile.Close(); err != nil { log.Printf("closing %s: %s\n", tmpfile.Name(), err) - os.Remove(tmpfile.Name()) + v.os.Remove(tmpfile.Name()) return err } - if err := os.Rename(tmpfile.Name(), bpath); err != nil { + if err := v.os.Rename(tmpfile.Name(), bpath); err != nil { log.Printf("rename %s %s: %s\n", tmpfile.Name(), bpath, err) - os.Remove(tmpfile.Name()) - return err + return v.os.Remove(tmpfile.Name()) } return nil } // Status returns a VolumeStatus struct describing the volume's -// current state. +// current state, or nil if an error occurs. // func (v *UnixVolume) Status() *VolumeStatus { - var fs syscall.Statfs_t - var devnum uint64 - - if fi, err := os.Stat(v.root); err == nil { - devnum = fi.Sys().(*syscall.Stat_t).Dev - } else { + fi, err := v.os.Stat(v.Root) + if err != nil { log.Printf("%s: os.Stat: %s\n", v, err) return nil } + devnum := fi.Sys().(*syscall.Stat_t).Dev - err := syscall.Statfs(v.root, &fs) - if err != nil { + var fs syscall.Statfs_t + if err := syscall.Statfs(v.Root, &fs); err != nil { log.Printf("%s: statfs: %s\n", v, err) return nil } @@ -202,17 +407,24 @@ func (v *UnixVolume) Status() *VolumeStatus { // uses fs.Blocks - fs.Bfree. free := fs.Bavail * uint64(fs.Bsize) used := (fs.Blocks - fs.Bfree) * uint64(fs.Bsize) - return &VolumeStatus{v.root, devnum, free, used} + return &VolumeStatus{ + MountPoint: v.Root, + DeviceNum: devnum, + BytesFree: free, + BytesUsed: used, + } } -// Index returns a list of blocks found on this volume which begin with -// the specified prefix. If the prefix is an empty string, Index returns -// a complete list of blocks. +var blockDirRe = regexp.MustCompile(`^[0-9a-f]+$`) +var blockFileRe = regexp.MustCompile(`^[0-9a-f]{32}$`) + +// IndexTo writes (to the given Writer) a list of blocks found on this +// volume which begin with the specified prefix. If the prefix is an +// empty string, IndexTo writes a complete list of blocks. // -// The return value is a multiline string (separated by -// newlines). Each line is in the format +// Each block is given in the format // -// locator+size modification-time +// locator+size modification-time {newline} // // e.g.: // @@ -220,79 +432,156 @@ func (v *UnixVolume) Status() *VolumeStatus { // e4d41e6fd68460e0e3fc18cc746959d2+67108864 1377796043 // e4de7a2810f5554cd39b36d8ddb132ff+67108864 1388701136 // -func (v *UnixVolume) Index(prefix string) (output string) { - filepath.Walk(v.root, - func(path string, info os.FileInfo, err error) error { - // This WalkFunc inspects each path in the volume - // and prints an index line for all files that begin - // with prefix. - if err != nil { - log.Printf("IndexHandler: %s: walking to %s: %s", - v, path, err) - return nil +func (v *UnixVolume) IndexTo(prefix string, w io.Writer) error { + var lastErr error + rootdir, err := v.os.Open(v.Root) + if err != nil { + return err + } + defer rootdir.Close() + v.os.stats.Tick(&v.os.stats.ReaddirOps) + for { + names, err := rootdir.Readdirnames(1) + if err == io.EOF { + return lastErr + } else if err != nil { + return err + } + if !strings.HasPrefix(names[0], prefix) && !strings.HasPrefix(prefix, names[0]) { + // prefix excludes all blocks stored in this dir + continue + } + if !blockDirRe.MatchString(names[0]) { + continue + } + blockdirpath := filepath.Join(v.Root, names[0]) + blockdir, err := v.os.Open(blockdirpath) + if err != nil { + log.Print("Error reading ", blockdirpath, ": ", err) + lastErr = err + continue + } + v.os.stats.Tick(&v.os.stats.ReaddirOps) + for { + fileInfo, err := blockdir.Readdir(1) + if err == io.EOF { + break + } else if err != nil { + log.Print("Error reading ", blockdirpath, ": ", err) + lastErr = err + break } - locator := filepath.Base(path) - // Skip directories that do not match prefix. - // We know there is nothing interesting inside. - if info.IsDir() && - !strings.HasPrefix(locator, prefix) && - !strings.HasPrefix(prefix, locator) { - return filepath.SkipDir + name := fileInfo[0].Name() + if !strings.HasPrefix(name, prefix) { + continue } - // Skip any file that is not apparently a locator, e.g. .meta files - if !IsValidLocator(locator) { - return nil + if !blockFileRe.MatchString(name) { + continue } - // Print filenames beginning with prefix - if !info.IsDir() && strings.HasPrefix(locator, prefix) { - output = output + fmt.Sprintf( - "%s+%d %d\n", locator, info.Size(), info.ModTime().Unix()) + _, err = fmt.Fprint(w, + name, + "+", fileInfo[0].Size(), + " ", fileInfo[0].ModTime().UnixNano(), + "\n") + if err != nil { + log.Print("Error writing : ", err) + lastErr = err + break } - return nil - }) - - return + } + blockdir.Close() + } } -func (v *UnixVolume) Delete(loc string) error { +// Trash trashes the block data from the unix storage +// If TrashLifetime == 0, the block is deleted +// Else, the block is renamed as path/{loc}.trash.{deadline}, +// where deadline = now + TrashLifetime +func (v *UnixVolume) Trash(loc string) error { // Touch() must be called before calling Write() on a block. Touch() // also uses lockfile(). This avoids a race condition between Write() - // and Delete() because either (a) the file will be deleted and Touch() + // and Trash() because either (a) the file will be trashed and Touch() // will signal to the caller that the file is not present (and needs to // be re-written), or (b) Touch() will update the file's timestamp and - // Delete() will read the correct up-to-date timestamp and choose not to - // delete the file. + // Trash() will read the correct up-to-date timestamp and choose not to + // trash the file. + if v.ReadOnly { + return MethodDisabledError + } + if err := v.lock(context.TODO()); err != nil { + return err + } + defer v.unlock() p := v.blockPath(loc) - f, err := os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644) + f, err := v.os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644) if err != nil { return err } defer f.Close() - if e := lockfile(f); e != nil { + if e := v.lockfile(f); e != nil { return e } - defer unlockfile(f) + defer v.unlockfile(f) + + // If the block has been PUT in the last blobSignatureTTL + // seconds, return success without removing the block. This + // protects data from garbage collection until it is no longer + // possible for clients to retrieve the unreferenced blocks + // anyway (because the permission signatures have expired). + if fi, err := v.os.Stat(p); err != nil { + return err + } else if time.Since(fi.ModTime()) < time.Duration(theConfig.BlobSignatureTTL) { + return nil + } + + if theConfig.TrashLifetime == 0 { + return v.os.Remove(p) + } + return v.os.Rename(p, fmt.Sprintf("%v.trash.%d", p, time.Now().Add(theConfig.TrashLifetime.Duration()).Unix())) +} + +// Untrash moves block from trash back into store +// Look for path/{loc}.trash.{deadline} in storage, +// and rename the first such file as path/{loc} +func (v *UnixVolume) Untrash(loc string) (err error) { + if v.ReadOnly { + return MethodDisabledError + } - // If the block has been PUT more recently than -permission_ttl, - // return success without removing the block. This guards against - // a race condition where a block is old enough that Data Manager - // has added it to the trash list, but the user submitted a PUT - // for the block since then. - if fi, err := os.Stat(p); err != nil { + v.os.stats.Tick(&v.os.stats.ReaddirOps) + files, err := ioutil.ReadDir(v.blockDir(loc)) + if err != nil { return err - } else { - if time.Since(fi.ModTime()) < permission_ttl { - return nil + } + + if len(files) == 0 { + return os.ErrNotExist + } + + foundTrash := false + prefix := fmt.Sprintf("%v.trash.", loc) + for _, f := range files { + if strings.HasPrefix(f.Name(), prefix) { + foundTrash = true + err = v.os.Rename(v.blockPath(f.Name()), v.blockPath(loc)) + if err == nil { + break + } } } - return os.Remove(p) + + if foundTrash == false { + return os.ErrNotExist + } + + return } // blockDir returns the fully qualified directory name for the directory // where loc is (or would be) stored on this volume. func (v *UnixVolume) blockDir(loc string) string { - return filepath.Join(v.root, loc[0:3]) + return filepath.Join(v.Root, loc[0:3]) } // blockPath returns the fully qualified pathname for the path to loc @@ -302,10 +591,10 @@ func (v *UnixVolume) blockPath(loc string) string { } // IsFull returns true if the free space on the volume is less than -// MIN_FREE_KILOBYTES. +// MinFreeKilobytes. // func (v *UnixVolume) IsFull() (isFull bool) { - fullSymlink := v.root + "/full" + fullSymlink := v.Root + "/full" // Check if the volume has been marked as full in the last hour. if link, err := os.Readlink(fullSymlink); err == nil { @@ -318,7 +607,7 @@ func (v *UnixVolume) IsFull() (isFull bool) { } if avail, err := v.FreeDiskSpace(); err == nil { - isFull = avail < MIN_FREE_KILOBYTES + isFull = avail < MinFreeKilobytes } else { log.Printf("%s: FreeDiskSpace: %s\n", v, err) isFull = false @@ -337,7 +626,7 @@ func (v *UnixVolume) IsFull() (isFull bool) { // func (v *UnixVolume) FreeDiskSpace() (free uint64, err error) { var fs syscall.Statfs_t - err = syscall.Statfs(v.root, &fs) + err = syscall.Statfs(v.Root, &fs) if err == nil { // Statfs output is not guaranteed to measure free // space in terms of 1K blocks. @@ -347,14 +636,220 @@ func (v *UnixVolume) FreeDiskSpace() (free uint64, err error) { } func (v *UnixVolume) String() string { - return fmt.Sprintf("[UnixVolume %s]", v.root) + return fmt.Sprintf("[UnixVolume %s]", v.Root) +} + +// Writable returns false if all future Put, Mtime, and Delete calls +// are expected to fail. +func (v *UnixVolume) Writable() bool { + return !v.ReadOnly +} + +// Replication returns the number of replicas promised by the +// underlying device (as specified in configuration). +func (v *UnixVolume) Replication() int { + return v.DirectoryReplication +} + +// GetStorageClasses implements Volume +func (v *UnixVolume) GetStorageClasses() []string { + return v.StorageClasses +} + +// InternalStats returns I/O and filesystem ops counters. +func (v *UnixVolume) InternalStats() interface{} { + return &v.os.stats +} + +// lock acquires the serialize lock, if one is in use. If ctx is done +// before the lock is acquired, lock returns ctx.Err() instead of +// acquiring the lock. +func (v *UnixVolume) lock(ctx context.Context) error { + if v.locker == nil { + return nil + } + locked := make(chan struct{}) + go func() { + v.locker.Lock() + close(locked) + }() + select { + case <-ctx.Done(): + go func() { + <-locked + v.locker.Unlock() + }() + return ctx.Err() + case <-locked: + return nil + } +} + +// unlock releases the serialize lock, if one is in use. +func (v *UnixVolume) unlock() { + if v.locker == nil { + return + } + v.locker.Unlock() } // lockfile and unlockfile use flock(2) to manage kernel file locks. -func lockfile(f *os.File) error { - return syscall.Flock(int(f.Fd()), syscall.LOCK_EX) +func (v *UnixVolume) lockfile(f *os.File) error { + v.os.stats.Tick(&v.os.stats.FlockOps) + err := syscall.Flock(int(f.Fd()), syscall.LOCK_EX) + v.os.stats.TickErr(err) + return err +} + +func (v *UnixVolume) unlockfile(f *os.File) error { + err := syscall.Flock(int(f.Fd()), syscall.LOCK_UN) + v.os.stats.TickErr(err) + return err +} + +// Where appropriate, translate a more specific filesystem error to an +// error recognized by handlers, like os.ErrNotExist. +func (v *UnixVolume) translateError(err error) error { + switch err.(type) { + case *os.PathError: + // stat() returns a PathError if the parent directory + // (not just the file itself) is missing + return os.ErrNotExist + default: + return err + } +} + +var unixTrashLocRegexp = regexp.MustCompile(`/([0-9a-f]{32})\.trash\.(\d+)$`) + +// EmptyTrash walks hierarchy looking for {hash}.trash.* +// and deletes those with deadline < now. +func (v *UnixVolume) EmptyTrash() { + var bytesDeleted, bytesInTrash int64 + var blocksDeleted, blocksInTrash int64 + + doFile := func(path string, info os.FileInfo) { + if info.Mode().IsDir() { + return + } + matches := unixTrashLocRegexp.FindStringSubmatch(path) + if len(matches) != 3 { + return + } + deadline, err := strconv.ParseInt(matches[2], 10, 64) + if err != nil { + log.Printf("EmptyTrash: %v: ParseInt(%v): %v", path, matches[2], err) + return + } + atomic.AddInt64(&bytesInTrash, info.Size()) + atomic.AddInt64(&blocksInTrash, 1) + if deadline > time.Now().Unix() { + return + } + err = v.os.Remove(path) + if err != nil { + log.Printf("EmptyTrash: Remove %v: %v", path, err) + return + } + atomic.AddInt64(&bytesDeleted, info.Size()) + atomic.AddInt64(&blocksDeleted, 1) + } + + type dirent struct { + path string + info os.FileInfo + } + var wg sync.WaitGroup + todo := make(chan dirent, theConfig.EmptyTrashWorkers) + for i := 0; i < 1 || i < theConfig.EmptyTrashWorkers; i++ { + wg.Add(1) + go func() { + defer wg.Done() + for e := range todo { + doFile(e.path, e.info) + } + }() + } + + err := filepath.Walk(v.Root, func(path string, info os.FileInfo, err error) error { + if err != nil { + log.Printf("EmptyTrash: filepath.Walk: %v: %v", path, err) + return nil + } + todo <- dirent{path, info} + return nil + }) + close(todo) + wg.Wait() + + if err != nil { + log.Printf("EmptyTrash error for %v: %v", v.String(), err) + } + + log.Printf("EmptyTrash stats for %v: Deleted %v bytes in %v blocks. Remaining in trash: %v bytes in %v blocks.", v.String(), bytesDeleted, blocksDeleted, bytesInTrash-bytesDeleted, blocksInTrash-blocksDeleted) +} + +type unixStats struct { + statsTicker + OpenOps uint64 + StatOps uint64 + FlockOps uint64 + UtimesOps uint64 + CreateOps uint64 + RenameOps uint64 + UnlinkOps uint64 + ReaddirOps uint64 +} + +func (s *unixStats) TickErr(err error) { + if err == nil { + return + } + s.statsTicker.TickErr(err, fmt.Sprintf("%T", err)) +} + +type osWithStats struct { + stats unixStats +} + +func (o *osWithStats) Open(name string) (*os.File, error) { + o.stats.Tick(&o.stats.OpenOps) + f, err := os.Open(name) + o.stats.TickErr(err) + return f, err +} + +func (o *osWithStats) OpenFile(name string, flag int, perm os.FileMode) (*os.File, error) { + o.stats.Tick(&o.stats.OpenOps) + f, err := os.OpenFile(name, flag, perm) + o.stats.TickErr(err) + return f, err +} + +func (o *osWithStats) Remove(path string) error { + o.stats.Tick(&o.stats.UnlinkOps) + err := os.Remove(path) + o.stats.TickErr(err) + return err +} + +func (o *osWithStats) Rename(a, b string) error { + o.stats.Tick(&o.stats.RenameOps) + err := os.Rename(a, b) + o.stats.TickErr(err) + return err +} + +func (o *osWithStats) Stat(path string) (os.FileInfo, error) { + o.stats.Tick(&o.stats.StatOps) + fi, err := os.Stat(path) + o.stats.TickErr(err) + return fi, err } -func unlockfile(f *os.File) error { - return syscall.Flock(int(f.Fd()), syscall.LOCK_UN) +func (o *osWithStats) TempFile(dir, base string) (*os.File, error) { + o.stats.Tick(&o.stats.CreateOps) + f, err := ioutil.TempFile(dir, base) + o.stats.TickErr(err) + return f, err }