Merge branch 'master' into 13937-keepstore-prometheus
[arvados.git] / services / keepstore / azure_blob_volume.go
index a28f8b7f732ba1c2df8e862b6fceb9d9b7aaf604..66956b89ee83928261bc67dcedba075c20b78397 100644 (file)
@@ -1,3 +1,7 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
 package main
 
 import (
@@ -14,11 +18,12 @@ import (
        "strconv"
        "strings"
        "sync"
+       "sync/atomic"
        "time"
 
        "git.curoverse.com/arvados.git/sdk/go/arvados"
-       log "github.com/Sirupsen/logrus"
-       "github.com/curoverse/azure-sdk-for-go/storage"
+       "github.com/Azure/azure-sdk-for-go/storage"
+       "github.com/prometheus/client_golang/prometheus"
 )
 
 const azureDefaultRequestTimeout = arvados.Duration(10 * time.Minute)
@@ -102,9 +107,18 @@ type AzureBlobVolume struct {
        AzureReplication      int
        ReadOnly              bool
        RequestTimeout        arvados.Duration
+       StorageClasses        []string
+
+       azClient  storage.Client
+       container *azureContainer
+}
 
-       azClient storage.Client
-       bsClient *azureBlobClient
+// 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)
 }
 
 // Examples implements VolumeWithExamples.
@@ -117,6 +131,14 @@ func (*AzureBlobVolume) Examples() []Volume {
                        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,
+               },
        }
 }
 
@@ -126,7 +148,7 @@ func (v *AzureBlobVolume) Type() string {
 }
 
 // Start implements Volume.
