X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/e231fef37079916d0dc6babf93d669d474598ced..da83807d6bcef1c1f0bb78479c5ec17f150f5eda:/services/keepstore/azure_blob_volume.go diff --git a/services/keepstore/azure_blob_volume.go b/services/keepstore/azure_blob_volume.go index 4f7339facf..56a52c913a 100644 --- a/services/keepstore/azure_blob_volume.go +++ b/services/keepstore/azure_blob_volume.go @@ -2,13 +2,13 @@ // // SPDX-License-Identifier: AGPL-3.0 -package main +package keepstore import ( "bytes" "context" + "encoding/json" "errors" - "flag" "fmt" "io" "io/ioutil" @@ -21,93 +21,97 @@ import ( "sync/atomic" "time" - "git.curoverse.com/arvados.git/sdk/go/arvados" + "git.arvados.org/arvados.git/sdk/go/arvados" + "git.arvados.org/arvados.git/sdk/go/ctxlog" "github.com/Azure/azure-sdk-for-go/storage" + "github.com/prometheus/client_golang/prometheus" + "github.com/sirupsen/logrus" ) -const azureDefaultRequestTimeout = arvados.Duration(10 * time.Minute) - -var ( - azureMaxGetBytes int - azureStorageAccountName string - azureStorageAccountKeyFile string - azureStorageReplication int - azureWriteRaceInterval = 15 * time.Second - azureWriteRacePollTime = time.Second -) +func init() { + driver["Azure"] = newAzureBlobVolume +} -func readKeyFromFile(file string) (string, error) { - buf, err := ioutil.ReadFile(file) +func newAzureBlobVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) { + v := &AzureBlobVolume{ + RequestTimeout: azureDefaultRequestTimeout, + WriteRaceInterval: azureDefaultWriteRaceInterval, + WriteRacePollTime: azureDefaultWriteRacePollTime, + cluster: cluster, + volume: volume, + logger: logger, + metrics: metrics, + } + err := json.Unmarshal(volume.DriverParameters, &v) if err != nil { - return "", errors.New("reading key from " + file + ": " + err.Error()) + return nil, err } - accountKey := strings.TrimSpace(string(buf)) - if accountKey == "" { - return "", errors.New("empty account key in " + file) + if v.ListBlobsRetryDelay == 0 { + v.ListBlobsRetryDelay = azureDefaultListBlobsRetryDelay + } + if v.ListBlobsMaxAttempts == 0 { + v.ListBlobsMaxAttempts = azureDefaultListBlobsMaxAttempts + } + if v.StorageBaseURL == "" { + v.StorageBaseURL = storage.DefaultBaseURL + } + if v.ContainerName == "" || v.StorageAccountName == "" || v.StorageAccountKey == "" { + return nil, errors.New("DriverParameters: ContainerName, StorageAccountName, and StorageAccountKey must be provided") + } + azc, err := storage.NewClient(v.StorageAccountName, v.StorageAccountKey, v.StorageBaseURL, storage.DefaultAPIVersion, true) + if err != nil { + return nil, fmt.Errorf("creating Azure storage client: %s", err) + } + v.azClient = azc + v.azClient.Sender = &singleSender{} + v.azClient.HTTPClient = &http.Client{ + Timeout: time.Duration(v.RequestTimeout), + } + bs := v.azClient.GetBlobService() + v.container = &azureContainer{ + ctr: bs.GetContainerReference(v.ContainerName), } - return accountKey, nil -} - -type azureVolumeAdder struct { - *Config -} -// String implements flag.Value -func (s *azureVolumeAdder) String() string { - return "-" + if ok, err := v.container.Exists(); err != nil { + return nil, err + } else if !ok { + return nil, fmt.Errorf("Azure container %q does not exist: %s", v.ContainerName, err) + } + return v, v.check() } -func (s *azureVolumeAdder) Set(containerName string) error { - s.Config.Volumes = append(s.Config.Volumes, &AzureBlobVolume{ - ContainerName: containerName, - StorageAccountName: azureStorageAccountName, - StorageAccountKeyFile: azureStorageAccountKeyFile, - AzureReplication: azureStorageReplication, - ReadOnly: deprecated.flagReadonly, - }) +func (v *AzureBlobVolume) check() error { + lbls := prometheus.Labels{"device_id": v.GetDeviceID()} + v.container.stats.opsCounters, v.container.stats.errCounters, v.container.stats.ioBytes = v.metrics.getCounterVecsFor(lbls) return nil } -func init() { - VolumeTypes = append(VolumeTypes, func() VolumeWithExamples { return &AzureBlobVolume{} }) - - flag.Var(&azureVolumeAdder{theConfig}, - "azure-storage-container-volume", - "Use the given container as a storage volume. Can be given multiple times.") - flag.StringVar( - &azureStorageAccountName, - "azure-storage-account-name", - "", - "Azure storage account name used for subsequent --azure-storage-container-volume arguments.") - flag.StringVar( - &azureStorageAccountKeyFile, - "azure-storage-account-key-file", - "", - "`File` containing the account key used for subsequent --azure-storage-container-volume arguments.") - flag.IntVar( - &azureStorageReplication, - "azure-storage-replication", - 3, - "Replication level to report to clients when data is stored in an Azure container.") - flag.IntVar( - &azureMaxGetBytes, - "azure-max-get-bytes", - BlockSize, - fmt.Sprintf("Maximum bytes to request in a single GET request. If smaller than %d, use multiple concurrent range requests to retrieve a block.", BlockSize)) -} +const ( + azureDefaultRequestTimeout = arvados.Duration(10 * time.Minute) + azureDefaultListBlobsMaxAttempts = 12 + azureDefaultListBlobsRetryDelay = arvados.Duration(10 * time.Second) + azureDefaultWriteRaceInterval = arvados.Duration(15 * time.Second) + azureDefaultWriteRacePollTime = arvados.Duration(time.Second) +) // An AzureBlobVolume stores and retrieves blocks in an Azure Blob // container. type AzureBlobVolume struct { - StorageAccountName string - StorageAccountKeyFile string - StorageBaseURL string // "" means default, "core.windows.net" - ContainerName string - AzureReplication int - ReadOnly bool - RequestTimeout arvados.Duration - StorageClasses []string - + StorageAccountName string + StorageAccountKey string + StorageBaseURL string // "" means default, "core.windows.net" + ContainerName string + RequestTimeout arvados.Duration + ListBlobsRetryDelay arvados.Duration + ListBlobsMaxAttempts int + MaxGetBytes int + WriteRaceInterval arvados.Duration + WriteRacePollTime arvados.Duration + + cluster *arvados.Cluster + volume arvados.Volume + logger logrus.FieldLogger + metrics *volumeMetricsVecs azClient storage.Client container *azureContainer } @@ -120,74 +124,13 @@ func (*singleSender) Send(c *storage.Client, req *http.Request) (resp *http.Resp return c.HTTPClient.Do(req) } -// Examples implements VolumeWithExamples. -func (*AzureBlobVolume) Examples() []Volume { - return []Volume{ - &AzureBlobVolume{ - StorageAccountName: "example-account-name", - StorageAccountKeyFile: "/etc/azure_storage_account_key.txt", - ContainerName: "example-container-name", - AzureReplication: 3, - RequestTimeout: azureDefaultRequestTimeout, - }, - &AzureBlobVolume{ - StorageAccountName: "cn-account-name", - StorageAccountKeyFile: "/etc/azure_cn_storage_account_key.txt", - StorageBaseURL: "core.chinacloudapi.cn", - ContainerName: "cn-container-name", - AzureReplication: 3, - RequestTimeout: azureDefaultRequestTimeout, - }, - } -} - // Type implements Volume. func (v *AzureBlobVolume) Type() string { return "Azure" } -// Start implements Volume. -func (v *AzureBlobVolume) Start() error { - if v.ContainerName == "" { - return errors.New("no container name given") - } - if v.StorageAccountName == "" || v.StorageAccountKeyFile == "" { - return errors.New("StorageAccountName and StorageAccountKeyFile must be given") - } - accountKey, err := readKeyFromFile(v.StorageAccountKeyFile) - if err != nil { - return err - } - if v.StorageBaseURL == "" { - v.StorageBaseURL = storage.DefaultBaseURL - } - v.azClient, err = storage.NewClient(v.StorageAccountName, accountKey, v.StorageBaseURL, storage.DefaultAPIVersion, true) - if err != nil { - return fmt.Errorf("creating Azure storage client: %s", err) - } - v.azClient.Sender = &singleSender{} - - if v.RequestTimeout == 0 { - v.RequestTimeout = azureDefaultRequestTimeout - } - v.azClient.HTTPClient = &http.Client{ - Timeout: time.Duration(v.RequestTimeout), - } - bs := v.azClient.GetBlobService() - v.container = &azureContainer{ - ctr: bs.GetContainerReference(v.ContainerName), - } - - if ok, err := v.container.Exists(); err != nil { - return err - } else if !ok { - return fmt.Errorf("Azure container %q does not exist", v.ContainerName) - } - return nil -} - -// DeviceID returns a globally unique ID for the storage container. -func (v *AzureBlobVolume) DeviceID() string { +// GetDeviceID returns a globally unique ID for the storage container. +func (v *AzureBlobVolume) GetDeviceID() string { return "azure://" + v.StorageBaseURL + "/" + v.StorageAccountName + "/" + v.ContainerName } @@ -228,14 +171,14 @@ func (v *AzureBlobVolume) Get(ctx context.Context, loc string, buf []byte) (int, if !haveDeadline { t, err := v.Mtime(loc) if err != nil { - log.Print("Got empty block (possible race) but Mtime failed: ", err) + ctxlog.FromContext(ctx).Print("Got empty block (possible race) but Mtime failed: ", err) break } - deadline = t.Add(azureWriteRaceInterval) + deadline = t.Add(v.WriteRaceInterval.Duration()) if time.Now().After(deadline) { break } - log.Printf("Race? Block %s is 0 bytes, %s old. Polling until %s", loc, time.Since(t), deadline) + ctxlog.FromContext(ctx).Printf("Race? Block %s is 0 bytes, %s old. Polling until %s", loc, time.Since(t), deadline) haveDeadline = true } else if time.Now().After(deadline) { break @@ -243,12 +186,12 @@ func (v *AzureBlobVolume) Get(ctx context.Context, loc string, buf []byte) (int, select { case <-ctx.Done(): return 0, ctx.Err() - case <-time.After(azureWriteRacePollTime): + case <-time.After(v.WriteRacePollTime.Duration()): } size, err = v.get(ctx, loc, buf) } if haveDeadline { - log.Printf("Race ended with size==%d", size) + ctxlog.FromContext(ctx).Printf("Race ended with size==%d", size) } return size, err } @@ -256,8 +199,15 @@ func (v *AzureBlobVolume) Get(ctx context.Context, loc string, buf []byte) (int, func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, error) { ctx, cancel := context.WithCancel(ctx) defer cancel() + + pieceSize := BlockSize + if v.MaxGetBytes > 0 && v.MaxGetBytes < BlockSize { + pieceSize = v.MaxGetBytes + } + + pieces := 1 expectSize := len(buf) - if azureMaxGetBytes < BlockSize { + if pieceSize < BlockSize { // Unfortunately the handler doesn't tell us how long the blob // is expected to be, so we have to ask Azure. props, err := v.container.GetBlobProperties(loc) @@ -268,6 +218,7 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, return 0, fmt.Errorf("block %s invalid size %d (max %d)", loc, props.ContentLength, BlockSize) } expectSize = int(props.ContentLength) + pieces = (expectSize + pieceSize - 1) / pieceSize } if expectSize == 0 { @@ -276,7 +227,6 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, // We'll update this actualSize if/when we get the last piece. actualSize := -1 - pieces := (expectSize + azureMaxGetBytes - 1) / azureMaxGetBytes errors := make(chan error, pieces) var wg sync.WaitGroup wg.Add(pieces) @@ -291,8 +241,8 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, // interrupted as a result. go func(p int) { defer wg.Done() - startPos := p * azureMaxGetBytes - endPos := startPos + azureMaxGetBytes + startPos := p * pieceSize + endPos := startPos + pieceSize if endPos > expectSize { endPos = expectSize } @@ -395,7 +345,7 @@ func (v *AzureBlobVolume) Compare(ctx context.Context, loc string, expect []byte // Put stores a Keep block as a block blob in the container. func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) error { - if v.ReadOnly { + if v.volume.ReadOnly { return MethodDisabledError } // Send the block data through a pipe, so that (if we need to) @@ -424,7 +374,7 @@ func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) err }() select { case <-ctx.Done(): - theConfig.debugLogf("%s: taking CreateBlockBlobFromReader's input away: %s", v, ctx.Err()) + ctxlog.FromContext(ctx).Debugf("%s: taking CreateBlockBlobFromReader's input away: %s", v, ctx.Err()) // Our pipe might be stuck in Write(), waiting for // io.Copy() to read. If so, un-stick it. This means // CreateBlockBlobFromReader will get corrupt data, @@ -433,7 +383,7 @@ func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) err go io.Copy(ioutil.Discard, bufr) // CloseWithError() will return once pending I/O is done. bufw.CloseWithError(ctx.Err()) - theConfig.debugLogf("%s: abandoning CreateBlockBlobFromReader goroutine", v) + ctxlog.FromContext(ctx).Debugf("%s: abandoning CreateBlockBlobFromReader goroutine", v) return ctx.Err() case err := <-errChan: return err @@ -442,7 +392,7 @@ func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) err // Touch updates the last-modified property of a block blob. func (v *AzureBlobVolume) Touch(loc string) error { - if v.ReadOnly { + if v.volume.ReadOnly { return MethodDisabledError } trashed, metadata, err := v.checkTrashed(loc) @@ -481,8 +431,8 @@ func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error { Prefix: prefix, Include: &storage.IncludeBlobDataset{Metadata: true}, } - for { - resp, err := v.container.ListBlobs(params) + for page := 1; ; page++ { + resp, err := v.listBlobs(page, params) if err != nil { return err } @@ -491,7 +441,7 @@ func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error { continue } modtime := time.Time(b.Properties.LastModified) - if b.Properties.ContentLength == 0 && modtime.Add(azureWriteRaceInterval).After(time.Now()) { + if b.Properties.ContentLength == 0 && modtime.Add(v.WriteRaceInterval.Duration()).After(time.Now()) { // A new zero-length blob is probably // just a new non-empty blob that // hasn't committed its data yet (see @@ -512,12 +462,27 @@ func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error { } } +// call v.container.ListBlobs, retrying if needed. +func (v *AzureBlobVolume) listBlobs(page int, params storage.ListBlobsParameters) (resp storage.BlobListResponse, err error) { + for i := 0; i < v.ListBlobsMaxAttempts; i++ { + resp, err = v.container.ListBlobs(params) + err = v.translateError(err) + if err == VolumeBusyError { + v.logger.Printf("ListBlobs: will retry page %d in %s after error: %s", page, v.ListBlobsRetryDelay, err) + time.Sleep(time.Duration(v.ListBlobsRetryDelay)) + continue + } else { + break + } + } + return +} + // Trash a Keep block. func (v *AzureBlobVolume) Trash(loc string) error { - if v.ReadOnly { + if v.volume.ReadOnly && !v.volume.AllowTrashWhenReadOnly { return MethodDisabledError } - // Ideally we would use If-Unmodified-Since, but that // particular condition seems to be ignored by Azure. Instead, // we get the Etag before checking Mtime, and use If-Match to @@ -529,12 +494,12 @@ func (v *AzureBlobVolume) Trash(loc string) error { } 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 TrashLifetime == 0, just delete it - if theConfig.TrashLifetime == 0 { + // If BlobTrashLifetime == 0, just delete it + if v.cluster.Collections.BlobTrashLifetime == 0 { return v.container.DeleteBlob(loc, &storage.DeleteBlobOptions{ IfMatch: props.Etag, }) @@ -542,7 +507,7 @@ func (v *AzureBlobVolume) Trash(loc string) error { // Otherwise, mark as trash return v.container.SetBlobMetadata(loc, storage.BlobMetadata{ - "expires_at": fmt.Sprintf("%d", time.Now().Add(theConfig.TrashLifetime.Duration()).Unix()), + "expires_at": fmt.Sprintf("%d", time.Now().Add(v.cluster.Collections.BlobTrashLifetime.Duration()).Unix()), }, &storage.SetBlobMetadataOptions{ IfMatch: props.Etag, }) @@ -580,23 +545,6 @@ func (v *AzureBlobVolume) String() string { return fmt.Sprintf("azure-storage-container:%+q", v.ContainerName) } -// Writable returns true, unless the -readonly flag was on when the -// volume was added. -func (v *AzureBlobVolume) Writable() bool { - return !v.ReadOnly -} - -// Replication returns the replication level of the container, as -// specified by the -azure-storage-replication argument. -func (v *AzureBlobVolume) Replication() int { - return v.AzureReplication -} - -// GetStorageClasses implements Volume -func (v *AzureBlobVolume) GetStorageClasses() []string { - return v.StorageClasses -} - // If possible, translate an Azure SDK error to a recognizable error // like os.ErrNotExist. func (v *AzureBlobVolume) translateError(err error) error { @@ -609,6 +557,9 @@ func (v *AzureBlobVolume) translateError(err error) error { case strings.Contains(err.Error(), "Not Found"): // "storage: service returned without a response body (404 Not Found)" return os.ErrNotExist + case strings.Contains(err.Error(), "ErrorCode=BlobNotFound"): + // "storage: service returned error: StatusCode=404, ErrorCode=BlobNotFound, ErrorMessage=The specified blob does not exist.\n..." + return os.ErrNotExist default: return err } @@ -620,7 +571,7 @@ func (v *AzureBlobVolume) isKeepBlock(s string) bool { return keepBlockRegexp.MatchString(s) } -// EmptyTrash looks for trashed blocks that exceeded TrashLifetime +// EmptyTrash looks for trashed blocks that exceeded BlobTrashLifetime // and deletes them from the volume. func (v *AzureBlobVolume) EmptyTrash() { var bytesDeleted, bytesInTrash int64 @@ -637,7 +588,7 @@ func (v *AzureBlobVolume) EmptyTrash() { expiresAt, err := strconv.ParseInt(b.Metadata["expires_at"], 10, 64) if err != nil { - log.Printf("EmptyTrash: ParseInt(%v): %v", b.Metadata["expires_at"], err) + v.logger.Printf("EmptyTrash: ParseInt(%v): %v", b.Metadata["expires_at"], err) return } @@ -649,7 +600,7 @@ func (v *AzureBlobVolume) EmptyTrash() { IfMatch: b.Properties.Etag, }) if err != nil { - log.Printf("EmptyTrash: DeleteBlob(%v): %v", b.Name, err) + v.logger.Printf("EmptyTrash: DeleteBlob(%v): %v", b.Name, err) return } atomic.AddInt64(&blocksDeleted, 1) @@ -657,8 +608,8 @@ func (v *AzureBlobVolume) EmptyTrash() { } var wg sync.WaitGroup - todo := make(chan storage.Blob, theConfig.EmptyTrashWorkers) - for i := 0; i < 1 || i < theConfig.EmptyTrashWorkers; i++ { + todo := make(chan storage.Blob, v.cluster.Collections.BlobDeleteConcurrency) + for i := 0; i < v.cluster.Collections.BlobDeleteConcurrency; i++ { wg.Add(1) go func() { defer wg.Done() @@ -669,10 +620,10 @@ func (v *AzureBlobVolume) EmptyTrash() { } params := storage.ListBlobsParameters{Include: &storage.IncludeBlobDataset{Metadata: true}} - for { - resp, err := v.container.ListBlobs(params) + for page := 1; ; page++ { + resp, err := v.listBlobs(page, params) if err != nil { - log.Printf("EmptyTrash: ListBlobs: %v", err) + v.logger.Printf("EmptyTrash: ListBlobs: %v", err) break } for _, b := range resp.Blobs { @@ -686,7 +637,7 @@ func (v *AzureBlobVolume) EmptyTrash() { close(todo) wg.Wait() - 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.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) } // InternalStats returns bucket I/O and API call counters. @@ -715,7 +666,6 @@ func (s *azureBlobStats) TickErr(err error) { if err, ok := err.(storage.AzureStorageServiceError); ok { errType = errType + fmt.Sprintf(" %d (%s)", err.StatusCode, err.Code) } - log.Printf("errType %T, err %s", err, err) s.statsTicker.TickErr(err, errType) } @@ -727,6 +677,7 @@ type azureContainer struct { } func (c *azureContainer) Exists() (bool, error) { + c.stats.TickOps("exists") c.stats.Tick(&c.stats.Ops) ok, err := c.ctr.Exists() c.stats.TickErr(err) @@ -734,6 +685,7 @@ func (c *azureContainer) Exists() (bool, error) { } func (c *azureContainer) GetBlobMetadata(bname string) (storage.BlobMetadata, error) { + c.stats.TickOps("get_metadata") c.stats.Tick(&c.stats.Ops, &c.stats.GetMetadataOps) b := c.ctr.GetBlobReference(bname) err := b.GetMetadata(nil) @@ -742,6 +694,7 @@ func (c *azureContainer) GetBlobMetadata(bname string) (storage.BlobMetadata, er } func (c *azureContainer) GetBlobProperties(bname string) (*storage.BlobProperties, error) { + c.stats.TickOps("get_properties") c.stats.Tick(&c.stats.Ops, &c.stats.GetPropertiesOps) b := c.ctr.GetBlobReference(bname) err := b.GetProperties(nil) @@ -750,6 +703,7 @@ func (c *azureContainer) GetBlobProperties(bname string) (*storage.BlobPropertie } func (c *azureContainer) GetBlob(bname string) (io.ReadCloser, error) { + c.stats.TickOps("get") c.stats.Tick(&c.stats.Ops, &c.stats.GetOps) b := c.ctr.GetBlobReference(bname) rdr, err := b.Get(nil) @@ -758,6 +712,7 @@ func (c *azureContainer) GetBlob(bname string) (io.ReadCloser, error) { } func (c *azureContainer) GetBlobRange(bname string, start, end int, opts *storage.GetBlobOptions) (io.ReadCloser, error) { + c.stats.TickOps("get_range") c.stats.Tick(&c.stats.Ops, &c.stats.GetRangeOps) b := c.ctr.GetBlobReference(bname) rdr, err := b.GetRange(&storage.GetBlobRangeOptions{ @@ -785,6 +740,7 @@ func (r *readerWithAzureLen) Len() int { } func (c *azureContainer) CreateBlockBlobFromReader(bname string, size int, rdr io.Reader, opts *storage.PutBlobOptions) error { + c.stats.TickOps("create") c.stats.Tick(&c.stats.Ops, &c.stats.CreateOps) if size != 0 { rdr = &readerWithAzureLen{ @@ -799,6 +755,7 @@ func (c *azureContainer) CreateBlockBlobFromReader(bname string, size int, rdr i } func (c *azureContainer) SetBlobMetadata(bname string, m storage.BlobMetadata, opts *storage.SetBlobMetadataOptions) error { + c.stats.TickOps("set_metadata") c.stats.Tick(&c.stats.Ops, &c.stats.SetMetadataOps) b := c.ctr.GetBlobReference(bname) b.Metadata = m @@ -808,6 +765,7 @@ func (c *azureContainer) SetBlobMetadata(bname string, m storage.BlobMetadata, o } func (c *azureContainer) ListBlobs(params storage.ListBlobsParameters) (storage.BlobListResponse, error) { + c.stats.TickOps("list") c.stats.Tick(&c.stats.Ops, &c.stats.ListOps) resp, err := c.ctr.ListBlobs(params) c.stats.TickErr(err) @@ -815,6 +773,7 @@ func (c *azureContainer) ListBlobs(params storage.ListBlobsParameters) (storage. } func (c *azureContainer) DeleteBlob(bname string, opts *storage.DeleteBlobOptions) error { + c.stats.TickOps("delete") c.stats.Tick(&c.stats.Ops, &c.stats.DelOps) b := c.ctr.GetBlobReference(bname) err := b.Delete(opts)