Refactor the multi-host salt install page.
[arvados.git] / services / keepstore / unix_volume.go
index 96f458720d38b56b97fa51fd63e76faa798987bf..dd62cf1319318fb4f5dbe52869311bb55b6b8008 100644 (file)
@@ -2,12 +2,12 @@
 //
 // SPDX-License-Identifier: AGPL-3.0
 
-package main
+package keepstore
 
 import (
-       "bufio"
        "context"
-       "flag"
+       "encoding/json"
+       "errors"
        "fmt"
        "io"
        "io/ioutil"
@@ -22,98 +22,53 @@ import (
        "syscall"
        "time"
 
+       "git.arvados.org/arvados.git/sdk/go/arvados"
        "github.com/prometheus/client_golang/prometheus"
+       "github.com/sirupsen/logrus"
 )
 
-type unixVolumeAdder struct {
-       *Config
-}
-
-// String implements flag.Value
-func (vs *unixVolumeAdder) String() string {
-       return "-"
+func init() {
+       driver["Directory"] = newDirectoryVolume
 }
 
-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
+func newDirectoryVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) {
+       v := &UnixVolume{cluster: cluster, volume: volume, logger: logger, metrics: metrics}
+       err := json.Unmarshal(volume.DriverParameters, &v)
+       if err != nil {
+               return nil, err
        }
-       vs.Config.Volumes = append(vs.Config.Volumes, &UnixVolume{
-               Root:      path,
-               ReadOnly:  deprecated.flagReadonly,
-               Serialize: deprecated.flagSerializeIO,
-       })
-       return nil
+       v.logger = v.logger.WithField("Volume", v.String())
+       return v, v.check()
 }
 
-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")
-}
-
-// 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)
+func (v *UnixVolume) check() error {
+       if v.Root == "" {
+               return errors.New("DriverParameters.Root was not provided")
        }
-       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
-               }
-               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
+       if v.Serialize {
+               v.locker = &sync.Mutex{}
        }
-       return added
+       if !strings.HasPrefix(v.Root, "/") {
+               return fmt.Errorf("DriverParameters.Root %q does not start with '/'", v.Root)
+       }
+
+       // Set up prometheus metrics
+       lbls := prometheus.Labels{"device_id": v.GetDeviceID()}
+       v.os.stats.opsCounters, v.os.stats.errCounters, v.os.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
+
+       _, err := v.os.Stat(v.Root)
+       return err
 }
 
 // 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
+       Root      string // path to the volume's root directory
+       Serialize bool
+
+       cluster *arvados.Cluster
+       volume  arvados.Volume
+       logger  logrus.FieldLogger
+       metrics *volumeMetricsVecs
 
        // something to lock during IO, typically a sync.Mutex (or nil
        // to skip locking)
@@ -122,14 +77,14 @@ type UnixVolume struct {
        os osWithStats
 }
 
-// DeviceID returns a globally unique ID for the volume's root
+// GetDeviceID 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
+// the device ID for a local directory "/mnt/xvda1/keep" might be
 // "fa0b6166-3b55-4994-bd3f-92f4e00a1bb0/keep".
