X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/5c1410e60eb06769207dbafd77501bf2fa45a29b..d3716fbfea120893e1a23915c5f9bcb7ca96c371:/services/keepstore/azure_blob_volume.go diff --git a/services/keepstore/azure_blob_volume.go b/services/keepstore/azure_blob_volume.go index f08cebff63..f9b383e70e 100644 --- a/services/keepstore/azure_blob_volume.go +++ b/services/keepstore/azure_blob_volume.go @@ -1,136 +1,149 @@ -package main +// Copyright (C) The Arvados Authors. All rights reserved. +// +// SPDX-License-Identifier: AGPL-3.0 + +package keepstore import ( "bytes" + "context" + "encoding/json" "errors" - "flag" "fmt" "io" "io/ioutil" - "log" + "net/http" "os" "regexp" + "strconv" "strings" "sync" + "sync/atomic" "time" - "github.com/curoverse/azure-sdk-for-go/storage" + "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" ) -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 } - return accountKey, nil -} - -type azureVolumeAdder struct { - *volumeSet -} - -func (s *azureVolumeAdder) Set(containerName string) error { - if trashLifetime != 0 { - return ErrNotImplemented + if v.ListBlobsMaxAttempts == 0 { + v.ListBlobsMaxAttempts = azureDefaultListBlobsMaxAttempts } - - if containerName == "" { - return errors.New("no container name given") + if v.StorageBaseURL == "" { + v.StorageBaseURL = storage.DefaultBaseURL } - if azureStorageAccountName == "" || azureStorageAccountKeyFile == "" { - return errors.New("-azure-storage-account-name and -azure-storage-account-key-file arguments must given before -azure-storage-container-volume") + if v.ContainerName == "" || v.StorageAccountName == "" || v.StorageAccountKey == "" { + return nil, errors.New("DriverParameters: ContainerName, StorageAccountName, and StorageAccountKey must be provided") } - accountKey, err := readKeyFromFile(azureStorageAccountKeyFile) + azc, err := storage.NewClient(v.StorageAccountName, v.StorageAccountKey, v.StorageBaseURL, storage.DefaultAPIVersion, true) if err != nil { - return err + return nil, fmt.Errorf("creating Azure storage client: %s", err) } - azClient, err := storage.NewBasicClient(azureStorageAccountName, accountKey) - if err != nil { - return errors.New("creating Azure storage client: " + err.Error()) + v.azClient = azc + v.azClient.Sender = &singleSender{} + v.azClient.HTTPClient = &http.Client{ + Timeout: time.Duration(v.RequestTimeout), } - if flagSerializeIO { - log.Print("Notice: -serialize is not supported by azure-blob-container volumes.") + bs := v.azClient.GetBlobService() + v.container = &azureContainer{ + ctr: bs.GetContainerReference(v.ContainerName), } - v := NewAzureBlobVolume(azClient, containerName, flagReadonly, azureStorageReplication) - if err := v.Check(); err != nil { - return err + + 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) } - *s.volumeSet = append(*s.volumeSet, v) - return nil + return v, v.check() } -func init() { - flag.Var(&azureVolumeAdder{&volumes}, - "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)) +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 } +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 { - azClient storage.Client - bsClient storage.BlobStorageClient - containerName string - readonly bool - replication int + 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 } -// NewAzureBlobVolume returns a new AzureBlobVolume using the given -// client and container name. The replication argument specifies the -// replication level to report when writing data. -func NewAzureBlobVolume(client storage.Client, containerName string, readonly bool, replication int) *AzureBlobVolume { - return &AzureBlobVolume{ - azClient: client, - bsClient: client.GetBlobService(), - containerName: containerName, - readonly: readonly, - replication: replication, - } +// 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) +} + +// Type implements Volume. +func (v *AzureBlobVolume) Type() string { + return "Azure" +} + +// GetDeviceID returns a globally unique ID for the storage container. +func (v *AzureBlobVolume) GetDeviceID() string { + return "azure://" + v.StorageBaseURL + "/" + v.StorageAccountName + "/" + v.ContainerName } -// Check returns nil if the volume is usable. -func (v *AzureBlobVolume) Check() error { - ok, err := v.bsClient.ContainerExists(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.container.GetBlobMetadata(loc) if err != nil { - return err + return false, metadata, v.translateError(err) } - if !ok { - return errors.New("container does not exist") + if metadata["expires_at"] != "" { + return true, metadata, nil } - return nil + return false, metadata, nil } // Get reads a Keep block that has been stored as a block blob in the @@ -139,11 +152,18 @@ func (v *AzureBlobVolume) Check() error { // 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(loc string) ([]byte, error) { +func (v *AzureBlobVolume) Get(ctx context.Context, loc string, buf []byte) (int, error) { + trashed, _, err := v.checkTrashed(loc) + if err != nil { + return 0, err + } + if trashed { + return 0, os.ErrNotExist + } var deadline time.Time haveDeadline := false - buf, err := v.get(loc) - for err == nil && len(buf) == 0 && loc != "d41d8cd98f00b204e9800998ecf8427e" { + size, err := v.get(ctx, loc, buf) + for err == nil && size == 0 && loc != "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" @@ -151,81 +171,128 @@ func (v *AzureBlobVolume) Get(loc string) ([]byte, error) { 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 } - bufs.Put(buf) - time.Sleep(azureWriteRacePollTime) - buf, err = v.get(loc) + select { + case <-ctx.Done(): + return 0, ctx.Err() + case <-time.After(v.WriteRacePollTime.Duration()): + } + size, err = v.get(ctx, loc, buf) } if haveDeadline { - log.Printf("Race ended with len(buf)==%d", len(buf)) + ctxlog.FromContext(ctx).Printf("Race ended with size==%d", size) } - return buf, err + return size, err } -func (v *AzureBlobVolume) get(loc string) ([]byte, error) { - expectSize := BlockSize - if azureMaxGetBytes < BlockSize { +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 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.bsClient.GetBlobProperties(v.containerName, loc) + props, err := v.container.GetBlobProperties(loc) if err != nil { - return nil, v.translateError(err) + return 0, v.translateError(err) } if props.ContentLength > int64(BlockSize) || props.ContentLength < 0 { - return nil, fmt.Errorf("block %s invalid size %d (max %d)", loc, props.ContentLength, BlockSize) + return 0, fmt.Errorf("block %s invalid size %d (max %d)", loc, props.ContentLength, BlockSize) } expectSize = int(props.ContentLength) + pieces = (expectSize + pieceSize - 1) / pieceSize } - buf := bufs.Get(expectSize) if expectSize == 0 { - return buf, nil + return 0, nil } // We'll update this actualSize if/when we get the last piece. actualSize := -1 - pieces := (expectSize + azureMaxGetBytes - 1) / azureMaxGetBytes - errors := make([]error, pieces) + errors := make(chan error, pieces) var wg sync.WaitGroup wg.Add(pieces) for p := 0; p < pieces; p++ { + // Each goroutine retrieves one piece. If we hit an + // error, it is sent to the errors chan so get() can + // return it -- but only if the error happens before + // ctx is done. This way, if ctx is done before we hit + // any other error (e.g., requesting client has hung + // up), we return the original ctx.Err() instead of + // the secondary errors from the transfers that got + // 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 } var rdr io.ReadCloser var err error - if startPos == 0 && endPos == expectSize { - rdr, err = v.bsClient.GetBlob(v.containerName, loc) - } else { - rdr, err = v.bsClient.GetBlobRange(v.containerName, loc, fmt.Sprintf("%d-%d", startPos, endPos-1)) + gotRdr := make(chan struct{}) + go func() { + defer close(gotRdr) + if startPos == 0 && endPos == expectSize { + rdr, err = v.container.GetBlob(loc) + } else { + rdr, err = v.container.GetBlobRange(loc, startPos, endPos-1, nil) + } + }() + select { + case <-ctx.Done(): + go func() { + <-gotRdr + if err == nil { + rdr.Close() + } + }() + return + case <-gotRdr: } if err != nil { - errors[p] = err + errors <- err + cancel() return } - defer rdr.Close() + go func() { + // Close the reader when the client + // hangs up or another piece fails + // (possibly interrupting ReadFull()) + // or when all pieces succeed and + // get() returns. + <-ctx.Done() + rdr.Close() + }() n, err := io.ReadFull(rdr, buf[startPos:endPos]) if pieces == 1 && (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 { - errors[p] = err + if ctx.Err() == nil { + errors <- err + } + cancel() + return } if p == pieces-1 { actualSize = startPos + n @@ -233,72 +300,148 @@ func (v *AzureBlobVolume) get(loc string) ([]byte, error) { }(p) } wg.Wait() - for _, err := range errors { - if err != nil { - bufs.Put(buf) - return nil, v.translateError(err) - } + close(errors) + if len(errors) > 0 { + return 0, v.translateError(<-errors) } - return buf[:actualSize], nil + if ctx.Err() != nil { + return 0, ctx.Err() + } + return actualSize, nil } // Compare the given data with existing stored data. -func (v *AzureBlobVolume) Compare(loc string, expect []byte) error { - rdr, err := v.bsClient.GetBlob(v.containerName, loc) +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) } defer rdr.Close() - return compareReaderWithBuf(rdr, expect, loc[:32]) + return compareReaderWithBuf(ctx, rdr, expect, loc[:32]) } // Put stores a Keep block as a block blob in the container. -func (v *AzureBlobVolume) Put(loc string, block []byte) error { - if v.readonly { +func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) error { + if v.volume.ReadOnly { return MethodDisabledError } - return v.bsClient.CreateBlockBlobFromReader(v.containerName, loc, uint64(len(block)), bytes.NewReader(block), nil) + // 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 + // buffer after we release it. + bufr, bufw := io.Pipe() + go func() { + io.Copy(bufw, bytes.NewReader(block)) + bufw.Close() + }() + errChan := make(chan error) + go func() { + 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(): + 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, + // 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. + bufw.CloseWithError(ctx.Err()) + 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 { + if v.volume.ReadOnly { return MethodDisabledError } - return v.bsClient.SetBlobMetadata(v.containerName, loc, map[string]string{ - "touch": fmt.Sprintf("%d", time.Now()), - }) + trashed, metadata, err := v.checkTrashed(loc) + if err != nil { + return err + } + if trashed { + return os.ErrNotExist + } + + 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. func (v *AzureBlobVolume) Mtime(loc string) (time.Time, error) { - props, err := v.bsClient.GetBlobProperties(v.containerName, loc) + trashed, _, err := v.checkTrashed(loc) + if err != nil { + return time.Time{}, err + } + if trashed { + return time.Time{}, os.ErrNotExist + } + + 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 // container. func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error { params := storage.ListBlobsParameters{ - Prefix: prefix, + Prefix: prefix, + Include: &storage.IncludeBlobDataset{Metadata: true}, } - for { - resp, err := v.bsClient.ListBlobs(v.containerName, params) + for page := 1; ; page++ { + resp, err := v.listBlobs(page, 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(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 @@ -306,7 +449,11 @@ func (v *AzureBlobVolume) IndexTo(prefix string, writer io.Writer) error { // value. continue } - fmt.Fprintf(writer, "%s+%d %d\n", b.Name, b.Properties.ContentLength, t.Unix()) + if b.Metadata["expires_at"] != "" { + // Trashed blob; exclude it from response + continue + } + fmt.Fprintf(writer, "%s+%d %d\n", b.Name, b.Properties.ContentLength, modtime.UnixNano()) } if resp.NextMarker == "" { return nil @@ -315,39 +462,74 @@ 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 { return MethodDisabledError } - if trashLifetime != 0 { - return ErrNotImplemented - } - // 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 // 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 } if t, err := v.Mtime(loc); err != nil { return err - } else if time.Since(t) < blobSignatureTTL { + } else if time.Since(t) < v.cluster.Collections.BlobSigningTTL.Duration() { return nil } - return v.bsClient.DeleteBlob(v.containerName, loc, map[string]string{ - "If-Match": props.Etag, + + // If BlobTrashLifetime == 0, just delete it + if v.cluster.Collections.BlobTrashLifetime == 0 { + return v.container.DeleteBlob(loc, &storage.DeleteBlobOptions{ + IfMatch: props.Etag, + }) + } + + // Otherwise, mark as trash + return v.container.SetBlobMetadata(loc, storage.BlobMetadata{ + "expires_at": fmt.Sprintf("%d", time.Now().Add(v.cluster.Collections.BlobTrashLifetime.Duration()).Unix()), + }, &storage.SetBlobMetadataOptions{ + IfMatch: props.Etag, }) } // Untrash a Keep block. -// TBD +// Delete the expires_at metadata attribute func (v *AzureBlobVolume) Untrash(loc string) error { - return ErrNotImplemented + // if expires_at does not exist, return NotFoundError + metadata, err := v.container.GetBlobMetadata(loc) + if err != nil { + return v.translateError(err) + } + if metadata["expires_at"] == "" { + return os.ErrNotExist + } + + // reset expires_at metadata attribute + metadata["expires_at"] = "" + err = v.container.SetBlobMetadata(loc, metadata, nil) + return v.translateError(err) } // Status returns a VolumeStatus struct with placeholder data. @@ -361,19 +543,7 @@ func (v *AzureBlobVolume) Status() *VolumeStatus { // 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.replication + return fmt.Sprintf("azure-storage-container:%+q", v.ContainerName) } // If possible, translate an Azure SDK error to a recognizable error @@ -382,9 +552,15 @@ func (v *AzureBlobVolume) translateError(err error) error { switch { case err == nil: return err - case strings.Contains(err.Error(), "404 Not Found"): + 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 + 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 } @@ -396,8 +572,216 @@ 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. -// TBD func (v *AzureBlobVolume) EmptyTrash() { + if v.cluster.Collections.BlobDeleteConcurrency < 1 { + return + } + + var bytesDeleted, bytesInTrash int64 + var blocksDeleted, blocksInTrash int64 + + 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 { + v.logger.Printf("EmptyTrash: ParseInt(%v): %v", b.Metadata["expires_at"], err) + return + } + + if expiresAt > time.Now().Unix() { + return + } + + err = v.container.DeleteBlob(b.Name, &storage.DeleteBlobOptions{ + IfMatch: b.Properties.Etag, + }) + if err != nil { + v.logger.Printf("EmptyTrash: DeleteBlob(%v): %v", b.Name, err) + return + } + atomic.AddInt64(&blocksDeleted, 1) + atomic.AddInt64(&bytesDeleted, b.Properties.ContentLength) + } + + var wg sync.WaitGroup + 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() + for b := range todo { + doBlob(b) + } + }() + } + + params := storage.ListBlobsParameters{Include: &storage.IncludeBlobDataset{Metadata: true}} + for page := 1; ; page++ { + resp, err := v.listBlobs(page, params) + if err != nil { + v.logger.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() + + 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. +func (v *AzureBlobVolume) InternalStats() interface{} { + return &v.container.stats +} + +type azureBlobStats struct { + statsTicker + Ops uint64 + GetOps uint64 + GetRangeOps uint64 + GetMetadataOps uint64 + GetPropertiesOps uint64 + CreateOps uint64 + SetMetadataOps uint64 + DelOps uint64 + ListOps uint64 +} + +func (s *azureBlobStats) TickErr(err error) { + if err == nil { + return + } + errType := fmt.Sprintf("%T", err) + if err, ok := err.(storage.AzureStorageServiceError); ok { + errType = errType + fmt.Sprintf(" %d (%s)", err.StatusCode, err.Code) + } + s.statsTicker.TickErr(err, errType) +} + +// azureContainer wraps storage.Container in order to count I/O and +// API usage stats. +type azureContainer struct { + ctr *storage.Container + stats azureBlobStats +} + +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) + return ok, err +} + +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) + c.stats.TickErr(err) + return b.Metadata, err +} + +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) + c.stats.TickErr(err) + return &b.Properties, err +} + +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 +} + +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{ + Range: &storage.BlobRange{ + Start: uint64(start), + End: uint64(end), + }, + GetBlobOptions: opts, + }) + c.stats.TickErr(err) + return NewCountingReader(rdr, c.stats.TickInBytes), err +} + +// 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) + 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 *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 + err := b.SetMetadata(opts) + c.stats.TickErr(err) + return err +} + +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) + return resp, err +} + +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) + c.stats.TickErr(err) + return err }