2960: Refactor keepstore into a streaming server.
[arvados.git] / services / keepstore / azure_blob_volume.go
1 // Copyright (C) The Arvados Authors. All rights reserved.
2 //
3 // SPDX-License-Identifier: AGPL-3.0
4
5 package keepstore
6
7 import (
8         "context"
9         "encoding/json"
10         "errors"
11         "fmt"
12         "io"
13         "net/http"
14         "os"
15         "regexp"
16         "strconv"
17         "strings"
18         "sync"
19         "sync/atomic"
20         "time"
21
22         "git.arvados.org/arvados.git/sdk/go/arvados"
23         "git.arvados.org/arvados.git/sdk/go/ctxlog"
24         "github.com/Azure/azure-sdk-for-go/storage"
25         "github.com/prometheus/client_golang/prometheus"
26         "github.com/sirupsen/logrus"
27 )
28
29 func init() {
30         driver["Azure"] = newAzureBlobVolume
31 }
32
33 func newAzureBlobVolume(params newVolumeParams) (volume, error) {
34         v := &AzureBlobVolume{
35                 RequestTimeout:    azureDefaultRequestTimeout,
36                 WriteRaceInterval: azureDefaultWriteRaceInterval,
37                 WriteRacePollTime: azureDefaultWriteRacePollTime,
38                 cluster:           params.Cluster,
39                 volume:            params.ConfigVolume,
40                 logger:            params.Logger,
41                 metrics:           params.MetricsVecs,
42                 bufferPool:        params.BufferPool,
43         }
44         err := json.Unmarshal(params.ConfigVolume.DriverParameters, &v)
45         if err != nil {
46                 return nil, err
47         }
48         if v.ListBlobsRetryDelay == 0 {
49                 v.ListBlobsRetryDelay = azureDefaultListBlobsRetryDelay
50         }
51         if v.ListBlobsMaxAttempts == 0 {
52                 v.ListBlobsMaxAttempts = azureDefaultListBlobsMaxAttempts
53         }
54         if v.StorageBaseURL == "" {
55                 v.StorageBaseURL = storage.DefaultBaseURL
56         }
57         if v.ContainerName == "" || v.StorageAccountName == "" || v.StorageAccountKey == "" {
58                 return nil, errors.New("DriverParameters: ContainerName, StorageAccountName, and StorageAccountKey must be provided")
59         }
60         azc, err := storage.NewClient(v.StorageAccountName, v.StorageAccountKey, v.StorageBaseURL, storage.DefaultAPIVersion, true)
61         if err != nil {
62                 return nil, fmt.Errorf("creating Azure storage client: %s", err)
63         }
64         v.azClient = azc
65         v.azClient.Sender = &singleSender{}
66         v.azClient.HTTPClient = &http.Client{
67                 Timeout: time.Duration(v.RequestTimeout),
68         }
69         bs := v.azClient.GetBlobService()
70         v.container = &azureContainer{
71                 ctr: bs.GetContainerReference(v.ContainerName),
72         }
73
74         if ok, err := v.container.Exists(); err != nil {
75                 return nil, err
76         } else if !ok {
77                 return nil, fmt.Errorf("Azure container %q does not exist: %s", v.ContainerName, err)
78         }
79         return v, v.check()
80 }
81
82 func (v *AzureBlobVolume) check() error {
83         lbls := prometheus.Labels{"device_id": v.DeviceID()}
84         v.container.stats.opsCounters, v.container.stats.errCounters, v.container.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
85         return nil
86 }
87
88 const (
89         azureDefaultRequestTimeout       = arvados.Duration(10 * time.Minute)
90         azureDefaultListBlobsMaxAttempts = 12
91         azureDefaultListBlobsRetryDelay  = arvados.Duration(10 * time.Second)
92         azureDefaultWriteRaceInterval    = arvados.Duration(15 * time.Second)
93         azureDefaultWriteRacePollTime    = arvados.Duration(time.Second)
94 )
95
96 // An AzureBlobVolume stores and retrieves blocks in an Azure Blob
97 // container.
98 type AzureBlobVolume struct {
99         StorageAccountName   string
100         StorageAccountKey    string
101         StorageBaseURL       string // "" means default, "core.windows.net"
102         ContainerName        string
103         RequestTimeout       arvados.Duration
104         ListBlobsRetryDelay  arvados.Duration
105         ListBlobsMaxAttempts int
106         MaxGetBytes          int
107         WriteRaceInterval    arvados.Duration
108         WriteRacePollTime    arvados.Duration
109
110         cluster    *arvados.Cluster
111         volume     arvados.Volume
112         logger     logrus.FieldLogger
113         metrics    *volumeMetricsVecs
114         bufferPool *bufferPool
115         azClient   storage.Client
116         container  *azureContainer
117 }
118
119 // singleSender is a single-attempt storage.Sender.
120 type singleSender struct{}
121
122 // Send performs req exactly once.
123 func (*singleSender) Send(c *storage.Client, req *http.Request) (resp *http.Response, err error) {
124         return c.HTTPClient.Do(req)
125 }
126
127 // DeviceID returns a globally unique ID for the storage container.
128 func (v *AzureBlobVolume) DeviceID() string {
129         return "azure://" + v.StorageBaseURL + "/" + v.StorageAccountName + "/" + v.ContainerName
130 }
131
132 // Return true if expires_at metadata attribute is found on the block
133 func (v *AzureBlobVolume) checkTrashed(loc string) (bool, map[string]string, error) {
134         metadata, err := v.container.GetBlobMetadata(loc)
135         if err != nil {
136                 return false, metadata, v.translateError(err)
137         }
138         if metadata["expires_at"] != "" {
139                 return true, metadata, nil
140         }
141         return false, metadata, nil
142 }
143
144 // BlockRead reads a Keep block that has been stored as a block blob
145 // in the container.
146 //
147 // If the block is younger than azureWriteRaceInterval and is
148 // unexpectedly empty, assume a BlockWrite operation is in progress,
149 // and wait for it to finish writing.
150 func (v *AzureBlobVolume) BlockRead(ctx context.Context, hash string, writeTo io.Writer) (int, error) {
151         trashed, _, err := v.checkTrashed(hash)
152         if err != nil {
153                 return 0, err
154         }
155         if trashed {
156                 return 0, os.ErrNotExist
157         }
158         buf, err := v.bufferPool.GetContext(ctx)
159         if err != nil {
160                 return 0, err
161         }
162         defer v.bufferPool.Put(buf)
163         streamer := newStreamWriterAt(writeTo, 65536, buf)
164         defer streamer.Close()
165         var deadline time.Time
166         size, err := v.get(ctx, hash, streamer)
167         for err == nil && size == 0 && streamer.WroteAt() == 0 && hash != "d41d8cd98f00b204e9800998ecf8427e" {
168                 // Seeing a brand new empty block probably means we're
169                 // in a race with CreateBlob, which under the hood
170                 // (apparently) does "CreateEmpty" and "CommitData"
171                 // with no additional transaction locking.
172                 if deadline.IsZero() {
173                         t, err := v.Mtime(hash)
174                         if err != nil {
175                                 ctxlog.FromContext(ctx).Print("Got empty block (possible race) but Mtime failed: ", err)
176                                 break
177                         }
178                         deadline = t.Add(v.WriteRaceInterval.Duration())
179                         if time.Now().After(deadline) {
180                                 break
181                         }
182                         ctxlog.FromContext(ctx).Printf("Race? Block %s is 0 bytes, %s old. Polling until %s", hash, time.Since(t), deadline)
183                 } else if time.Now().After(deadline) {
184                         break
185                 }
186                 select {
187                 case <-ctx.Done():
188                         return 0, ctx.Err()
189                 case <-time.After(v.WriteRacePollTime.Duration()):
190                 }
191                 size, err = v.get(ctx, hash, streamer)
192         }
193         if !deadline.IsZero() {
194                 ctxlog.FromContext(ctx).Printf("Race ended with size==%d", size)
195         }
196         if err != nil {
197                 streamer.Close()
198                 return streamer.Wrote(), err
199         }
200         err = streamer.Close()
201         return streamer.Wrote(), err
202 }
203
204 func (v *AzureBlobVolume) get(ctx context.Context, hash string, dst io.WriterAt) (int, error) {
205         ctx, cancel := context.WithCancel(ctx)
206         defer cancel()
207
208         pieceSize := BlockSize
209         if v.MaxGetBytes > 0 && v.MaxGetBytes < BlockSize {
210                 pieceSize = v.MaxGetBytes
211         }
212
213         pieces := 1
214         expectSize := BlockSize
215         if pieceSize < BlockSize {
216                 // Unfortunately the handler doesn't tell us how long
217                 // the blob is expected to be, so we have to ask
218                 // Azure.
219                 props, err := v.container.GetBlobProperties(hash)
220                 if err != nil {
221                         return 0, v.translateError(err)
222                 }
223                 if props.ContentLength > int64(BlockSize) || props.ContentLength < 0 {
224                         return 0, fmt.Errorf("block %s invalid size %d (max %d)", hash, props.ContentLength, BlockSize)
225                 }
226                 expectSize = int(props.ContentLength)
227                 pieces = (expectSize + pieceSize - 1) / pieceSize
228         }
229
230         if expectSize == 0 {
231                 return 0, nil
232         }
233
234         // We'll update this actualSize if/when we get the last piece.
235         actualSize := -1
236         errors := make(chan error, pieces)
237         var wg sync.WaitGroup
238         wg.Add(pieces)
239         for p := 0; p < pieces; p++ {
240                 // Each goroutine retrieves one piece. If we hit an
241                 // error, it is sent to the errors chan so get() can
242                 // return it -- but only if the error happens before
243                 // ctx is done. This way, if ctx is done before we hit
244                 // any other error (e.g., requesting client has hung
245                 // up), we return the original ctx.Err() instead of
246                 // the secondary errors from the transfers that got
247                 // interrupted as a result.
248                 go func(p int) {
249                         defer wg.Done()
250                         startPos := p * pieceSize
251                         endPos := startPos + pieceSize
252                         if endPos > expectSize {
253                                 endPos = expectSize
254                         }
255                         var rdr io.ReadCloser
256                         var err error
257                         gotRdr := make(chan struct{})
258                         go func() {
259                                 defer close(gotRdr)
260                                 if startPos == 0 && endPos == expectSize {
261                                         rdr, err = v.container.GetBlob(hash)
262                                 } else {
263                                         rdr, err = v.container.GetBlobRange(hash, startPos, endPos-1, nil)
264                                 }
265                         }()
266                         select {
267                         case <-ctx.Done():
268                                 go func() {
269                                         <-gotRdr
270                                         if err == nil {
271                                                 rdr.Close()
272                                         }
273                                 }()
274                                 return
275                         case <-gotRdr:
276                         }
277                         if err != nil {
278                                 errors <- err
279                                 cancel()
280                                 return
281                         }
282                         go func() {
283                                 // Close the reader when the client
284                                 // hangs up or another piece fails
285                                 // (possibly interrupting ReadFull())
286                                 // or when all pieces succeed and
287                                 // get() returns.
288                                 <-ctx.Done()
289                                 rdr.Close()
290                         }()
291                         n, err := io.CopyN(io.NewOffsetWriter(dst, int64(startPos)), rdr, int64(endPos-startPos))
292                         if pieces == 1 && (err == io.ErrUnexpectedEOF || err == io.EOF) {
293                                 // If we don't know the actual size,
294                                 // and just tried reading 64 MiB, it's
295                                 // normal to encounter EOF.
296                         } else if err != nil {
297                                 if ctx.Err() == nil {
298                                         errors <- err
299                                 }
300                                 cancel()
301                                 return
302                         }
303                         if p == pieces-1 {
304                                 actualSize = startPos + int(n)
305                         }
306                 }(p)
307         }
308         wg.Wait()
309         close(errors)
310         if len(errors) > 0 {
311                 return 0, v.translateError(<-errors)
312         }
313         if ctx.Err() != nil {
314                 return 0, ctx.Err()
315         }
316         return actualSize, nil
317 }
318
319 // BlockWrite stores a block on the volume. If it already exists, its
320 // timestamp is updated.
321 func (v *AzureBlobVolume) BlockWrite(ctx context.Context, hash string, data []byte) error {
322         // Send the block data through a pipe, so that (if we need to)
323         // we can close the pipe early and abandon our
324         // CreateBlockBlobFromReader() goroutine, without worrying
325         // about CreateBlockBlobFromReader() accessing our data
326         // buffer after we release it.
327         bufr, bufw := io.Pipe()
328         go func() {
329                 bufw.Write(data)
330                 bufw.Close()
331         }()
332         errChan := make(chan error, 1)
333         go func() {
334                 var body io.Reader = bufr
335                 if len(data) == 0 {
336                         // We must send a "Content-Length: 0" header,
337                         // but the http client interprets
338                         // ContentLength==0 as "unknown" unless it can
339                         // confirm by introspection that Body will
340                         // read 0 bytes.
341                         body = http.NoBody
342                         bufr.Close()
343                 }
344                 errChan <- v.container.CreateBlockBlobFromReader(hash, len(data), body, nil)
345         }()
346         select {
347         case <-ctx.Done():
348                 ctxlog.FromContext(ctx).Debugf("%s: taking CreateBlockBlobFromReader's input away: %s", v, ctx.Err())
349                 // bufw.CloseWithError() interrupts bufw.Write() if
350                 // necessary, ensuring CreateBlockBlobFromReader can't
351                 // read any more of our data slice via bufr after we
352                 // return.
353                 bufw.CloseWithError(ctx.Err())
354                 ctxlog.FromContext(ctx).Debugf("%s: abandoning CreateBlockBlobFromReader goroutine", v)
355                 return ctx.Err()
356         case err := <-errChan:
357                 return err
358         }
359 }
360
361 // BlockTouch updates the last-modified property of a block blob.
362 func (v *AzureBlobVolume) BlockTouch(hash string) error {
363         trashed, metadata, err := v.checkTrashed(hash)
364         if err != nil {
365                 return err
366         }
367         if trashed {
368                 return os.ErrNotExist
369         }
370
371         metadata["touch"] = fmt.Sprintf("%d", time.Now().Unix())
372         return v.container.SetBlobMetadata(hash, metadata, nil)
373 }
374
375 // Mtime returns the last-modified property of a block blob.
376 func (v *AzureBlobVolume) Mtime(hash string) (time.Time, error) {
377         trashed, _, err := v.checkTrashed(hash)
378         if err != nil {
379                 return time.Time{}, err
380         }
381         if trashed {
382                 return time.Time{}, os.ErrNotExist
383         }
384
385         props, err := v.container.GetBlobProperties(hash)
386         if err != nil {
387                 return time.Time{}, err
388         }
389         return time.Time(props.LastModified), nil
390 }
391
392 // Index writes a list of Keep blocks that are stored in the
393 // container.
394 func (v *AzureBlobVolume) Index(ctx context.Context, prefix string, writer io.Writer) error {
395         params := storage.ListBlobsParameters{
396                 Prefix:  prefix,
397                 Include: &storage.IncludeBlobDataset{Metadata: true},
398         }
399         for page := 1; ; page++ {
400                 err := ctx.Err()
401                 if err != nil {
402                         return err
403                 }
404                 resp, err := v.listBlobs(page, params)
405                 if err != nil {
406                         return err
407                 }
408                 for _, b := range resp.Blobs {
409                         if !v.isKeepBlock(b.Name) {
410                                 continue
411                         }
412                         modtime := time.Time(b.Properties.LastModified)
413                         if b.Properties.ContentLength == 0 && modtime.Add(v.WriteRaceInterval.Duration()).After(time.Now()) {
414                                 // A new zero-length blob is probably
415                                 // just a new non-empty blob that
416                                 // hasn't committed its data yet (see
417                                 // Get()), and in any case has no
418                                 // value.
419                                 continue
420                         }
421                         if b.Metadata["expires_at"] != "" {
422                                 // Trashed blob; exclude it from response
423                                 continue
424                         }
425                         fmt.Fprintf(writer, "%s+%d %d\n", b.Name, b.Properties.ContentLength, modtime.UnixNano())
426                 }
427                 if resp.NextMarker == "" {
428                         return nil
429                 }
430                 params.Marker = resp.NextMarker
431         }
432 }
433
434 // call v.container.ListBlobs, retrying if needed.
435 func (v *AzureBlobVolume) listBlobs(page int, params storage.ListBlobsParameters) (resp storage.BlobListResponse, err error) {
436         for i := 0; i < v.ListBlobsMaxAttempts; i++ {
437                 resp, err = v.container.ListBlobs(params)
438                 err = v.translateError(err)
439                 if err == errVolumeUnavailable {
440                         v.logger.Printf("ListBlobs: will retry page %d in %s after error: %s", page, v.ListBlobsRetryDelay, err)
441                         time.Sleep(time.Duration(v.ListBlobsRetryDelay))
442                         continue
443                 } else {
444                         break
445                 }
446         }
447         return
448 }
449
450 // Trash a Keep block.
451 func (v *AzureBlobVolume) BlockTrash(loc string) error {
452         // Ideally we would use If-Unmodified-Since, but that
453         // particular condition seems to be ignored by Azure. Instead,
454         // we get the Etag before checking Mtime, and use If-Match to
455         // ensure we don't delete data if Put() or Touch() happens
456         // between our calls to Mtime() and DeleteBlob().
457         props, err := v.container.GetBlobProperties(loc)
458         if err != nil {
459                 return err
460         }
461         if t, err := v.Mtime(loc); err != nil {
462                 return err
463         } else if time.Since(t) < v.cluster.Collections.BlobSigningTTL.Duration() {
464                 return nil
465         }
466
467         // If BlobTrashLifetime == 0, just delete it
468         if v.cluster.Collections.BlobTrashLifetime == 0 {
469                 return v.container.DeleteBlob(loc, &storage.DeleteBlobOptions{
470                         IfMatch: props.Etag,
471                 })
472         }
473
474         // Otherwise, mark as trash
475         return v.container.SetBlobMetadata(loc, storage.BlobMetadata{
476                 "expires_at": fmt.Sprintf("%d", time.Now().Add(v.cluster.Collections.BlobTrashLifetime.Duration()).Unix()),
477         }, &storage.SetBlobMetadataOptions{
478                 IfMatch: props.Etag,
479         })
480 }
481
482 // BlockUntrash deletes the expires_at metadata attribute for the
483 // specified block blob.
484 func (v *AzureBlobVolume) BlockUntrash(hash string) error {
485         // if expires_at does not exist, return NotFoundError
486         metadata, err := v.container.GetBlobMetadata(hash)
487         if err != nil {
488                 return v.translateError(err)
489         }
490         if metadata["expires_at"] == "" {
491                 return os.ErrNotExist
492         }
493
494         // reset expires_at metadata attribute
495         metadata["expires_at"] = ""
496         err = v.container.SetBlobMetadata(hash, metadata, nil)
497         return v.translateError(err)
498 }
499
500 // If possible, translate an Azure SDK error to a recognizable error
501 // like os.ErrNotExist.
502 func (v *AzureBlobVolume) translateError(err error) error {
503         switch {
504         case err == nil:
505                 return err
506         case strings.Contains(err.Error(), "StatusCode=503"):
507                 // "storage: service returned error: StatusCode=503, ErrorCode=ServerBusy, ErrorMessage=The server is busy" (See #14804)
508                 return errVolumeUnavailable
509         case strings.Contains(err.Error(), "Not Found"):
510                 // "storage: service returned without a response body (404 Not Found)"
511                 return os.ErrNotExist
512         case strings.Contains(err.Error(), "ErrorCode=BlobNotFound"):
513                 // "storage: service returned error: StatusCode=404, ErrorCode=BlobNotFound, ErrorMessage=The specified blob does not exist.\n..."
514                 return os.ErrNotExist
515         default:
516                 return err
517         }
518 }
519
520 var keepBlockRegexp = regexp.MustCompile(`^[0-9a-f]{32}$`)
521
522 func (v *AzureBlobVolume) isKeepBlock(s string) bool {
523         return keepBlockRegexp.MatchString(s)
524 }
525
526 // EmptyTrash looks for trashed blocks that exceeded BlobTrashLifetime
527 // and deletes them from the volume.
528 func (v *AzureBlobVolume) EmptyTrash() {
529         var bytesDeleted, bytesInTrash int64
530         var blocksDeleted, blocksInTrash int64
531
532         doBlob := func(b storage.Blob) {
533                 // Check whether the block is flagged as trash
534                 if b.Metadata["expires_at"] == "" {
535                         return
536                 }
537
538                 atomic.AddInt64(&blocksInTrash, 1)
539                 atomic.AddInt64(&bytesInTrash, b.Properties.ContentLength)
540
541                 expiresAt, err := strconv.ParseInt(b.Metadata["expires_at"], 10, 64)
542                 if err != nil {
543                         v.logger.Printf("EmptyTrash: ParseInt(%v): %v", b.Metadata["expires_at"], err)
544                         return
545                 }
546
547                 if expiresAt > time.Now().Unix() {
548                         return
549                 }
550
551                 err = v.container.DeleteBlob(b.Name, &storage.DeleteBlobOptions{
552                         IfMatch: b.Properties.Etag,
553                 })
554                 if err != nil {
555                         v.logger.Printf("EmptyTrash: DeleteBlob(%v): %v", b.Name, err)
556                         return
557                 }
558                 atomic.AddInt64(&blocksDeleted, 1)
559                 atomic.AddInt64(&bytesDeleted, b.Properties.ContentLength)
560         }
561
562         var wg sync.WaitGroup
563         todo := make(chan storage.Blob, v.cluster.Collections.BlobDeleteConcurrency)
564         for i := 0; i < v.cluster.Collections.BlobDeleteConcurrency; i++ {
565                 wg.Add(1)
566                 go func() {
567                         defer wg.Done()
568                         for b := range todo {
569                                 doBlob(b)
570                         }
571                 }()
572         }
573
574         params := storage.ListBlobsParameters{Include: &storage.IncludeBlobDataset{Metadata: true}}
575         for page := 1; ; page++ {
576                 resp, err := v.listBlobs(page, params)
577                 if err != nil {
578                         v.logger.Printf("EmptyTrash: ListBlobs: %v", err)
579                         break
580                 }
581                 for _, b := range resp.Blobs {
582                         todo <- b
583                 }
584                 if resp.NextMarker == "" {
585                         break
586                 }
587                 params.Marker = resp.NextMarker
588         }
589         close(todo)
590         wg.Wait()
591
592         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)
593 }
594
595 // InternalStats returns bucket I/O and API call counters.
596 func (v *AzureBlobVolume) InternalStats() interface{} {
597         return &v.container.stats
598 }
599
600 type azureBlobStats struct {
601         statsTicker
602         Ops              uint64
603         GetOps           uint64
604         GetRangeOps      uint64
605         GetMetadataOps   uint64
606         GetPropertiesOps uint64
607         CreateOps        uint64
608         SetMetadataOps   uint64
609         DelOps           uint64
610         ListOps          uint64
611 }
612
613 func (s *azureBlobStats) TickErr(err error) {
614         if err == nil {
615                 return
616         }
617         errType := fmt.Sprintf("%T", err)
618         if err, ok := err.(storage.AzureStorageServiceError); ok {
619                 errType = errType + fmt.Sprintf(" %d (%s)", err.StatusCode, err.Code)
620         }
621         s.statsTicker.TickErr(err, errType)
622 }
623
624 // azureContainer wraps storage.Container in order to count I/O and
625 // API usage stats.
626 type azureContainer struct {
627         ctr   *storage.Container
628         stats azureBlobStats
629 }
630
631 func (c *azureContainer) Exists() (bool, error) {
632         c.stats.TickOps("exists")
633         c.stats.Tick(&c.stats.Ops)
634         ok, err := c.ctr.Exists()
635         c.stats.TickErr(err)
636         return ok, err
637 }
638
639 func (c *azureContainer) GetBlobMetadata(bname string) (storage.BlobMetadata, error) {
640         c.stats.TickOps("get_metadata")
641         c.stats.Tick(&c.stats.Ops, &c.stats.GetMetadataOps)
642         b := c.ctr.GetBlobReference(bname)
643         err := b.GetMetadata(nil)
644         c.stats.TickErr(err)
645         return b.Metadata, err
646 }
647
648 func (c *azureContainer) GetBlobProperties(bname string) (*storage.BlobProperties, error) {
649         c.stats.TickOps("get_properties")
650         c.stats.Tick(&c.stats.Ops, &c.stats.GetPropertiesOps)
651         b := c.ctr.GetBlobReference(bname)
652         err := b.GetProperties(nil)
653         c.stats.TickErr(err)
654         return &b.Properties, err
655 }
656
657 func (c *azureContainer) GetBlob(bname string) (io.ReadCloser, error) {
658         c.stats.TickOps("get")
659         c.stats.Tick(&c.stats.Ops, &c.stats.GetOps)
660         b := c.ctr.GetBlobReference(bname)
661         rdr, err := b.Get(nil)
662         c.stats.TickErr(err)
663         return newCountingReader(rdr, c.stats.TickInBytes), err
664 }
665
666 func (c *azureContainer) GetBlobRange(bname string, start, end int, opts *storage.GetBlobOptions) (io.ReadCloser, error) {
667         c.stats.TickOps("get_range")
668         c.stats.Tick(&c.stats.Ops, &c.stats.GetRangeOps)
669         b := c.ctr.GetBlobReference(bname)
670         rdr, err := b.GetRange(&storage.GetBlobRangeOptions{
671                 Range: &storage.BlobRange{
672                         Start: uint64(start),
673                         End:   uint64(end),
674                 },
675                 GetBlobOptions: opts,
676         })
677         c.stats.TickErr(err)
678         return newCountingReader(rdr, c.stats.TickInBytes), err
679 }
680
681 // If we give it an io.Reader that doesn't also have a Len() int
682 // method, the Azure SDK determines data size by copying the data into
683 // a new buffer, which is not a good use of memory.
684 type readerWithAzureLen struct {
685         io.Reader
686         len int
687 }
688
689 // Len satisfies the private lener interface in azure-sdk-for-go.
690 func (r *readerWithAzureLen) Len() int {
691         return r.len
692 }
693
694 func (c *azureContainer) CreateBlockBlobFromReader(bname string, size int, rdr io.Reader, opts *storage.PutBlobOptions) error {
695         c.stats.TickOps("create")
696         c.stats.Tick(&c.stats.Ops, &c.stats.CreateOps)
697         if size != 0 {
698                 rdr = &readerWithAzureLen{
699                         Reader: newCountingReader(rdr, c.stats.TickOutBytes),
700                         len:    size,
701                 }
702         }
703         b := c.ctr.GetBlobReference(bname)
704         err := b.CreateBlockBlobFromReader(rdr, opts)
705         c.stats.TickErr(err)
706         return err
707 }
708
709 func (c *azureContainer) SetBlobMetadata(bname string, m storage.BlobMetadata, opts *storage.SetBlobMetadataOptions) error {
710         c.stats.TickOps("set_metadata")
711         c.stats.Tick(&c.stats.Ops, &c.stats.SetMetadataOps)
712         b := c.ctr.GetBlobReference(bname)
713         b.Metadata = m
714         err := b.SetMetadata(opts)
715         c.stats.TickErr(err)
716         return err
717 }
718
719 func (c *azureContainer) ListBlobs(params storage.ListBlobsParameters) (storage.BlobListResponse, error) {
720         c.stats.TickOps("list")
721         c.stats.Tick(&c.stats.Ops, &c.stats.ListOps)
722         resp, err := c.ctr.ListBlobs(params)
723         c.stats.TickErr(err)
724         return resp, err
725 }
726
727 func (c *azureContainer) DeleteBlob(bname string, opts *storage.DeleteBlobOptions) error {
728         c.stats.TickOps("delete")
729         c.stats.Tick(&c.stats.Ops, &c.stats.DelOps)
730         b := c.ctr.GetBlobReference(bname)
731         err := b.Delete(opts)
732         c.stats.TickErr(err)
733         return err
734 }