X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/90d84d8578b760b493ac76b22c42bc284868bc0c..39f6e9f70f683237d9488faac1c549ca19ac9dae:/services/keepstore/azure_blob_volume.go diff --git a/services/keepstore/azure_blob_volume.go b/services/keepstore/azure_blob_volume.go index b52b706b26..31660614f3 100644 --- a/services/keepstore/azure_blob_volume.go +++ b/services/keepstore/azure_blob_volume.go @@ -2,16 +2,14 @@ // // SPDX-License-Identifier: AGPL-3.0 -package main +package keepstore import ( - "bytes" "context" "encoding/json" "errors" "fmt" "io" - "io/ioutil" "net/http" "os" "regexp" @@ -21,8 +19,8 @@ import ( "sync/atomic" "time" - "git.curoverse.com/arvados.git/sdk/go/arvados" - "git.curoverse.com/arvados.git/sdk/go/ctxlog" + "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" @@ -32,18 +30,18 @@ func init() { driver["Azure"] = newAzureBlobVolume } -func newAzureBlobVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) { - v := &AzureBlobVolume{ - StorageBaseURL: storage.DefaultBaseURL, +func newAzureBlobVolume(params newVolumeParams) (volume, error) { + v := &azureBlobVolume{ RequestTimeout: azureDefaultRequestTimeout, WriteRaceInterval: azureDefaultWriteRaceInterval, WriteRacePollTime: azureDefaultWriteRacePollTime, - cluster: cluster, - volume: volume, - logger: logger, - metrics: metrics, + cluster: params.Cluster, + volume: params.ConfigVolume, + logger: params.Logger, + metrics: params.MetricsVecs, + bufferPool: params.BufferPool, } - err := json.Unmarshal(volume.DriverParameters, &v) + err := json.Unmarshal(params.ConfigVolume.DriverParameters, &v) if err != nil { return nil, err } @@ -53,6 +51,9 @@ func newAzureBlobVolume(cluster *arvados.Cluster, volume arvados.Volume, logger if v.ListBlobsMaxAttempts == 0 { v.ListBlobsMaxAttempts = azureDefaultListBlobsMaxAttempts } + if v.StorageBaseURL == "" { + v.StorageBaseURL = storage.DefaultBaseURL + } if v.ContainerName == "" || v.StorageAccountName == "" || v.StorageAccountKey == "" { return nil, errors.New("DriverParameters: ContainerName, StorageAccountName, and StorageAccountKey must be provided") } @@ -78,8 +79,8 @@ func newAzureBlobVolume(cluster *arvados.Cluster, volume arvados.Volume, logger return v, v.check() } -func (v *AzureBlobVolume) check() error { - lbls := prometheus.Labels{"device_id": v.GetDeviceID()} +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 } @@ -92,9 +93,9 @@ const ( azureDefaultWriteRacePollTime = arvados.Duration(time.Second) ) -// An AzureBlobVolume stores and retrieves blocks in an Azure Blob +// An azureBlobVolume stores and retrieves blocks in an Azure Blob // container. -type AzureBlobVolume struct { +type azureBlobVolume struct { StorageAccountName string StorageAccountKey string StorageBaseURL string // "" means default, "core.windows.net" @@ -106,12 +107,13 @@ type AzureBlobVolume struct { WriteRaceInterval arvados.Duration WriteRacePollTime arvados.Duration - cluster *arvados.Cluster - volume arvados.Volume - logger logrus.FieldLogger - metrics *volumeMetricsVecs - azClient storage.Client - container *azureContainer + 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. @@ -122,18 +124,13 @@ func (*singleSender) Send(c *storage.Client, req *http.Request) (resp *http.Resp 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 { +// 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) { +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) @@ -144,30 +141,36 @@ 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, writeTo io.Writer) (int, error) { + trashed, _, err := v.checkTrashed(hash) if err != nil { return 0, err } if trashed { return 0, os.ErrNotExist } + buf, err := v.bufferPool.GetContext(ctx) + if err != nil { + return 0, err + } + defer v.bufferPool.Put(buf) + streamer := newStreamWriterAt(writeTo, 65536, buf) + defer streamer.Close() var deadline time.Time - haveDeadline := false - size, err := v.get(ctx, loc, buf) - for err == nil && size == 0 && loc != "d41d8cd98f00b204e9800998ecf8427e" { + size, err := v.get(ctx, hash, streamer) + for err == nil && size == 0 && streamer.WroteAt() == 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 { ctxlog.FromContext(ctx).Print("Got empty block (possible race) but Mtime failed: ", err) break @@ -176,8 +179,7 @@ func (v *AzureBlobVolume) Get(ctx context.Context, loc string, buf []byte) (int, if time.Now().After(deadline) { break } - ctxlog.FromContext(ctx).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 } @@ -186,15 +188,20 @@ func (v *AzureBlobVolume) Get(ctx context.Context, loc string, buf []byte) (int, return 0, ctx.Err() case <-time.After(v.WriteRacePollTime.Duration()): } - size, err = v.get(ctx, loc, buf) + size, err = v.get(ctx, hash, streamer) } - if haveDeadline { + if !deadline.IsZero() { ctxlog.FromContext(ctx).Printf("Race ended with size==%d", size) } - return size, err + if err != nil { + streamer.Close() + return streamer.Wrote(), err + } + err = streamer.Close() + return streamer.Wrote(), 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() @@ -204,16 +211,17 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, } pieces := 1 - expectSize := len(buf) + expectSize := BlockSize if pieceSize < BlockSize { - // Unfortunately the handler doesn't tell us how long the blob - // is expected to be, so we have to ask Azure. - props, err := v.container.GetBlobProperties(loc) + // 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 @@ -250,9 +258,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 { @@ -280,7 +288,7 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, <-ctx.Done() rdr.Close() }() - n, err := io.ReadFull(rdr, buf[startPos:endPos]) + n, err := io.CopyN(io.NewOffsetWriter(dst, int64(startPos)), rdr, int64(endPos-startPos)) 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 @@ -293,7 +301,7 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, return } if p == pieces-1 { - actualSize = startPos + n + actualSize = startPos + int(n) } }(p) } @@ -308,58 +316,23 @@ func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, 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) - } - defer rdr.Close() - return compareReaderWithBuf(ctx, rdr, expect, loc[:32]) -} - -// 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.volume.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 @@ -368,18 +341,15 @@ 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(): 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() 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()) ctxlog.FromContext(ctx).Debugf("%s: abandoning CreateBlockBlobFromReader goroutine", v) return ctx.Err() @@ -388,12 +358,9 @@ func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) err } } -// Touch updates the last-modified property of a block blob. -func (v *AzureBlobVolume) Touch(loc string) error { - if v.volume.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 } @@ -402,12 +369,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 } @@ -415,21 +382,25 @@ 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 page := 1; ; page++ { + err := ctx.Err() + if err != nil { + return err + } resp, err := v.listBlobs(page, params) if err != nil { return err @@ -461,11 +432,11 @@ 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) { +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 { + 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 @@ -477,11 +448,7 @@ func (v *AzureBlobVolume) listBlobs(page int, params storage.ListBlobsParameters } // Trash a Keep block. -func (v *AzureBlobVolume) Trash(loc string) error { - if v.volume.ReadOnly { - return MethodDisabledError - } - +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 @@ -497,7 +464,7 @@ func (v *AzureBlobVolume) Trash(loc string) error { return nil } - // If TrashLifetime == 0, just delete it + // If BlobTrashLifetime == 0, just delete it if v.cluster.Collections.BlobTrashLifetime == 0 { return v.container.DeleteBlob(loc, &storage.DeleteBlobOptions{ IfMatch: props.Etag, @@ -512,11 +479,11 @@ func (v *AzureBlobVolume) Trash(loc string) error { }) } -// 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) } @@ -526,36 +493,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) -} - // 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 } @@ -563,17 +519,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() { - if v.cluster.Collections.BlobDeleteConcurrency < 1 { - return - } - +func (v *azureBlobVolume) EmptyTrash() { var bytesDeleted, bytesInTrash int64 var blocksDeleted, blocksInTrash int64 @@ -637,11 +589,11 @@ func (v *AzureBlobVolume) EmptyTrash() { 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) + 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 } @@ -708,7 +660,7 @@ func (c *azureContainer) GetBlob(bname string) (io.ReadCloser, error) { 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) { @@ -723,7 +675,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 @@ -744,7 +696,7 @@ func (c *azureContainer) CreateBlockBlobFromReader(bname string, size int, rdr i 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, } }