2960: Move streaming from volume to keepstore layer.
[arvados.git] / services / keepstore / azure_blob_volume.go
index 4f7339facf4ace001ac886a5076afc217e040c18..2c8a79350c86b02e08eea2007c58a8f2e632ca47 100644 (file)
@@ -2,16 +2,14 @@
 //
 // SPDX-License-Identifier: AGPL-3.0
 
-package main
+package keepstore
 
 import (
-       "bytes"
        "context"
+       "encoding/json"
        "errors"
-       "flag"
        "fmt"
        "io"
-       "io/ioutil"
        "net/http"
        "os"
        "regexp"
@@ -21,155 +19,50 @@ 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 readKeyFromFile(file string) (string, error) {
-       buf, err := ioutil.ReadFile(file)
-       if err != nil {
-               return "", errors.New("reading key from " + file + ": " + err.Error())
-       }
-       accountKey := strings.TrimSpace(string(buf))
-       if accountKey == "" {
-               return "", errors.New("empty account key in " + file)
-       }
-       return accountKey, nil
-}
-
-type azureVolumeAdder struct {
-       *Config
-}
-
-// String implements flag.Value
-func (s *azureVolumeAdder) String() string {
-       return "-"
-}
-
-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,
-       })
-       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))
-}
-
-// 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
-
-       azClient  storage.Client
-       container *azureContainer
-}
-
-// singleSender is a single-attempt storage.Sender.
-type singleSender struct{}
-
-// Send performs req exactly once.
-func (*singleSender) Send(c *storage.Client, req *http.Request) (resp *http.Response, err error) {
-       return c.HTTPClient.Do(req)
+       driver["Azure"] = newAzureBlobVolume
 }
 
-// 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,
-               },
+func newAzureBlobVolume(params newVolumeParams) (volume, error) {
+       v := &azureBlobVolume{
+               RequestTimeout:    azureDefaultRequestTimeout,
+               WriteRaceInterval: azureDefaultWriteRaceInterval,
+               WriteRacePollTime: azureDefaultWriteRacePollTime,
+               cluster:           params.Cluster,
+               volume:            params.ConfigVolume,
+               logger:            params.Logger,
+               metrics:           params.MetricsVecs,
+               bufferPool:        params.BufferPool,
        }
-}
-
-// 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")
+       err := json.Unmarshal(params.ConfigVolume.DriverParameters, &v)
+       if err != nil {
+               return nil, err
        }
-       if v.StorageAccountName == "" || v.StorageAccountKeyFile == "" {
-               return errors.New("StorageAccountName and StorageAccountKeyFile must be given")
+       if v.ListBlobsRetryDelay == 0 {
+               v.ListBlobsRetryDelay = azureDefaultListBlobsRetryDelay
        }
-       accountKey, err := readKeyFromFile(v.StorageAccountKeyFile)
-       if err != nil {
-               return err
+       if v.ListBlobsMaxAttempts == 0 {
+               v.ListBlobsMaxAttempts = azureDefaultListBlobsMaxAttempts
        }
        if v.StorageBaseURL == "" {
                v.StorageBaseURL = storage.DefaultBaseURL
        }
-       v.azClient, err = storage.NewClient(v.StorageAccountName, accountKey, v.StorageBaseURL, storage.DefaultAPIVersion, true)
+       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 fmt.Errorf("creating Azure storage client: %s", err)
+               return nil, fmt.Errorf("creating Azure storage client: %s", err)
        }
+       v.azClient = azc
        v.azClient.Sender = &singleSender{}
-
-       if v.RequestTimeout == 0 {
-               v.RequestTimeout = azureDefaultRequestTimeout
-       }
        v.azClient.HTTPClient = &http.Client{
                Timeout: time.Duration(v.RequestTimeout),
        }
@@ -179,20 +72,65 @@ func (v *AzureBlobVolume) Start() error {
        }
 
        if ok, err := v.container.Exists(); err != nil {
-               return err
+               return nil, err
        } else if !ok {
-               return fmt.Errorf("Azure container %q does not exist", v.ContainerName)
+               return nil, fmt.Errorf("Azure container %q does not exist: %s", v.ContainerName, err)
        }
+       return v, v.check()
+}
+
+func (v *azureBlobVolume) check() error {
+       lbls := prometheus.Labels{"device_id": v.DeviceID()}
+       v.container.stats.opsCounters, v.container.stats.errCounters, v.container.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
        return nil
 }
 
