13647: Use cluster config instead of custom keepstore config.
[arvados.git] / services / keepstore / s3_volume.go
index bdab58927bdc243605b8cf1d7e95b34d2f610272..22a38e20859dfaceeda16bedc90a59f035fda360 100644 (file)
@@ -7,12 +7,15 @@ package main
 import (
        "bytes"
        "context"
+       "crypto/sha256"
        "encoding/base64"
        "encoding/hex"
-       "flag"
+       "encoding/json"
+       "errors"
        "fmt"
        "io"
        "io/ioutil"
+       "log"
        "net/http"
        "os"
        "regexp"
@@ -24,188 +27,42 @@ import (
        "git.curoverse.com/arvados.git/sdk/go/arvados"
        "github.com/AdRoll/goamz/aws"
        "github.com/AdRoll/goamz/s3"
+       "github.com/prometheus/client_golang/prometheus"
+       "github.com/sirupsen/logrus"
 )
 
-const (
-       s3DefaultReadTimeout    = arvados.Duration(10 * time.Minute)
-       s3DefaultConnectTimeout = arvados.Duration(time.Minute)
-)
-
-var (
-       // ErrS3TrashDisabled is returned by Trash if that operation
-       // is impossible with the current config.
-       ErrS3TrashDisabled = fmt.Errorf("trash function is disabled because -trash-lifetime=0 and -s3-unsafe-delete=false")
-
-       s3AccessKeyFile string
-       s3SecretKeyFile string
-       s3RegionName    string
-       s3Endpoint      string
-       s3Replication   int
-       s3UnsafeDelete  bool
-       s3RaceWindow    time.Duration
-
-       s3ACL = s3.Private
-
-       zeroTime time.Time
-)
-
-const (
-       maxClockSkew  = 600 * time.Second
-       nearlyRFC1123 = "Mon, 2 Jan 2006 15:04:05 GMT"
-)
-
-type s3VolumeAdder struct {
-       *Config
-}
-
-// String implements flag.Value
-func (s *s3VolumeAdder) String() string {
-       return "-"
+func init() {
+       driver["S3"] = newS3Volume
 }
 
-func (s *s3VolumeAdder) Set(bucketName string) error {
-       if bucketName == "" {
-               return fmt.Errorf("no container name given")
-       }
-       if s3AccessKeyFile == "" || s3SecretKeyFile == "" {
-               return fmt.Errorf("-s3-access-key-file and -s3-secret-key-file arguments must given before -s3-bucket-volume")
-       }
-       if deprecated.flagSerializeIO {
-               log.Print("Notice: -serialize is not supported by s3-bucket volumes.")
+func newS3Volume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) {
+       v := &S3Volume{cluster: cluster, volume: volume, logger: logger, metrics: metrics}
+       err := json.Unmarshal(volume.DriverParameters, &v)
+       if err != nil {
+               return nil, err
        }
-       s.Config.Volumes = append(s.Config.Volumes, &S3Volume{
-               Bucket:        bucketName,
-               AccessKeyFile: s3AccessKeyFile,
-               SecretKeyFile: s3SecretKeyFile,
-               Endpoint:      s3Endpoint,
-               Region:        s3RegionName,
-               RaceWindow:    arvados.Duration(s3RaceWindow),
-               S3Replication: s3Replication,
-               UnsafeDelete:  s3UnsafeDelete,
-               ReadOnly:      deprecated.flagReadonly,
-               IndexPageSize: 1000,
-       })
-       return nil
+       return v, v.check()
 }
 
-func s3regions() (okList []string) {
-       for r := range aws.Regions {
-               okList = append(okList, r)
+func (v *S3Volume) check() error {
+       if v.Bucket == "" || v.AccessKey == "" || v.SecretKey == "" {
+               return errors.New("DriverParameters: Bucket, AccessKey, and SecretKey must be provided")
        }
-       return
-}
-
-func init() {
-       VolumeTypes = append(VolumeTypes, func() VolumeWithExamples { return &S3Volume{} })
-
-       flag.Var(&s3VolumeAdder{theConfig},
-               "s3-bucket-volume",
-               "Use the given bucket as a storage volume. Can be given multiple times.")
-       flag.StringVar(
-               &s3RegionName,
-               "s3-region",
-               "",
-               fmt.Sprintf("AWS region used for subsequent -s3-bucket-volume arguments. Allowed values are %+q.", s3regions()))
-       flag.StringVar(
-               &s3Endpoint,
-               "s3-endpoint",
-               "",
-               "Endpoint URL used for subsequent -s3-bucket-volume arguments. If blank, use the AWS endpoint corresponding to the -s3-region argument. For Google Storage, use \"https://storage.googleapis.com\".")
-       flag.StringVar(
-               &s3AccessKeyFile,
-               "s3-access-key-file",
-               "",
-               "`File` containing the access key used for subsequent -s3-bucket-volume arguments.")
-       flag.StringVar(
-               &s3SecretKeyFile,
-               "s3-secret-key-file",
-               "",
-               "`File` containing the secret key used for subsequent -s3-bucket-volume arguments.")
-       flag.DurationVar(
-               &s3RaceWindow,
-               "s3-race-window",
-               24*time.Hour,
-               "Maximum eventual consistency latency for subsequent -s3-bucket-volume arguments.")
-       flag.IntVar(
-               &s3Replication,
-               "s3-replication",
-               2,
-               "Replication level reported to clients for subsequent -s3-bucket-volume arguments.")
-       flag.BoolVar(
-               &s3UnsafeDelete,
-               "s3-unsafe-delete",
-               false,
-               "EXPERIMENTAL. Enable deletion (garbage collection) even when trash lifetime is zero, even though there are known race conditions that can cause data loss.")
-}
-
-// S3Volume implements Volume using an S3 bucket.
-type S3Volume struct {
-       AccessKeyFile      string
-       SecretKeyFile      string
-       Endpoint           string
-       Region             string
-       Bucket             string
-       LocationConstraint bool
-       IndexPageSize      int
-       S3Replication      int
-       ConnectTimeout     arvados.Duration
-       ReadTimeout        arvados.Duration
-       RaceWindow         arvados.Duration
-       ReadOnly           bool
-       UnsafeDelete       bool
-       StorageClasses     []string
-
-       bucket *s3bucket
-
-       startOnce sync.Once
-}
-
-// Examples implements VolumeWithExamples.
-func (*S3Volume) Examples() []Volume {
-       return []Volume{
-               &S3Volume{
-                       AccessKeyFile:  "/etc/aws_s3_access_key.txt",
-                       SecretKeyFile:  "/etc/aws_s3_secret_key.txt",
-                       Endpoint:       "",
-                       Region:         "us-east-1",
-                       Bucket:         "example-bucket-name",
-                       IndexPageSize:  1000,
-                       S3Replication:  2,
-                       RaceWindow:     arvados.Duration(24 * time.Hour),
-                       ConnectTimeout: arvados.Duration(time.Minute),
-                       ReadTimeout:    arvados.Duration(5 * time.Minute),
-               },
-               &S3Volume{
-                       AccessKeyFile:  "/etc/gce_s3_access_key.txt",
-                       SecretKeyFile:  "/etc/gce_s3_secret_key.txt",
-                       Endpoint:       "https://storage.googleapis.com",
-                       Region:         "",
-                       Bucket:         "example-bucket-name",
-                       IndexPageSize:  1000,
-                       S3Replication:  2,
-                       RaceWindow:     arvados.Duration(24 * time.Hour),
-                       ConnectTimeout: arvados.Duration(time.Minute),
-                       ReadTimeout:    arvados.Duration(5 * time.Minute),
-               },
+       if v.IndexPageSize == 0 {
+               v.IndexPageSize = 1000
+       }
+       if v.RaceWindow < 0 {
+               return errors.New("DriverParameters: RaceWindow must not be negative")
        }
-}
-
-// Type implements Volume.
-func (*S3Volume) Type() string {
-       return "S3"
-}
 
-// Start populates private fields and verifies the configuration is
-// valid.
-func (v *S3Volume) Start() error {
        region, ok := aws.Regions[v.Region]
        if v.Endpoint == "" {
                if !ok {
-                       return fmt.Errorf("unrecognized region %+q; try specifying -s3-endpoint instead", v.Region)
+                       return fmt.Errorf("unrecognized region %+q; try specifying endpoint instead", v.Region)
                }
        } else if ok {
                return fmt.Errorf("refusing to use AWS region name %+q with endpoint %+q; "+
-                       "specify empty endpoint (\"-s3-endpoint=\") or use a different region name", v.Region, v.Endpoint)
+                       "specify empty endpoint or use a different region name", v.Region, v.Endpoint)
        } else {
                region = aws.Region{
                        Name:                 v.Region,
@@ -214,15 +71,9 @@ func (v *S3Volume) Start() error {
                }
        }
 
-       var err error
-       var auth aws.Auth
-       auth.AccessKey, err = readKeyFromFile(v.AccessKeyFile)
-       if err != nil {
-               return err
-       }
-       auth.SecretKey, err = readKeyFromFile(v.SecretKeyFile)
-       if err != nil {
-               return err
+       auth := aws.Auth{
+               AccessKey: v.AccessKey,
+               SecretKey: v.SecretKey,
        }
 
        // Zero timeouts mean "wait forever", which is a bad
@@ -235,6 +86,10 @@ func (v *S3Volume) Start() error {
        }
 
        client := s3.New(auth, region)
+       if region.EC2Endpoint.Signer == aws.V4Signature {
+               // Currently affects only eu-central-1
+               client.Signature = aws.V4Signature
+       }
        client.ConnectTimeout = time.Duration(v.ConnectTimeout)
        client.ReadTimeout = time.Duration(v.ReadTimeout)
        v.bucket = &s3bucket{
@@ -243,11 +98,64 @@ func (v *S3Volume) Start() error {
                        Name: v.Bucket,
                },
        }
+       // Set up prometheus metrics
+       lbls := prometheus.Labels{"device_id": v.GetDeviceID()}
+       v.bucket.stats.opsCounters, v.bucket.stats.errCounters, v.bucket.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
+
        return nil
 }
 
-// DeviceID returns a globally unique ID for the storage bucket.
-func (v *S3Volume) DeviceID() string {
+const (
+       s3DefaultReadTimeout    = arvados.Duration(10 * time.Minute)
+       s3DefaultConnectTimeout = arvados.Duration(time.Minute)
+)
+
+var (
+       // ErrS3TrashDisabled is returned by Trash if that operation
+       // is impossible with the current config.
+       ErrS3TrashDisabled = fmt.Errorf("trash function is disabled because -trash-lifetime=0 and -s3-unsafe-delete=false")
+
+       s3ACL = s3.Private
+
+       zeroTime time.Time
+)
+
+const (
+       maxClockSkew  = 600 * time.Second
+       nearlyRFC1123 = "Mon, 2 Jan 2006 15:04:05 GMT"
+)
+
+func s3regions() (okList []string) {
+       for r := range aws.Regions {
+               okList = append(okList, r)
+       }
+       return
+}
+
+// S3Volume implements Volume using an S3 bucket.
+type S3Volume struct {
+       AccessKey          string
+       SecretKey          string
+       Endpoint           string
+       Region             string
+       Bucket             string
+       LocationConstraint bool
+       IndexPageSize      int
+       ConnectTimeout     arvados.Duration
+       ReadTimeout        arvados.Duration
+       RaceWindow         arvados.Duration
+       UnsafeDelete       bool
+
+       cluster   *arvados.Cluster
+       volume    arvados.Volume
+       logger    logrus.FieldLogger
+       metrics   *volumeMetricsVecs
+       bucket    *s3bucket
+       startOnce sync.Once
+}
+
+// GetDeviceID returns a globally unique ID for the storage bucket.
+func (v *S3Volume) GetDeviceID() string {
        return "s3://" + v.Endpoint + "/" + v.Bucket
 }
 
@@ -261,7 +169,7 @@ func (v *S3Volume) getReaderWithContext(ctx context.Context, loc string) (rdr io
        case <-ready:
                return
        case <-ctx.Done():
-               theConfig.debugLogf("s3: abandoning getReader(): %s", ctx.Err())
+               v.logger.Debugf("s3: abandoning getReader(): %s", ctx.Err())
                go func() {
                        <-ready
                        if err == nil {
@@ -329,11 +237,11 @@ func (v *S3Volume) Get(ctx context.Context, loc string, buf []byte) (int, error)
        }()
        select {
        case <-ctx.Done():
-               theConfig.debugLogf("s3: interrupting ReadFull() with Close() because %s", ctx.Err())
+               v.logger.Debugf("s3: interrupting ReadFull() with Close() because %s", ctx.Err())
                rdr.Close()
                // Must wait for ReadFull to return, to ensure it
                // doesn't write to buf after we return.
-               theConfig.debugLogf("s3: waiting for ReadFull() to fail")
+               v.logger.Debug("s3: waiting for ReadFull() to fail")
                <-ready
                return 0, ctx.Err()
        case <-ready:
@@ -387,7 +295,7 @@ func (v *S3Volume) Compare(ctx context.Context, loc string, expect []byte) error
 
 // Put writes a block.
 func (v *S3Volume) Put(ctx context.Context, loc string, block []byte) error {
-       if v.ReadOnly {
+       if v.volume.ReadOnly {
                return MethodDisabledError
        }
        var opts s3.Options
@@ -398,6 +306,14 @@ func (v *S3Volume) Put(ctx context.Context, loc string, block []byte) error {
                        return err
                }
                opts.ContentMD5 = base64.StdEncoding.EncodeToString(md5)
+               // In AWS regions that use V4 signatures, we need to
+               // provide ContentSHA256 up front. Otherwise, the S3
+               // library reads the request body (from our buffer)
+               // into another new buffer in order to compute the
+               // SHA256 before sending the request -- which would
+               // mean consuming 128 MiB of memory for the duration
+               // of a 64 MiB write.
+               opts.ContentSHA256 = fmt.Sprintf("%x", sha256.Sum256(block))
        }
 
        // Send the block data through a pipe, so that (if we need to)
@@ -415,7 +331,7 @@ func (v *S3Volume) Put(ctx context.Context, loc string, block []byte) error {
        go func() {
                defer func() {
                        if ctx.Err() != nil {
-                               theConfig.debugLogf("%s: abandoned PutReader goroutine finished with err: %s", v, err)
+                               v.logger.Debugf("%s: abandoned PutReader goroutine finished with err: %s", v, err)
                        }
                }()
                defer close(ready)
@@ -427,7 +343,7 @@ func (v *S3Volume) Put(ctx context.Context, loc string, block []byte) error {
        }()
        select {
        case <-ctx.Done():
-               theConfig.debugLogf("%s: taking PutReader's input away: %s", v, ctx.Err())
+               v.logger.Debugf("%s: taking PutReader's input away: %s", v, ctx.Err())
                // Our pipe might be stuck in Write(), waiting for
                // PutReader() to read. If so, un-stick it. This means
                // PutReader will get corrupt data, but that's OK: the
@@ -435,7 +351,7 @@ func (v *S3Volume) Put(ctx context.Context, loc string, block []byte) error {
                go io.Copy(ioutil.Discard, bufr)
                // CloseWithError() will return once pending I/O is done.
                bufw.CloseWithError(ctx.Err())
-               theConfig.debugLogf("%s: abandoning PutReader goroutine", v)
+               v.logger.Debugf("%s: abandoning PutReader goroutine", v)
                return ctx.Err()
        case <-ready:
                // Unblock pipe in case PutReader did not consume it.
@@ -446,7 +362,7 @@ func (v *S3Volume) Put(ctx context.Context, loc string, block []byte) error {
 
 // Touch sets the timestamp for the given locator to the current time.
 func (v *S3Volume) Touch(loc string) error {
-       if v.ReadOnly {
+       if v.volume.ReadOnly {
                return MethodDisabledError
        }
        _, err := v.bucket.Head(loc, nil)
@@ -497,16 +413,15 @@ func (v *S3Volume) IndexTo(prefix string, writer io.Writer) error {
                Bucket:   v.bucket.Bucket,
                Prefix:   prefix,
                PageSize: v.IndexPageSize,
+               Stats:    &v.bucket.stats,
        }
        recentL := s3Lister{
                Bucket:   v.bucket.Bucket,
                Prefix:   "recent/" + prefix,
                PageSize: v.IndexPageSize,
+               Stats:    &v.bucket.stats,
        }
-       v.bucket.stats.Tick(&v.bucket.stats.Ops, &v.bucket.stats.ListOps)
-       v.bucket.stats.Tick(&v.bucket.stats.Ops, &v.bucket.stats.ListOps)
-       for data, recent := dataL.First(), recentL.First(); data != nil; data = dataL.Next() {
-               v.bucket.stats.Tick(&v.bucket.stats.Ops, &v.bucket.stats.ListOps)
+       for data, recent := dataL.First(), recentL.First(); data != nil && dataL.Error() == nil; data = dataL.Next() {
                if data.Key >= "g" {
                        // Conveniently, "recent/*" and "trash/*" are
                        // lexically greater than all hex-encoded data
@@ -525,15 +440,13 @@ func (v *S3Volume) IndexTo(prefix string, writer io.Writer) error {
                stamp := data
 
                // Advance to the corresponding recent/X marker, if any
-               for recent != nil {
+               for recent != nil && recentL.Error() == nil {
                        if cmp := strings.Compare(recent.Key[7:], data.Key); cmp < 0 {
                                recent = recentL.Next()
-                               v.bucket.stats.Tick(&v.bucket.stats.Ops, &v.bucket.stats.ListOps)
                                continue
                        } else if cmp == 0 {
                                stamp = recent
                                recent = recentL.Next()
-                               v.bucket.stats.Tick(&v.bucket.stats.Ops, &v.bucket.stats.ListOps)
                                break
                        } else {
                                // recent/X marker is missing: we'll
@@ -542,27 +455,30 @@ func (v *S3Volume) IndexTo(prefix string, writer io.Writer) error {
                                break
                        }
                }
+               if err := recentL.Error(); err != nil {
+                       return err
+               }
                t, err := time.Parse(time.RFC3339, stamp.LastModified)
                if err != nil {
                        return err
                }
                fmt.Fprintf(writer, "%s+%d %d\n", data.Key, data.Size, t.UnixNano())
        }
-       return nil
+       return dataL.Error()
 }
 
 // Trash a Keep block.
 func (v *S3Volume) Trash(loc string) error {
-       if v.ReadOnly {
+       if v.volume.ReadOnly {
                return MethodDisabledError
        }
        if t, err := v.Mtime(loc); err != nil {
                return err
-       } else if time.Since(t) < theConfig.BlobSignatureTTL.Duration() {
+       } else if time.Since(t) < v.cluster.Collections.BlobSigningTTL.Duration() {
                return nil
        }
-       if theConfig.TrashLifetime == 0 {
-               if !s3UnsafeDelete {
+       if v.cluster.Collections.BlobTrashLifetime == 0 {
+               if !v.UnsafeDelete {
                        return ErrS3TrashDisabled
                }
                return v.translateError(v.bucket.Del(loc))
@@ -597,7 +513,7 @@ func (v *S3Volume) checkRaceWindow(loc string) error {
                // Can't parse timestamp
                return err
        }
-       safeWindow := t.Add(theConfig.TrashLifetime.Duration()).Sub(time.Now().Add(time.Duration(v.RaceWindow)))
+       safeWindow := t.Add(v.cluster.Collections.BlobTrashLifetime.Duration()).Sub(time.Now().Add(time.Duration(v.RaceWindow)))
        if safeWindow <= 0 {
                // We can't count on "touch trash/X" to prolong
                // trash/X's lifetime. The new timestamp might not
@@ -620,8 +536,10 @@ func (v *S3Volume) safeCopy(dst, src string) error {
                MetadataDirective: "REPLACE",
        }, v.bucket.Name+"/"+src)
        err = v.translateError(err)
-       if err != nil {
+       if os.IsNotExist(err) {
                return err
+       } else if err != nil {
+               return fmt.Errorf("PutCopy(%q ← %q): %s", dst, v.bucket.Name+"/"+src, err)
        }
        if t, err := time.Parse(time.RFC3339Nano, resp.LastModified); err != nil {
                return fmt.Errorf("PutCopy succeeded but did not return a timestamp: %q: %s", resp.LastModified, err)
@@ -678,23 +596,6 @@ func (v *S3Volume) String() string {
        return fmt.Sprintf("s3-bucket:%+q", v.Bucket)
 }
 
-// Writable returns false if all future Put, Mtime, and Delete calls
-// are expected to fail.
-func (v *S3Volume) Writable() bool {
-       return !v.ReadOnly
-}
-
-// Replication returns the storage redundancy of the underlying
-// device. Configured via command line flag.
-func (v *S3Volume) Replication() int {
-       return v.S3Replication
-}
-
-// GetStorageClasses implements Volume
-func (v *S3Volume) GetStorageClasses() []string {
-       return v.StorageClasses
-}
-
 var s3KeepBlockRegexp = regexp.MustCompile(`^[0-9a-f]{32}$`)
 
 func (v *S3Volume) isKeepBlock(s string) bool {
@@ -731,13 +632,13 @@ func (v *S3Volume) fixRace(loc string) bool {
        }
 
        ageWhenTrashed := trashTime.Sub(recentTime)
-       if ageWhenTrashed >= theConfig.BlobSignatureTTL.Duration() {
+       if ageWhenTrashed >= v.cluster.Collections.BlobSigningTTL.Duration() {
                // No evidence of a race: block hasn't been written
                // since it became eligible for Trash. No fix needed.
                return false
        }
 
-       log.Printf("notice: fixRace: %q: trashed at %s but touched at %s (age when trashed = %s < %s)", loc, trashTime, recentTime, ageWhenTrashed, theConfig.BlobSignatureTTL)
+       log.Printf("notice: fixRace: %q: trashed at %s but touched at %s (age when trashed = %s < %s)", loc, trashTime, recentTime, ageWhenTrashed, v.cluster.Collections.BlobSigningTTL)
        log.Printf("notice: fixRace: copying %q to %q to recover from race between Put/Touch and Trash", "recent/"+loc, loc)
        err = v.safeCopy(loc, "trash/"+loc)
        if err != nil {
@@ -765,6 +666,10 @@ func (v *S3Volume) translateError(err error) error {
 // EmptyTrash looks for trashed blocks that exceeded TrashLifetime
 // and deletes them from the volume.
 func (v *S3Volume) EmptyTrash() {
+       if v.cluster.Collections.BlobDeleteConcurrency < 1 {
+               return
+       }
+
        var bytesInTrash, blocksInTrash, bytesDeleted, blocksDeleted int64
 
        // Define "ready to delete" as "...when EmptyTrash started".
@@ -800,8 +705,8 @@ func (v *S3Volume) EmptyTrash() {
                        log.Printf("warning: %s: EmptyTrash: %q: parse %q: %s", v, "recent/"+loc, recent.Header.Get("Last-Modified"), err)
                        return
                }
-               if trashT.Sub(recentT) < theConfig.BlobSignatureTTL.Duration() {
-                       if age := startT.Sub(recentT); age >= theConfig.BlobSignatureTTL.Duration()-time.Duration(v.RaceWindow) {
+               if trashT.Sub(recentT) < v.cluster.Collections.BlobSigningTTL.Duration() {
+                       if age := startT.Sub(recentT); age >= v.cluster.Collections.BlobSigningTTL.Duration()-time.Duration(v.RaceWindow) {
                                // recent/loc is too old to protect
                                // loc from being Trashed again during
                                // the raceWindow that starts if we
@@ -825,7 +730,7 @@ func (v *S3Volume) EmptyTrash() {
                                return
                        }
                }
-               if startT.Sub(trashT) < theConfig.TrashLifetime.Duration() {
+               if startT.Sub(trashT) < v.cluster.Collections.BlobTrashLifetime.Duration() {
                        return
                }
                err = v.bucket.Del(trash.Key)
@@ -852,8 +757,8 @@ func (v *S3Volume) EmptyTrash() {
        }
 
        var wg sync.WaitGroup
-       todo := make(chan *s3.Key, theConfig.EmptyTrashWorkers)
-       for i := 0; i < 1 || i < theConfig.EmptyTrashWorkers; i++ {
+       todo := make(chan *s3.Key, v.cluster.Collections.BlobDeleteConcurrency)
+       for i := 0; i < v.cluster.Collections.BlobDeleteConcurrency; i++ {
                wg.Add(1)
                go func() {
                        defer wg.Done()
@@ -867,6 +772,7 @@ func (v *S3Volume) EmptyTrash() {
                Bucket:   v.bucket.Bucket,
                Prefix:   "trash/",
                PageSize: v.IndexPageSize,
+               Stats:    &v.bucket.stats,
        }
        for trash := trashL.First(); trash != nil; trash = trashL.Next() {
                todo <- trash
@@ -884,6 +790,7 @@ type s3Lister struct {
        Bucket     *s3.Bucket
        Prefix     string
        PageSize   int
+       Stats      *s3bucketStats
        nextMarker string
        buf        []s3.Key
        err        error
@@ -912,6 +819,8 @@ func (lister *s3Lister) Error() error {
 }
 
 func (lister *s3Lister) getPage() {
+       lister.Stats.TickOps("list")
+       lister.Stats.Tick(&lister.Stats.Ops, &lister.Stats.ListOps)
        resp, err := lister.Bucket.List(lister.Prefix, "", lister.nextMarker, lister.PageSize)
        lister.nextMarker = ""
        if err != nil {
@@ -947,6 +856,7 @@ type s3bucket struct {
 
 func (b *s3bucket) GetReader(path string) (io.ReadCloser, error) {
        rdr, err := b.Bucket.GetReader(path)
+       b.stats.TickOps("get")
        b.stats.Tick(&b.stats.Ops, &b.stats.GetOps)
        b.stats.TickErr(err)
        return NewCountingReader(rdr, b.stats.TickInBytes), err
@@ -954,6 +864,7 @@ func (b *s3bucket) GetReader(path string) (io.ReadCloser, error) {
 
 func (b *s3bucket) Head(path string, headers map[string][]string) (*http.Response, error) {
        resp, err := b.Bucket.Head(path, headers)
+       b.stats.TickOps("head")
        b.stats.Tick(&b.stats.Ops, &b.stats.HeadOps)
        b.stats.TickErr(err)
        return resp, err
@@ -972,6 +883,7 @@ func (b *s3bucket) PutReader(path string, r io.Reader, length int64, contType st
                r = NewCountingReader(r, b.stats.TickOutBytes)
        }
        err := b.Bucket.PutReader(path, r, length, contType, perm, options)
+       b.stats.TickOps("put")
        b.stats.Tick(&b.stats.Ops, &b.stats.PutOps)
        b.stats.TickErr(err)
        return err
@@ -979,6 +891,7 @@ func (b *s3bucket) PutReader(path string, r io.Reader, length int64, contType st
 
 func (b *s3bucket) Del(path string) error {
        err := b.Bucket.Del(path)
+       b.stats.TickOps("delete")
        b.stats.Tick(&b.stats.Ops, &b.stats.DelOps)
        b.stats.TickErr(err)
        return err