2960: Refactor keepstore into a streaming server.
[arvados.git] / services / keepstore / unix_volume.go
1 // Copyright (C) The Arvados Authors. All rights reserved.
2 //
3 // SPDX-License-Identifier: AGPL-3.0
4
5 package keepstore
6
7 import (
8         "context"
9         "encoding/json"
10         "errors"
11         "fmt"
12         "io"
13         "io/ioutil"
14         "os"
15         "os/exec"
16         "path/filepath"
17         "regexp"
18         "strconv"
19         "strings"
20         "sync"
21         "sync/atomic"
22         "syscall"
23         "time"
24
25         "git.arvados.org/arvados.git/sdk/go/arvados"
26         "github.com/prometheus/client_golang/prometheus"
27         "github.com/sirupsen/logrus"
28 )
29
30 func init() {
31         driver["Directory"] = newUnixVolume
32 }
33
34 func newUnixVolume(params newVolumeParams) (volume, error) {
35         v := &UnixVolume{
36                 uuid:    params.UUID,
37                 cluster: params.Cluster,
38                 volume:  params.ConfigVolume,
39                 logger:  params.Logger,
40                 metrics: params.MetricsVecs,
41         }
42         err := json.Unmarshal(params.ConfigVolume.DriverParameters, &v)
43         if err != nil {
44                 return nil, err
45         }
46         v.logger = v.logger.WithField("Volume", v.DeviceID())
47         return v, v.check()
48 }
49
50 func (v *UnixVolume) check() error {
51         if v.Root == "" {
52                 return errors.New("DriverParameters.Root was not provided")
53         }
54         if v.Serialize {
55                 v.locker = &sync.Mutex{}
56         }
57         if !strings.HasPrefix(v.Root, "/") {
58                 return fmt.Errorf("DriverParameters.Root %q does not start with '/'", v.Root)
59         }
60
61         // Set up prometheus metrics
62         lbls := prometheus.Labels{"device_id": v.DeviceID()}
63         v.os.stats.opsCounters, v.os.stats.errCounters, v.os.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
64
65         _, err := v.os.Stat(v.Root)
66         return err
67 }
68
69 // A UnixVolume stores and retrieves blocks in a local directory.
70 type UnixVolume struct {
71         Root      string // path to the volume's root directory
72         Serialize bool
73
74         uuid    string
75         cluster *arvados.Cluster
76         volume  arvados.Volume
77         logger  logrus.FieldLogger
78         metrics *volumeMetricsVecs
79
80         // something to lock during IO, typically a sync.Mutex (or nil
81         // to skip locking)
82         locker sync.Locker
83
84         os osWithStats
85 }
86
87 // DeviceID returns a globally unique ID for the volume's root
88 // directory, consisting of the filesystem's UUID and the path from
89 // filesystem root to storage directory, joined by "/". For example,
90 // the device ID for a local directory "/mnt/xvda1/keep" might be
91 // "fa0b6166-3b55-4994-bd3f-92f4e00a1bb0/keep".
92 func (v *UnixVolume) DeviceID() string {
93         giveup := func(f string, args ...interface{}) string {
94                 v.logger.Infof(f+"; using hostname:path for volume %s", append(args, v.uuid)...)
95                 host, _ := os.Hostname()
96                 return host + ":" + v.Root
97         }
98         buf, err := exec.Command("findmnt", "--noheadings", "--target", v.Root).CombinedOutput()
99         if err != nil {
100                 return giveup("findmnt: %s (%q)", err, buf)
101         }
102         findmnt := strings.Fields(string(buf))
103         if len(findmnt) < 2 {
104                 return giveup("could not parse findmnt output: %q", buf)
105         }
106         fsRoot, dev := findmnt[0], findmnt[1]
107
108         absRoot, err := filepath.Abs(v.Root)
109         if err != nil {
110                 return giveup("resolving relative path %q: %s", v.Root, err)
111         }
112         realRoot, err := filepath.EvalSymlinks(absRoot)
113         if err != nil {
114                 return giveup("resolving symlinks in %q: %s", absRoot, err)
115         }
116
117         // Find path from filesystem root to realRoot
118         var fsPath string
119         if strings.HasPrefix(realRoot, fsRoot+"/") {
120                 fsPath = realRoot[len(fsRoot):]
121         } else if fsRoot == "/" {
122                 fsPath = realRoot
123         } else if fsRoot == realRoot {
124                 fsPath = ""
125         } else {
126                 return giveup("findmnt reports mount point %q which is not a prefix of volume root %q", fsRoot, realRoot)
127         }
128
129         if !strings.HasPrefix(dev, "/") {
130                 return giveup("mount %q device %q is not a path", fsRoot, dev)
131         }
132
133         fi, err := os.Stat(dev)
134         if err != nil {
135                 return giveup("stat %q: %s", dev, err)
136         }
137         ino := fi.Sys().(*syscall.Stat_t).Ino
138
139         // Find a symlink in /dev/disk/by-uuid/ whose target is (i.e.,
140         // has the same inode as) the mounted device
141         udir := "/dev/disk/by-uuid"
142         d, err := os.Open(udir)
143         if err != nil {
144                 return giveup("opening %q: %s", udir, err)
145         }
146         defer d.Close()
147         uuids, err := d.Readdirnames(0)
148         if err != nil {
149                 return giveup("reading %q: %s", udir, err)
150         }
151         for _, uuid := range uuids {
152                 link := filepath.Join(udir, uuid)
153                 fi, err = os.Stat(link)
154                 if err != nil {
155                         v.logger.WithError(err).Errorf("stat(%q) failed", link)
156                         continue
157                 }
158                 if fi.Sys().(*syscall.Stat_t).Ino == ino {
159                         return uuid + fsPath
160                 }
161         }
162         return giveup("could not find entry in %q matching %q", udir, dev)
163 }
164
165 // BlockTouch sets the timestamp for the given locator to the current time
166 func (v *UnixVolume) BlockTouch(hash string) error {
167         p := v.blockPath(hash)
168         f, err := v.os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644)
169         if err != nil {
170                 return err
171         }
172         defer f.Close()
173         if err := v.lock(context.TODO()); err != nil {
174                 return err
175         }
176         defer v.unlock()
177         if e := v.lockfile(f); e != nil {
178                 return e
179         }
180         defer v.unlockfile(f)
181         ts := time.Now()
182         v.os.stats.TickOps("utimes")
183         v.os.stats.Tick(&v.os.stats.UtimesOps)
184         err = os.Chtimes(p, ts, ts)
185         v.os.stats.TickErr(err)
186         return err
187 }
188
189 // Mtime returns the stored timestamp for the given locator.
190 func (v *UnixVolume) Mtime(loc string) (time.Time, error) {
191         p := v.blockPath(loc)
192         fi, err := v.os.Stat(p)
193         if err != nil {
194                 return time.Time{}, err
195         }
196         return fi.ModTime(), nil
197 }
198
199 // Lock the locker (if one is in use), open the file for reading, and
200 // call the given function if and when the file is ready to read.
201 func (v *UnixVolume) getFunc(ctx context.Context, path string, fn func(io.Reader) error) error {
202         if err := v.lock(ctx); err != nil {
203                 return err
204         }
205         defer v.unlock()
206         f, err := v.os.Open(path)
207         if err != nil {
208                 return err
209         }
210         defer f.Close()
211         return fn(newCountingReader(ioutil.NopCloser(f), v.os.stats.TickInBytes))
212 }
213
214 // stat is os.Stat() with some extra sanity checks.
215 func (v *UnixVolume) stat(path string) (os.FileInfo, error) {
216         stat, err := v.os.Stat(path)
217         if err == nil {
218                 if stat.Size() < 0 {
219                         err = os.ErrInvalid
220                 } else if stat.Size() > BlockSize {
221                         err = errTooLarge
222                 }
223         }
224         return stat, err
225 }
226
227 // BlockRead reads a block from the volume.
228 func (v *UnixVolume) BlockRead(ctx context.Context, hash string, w io.Writer) (int, error) {
229         path := v.blockPath(hash)
230         stat, err := v.stat(path)
231         if err != nil {
232                 return 0, v.translateError(err)
233         }
234         var n int64
235         err = v.getFunc(ctx, path, func(rdr io.Reader) error {
236                 n, err = io.Copy(w, rdr)
237                 if err == nil && n != stat.Size() {
238                         err = io.ErrUnexpectedEOF
239                 }
240                 return err
241         })
242         return int(n), err
243 }
244
245 // BlockWrite stores a block on the volume. If it already exists, its
246 // timestamp is updated.
247 func (v *UnixVolume) BlockWrite(ctx context.Context, hash string, data []byte) error {
248         if v.isFull() {
249                 return errFull
250         }
251         bdir := v.blockDir(hash)
252         if err := os.MkdirAll(bdir, 0755); err != nil {
253                 return fmt.Errorf("error creating directory %s: %s", bdir, err)
254         }
255
256         bpath := v.blockPath(hash)
257         tmpfile, err := v.os.TempFile(bdir, "tmp"+hash)
258         if err != nil {
259                 return fmt.Errorf("TempFile(%s, tmp%s) failed: %s", bdir, hash, err)
260         }
261         defer v.os.Remove(tmpfile.Name())
262         defer tmpfile.Close()
263
264         if err = v.lock(ctx); err != nil {
265                 return err
266         }
267         defer v.unlock()
268         n, err := tmpfile.Write(data)
269         v.os.stats.TickOutBytes(uint64(n))
270         if err != nil {
271                 return fmt.Errorf("error writing %s: %s", bpath, err)
272         }
273         if err = tmpfile.Close(); err != nil {
274                 return fmt.Errorf("error closing %s: %s", tmpfile.Name(), err)
275         }
276         // ext4 uses a low-precision clock and effectively backdates
277         // files by up to 10 ms, sometimes across a 1-second boundary,
278         // which produces confusing results in logs and tests.  We
279         // avoid this by setting the output file's timestamps
280         // explicitly, using a higher resolution clock.
281         ts := time.Now()
282         v.os.stats.TickOps("utimes")
283         v.os.stats.Tick(&v.os.stats.UtimesOps)
284         if err = os.Chtimes(tmpfile.Name(), ts, ts); err != nil {
285                 return fmt.Errorf("error setting timestamps on %s: %s", tmpfile.Name(), err)
286         }
287         if err = v.os.Rename(tmpfile.Name(), bpath); err != nil {
288                 return fmt.Errorf("error renaming %s to %s: %s", tmpfile.Name(), bpath, err)
289         }
290         return nil
291 }
292
293 var blockDirRe = regexp.MustCompile(`^[0-9a-f]+$`)
294 var blockFileRe = regexp.MustCompile(`^[0-9a-f]{32}$`)
295
296 func (v *UnixVolume) Index(ctx context.Context, prefix string, w io.Writer) error {
297         rootdir, err := v.os.Open(v.Root)
298         if err != nil {
299                 return err
300         }
301         v.os.stats.TickOps("readdir")
302         v.os.stats.Tick(&v.os.stats.ReaddirOps)
303         subdirs, err := rootdir.Readdirnames(-1)
304         rootdir.Close()
305         if err != nil {
306                 return err
307         }
308         for _, subdir := range subdirs {
309                 if ctx.Err() != nil {
310                         return ctx.Err()
311                 }
312                 if !strings.HasPrefix(subdir, prefix) && !strings.HasPrefix(prefix, subdir) {
313                         // prefix excludes all blocks stored in this dir
314                         continue
315                 }
316                 if !blockDirRe.MatchString(subdir) {
317                         continue
318                 }
319                 blockdirpath := filepath.Join(v.Root, subdir)
320
321                 var dirents []os.DirEntry
322                 for attempt := 0; ; attempt++ {
323                         v.os.stats.TickOps("readdir")
324                         v.os.stats.Tick(&v.os.stats.ReaddirOps)
325                         dirents, err = os.ReadDir(blockdirpath)
326                         if ctx.Err() != nil {
327                                 return ctx.Err()
328                         } else if err == nil {
329                                 break
330                         } else if attempt < 5 && strings.Contains(err.Error(), "errno 523") {
331                                 // EBADCOOKIE (NFS stopped accepting
332                                 // our readdirent cookie) -- retry a
333                                 // few times before giving up
334                                 v.logger.WithError(err).Printf("retry after error reading %s", blockdirpath)
335                                 continue
336                         } else {
337                                 return err
338                         }
339                 }
340
341                 for _, dirent := range dirents {
342                         if ctx.Err() != nil {
343                                 return ctx.Err()
344                         }
345                         fileInfo, err := dirent.Info()
346                         if os.IsNotExist(err) {
347                                 // File disappeared between ReadDir() and now
348                                 continue
349                         } else if err != nil {
350                                 v.logger.WithError(err).Errorf("error getting FileInfo for %q in %q", dirent.Name(), blockdirpath)
351                                 return err
352                         }
353                         name := fileInfo.Name()
354                         if !strings.HasPrefix(name, prefix) {
355                                 continue
356                         }
357                         if !blockFileRe.MatchString(name) {
358                                 continue
359                         }
360                         _, err = fmt.Fprint(w,
361                                 name,
362                                 "+", fileInfo.Size(),
363                                 " ", fileInfo.ModTime().UnixNano(),
364                                 "\n")
365                         if err != nil {
366                                 return fmt.Errorf("error writing: %s", err)
367                         }
368                 }
369         }
370         return nil
371 }
372
373 // BlockTrash trashes the block data from the unix storage.  If
374 // BlobTrashLifetime == 0, the block is deleted; otherwise, the block
375 // is renamed as path/{loc}.trash.{deadline}, where deadline = now +
376 // BlobTrashLifetime.
377 func (v *UnixVolume) BlockTrash(loc string) error {
378         // Touch() must be called before calling Write() on a block.  Touch()
379         // also uses lockfile().  This avoids a race condition between Write()
380         // and Trash() because either (a) the file will be trashed and Touch()
381         // will signal to the caller that the file is not present (and needs to
382         // be re-written), or (b) Touch() will update the file's timestamp and
383         // Trash() will read the correct up-to-date timestamp and choose not to
384         // trash the file.
385         if err := v.lock(context.TODO()); err != nil {
386                 return err
387         }
388         defer v.unlock()
389         p := v.blockPath(loc)
390         f, err := v.os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644)
391         if err != nil {
392                 return err
393         }
394         defer f.Close()
395         if e := v.lockfile(f); e != nil {
396                 return e
397         }
398         defer v.unlockfile(f)
399
400         // If the block has been PUT in the last blobSignatureTTL
401         // seconds, return success without removing the block. This
402         // protects data from garbage collection until it is no longer
403         // possible for clients to retrieve the unreferenced blocks
404         // anyway (because the permission signatures have expired).
405         if fi, err := v.os.Stat(p); err != nil {
406                 return err
407         } else if time.Since(fi.ModTime()) < v.cluster.Collections.BlobSigningTTL.Duration() {
408                 return nil
409         }
410
411         if v.cluster.Collections.BlobTrashLifetime == 0 {
412                 return v.os.Remove(p)
413         }
414         return v.os.Rename(p, fmt.Sprintf("%v.trash.%d", p, time.Now().Add(v.cluster.Collections.BlobTrashLifetime.Duration()).Unix()))
415 }
416
417 // BlockUntrash moves block from trash back into store
418 // Look for path/{loc}.trash.{deadline} in storage,
419 // and rename the first such file as path/{loc}
420 func (v *UnixVolume) BlockUntrash(hash string) error {
421         v.os.stats.TickOps("readdir")
422         v.os.stats.Tick(&v.os.stats.ReaddirOps)
423         files, err := ioutil.ReadDir(v.blockDir(hash))
424         if err != nil {
425                 return err
426         }
427
428         if len(files) == 0 {
429                 return os.ErrNotExist
430         }
431
432         foundTrash := false
433         prefix := fmt.Sprintf("%v.trash.", hash)
434         for _, f := range files {
435                 if strings.HasPrefix(f.Name(), prefix) {
436                         foundTrash = true
437                         err = v.os.Rename(v.blockPath(f.Name()), v.blockPath(hash))
438                         if err == nil {
439                                 break
440                         }
441                 }
442         }
443
444         if foundTrash == false {
445                 return os.ErrNotExist
446         }
447
448         return nil
449 }
450
451 // blockDir returns the fully qualified directory name for the directory
452 // where loc is (or would be) stored on this volume.
453 func (v *UnixVolume) blockDir(loc string) string {
454         return filepath.Join(v.Root, loc[0:3])
455 }
456
457 // blockPath returns the fully qualified pathname for the path to loc
458 // on this volume.
459 func (v *UnixVolume) blockPath(loc string) string {
460         return filepath.Join(v.blockDir(loc), loc)
461 }
462
463 // isFull returns true if the free space on the volume is less than
464 // MinFreeKilobytes.
465 func (v *UnixVolume) isFull() (isFull bool) {
466         fullSymlink := v.Root + "/full"
467
468         // Check if the volume has been marked as full in the last hour.
469         if link, err := os.Readlink(fullSymlink); err == nil {
470                 if ts, err := strconv.Atoi(link); err == nil {
471                         fulltime := time.Unix(int64(ts), 0)
472                         if time.Since(fulltime).Hours() < 1.0 {
473                                 return true
474                         }
475                 }
476         }
477
478         if avail, err := v.FreeDiskSpace(); err == nil {
479                 isFull = avail < BlockSize
480         } else {
481                 v.logger.WithError(err).Errorf("%s: FreeDiskSpace failed", v.DeviceID())
482                 isFull = false
483         }
484
485         // If the volume is full, timestamp it.
486         if isFull {
487                 now := fmt.Sprintf("%d", time.Now().Unix())
488                 os.Symlink(now, fullSymlink)
489         }
490         return
491 }
492
493 // FreeDiskSpace returns the number of unused 1k blocks available on
494 // the volume.
495 func (v *UnixVolume) FreeDiskSpace() (free uint64, err error) {
496         var fs syscall.Statfs_t
497         err = syscall.Statfs(v.Root, &fs)
498         if err == nil {
499                 // Statfs output is not guaranteed to measure free
500                 // space in terms of 1K blocks.
501                 free = fs.Bavail * uint64(fs.Bsize)
502         }
503         return
504 }
505
506 // InternalStats returns I/O and filesystem ops counters.
507 func (v *UnixVolume) InternalStats() interface{} {
508         return &v.os.stats
509 }
510
511 // lock acquires the serialize lock, if one is in use. If ctx is done
512 // before the lock is acquired, lock returns ctx.Err() instead of
513 // acquiring the lock.
514 func (v *UnixVolume) lock(ctx context.Context) error {
515         if v.locker == nil {
516                 return nil
517         }
518         t0 := time.Now()
519         locked := make(chan struct{})
520         go func() {
521                 v.locker.Lock()
522                 close(locked)
523         }()
524         select {
525         case <-ctx.Done():
526                 v.logger.Infof("client hung up while waiting for Serialize lock (%s)", time.Since(t0))
527                 go func() {
528                         <-locked
529                         v.locker.Unlock()
530                 }()
531                 return ctx.Err()
532         case <-locked:
533                 return nil
534         }
535 }
536
537 // unlock releases the serialize lock, if one is in use.
538 func (v *UnixVolume) unlock() {
539         if v.locker == nil {
540                 return
541         }
542         v.locker.Unlock()
543 }
544
545 // lockfile and unlockfile use flock(2) to manage kernel file locks.
546 func (v *UnixVolume) lockfile(f *os.File) error {
547         v.os.stats.TickOps("flock")
548         v.os.stats.Tick(&v.os.stats.FlockOps)
549         err := syscall.Flock(int(f.Fd()), syscall.LOCK_EX)
550         v.os.stats.TickErr(err)
551         return err
552 }
553
554 func (v *UnixVolume) unlockfile(f *os.File) error {
555         err := syscall.Flock(int(f.Fd()), syscall.LOCK_UN)
556         v.os.stats.TickErr(err)
557         return err
558 }
559
560 // Where appropriate, translate a more specific filesystem error to an
561 // error recognized by handlers, like os.ErrNotExist.
562 func (v *UnixVolume) translateError(err error) error {
563         switch err.(type) {
564         case *os.PathError:
565                 // stat() returns a PathError if the parent directory
566                 // (not just the file itself) is missing
567                 return os.ErrNotExist
568         default:
569                 return err
570         }
571 }
572
573 var unixTrashLocRegexp = regexp.MustCompile(`/([0-9a-f]{32})\.trash\.(\d+)$`)
574
575 // EmptyTrash walks hierarchy looking for {hash}.trash.*
576 // and deletes those with deadline < now.
577 func (v *UnixVolume) EmptyTrash() {
578         var bytesDeleted, bytesInTrash int64
579         var blocksDeleted, blocksInTrash int64
580
581         doFile := func(path string, info os.FileInfo) {
582                 if info.Mode().IsDir() {
583                         return
584                 }
585                 matches := unixTrashLocRegexp.FindStringSubmatch(path)
586                 if len(matches) != 3 {
587                         return
588                 }
589                 deadline, err := strconv.ParseInt(matches[2], 10, 64)
590                 if err != nil {
591                         v.logger.WithError(err).Errorf("EmptyTrash: %v: ParseInt(%q) failed", path, matches[2])
592                         return
593                 }
594                 atomic.AddInt64(&bytesInTrash, info.Size())
595                 atomic.AddInt64(&blocksInTrash, 1)
596                 if deadline > time.Now().Unix() {
597                         return
598                 }
599                 err = v.os.Remove(path)
600                 if err != nil {
601                         v.logger.WithError(err).Errorf("EmptyTrash: Remove(%q) failed", path)
602                         return
603                 }
604                 atomic.AddInt64(&bytesDeleted, info.Size())
605                 atomic.AddInt64(&blocksDeleted, 1)
606         }
607
608         type dirent struct {
609                 path string
610                 info os.FileInfo
611         }
612         var wg sync.WaitGroup
613         todo := make(chan dirent, v.cluster.Collections.BlobDeleteConcurrency)
614         for i := 0; i < v.cluster.Collections.BlobDeleteConcurrency; i++ {
615                 wg.Add(1)
616                 go func() {
617                         defer wg.Done()
618                         for e := range todo {
619                                 doFile(e.path, e.info)
620                         }
621                 }()
622         }
623
624         err := filepath.Walk(v.Root, func(path string, info os.FileInfo, err error) error {
625                 if err != nil {
626                         v.logger.WithError(err).Errorf("EmptyTrash: filepath.Walk(%q) failed", path)
627                         // Don't give up -- keep walking other
628                         // files/dirs
629                         return nil
630                 } else if !info.Mode().IsDir() {
631                         todo <- dirent{path, info}
632                         return nil
633                 } else if path == v.Root || blockDirRe.MatchString(info.Name()) {
634                         // Descend into a directory that we might have
635                         // put trash in.
636                         return nil
637                 } else {
638                         // Don't descend into other dirs.
639                         return filepath.SkipDir
640                 }
641         })
642         close(todo)
643         wg.Wait()
644
645         if err != nil {
646                 v.logger.WithError(err).Error("EmptyTrash failed")
647         }
648
649         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)
650 }
651
652 type unixStats struct {
653         statsTicker
654         OpenOps    uint64
655         StatOps    uint64
656         FlockOps   uint64
657         UtimesOps  uint64
658         CreateOps  uint64
659         RenameOps  uint64
660         UnlinkOps  uint64
661         ReaddirOps uint64
662 }
663
664 func (s *unixStats) TickErr(err error) {
665         if err == nil {
666                 return
667         }
668         s.statsTicker.TickErr(err, fmt.Sprintf("%T", err))
669 }
670
671 type osWithStats struct {
672         stats unixStats
673 }
674
675 func (o *osWithStats) Open(name string) (*os.File, error) {
676         o.stats.TickOps("open")
677         o.stats.Tick(&o.stats.OpenOps)
678         f, err := os.Open(name)
679         o.stats.TickErr(err)
680         return f, err
681 }
682
683 func (o *osWithStats) OpenFile(name string, flag int, perm os.FileMode) (*os.File, error) {
684         o.stats.TickOps("open")
685         o.stats.Tick(&o.stats.OpenOps)
686         f, err := os.OpenFile(name, flag, perm)
687         o.stats.TickErr(err)
688         return f, err
689 }
690
691 func (o *osWithStats) Remove(path string) error {
692         o.stats.TickOps("unlink")
693         o.stats.Tick(&o.stats.UnlinkOps)
694         err := os.Remove(path)
695         o.stats.TickErr(err)
696         return err
697 }
698
699 func (o *osWithStats) Rename(a, b string) error {
700         o.stats.TickOps("rename")
701         o.stats.Tick(&o.stats.RenameOps)
702         err := os.Rename(a, b)
703         o.stats.TickErr(err)
704         return err
705 }
706
707 func (o *osWithStats) Stat(path string) (os.FileInfo, error) {
708         o.stats.TickOps("stat")
709         o.stats.Tick(&o.stats.StatOps)
710         fi, err := os.Stat(path)
711         o.stats.TickErr(err)
712         return fi, err
713 }
714
715 func (o *osWithStats) TempFile(dir, base string) (*os.File, error) {
716         o.stats.TickOps("create")
717         o.stats.Tick(&o.stats.CreateOps)
718         f, err := ioutil.TempFile(dir, base)
719         o.stats.TickErr(err)
720         return f, err
721 }