+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
+       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
+       bufferPool *bufferPool
+       azClient   storage.Client
+       container  *azureContainer
+}
+
+// singleSender is a single-attempt storage.Sender.
+type singleSender struct{}
+
+// Send performs req exactly once.
+func (*singleSender) Send(c *storage.Client, req *http.Request) (resp *http.Response, err error) {
+       return c.HTTPClient.Do(req)
+}
+
 // DeviceID returns a globally unique ID for the storage container.
-func (v *AzureBlobVolume) DeviceID() string {
+func (v *azureBlobVolume) DeviceID() string {
        return "azure://" + v.StorageBaseURL + "/" + v.StorageAccountName + "/" + v.ContainerName
 }
 
 // Return true if expires_at metadata attribute is found on the block
-func (v *AzureBlobVolume) checkTrashed(loc string) (bool, map[string]string, error) {
+func (v *azureBlobVolume) checkTrashed(loc string) (bool, map[string]string, error) {
        metadata, err := v.container.GetBlobMetadata(loc)
        if err != nil {
                return false, metadata, v.translateError(err)
@@ -203,81 +141,93 @@ func (v *AzureBlobVolume) checkTrashed(loc string) (bool, map[string]string, err
        return false, metadata, nil
 }
 
-// Get reads a Keep block that has been stored as a block blob in the
-// container.
+// BlockRead reads a Keep block that has been stored as a block blob
+// in the container.
 //
 // If the block is younger than azureWriteRaceInterval and is
-// unexpectedly empty, assume a PutBlob operation is in progress, and
-// wait for it to finish writing.
-func (v *AzureBlobVolume) Get(ctx context.Context, loc string, buf []byte) (int, error) {
-       trashed, _, err := v.checkTrashed(loc)
+// unexpectedly empty, assume a BlockWrite operation is in progress,
+// and wait for it to finish writing.
+func (v *azureBlobVolume) BlockRead(ctx context.Context, hash string, w io.WriterAt) error {
+       trashed, _, err := v.checkTrashed(hash)
        if err != nil {
-               return 0, err
+               return err
        }
        if trashed {
-               return 0, os.ErrNotExist
+               return os.ErrNotExist
        }
+       buf, err := v.bufferPool.GetContext(ctx)
+       if err != nil {
+               return err
+       }
+       defer v.bufferPool.Put(buf)
        var deadline time.Time
-       haveDeadline := false
-       size, err := v.get(ctx, loc, buf)
-       for err == nil && size == 0 && loc != "d41d8cd98f00b204e9800998ecf8427e" {
+       wrote, err := v.get(ctx, hash, w)
+       for err == nil && wrote == 0 && hash != "d41d8cd98f00b204e9800998ecf8427e" {
                // Seeing a brand new empty block probably means we're
                // in a race with CreateBlob, which under the hood
                // (apparently) does "CreateEmpty" and "CommitData"
                // with no additional transaction locking.
-               if !haveDeadline {
-                       t, err := v.Mtime(loc)
+               if deadline.IsZero() {
+                       t, err := v.Mtime(hash)
                        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)
-                       haveDeadline = true
+                       ctxlog.FromContext(ctx).Printf("Race? Block %s is 0 bytes, %s old. Polling until %s", hash, time.Since(t), deadline)
                } else if time.Now().After(deadline) {
                        break
                }
                select {
                case <-ctx.Done():
-                       return 0, ctx.Err()
-               case <-time.After(azureWriteRacePollTime):
+                       return ctx.Err()
+               case <-time.After(v.WriteRacePollTime.Duration()):
                }
-               size, err = v.get(ctx, loc, buf)
+               wrote, err = v.get(ctx, hash, w)
        }
-       if haveDeadline {
-               log.Printf("Race ended with size==%d", size)
+       if !deadline.IsZero() {
+               ctxlog.FromContext(ctx).Printf("Race ended with size==%d", wrote)
        }
-       return size, err
+       return err
 }
 
-func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, error) {
+func (v *azureBlobVolume) get(ctx context.Context, hash string, dst io.WriterAt) (int, error) {
        ctx, cancel := context.WithCancel(ctx)
        defer cancel()
-       expectSize := len(buf)
-       if azureMaxGetBytes < 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)
+
+       pieceSize := BlockSize
+       if v.MaxGetBytes > 0 && v.MaxGetBytes < BlockSize {
+               pieceSize = v.MaxGetBytes
+       }
+
+       pieces := 1
+       expectSize := BlockSize
+       sizeKnown := false
+       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(hash)
                if err != nil {
                        return 0, v.translateError(err)
                }
                if props.ContentLength > int64(BlockSize) || props.ContentLength < 0 {
-                       return 0, fmt.Errorf("block %s invalid size %d (max %d)", loc, props.ContentLength, BlockSize)
+                       return 0, fmt.Errorf("block %s invalid size %d (max %d)", hash, props.ContentLength, BlockSize)
                }
                expectSize = int(props.ContentLength)
+               pieces = (expectSize + pieceSize - 1) / pieceSize
+               sizeKnown = true
        }
 
        if expectSize == 0 {
                return 0, nil
        }
 
-       // 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 wrote atomic.Int64
        var wg sync.WaitGroup
        wg.Add(pieces)
        for p := 0; p < pieces; p++ {
@@ -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
                        }
@@ -302,9 +252,9 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int,
                        go func() {
                                defer close(gotRdr)
                                if startPos == 0 && endPos == expectSize {
-                                       rdr, err = v.container.GetBlob(loc)
+                                       rdr, err = v.container.GetBlob(hash)
                                } else {
-                                       rdr, err = v.container.GetBlobRange(loc, startPos, endPos-1, nil)
+                                       rdr, err = v.container.GetBlobRange(hash, startPos, endPos-1, nil)
                                }
                        }()
                        select {
@@ -332,86 +282,44 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int,
                                <-ctx.Done()
                                rdr.Close()
                        }()
-                       n, err := io.ReadFull(rdr, buf[startPos:endPos])
-                       if pieces == 1 && (err == io.ErrUnexpectedEOF || err == io.EOF) {
+                       n, err := io.CopyN(io.NewOffsetWriter(dst, int64(startPos)), rdr, int64(endPos-startPos))
+                       wrote.Add(n)
+                       if pieces == 1 && !sizeKnown && (err == io.ErrUnexpectedEOF || err == io.EOF) {
                                // If we don't know the actual size,
                                // and just tried reading 64 MiB, it's
                                // normal to encounter EOF.
                        } else if err != nil {
-                               if ctx.Err() == nil {
-                                       errors <- err
-                               }
+                               errors <- err
                                cancel()
                                return
                        }
-                       if p == pieces-1 {
-                               actualSize = startPos + n
-                       }
                }(p)
        }
        wg.Wait()
        close(errors)
        if len(errors) > 0 {
-               return 0, v.translateError(<-errors)
-       }
-       if ctx.Err() != nil {
-               return 0, ctx.Err()
-       }
-       return actualSize, nil
-}
-
-// Compare the given data with existing stored data.
-func (v *AzureBlobVolume) Compare(ctx context.Context, loc string, expect []byte) error {
-       trashed, _, err := v.checkTrashed(loc)
-       if err != nil {
-               return err
-       }
-       if trashed {
-               return os.ErrNotExist
-       }
-       var rdr io.ReadCloser
-       gotRdr := make(chan struct{})
-       go func() {
-               defer close(gotRdr)
-               rdr, err = v.container.GetBlob(loc)
-       }()
-       select {
-       case <-ctx.Done():
-               go func() {
-                       <-gotRdr
-                       if err == nil {
-                               rdr.Close()
-                       }
-               }()
-               return ctx.Err()
-       case <-gotRdr:
-       }
-       if err != nil {
-               return v.translateError(err)
+               return int(wrote.Load()), v.translateError(<-errors)
        }
-       defer rdr.Close()
-       return compareReaderWithBuf(ctx, rdr, expect, loc[:32])
+       return int(wrote.Load()), ctx.Err()
 }
 
-// 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 {
-               return MethodDisabledError
-       }
+// BlockWrite stores a block on the volume. If it already exists, its
+// timestamp is updated.
+func (v *azureBlobVolume) BlockWrite(ctx context.Context, hash string, data []byte) error {
        // Send the block data through a pipe, so that (if we need to)
        // we can close the pipe early and abandon our
        // CreateBlockBlobFromReader() goroutine, without worrying
-       // about CreateBlockBlobFromReader() accessing our block
+       // about CreateBlockBlobFromReader() accessing our data
        // buffer after we release it.
        bufr, bufw := io.Pipe()
        go func() {
-               io.Copy(bufw, bytes.NewReader(block))
+               bufw.Write(data)
                bufw.Close()
        }()
-       errChan := make(chan error)
+       errChan := make(chan error, 1)
        go func() {
                var body io.Reader = bufr
-               if len(block) == 0 {
+               if len(data) == 0 {
                        // We must send a "Content-Length: 0" header,
                        // but the http client interprets
                        // ContentLength==0 as "unknown" unless it can
@@ -420,32 +328,26 @@ func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) err
                        body = http.NoBody
                        bufr.Close()
                }
-               errChan <- v.container.CreateBlockBlobFromReader(loc, len(block), body, nil)
+               errChan <- v.container.CreateBlockBlobFromReader(hash, len(data), body, nil)
        }()
        select {
        case <-ctx.Done():
-               theConfig.debugLogf("%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,
-               // but that's OK: the size won't match, so the write
-               // will fail.
-               go io.Copy(ioutil.Discard, bufr)
-               // CloseWithError() will return once pending I/O is done.
+               ctxlog.FromContext(ctx).Debugf("%s: taking CreateBlockBlobFromReader's input away: %s", v, ctx.Err())
+               // bufw.CloseWithError() interrupts bufw.Write() if
+               // necessary, ensuring CreateBlockBlobFromReader can't
+               // read any more of our data slice via bufr after we
+               // return.
                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
        }
 }
 
-// Touch updates the last-modified property of a block blob.
-func (v *AzureBlobVolume) Touch(loc string) error {
-       if v.ReadOnly {
-               return MethodDisabledError
-       }
-       trashed, metadata, err := v.checkTrashed(loc)
+// BlockTouch updates the last-modified property of a block blob.
+func (v *azureBlobVolume) BlockTouch(hash string) error {
+       trashed, metadata, err := v.checkTrashed(hash)
        if err != nil {
                return err
        }
@@ -454,12 +356,12 @@ func (v *AzureBlobVolume) Touch(loc string) error {
        }
 
        metadata["touch"] = fmt.Sprintf("%d", time.Now().Unix())
-       return v.container.SetBlobMetadata(loc, metadata, nil)
+       return v.container.SetBlobMetadata(hash, metadata, nil)
 }
 
 // Mtime returns the last-modified property of a block blob.
-func (v *AzureBlobVolume) Mtime(loc string) (time.Time, error) {
-       trashed, _, err := v.checkTrashed(loc)
+func (v *azureBlobVolume) Mtime(hash string) (time.Time, error) {
+       trashed, _, err := v.checkTrashed(hash)
        if err != nil {
                return time.Time{}, err
        }
@@ -467,22 +369,26 @@ func (v *AzureBlobVolume) Mtime(loc string) (time.Time, error) {
                return time.Time{}, os.ErrNotExist
        }
 
-       props, err := v.container.GetBlobProperties(loc)
+       props, err := v.container.GetBlobProperties(hash)
        if err != nil {
                return time.Time{}, err
        }
        return time.Time(props.LastModified), nil
 }
 
-// IndexTo writes a list of Keep blocks that are stored in the
+// Index writes a list of Keep blocks that are stored in the
 // container.
-func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error {
+func (v *azureBlobVolume) Index(ctx context.Context, prefix string, writer io.Writer) error {
        params := storage.ListBlobsParameters{
                Prefix:  prefix,
                Include: &storage.IncludeBlobDataset{Metadata: true},
        }
-       for {
-               resp, err := v.container.ListBlobs(params)
+       for page := 1; ; page++ {
+               err := ctx.Err()
+               if err != nil {
+                       return err
+               }
+               resp, err := v.listBlobs(page, params)
                if err != nil {
                        return err
                }
@@ -491,7 +397,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 +418,24 @@ func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error {
        }
 }
 
-// Trash a Keep block.
-func (v *AzureBlobVolume) Trash(loc string) error {
-       if v.ReadOnly {
-               return MethodDisabledError
+// 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 == errVolumeUnavailable {
+                       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) BlockTrash(loc string) error {
        // 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 +447,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,17 +460,17 @@ 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,
        })
 }
 
-// Untrash a Keep block.
-// Delete the expires_at metadata attribute
-func (v *AzureBlobVolume) Untrash(loc string) error {
+// BlockUntrash deletes the expires_at metadata attribute for the
+// specified block blob.
+func (v *azureBlobVolume) BlockUntrash(hash string) error {
        // if expires_at does not exist, return NotFoundError
-       metadata, err := v.container.GetBlobMetadata(loc)
+       metadata, err := v.container.GetBlobMetadata(hash)
        if err != nil {
                return v.translateError(err)
        }
@@ -562,53 +480,25 @@ func (v *AzureBlobVolume) Untrash(loc string) error {
 
        // reset expires_at metadata attribute
        metadata["expires_at"] = ""
-       err = v.container.SetBlobMetadata(loc, metadata, nil)
+       err = v.container.SetBlobMetadata(hash, metadata, nil)
        return v.translateError(err)
 }
 
-// Status returns a VolumeStatus struct with placeholder data.
-func (v *AzureBlobVolume) Status() *VolumeStatus {
-       return &VolumeStatus{
-               DeviceNum: 1,
-               BytesFree: BlockSize * 1000,
-               BytesUsed: 1,
-       }
-}
-
-// String returns a volume label, including the container name.
-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 {
+func (v *azureBlobVolume) translateError(err error) error {
        switch {
        case err == nil:
                return err
        case strings.Contains(err.Error(), "StatusCode=503"):
                // "storage: service returned error: StatusCode=503, ErrorCode=ServerBusy, ErrorMessage=The server is busy" (See #14804)
-               return VolumeBusyError
+               return errVolumeUnavailable
        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
        }
@@ -616,13 +506,13 @@ func (v *AzureBlobVolume) translateError(err error) error {
 
 var keepBlockRegexp = regexp.MustCompile(`^[0-9a-f]{32}$`)
 
-func (v *AzureBlobVolume) isKeepBlock(s string) bool {
+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() {
+func (v *azureBlobVolume) EmptyTrash() {
        var bytesDeleted, bytesInTrash int64
        var blocksDeleted, blocksInTrash int64
 
@@ -637,7 +527,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 +539,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 +547,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 +559,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,11 +576,11 @@ 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.DeviceID(), bytesDeleted, blocksDeleted, bytesInTrash-bytesDeleted, blocksInTrash-blocksDeleted)
 }
 
 // InternalStats returns bucket I/O and API call counters.
-func (v *AzureBlobVolume) InternalStats() interface{} {
+func (v *azureBlobVolume) InternalStats() interface{} {
        return &v.container.stats
 }
 
@@ -715,7 +605,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 +616,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 +624,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 +633,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,14 +642,16 @@ 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)
        c.stats.TickErr(err)
-       return NewCountingReader(rdr, c.stats.TickInBytes), err
+       return newCountingReader(rdr, c.stats.TickInBytes), err
 }
 
 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{
@@ -768,7 +662,7 @@ func (c *azureContainer) GetBlobRange(bname string, start, end int, opts *storag
                GetBlobOptions: opts,
        })
        c.stats.TickErr(err)
-       return NewCountingReader(rdr, c.stats.TickInBytes), err
+       return newCountingReader(rdr, c.stats.TickInBytes), err
 }
 
 // If we give it an io.Reader that doesn't also have a Len() int
@@ -785,10 +679,11 @@ 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{
-                       Reader: NewCountingReader(rdr, c.stats.TickOutBytes),
+                       Reader: newCountingReader(rdr, c.stats.TickOutBytes),
                        len:    size,
                }
        }
@@ -799,6 +694,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 +704,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 +712,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)