import (
"bytes"
"context"
+ "encoding/json"
"errors"
- "flag"
"fmt"
"io"
"io/ioutil"
"strconv"
"strings"
"sync"
+ "sync/atomic"
"time"
- "git.curoverse.com/arvados.git/sdk/go/arvados"
+ "git.arvados.org/arvados.git/sdk/go/arvados"
+ "git.arvados.org/arvados.git/sdk/go/ctxlog"
"github.com/Azure/azure-sdk-for-go/storage"
+ "github.com/prometheus/client_golang/prometheus"
+ "github.com/sirupsen/logrus"
)
-const azureDefaultRequestTimeout = arvados.Duration(10 * time.Minute)
-
-var (
- azureMaxGetBytes int
- azureStorageAccountName string
- azureStorageAccountKeyFile string
- azureStorageReplication int
- azureWriteRaceInterval = 15 * time.Second
- azureWriteRacePollTime = time.Second
-)
+func init() {
+ driver["Azure"] = newAzureBlobVolume
+}
-func readKeyFromFile(file string) (string, error) {
- buf, err := ioutil.ReadFile(file)
+func newAzureBlobVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) {
+ v := &AzureBlobVolume{
+ RequestTimeout: azureDefaultRequestTimeout,
+ WriteRaceInterval: azureDefaultWriteRaceInterval,
+ WriteRacePollTime: azureDefaultWriteRacePollTime,
+ cluster: cluster,
+ volume: volume,
+ logger: logger,
+ metrics: metrics,
+ }
+ err := json.Unmarshal(volume.DriverParameters, &v)
if err != nil {
- return "", errors.New("reading key from " + file + ": " + err.Error())
+ return nil, err
}
- accountKey := strings.TrimSpace(string(buf))
- if accountKey == "" {
- return "", errors.New("empty account key in " + file)
+ if v.ListBlobsRetryDelay == 0 {
+ v.ListBlobsRetryDelay = azureDefaultListBlobsRetryDelay
+ }
+ if v.ListBlobsMaxAttempts == 0 {
+ v.ListBlobsMaxAttempts = azureDefaultListBlobsMaxAttempts
+ }
+ if v.StorageBaseURL == "" {
+ v.StorageBaseURL = storage.DefaultBaseURL
+ }
+ if v.ContainerName == "" || v.StorageAccountName == "" || v.StorageAccountKey == "" {
+ return nil, errors.New("DriverParameters: ContainerName, StorageAccountName, and StorageAccountKey must be provided")
+ }
+ azc, err := storage.NewClient(v.StorageAccountName, v.StorageAccountKey, v.StorageBaseURL, storage.DefaultAPIVersion, true)
+ if err != nil {
+ return nil, fmt.Errorf("creating Azure storage client: %s", err)
+ }
+ v.azClient = azc
+ v.azClient.Sender = &singleSender{}
+ v.azClient.HTTPClient = &http.Client{
+ Timeout: time.Duration(v.RequestTimeout),
+ }
+ bs := v.azClient.GetBlobService()
+ v.container = &azureContainer{
+ ctr: bs.GetContainerReference(v.ContainerName),
}
- return accountKey, nil
-}
-
-type azureVolumeAdder struct {
- *Config
-}
-// String implements flag.Value
-func (s *azureVolumeAdder) String() string {
- return "-"
+ if ok, err := v.container.Exists(); err != nil {
+ return nil, err
+ } else if !ok {
+ return nil, fmt.Errorf("Azure container %q does not exist: %s", v.ContainerName, err)
+ }
+ return v, v.check()
}
-func (s *azureVolumeAdder) Set(containerName string) error {
- s.Config.Volumes = append(s.Config.Volumes, &AzureBlobVolume{
- ContainerName: containerName,
- StorageAccountName: azureStorageAccountName,
- StorageAccountKeyFile: azureStorageAccountKeyFile,
- AzureReplication: azureStorageReplication,
- ReadOnly: deprecated.flagReadonly,
- })
+func (v *AzureBlobVolume) check() error {
+ lbls := prometheus.Labels{"device_id": v.GetDeviceID()}
+ v.container.stats.opsCounters, v.container.stats.errCounters, v.container.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
return nil
}
-func init() {
- VolumeTypes = append(VolumeTypes, func() VolumeWithExamples { return &AzureBlobVolume{} })
-
- flag.Var(&azureVolumeAdder{theConfig},
- "azure-storage-container-volume",
- "Use the given container as a storage volume. Can be given multiple times.")
- flag.StringVar(
- &azureStorageAccountName,
- "azure-storage-account-name",
- "",
- "Azure storage account name used for subsequent --azure-storage-container-volume arguments.")
- flag.StringVar(
- &azureStorageAccountKeyFile,
- "azure-storage-account-key-file",
- "",
- "`File` containing the account key used for subsequent --azure-storage-container-volume arguments.")
- flag.IntVar(
- &azureStorageReplication,
- "azure-storage-replication",
- 3,
- "Replication level to report to clients when data is stored in an Azure container.")
- flag.IntVar(
- &azureMaxGetBytes,
- "azure-max-get-bytes",
- BlockSize,
- fmt.Sprintf("Maximum bytes to request in a single GET request. If smaller than %d, use multiple concurrent range requests to retrieve a block.", BlockSize))
-}
+const (
+ azureDefaultRequestTimeout = arvados.Duration(10 * time.Minute)
+ azureDefaultListBlobsMaxAttempts = 12
+ azureDefaultListBlobsRetryDelay = arvados.Duration(10 * time.Second)
+ azureDefaultWriteRaceInterval = arvados.Duration(15 * time.Second)
+ azureDefaultWriteRacePollTime = arvados.Duration(time.Second)
+)
// An AzureBlobVolume stores and retrieves blocks in an Azure Blob
// container.
type AzureBlobVolume struct {
- StorageAccountName string
- StorageAccountKeyFile string
- StorageBaseURL string // "" means default, "core.windows.net"
- ContainerName string
- AzureReplication int
- ReadOnly bool
- RequestTimeout arvados.Duration
-
+ 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
}
return c.HTTPClient.Do(req)
}
-// Examples implements VolumeWithExamples.
-func (*AzureBlobVolume) Examples() []Volume {
- return []Volume{
- &AzureBlobVolume{
- StorageAccountName: "example-account-name",
- StorageAccountKeyFile: "/etc/azure_storage_account_key.txt",
- ContainerName: "example-container-name",
- AzureReplication: 3,
- RequestTimeout: azureDefaultRequestTimeout,
- },
- &AzureBlobVolume{
- StorageAccountName: "cn-account-name",
- StorageAccountKeyFile: "/etc/azure_cn_storage_account_key.txt",
- StorageBaseURL: "core.chinacloudapi.cn",
- ContainerName: "cn-container-name",
- AzureReplication: 3,
- RequestTimeout: azureDefaultRequestTimeout,
- },
- }
-}
-
// Type implements Volume.
func (v *AzureBlobVolume) Type() string {
return "Azure"
}
-// Start implements Volume.
-func (v *AzureBlobVolume) Start() error {
- if v.ContainerName == "" {
- return errors.New("no container name given")
- }
- if v.StorageAccountName == "" || v.StorageAccountKeyFile == "" {
- return errors.New("StorageAccountName and StorageAccountKeyFile must be given")
- }
- accountKey, err := readKeyFromFile(v.StorageAccountKeyFile)
- if err != nil {
- return err
- }
- if v.StorageBaseURL == "" {
- v.StorageBaseURL = storage.DefaultBaseURL
- }
- v.azClient, err = storage.NewClient(v.StorageAccountName, accountKey, v.StorageBaseURL, storage.DefaultAPIVersion, true)
- if err != nil {
- return fmt.Errorf("creating Azure storage client: %s", err)
- }
- v.azClient.Sender = &singleSender{}
-
- if v.RequestTimeout == 0 {
- v.RequestTimeout = azureDefaultRequestTimeout
- }
- v.azClient.HTTPClient = &http.Client{
- Timeout: time.Duration(v.RequestTimeout),
- }
- bs := v.azClient.GetBlobService()
- v.container = &azureContainer{
- ctr: bs.GetContainerReference(v.ContainerName),
- }
-
- if ok, err := v.container.Exists(); err != nil {
- return err
- } else if !ok {
- return fmt.Errorf("Azure container %q does not exist", v.ContainerName)
- }
- return nil
-}
-
-// DeviceID returns a globally unique ID for the storage container.
-func (v *AzureBlobVolume) DeviceID() string {
+// GetDeviceID returns a globally unique ID for the storage container.
+func (v *AzureBlobVolume) GetDeviceID() string {
return "azure://" + v.StorageBaseURL + "/" + v.StorageAccountName + "/" + v.ContainerName
}
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
select {
case <-ctx.Done():
return 0, ctx.Err()
- case <-time.After(azureWriteRacePollTime):
+ case <-time.After(v.WriteRacePollTime.Duration()):
}
size, err = v.get(ctx, loc, buf)
}
if haveDeadline {
- log.Printf("Race ended with size==%d", size)
+ ctxlog.FromContext(ctx).Printf("Race ended with size==%d", size)
}
return size, err
}
func (v *AzureBlobVolume) get(ctx context.Context, loc string, buf []byte) (int, error) {
ctx, cancel := context.WithCancel(ctx)
defer cancel()
+
+ pieceSize := BlockSize
+ if v.MaxGetBytes > 0 && v.MaxGetBytes < BlockSize {
+ pieceSize = v.MaxGetBytes
+ }
+
+ pieces := 1
expectSize := len(buf)
- if azureMaxGetBytes < BlockSize {
+ if pieceSize < BlockSize {
// Unfortunately the handler doesn't tell us how long the blob
// is expected to be, so we have to ask Azure.
props, err := v.container.GetBlobProperties(loc)
return 0, fmt.Errorf("block %s invalid size %d (max %d)", loc, props.ContentLength, BlockSize)
}
expectSize = int(props.ContentLength)
+ pieces = (expectSize + pieceSize - 1) / pieceSize
}
if expectSize == 0 {
// We'll update this actualSize if/when we get the last piece.
actualSize := -1
- pieces := (expectSize + azureMaxGetBytes - 1) / azureMaxGetBytes
errors := make(chan error, pieces)
var wg sync.WaitGroup
wg.Add(pieces)
// 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
}
// Put stores a Keep block as a block blob in the container.
func (v *AzureBlobVolume) Put(ctx context.Context, loc string, block []byte) error {
- if v.ReadOnly {
+ if v.volume.ReadOnly {
return MethodDisabledError
}
// Send the block data through a pipe, so that (if we need to)
}()
select {
case <-ctx.Done():
- theConfig.debugLogf("%s: taking CreateBlockBlobFromReader's input away: %s", v, ctx.Err())
+ ctxlog.FromContext(ctx).Debugf("%s: taking CreateBlockBlobFromReader's input away: %s", v, ctx.Err())
// Our pipe might be stuck in Write(), waiting for
// io.Copy() to read. If so, un-stick it. This means
// CreateBlockBlobFromReader will get corrupt data,
go io.Copy(ioutil.Discard, bufr)
// CloseWithError() will return once pending I/O is done.
bufw.CloseWithError(ctx.Err())
- theConfig.debugLogf("%s: abandoning CreateBlockBlobFromReader goroutine", v)
+ ctxlog.FromContext(ctx).Debugf("%s: abandoning CreateBlockBlobFromReader goroutine", v)
return ctx.Err()
case err := <-errChan:
return err
// Touch updates the last-modified property of a block blob.
func (v *AzureBlobVolume) Touch(loc string) error {
- if v.ReadOnly {
+ if v.volume.ReadOnly {
return MethodDisabledError
}
trashed, metadata, err := v.checkTrashed(loc)
return os.ErrNotExist
}
- metadata["touch"] = fmt.Sprintf("%d", time.Now())
+ metadata["touch"] = fmt.Sprintf("%d", time.Now().Unix())
return v.container.SetBlobMetadata(loc, metadata, nil)
}
Prefix: prefix,
Include: &storage.IncludeBlobDataset{Metadata: true},
}
- for {
- resp, err := v.container.ListBlobs(params)
+ for page := 1; ; page++ {
+ resp, err := v.listBlobs(page, params)
if err != nil {
return err
}
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
}
}
+// 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 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,
})
// 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,
})
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
-}
-
// 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
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
+ }
+
var bytesDeleted, bytesInTrash int64
- var blocksDeleted, blocksInTrash int
- params := storage.ListBlobsParameters{Include: &storage.IncludeBlobDataset{Metadata: true}}
+ 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)
- for {
- resp, err := v.container.ListBlobs(params)
+ expiresAt, err := strconv.ParseInt(b.Metadata["expires_at"], 10, 64)
if err != nil {
- log.Printf("EmptyTrash: ListBlobs: %v", err)
- break
+ v.logger.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 {
+ v.logger.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, 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)
}
+ }()
+ }
- err = v.container.DeleteBlob(b.Name, &storage.DeleteBlobOptions{
- IfMatch: 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 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()
- log.Printf("EmptyTrash stats for %v: Deleted %v bytes in %v blocks. Remaining in trash: %v bytes in %v blocks.", v.String(), bytesDeleted, blocksDeleted, bytesInTrash-bytesDeleted, blocksInTrash-blocksDeleted)
+ v.logger.Printf("EmptyTrash stats for %v: Deleted %v bytes in %v blocks. Remaining in trash: %v bytes in %v blocks.", v.String(), bytesDeleted, blocksDeleted, bytesInTrash-bytesDeleted, blocksInTrash-blocksDeleted)
}
// InternalStats returns bucket I/O and API call counters.
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)
}
}
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)
}
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)
}
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)
}
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)
}
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{
}
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{
}
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
}
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)
}
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)