-func (v *AzureBlobVolume) Start() error {
+func (v *AzureBlobVolume) Start(opsCounters, errCounters, ioBytes *prometheus.CounterVec) error {
        if v.ContainerName == "" {
                return errors.New("no container name given")
        }
@@ -144,6 +166,7 @@ func (v *AzureBlobVolume) Start() error {
        if err != nil {
                return fmt.Errorf("creating Azure storage client: %s", err)
        }
+       v.azClient.Sender = &singleSender{}
 
        if v.RequestTimeout == 0 {
                v.RequestTimeout = azureDefaultRequestTimeout
@@ -152,23 +175,32 @@ func (v *AzureBlobVolume) Start() error {
                Timeout: time.Duration(v.RequestTimeout),
        }
        bs := v.azClient.GetBlobService()
-       v.bsClient = &azureBlobClient{
-               client: &bs,
+       v.container = &azureContainer{
+               ctr: bs.GetContainerReference(v.ContainerName),
        }
 
-       ok, err := v.bsClient.ContainerExists(v.ContainerName)
-       if err != nil {
+       if ok, err := v.container.Exists(); err != nil {
                return err
-       }
-       if !ok {
+       } else if !ok {
                return fmt.Errorf("Azure container %q does not exist", v.ContainerName)
        }
+       // Set up prometheus metrics
+       lbls := prometheus.Labels{"device_id": v.DeviceID()}
+       v.container.stats.opsCounters = opsCounters.MustCurryWith(lbls)
+       v.container.stats.errCounters = errCounters.MustCurryWith(lbls)
+       v.container.stats.ioBytes = ioBytes.MustCurryWith(lbls)
+
        return nil
 }
 
+// DeviceID returns a globally unique ID for the storage container.
+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) {
-       metadata, err := v.bsClient.GetBlobMetadata(v.ContainerName, loc)
+       metadata, err := v.container.GetBlobMetadata(loc)
        if err != nil {
                return false, metadata, v.translateError(err)
        }
@@ -235,7 +267,7 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int,
        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.bsClient.GetBlobProperties(v.ContainerName, loc)
+               props, err := v.container.GetBlobProperties(loc)
                if err != nil {
                        return 0, v.translateError(err)
                }
@@ -277,9 +309,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.bsClient.GetBlob(v.ContainerName, loc)
+                                       rdr, err = v.container.GetBlob(loc)
                                } else {
-                                       rdr, err = v.bsClient.GetBlobRange(v.ContainerName, loc, fmt.Sprintf("%d-%d", startPos, endPos-1), nil)
+                                       rdr, err = v.container.GetBlobRange(loc, startPos, endPos-1, nil)
                                }
                        }()
                        select {
@@ -348,7 +380,7 @@ func (v *AzureBlobVolume) Compare(ctx context.Context, loc string, expect []byte
        gotRdr := make(chan struct{})
        go func() {
                defer close(gotRdr)
-               rdr, err = v.bsClient.GetBlob(v.ContainerName, loc)
+               rdr, err = v.container.GetBlob(loc)
        }()
        select {
        case <-ctx.Done():
@@ -385,7 +417,17 @@ func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) err
        }()
        errChan := make(chan error)
        go func() {
-               errChan <- v.bsClient.CreateBlockBlobFromReader(v.ContainerName, loc, uint64(len(block)), bufr, nil)
+               var body io.Reader = bufr
+               if len(block) == 0 {
+                       // We must send a "Content-Length: 0" header,
+                       // but the http client interprets
+                       // ContentLength==0 as "unknown" unless it can
+                       // confirm by introspection that Body will
+                       // read 0 bytes.
+                       body = http.NoBody
+                       bufr.Close()
+               }
+               errChan <- v.container.CreateBlockBlobFromReader(loc, len(block), body, nil)
        }()
        select {
        case <-ctx.Done():
@@ -418,8 +460,8 @@ func (v *AzureBlobVolume) Touch(loc string) error {
                return os.ErrNotExist
        }
 
-       metadata["touch"] = fmt.Sprintf("%d", time.Now())
-       return v.bsClient.SetBlobMetadata(v.ContainerName, loc, metadata, nil)
+       metadata["touch"] = fmt.Sprintf("%d", time.Now().Unix())
+       return v.container.SetBlobMetadata(loc, metadata, nil)
 }
 
 // Mtime returns the last-modified property of a block blob.
@@ -432,11 +474,11 @@ func (v *AzureBlobVolume) Mtime(loc string) (time.Time, error) {
                return time.Time{}, os.ErrNotExist
        }
 
-       props, err := v.bsClient.GetBlobProperties(v.ContainerName, loc)
+       props, err := v.container.GetBlobProperties(loc)
        if err != nil {
                return time.Time{}, err
        }
-       return time.Parse(time.RFC1123, props.LastModified)
+       return time.Time(props.LastModified), nil
 }
 
 // IndexTo writes a list of Keep blocks that are stored in the
@@ -444,22 +486,19 @@ func (v *AzureBlobVolume) Mtime(loc string) (time.Time, error) {
 func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error {
        params := storage.ListBlobsParameters{
                Prefix:  prefix,
-               Include: "metadata",
+               Include: &storage.IncludeBlobDataset{Metadata: true},
        }
        for {
-               resp, err := v.bsClient.ListBlobs(v.ContainerName, params)
+               resp, err := v.container.ListBlobs(params)
                if err != nil {
                        return err
                }
                for _, b := range resp.Blobs {
-                       t, err := time.Parse(time.RFC1123, b.Properties.LastModified)
-                       if err != nil {
-                               return err
-                       }
                        if !v.isKeepBlock(b.Name) {
                                continue
                        }
-                       if b.Properties.ContentLength == 0 && t.Add(azureWriteRaceInterval).After(time.Now()) {
+                       modtime := time.Time(b.Properties.LastModified)
+                       if b.Properties.ContentLength == 0 && modtime.Add(azureWriteRaceInterval).After(time.Now()) {
                                // A new zero-length blob is probably
                                // just a new non-empty blob that
                                // hasn't committed its data yet (see
@@ -471,7 +510,7 @@ func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error {
                                // Trashed blob; exclude it from response
                                continue
                        }
-                       fmt.Fprintf(writer, "%s+%d %d\n", b.Name, b.Properties.ContentLength, t.UnixNano())
+                       fmt.Fprintf(writer, "%s+%d %d\n", b.Name, b.Properties.ContentLength, modtime.UnixNano())
                }
                if resp.NextMarker == "" {
                        return nil
@@ -491,7 +530,7 @@ func (v *AzureBlobVolume) Trash(loc string) error {
        // we get the Etag before checking Mtime, and use If-Match to
        // ensure we don't delete data if Put() or Touch() happens
        // between our calls to Mtime() and DeleteBlob().
-       props, err := v.bsClient.GetBlobProperties(v.ContainerName, loc)
+       props, err := v.container.GetBlobProperties(loc)
        if err != nil {
                return err
        }
@@ -503,16 +542,16 @@ func (v *AzureBlobVolume) Trash(loc string) error {
 
        // If TrashLifetime == 0, just delete it
        if theConfig.TrashLifetime == 0 {
-               return v.bsClient.DeleteBlob(v.ContainerName, loc, map[string]string{
-                       "If-Match": props.Etag,
+               return v.container.DeleteBlob(loc, &storage.DeleteBlobOptions{
+                       IfMatch: props.Etag,
                })
        }
 
        // Otherwise, mark as trash
-       return v.bsClient.SetBlobMetadata(v.ContainerName, loc, map[string]string{
+       return v.container.SetBlobMetadata(loc, storage.BlobMetadata{
                "expires_at": fmt.Sprintf("%d", time.Now().Add(theConfig.TrashLifetime.Duration()).Unix()),
-       }, map[string]string{
-               "If-Match": props.Etag,
+       }, &storage.SetBlobMetadataOptions{
+               IfMatch: props.Etag,
        })
 }
 
@@ -520,7 +559,7 @@ func (v *AzureBlobVolume) Trash(loc string) error {
 // Delete the expires_at metadata attribute
 func (v *AzureBlobVolume) Untrash(loc string) error {
        // if expires_at does not exist, return NotFoundError
-       metadata, err := v.bsClient.GetBlobMetadata(v.ContainerName, loc)
+       metadata, err := v.container.GetBlobMetadata(loc)
        if err != nil {
                return v.translateError(err)
        }
@@ -530,7 +569,7 @@ func (v *AzureBlobVolume) Untrash(loc string) error {
 
        // reset expires_at metadata attribute
        metadata["expires_at"] = ""
-       err = v.bsClient.SetBlobMetadata(v.ContainerName, loc, metadata, nil)
+       err = v.container.SetBlobMetadata(loc, metadata, nil)
        return v.translateError(err)
 }
 
@@ -560,12 +599,20 @@ 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 {
        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
        case strings.Contains(err.Error(), "Not Found"):
                // "storage: service returned without a response body (404 Not Found)"
                return os.ErrNotExist
@@ -584,56 +631,74 @@ func (v *AzureBlobVolume) isKeepBlock(s string) bool {
 // and deletes them from the volume.
 func (v *AzureBlobVolume) EmptyTrash() {
        var bytesDeleted, bytesInTrash int64
-       var blocksDeleted, blocksInTrash int
-       params := storage.ListBlobsParameters{Include: "metadata"}
+       var blocksDeleted, blocksInTrash int64
 
-       for {
-               resp, err := v.bsClient.ListBlobs(v.ContainerName, params)
+       doBlob := func(b storage.Blob) {
+               // Check whether the block is flagged as trash
+               if b.Metadata["expires_at"] == "" {
+                       return
+               }
+
+               atomic.AddInt64(&blocksInTrash, 1)
+               atomic.AddInt64(&bytesInTrash, b.Properties.ContentLength)
+
+               expiresAt, err := strconv.ParseInt(b.Metadata["expires_at"], 10, 64)
                if err != nil {
-                       log.Printf("EmptyTrash: ListBlobs: %v", err)
-                       break
+                       log.Printf("EmptyTrash: ParseInt(%v): %v", b.Metadata["expires_at"], err)
+                       return
                }
-               for _, b := range resp.Blobs {
-                       // Check if the block is expired
-                       if b.Metadata["expires_at"] == "" {
-                               continue
-                       }
 
-                       blocksInTrash++
-                       bytesInTrash += b.Properties.ContentLength
+               if expiresAt > time.Now().Unix() {
+                       return
+               }
 
-                       expiresAt, err := strconv.ParseInt(b.Metadata["expires_at"], 10, 64)
-                       if err != nil {
-                               log.Printf("EmptyTrash: ParseInt(%v): %v", b.Metadata["expires_at"], err)
-                               continue
-                       }
+               err = v.container.DeleteBlob(b.Name, &storage.DeleteBlobOptions{
+                       IfMatch: b.Properties.Etag,
+               })
+               if err != nil {
+                       log.Printf("EmptyTrash: DeleteBlob(%v): %v", b.Name, err)
+                       return
+               }
+               atomic.AddInt64(&blocksDeleted, 1)
+               atomic.AddInt64(&bytesDeleted, b.Properties.ContentLength)
+       }
 
-                       if expiresAt > time.Now().Unix() {
-                               continue
+       var wg sync.WaitGroup
+       todo := make(chan storage.Blob, theConfig.EmptyTrashWorkers)
+       for i := 0; i < 1 || i < theConfig.EmptyTrashWorkers; i++ {
+               wg.Add(1)
+               go func() {
+                       defer wg.Done()
+                       for b := range todo {
+                               doBlob(b)
                        }
+               }()
+       }
 
-                       err = v.bsClient.DeleteBlob(v.ContainerName, b.Name, map[string]string{
-                               "If-Match": b.Properties.Etag,
-                       })
-                       if err != nil {
-                               log.Printf("EmptyTrash: DeleteBlob(%v): %v", b.Name, err)
-                               continue
-                       }
-                       blocksDeleted++
-                       bytesDeleted += b.Properties.ContentLength
+       params := storage.ListBlobsParameters{Include: &storage.IncludeBlobDataset{Metadata: true}}
+       for {
+               resp, err := v.container.ListBlobs(params)
+               if err != nil {
+                       log.Printf("EmptyTrash: ListBlobs: %v", err)
+                       break
+               }
+               for _, b := range resp.Blobs {
+                       todo <- b
                }
                if resp.NextMarker == "" {
                        break
                }
                params.Marker = resp.NextMarker
        }
+       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)
 }
 
 // InternalStats returns bucket I/O and API call counters.
 func (v *AzureBlobVolume) InternalStats() interface{} {
-       return &v.bsClient.stats
+       return &v.container.stats
 }
 
 type azureBlobStats struct {
@@ -661,73 +726,114 @@ func (s *azureBlobStats) TickErr(err error) {
        s.statsTicker.TickErr(err, errType)
 }
 
-// azureBlobClient wraps storage.BlobStorageClient in order to count
-// I/O and API usage stats.
-type azureBlobClient struct {
-       client *storage.BlobStorageClient
-       stats  azureBlobStats
+// azureContainer wraps storage.Container in order to count I/O and
+// API usage stats.
+type azureContainer struct {
+       ctr   *storage.Container
+       stats azureBlobStats
 }
 
-func (c *azureBlobClient) ContainerExists(cname string) (bool, error) {
+func (c *azureContainer) Exists() (bool, error) {
+       c.stats.TickOps("exists")
        c.stats.Tick(&c.stats.Ops)
-       ok, err := c.client.ContainerExists(cname)
+       ok, err := c.ctr.Exists()
        c.stats.TickErr(err)
        return ok, err
 }
 
-func (c *azureBlobClient) GetBlobMetadata(cname, bname string) (map[string]string, error) {
+func (c *azureContainer) GetBlobMetadata(bname string) (storage.BlobMetadata, error) {
+       c.stats.TickOps("get_metadata")
        c.stats.Tick(&c.stats.Ops, &c.stats.GetMetadataOps)
-       m, err := c.client.GetBlobMetadata(cname, bname)
+       b := c.ctr.GetBlobReference(bname)
+       err := b.GetMetadata(nil)
        c.stats.TickErr(err)
-       return m, err
+       return b.Metadata, err
 }
 
-func (c *azureBlobClient) GetBlobProperties(cname, bname string) (*storage.BlobProperties, error) {
+func (c *azureContainer) GetBlobProperties(bname string) (*storage.BlobProperties, error) {
+       c.stats.TickOps("get_properties")
        c.stats.Tick(&c.stats.Ops, &c.stats.GetPropertiesOps)
-       p, err := c.client.GetBlobProperties(cname, bname)
+       b := c.ctr.GetBlobReference(bname)
+       err := b.GetProperties(nil)
        c.stats.TickErr(err)
-       return p, err
+       return &b.Properties, err
 }
 
-func (c *azureBlobClient) GetBlob(cname, bname string) (io.ReadCloser, error) {
+func (c *azureContainer) GetBlob(bname string) (io.ReadCloser, error) {
+       c.stats.TickOps("get")
        c.stats.Tick(&c.stats.Ops, &c.stats.GetOps)
-       rdr, err := c.client.GetBlob(cname, bname)
+       b := c.ctr.GetBlobReference(bname)
+       rdr, err := b.Get(nil)
        c.stats.TickErr(err)
        return NewCountingReader(rdr, c.stats.TickInBytes), err
 }
 
-func (c *azureBlobClient) GetBlobRange(cname, bname, byterange string, hdrs map[string]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)
-       rdr, err := c.client.GetBlobRange(cname, bname, byterange, hdrs)
+       b := c.ctr.GetBlobReference(bname)
+       rdr, err := b.GetRange(&storage.GetBlobRangeOptions{
+               Range: &storage.BlobRange{
+                       Start: uint64(start),
+                       End:   uint64(end),
+               },
+               GetBlobOptions: opts,
+       })
        c.stats.TickErr(err)
        return NewCountingReader(rdr, c.stats.TickInBytes), err
 }
 
-func (c *azureBlobClient) CreateBlockBlobFromReader(cname, bname string, size uint64, rdr io.Reader, hdrs map[string]string) error {
+// If we give it an io.Reader that doesn't also have a Len() int
+// method, the Azure SDK determines data size by copying the data into
+// a new buffer, which is not a good use of memory.
+type readerWithAzureLen struct {
+       io.Reader
+       len int
+}
+
+// Len satisfies the private lener interface in azure-sdk-for-go.
+func (r *readerWithAzureLen) Len() int {
+       return r.len
+}
+
+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)
-       rdr = NewCountingReader(rdr, c.stats.TickOutBytes)
-       err := c.client.CreateBlockBlobFromReader(cname, bname, size, rdr, hdrs)
+       if size != 0 {
+               rdr = &readerWithAzureLen{
+                       Reader: NewCountingReader(rdr, c.stats.TickOutBytes),
+                       len:    size,
+               }
+       }
+       b := c.ctr.GetBlobReference(bname)
+       err := b.CreateBlockBlobFromReader(rdr, opts)
        c.stats.TickErr(err)
        return err
 }
 
-func (c *azureBlobClient) SetBlobMetadata(cname, bname string, m, hdrs map[string]string) error {
+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)
-       err := c.client.SetBlobMetadata(cname, bname, m, hdrs)
+       b := c.ctr.GetBlobReference(bname)
+       b.Metadata = m
+       err := b.SetMetadata(opts)
        c.stats.TickErr(err)
        return err
 }
 
-func (c *azureBlobClient) ListBlobs(cname string, params storage.ListBlobsParameters) (storage.BlobListResponse, error) {
+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.client.ListBlobs(cname, params)
+       resp, err := c.ctr.ListBlobs(params)
        c.stats.TickErr(err)
        return resp, err
 }
 
-func (c *azureBlobClient) DeleteBlob(cname, bname string, hdrs map[string]string) error {
+func (c *azureContainer) DeleteBlob(bname string, opts *storage.DeleteBlobOptions) error {
+       c.stats.TickOps("delete")
        c.stats.Tick(&c.stats.Ops, &c.stats.DelOps)
-       err := c.client.DeleteBlob(cname, bname, hdrs)
+       b := c.ctr.GetBlobReference(bname)
+       err := b.Delete(opts)
        c.stats.TickErr(err)
        return err
 }