-func (v *UnixVolume) DeviceID() string {
+func (v *UnixVolume) GetDeviceID() string {
        giveup := func(f string, args ...interface{}) string {
-               log.Printf(f+"; using blank DeviceID for volume %s", append(args, v)...)
+               v.logger.Infof(f+"; using blank DeviceID for volume %s", append(args, v)...)
                return ""
        }
        buf, err := exec.Command("findmnt", "--noheadings", "--target", v.Root).CombinedOutput()
@@ -169,7 +124,7 @@ func (v *UnixVolume) DeviceID() string {
 
        fi, err := os.Stat(dev)
        if err != nil {
-               return giveup("stat %q: %s\n", dev, err)
+               return giveup("stat %q: %s", dev, err)
        }
        ino := fi.Sys().(*syscall.Stat_t).Ino
 
@@ -180,6 +135,7 @@ func (v *UnixVolume) DeviceID() string {
        if err != nil {
                return giveup("opening %q: %s", udir, err)
        }
+       defer d.Close()
        uuids, err := d.Readdirnames(0)
        if err != nil {
                return giveup("reading %q: %s", udir, err)
@@ -188,7 +144,7 @@ func (v *UnixVolume) DeviceID() string {
                link := filepath.Join(udir, uuid)
                fi, err = os.Stat(link)
                if err != nil {
-                       log.Printf("error: stat %q: %s", link, err)
+                       v.logger.WithError(err).Errorf("stat(%q) failed", link)
                        continue
                }
                if fi.Sys().(*syscall.Stat_t).Ino == ino {
@@ -198,50 +154,9 @@ func (v *UnixVolume) DeviceID() string {
        return giveup("could not find entry in %q matching %q", udir, dev)
 }
 
-// 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,
-               },
-       }
-}
-
-// Type implements Volume
-func (v *UnixVolume) Type() string {
-       return "Directory"
-}
-
-// Start implements Volume
-func (v *UnixVolume) Start(vm *volumeMetricsVecs) 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)
-       }
-       if v.DirectoryReplication == 0 {
-               v.DirectoryReplication = 1
-       }
-       // Set up prometheus metrics
-       lbls := prometheus.Labels{"device_id": v.DeviceID()}
-       v.os.stats.opsCounters, v.os.stats.errCounters, v.os.stats.ioBytes = vm.getCounterVecsFor(lbls)
-
-       _, 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 {
+       if v.volume.ReadOnly {
                return MethodDisabledError
        }
        p := v.blockPath(loc)
@@ -258,10 +173,10 @@ func (v *UnixVolume) Touch(loc string) error {
                return e
        }
        defer v.unlockfile(f)
-       ts := syscall.NsecToTimespec(time.Now().UnixNano())
+       ts := time.Now()
        v.os.stats.TickOps("utimes")
        v.os.stats.Tick(&v.os.stats.UtimesOps)
-       err = syscall.UtimesNano(p, []syscall.Timespec{ts, ts})
+       err = os.Chtimes(p, ts, ts)
        v.os.stats.TickErr(err)
        return err
 }
@@ -349,7 +264,7 @@ func (v *UnixVolume) Put(ctx context.Context, loc string, block []byte) error {
 
 // WriteBlock implements BlockWriter.
 func (v *UnixVolume) WriteBlock(ctx context.Context, loc string, rdr io.Reader) error {
-       if v.ReadOnly {
+       if v.volume.ReadOnly {
                return MethodDisabledError
        }
        if v.IsFull() {
@@ -357,39 +272,42 @@ func (v *UnixVolume) WriteBlock(ctx context.Context, loc string, rdr io.Reader)
        }
        bdir := v.blockDir(loc)
        if err := os.MkdirAll(bdir, 0755); err != nil {
-               log.Printf("%s: could not create directory %s: %s",
-                       loc, bdir, err)
-               return err
-       }
-
-       tmpfile, tmperr := v.os.TempFile(bdir, "tmp"+loc)
-       if tmperr != nil {
-               log.Printf("ioutil.TempFile(%s, tmp%s): %s", bdir, loc, tmperr)
-               return tmperr
+               return fmt.Errorf("error creating directory %s: %s", bdir, err)
        }
 
        bpath := v.blockPath(loc)
+       tmpfile, err := v.os.TempFile(bdir, "tmp"+loc)
+       if err != nil {
+               return fmt.Errorf("TempFile(%s, tmp%s) failed: %s", bdir, loc, err)
+       }
+       defer v.os.Remove(tmpfile.Name())
+       defer tmpfile.Close()
 
-       if err := v.lock(ctx); 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
+               return fmt.Errorf("error writing %s: %s", bpath, err)
        }
-       if err := tmpfile.Close(); err != nil {
-               log.Printf("closing %s: %s\n", tmpfile.Name(), err)
-               v.os.Remove(tmpfile.Name())
-               return err
+       if err = tmpfile.Close(); err != nil {
+               return fmt.Errorf("error closing %s: %s", tmpfile.Name(), err)
+       }
+       // ext4 uses a low-precision clock and effectively backdates
+       // files by up to 10 ms, sometimes across a 1-second boundary,
+       // which produces confusing results in logs and tests.  We
+       // avoid this by setting the output file's timestamps
+       // explicitly, using a higher resolution clock.
+       ts := time.Now()
+       v.os.stats.TickOps("utimes")
+       v.os.stats.Tick(&v.os.stats.UtimesOps)
+       if err = os.Chtimes(tmpfile.Name(), ts, ts); err != nil {
+               return fmt.Errorf("error setting timestamps on %s: %s", tmpfile.Name(), err)
        }
-       if err := v.os.Rename(tmpfile.Name(), bpath); err != nil {
-               log.Printf("rename %s %s: %s\n", tmpfile.Name(), bpath, err)
-               return v.os.Remove(tmpfile.Name())
+       if err = v.os.Rename(tmpfile.Name(), bpath); err != nil {
+               return fmt.Errorf("error renaming %s to %s: %s", tmpfile.Name(), bpath, err)
        }
        return nil
 }
@@ -400,14 +318,19 @@ func (v *UnixVolume) WriteBlock(ctx context.Context, loc string, rdr io.Reader)
 func (v *UnixVolume) Status() *VolumeStatus {
        fi, err := v.os.Stat(v.Root)
        if err != nil {
-               log.Printf("%s: os.Stat: %s\n", v, err)
+               v.logger.WithError(err).Error("stat failed")
                return nil
        }
-       devnum := fi.Sys().(*syscall.Stat_t).Dev
+       // uint64() cast here supports GOOS=darwin where Dev is
+       // int32. If the device number is negative, the unsigned
+       // devnum won't be the real device number any more, but that's
+       // fine -- all we care about is getting the same number each
+       // time.
+       devnum := uint64(fi.Sys().(*syscall.Stat_t).Dev)
 
        var fs syscall.Statfs_t
        if err := syscall.Statfs(v.Root, &fs); err != nil {
-               log.Printf("%s: statfs: %s\n", v, err)
+               v.logger.WithError(err).Error("statfs failed")
                return nil
        }
        // These calculations match the way df calculates disk usage:
@@ -441,47 +364,55 @@ var blockFileRe = regexp.MustCompile(`^[0-9a-f]{32}$`)
 //     e4de7a2810f5554cd39b36d8ddb132ff+67108864 1388701136
 //
 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.TickOps("readdir")
        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]) {
+       subdirs, err := rootdir.Readdirnames(-1)
+       rootdir.Close()
+       if err != nil {
+               return err
+       }
+       for _, subdir := range subdirs {
+               if !strings.HasPrefix(subdir, prefix) && !strings.HasPrefix(prefix, subdir) {
                        // 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
+               if !blockDirRe.MatchString(subdir) {
                        continue
                }
-               v.os.stats.TickOps("readdir")
-               v.os.stats.Tick(&v.os.stats.ReaddirOps)
-               for {
-                       fileInfo, err := blockdir.Readdir(1)
-                       if err == io.EOF {
+               blockdirpath := filepath.Join(v.Root, subdir)
+
+               var dirents []os.DirEntry
+               for attempt := 0; ; attempt++ {
+                       v.os.stats.TickOps("readdir")
+                       v.os.stats.Tick(&v.os.stats.ReaddirOps)
+                       dirents, err = os.ReadDir(blockdirpath)
+                       if err == nil {
                                break
+                       } else if attempt < 5 && strings.Contains(err.Error(), "errno 523") {
+                               // EBADCOOKIE (NFS stopped accepting
+                               // our readdirent cookie) -- retry a
+                               // few times before giving up
+                               v.logger.WithError(err).Printf("retry after error reading %s", blockdirpath)
+                               continue
+                       } else {
+                               return err
+                       }
+               }
+
+               for _, dirent := range dirents {
+                       fileInfo, err := dirent.Info()
+                       if os.IsNotExist(err) {
+                               // File disappeared between ReadDir() and now
+                               continue
                        } else if err != nil {
-                               log.Print("Error reading ", blockdirpath, ": ", err)
-                               lastErr = err
-                               break
+                               v.logger.WithError(err).Errorf("error getting FileInfo for %q in %q", dirent.Name(), blockdirpath)
+                               return err
                        }
-                       name := fileInfo[0].Name()
+                       name := fileInfo.Name()
                        if !strings.HasPrefix(name, prefix) {
                                continue
                        }
@@ -490,23 +421,21 @@ func (v *UnixVolume) IndexTo(prefix string, w io.Writer) error {
                        }
                        _, err = fmt.Fprint(w,
                                name,
-                               "+", fileInfo[0].Size(),
-                               " ", fileInfo[0].ModTime().UnixNano(),
+                               "+", fileInfo.Size(),
+                               " ", fileInfo.ModTime().UnixNano(),
                                "\n")
                        if err != nil {
-                               log.Print("Error writing : ", err)
-                               lastErr = err
-                               break
+                               return fmt.Errorf("error writing: %s", err)
                        }
                }
-               blockdir.Close()
        }
+       return nil
 }
 
 // Trash trashes the block data from the unix storage
-// If TrashLifetime == 0, the block is deleted
+// If BlobTrashLifetime == 0, the block is deleted
 // Else, the block is renamed as path/{loc}.trash.{deadline},
-// where deadline = now + TrashLifetime
+// where deadline = now + BlobTrashLifetime
 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()
@@ -516,7 +445,7 @@ func (v *UnixVolume) Trash(loc string) error {
        // Trash() will read the correct up-to-date timestamp and choose not to
        // trash the file.
 
-       if v.ReadOnly {
+       if v.volume.ReadOnly || !v.cluster.Collections.BlobTrash {
                return MethodDisabledError
        }
        if err := v.lock(context.TODO()); err != nil {
@@ -541,21 +470,21 @@ func (v *UnixVolume) Trash(loc string) error {
        // 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) {
+       } else if time.Since(fi.ModTime()) < v.cluster.Collections.BlobSigningTTL.Duration() {
                return nil
        }
 
-       if theConfig.TrashLifetime == 0 {
+       if v.cluster.Collections.BlobTrashLifetime == 0 {
                return v.os.Remove(p)
        }
-       return v.os.Rename(p, fmt.Sprintf("%v.trash.%d", p, time.Now().Add(theConfig.TrashLifetime.Duration()).Unix()))
+       return v.os.Rename(p, fmt.Sprintf("%v.trash.%d", p, time.Now().Add(v.cluster.Collections.BlobTrashLifetime.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 {
+       if v.volume.ReadOnly {
                return MethodDisabledError
        }
 
@@ -620,7 +549,7 @@ func (v *UnixVolume) IsFull() (isFull bool) {
        if avail, err := v.FreeDiskSpace(); err == nil {
                isFull = avail < MinFreeKilobytes
        } else {
-               log.Printf("%s: FreeDiskSpace: %s\n", v, err)
+               v.logger.WithError(err).Errorf("%s: FreeDiskSpace failed", v)
                isFull = false
        }
 
@@ -650,23 +579,6 @@ func (v *UnixVolume) String() string {
        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
@@ -679,6 +591,7 @@ func (v *UnixVolume) lock(ctx context.Context) error {
        if v.locker == nil {
                return nil
        }
+       t0 := time.Now()
        locked := make(chan struct{})
        go func() {
                v.locker.Lock()
@@ -686,6 +599,7 @@ func (v *UnixVolume) lock(ctx context.Context) error {
        }()
        select {
        case <-ctx.Done():
+               v.logger.Infof("client hung up while waiting for Serialize lock (%s)", time.Since(t0))
                go func() {
                        <-locked
                        v.locker.Unlock()
@@ -737,6 +651,10 @@ 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() {
+       if v.cluster.Collections.BlobDeleteConcurrency < 1 {
+               return
+       }
+
        var bytesDeleted, bytesInTrash int64
        var blocksDeleted, blocksInTrash int64
 
@@ -750,7 +668,7 @@ func (v *UnixVolume) EmptyTrash() {
                }
                deadline, err := strconv.ParseInt(matches[2], 10, 64)
                if err != nil {
-                       log.Printf("EmptyTrash: %v: ParseInt(%v): %v", path, matches[2], err)
+                       v.logger.WithError(err).Errorf("EmptyTrash: %v: ParseInt(%q) failed", path, matches[2])
                        return
                }
                atomic.AddInt64(&bytesInTrash, info.Size())
@@ -760,7 +678,7 @@ func (v *UnixVolume) EmptyTrash() {
                }
                err = v.os.Remove(path)
                if err != nil {
-                       log.Printf("EmptyTrash: Remove %v: %v", path, err)
+                       v.logger.WithError(err).Errorf("EmptyTrash: Remove(%q) failed", path)
                        return
                }
                atomic.AddInt64(&bytesDeleted, info.Size())
@@ -772,8 +690,8 @@ func (v *UnixVolume) EmptyTrash() {
                info os.FileInfo
        }
        var wg sync.WaitGroup
-       todo := make(chan dirent, theConfig.EmptyTrashWorkers)
-       for i := 0; i < 1 || i < theConfig.EmptyTrashWorkers; i++ {
+       todo := make(chan dirent, v.cluster.Collections.BlobDeleteConcurrency)
+       for i := 0; i < v.cluster.Collections.BlobDeleteConcurrency; i++ {
                wg.Add(1)
                go func() {
                        defer wg.Done()
@@ -785,20 +703,30 @@ func (v *UnixVolume) EmptyTrash() {
 
        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)
+                       v.logger.WithError(err).Errorf("EmptyTrash: filepath.Walk(%q) failed", path)
+                       // Don't give up -- keep walking other
+                       // files/dirs
+                       return nil
+               } else if !info.Mode().IsDir() {
+                       todo <- dirent{path, info}
                        return nil
+               } else if path == v.Root || blockDirRe.MatchString(info.Name()) {
+                       // Descend into a directory that we might have
+                       // put trash in.
+                       return nil
+               } else {
+                       // Don't descend into other dirs.
+                       return filepath.SkipDir
                }
-               todo <- dirent{path, info}
-               return nil
        })
        close(todo)
        wg.Wait()
 
        if err != nil {
-               log.Printf("EmptyTrash error for %v: %v", v.String(), err)
+               v.logger.WithError(err).Error("EmptyTrash failed")
        }
 
-       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)
+       v.logger.Infof("EmptyTrash stats: Deleted %v bytes in %v blocks. Remaining in trash: %v bytes in %v blocks.", bytesDeleted, blocksDeleted, bytesInTrash-bytesDeleted, blocksInTrash-blocksDeleted)
 }
 
 type unixStats struct {