2960: Refactor keepstore into a streaming server.
authorTom Clegg <tom@curii.com>
Mon, 12 Feb 2024 19:46:30 +0000 (14:46 -0500)
committerTom Clegg <tom@curii.com>
Tue, 13 Feb 2024 20:40:02 +0000 (15:40 -0500)
Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tom@curii.com>

51 files changed:
sdk/go/arvados/api.go
sdk/go/keepclient/keepclient.go
sdk/python/tests/test_keep_client.py
services/keep-balance/change_set.go
services/keep-balance/change_set_test.go
services/keepproxy/keepproxy_test.go
services/keepstore/azure_blob_volume.go
services/keepstore/azure_blob_volume_test.go
services/keepstore/bufferpool.go
services/keepstore/bufferpool_test.go
services/keepstore/collision.go [deleted file]
services/keepstore/collision_test.go [deleted file]
services/keepstore/command.go
services/keepstore/count.go
services/keepstore/gocheck_test.go [deleted file]
services/keepstore/handler_test.go [deleted file]
services/keepstore/handlers.go [deleted file]
services/keepstore/hashcheckwriter.go [new file with mode: 0644]
services/keepstore/keepstore.go
services/keepstore/keepstore_test.go [new file with mode: 0644]
services/keepstore/metrics.go
services/keepstore/metrics_test.go [new file with mode: 0644]
services/keepstore/mock_mutex_for_test.go [deleted file]
services/keepstore/mounts_test.go
services/keepstore/perms.go [deleted file]
services/keepstore/perms_test.go [deleted file]
services/keepstore/pipe_adapters.go [deleted file]
services/keepstore/proxy_remote.go [deleted file]
services/keepstore/proxy_remote_test.go
services/keepstore/pull_worker.go
services/keepstore/pull_worker_integration_test.go [deleted file]
services/keepstore/pull_worker_test.go
services/keepstore/putprogress.go [new file with mode: 0644]
services/keepstore/router.go [new file with mode: 0644]
services/keepstore/router_test.go [new file with mode: 0644]
services/keepstore/s3aws_volume.go
services/keepstore/s3aws_volume_test.go
services/keepstore/status_test.go [deleted file]
services/keepstore/streamwriterat.go [new file with mode: 0644]
services/keepstore/streamwriterat_test.go [new file with mode: 0644]
services/keepstore/trash_worker.go
services/keepstore/trash_worker_test.go
services/keepstore/unix_volume.go
services/keepstore/unix_volume_test.go
services/keepstore/volume.go
services/keepstore/volume_generic_test.go
services/keepstore/volume_test.go
services/keepstore/work_queue.go [deleted file]
services/keepstore/work_queue_test.go [deleted file]
tools/keep-block-check/keep-block-check_test.go
tools/keep-rsync/keep-rsync_test.go

index a6b240e2149d5892d3fc2a2dfdbb3d3b22feb857..e7310818f7d745b55fda3ceed59bf2438bfab9e1 100644 (file)
@@ -242,8 +242,9 @@ type LogoutOptions struct {
 }
 
 type BlockReadOptions struct {
-       Locator string
-       WriteTo io.Writer
+       Locator      string
+       WriteTo      io.Writer
+       LocalLocator func(string)
 }
 
 type BlockWriteOptions struct {
@@ -258,8 +259,9 @@ type BlockWriteOptions struct {
 }
 
 type BlockWriteResponse struct {
-       Locator  string
-       Replicas int
+       Locator        string
+       Replicas       int
+       StorageClasses map[string]int
 }
 
 type WebDAVOptions struct {
index 2bd7996b59c0260caf1d61560316c3bc42e09357..64f7e47b7e14e85aa86d369da58c6fd8d1273ad7 100644 (file)
@@ -75,6 +75,8 @@ type ErrNotFound struct {
        multipleResponseError
 }
 
+func (*ErrNotFound) HTTPStatus() int { return http.StatusNotFound }
+
 type InsufficientReplicasError struct{ error }
 
 type OversizeBlockError struct{ error }
index f472c0830e65b02d9394c50e45a361c67ab48289..6b1ebf56c0826ee4e23523168b729855f6368bf8 100644 (file)
@@ -167,30 +167,30 @@ class KeepPermissionTestCase(run_test_server.TestCaseWithServers, DiskCacheBase)
                          b'foo',
                          'wrong content from Keep.get(md5("foo"))')
 
-        # GET with an unsigned locator => NotFound
+        # GET with an unsigned locator => bad request
         bar_locator = keep_client.put('bar')
         unsigned_bar_locator = "37b51d194a7513e45b56f6524f2d51f2+3"
         self.assertRegex(
             bar_locator,
             r'^37b51d194a7513e45b56f6524f2d51f2\+3\+A[a-f0-9]+@[a-f0-9]+$',
             'invalid locator from Keep.put("bar"): ' + bar_locator)
-        self.assertRaises(arvados.errors.NotFoundError,
+        self.assertRaises(arvados.errors.KeepReadError,
                           keep_client.get,
                           unsigned_bar_locator)
 
-        # GET from a different user => NotFound
+        # GET from a different user => bad request
         run_test_server.authorize_with('spectator')
-        self.assertRaises(arvados.errors.NotFoundError,
+        self.assertRaises(arvados.errors.KeepReadError,
                           arvados.Keep.get,
                           bar_locator)
 
-        # Unauthenticated GET for a signed locator => NotFound
-        # Unauthenticated GET for an unsigned locator => NotFound
+        # Unauthenticated GET for a signed locator => bad request
+        # Unauthenticated GET for an unsigned locator => bad request
         keep_client.api_token = ''
-        self.assertRaises(arvados.errors.NotFoundError,
+        self.assertRaises(arvados.errors.KeepReadError,
                           keep_client.get,
                           bar_locator)
-        self.assertRaises(arvados.errors.NotFoundError,
+        self.assertRaises(arvados.errors.KeepReadError,
                           keep_client.get,
                           unsigned_bar_locator)
 
index c3579556bb5f174781753676f0208d794a5ee620..771e277d60a4befe5367bfc2299ec80da145b2bb 100644 (file)
@@ -10,6 +10,7 @@ import (
        "sync"
 
        "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/services/keepstore"
 )
 
 // Pull is a request to retrieve a block from a remote server, and
@@ -23,13 +24,8 @@ type Pull struct {
 // MarshalJSON formats a pull request the way keepstore wants to see
 // it.
 func (p Pull) MarshalJSON() ([]byte, error) {
-       type KeepstorePullRequest struct {
-               Locator   string   `json:"locator"`
-               Servers   []string `json:"servers"`
-               MountUUID string   `json:"mount_uuid"`
-       }
-       return json.Marshal(KeepstorePullRequest{
-               Locator:   string(p.SizedDigest[:32]),
+       return json.Marshal(keepstore.PullListItem{
+               Locator:   string(p.SizedDigest),
                Servers:   []string{p.From.URLBase()},
                MountUUID: p.To.KeepMount.UUID,
        })
@@ -45,13 +41,8 @@ type Trash struct {
 // MarshalJSON formats a trash request the way keepstore wants to see
 // it, i.e., as a bare locator with no +size hint.
 func (t Trash) MarshalJSON() ([]byte, error) {
-       type KeepstoreTrashRequest struct {
-               Locator    string `json:"locator"`
-               BlockMtime int64  `json:"block_mtime"`
-               MountUUID  string `json:"mount_uuid"`
-       }
-       return json.Marshal(KeepstoreTrashRequest{
-               Locator:    string(t.SizedDigest[:32]),
+       return json.Marshal(keepstore.TrashListItem{
+               Locator:    string(t.SizedDigest),
                BlockMtime: t.Mtime,
                MountUUID:  t.From.KeepMount.UUID,
        })
index 5474d29fb57e2d64a67286382b1d53907afe3ae7..f2b9429017cf52a4b21398a4d8106b70a3e34757 100644 (file)
@@ -33,12 +33,12 @@ func (s *changeSetSuite) TestJSONFormat(c *check.C) {
                To:          mnt,
                From:        srv}})
        c.Check(err, check.IsNil)
-       c.Check(string(buf), check.Equals, `[{"locator":"acbd18db4cc2f85cedef654fccc4a4d8","servers":["http://keep1.zzzzz.arvadosapi.com:25107"],"mount_uuid":"zzzzz-mount-abcdefghijklmno"}]`)
+       c.Check(string(buf), check.Equals, `[{"locator":"acbd18db4cc2f85cedef654fccc4a4d8+3","servers":["http://keep1.zzzzz.arvadosapi.com:25107"],"mount_uuid":"zzzzz-mount-abcdefghijklmno"}]`)
 
        buf, err = json.Marshal([]Trash{{
                SizedDigest: arvados.SizedDigest("acbd18db4cc2f85cedef654fccc4a4d8+3"),
                From:        mnt,
                Mtime:       123456789}})
        c.Check(err, check.IsNil)
-       c.Check(string(buf), check.Equals, `[{"locator":"acbd18db4cc2f85cedef654fccc4a4d8","block_mtime":123456789,"mount_uuid":"zzzzz-mount-abcdefghijklmno"}]`)
+       c.Check(string(buf), check.Equals, `[{"locator":"acbd18db4cc2f85cedef654fccc4a4d8+3","block_mtime":123456789,"mount_uuid":"zzzzz-mount-abcdefghijklmno"}]`)
 }
index 7efba2348b6593a1232edb6c719253ef388674dd..0390bd1fd0da0a728dcb146938b20bd54da7bb65 100644 (file)
@@ -346,7 +346,7 @@ func (s *ServerRequiredSuite) TestPutAskGet(c *C) {
        }
 
        {
-               reader, _, _, err := kc.Get(hash)
+               reader, _, _, err := kc.Get(hash + "+3")
                c.Check(reader, Equals, nil)
                c.Check(err, Equals, keepclient.BlockNotFound)
                c.Log("Finished Get (expected BlockNotFound)")
index 56a52c913a196149d3d4bbe03ec1f8f382018b72..bdd669bb4649e2471f0a36035a7f96dfcf6ccd8a 100644 (file)
@@ -5,13 +5,11 @@
 package keepstore
 
 import (
-       "bytes"
        "context"
        "encoding/json"
        "errors"
        "fmt"
        "io"
-       "io/ioutil"
        "net/http"
        "os"
        "regexp"
@@ -32,17 +30,18 @@ func init() {
        driver["Azure"] = newAzureBlobVolume
 }
 
-func newAzureBlobVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) {
+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
        }
@@ -81,7 +80,7 @@ func newAzureBlobVolume(cluster *arvados.Cluster, volume arvados.Volume, logger
 }
 
 func (v *AzureBlobVolume) check() error {
-       lbls := prometheus.Labels{"device_id": v.GetDeviceID()}
+       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
 }
@@ -108,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.
@@ -124,13 +124,8 @@ 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
 }
 
@@ -146,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
@@ -178,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
                }
@@ -188,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()
 
@@ -206,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
@@ -252,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 {
@@ -282,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
@@ -295,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)
        }
@@ -310,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
@@ -370,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()
@@ -390,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
        }
@@ -404,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
        }
@@ -417,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
@@ -467,7 +436,7 @@ func (v *AzureBlobVolume) listBlobs(page int, params storage.ListBlobsParameters
        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
@@ -479,10 +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 && !v.volume.AllowTrashWhenReadOnly {
-               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
@@ -513,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)
        }
@@ -527,24 +493,10 @@ 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 {
@@ -553,7 +505,7 @@ func (v *AzureBlobVolume) translateError(err error) error {
                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
@@ -637,7 +589,7 @@ 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.
@@ -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,
                }
        }
index 48d58ee9bfc454e5b2972e6d36867a578c29e6bb..a543dfc245174d26f6d8a66fcfa47b6d54215d0e 100644 (file)
@@ -13,6 +13,7 @@ import (
        "encoding/xml"
        "flag"
        "fmt"
+       "io"
        "io/ioutil"
        "math/rand"
        "net"
@@ -87,7 +88,7 @@ func (h *azStubHandler) TouchWithDate(container, hash string, t time.Time) {
        blob.Mtime = t
 }
 
-func (h *azStubHandler) PutRaw(container, hash string, data []byte) {
+func (h *azStubHandler) BlockWriteRaw(container, hash string, data []byte) {
        h.Lock()
        defer h.Unlock()
        h.blobs[container+"|"+hash] = &azBlob{
@@ -365,14 +366,14 @@ func (d *azStubDialer) Dial(network, address string) (net.Conn, error) {
        return d.Dialer.Dial(network, address)
 }
 
-type TestableAzureBlobVolume struct {
+type testableAzureBlobVolume struct {
        *AzureBlobVolume
        azHandler *azStubHandler
        azStub    *httptest.Server
        t         TB
 }
 
-func (s *StubbedAzureBlobSuite) newTestableAzureBlobVolume(t TB, cluster *arvados.Cluster, volume arvados.Volume, metrics *volumeMetricsVecs) *TestableAzureBlobVolume {
+func (s *stubbedAzureBlobSuite) newTestableAzureBlobVolume(t TB, params newVolumeParams) *testableAzureBlobVolume {
        azHandler := newAzStubHandler(t.(*check.C))
        azStub := httptest.NewServer(azHandler)
 
@@ -404,16 +405,17 @@ func (s *StubbedAzureBlobSuite) newTestableAzureBlobVolume(t TB, cluster *arvado
                ListBlobsRetryDelay:  arvados.Duration(time.Millisecond),
                azClient:             azClient,
                container:            &azureContainer{ctr: bs.GetContainerReference(container)},
-               cluster:              cluster,
-               volume:               volume,
+               cluster:              params.Cluster,
+               volume:               params.ConfigVolume,
                logger:               ctxlog.TestLogger(t),
-               metrics:              metrics,
+               metrics:              params.MetricsVecs,
+               bufferPool:           params.BufferPool,
        }
        if err = v.check(); err != nil {
                t.Fatal(err)
        }
 
-       return &TestableAzureBlobVolume{
+       return &testableAzureBlobVolume{
                AzureBlobVolume: v,
                azHandler:       azHandler,
                azStub:          azStub,
@@ -421,48 +423,54 @@ func (s *StubbedAzureBlobSuite) newTestableAzureBlobVolume(t TB, cluster *arvado
        }
 }
 
-var _ = check.Suite(&StubbedAzureBlobSuite{})
+var _ = check.Suite(&stubbedAzureBlobSuite{})
 
-type StubbedAzureBlobSuite struct {
+type stubbedAzureBlobSuite struct {
        origHTTPTransport http.RoundTripper
 }
 
-func (s *StubbedAzureBlobSuite) SetUpTest(c *check.C) {
+func (s *stubbedAzureBlobSuite) SetUpSuite(c *check.C) {
        s.origHTTPTransport = http.DefaultTransport
        http.DefaultTransport = &http.Transport{
                Dial: (&azStubDialer{logger: ctxlog.TestLogger(c)}).Dial,
        }
 }
 
-func (s *StubbedAzureBlobSuite) TearDownTest(c *check.C) {
+func (s *stubbedAzureBlobSuite) TearDownSuite(c *check.C) {
        http.DefaultTransport = s.origHTTPTransport
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeWithGeneric(c *check.C) {
-       DoGenericVolumeTests(c, false, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               return s.newTestableAzureBlobVolume(t, cluster, volume, metrics)
+func (s *stubbedAzureBlobSuite) TestAzureBlobVolumeWithGeneric(c *check.C) {
+       DoGenericVolumeTests(c, false, func(t TB, params newVolumeParams) TestableVolume {
+               return s.newTestableAzureBlobVolume(t, params)
        })
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeConcurrentRanges(c *check.C) {
+func (s *stubbedAzureBlobSuite) TestAzureBlobVolumeConcurrentRanges(c *check.C) {
        // Test (BlockSize mod azureMaxGetBytes)==0 and !=0 cases
-       for _, b := range []int{2 << 22, 2<<22 - 1} {
-               DoGenericVolumeTests(c, false, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-                       v := s.newTestableAzureBlobVolume(t, cluster, volume, metrics)
+       for _, b := range []int{2<<22 - 1, 2<<22 - 1} {
+               c.Logf("=== MaxGetBytes=%d", b)
+               DoGenericVolumeTests(c, false, func(t TB, params newVolumeParams) TestableVolume {
+                       v := s.newTestableAzureBlobVolume(t, params)
                        v.MaxGetBytes = b
                        return v
                })
        }
 }
 
-func (s *StubbedAzureBlobSuite) TestReadonlyAzureBlobVolumeWithGeneric(c *check.C) {
-       DoGenericVolumeTests(c, false, func(c TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               return s.newTestableAzureBlobVolume(c, cluster, volume, metrics)
+func (s *stubbedAzureBlobSuite) TestReadonlyAzureBlobVolumeWithGeneric(c *check.C) {
+       DoGenericVolumeTests(c, false, func(c TB, params newVolumeParams) TestableVolume {
+               return s.newTestableAzureBlobVolume(c, params)
        })
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeRangeFenceposts(c *check.C) {
-       v := s.newTestableAzureBlobVolume(c, testCluster(c), arvados.Volume{Replication: 3}, newVolumeMetricsVecs(prometheus.NewRegistry()))
+func (s *stubbedAzureBlobSuite) TestAzureBlobVolumeRangeFenceposts(c *check.C) {
+       v := s.newTestableAzureBlobVolume(c, newVolumeParams{
+               Cluster:      testCluster(c),
+               ConfigVolume: arvados.Volume{Replication: 3},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       })
        defer v.Teardown()
 
        for _, size := range []int{
@@ -478,16 +486,16 @@ func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeRangeFenceposts(c *check.C) {
                        data[i] = byte((i + 7) & 0xff)
                }
                hash := fmt.Sprintf("%x", md5.Sum(data))
-               err := v.Put(context.Background(), hash, data)
+               err := v.BlockWrite(context.Background(), hash, data)
                if err != nil {
                        c.Error(err)
                }
-               gotData := make([]byte, len(data))
-               gotLen, err := v.Get(context.Background(), hash, gotData)
+               gotData := bytes.NewBuffer(nil)
+               gotLen, err := v.BlockRead(context.Background(), hash, gotData)
                if err != nil {
                        c.Error(err)
                }
-               gotHash := fmt.Sprintf("%x", md5.Sum(gotData))
+               gotHash := fmt.Sprintf("%x", md5.Sum(gotData.Bytes()))
                if gotLen != size {
                        c.Errorf("length mismatch: got %d != %d", gotLen, size)
                }
@@ -497,8 +505,13 @@ func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeRangeFenceposts(c *check.C) {
        }
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeCreateBlobRace(c *check.C) {
-       v := s.newTestableAzureBlobVolume(c, testCluster(c), arvados.Volume{Replication: 3}, newVolumeMetricsVecs(prometheus.NewRegistry()))
+func (s *stubbedAzureBlobSuite) TestAzureBlobVolumeCreateBlobRace(c *check.C) {
+       v := s.newTestableAzureBlobVolume(c, newVolumeParams{
+               Cluster:      testCluster(c),
+               ConfigVolume: arvados.Volume{Replication: 3},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       })
        defer v.Teardown()
 
        var wg sync.WaitGroup
@@ -508,42 +521,46 @@ func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeCreateBlobRace(c *check.C) {
        wg.Add(1)
        go func() {
                defer wg.Done()
-               err := v.Put(context.Background(), TestHash, TestBlock)
+               err := v.BlockWrite(context.Background(), TestHash, TestBlock)
                if err != nil {
                        c.Error(err)
                }
        }()
-       continuePut := make(chan struct{})
-       // Wait for the stub's Put to create the empty blob
-       v.azHandler.race <- continuePut
+       continueBlockWrite := make(chan struct{})
+       // Wait for the stub's BlockWrite to create the empty blob
+       v.azHandler.race <- continueBlockWrite
        wg.Add(1)
        go func() {
                defer wg.Done()
-               buf := make([]byte, len(TestBlock))
-               _, err := v.Get(context.Background(), TestHash, buf)
+               _, err := v.BlockRead(context.Background(), TestHash, io.Discard)
                if err != nil {
                        c.Error(err)
                }
        }()
-       // Wait for the stub's Get to get the empty blob
+       // Wait for the stub's BlockRead to get the empty blob
        close(v.azHandler.race)
-       // Allow stub's Put to continue, so the real data is ready
-       // when the volume's Get retries
-       <-continuePut
-       // Wait for Get() and Put() to finish
+       // Allow stub's BlockWrite to continue, so the real data is ready
+       // when the volume's BlockRead retries
+       <-continueBlockWrite
+       // Wait for BlockRead() and BlockWrite() to finish
        wg.Wait()
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeCreateBlobRaceDeadline(c *check.C) {
-       v := s.newTestableAzureBlobVolume(c, testCluster(c), arvados.Volume{Replication: 3}, newVolumeMetricsVecs(prometheus.NewRegistry()))
+func (s *stubbedAzureBlobSuite) TestAzureBlobVolumeCreateBlobRaceDeadline(c *check.C) {
+       v := s.newTestableAzureBlobVolume(c, newVolumeParams{
+               Cluster:      testCluster(c),
+               ConfigVolume: arvados.Volume{Replication: 3},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       })
        v.AzureBlobVolume.WriteRaceInterval.Set("2s")
        v.AzureBlobVolume.WriteRacePollTime.Set("5ms")
        defer v.Teardown()
 
-       v.PutRaw(TestHash, nil)
+       v.BlockWriteRaw(TestHash, nil)
 
        buf := new(bytes.Buffer)
-       v.IndexTo("", buf)
+       v.Index(context.Background(), "", buf)
        if buf.Len() != 0 {
                c.Errorf("Index %+q should be empty", buf.Bytes())
        }
@@ -553,52 +570,50 @@ func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeCreateBlobRaceDeadline(c *che
        allDone := make(chan struct{})
        go func() {
                defer close(allDone)
-               buf := make([]byte, BlockSize)
-               n, err := v.Get(context.Background(), TestHash, buf)
+               buf := bytes.NewBuffer(nil)
+               n, err := v.BlockRead(context.Background(), TestHash, buf)
                if err != nil {
                        c.Error(err)
                        return
                }
                if n != 0 {
-                       c.Errorf("Got %+q, expected empty buf", buf[:n])
+                       c.Errorf("Got %+q (n=%d), expected empty buf", buf.Bytes(), n)
                }
        }()
        select {
        case <-allDone:
        case <-time.After(time.Second):
-               c.Error("Get should have stopped waiting for race when block was 2s old")
+               c.Error("BlockRead should have stopped waiting for race when block was 2s old")
        }
 
        buf.Reset()
-       v.IndexTo("", buf)
+       v.Index(context.Background(), "", buf)
        if !bytes.HasPrefix(buf.Bytes(), []byte(TestHash+"+0")) {
                c.Errorf("Index %+q should have %+q", buf.Bytes(), TestHash+"+0")
        }
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeContextCancelGet(c *check.C) {
-       s.testAzureBlobVolumeContextCancel(c, func(ctx context.Context, v *TestableAzureBlobVolume) error {
-               v.PutRaw(TestHash, TestBlock)
-               _, err := v.Get(ctx, TestHash, make([]byte, BlockSize))
+func (s *stubbedAzureBlobSuite) TestAzureBlobVolumeContextCancelBlockRead(c *check.C) {
+       s.testAzureBlobVolumeContextCancel(c, func(ctx context.Context, v *testableAzureBlobVolume) error {
+               v.BlockWriteRaw(TestHash, TestBlock)
+               _, err := v.BlockRead(ctx, TestHash, io.Discard)
                return err
        })
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeContextCancelPut(c *check.C) {
-       s.testAzureBlobVolumeContextCancel(c, func(ctx context.Context, v *TestableAzureBlobVolume) error {
-               return v.Put(ctx, TestHash, make([]byte, BlockSize))
+func (s *stubbedAzureBlobSuite) TestAzureBlobVolumeContextCancelBlockWrite(c *check.C) {
+       s.testAzureBlobVolumeContextCancel(c, func(ctx context.Context, v *testableAzureBlobVolume) error {
+               return v.BlockWrite(ctx, TestHash, make([]byte, BlockSize))
        })
 }
 
-func (s *StubbedAzureBlobSuite) TestAzureBlobVolumeContextCancelCompare(c *check.C) {
-       s.testAzureBlobVolumeContextCancel(c, func(ctx context.Context, v *TestableAzureBlobVolume) error {
-               v.PutRaw(TestHash, TestBlock)
-               return v.Compare(ctx, TestHash, TestBlock2)
+func (s *stubbedAzureBlobSuite) testAzureBlobVolumeContextCancel(c *check.C, testFunc func(context.Context, *testableAzureBlobVolume) error) {
+       v := s.newTestableAzureBlobVolume(c, newVolumeParams{
+               Cluster:      testCluster(c),
+               ConfigVolume: arvados.Volume{Replication: 3},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
        })
-}
-
-func (s *StubbedAzureBlobSuite) testAzureBlobVolumeContextCancel(c *check.C, testFunc func(context.Context, *TestableAzureBlobVolume) error) {
-       v := s.newTestableAzureBlobVolume(c, testCluster(c), arvados.Volume{Replication: 3}, newVolumeMetricsVecs(prometheus.NewRegistry()))
        defer v.Teardown()
        v.azHandler.race = make(chan chan struct{})
 
@@ -633,8 +648,13 @@ func (s *StubbedAzureBlobSuite) testAzureBlobVolumeContextCancel(c *check.C, tes
        }()
 }
 
-func (s *StubbedAzureBlobSuite) TestStats(c *check.C) {
-       volume := s.newTestableAzureBlobVolume(c, testCluster(c), arvados.Volume{Replication: 3}, newVolumeMetricsVecs(prometheus.NewRegistry()))
+func (s *stubbedAzureBlobSuite) TestStats(c *check.C) {
+       volume := s.newTestableAzureBlobVolume(c, newVolumeParams{
+               Cluster:      testCluster(c),
+               ConfigVolume: arvados.Volume{Replication: 3},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       })
        defer volume.Teardown()
 
        stats := func() string {
@@ -647,38 +667,38 @@ func (s *StubbedAzureBlobSuite) TestStats(c *check.C) {
        c.Check(stats(), check.Matches, `.*"Errors":0,.*`)
 
        loc := "acbd18db4cc2f85cedef654fccc4a4d8"
-       _, err := volume.Get(context.Background(), loc, make([]byte, 3))
+       _, err := volume.BlockRead(context.Background(), loc, io.Discard)
        c.Check(err, check.NotNil)
        c.Check(stats(), check.Matches, `.*"Ops":[^0],.*`)
        c.Check(stats(), check.Matches, `.*"Errors":[^0],.*`)
        c.Check(stats(), check.Matches, `.*"storage\.AzureStorageServiceError 404 \(404 Not Found\)":[^0].*`)
        c.Check(stats(), check.Matches, `.*"InBytes":0,.*`)
 
-       err = volume.Put(context.Background(), loc, []byte("foo"))
+       err = volume.BlockWrite(context.Background(), loc, []byte("foo"))
        c.Check(err, check.IsNil)
        c.Check(stats(), check.Matches, `.*"OutBytes":3,.*`)
        c.Check(stats(), check.Matches, `.*"CreateOps":1,.*`)
 
-       _, err = volume.Get(context.Background(), loc, make([]byte, 3))
+       _, err = volume.BlockRead(context.Background(), loc, io.Discard)
        c.Check(err, check.IsNil)
-       _, err = volume.Get(context.Background(), loc, make([]byte, 3))
+       _, err = volume.BlockRead(context.Background(), loc, io.Discard)
        c.Check(err, check.IsNil)
        c.Check(stats(), check.Matches, `.*"InBytes":6,.*`)
 }
 
-func (v *TestableAzureBlobVolume) PutRaw(locator string, data []byte) {
-       v.azHandler.PutRaw(v.ContainerName, locator, data)
+func (v *testableAzureBlobVolume) BlockWriteRaw(locator string, data []byte) {
+       v.azHandler.BlockWriteRaw(v.ContainerName, locator, data)
 }
 
-func (v *TestableAzureBlobVolume) TouchWithDate(locator string, lastPut time.Time) {
-       v.azHandler.TouchWithDate(v.ContainerName, locator, lastPut)
+func (v *testableAzureBlobVolume) TouchWithDate(locator string, lastBlockWrite time.Time) {
+       v.azHandler.TouchWithDate(v.ContainerName, locator, lastBlockWrite)
 }
 
-func (v *TestableAzureBlobVolume) Teardown() {
+func (v *testableAzureBlobVolume) Teardown() {
        v.azStub.Close()
 }
 
-func (v *TestableAzureBlobVolume) ReadWriteOperationLabelValues() (r, w string) {
+func (v *testableAzureBlobVolume) ReadWriteOperationLabelValues() (r, w string) {
        return "get", "create"
 }
 
index b4cc5d38e1670034212816bd96b95cdc838a2cfb..811715b191c7384cfe904744221cbffcd3ac1b46 100644 (file)
@@ -5,13 +5,17 @@
 package keepstore
 
 import (
+       "context"
        "sync"
        "sync/atomic"
        "time"
 
+       "github.com/prometheus/client_golang/prometheus"
        "github.com/sirupsen/logrus"
 )
 
+var bufferPoolBlockSize = BlockSize // modified by tests
+
 type bufferPool struct {
        log logrus.FieldLogger
        // limiter has a "true" placeholder for each in-use buffer.
@@ -22,17 +26,67 @@ type bufferPool struct {
        sync.Pool
 }
 
-func newBufferPool(log logrus.FieldLogger, count int, bufSize int) *bufferPool {
+func newBufferPool(log logrus.FieldLogger, count int, reg *prometheus.Registry) *bufferPool {
        p := bufferPool{log: log}
        p.Pool.New = func() interface{} {
-               atomic.AddUint64(&p.allocated, uint64(bufSize))
-               return make([]byte, bufSize)
+               atomic.AddUint64(&p.allocated, uint64(bufferPoolBlockSize))
+               return make([]byte, bufferPoolBlockSize)
        }
        p.limiter = make(chan bool, count)
+       if reg != nil {
+               reg.MustRegister(prometheus.NewGaugeFunc(
+                       prometheus.GaugeOpts{
+                               Namespace: "arvados",
+                               Subsystem: "keepstore",
+                               Name:      "bufferpool_allocated_bytes",
+                               Help:      "Number of bytes allocated to buffers",
+                       },
+                       func() float64 { return float64(p.Alloc()) },
+               ))
+               reg.MustRegister(prometheus.NewGaugeFunc(
+                       prometheus.GaugeOpts{
+                               Namespace: "arvados",
+                               Subsystem: "keepstore",
+                               Name:      "bufferpool_max_buffers",
+                               Help:      "Maximum number of buffers allowed",
+                       },
+                       func() float64 { return float64(p.Cap()) },
+               ))
+               reg.MustRegister(prometheus.NewGaugeFunc(
+                       prometheus.GaugeOpts{
+                               Namespace: "arvados",
+                               Subsystem: "keepstore",
+                               Name:      "bufferpool_inuse_buffers",
+                               Help:      "Number of buffers in use",
+                       },
+                       func() float64 { return float64(p.Len()) },
+               ))
+       }
        return &p
 }
 
-func (p *bufferPool) Get(size int) []byte {
+// GetContext gets a buffer from the pool -- but gives up and returns
+// ctx.Err() if ctx ends before a buffer is available.
+func (p *bufferPool) GetContext(ctx context.Context) ([]byte, error) {
+       bufReady := make(chan []byte)
+       go func() {
+               bufReady <- p.Get()
+       }()
+       select {
+       case buf := <-bufReady:
+               return buf, nil
+       case <-ctx.Done():
+               go func() {
+                       // Even if closeNotifier happened first, we
+                       // need to keep waiting for our buf so we can
+                       // return it to the pool.
+                       p.Put(<-bufReady)
+               }()
+               return nil, ctx.Err()
+       }
+}
+
+func (p *bufferPool) Get() []byte {
        select {
        case p.limiter <- true:
        default:
@@ -42,14 +96,14 @@ func (p *bufferPool) Get(size int) []byte {
                p.log.Printf("waited %v for a buffer", time.Since(t0))
        }
        buf := p.Pool.Get().([]byte)
-       if cap(buf) < size {
-               p.log.Fatalf("bufferPool Get(size=%d) but max=%d", size, cap(buf))
+       if len(buf) < bufferPoolBlockSize {
+               p.log.Fatalf("bufferPoolBlockSize=%d but cap(buf)=%d", bufferPoolBlockSize, len(buf))
        }
-       return buf[:size]
+       return buf
 }
 
 func (p *bufferPool) Put(buf []byte) {
-       p.Pool.Put(buf)
+       p.Pool.Put(buf[:cap(buf)])
        <-p.limiter
 }
 
index 13e1cb4f332ba180857aef747b3086e9251466ee..8ecc833228f5b07218a6b0ea8ba58f2f44616c7e 100644 (file)
@@ -5,55 +5,54 @@
 package keepstore
 
 import (
-       "context"
        "time"
 
        "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "github.com/prometheus/client_golang/prometheus"
        . "gopkg.in/check.v1"
 )
 
 var _ = Suite(&BufferPoolSuite{})
 
+var bufferPoolTestSize = 10
+
 type BufferPoolSuite struct{}
 
-// Initialize a default-sized buffer pool for the benefit of test
-// suites that don't run main().
-func init() {
-       bufs = newBufferPool(ctxlog.FromContext(context.Background()), 12, BlockSize)
+func (s *BufferPoolSuite) SetUpTest(c *C) {
+       bufferPoolBlockSize = bufferPoolTestSize
 }
 
-// Restore sane default after bufferpool's own tests
 func (s *BufferPoolSuite) TearDownTest(c *C) {
-       bufs = newBufferPool(ctxlog.FromContext(context.Background()), 12, BlockSize)
+       bufferPoolBlockSize = BlockSize
 }
 
 func (s *BufferPoolSuite) TestBufferPoolBufSize(c *C) {
-       bufs := newBufferPool(ctxlog.TestLogger(c), 2, 10)
-       b1 := bufs.Get(1)
-       bufs.Get(2)
+       bufs := newBufferPool(ctxlog.TestLogger(c), 2, prometheus.NewRegistry())
+       b1 := bufs.Get()
+       bufs.Get()
        bufs.Put(b1)
-       b3 := bufs.Get(3)
-       c.Check(len(b3), Equals, 3)
+       b3 := bufs.Get()
+       c.Check(len(b3), Equals, bufferPoolTestSize)
 }
 
 func (s *BufferPoolSuite) TestBufferPoolUnderLimit(c *C) {
-       bufs := newBufferPool(ctxlog.TestLogger(c), 3, 10)
-       b1 := bufs.Get(10)
-       bufs.Get(10)
+       bufs := newBufferPool(ctxlog.TestLogger(c), 3, prometheus.NewRegistry())
+       b1 := bufs.Get()
+       bufs.Get()
        testBufferPoolRace(c, bufs, b1, "Get")
 }
 
 func (s *BufferPoolSuite) TestBufferPoolAtLimit(c *C) {
-       bufs := newBufferPool(ctxlog.TestLogger(c), 2, 10)
-       b1 := bufs.Get(10)
-       bufs.Get(10)
+       bufs := newBufferPool(ctxlog.TestLogger(c), 2, prometheus.NewRegistry())
+       b1 := bufs.Get()
+       bufs.Get()
        testBufferPoolRace(c, bufs, b1, "Put")
 }
 
 func testBufferPoolRace(c *C, bufs *bufferPool, unused []byte, expectWin string) {
        race := make(chan string)
        go func() {
-               bufs.Get(10)
+               bufs.Get()
                time.Sleep(time.Millisecond)
                race <- "Get"
        }()
@@ -68,9 +67,9 @@ func testBufferPoolRace(c *C, bufs *bufferPool, unused []byte, expectWin string)
 }
 
 func (s *BufferPoolSuite) TestBufferPoolReuse(c *C) {
-       bufs := newBufferPool(ctxlog.TestLogger(c), 2, 10)
-       bufs.Get(10)
-       last := bufs.Get(10)
+       bufs := newBufferPool(ctxlog.TestLogger(c), 2, prometheus.NewRegistry())
+       bufs.Get()
+       last := bufs.Get()
        // The buffer pool is allowed to throw away unused buffers
        // (e.g., during sync.Pool's garbage collection hook, in the
        // the current implementation). However, if unused buffers are
@@ -81,7 +80,7 @@ func (s *BufferPoolSuite) TestBufferPoolReuse(c *C) {
        reuses := 0
        for i := 0; i < allocs; i++ {
                bufs.Put(last)
-               next := bufs.Get(10)
+               next := bufs.Get()
                copy(last, []byte("last"))
                copy(next, []byte("next"))
                if last[0] == 'n' {
diff --git a/services/keepstore/collision.go b/services/keepstore/collision.go
deleted file mode 100644 (file)
index 16f2d09..0000000
+++ /dev/null
@@ -1,100 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "bytes"
-       "context"
-       "crypto/md5"
-       "fmt"
-       "io"
-)
-
-// Compute the MD5 digest of a data block (consisting of buf1 + buf2 +
-// all bytes readable from rdr). If all data is read successfully,
-// return DiskHashError or CollisionError depending on whether it
-// matches expectMD5. If an error occurs while reading, return that
-// error.
-//
-// "content has expected MD5" is called a collision because this
-// function is used in cases where we have another block in hand with
-// the given MD5 but different content.
-func collisionOrCorrupt(expectMD5 string, buf1, buf2 []byte, rdr io.Reader) error {
-       outcome := make(chan error)
-       data := make(chan []byte, 1)
-       go func() {
-               h := md5.New()
-               for b := range data {
-                       h.Write(b)
-               }
-               if fmt.Sprintf("%x", h.Sum(nil)) == expectMD5 {
-                       outcome <- CollisionError
-               } else {
-                       outcome <- DiskHashError
-               }
-       }()
-       data <- buf1
-       if buf2 != nil {
-               data <- buf2
-       }
-       var err error
-       for rdr != nil && err == nil {
-               buf := make([]byte, 1<<18)
-               var n int
-               n, err = rdr.Read(buf)
-               data <- buf[:n]
-       }
-       close(data)
-       if rdr != nil && err != io.EOF {
-               <-outcome
-               return err
-       }
-       return <-outcome
-}
-
-func compareReaderWithBuf(ctx context.Context, rdr io.Reader, expect []byte, hash string) error {
-       bufLen := 1 << 20
-       if bufLen > len(expect) && len(expect) > 0 {
-               // No need for bufLen to be longer than
-               // expect, except that len(buf)==0 would
-               // prevent us from handling empty readers the
-               // same way as non-empty readers: reading 0
-               // bytes at a time never reaches EOF.
-               bufLen = len(expect)
-       }
-       buf := make([]byte, bufLen)
-       cmp := expect
-
-       // Loop invariants: all data read so far matched what
-       // we expected, and the first N bytes of cmp are
-       // expected to equal the next N bytes read from
-       // rdr.
-       for {
-               ready := make(chan bool)
-               var n int
-               var err error
-               go func() {
-                       n, err = rdr.Read(buf)
-                       close(ready)
-               }()
-               select {
-               case <-ready:
-               case <-ctx.Done():
-                       return ctx.Err()
-               }
-               if n > len(cmp) || bytes.Compare(cmp[:n], buf[:n]) != 0 {
-                       return collisionOrCorrupt(hash, expect[:len(expect)-len(cmp)], buf[:n], rdr)
-               }
-               cmp = cmp[n:]
-               if err == io.EOF {
-                       if len(cmp) != 0 {
-                               return collisionOrCorrupt(hash, expect[:len(expect)-len(cmp)], nil, nil)
-                       }
-                       return nil
-               } else if err != nil {
-                       return err
-               }
-       }
-}
diff --git a/services/keepstore/collision_test.go b/services/keepstore/collision_test.go
deleted file mode 100644 (file)
index aa8f0cb..0000000
+++ /dev/null
@@ -1,51 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "bytes"
-       "testing/iotest"
-
-       check "gopkg.in/check.v1"
-)
-
-var _ = check.Suite(&CollisionSuite{})
-
-type CollisionSuite struct{}
-
-func (s *CollisionSuite) TestCollisionOrCorrupt(c *check.C) {
-       fooMD5 := "acbd18db4cc2f85cedef654fccc4a4d8"
-
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f'}, []byte{'o'}, bytes.NewBufferString("o")),
-               check.Equals, CollisionError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f'}, nil, bytes.NewBufferString("oo")),
-               check.Equals, CollisionError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f'}, []byte{'o', 'o'}, nil),
-               check.Equals, CollisionError)
-       c.Check(collisionOrCorrupt(fooMD5, nil, []byte{}, bytes.NewBufferString("foo")),
-               check.Equals, CollisionError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f', 'o', 'o'}, nil, bytes.NewBufferString("")),
-               check.Equals, CollisionError)
-       c.Check(collisionOrCorrupt(fooMD5, nil, nil, iotest.NewReadLogger("foo: ", iotest.DataErrReader(iotest.OneByteReader(bytes.NewBufferString("foo"))))),
-               check.Equals, CollisionError)
-
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f', 'o', 'o'}, nil, bytes.NewBufferString("bar")),
-               check.Equals, DiskHashError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f', 'o'}, nil, nil),
-               check.Equals, DiskHashError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{}, nil, bytes.NewBufferString("")),
-               check.Equals, DiskHashError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f', 'O'}, nil, bytes.NewBufferString("o")),
-               check.Equals, DiskHashError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f', 'O', 'o'}, nil, nil),
-               check.Equals, DiskHashError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f', 'o'}, []byte{'O'}, nil),
-               check.Equals, DiskHashError)
-       c.Check(collisionOrCorrupt(fooMD5, []byte{'f', 'o'}, nil, bytes.NewBufferString("O")),
-               check.Equals, DiskHashError)
-
-       c.Check(collisionOrCorrupt(fooMD5, []byte{}, nil, iotest.TimeoutReader(iotest.OneByteReader(bytes.NewBufferString("foo")))),
-               check.Equals, iotest.ErrTimeout)
-}
index 48c8256a3ca1e22e524b60fe89ebc7de26be54e7..d01b30c907fcee9da5215bc442e0d8462954ef3e 100644 (file)
@@ -8,20 +8,13 @@ import (
        "context"
        "errors"
        "flag"
-       "fmt"
        "io"
-       "math/rand"
-       "net/http"
-       "os"
-       "sync"
 
        "git.arvados.org/arvados.git/lib/cmd"
        "git.arvados.org/arvados.git/lib/config"
        "git.arvados.org/arvados.git/lib/service"
        "git.arvados.org/arvados.git/sdk/go/arvados"
-       "git.arvados.org/arvados.git/sdk/go/arvadosclient"
        "git.arvados.org/arvados.git/sdk/go/ctxlog"
-       "git.arvados.org/arvados.git/sdk/go/keepclient"
        "github.com/prometheus/client_golang/prometheus"
        "github.com/sirupsen/logrus"
 )
@@ -108,112 +101,17 @@ func convertKeepstoreFlagsToServiceFlags(prog string, args []string, lgr logrus.
        return loader.MungeLegacyConfigArgs(lgr, args, "-legacy-keepstore-config"), true, 0
 }
 
-type handler struct {
-       http.Handler
-       Cluster *arvados.Cluster
-       Logger  logrus.FieldLogger
-
-       pullq      *WorkQueue
-       trashq     *WorkQueue
-       volmgr     *RRVolumeManager
-       keepClient *keepclient.KeepClient
-
-       err       error
-       setupOnce sync.Once
-}
-
-func (h *handler) CheckHealth() error {
-       return h.err
-}
-
-func (h *handler) Done() <-chan struct{} {
-       return nil
-}
-
 func newHandlerOrErrorHandler(ctx context.Context, cluster *arvados.Cluster, token string, reg *prometheus.Registry) service.Handler {
-       var h handler
        serviceURL, ok := service.URLFromContext(ctx)
        if !ok {
                return service.ErrorHandler(ctx, cluster, errors.New("BUG: no URL from service.URLFromContext"))
        }
-       err := h.setup(ctx, cluster, token, reg, serviceURL)
+       ks, err := newKeepstore(ctx, cluster, token, reg, serviceURL)
        if err != nil {
                return service.ErrorHandler(ctx, cluster, err)
        }
-       return &h
-}
-
-func (h *handler) setup(ctx context.Context, cluster *arvados.Cluster, token string, reg *prometheus.Registry, serviceURL arvados.URL) error {
-       h.Cluster = cluster
-       h.Logger = ctxlog.FromContext(ctx)
-       if h.Cluster.API.MaxKeepBlobBuffers <= 0 {
-               return fmt.Errorf("API.MaxKeepBlobBuffers must be greater than zero")
-       }
-       bufs = newBufferPool(h.Logger, h.Cluster.API.MaxKeepBlobBuffers, BlockSize)
-
-       if h.Cluster.API.MaxConcurrentRequests > 0 && h.Cluster.API.MaxConcurrentRequests < h.Cluster.API.MaxKeepBlobBuffers {
-               h.Logger.Warnf("Possible configuration mistake: not useful to set API.MaxKeepBlobBuffers (%d) higher than API.MaxConcurrentRequests (%d)", h.Cluster.API.MaxKeepBlobBuffers, h.Cluster.API.MaxConcurrentRequests)
-       }
-
-       if h.Cluster.Collections.BlobSigningKey != "" {
-       } else if h.Cluster.Collections.BlobSigning {
-               return errors.New("cannot enable Collections.BlobSigning with no Collections.BlobSigningKey")
-       } else {
-               h.Logger.Warn("Running without a blob signing key. Block locators returned by this server will not be signed, and will be rejected by a server that enforces permissions. To fix this, configure Collections.BlobSigning and Collections.BlobSigningKey.")
-       }
-
-       if len(h.Cluster.Volumes) == 0 {
-               return errors.New("no volumes configured")
-       }
-
-       h.Logger.Printf("keepstore %s starting, pid %d", cmd.Version.String(), os.Getpid())
-
-       // Start a round-robin VolumeManager with the configured volumes.
-       vm, err := makeRRVolumeManager(h.Logger, h.Cluster, serviceURL, newVolumeMetricsVecs(reg))
-       if err != nil {
-               return err
-       }
-       if len(vm.readables) == 0 {
-               return fmt.Errorf("no volumes configured for %s", serviceURL)
-       }
-       h.volmgr = vm
-
-       // Initialize the pullq and workers
-       h.pullq = NewWorkQueue()
-       for i := 0; i < 1 || i < h.Cluster.Collections.BlobReplicateConcurrency; i++ {
-               go h.runPullWorker(h.pullq)
-       }
-
-       // Initialize the trashq and workers
-       h.trashq = NewWorkQueue()
-       for i := 0; i < h.Cluster.Collections.BlobTrashConcurrency; i++ {
-               go RunTrashWorker(h.volmgr, h.Logger, h.Cluster, h.trashq)
-       }
-
-       // Set up routes and metrics
-       h.Handler = MakeRESTRouter(ctx, cluster, reg, vm, h.pullq, h.trashq)
-
-       // Initialize keepclient for pull workers
-       c, err := arvados.NewClientFromConfig(cluster)
-       if err != nil {
-               return err
-       }
-       ac, err := arvadosclient.New(c)
-       if err != nil {
-               return err
-       }
-       h.keepClient = &keepclient.KeepClient{
-               Arvados:       ac,
-               Want_replicas: 1,
-               DiskCacheSize: keepclient.DiskCacheDisabled,
-       }
-       h.keepClient.Arvados.ApiToken = fmt.Sprintf("%x", rand.Int63())
-
-       if d := h.Cluster.Collections.BlobTrashCheckInterval.Duration(); d > 0 &&
-               h.Cluster.Collections.BlobTrash &&
-               h.Cluster.Collections.BlobDeleteConcurrency > 0 {
-               go emptyTrash(h.volmgr.mounts, d)
-       }
-
-       return nil
+       puller := newPuller(ctx, ks, reg)
+       trasher := newTrasher(ctx, ks, reg)
+       _ = newTrashEmptier(ctx, ks, reg)
+       return newRouter(ks, puller, trasher)
 }
index 700ca19dec958cbf978bc875e2b1f8e71f24bf49..51434a803e681d1fc2b5d9276dcb9ddbcddd6cca 100644 (file)
@@ -8,21 +8,21 @@ import (
        "io"
 )
 
-func NewCountingWriter(w io.Writer, f func(uint64)) io.WriteCloser {
+func newCountingWriter(w io.Writer, f func(uint64)) io.WriteCloser {
        return &countingReadWriter{
                writer:  w,
                counter: f,
        }
 }
 
-func NewCountingReader(r io.Reader, f func(uint64)) io.ReadCloser {
+func newCountingReader(r io.Reader, f func(uint64)) io.ReadCloser {
        return &countingReadWriter{
                reader:  r,
                counter: f,
        }
 }
 
-func NewCountingReaderAtSeeker(r readerAtSeeker, f func(uint64)) *countingReaderAtSeeker {
+func newCountingReaderAtSeeker(r readerAtSeeker, f func(uint64)) *countingReaderAtSeeker {
        return &countingReaderAtSeeker{readerAtSeeker: r, counter: f}
 }
 
diff --git a/services/keepstore/gocheck_test.go b/services/keepstore/gocheck_test.go
deleted file mode 100644 (file)
index 90076db..0000000
+++ /dev/null
@@ -1,14 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "gopkg.in/check.v1"
-       "testing"
-)
-
-func TestGocheck(t *testing.T) {
-       check.TestingT(t)
-}
diff --git a/services/keepstore/handler_test.go b/services/keepstore/handler_test.go
deleted file mode 100644 (file)
index 5bdafb7..0000000
+++ /dev/null
@@ -1,1405 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-// Tests for Keep HTTP handlers:
-//
-// - GetBlockHandler
-// - PutBlockHandler
-// - IndexHandler
-//
-// The HTTP handlers are responsible for enforcing permission policy,
-// so these tests must exercise all possible permission permutations.
-
-package keepstore
-
-import (
-       "bytes"
-       "context"
-       "encoding/json"
-       "fmt"
-       "net/http"
-       "net/http/httptest"
-       "os"
-       "sort"
-       "strings"
-       "sync/atomic"
-       "time"
-
-       "git.arvados.org/arvados.git/lib/config"
-       "git.arvados.org/arvados.git/sdk/go/arvados"
-       "git.arvados.org/arvados.git/sdk/go/arvadostest"
-       "git.arvados.org/arvados.git/sdk/go/ctxlog"
-       "github.com/prometheus/client_golang/prometheus"
-       check "gopkg.in/check.v1"
-)
-
-var testServiceURL = func() arvados.URL {
-       return arvados.URL{Host: "localhost:12345", Scheme: "http"}
-}()
-
-func testCluster(t TB) *arvados.Cluster {
-       cfg, err := config.NewLoader(bytes.NewBufferString("Clusters: {zzzzz: {}}"), ctxlog.TestLogger(t)).Load()
-       if err != nil {
-               t.Fatal(err)
-       }
-       cluster, err := cfg.GetCluster("")
-       if err != nil {
-               t.Fatal(err)
-       }
-       cluster.SystemRootToken = arvadostest.SystemRootToken
-       cluster.ManagementToken = arvadostest.ManagementToken
-       cluster.Collections.BlobSigning = false
-       return cluster
-}
-
-var _ = check.Suite(&HandlerSuite{})
-
-type HandlerSuite struct {
-       cluster *arvados.Cluster
-       handler *handler
-}
-
-func (s *HandlerSuite) SetUpTest(c *check.C) {
-       s.cluster = testCluster(c)
-       s.cluster.Volumes = map[string]arvados.Volume{
-               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "mock"},
-               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "mock"},
-       }
-       s.handler = &handler{}
-}
-
-// A RequestTester represents the parameters for an HTTP request to
-// be issued on behalf of a unit test.
-type RequestTester struct {
-       uri            string
-       apiToken       string
-       method         string
-       requestBody    []byte
-       storageClasses string
-}
-
-// Test GetBlockHandler on the following situations:
-//   - permissions off, unauthenticated request, unsigned locator
-//   - permissions on, authenticated request, signed locator
-//   - permissions on, authenticated request, unsigned locator
-//   - permissions on, unauthenticated request, signed locator
-//   - permissions on, authenticated request, expired locator
-//   - permissions on, authenticated request, signed locator, transient error from backend
-func (s *HandlerSuite) TestGetHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       vols := s.handler.volmgr.AllWritable()
-       err := vols[0].Put(context.Background(), TestHash, TestBlock)
-       c.Check(err, check.IsNil)
-
-       // Create locators for testing.
-       // Turn on permission settings so we can generate signed locators.
-       s.cluster.Collections.BlobSigning = true
-       s.cluster.Collections.BlobSigningKey = knownKey
-       s.cluster.Collections.BlobSigningTTL.Set("5m")
-
-       var (
-               unsignedLocator  = "/" + TestHash
-               validTimestamp   = time.Now().Add(s.cluster.Collections.BlobSigningTTL.Duration())
-               expiredTimestamp = time.Now().Add(-time.Hour)
-               signedLocator    = "/" + SignLocator(s.cluster, TestHash, knownToken, validTimestamp)
-               expiredLocator   = "/" + SignLocator(s.cluster, TestHash, knownToken, expiredTimestamp)
-       )
-
-       // -----------------
-       // Test unauthenticated request with permissions off.
-       s.cluster.Collections.BlobSigning = false
-
-       // Unauthenticated request, unsigned locator
-       // => OK
-       response := IssueRequest(s.handler,
-               &RequestTester{
-                       method: "GET",
-                       uri:    unsignedLocator,
-               })
-       ExpectStatusCode(c,
-               "Unauthenticated request, unsigned locator", http.StatusOK, response)
-       ExpectBody(c,
-               "Unauthenticated request, unsigned locator",
-               string(TestBlock),
-               response)
-
-       receivedLen := response.Header().Get("Content-Length")
-       expectedLen := fmt.Sprintf("%d", len(TestBlock))
-       if receivedLen != expectedLen {
-               c.Errorf("expected Content-Length %s, got %s", expectedLen, receivedLen)
-       }
-
-       // ----------------
-       // Permissions: on.
-       s.cluster.Collections.BlobSigning = true
-
-       // Authenticated request, signed locator
-       // => OK
-       response = IssueRequest(s.handler, &RequestTester{
-               method:   "GET",
-               uri:      signedLocator,
-               apiToken: knownToken,
-       })
-       ExpectStatusCode(c,
-               "Authenticated request, signed locator", http.StatusOK, response)
-       ExpectBody(c,
-               "Authenticated request, signed locator", string(TestBlock), response)
-
-       receivedLen = response.Header().Get("Content-Length")
-       expectedLen = fmt.Sprintf("%d", len(TestBlock))
-       if receivedLen != expectedLen {
-               c.Errorf("expected Content-Length %s, got %s", expectedLen, receivedLen)
-       }
-
-       // Authenticated request, unsigned locator
-       // => PermissionError
-       response = IssueRequest(s.handler, &RequestTester{
-               method:   "GET",
-               uri:      unsignedLocator,
-               apiToken: knownToken,
-       })
-       ExpectStatusCode(c, "unsigned locator", PermissionError.HTTPCode, response)
-
-       // Unauthenticated request, signed locator
-       // => PermissionError
-       response = IssueRequest(s.handler, &RequestTester{
-               method: "GET",
-               uri:    signedLocator,
-       })
-       ExpectStatusCode(c,
-               "Unauthenticated request, signed locator",
-               PermissionError.HTTPCode, response)
-
-       // Authenticated request, expired locator
-       // => ExpiredError
-       response = IssueRequest(s.handler, &RequestTester{
-               method:   "GET",
-               uri:      expiredLocator,
-               apiToken: knownToken,
-       })
-       ExpectStatusCode(c,
-               "Authenticated request, expired locator",
-               ExpiredError.HTTPCode, response)
-
-       // Authenticated request, signed locator
-       // => 503 Server busy (transient error)
-
-       // Set up the block owning volume to respond with errors
-       vols[0].Volume.(*MockVolume).Bad = true
-       vols[0].Volume.(*MockVolume).BadVolumeError = VolumeBusyError
-       response = IssueRequest(s.handler, &RequestTester{
-               method:   "GET",
-               uri:      signedLocator,
-               apiToken: knownToken,
-       })
-       // A transient error from one volume while the other doesn't find the block
-       // should make the service return a 503 so that clients can retry.
-       ExpectStatusCode(c,
-               "Volume backend busy",
-               503, response)
-}
-
-// Test PutBlockHandler on the following situations:
-//   - no server key
-//   - with server key, authenticated request, unsigned locator
-//   - with server key, unauthenticated request, unsigned locator
-func (s *HandlerSuite) TestPutHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       // --------------
-       // No server key.
-
-       s.cluster.Collections.BlobSigningKey = ""
-
-       // Unauthenticated request, no server key
-       // => OK (unsigned response)
-       unsignedLocator := "/" + TestHash
-       response := IssueRequest(s.handler,
-               &RequestTester{
-                       method:      "PUT",
-                       uri:         unsignedLocator,
-                       requestBody: TestBlock,
-               })
-
-       ExpectStatusCode(c,
-               "Unauthenticated request, no server key", http.StatusOK, response)
-       ExpectBody(c,
-               "Unauthenticated request, no server key",
-               TestHashPutResp, response)
-
-       // ------------------
-       // With a server key.
-
-       s.cluster.Collections.BlobSigningKey = knownKey
-       s.cluster.Collections.BlobSigningTTL.Set("5m")
-
-       // When a permission key is available, the locator returned
-       // from an authenticated PUT request will be signed.
-
-       // Authenticated PUT, signed locator
-       // => OK (signed response)
-       response = IssueRequest(s.handler,
-               &RequestTester{
-                       method:      "PUT",
-                       uri:         unsignedLocator,
-                       requestBody: TestBlock,
-                       apiToken:    knownToken,
-               })
-
-       ExpectStatusCode(c,
-               "Authenticated PUT, signed locator, with server key",
-               http.StatusOK, response)
-       responseLocator := strings.TrimSpace(response.Body.String())
-       if VerifySignature(s.cluster, responseLocator, knownToken) != nil {
-               c.Errorf("Authenticated PUT, signed locator, with server key:\n"+
-                       "response '%s' does not contain a valid signature",
-                       responseLocator)
-       }
-
-       // Unauthenticated PUT, unsigned locator
-       // => OK
-       response = IssueRequest(s.handler,
-               &RequestTester{
-                       method:      "PUT",
-                       uri:         unsignedLocator,
-                       requestBody: TestBlock,
-               })
-
-       ExpectStatusCode(c,
-               "Unauthenticated PUT, unsigned locator, with server key",
-               http.StatusOK, response)
-       ExpectBody(c,
-               "Unauthenticated PUT, unsigned locator, with server key",
-               TestHashPutResp, response)
-}
-
-func (s *HandlerSuite) TestPutAndDeleteSkipReadonlyVolumes(c *check.C) {
-       s.cluster.Volumes["zzzzz-nyw5e-000000000000000"] = arvados.Volume{Driver: "mock", ReadOnly: true}
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       s.cluster.SystemRootToken = "fake-data-manager-token"
-       IssueRequest(s.handler,
-               &RequestTester{
-                       method:      "PUT",
-                       uri:         "/" + TestHash,
-                       requestBody: TestBlock,
-               })
-
-       s.cluster.Collections.BlobTrash = true
-       IssueRequest(s.handler,
-               &RequestTester{
-                       method:      "DELETE",
-                       uri:         "/" + TestHash,
-                       requestBody: TestBlock,
-                       apiToken:    s.cluster.SystemRootToken,
-               })
-       type expect struct {
-               volid     string
-               method    string
-               callcount int
-       }
-       for _, e := range []expect{
-               {"zzzzz-nyw5e-000000000000000", "Get", 0},
-               {"zzzzz-nyw5e-000000000000000", "Compare", 0},
-               {"zzzzz-nyw5e-000000000000000", "Touch", 0},
-               {"zzzzz-nyw5e-000000000000000", "Put", 0},
-               {"zzzzz-nyw5e-000000000000000", "Delete", 0},
-               {"zzzzz-nyw5e-111111111111111", "Get", 0},
-               {"zzzzz-nyw5e-111111111111111", "Compare", 1},
-               {"zzzzz-nyw5e-111111111111111", "Touch", 1},
-               {"zzzzz-nyw5e-111111111111111", "Put", 1},
-               {"zzzzz-nyw5e-111111111111111", "Delete", 1},
-       } {
-               if calls := s.handler.volmgr.mountMap[e.volid].Volume.(*MockVolume).CallCount(e.method); calls != e.callcount {
-                       c.Errorf("Got %d %s() on vol %s, expect %d", calls, e.method, e.volid, e.callcount)
-               }
-       }
-}
-
-func (s *HandlerSuite) TestReadsOrderedByStorageClassPriority(c *check.C) {
-       s.cluster.Volumes = map[string]arvados.Volume{
-               "zzzzz-nyw5e-111111111111111": {
-                       Driver:         "mock",
-                       Replication:    1,
-                       StorageClasses: map[string]bool{"class1": true}},
-               "zzzzz-nyw5e-222222222222222": {
-                       Driver:         "mock",
-                       Replication:    1,
-                       StorageClasses: map[string]bool{"class2": true, "class3": true}},
-       }
-
-       for _, trial := range []struct {
-               priority1 int // priority of class1, thus vol1
-               priority2 int // priority of class2
-               priority3 int // priority of class3 (vol2 priority will be max(priority2, priority3))
-               get1      int // expected number of "get" ops on vol1
-               get2      int // expected number of "get" ops on vol2
-       }{
-               {100, 50, 50, 1, 0},   // class1 has higher priority => try vol1 first, no need to try vol2
-               {100, 100, 100, 1, 0}, // same priority, vol1 is first lexicographically => try vol1 first and succeed
-               {66, 99, 33, 1, 1},    // class2 has higher priority => try vol2 first, then try vol1
-               {66, 33, 99, 1, 1},    // class3 has highest priority => vol2 has highest => try vol2 first, then try vol1
-       } {
-               c.Logf("%+v", trial)
-               s.cluster.StorageClasses = map[string]arvados.StorageClassConfig{
-                       "class1": {Priority: trial.priority1},
-                       "class2": {Priority: trial.priority2},
-                       "class3": {Priority: trial.priority3},
-               }
-               c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-               IssueRequest(s.handler,
-                       &RequestTester{
-                               method:         "PUT",
-                               uri:            "/" + TestHash,
-                               requestBody:    TestBlock,
-                               storageClasses: "class1",
-                       })
-               IssueRequest(s.handler,
-                       &RequestTester{
-                               method: "GET",
-                               uri:    "/" + TestHash,
-                       })
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-111111111111111"].Volume.(*MockVolume).CallCount("Get"), check.Equals, trial.get1)
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-222222222222222"].Volume.(*MockVolume).CallCount("Get"), check.Equals, trial.get2)
-       }
-}
-
-func (s *HandlerSuite) TestPutWithNoWritableVolumes(c *check.C) {
-       s.cluster.Volumes = map[string]arvados.Volume{
-               "zzzzz-nyw5e-111111111111111": {
-                       Driver:         "mock",
-                       Replication:    1,
-                       ReadOnly:       true,
-                       StorageClasses: map[string]bool{"class1": true}},
-       }
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       resp := IssueRequest(s.handler,
-               &RequestTester{
-                       method:         "PUT",
-                       uri:            "/" + TestHash,
-                       requestBody:    TestBlock,
-                       storageClasses: "class1",
-               })
-       c.Check(resp.Code, check.Equals, FullError.HTTPCode)
-       c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-111111111111111"].Volume.(*MockVolume).CallCount("Put"), check.Equals, 0)
-}
-
-func (s *HandlerSuite) TestConcurrentWritesToMultipleStorageClasses(c *check.C) {
-       s.cluster.Volumes = map[string]arvados.Volume{
-               "zzzzz-nyw5e-111111111111111": {
-                       Driver:         "mock",
-                       Replication:    1,
-                       StorageClasses: map[string]bool{"class1": true}},
-               "zzzzz-nyw5e-121212121212121": {
-                       Driver:         "mock",
-                       Replication:    1,
-                       StorageClasses: map[string]bool{"class1": true, "class2": true}},
-               "zzzzz-nyw5e-222222222222222": {
-                       Driver:         "mock",
-                       Replication:    1,
-                       StorageClasses: map[string]bool{"class2": true}},
-       }
-
-       for _, trial := range []struct {
-               setCounter uint32 // value to stuff vm.counter, to control offset
-               classes    string // desired classes
-               put111     int    // expected number of "put" ops on 11111... after 2x put reqs
-               put121     int    // expected number of "put" ops on 12121...
-               put222     int    // expected number of "put" ops on 22222...
-               cmp111     int    // expected number of "compare" ops on 11111... after 2x put reqs
-               cmp121     int    // expected number of "compare" ops on 12121...
-               cmp222     int    // expected number of "compare" ops on 22222...
-       }{
-               {0, "class1",
-                       1, 0, 0,
-                       2, 1, 0}, // first put compares on all vols with class2; second put succeeds after checking 121
-               {0, "class2",
-                       0, 1, 0,
-                       0, 2, 1}, // first put compares on all vols with class2; second put succeeds after checking 121
-               {0, "class1,class2",
-                       1, 1, 0,
-                       2, 2, 1}, // first put compares on all vols; second put succeeds after checking 111 and 121
-               {1, "class1,class2",
-                       0, 1, 0, // vm.counter offset is 1 so the first volume attempted is 121
-                       2, 2, 1}, // first put compares on all vols; second put succeeds after checking 111 and 121
-               {0, "class1,class2,class404",
-                       1, 1, 0,
-                       2, 2, 1}, // first put compares on all vols; second put doesn't compare on 222 because it already satisfied class2 on 121
-       } {
-               c.Logf("%+v", trial)
-               s.cluster.StorageClasses = map[string]arvados.StorageClassConfig{
-                       "class1": {},
-                       "class2": {},
-                       "class3": {},
-               }
-               c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-               atomic.StoreUint32(&s.handler.volmgr.counter, trial.setCounter)
-               for i := 0; i < 2; i++ {
-                       IssueRequest(s.handler,
-                               &RequestTester{
-                                       method:         "PUT",
-                                       uri:            "/" + TestHash,
-                                       requestBody:    TestBlock,
-                                       storageClasses: trial.classes,
-                               })
-               }
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-111111111111111"].Volume.(*MockVolume).CallCount("Put"), check.Equals, trial.put111)
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-121212121212121"].Volume.(*MockVolume).CallCount("Put"), check.Equals, trial.put121)
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-222222222222222"].Volume.(*MockVolume).CallCount("Put"), check.Equals, trial.put222)
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-111111111111111"].Volume.(*MockVolume).CallCount("Compare"), check.Equals, trial.cmp111)
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-121212121212121"].Volume.(*MockVolume).CallCount("Compare"), check.Equals, trial.cmp121)
-               c.Check(s.handler.volmgr.mountMap["zzzzz-nyw5e-222222222222222"].Volume.(*MockVolume).CallCount("Compare"), check.Equals, trial.cmp222)
-       }
-}
-
-// Test TOUCH requests.
-func (s *HandlerSuite) TestTouchHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       vols := s.handler.volmgr.AllWritable()
-       vols[0].Put(context.Background(), TestHash, TestBlock)
-       vols[0].Volume.(*MockVolume).TouchWithDate(TestHash, time.Now().Add(-time.Hour))
-       afterPut := time.Now()
-       t, err := vols[0].Mtime(TestHash)
-       c.Assert(err, check.IsNil)
-       c.Assert(t.Before(afterPut), check.Equals, true)
-
-       ExpectStatusCode(c,
-               "touch with no credentials",
-               http.StatusUnauthorized,
-               IssueRequest(s.handler, &RequestTester{
-                       method: "TOUCH",
-                       uri:    "/" + TestHash,
-               }))
-
-       ExpectStatusCode(c,
-               "touch with non-root credentials",
-               http.StatusUnauthorized,
-               IssueRequest(s.handler, &RequestTester{
-                       method:   "TOUCH",
-                       uri:      "/" + TestHash,
-                       apiToken: arvadostest.ActiveTokenV2,
-               }))
-
-       ExpectStatusCode(c,
-               "touch non-existent block",
-               http.StatusNotFound,
-               IssueRequest(s.handler, &RequestTester{
-                       method:   "TOUCH",
-                       uri:      "/aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
-                       apiToken: s.cluster.SystemRootToken,
-               }))
-
-       beforeTouch := time.Now()
-       ExpectStatusCode(c,
-               "touch block",
-               http.StatusOK,
-               IssueRequest(s.handler, &RequestTester{
-                       method:   "TOUCH",
-                       uri:      "/" + TestHash,
-                       apiToken: s.cluster.SystemRootToken,
-               }))
-       t, err = vols[0].Mtime(TestHash)
-       c.Assert(err, check.IsNil)
-       c.Assert(t.After(beforeTouch), check.Equals, true)
-}
-
-// Test /index requests:
-//   - unauthenticated /index request
-//   - unauthenticated /index/prefix request
-//   - authenticated   /index request        | non-superuser
-//   - authenticated   /index/prefix request | non-superuser
-//   - authenticated   /index request        | superuser
-//   - authenticated   /index/prefix request | superuser
-//
-// The only /index requests that should succeed are those issued by the
-// superuser. They should pass regardless of the value of BlobSigning.
-func (s *HandlerSuite) TestIndexHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       // Include multiple blocks on different volumes, and
-       // some metadata files (which should be omitted from index listings)
-       vols := s.handler.volmgr.AllWritable()
-       vols[0].Put(context.Background(), TestHash, TestBlock)
-       vols[1].Put(context.Background(), TestHash2, TestBlock2)
-       vols[0].Put(context.Background(), TestHash+".meta", []byte("metadata"))
-       vols[1].Put(context.Background(), TestHash2+".meta", []byte("metadata"))
-
-       s.cluster.SystemRootToken = "DATA MANAGER TOKEN"
-
-       unauthenticatedReq := &RequestTester{
-               method: "GET",
-               uri:    "/index",
-       }
-       authenticatedReq := &RequestTester{
-               method:   "GET",
-               uri:      "/index",
-               apiToken: knownToken,
-       }
-       superuserReq := &RequestTester{
-               method:   "GET",
-               uri:      "/index",
-               apiToken: s.cluster.SystemRootToken,
-       }
-       unauthPrefixReq := &RequestTester{
-               method: "GET",
-               uri:    "/index/" + TestHash[0:3],
-       }
-       authPrefixReq := &RequestTester{
-               method:   "GET",
-               uri:      "/index/" + TestHash[0:3],
-               apiToken: knownToken,
-       }
-       superuserPrefixReq := &RequestTester{
-               method:   "GET",
-               uri:      "/index/" + TestHash[0:3],
-               apiToken: s.cluster.SystemRootToken,
-       }
-       superuserNoSuchPrefixReq := &RequestTester{
-               method:   "GET",
-               uri:      "/index/abcd",
-               apiToken: s.cluster.SystemRootToken,
-       }
-       superuserInvalidPrefixReq := &RequestTester{
-               method:   "GET",
-               uri:      "/index/xyz",
-               apiToken: s.cluster.SystemRootToken,
-       }
-
-       // -------------------------------------------------------------
-       // Only the superuser should be allowed to issue /index requests.
-
-       // ---------------------------
-       // BlobSigning enabled
-       // This setting should not affect tests passing.
-       s.cluster.Collections.BlobSigning = true
-
-       // unauthenticated /index request
-       // => UnauthorizedError
-       response := IssueRequest(s.handler, unauthenticatedReq)
-       ExpectStatusCode(c,
-               "permissions on, unauthenticated request",
-               UnauthorizedError.HTTPCode,
-               response)
-
-       // unauthenticated /index/prefix request
-       // => UnauthorizedError
-       response = IssueRequest(s.handler, unauthPrefixReq)
-       ExpectStatusCode(c,
-               "permissions on, unauthenticated /index/prefix request",
-               UnauthorizedError.HTTPCode,
-               response)
-
-       // authenticated /index request, non-superuser
-       // => UnauthorizedError
-       response = IssueRequest(s.handler, authenticatedReq)
-       ExpectStatusCode(c,
-               "permissions on, authenticated request, non-superuser",
-               UnauthorizedError.HTTPCode,
-               response)
-
-       // authenticated /index/prefix request, non-superuser
-       // => UnauthorizedError
-       response = IssueRequest(s.handler, authPrefixReq)
-       ExpectStatusCode(c,
-               "permissions on, authenticated /index/prefix request, non-superuser",
-               UnauthorizedError.HTTPCode,
-               response)
-
-       // superuser /index request
-       // => OK
-       response = IssueRequest(s.handler, superuserReq)
-       ExpectStatusCode(c,
-               "permissions on, superuser request",
-               http.StatusOK,
-               response)
-
-       // ----------------------------
-       // BlobSigning disabled
-       // Valid Request should still pass.
-       s.cluster.Collections.BlobSigning = false
-
-       // superuser /index request
-       // => OK
-       response = IssueRequest(s.handler, superuserReq)
-       ExpectStatusCode(c,
-               "permissions on, superuser request",
-               http.StatusOK,
-               response)
-
-       expected := `^` + TestHash + `\+\d+ \d+\n` +
-               TestHash2 + `\+\d+ \d+\n\n$`
-       c.Check(response.Body.String(), check.Matches, expected, check.Commentf(
-               "permissions on, superuser request"))
-
-       // superuser /index/prefix request
-       // => OK
-       response = IssueRequest(s.handler, superuserPrefixReq)
-       ExpectStatusCode(c,
-               "permissions on, superuser request",
-               http.StatusOK,
-               response)
-
-       expected = `^` + TestHash + `\+\d+ \d+\n\n$`
-       c.Check(response.Body.String(), check.Matches, expected, check.Commentf(
-               "permissions on, superuser /index/prefix request"))
-
-       // superuser /index/{no-such-prefix} request
-       // => OK
-       response = IssueRequest(s.handler, superuserNoSuchPrefixReq)
-       ExpectStatusCode(c,
-               "permissions on, superuser request",
-               http.StatusOK,
-               response)
-
-       if "\n" != response.Body.String() {
-               c.Errorf("Expected empty response for %s. Found %s", superuserNoSuchPrefixReq.uri, response.Body.String())
-       }
-
-       // superuser /index/{invalid-prefix} request
-       // => StatusBadRequest
-       response = IssueRequest(s.handler, superuserInvalidPrefixReq)
-       ExpectStatusCode(c,
-               "permissions on, superuser request",
-               http.StatusBadRequest,
-               response)
-}
-
-// TestDeleteHandler
-//
-// Cases tested:
-//
-//     With no token and with a non-data-manager token:
-//     * Delete existing block
-//       (test for 403 Forbidden, confirm block not deleted)
-//
-//     With data manager token:
-//
-//     * Delete existing block
-//       (test for 200 OK, response counts, confirm block deleted)
-//
-//     * Delete nonexistent block
-//       (test for 200 OK, response counts)
-//
-//     TODO(twp):
-//
-//     * Delete block on read-only and read-write volume
-//       (test for 200 OK, response with copies_deleted=1,
-//       copies_failed=1, confirm block deleted only on r/w volume)
-//
-//     * Delete block on read-only volume only
-//       (test for 200 OK, response with copies_deleted=0, copies_failed=1,
-//       confirm block not deleted)
-func (s *HandlerSuite) TestDeleteHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       vols := s.handler.volmgr.AllWritable()
-       vols[0].Put(context.Background(), TestHash, TestBlock)
-
-       // Explicitly set the BlobSigningTTL to 0 for these
-       // tests, to ensure the MockVolume deletes the blocks
-       // even though they have just been created.
-       s.cluster.Collections.BlobSigningTTL = arvados.Duration(0)
-
-       var userToken = "NOT DATA MANAGER TOKEN"
-       s.cluster.SystemRootToken = "DATA MANAGER TOKEN"
-
-       s.cluster.Collections.BlobTrash = true
-
-       unauthReq := &RequestTester{
-               method: "DELETE",
-               uri:    "/" + TestHash,
-       }
-
-       userReq := &RequestTester{
-               method:   "DELETE",
-               uri:      "/" + TestHash,
-               apiToken: userToken,
-       }
-
-       superuserExistingBlockReq := &RequestTester{
-               method:   "DELETE",
-               uri:      "/" + TestHash,
-               apiToken: s.cluster.SystemRootToken,
-       }
-
-       superuserNonexistentBlockReq := &RequestTester{
-               method:   "DELETE",
-               uri:      "/" + TestHash2,
-               apiToken: s.cluster.SystemRootToken,
-       }
-
-       // Unauthenticated request returns PermissionError.
-       var response *httptest.ResponseRecorder
-       response = IssueRequest(s.handler, unauthReq)
-       ExpectStatusCode(c,
-               "unauthenticated request",
-               PermissionError.HTTPCode,
-               response)
-
-       // Authenticated non-admin request returns PermissionError.
-       response = IssueRequest(s.handler, userReq)
-       ExpectStatusCode(c,
-               "authenticated non-admin request",
-               PermissionError.HTTPCode,
-               response)
-
-       // Authenticated admin request for nonexistent block.
-       type deletecounter struct {
-               Deleted int `json:"copies_deleted"`
-               Failed  int `json:"copies_failed"`
-       }
-       var responseDc, expectedDc deletecounter
-
-       response = IssueRequest(s.handler, superuserNonexistentBlockReq)
-       ExpectStatusCode(c,
-               "data manager request, nonexistent block",
-               http.StatusNotFound,
-               response)
-
-       // Authenticated admin request for existing block while BlobTrash is false.
-       s.cluster.Collections.BlobTrash = false
-       response = IssueRequest(s.handler, superuserExistingBlockReq)
-       ExpectStatusCode(c,
-               "authenticated request, existing block, method disabled",
-               MethodDisabledError.HTTPCode,
-               response)
-       s.cluster.Collections.BlobTrash = true
-
-       // Authenticated admin request for existing block.
-       response = IssueRequest(s.handler, superuserExistingBlockReq)
-       ExpectStatusCode(c,
-               "data manager request, existing block",
-               http.StatusOK,
-               response)
-       // Expect response {"copies_deleted":1,"copies_failed":0}
-       expectedDc = deletecounter{1, 0}
-       json.NewDecoder(response.Body).Decode(&responseDc)
-       if responseDc != expectedDc {
-               c.Errorf("superuserExistingBlockReq\nexpected: %+v\nreceived: %+v",
-                       expectedDc, responseDc)
-       }
-       // Confirm the block has been deleted
-       buf := make([]byte, BlockSize)
-       _, err := vols[0].Get(context.Background(), TestHash, buf)
-       var blockDeleted = os.IsNotExist(err)
-       if !blockDeleted {
-               c.Error("superuserExistingBlockReq: block not deleted")
-       }
-
-       // A DELETE request on a block newer than BlobSigningTTL
-       // should return success but leave the block on the volume.
-       vols[0].Put(context.Background(), TestHash, TestBlock)
-       s.cluster.Collections.BlobSigningTTL = arvados.Duration(time.Hour)
-
-       response = IssueRequest(s.handler, superuserExistingBlockReq)
-       ExpectStatusCode(c,
-               "data manager request, existing block",
-               http.StatusOK,
-               response)
-       // Expect response {"copies_deleted":1,"copies_failed":0}
-       expectedDc = deletecounter{1, 0}
-       json.NewDecoder(response.Body).Decode(&responseDc)
-       if responseDc != expectedDc {
-               c.Errorf("superuserExistingBlockReq\nexpected: %+v\nreceived: %+v",
-                       expectedDc, responseDc)
-       }
-       // Confirm the block has NOT been deleted.
-       _, err = vols[0].Get(context.Background(), TestHash, buf)
-       if err != nil {
-               c.Errorf("testing delete on new block: %s\n", err)
-       }
-}
-
-// TestPullHandler
-//
-// Test handling of the PUT /pull statement.
-//
-// Cases tested: syntactically valid and invalid pull lists, from the
-// data manager and from unprivileged users:
-//
-//  1. Valid pull list from an ordinary user
-//     (expected result: 401 Unauthorized)
-//
-//  2. Invalid pull request from an ordinary user
-//     (expected result: 401 Unauthorized)
-//
-//  3. Valid pull request from the data manager
-//     (expected result: 200 OK with request body "Received 3 pull
-//     requests"
-//
-//  4. Invalid pull request from the data manager
-//     (expected result: 400 Bad Request)
-//
-// Test that in the end, the pull manager received a good pull list with
-// the expected number of requests.
-//
-// TODO(twp): test concurrency: launch 100 goroutines to update the
-// pull list simultaneously.  Make sure that none of them return 400
-// Bad Request and that pullq.GetList() returns a valid list.
-func (s *HandlerSuite) TestPullHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       // Replace the router's pullq -- which the worker goroutines
-       // started by setup() are now receiving from -- with a new
-       // one, so we can see what the handler sends to it.
-       pullq := NewWorkQueue()
-       s.handler.Handler.(*router).pullq = pullq
-
-       var userToken = "USER TOKEN"
-       s.cluster.SystemRootToken = "DATA MANAGER TOKEN"
-
-       goodJSON := []byte(`[
-               {
-                       "locator":"aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa+12345",
-                       "servers":[
-                               "http://server1",
-                               "http://server2"
-                       ]
-               },
-               {
-                       "locator":"bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb+12345",
-                       "servers":[]
-               },
-               {
-                       "locator":"cccccccccccccccccccccccccccccccc+12345",
-                       "servers":["http://server1"]
-               }
-       ]`)
-
-       badJSON := []byte(`{ "key":"I'm a little teapot" }`)
-
-       type pullTest struct {
-               name         string
-               req          RequestTester
-               responseCode int
-               responseBody string
-       }
-       var testcases = []pullTest{
-               {
-                       "Valid pull list from an ordinary user",
-                       RequestTester{"/pull", userToken, "PUT", goodJSON, ""},
-                       http.StatusUnauthorized,
-                       "Unauthorized\n",
-               },
-               {
-                       "Invalid pull request from an ordinary user",
-                       RequestTester{"/pull", userToken, "PUT", badJSON, ""},
-                       http.StatusUnauthorized,
-                       "Unauthorized\n",
-               },
-               {
-                       "Valid pull request from the data manager",
-                       RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", goodJSON, ""},
-                       http.StatusOK,
-                       "Received 3 pull requests\n",
-               },
-               {
-                       "Invalid pull request from the data manager",
-                       RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", badJSON, ""},
-                       http.StatusBadRequest,
-                       "",
-               },
-       }
-
-       for _, tst := range testcases {
-               response := IssueRequest(s.handler, &tst.req)
-               ExpectStatusCode(c, tst.name, tst.responseCode, response)
-               ExpectBody(c, tst.name, tst.responseBody, response)
-       }
-
-       // The Keep pull manager should have received one good list with 3
-       // requests on it.
-       for i := 0; i < 3; i++ {
-               var item interface{}
-               select {
-               case item = <-pullq.NextItem:
-               case <-time.After(time.Second):
-                       c.Error("timed out")
-               }
-               if _, ok := item.(PullRequest); !ok {
-                       c.Errorf("item %v could not be parsed as a PullRequest", item)
-               }
-       }
-
-       expectChannelEmpty(c, pullq.NextItem)
-}
-
-// TestTrashHandler
-//
-// Test cases:
-//
-// Cases tested: syntactically valid and invalid trash lists, from the
-// data manager and from unprivileged users:
-//
-//  1. Valid trash list from an ordinary user
-//     (expected result: 401 Unauthorized)
-//
-//  2. Invalid trash list from an ordinary user
-//     (expected result: 401 Unauthorized)
-//
-//  3. Valid trash list from the data manager
-//     (expected result: 200 OK with request body "Received 3 trash
-//     requests"
-//
-//  4. Invalid trash list from the data manager
-//     (expected result: 400 Bad Request)
-//
-// Test that in the end, the trash collector received a good list
-// trash list with the expected number of requests.
-//
-// TODO(twp): test concurrency: launch 100 goroutines to update the
-// pull list simultaneously.  Make sure that none of them return 400
-// Bad Request and that replica.Dump() returns a valid list.
-func (s *HandlerSuite) TestTrashHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       // Replace the router's trashq -- which the worker goroutines
-       // started by setup() are now receiving from -- with a new
-       // one, so we can see what the handler sends to it.
-       trashq := NewWorkQueue()
-       s.handler.Handler.(*router).trashq = trashq
-
-       var userToken = "USER TOKEN"
-       s.cluster.SystemRootToken = "DATA MANAGER TOKEN"
-
-       goodJSON := []byte(`[
-               {
-                       "locator":"block1",
-                       "block_mtime":1409082153
-               },
-               {
-                       "locator":"block2",
-                       "block_mtime":1409082153
-               },
-               {
-                       "locator":"block3",
-                       "block_mtime":1409082153
-               }
-       ]`)
-
-       badJSON := []byte(`I am not a valid JSON string`)
-
-       type trashTest struct {
-               name         string
-               req          RequestTester
-               responseCode int
-               responseBody string
-       }
-
-       var testcases = []trashTest{
-               {
-                       "Valid trash list from an ordinary user",
-                       RequestTester{"/trash", userToken, "PUT", goodJSON, ""},
-                       http.StatusUnauthorized,
-                       "Unauthorized\n",
-               },
-               {
-                       "Invalid trash list from an ordinary user",
-                       RequestTester{"/trash", userToken, "PUT", badJSON, ""},
-                       http.StatusUnauthorized,
-                       "Unauthorized\n",
-               },
-               {
-                       "Valid trash list from the data manager",
-                       RequestTester{"/trash", s.cluster.SystemRootToken, "PUT", goodJSON, ""},
-                       http.StatusOK,
-                       "Received 3 trash requests\n",
-               },
-               {
-                       "Invalid trash list from the data manager",
-                       RequestTester{"/trash", s.cluster.SystemRootToken, "PUT", badJSON, ""},
-                       http.StatusBadRequest,
-                       "",
-               },
-       }
-
-       for _, tst := range testcases {
-               response := IssueRequest(s.handler, &tst.req)
-               ExpectStatusCode(c, tst.name, tst.responseCode, response)
-               ExpectBody(c, tst.name, tst.responseBody, response)
-       }
-
-       // The trash collector should have received one good list with 3
-       // requests on it.
-       for i := 0; i < 3; i++ {
-               item := <-trashq.NextItem
-               if _, ok := item.(TrashRequest); !ok {
-                       c.Errorf("item %v could not be parsed as a TrashRequest", item)
-               }
-       }
-
-       expectChannelEmpty(c, trashq.NextItem)
-}
-
-// ====================
-// Helper functions
-// ====================
-
-// IssueTestRequest executes an HTTP request described by rt, to a
-// REST router.  It returns the HTTP response to the request.
-func IssueRequest(handler http.Handler, rt *RequestTester) *httptest.ResponseRecorder {
-       response := httptest.NewRecorder()
-       body := bytes.NewReader(rt.requestBody)
-       req, _ := http.NewRequest(rt.method, rt.uri, body)
-       if rt.apiToken != "" {
-               req.Header.Set("Authorization", "OAuth2 "+rt.apiToken)
-       }
-       if rt.storageClasses != "" {
-               req.Header.Set("X-Keep-Storage-Classes", rt.storageClasses)
-       }
-       handler.ServeHTTP(response, req)
-       return response
-}
-
-func IssueHealthCheckRequest(handler http.Handler, rt *RequestTester) *httptest.ResponseRecorder {
-       response := httptest.NewRecorder()
-       body := bytes.NewReader(rt.requestBody)
-       req, _ := http.NewRequest(rt.method, rt.uri, body)
-       if rt.apiToken != "" {
-               req.Header.Set("Authorization", "Bearer "+rt.apiToken)
-       }
-       handler.ServeHTTP(response, req)
-       return response
-}
-
-// ExpectStatusCode checks whether a response has the specified status code,
-// and reports a test failure if not.
-func ExpectStatusCode(
-       c *check.C,
-       testname string,
-       expectedStatus int,
-       response *httptest.ResponseRecorder) {
-       c.Check(response.Code, check.Equals, expectedStatus, check.Commentf("%s", testname))
-}
-
-func ExpectBody(
-       c *check.C,
-       testname string,
-       expectedBody string,
-       response *httptest.ResponseRecorder) {
-       if expectedBody != "" && response.Body.String() != expectedBody {
-               c.Errorf("%s: expected response body '%s', got %+v",
-                       testname, expectedBody, response)
-       }
-}
-
-// See #7121
-func (s *HandlerSuite) TestPutNeedsOnlyOneBuffer(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       defer func(orig *bufferPool) {
-               bufs = orig
-       }(bufs)
-       bufs = newBufferPool(ctxlog.TestLogger(c), 1, BlockSize)
-
-       ok := make(chan struct{})
-       go func() {
-               for i := 0; i < 2; i++ {
-                       response := IssueRequest(s.handler,
-                               &RequestTester{
-                                       method:      "PUT",
-                                       uri:         "/" + TestHash,
-                                       requestBody: TestBlock,
-                               })
-                       ExpectStatusCode(c,
-                               "TestPutNeedsOnlyOneBuffer", http.StatusOK, response)
-               }
-               ok <- struct{}{}
-       }()
-
-       select {
-       case <-ok:
-       case <-time.After(time.Second):
-               c.Fatal("PUT deadlocks with MaxKeepBlobBuffers==1")
-       }
-}
-
-// Invoke the PutBlockHandler a bunch of times to test for bufferpool resource
-// leak.
-func (s *HandlerSuite) TestPutHandlerNoBufferleak(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       ok := make(chan bool)
-       go func() {
-               for i := 0; i < s.cluster.API.MaxKeepBlobBuffers+1; i++ {
-                       // Unauthenticated request, no server key
-                       // => OK (unsigned response)
-                       unsignedLocator := "/" + TestHash
-                       response := IssueRequest(s.handler,
-                               &RequestTester{
-                                       method:      "PUT",
-                                       uri:         unsignedLocator,
-                                       requestBody: TestBlock,
-                               })
-                       ExpectStatusCode(c,
-                               "TestPutHandlerBufferleak", http.StatusOK, response)
-                       ExpectBody(c,
-                               "TestPutHandlerBufferleak",
-                               TestHashPutResp, response)
-               }
-               ok <- true
-       }()
-       select {
-       case <-time.After(20 * time.Second):
-               // If the buffer pool leaks, the test goroutine hangs.
-               c.Fatal("test did not finish, assuming pool leaked")
-       case <-ok:
-       }
-}
-
-func (s *HandlerSuite) TestGetHandlerClientDisconnect(c *check.C) {
-       s.cluster.Collections.BlobSigning = false
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       defer func(orig *bufferPool) {
-               bufs = orig
-       }(bufs)
-       bufs = newBufferPool(ctxlog.TestLogger(c), 1, BlockSize)
-       defer bufs.Put(bufs.Get(BlockSize))
-
-       err := s.handler.volmgr.AllWritable()[0].Put(context.Background(), TestHash, TestBlock)
-       c.Assert(err, check.IsNil)
-
-       resp := httptest.NewRecorder()
-       ok := make(chan struct{})
-       go func() {
-               ctx, cancel := context.WithCancel(context.Background())
-               req, _ := http.NewRequestWithContext(ctx, "GET", fmt.Sprintf("/%s+%d", TestHash, len(TestBlock)), nil)
-               cancel()
-               s.handler.ServeHTTP(resp, req)
-               ok <- struct{}{}
-       }()
-
-       select {
-       case <-time.After(20 * time.Second):
-               c.Fatal("request took >20s, close notifier must be broken")
-       case <-ok:
-       }
-
-       ExpectStatusCode(c, "client disconnect", http.StatusServiceUnavailable, resp)
-       for i, v := range s.handler.volmgr.AllWritable() {
-               if calls := v.Volume.(*MockVolume).called["GET"]; calls != 0 {
-                       c.Errorf("volume %d got %d calls, expected 0", i, calls)
-               }
-       }
-}
-
-// Invoke the GetBlockHandler a bunch of times to test for bufferpool resource
-// leak.
-func (s *HandlerSuite) TestGetHandlerNoBufferLeak(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       vols := s.handler.volmgr.AllWritable()
-       if err := vols[0].Put(context.Background(), TestHash, TestBlock); err != nil {
-               c.Error(err)
-       }
-
-       ok := make(chan bool)
-       go func() {
-               for i := 0; i < s.cluster.API.MaxKeepBlobBuffers+1; i++ {
-                       // Unauthenticated request, unsigned locator
-                       // => OK
-                       unsignedLocator := "/" + TestHash
-                       response := IssueRequest(s.handler,
-                               &RequestTester{
-                                       method: "GET",
-                                       uri:    unsignedLocator,
-                               })
-                       ExpectStatusCode(c,
-                               "Unauthenticated request, unsigned locator", http.StatusOK, response)
-                       ExpectBody(c,
-                               "Unauthenticated request, unsigned locator",
-                               string(TestBlock),
-                               response)
-               }
-               ok <- true
-       }()
-       select {
-       case <-time.After(20 * time.Second):
-               // If the buffer pool leaks, the test goroutine hangs.
-               c.Fatal("test did not finish, assuming pool leaked")
-       case <-ok:
-       }
-}
-
-func (s *HandlerSuite) TestPutStorageClasses(c *check.C) {
-       s.cluster.Volumes = map[string]arvados.Volume{
-               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "mock"}, // "default" is implicit
-               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "mock", StorageClasses: map[string]bool{"special": true, "extra": true}},
-               "zzzzz-nyw5e-222222222222222": {Replication: 1, Driver: "mock", StorageClasses: map[string]bool{"readonly": true}, ReadOnly: true},
-       }
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       rt := RequestTester{
-               method:      "PUT",
-               uri:         "/" + TestHash,
-               requestBody: TestBlock,
-       }
-
-       for _, trial := range []struct {
-               ask    string
-               expect string
-       }{
-               {"", ""},
-               {"default", "default=1"},
-               {" , default , default , ", "default=1"},
-               {"special", "extra=1, special=1"},
-               {"special, readonly", "extra=1, special=1"},
-               {"special, nonexistent", "extra=1, special=1"},
-               {"extra, special", "extra=1, special=1"},
-               {"default, special", "default=1, extra=1, special=1"},
-       } {
-               c.Logf("success case %#v", trial)
-               rt.storageClasses = trial.ask
-               resp := IssueRequest(s.handler, &rt)
-               if trial.expect == "" {
-                       // any non-empty value is correct
-                       c.Check(resp.Header().Get("X-Keep-Storage-Classes-Confirmed"), check.Not(check.Equals), "")
-               } else {
-                       c.Check(sortCommaSeparated(resp.Header().Get("X-Keep-Storage-Classes-Confirmed")), check.Equals, trial.expect)
-               }
-       }
-
-       for _, trial := range []struct {
-               ask string
-       }{
-               {"doesnotexist"},
-               {"doesnotexist, readonly"},
-               {"readonly"},
-       } {
-               c.Logf("failure case %#v", trial)
-               rt.storageClasses = trial.ask
-               resp := IssueRequest(s.handler, &rt)
-               c.Check(resp.Code, check.Equals, http.StatusServiceUnavailable)
-       }
-}
-
-func sortCommaSeparated(s string) string {
-       slice := strings.Split(s, ", ")
-       sort.Strings(slice)
-       return strings.Join(slice, ", ")
-}
-
-func (s *HandlerSuite) TestPutResponseHeader(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       resp := IssueRequest(s.handler, &RequestTester{
-               method:      "PUT",
-               uri:         "/" + TestHash,
-               requestBody: TestBlock,
-       })
-       c.Logf("%#v", resp)
-       c.Check(resp.Header().Get("X-Keep-Replicas-Stored"), check.Equals, "1")
-       c.Check(resp.Header().Get("X-Keep-Storage-Classes-Confirmed"), check.Equals, "default=1")
-}
-
-func (s *HandlerSuite) TestUntrashHandler(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       // Set up Keep volumes
-       vols := s.handler.volmgr.AllWritable()
-       vols[0].Put(context.Background(), TestHash, TestBlock)
-
-       s.cluster.SystemRootToken = "DATA MANAGER TOKEN"
-
-       // unauthenticatedReq => UnauthorizedError
-       unauthenticatedReq := &RequestTester{
-               method: "PUT",
-               uri:    "/untrash/" + TestHash,
-       }
-       response := IssueRequest(s.handler, unauthenticatedReq)
-       ExpectStatusCode(c,
-               "Unauthenticated request",
-               UnauthorizedError.HTTPCode,
-               response)
-
-       // notDataManagerReq => UnauthorizedError
-       notDataManagerReq := &RequestTester{
-               method:   "PUT",
-               uri:      "/untrash/" + TestHash,
-               apiToken: knownToken,
-       }
-
-       response = IssueRequest(s.handler, notDataManagerReq)
-       ExpectStatusCode(c,
-               "Non-datamanager token",
-               UnauthorizedError.HTTPCode,
-               response)
-
-       // datamanagerWithBadHashReq => StatusBadRequest
-       datamanagerWithBadHashReq := &RequestTester{
-               method:   "PUT",
-               uri:      "/untrash/thisisnotalocator",
-               apiToken: s.cluster.SystemRootToken,
-       }
-       response = IssueRequest(s.handler, datamanagerWithBadHashReq)
-       ExpectStatusCode(c,
-               "Bad locator in untrash request",
-               http.StatusBadRequest,
-               response)
-
-       // datamanagerWrongMethodReq => StatusBadRequest
-       datamanagerWrongMethodReq := &RequestTester{
-               method:   "GET",
-               uri:      "/untrash/" + TestHash,
-               apiToken: s.cluster.SystemRootToken,
-       }
-       response = IssueRequest(s.handler, datamanagerWrongMethodReq)
-       ExpectStatusCode(c,
-               "Only PUT method is supported for untrash",
-               http.StatusMethodNotAllowed,
-               response)
-
-       // datamanagerReq => StatusOK
-       datamanagerReq := &RequestTester{
-               method:   "PUT",
-               uri:      "/untrash/" + TestHash,
-               apiToken: s.cluster.SystemRootToken,
-       }
-       response = IssueRequest(s.handler, datamanagerReq)
-       ExpectStatusCode(c,
-               "",
-               http.StatusOK,
-               response)
-       c.Check(response.Body.String(), check.Equals, "Successfully untrashed on: [MockVolume], [MockVolume]\n")
-}
-
-func (s *HandlerSuite) TestUntrashHandlerWithNoWritableVolumes(c *check.C) {
-       // Change all volumes to read-only
-       for uuid, v := range s.cluster.Volumes {
-               v.ReadOnly = true
-               s.cluster.Volumes[uuid] = v
-       }
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       // datamanagerReq => StatusOK
-       datamanagerReq := &RequestTester{
-               method:   "PUT",
-               uri:      "/untrash/" + TestHash,
-               apiToken: s.cluster.SystemRootToken,
-       }
-       response := IssueRequest(s.handler, datamanagerReq)
-       ExpectStatusCode(c,
-               "No writable volumes",
-               http.StatusNotFound,
-               response)
-}
-
-func (s *HandlerSuite) TestHealthCheckPing(c *check.C) {
-       s.cluster.ManagementToken = arvadostest.ManagementToken
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       pingReq := &RequestTester{
-               method:   "GET",
-               uri:      "/_health/ping",
-               apiToken: arvadostest.ManagementToken,
-       }
-       response := IssueHealthCheckRequest(s.handler, pingReq)
-       ExpectStatusCode(c,
-               "",
-               http.StatusOK,
-               response)
-       want := `{"health":"OK"}`
-       if !strings.Contains(response.Body.String(), want) {
-               c.Errorf("expected response to include %s: got %s", want, response.Body.String())
-       }
-}
diff --git a/services/keepstore/handlers.go b/services/keepstore/handlers.go
deleted file mode 100644 (file)
index abeb20f..0000000
+++ /dev/null
@@ -1,1056 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "container/list"
-       "context"
-       "crypto/md5"
-       "encoding/json"
-       "fmt"
-       "io"
-       "net/http"
-       "os"
-       "regexp"
-       "runtime"
-       "strconv"
-       "strings"
-       "sync"
-       "sync/atomic"
-       "time"
-
-       "git.arvados.org/arvados.git/lib/cmd"
-       "git.arvados.org/arvados.git/sdk/go/arvados"
-       "git.arvados.org/arvados.git/sdk/go/ctxlog"
-       "git.arvados.org/arvados.git/sdk/go/health"
-       "git.arvados.org/arvados.git/sdk/go/httpserver"
-       "github.com/gorilla/mux"
-       "github.com/prometheus/client_golang/prometheus"
-       "github.com/sirupsen/logrus"
-)
-
-type router struct {
-       *mux.Router
-       cluster     *arvados.Cluster
-       logger      logrus.FieldLogger
-       remoteProxy remoteProxy
-       metrics     *nodeMetrics
-       volmgr      *RRVolumeManager
-       pullq       *WorkQueue
-       trashq      *WorkQueue
-}
-
-// MakeRESTRouter returns a new router that forwards all Keep requests
-// to the appropriate handlers.
-func MakeRESTRouter(ctx context.Context, cluster *arvados.Cluster, reg *prometheus.Registry, volmgr *RRVolumeManager, pullq, trashq *WorkQueue) http.Handler {
-       rtr := &router{
-               Router:  mux.NewRouter(),
-               cluster: cluster,
-               logger:  ctxlog.FromContext(ctx),
-               metrics: &nodeMetrics{reg: reg},
-               volmgr:  volmgr,
-               pullq:   pullq,
-               trashq:  trashq,
-       }
-
-       rtr.HandleFunc(
-               `/{hash:[0-9a-f]{32}}`, rtr.handleGET).Methods("GET", "HEAD")
-       rtr.HandleFunc(
-               `/{hash:[0-9a-f]{32}}+{hints}`,
-               rtr.handleGET).Methods("GET", "HEAD")
-
-       rtr.HandleFunc(`/{hash:[0-9a-f]{32}}`, rtr.handlePUT).Methods("PUT")
-       rtr.HandleFunc(`/{hash:[0-9a-f]{32}}`, rtr.handleDELETE).Methods("DELETE")
-       // List all blocks stored here. Privileged client only.
-       rtr.HandleFunc(`/index`, rtr.handleIndex).Methods("GET", "HEAD")
-       // List blocks stored here whose hash has the given prefix.
-       // Privileged client only.
-       rtr.HandleFunc(`/index/{prefix:[0-9a-f]{0,32}}`, rtr.handleIndex).Methods("GET", "HEAD")
-       // Update timestamp on existing block. Privileged client only.
-       rtr.HandleFunc(`/{hash:[0-9a-f]{32}}`, rtr.handleTOUCH).Methods("TOUCH")
-
-       // Internals/debugging info (runtime.MemStats)
-       rtr.HandleFunc(`/debug.json`, rtr.DebugHandler).Methods("GET", "HEAD")
-
-       // List volumes: path, device number, bytes used/avail.
-       rtr.HandleFunc(`/status.json`, rtr.StatusHandler).Methods("GET", "HEAD")
-
-       // List mounts: UUID, readonly, tier, device ID, ...
-       rtr.HandleFunc(`/mounts`, rtr.MountsHandler).Methods("GET")
-       rtr.HandleFunc(`/mounts/{uuid}/blocks`, rtr.handleIndex).Methods("GET")
-       rtr.HandleFunc(`/mounts/{uuid}/blocks/`, rtr.handleIndex).Methods("GET")
-
-       // Replace the current pull queue.
-       rtr.HandleFunc(`/pull`, rtr.handlePull).Methods("PUT")
-
-       // Replace the current trash queue.
-       rtr.HandleFunc(`/trash`, rtr.handleTrash).Methods("PUT")
-
-       // Untrash moves blocks from trash back into store
-       rtr.HandleFunc(`/untrash/{hash:[0-9a-f]{32}}`, rtr.handleUntrash).Methods("PUT")
-
-       rtr.Handle("/_health/{check}", &health.Handler{
-               Token:  cluster.ManagementToken,
-               Prefix: "/_health/",
-       }).Methods("GET")
-
-       // Any request which does not match any of these routes gets
-       // 400 Bad Request.
-       rtr.NotFoundHandler = http.HandlerFunc(BadRequestHandler)
-
-       rtr.metrics.setupBufferPoolMetrics(bufs)
-       rtr.metrics.setupWorkQueueMetrics(rtr.pullq, "pull")
-       rtr.metrics.setupWorkQueueMetrics(rtr.trashq, "trash")
-
-       return rtr
-}
-
-// BadRequestHandler is a HandleFunc to address bad requests.
-func BadRequestHandler(w http.ResponseWriter, r *http.Request) {
-       http.Error(w, BadRequestError.Error(), BadRequestError.HTTPCode)
-}
-
-func (rtr *router) handleGET(resp http.ResponseWriter, req *http.Request) {
-       locator := req.URL.Path[1:]
-       if strings.Contains(locator, "+R") && !strings.Contains(locator, "+A") {
-               rtr.remoteProxy.Get(req.Context(), resp, req, rtr.cluster, rtr.volmgr)
-               return
-       }
-
-       if rtr.cluster.Collections.BlobSigning {
-               locator := req.URL.Path[1:] // strip leading slash
-               if err := VerifySignature(rtr.cluster, locator, GetAPIToken(req)); err != nil {
-                       http.Error(resp, err.Error(), err.(*KeepError).HTTPCode)
-                       return
-               }
-       }
-
-       // TODO: Probe volumes to check whether the block _might_
-       // exist. Some volumes/types could support a quick existence
-       // check without causing other operations to suffer. If all
-       // volumes support that, and assure us the block definitely
-       // isn't here, we can return 404 now instead of waiting for a
-       // buffer.
-
-       buf, err := getBufferWithContext(req.Context(), bufs, BlockSize)
-       if err != nil {
-               http.Error(resp, err.Error(), http.StatusServiceUnavailable)
-               return
-       }
-       defer bufs.Put(buf)
-
-       size, err := GetBlock(req.Context(), rtr.volmgr, mux.Vars(req)["hash"], buf, resp)
-       if err != nil {
-               code := http.StatusInternalServerError
-               if err, ok := err.(*KeepError); ok {
-                       code = err.HTTPCode
-               }
-               http.Error(resp, err.Error(), code)
-               return
-       }
-
-       resp.Header().Set("Content-Length", strconv.Itoa(size))
-       resp.Header().Set("Content-Type", "application/octet-stream")
-       resp.Write(buf[:size])
-}
-
-// Get a buffer from the pool -- but give up and return a non-nil
-// error if ctx ends before we get a buffer.
-func getBufferWithContext(ctx context.Context, bufs *bufferPool, bufSize int) ([]byte, error) {
-       bufReady := make(chan []byte)
-       go func() {
-               bufReady <- bufs.Get(bufSize)
-       }()
-       select {
-       case buf := <-bufReady:
-               return buf, nil
-       case <-ctx.Done():
-               go func() {
-                       // Even if closeNotifier happened first, we
-                       // need to keep waiting for our buf so we can
-                       // return it to the pool.
-                       bufs.Put(<-bufReady)
-               }()
-               return nil, ErrClientDisconnect
-       }
-}
-
-func (rtr *router) handleTOUCH(resp http.ResponseWriter, req *http.Request) {
-       if !rtr.isSystemAuth(GetAPIToken(req)) {
-               http.Error(resp, UnauthorizedError.Error(), UnauthorizedError.HTTPCode)
-               return
-       }
-       hash := mux.Vars(req)["hash"]
-       vols := rtr.volmgr.AllWritable()
-       if len(vols) == 0 {
-               http.Error(resp, "no volumes", http.StatusNotFound)
-               return
-       }
-       var err error
-       for _, mnt := range vols {
-               err = mnt.Touch(hash)
-               if err == nil {
-                       break
-               }
-       }
-       switch {
-       case err == nil:
-               return
-       case os.IsNotExist(err):
-               http.Error(resp, err.Error(), http.StatusNotFound)
-       default:
-               http.Error(resp, err.Error(), http.StatusInternalServerError)
-       }
-}
-
-func (rtr *router) handlePUT(resp http.ResponseWriter, req *http.Request) {
-       hash := mux.Vars(req)["hash"]
-
-       // Detect as many error conditions as possible before reading
-       // the body: avoid transmitting data that will not end up
-       // being written anyway.
-
-       if req.ContentLength == -1 {
-               http.Error(resp, SizeRequiredError.Error(), SizeRequiredError.HTTPCode)
-               return
-       }
-
-       if req.ContentLength > BlockSize {
-               http.Error(resp, TooLongError.Error(), TooLongError.HTTPCode)
-               return
-       }
-
-       if len(rtr.volmgr.AllWritable()) == 0 {
-               http.Error(resp, FullError.Error(), FullError.HTTPCode)
-               return
-       }
-
-       var wantStorageClasses []string
-       if hdr := req.Header.Get("X-Keep-Storage-Classes"); hdr != "" {
-               wantStorageClasses = strings.Split(hdr, ",")
-               for i, sc := range wantStorageClasses {
-                       wantStorageClasses[i] = strings.TrimSpace(sc)
-               }
-       } else {
-               // none specified -- use configured default
-               for class, cfg := range rtr.cluster.StorageClasses {
-                       if cfg.Default {
-                               wantStorageClasses = append(wantStorageClasses, class)
-                       }
-               }
-       }
-
-       buf, err := getBufferWithContext(req.Context(), bufs, int(req.ContentLength))
-       if err != nil {
-               http.Error(resp, err.Error(), http.StatusServiceUnavailable)
-               return
-       }
-
-       _, err = io.ReadFull(req.Body, buf)
-       if err != nil {
-               http.Error(resp, err.Error(), 500)
-               bufs.Put(buf)
-               return
-       }
-
-       result, err := PutBlock(req.Context(), rtr.volmgr, buf, hash, wantStorageClasses)
-       bufs.Put(buf)
-
-       if err != nil {
-               code := http.StatusInternalServerError
-               if err, ok := err.(*KeepError); ok {
-                       code = err.HTTPCode
-               }
-               http.Error(resp, err.Error(), code)
-               return
-       }
-
-       // Success; add a size hint, sign the locator if possible, and
-       // return it to the client.
-       returnHash := fmt.Sprintf("%s+%d", hash, req.ContentLength)
-       apiToken := GetAPIToken(req)
-       if rtr.cluster.Collections.BlobSigningKey != "" && apiToken != "" {
-               expiry := time.Now().Add(rtr.cluster.Collections.BlobSigningTTL.Duration())
-               returnHash = SignLocator(rtr.cluster, returnHash, apiToken, expiry)
-       }
-       resp.Header().Set("X-Keep-Replicas-Stored", result.TotalReplication())
-       resp.Header().Set("X-Keep-Storage-Classes-Confirmed", result.ClassReplication())
-       resp.Write([]byte(returnHash + "\n"))
-}
-
-// IndexHandler responds to "/index", "/index/{prefix}", and
-// "/mounts/{uuid}/blocks" requests.
-func (rtr *router) handleIndex(resp http.ResponseWriter, req *http.Request) {
-       if !rtr.isSystemAuth(GetAPIToken(req)) {
-               http.Error(resp, UnauthorizedError.Error(), UnauthorizedError.HTTPCode)
-               return
-       }
-
-       prefix := mux.Vars(req)["prefix"]
-       if prefix == "" {
-               req.ParseForm()
-               prefix = req.Form.Get("prefix")
-       }
-
-       uuid := mux.Vars(req)["uuid"]
-
-       var vols []*VolumeMount
-       if uuid == "" {
-               vols = rtr.volmgr.AllReadable()
-       } else if mnt := rtr.volmgr.Lookup(uuid, false); mnt == nil {
-               http.Error(resp, "mount not found", http.StatusNotFound)
-               return
-       } else {
-               vols = []*VolumeMount{mnt}
-       }
-
-       for _, v := range vols {
-               if err := v.IndexTo(prefix, resp); err != nil {
-                       // We can't send an error status/message to
-                       // the client because IndexTo() might have
-                       // already written body content. All we can do
-                       // is log the error in our own logs.
-                       //
-                       // The client must notice the lack of trailing
-                       // newline as an indication that the response
-                       // is incomplete.
-                       ctxlog.FromContext(req.Context()).WithError(err).Errorf("truncating index response after error from volume %s", v)
-                       return
-               }
-       }
-       // An empty line at EOF is the only way the client can be
-       // assured the entire index was received.
-       resp.Write([]byte{'\n'})
-}
-
-// MountsHandler responds to "GET /mounts" requests.
-func (rtr *router) MountsHandler(resp http.ResponseWriter, req *http.Request) {
-       err := json.NewEncoder(resp).Encode(rtr.volmgr.Mounts())
-       if err != nil {
-               httpserver.Error(resp, err.Error(), http.StatusInternalServerError)
-       }
-}
-
-// PoolStatus struct
-type PoolStatus struct {
-       Alloc uint64 `json:"BytesAllocatedCumulative"`
-       Cap   int    `json:"BuffersMax"`
-       Len   int    `json:"BuffersInUse"`
-}
-
-type volumeStatusEnt struct {
-       Label         string
-       Status        *VolumeStatus `json:",omitempty"`
-       VolumeStats   *ioStats      `json:",omitempty"`
-       InternalStats interface{}   `json:",omitempty"`
-}
-
-// NodeStatus struct
-type NodeStatus struct {
-       Volumes         []*volumeStatusEnt
-       BufferPool      PoolStatus
-       PullQueue       WorkQueueStatus
-       TrashQueue      WorkQueueStatus
-       RequestsCurrent int
-       RequestsMax     int
-       Version         string
-}
-
-var st NodeStatus
-var stLock sync.Mutex
-
-// DebugHandler addresses /debug.json requests.
-func (rtr *router) DebugHandler(resp http.ResponseWriter, req *http.Request) {
-       type debugStats struct {
-               MemStats runtime.MemStats
-       }
-       var ds debugStats
-       runtime.ReadMemStats(&ds.MemStats)
-       data, err := json.Marshal(&ds)
-       if err != nil {
-               http.Error(resp, err.Error(), http.StatusInternalServerError)
-               return
-       }
-       resp.Write(data)
-}
-
-// StatusHandler addresses /status.json requests.
-func (rtr *router) StatusHandler(resp http.ResponseWriter, req *http.Request) {
-       stLock.Lock()
-       rtr.readNodeStatus(&st)
-       data, err := json.Marshal(&st)
-       stLock.Unlock()
-       if err != nil {
-               http.Error(resp, err.Error(), http.StatusInternalServerError)
-               return
-       }
-       resp.Write(data)
-}
-
-// populate the given NodeStatus struct with current values.
-func (rtr *router) readNodeStatus(st *NodeStatus) {
-       st.Version = strings.SplitN(cmd.Version.String(), " ", 2)[0]
-       vols := rtr.volmgr.AllReadable()
-       if cap(st.Volumes) < len(vols) {
-               st.Volumes = make([]*volumeStatusEnt, len(vols))
-       }
-       st.Volumes = st.Volumes[:0]
-       for _, vol := range vols {
-               var internalStats interface{}
-               if vol, ok := vol.Volume.(InternalStatser); ok {
-                       internalStats = vol.InternalStats()
-               }
-               st.Volumes = append(st.Volumes, &volumeStatusEnt{
-                       Label:         vol.String(),
-                       Status:        vol.Status(),
-                       InternalStats: internalStats,
-                       //VolumeStats: rtr.volmgr.VolumeStats(vol),
-               })
-       }
-       st.BufferPool.Alloc = bufs.Alloc()
-       st.BufferPool.Cap = bufs.Cap()
-       st.BufferPool.Len = bufs.Len()
-       st.PullQueue = getWorkQueueStatus(rtr.pullq)
-       st.TrashQueue = getWorkQueueStatus(rtr.trashq)
-}
-
-// return a WorkQueueStatus for the given queue. If q is nil (which
-// should never happen except in test suites), return a zero status
-// value instead of crashing.
-func getWorkQueueStatus(q *WorkQueue) WorkQueueStatus {
-       if q == nil {
-               // This should only happen during tests.
-               return WorkQueueStatus{}
-       }
-       return q.Status()
-}
-
-// handleDELETE processes DELETE requests.
-//
-// DELETE /{hash:[0-9a-f]{32} will delete the block with the specified hash
-// from all connected volumes.
-//
-// Only the Data Manager, or an Arvados admin with scope "all", are
-// allowed to issue DELETE requests.  If a DELETE request is not
-// authenticated or is issued by a non-admin user, the server returns
-// a PermissionError.
-//
-// Upon receiving a valid request from an authorized user,
-// handleDELETE deletes all copies of the specified block on local
-// writable volumes.
-//
-// Response format:
-//
-// If the requested blocks was not found on any volume, the response
-// code is HTTP 404 Not Found.
-//
-// Otherwise, the response code is 200 OK, with a response body
-// consisting of the JSON message
-//
-//     {"copies_deleted":d,"copies_failed":f}
-//
-// where d and f are integers representing the number of blocks that
-// were successfully and unsuccessfully deleted.
-func (rtr *router) handleDELETE(resp http.ResponseWriter, req *http.Request) {
-       hash := mux.Vars(req)["hash"]
-
-       // Confirm that this user is an admin and has a token with unlimited scope.
-       var tok = GetAPIToken(req)
-       if tok == "" || !rtr.canDelete(tok) {
-               http.Error(resp, PermissionError.Error(), PermissionError.HTTPCode)
-               return
-       }
-
-       if !rtr.cluster.Collections.BlobTrash {
-               http.Error(resp, MethodDisabledError.Error(), MethodDisabledError.HTTPCode)
-               return
-       }
-
-       // Delete copies of this block from all available volumes.
-       // Report how many blocks were successfully deleted, and how
-       // many were found on writable volumes but not deleted.
-       var result struct {
-               Deleted int `json:"copies_deleted"`
-               Failed  int `json:"copies_failed"`
-       }
-       for _, vol := range rtr.volmgr.Mounts() {
-               if !vol.KeepMount.AllowTrash {
-                       continue
-               } else if err := vol.Trash(hash); err == nil {
-                       result.Deleted++
-               } else if os.IsNotExist(err) {
-                       continue
-               } else {
-                       result.Failed++
-                       ctxlog.FromContext(req.Context()).WithError(err).Errorf("Trash(%s) failed on volume %s", hash, vol)
-               }
-       }
-       if result.Deleted == 0 && result.Failed == 0 {
-               resp.WriteHeader(http.StatusNotFound)
-               return
-       }
-       body, err := json.Marshal(result)
-       if err != nil {
-               http.Error(resp, err.Error(), http.StatusInternalServerError)
-               return
-       }
-       resp.Write(body)
-}
-
-/* PullHandler processes "PUT /pull" requests for the data manager.
-   The request body is a JSON message containing a list of pull
-   requests in the following format:
-
-   [
-      {
-         "locator":"e4d909c290d0fb1ca068ffaddf22cbd0+4985",
-         "servers":[
-                       "keep0.qr1hi.arvadosapi.com:25107",
-                       "keep1.qr1hi.arvadosapi.com:25108"
-                ]
-         },
-         {
-                "locator":"55ae4d45d2db0793d53f03e805f656e5+658395",
-                "servers":[
-                       "10.0.1.5:25107",
-                       "10.0.1.6:25107",
-                       "10.0.1.7:25108"
-                ]
-         },
-         ...
-   ]
-
-   Each pull request in the list consists of a block locator string
-   and an ordered list of servers.  Keepstore should try to fetch the
-   block from each server in turn.
-
-   If the request has not been sent by the Data Manager, return 401
-   Unauthorized.
-
-   If the JSON unmarshalling fails, return 400 Bad Request.
-*/
-
-// PullRequest consists of a block locator and an ordered list of servers
-type PullRequest struct {
-       Locator string   `json:"locator"`
-       Servers []string `json:"servers"`
-
-       // Destination mount, or "" for "anywhere"
-       MountUUID string `json:"mount_uuid"`
-}
-
-// PullHandler processes "PUT /pull" requests for the data manager.
-func (rtr *router) handlePull(resp http.ResponseWriter, req *http.Request) {
-       // Reject unauthorized requests.
-       if !rtr.isSystemAuth(GetAPIToken(req)) {
-               http.Error(resp, UnauthorizedError.Error(), UnauthorizedError.HTTPCode)
-               return
-       }
-
-       // Parse the request body.
-       var pr []PullRequest
-       r := json.NewDecoder(req.Body)
-       if err := r.Decode(&pr); err != nil {
-               http.Error(resp, err.Error(), BadRequestError.HTTPCode)
-               return
-       }
-
-       // We have a properly formatted pull list sent from the data
-       // manager.  Report success and send the list to the pull list
-       // manager for further handling.
-       resp.WriteHeader(http.StatusOK)
-       resp.Write([]byte(
-               fmt.Sprintf("Received %d pull requests\n", len(pr))))
-
-       plist := list.New()
-       for _, p := range pr {
-               plist.PushBack(p)
-       }
-       rtr.pullq.ReplaceQueue(plist)
-}
-
-// TrashRequest consists of a block locator and its Mtime
-type TrashRequest struct {
-       Locator    string `json:"locator"`
-       BlockMtime int64  `json:"block_mtime"`
-
-       // Target mount, or "" for "everywhere"
-       MountUUID string `json:"mount_uuid"`
-}
-
-// TrashHandler processes /trash requests.
-func (rtr *router) handleTrash(resp http.ResponseWriter, req *http.Request) {
-       // Reject unauthorized requests.
-       if !rtr.isSystemAuth(GetAPIToken(req)) {
-               http.Error(resp, UnauthorizedError.Error(), UnauthorizedError.HTTPCode)
-               return
-       }
-
-       // Parse the request body.
-       var trash []TrashRequest
-       r := json.NewDecoder(req.Body)
-       if err := r.Decode(&trash); err != nil {
-               http.Error(resp, err.Error(), BadRequestError.HTTPCode)
-               return
-       }
-
-       // We have a properly formatted trash list sent from the data
-       // manager.  Report success and send the list to the trash work
-       // queue for further handling.
-       resp.WriteHeader(http.StatusOK)
-       resp.Write([]byte(
-               fmt.Sprintf("Received %d trash requests\n", len(trash))))
-
-       tlist := list.New()
-       for _, t := range trash {
-               tlist.PushBack(t)
-       }
-       rtr.trashq.ReplaceQueue(tlist)
-}
-
-// UntrashHandler processes "PUT /untrash/{hash:[0-9a-f]{32}}" requests for the data manager.
-func (rtr *router) handleUntrash(resp http.ResponseWriter, req *http.Request) {
-       // Reject unauthorized requests.
-       if !rtr.isSystemAuth(GetAPIToken(req)) {
-               http.Error(resp, UnauthorizedError.Error(), UnauthorizedError.HTTPCode)
-               return
-       }
-
-       log := ctxlog.FromContext(req.Context())
-       hash := mux.Vars(req)["hash"]
-
-       if len(rtr.volmgr.AllWritable()) == 0 {
-               http.Error(resp, "No writable volumes", http.StatusNotFound)
-               return
-       }
-
-       var untrashedOn, failedOn []string
-       var numNotFound int
-       for _, vol := range rtr.volmgr.AllWritable() {
-               err := vol.Untrash(hash)
-
-               if os.IsNotExist(err) {
-                       numNotFound++
-               } else if err != nil {
-                       log.WithError(err).Errorf("Error untrashing %v on volume %s", hash, vol)
-                       failedOn = append(failedOn, vol.String())
-               } else {
-                       log.Infof("Untrashed %v on volume %v", hash, vol.String())
-                       untrashedOn = append(untrashedOn, vol.String())
-               }
-       }
-
-       if numNotFound == len(rtr.volmgr.AllWritable()) {
-               http.Error(resp, "Block not found on any of the writable volumes", http.StatusNotFound)
-       } else if len(failedOn) == len(rtr.volmgr.AllWritable()) {
-               http.Error(resp, "Failed to untrash on all writable volumes", http.StatusInternalServerError)
-       } else {
-               respBody := "Successfully untrashed on: " + strings.Join(untrashedOn, ", ")
-               if len(failedOn) > 0 {
-                       respBody += "; Failed to untrash on: " + strings.Join(failedOn, ", ")
-                       http.Error(resp, respBody, http.StatusInternalServerError)
-               } else {
-                       fmt.Fprintln(resp, respBody)
-               }
-       }
-}
-
-// GetBlock and PutBlock implement lower-level code for handling
-// blocks by rooting through volumes connected to the local machine.
-// Once the handler has determined that system policy permits the
-// request, it calls these methods to perform the actual operation.
-//
-// TODO(twp): this code would probably be better located in the
-// VolumeManager interface. As an abstraction, the VolumeManager
-// should be the only part of the code that cares about which volume a
-// block is stored on, so it should be responsible for figuring out
-// which volume to check for fetching blocks, storing blocks, etc.
-
-// GetBlock fetches the block identified by "hash" into the provided
-// buf, and returns the data size.
-//
-// If the block cannot be found on any volume, returns NotFoundError.
-//
-// If the block found does not have the correct MD5 hash, returns
-// DiskHashError.
-func GetBlock(ctx context.Context, volmgr *RRVolumeManager, hash string, buf []byte, resp http.ResponseWriter) (int, error) {
-       log := ctxlog.FromContext(ctx)
-
-       // Attempt to read the requested hash from a keep volume.
-       errorToCaller := NotFoundError
-
-       for _, vol := range volmgr.AllReadable() {
-               size, err := vol.Get(ctx, hash, buf)
-               select {
-               case <-ctx.Done():
-                       return 0, ErrClientDisconnect
-               default:
-               }
-               if err != nil {
-                       // IsNotExist is an expected error and may be
-                       // ignored. All other errors are logged. In
-                       // any case we continue trying to read other
-                       // volumes. If all volumes report IsNotExist,
-                       // we return a NotFoundError.
-                       if !os.IsNotExist(err) {
-                               log.WithError(err).Errorf("Get(%s) failed on %s", hash, vol)
-                       }
-                       // If some volume returns a transient error, return it to the caller
-                       // instead of "Not found" so it can retry.
-                       if err == VolumeBusyError {
-                               errorToCaller = err.(*KeepError)
-                       }
-                       continue
-               }
-               // Check the file checksum.
-               filehash := fmt.Sprintf("%x", md5.Sum(buf[:size]))
-               if filehash != hash {
-                       // TODO: Try harder to tell a sysadmin about
-                       // this.
-                       log.Errorf("checksum mismatch for block %s (actual %s), size %d on %s", hash, filehash, size, vol)
-                       errorToCaller = DiskHashError
-                       continue
-               }
-               if errorToCaller == DiskHashError {
-                       log.Warn("after checksum mismatch for block %s on a different volume, a good copy was found on volume %s and returned", hash, vol)
-               }
-               return size, nil
-       }
-       return 0, errorToCaller
-}
-
-type putProgress struct {
-       classNeeded      map[string]bool
-       classTodo        map[string]bool
-       mountUsed        map[*VolumeMount]bool
-       totalReplication int
-       classDone        map[string]int
-}
-
-// Number of distinct replicas stored. "2" can mean the block was
-// stored on 2 different volumes with replication 1, or on 1 volume
-// with replication 2.
-func (pr putProgress) TotalReplication() string {
-       return strconv.Itoa(pr.totalReplication)
-}
-
-// Number of replicas satisfying each storage class, formatted like
-// "default=2; special=1".
-func (pr putProgress) ClassReplication() string {
-       s := ""
-       for k, v := range pr.classDone {
-               if len(s) > 0 {
-                       s += ", "
-               }
-               s += k + "=" + strconv.Itoa(v)
-       }
-       return s
-}
-
-func (pr *putProgress) Add(mnt *VolumeMount) {
-       if pr.mountUsed[mnt] {
-               logrus.Warnf("BUG? superfluous extra write to mount %s", mnt.UUID)
-               return
-       }
-       pr.mountUsed[mnt] = true
-       pr.totalReplication += mnt.Replication
-       for class := range mnt.StorageClasses {
-               pr.classDone[class] += mnt.Replication
-               delete(pr.classTodo, class)
-       }
-}
-
-func (pr *putProgress) Sub(mnt *VolumeMount) {
-       if !pr.mountUsed[mnt] {
-               logrus.Warnf("BUG? Sub called with no prior matching Add: %s", mnt.UUID)
-               return
-       }
-       pr.mountUsed[mnt] = false
-       pr.totalReplication -= mnt.Replication
-       for class := range mnt.StorageClasses {
-               pr.classDone[class] -= mnt.Replication
-               if pr.classNeeded[class] {
-                       pr.classTodo[class] = true
-               }
-       }
-}
-
-func (pr *putProgress) Done() bool {
-       return len(pr.classTodo) == 0 && pr.totalReplication > 0
-}
-
-func (pr *putProgress) Want(mnt *VolumeMount) bool {
-       if pr.Done() || pr.mountUsed[mnt] {
-               return false
-       }
-       if len(pr.classTodo) == 0 {
-               // none specified == "any"
-               return true
-       }
-       for class := range mnt.StorageClasses {
-               if pr.classTodo[class] {
-                       return true
-               }
-       }
-       return false
-}
-
-func (pr *putProgress) Copy() *putProgress {
-       cp := putProgress{
-               classNeeded:      pr.classNeeded,
-               classTodo:        make(map[string]bool, len(pr.classTodo)),
-               classDone:        make(map[string]int, len(pr.classDone)),
-               mountUsed:        make(map[*VolumeMount]bool, len(pr.mountUsed)),
-               totalReplication: pr.totalReplication,
-       }
-       for k, v := range pr.classTodo {
-               cp.classTodo[k] = v
-       }
-       for k, v := range pr.classDone {
-               cp.classDone[k] = v
-       }
-       for k, v := range pr.mountUsed {
-               cp.mountUsed[k] = v
-       }
-       return &cp
-}
-
-func newPutProgress(classes []string) putProgress {
-       pr := putProgress{
-               classNeeded: make(map[string]bool, len(classes)),
-               classTodo:   make(map[string]bool, len(classes)),
-               classDone:   map[string]int{},
-               mountUsed:   map[*VolumeMount]bool{},
-       }
-       for _, c := range classes {
-               if c != "" {
-                       pr.classNeeded[c] = true
-                       pr.classTodo[c] = true
-               }
-       }
-       return pr
-}
-
-// PutBlock stores the given block on one or more volumes.
-//
-// The MD5 checksum of the block must match the given hash.
-//
-// The block is written to each writable volume (ordered by priority
-// and then UUID, see volume.go) until at least one replica has been
-// stored in each of the requested storage classes.
-//
-// The returned error, if any, is a KeepError with one of the
-// following codes:
-//
-// 500 Collision
-//
-//     A different block with the same hash already exists on this
-//     Keep server.
-//
-// 422 MD5Fail
-//
-//     The MD5 hash of the BLOCK does not match the argument HASH.
-//
-// 503 Full
-//
-//     There was not enough space left in any Keep volume to store
-//     the object.
-//
-// 500 Fail
-//
-//     The object could not be stored for some other reason (e.g.
-//     all writes failed). The text of the error message should
-//     provide as much detail as possible.
-func PutBlock(ctx context.Context, volmgr *RRVolumeManager, block []byte, hash string, wantStorageClasses []string) (putProgress, error) {
-       log := ctxlog.FromContext(ctx)
-
-       // Check that BLOCK's checksum matches HASH.
-       blockhash := fmt.Sprintf("%x", md5.Sum(block))
-       if blockhash != hash {
-               log.Printf("%s: MD5 checksum %s did not match request", hash, blockhash)
-               return putProgress{}, RequestHashError
-       }
-
-       result := newPutProgress(wantStorageClasses)
-
-       // If we already have this data, it's intact on disk, and we
-       // can update its timestamp, return success. If we have
-       // different data with the same hash, return failure.
-       if err := CompareAndTouch(ctx, volmgr, hash, block, &result); err != nil || result.Done() {
-               return result, err
-       }
-       if ctx.Err() != nil {
-               return result, ErrClientDisconnect
-       }
-
-       writables := volmgr.NextWritable()
-       if len(writables) == 0 {
-               log.Error("no writable volumes")
-               return result, FullError
-       }
-
-       var wg sync.WaitGroup
-       var mtx sync.Mutex
-       cond := sync.Cond{L: &mtx}
-       // pending predicts what result will be if all pending writes
-       // succeed.
-       pending := result.Copy()
-       var allFull atomic.Value
-       allFull.Store(true)
-
-       // We hold the lock for the duration of the "each volume" loop
-       // below, except when it is released during cond.Wait().
-       mtx.Lock()
-
-       for _, mnt := range writables {
-               // Wait until our decision to use this mount does not
-               // depend on the outcome of pending writes.
-               for result.Want(mnt) && !pending.Want(mnt) {
-                       cond.Wait()
-               }
-               if !result.Want(mnt) {
-                       continue
-               }
-               mnt := mnt
-               pending.Add(mnt)
-               wg.Add(1)
-               go func() {
-                       log.Debugf("PutBlock: start write to %s", mnt.UUID)
-                       defer wg.Done()
-                       err := mnt.Put(ctx, hash, block)
-
-                       mtx.Lock()
-                       if err != nil {
-                               log.Debugf("PutBlock: write to %s failed", mnt.UUID)
-                               pending.Sub(mnt)
-                       } else {
-                               log.Debugf("PutBlock: write to %s succeeded", mnt.UUID)
-                               result.Add(mnt)
-                       }
-                       cond.Broadcast()
-                       mtx.Unlock()
-
-                       if err != nil && err != FullError && ctx.Err() == nil {
-                               // The volume is not full but the
-                               // write did not succeed.  Report the
-                               // error and continue trying.
-                               allFull.Store(false)
-                               log.WithError(err).Errorf("%s: Put(%s) failed", mnt.Volume, hash)
-                       }
-               }()
-       }
-       mtx.Unlock()
-       wg.Wait()
-       if ctx.Err() != nil {
-               return result, ErrClientDisconnect
-       }
-       if result.Done() {
-               return result, nil
-       }
-
-       if result.totalReplication > 0 {
-               // Some, but not all, of the storage classes were
-               // satisfied. This qualifies as success.
-               return result, nil
-       } else if allFull.Load().(bool) {
-               log.Error("all volumes with qualifying storage classes are full")
-               return putProgress{}, FullError
-       } else {
-               // Already logged the non-full errors.
-               return putProgress{}, GenericError
-       }
-}
-
-// CompareAndTouch looks for volumes where the given content already
-// exists and its modification time can be updated (i.e., it is
-// protected from garbage collection), and updates result accordingly.
-// It returns when the result is Done() or all volumes have been
-// checked.
-func CompareAndTouch(ctx context.Context, volmgr *RRVolumeManager, hash string, buf []byte, result *putProgress) error {
-       log := ctxlog.FromContext(ctx)
-       for _, mnt := range volmgr.AllWritable() {
-               if !result.Want(mnt) {
-                       continue
-               }
-               err := mnt.Compare(ctx, hash, buf)
-               if ctx.Err() != nil {
-                       return nil
-               } else if err == CollisionError {
-                       // Stop if we have a block with same hash but
-                       // different content. (It will be impossible
-                       // to tell which one is wanted if we have
-                       // both, so there's no point writing it even
-                       // on a different volume.)
-                       log.Errorf("collision in Compare(%s) on volume %s", hash, mnt.Volume)
-                       return CollisionError
-               } else if os.IsNotExist(err) {
-                       // Block does not exist. This is the only
-                       // "normal" error: we don't log anything.
-                       continue
-               } else if err != nil {
-                       // Couldn't open file, data is corrupt on
-                       // disk, etc.: log this abnormal condition,
-                       // and try the next volume.
-                       log.WithError(err).Warnf("error in Compare(%s) on volume %s", hash, mnt.Volume)
-                       continue
-               }
-               if err := mnt.Touch(hash); err != nil {
-                       log.WithError(err).Errorf("error in Touch(%s) on volume %s", hash, mnt.Volume)
-                       continue
-               }
-               // Compare and Touch both worked --> done.
-               result.Add(mnt)
-               if result.Done() {
-                       return nil
-               }
-       }
-       return nil
-}
-
-var validLocatorRe = regexp.MustCompile(`^[0-9a-f]{32}$`)
-
-// IsValidLocator returns true if the specified string is a valid Keep
-// locator.  When Keep is extended to support hash types other than
-// MD5, this should be updated to cover those as well.
-func IsValidLocator(loc string) bool {
-       return validLocatorRe.MatchString(loc)
-}
-
-var authRe = regexp.MustCompile(`^(OAuth2|Bearer)\s+(.*)`)
-
-// GetAPIToken returns the OAuth2 token from the Authorization
-// header of a HTTP request, or an empty string if no matching
-// token is found.
-func GetAPIToken(req *http.Request) string {
-       if auth, ok := req.Header["Authorization"]; ok {
-               if match := authRe.FindStringSubmatch(auth[0]); match != nil {
-                       return match[2]
-               }
-       }
-       return ""
-}
-
-// canDelete returns true if the user identified by apiToken is
-// allowed to delete blocks.
-func (rtr *router) canDelete(apiToken string) bool {
-       if apiToken == "" {
-               return false
-       }
-       // Blocks may be deleted only when Keep has been configured with a
-       // data manager.
-       if rtr.isSystemAuth(apiToken) {
-               return true
-       }
-       // TODO(twp): look up apiToken with the API server
-       // return true if is_admin is true and if the token
-       // has unlimited scope
-       return false
-}
-
-// isSystemAuth returns true if the given token is allowed to perform
-// system level actions like deleting data.
-func (rtr *router) isSystemAuth(token string) bool {
-       return token != "" && token == rtr.cluster.SystemRootToken
-}
diff --git a/services/keepstore/hashcheckwriter.go b/services/keepstore/hashcheckwriter.go
new file mode 100644 (file)
index 0000000..f191c98
--- /dev/null
@@ -0,0 +1,68 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "fmt"
+       "hash"
+       "io"
+)
+
+type hashCheckWriter struct {
+       writer       io.Writer
+       hash         hash.Hash
+       expectSize   int64
+       expectDigest string
+
+       offset int64
+}
+
+// newHashCheckWriter returns a writer that writes through to w, but
+// stops short if the written content reaches expectSize bytes and
+// does not match expectDigest according to the given hash
+// function.
+//
+// It returns a write error if more than expectSize bytes are written.
+//
+// Thus, in case of a hash mismatch, fewer than expectSize will be
+// written through.
+func newHashCheckWriter(writer io.Writer, hash hash.Hash, expectSize int64, expectDigest string) io.Writer {
+       return &hashCheckWriter{
+               writer:       writer,
+               hash:         hash,
+               expectSize:   expectSize,
+               expectDigest: expectDigest,
+       }
+}
+
+func (hcw *hashCheckWriter) Write(p []byte) (int, error) {
+       if todo := hcw.expectSize - hcw.offset - int64(len(p)); todo < 0 {
+               // Writing beyond expected size returns a checksum
+               // error without even checking the hash.
+               return 0, errChecksum
+       } else if todo > 0 {
+               // This isn't the last write, so we pass it through.
+               _, err := hcw.hash.Write(p)
+               if err != nil {
+                       return 0, err
+               }
+               n, err := hcw.writer.Write(p)
+               hcw.offset += int64(n)
+               return n, err
+       } else {
+               // This is the last write, so we check the hash before
+               // writing through.
+               _, err := hcw.hash.Write(p)
+               if err != nil {
+                       return 0, err
+               }
+               if digest := fmt.Sprintf("%x", hcw.hash.Sum(nil)); digest != hcw.expectDigest {
+                       return 0, errChecksum
+               }
+               // Ensure subsequent write will fail
+               hcw.offset = hcw.expectSize + 1
+               return hcw.writer.Write(p)
+       }
+}
index 953aa047cbfa6ab6f7b55630aa30e83732adaf0e..89afa9089f973b968493c275fa13d0aaa4db8d45 100644 (file)
 package keepstore
 
 import (
+       "bytes"
+       "context"
+       "crypto/md5"
+       "errors"
+       "fmt"
+       "io"
+       "net/http"
+       "os"
+       "sort"
+       "strconv"
+       "strings"
+       "sync"
+       "sync/atomic"
        "time"
+
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/arvadosclient"
+       "git.arvados.org/arvados.git/sdk/go/auth"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "git.arvados.org/arvados.git/sdk/go/httpserver"
+       "git.arvados.org/arvados.git/sdk/go/keepclient"
+       "github.com/prometheus/client_golang/prometheus"
+       "github.com/sirupsen/logrus"
 )
 
-// BlockSize for a Keep "block" is 64MB.
-const BlockSize = 64 * 1024 * 1024
+// Maximum size of a keep block is 64 MiB.
+const BlockSize = 1 << 26
 
-// MinFreeKilobytes is the amount of space a Keep volume must have available
-// in order to permit writes.
-const MinFreeKilobytes = BlockSize / 1024
+var (
+       errChecksum          = httpserver.ErrorWithStatus(errors.New("checksum mismatch in stored data"), http.StatusBadGateway)
+       errNoTokenProvided   = httpserver.ErrorWithStatus(errors.New("no token provided in Authorization header"), http.StatusUnauthorized)
+       errMethodNotAllowed  = httpserver.ErrorWithStatus(errors.New("method not allowed"), http.StatusMethodNotAllowed)
+       errVolumeUnavailable = httpserver.ErrorWithStatus(errors.New("volume unavailable"), http.StatusServiceUnavailable)
+       errCollision         = httpserver.ErrorWithStatus(errors.New("hash collision"), http.StatusInternalServerError)
+       errExpiredSignature  = httpserver.ErrorWithStatus(errors.New("expired signature"), http.StatusUnauthorized)
+       errInvalidSignature  = httpserver.ErrorWithStatus(errors.New("invalid signature"), http.StatusBadRequest)
+       errInvalidLocator    = httpserver.ErrorWithStatus(errors.New("invalid locator"), http.StatusBadRequest)
+       errFull              = httpserver.ErrorWithStatus(errors.New("insufficient storage"), http.StatusInsufficientStorage)
+       errTooLarge          = httpserver.ErrorWithStatus(errors.New("request entity too large"), http.StatusRequestEntityTooLarge)
+       driver               = make(map[string]volumeDriver)
+)
 
-var bufs *bufferPool
+type IndexOptions struct {
+       MountUUID string
+       Prefix    string
+       WriteTo   io.Writer
+}
 
-type KeepError struct {
-       HTTPCode int
-       ErrMsg   string
+type mount struct {
+       arvados.KeepMount
+       volume
+       priority int
 }
 
-var (
-       BadRequestError     = &KeepError{400, "Bad Request"}
-       UnauthorizedError   = &KeepError{401, "Unauthorized"}
-       CollisionError      = &KeepError{500, "Collision"}
-       RequestHashError    = &KeepError{422, "Hash mismatch in request"}
-       PermissionError     = &KeepError{403, "Forbidden"}
-       DiskHashError       = &KeepError{500, "Hash mismatch in stored data"}
-       ExpiredError        = &KeepError{401, "Expired permission signature"}
-       NotFoundError       = &KeepError{404, "Not Found"}
-       VolumeBusyError     = &KeepError{503, "Volume backend busy"}
-       GenericError        = &KeepError{500, "Fail"}
-       FullError           = &KeepError{503, "Full"}
-       SizeRequiredError   = &KeepError{411, "Missing Content-Length"}
-       TooLongError        = &KeepError{413, "Block is too large"}
-       MethodDisabledError = &KeepError{405, "Method disabled"}
-       ErrNotImplemented   = &KeepError{500, "Unsupported configuration"}
-       ErrClientDisconnect = &KeepError{503, "Client disconnected"}
-)
+type keepstore struct {
+       cluster    *arvados.Cluster
+       logger     logrus.FieldLogger
+       serviceURL arvados.URL
+       mounts     map[string]*mount
+       mountsR    []*mount
+       mountsW    []*mount
+       bufferPool *bufferPool
 
-func (e *KeepError) Error() string {
-       return e.ErrMsg
+       iostats map[volume]*ioStats
+
+       remoteClients    map[string]*keepclient.KeepClient
+       remoteClientsMtx sync.Mutex
 }
 
-// Periodically (once per interval) invoke EmptyTrash on all volumes.
-func emptyTrash(mounts []*VolumeMount, interval time.Duration) {
-       for range time.NewTicker(interval).C {
-               for _, v := range mounts {
-                       if v.KeepMount.AllowTrash {
-                               v.EmptyTrash()
+func newKeepstore(ctx context.Context, cluster *arvados.Cluster, token string, reg *prometheus.Registry, serviceURL arvados.URL) (*keepstore, error) {
+       logger := ctxlog.FromContext(ctx)
+
+       if cluster.API.MaxConcurrentRequests > 0 && cluster.API.MaxConcurrentRequests < cluster.API.MaxKeepBlobBuffers {
+               logger.Warnf("Possible configuration mistake: not useful to set API.MaxKeepBlobBuffers (%d) higher than API.MaxConcurrentRequests (%d)", cluster.API.MaxKeepBlobBuffers, cluster.API.MaxConcurrentRequests)
+       }
+
+       if cluster.Collections.BlobSigningKey != "" {
+       } else if cluster.Collections.BlobSigning {
+               return nil, errors.New("cannot enable Collections.BlobSigning with no Collections.BlobSigningKey")
+       } else {
+               logger.Warn("Running without a blob signing key. Block locators returned by this server will not be signed, and will be rejected by a server that enforces permissions. To fix this, configure Collections.BlobSigning and Collections.BlobSigningKey.")
+       }
+
+       if cluster.API.MaxKeepBlobBuffers <= 0 {
+               return nil, fmt.Errorf("API.MaxKeepBlobBuffers must be greater than zero")
+       }
+       bufferPool := newBufferPool(logger, cluster.API.MaxKeepBlobBuffers, reg)
+
+       ks := &keepstore{
+               cluster:       cluster,
+               logger:        logger,
+               serviceURL:    serviceURL,
+               bufferPool:    bufferPool,
+               remoteClients: make(map[string]*keepclient.KeepClient),
+       }
+
+       err := ks.setupMounts(newVolumeMetricsVecs(reg))
+       if err != nil {
+               return nil, err
+       }
+
+       return ks, nil
+}
+
+func (ks *keepstore) setupMounts(metrics *volumeMetricsVecs) error {
+       ks.mounts = make(map[string]*mount)
+       if len(ks.cluster.Volumes) == 0 {
+               return errors.New("no volumes configured")
+       }
+       for uuid, cfgvol := range ks.cluster.Volumes {
+               va, ok := cfgvol.AccessViaHosts[ks.serviceURL]
+               if !ok && len(cfgvol.AccessViaHosts) > 0 {
+                       continue
+               }
+               dri, ok := driver[cfgvol.Driver]
+               if !ok {
+                       return fmt.Errorf("volume %s: invalid driver %q", uuid, cfgvol.Driver)
+               }
+               vol, err := dri(newVolumeParams{
+                       UUID:         uuid,
+                       Cluster:      ks.cluster,
+                       ConfigVolume: cfgvol,
+                       Logger:       ks.logger,
+                       MetricsVecs:  metrics,
+                       BufferPool:   ks.bufferPool,
+               })
+               if err != nil {
+                       return fmt.Errorf("error initializing volume %s: %s", uuid, err)
+               }
+               sc := cfgvol.StorageClasses
+               if len(sc) == 0 {
+                       sc = map[string]bool{"default": true}
+               }
+               repl := cfgvol.Replication
+               if repl < 1 {
+                       repl = 1
+               }
+               pri := 0
+               for class, in := range cfgvol.StorageClasses {
+                       p := ks.cluster.StorageClasses[class].Priority
+                       if in && p > pri {
+                               pri = p
                        }
                }
+               mnt := &mount{
+                       volume:   vol,
+                       priority: pri,
+                       KeepMount: arvados.KeepMount{
+                               UUID:           uuid,
+                               DeviceID:       vol.DeviceID(),
+                               AllowWrite:     !va.ReadOnly && !cfgvol.ReadOnly,
+                               AllowTrash:     !va.ReadOnly && (!cfgvol.ReadOnly || cfgvol.AllowTrashWhenReadOnly),
+                               Replication:    repl,
+                               StorageClasses: sc,
+                       },
+               }
+               ks.mounts[uuid] = mnt
+               ks.logger.Printf("started volume %s (%s), AllowWrite=%v, AllowTrash=%v", uuid, vol.DeviceID(), mnt.AllowWrite, mnt.AllowTrash)
+       }
+       if len(ks.mounts) == 0 {
+               return fmt.Errorf("no volumes configured for %s", ks.serviceURL)
+       }
+
+       ks.mountsR = nil
+       ks.mountsW = nil
+       for _, mnt := range ks.mounts {
+               ks.mountsR = append(ks.mountsR, mnt)
+               if mnt.AllowWrite {
+                       ks.mountsW = append(ks.mountsW, mnt)
+               }
+       }
+       // Sorting mounts by UUID makes behavior more predictable, and
+       // is convenient for testing -- for example, "index all
+       // volumes" and "trash block on all volumes" will visit
+       // volumes in predictable order.
+       sort.Slice(ks.mountsR, func(i, j int) bool { return ks.mountsR[i].UUID < ks.mountsR[j].UUID })
+       sort.Slice(ks.mountsW, func(i, j int) bool { return ks.mountsW[i].UUID < ks.mountsW[j].UUID })
+       return nil
+}
+
+// checkLocatorSignature checks that locator has a valid signature.
+// If the BlobSigning config is false, it returns nil even if the
+// signature is invalid or missing.
+func (ks *keepstore) checkLocatorSignature(ctx context.Context, locator string) error {
+       if !ks.cluster.Collections.BlobSigning {
+               return nil
+       }
+       token := ctxToken(ctx)
+       if token == "" {
+               return errNoTokenProvided
+       }
+       err := arvados.VerifySignature(locator, token, ks.cluster.Collections.BlobSigningTTL.Duration(), []byte(ks.cluster.Collections.BlobSigningKey))
+       if err == arvados.ErrSignatureExpired {
+               return errExpiredSignature
+       } else if err != nil {
+               return errInvalidSignature
+       }
+       return nil
+}
+
+// signLocator signs the locator for the given token, if possible.
+// Note this signs if the BlobSigningKey config is available, even if
+// the BlobSigning config is false.
+func (ks *keepstore) signLocator(token, locator string) string {
+       if token == "" || len(ks.cluster.Collections.BlobSigningKey) == 0 {
+               return locator
+       }
+       ttl := ks.cluster.Collections.BlobSigningTTL.Duration()
+       return arvados.SignLocator(locator, token, time.Now().Add(ttl), ttl, []byte(ks.cluster.Collections.BlobSigningKey))
+}
+
+func (ks *keepstore) BlockRead(ctx context.Context, opts arvados.BlockReadOptions) (n int, err error) {
+       li, err := parseLocator(opts.Locator)
+       if err != nil {
+               return 0, err
+       }
+       out := opts.WriteTo
+       if rw, ok := out.(http.ResponseWriter); ok && li.size > 0 {
+               out = &setSizeOnWrite{ResponseWriter: rw, size: li.size}
+       }
+       if li.remote && !li.signed {
+               return ks.blockReadRemote(ctx, opts)
+       }
+       if err := ks.checkLocatorSignature(ctx, opts.Locator); err != nil {
+               return 0, err
+       }
+       hashcheck := md5.New()
+       if li.size > 0 {
+               out = newHashCheckWriter(out, hashcheck, int64(li.size), li.hash)
+       } else {
+               out = io.MultiWriter(out, hashcheck)
+       }
+       var errToCaller error = os.ErrNotExist
+       for _, mnt := range ks.rendezvous(li.hash, ks.mountsR) {
+               if ctx.Err() != nil {
+                       return 0, ctx.Err()
+               }
+               n, err = mnt.BlockRead(ctx, li.hash, out)
+               if err == nil && li.size > 0 && n != li.size {
+                       // If the backend read fewer bytes than
+                       // expected but returns no error, we can
+                       // classify this as a checksum error (even
+                       // though hashCheckWriter doesn't know that
+                       // yet, it's just waiting for the next
+                       // write). If our caller is serving a GET
+                       // request it's too late to do anything about
+                       // it anyway, but if it's a HEAD request the
+                       // caller can still change the response status
+                       // code.
+                       return n, errChecksum
+               }
+               if err == nil && li.size == 0 {
+                       // hashCheckingWriter isn't in use because we
+                       // don't know the expected size. All we can do
+                       // is check after writing all the data, and
+                       // trust the caller is doing a HEAD request so
+                       // it's not too late to set an error code in
+                       // the response header.
+                       if hash := fmt.Sprintf("%x", hashcheck.Sum(nil)); hash != li.hash {
+                               return n, errChecksum
+                       }
+               }
+               if rw, ok := opts.WriteTo.(http.ResponseWriter); ok && li.size == 0 && err == nil {
+                       // We didn't set the content-length header
+                       // above because we didn't know the block size
+                       // until now.
+                       rw.Header().Set("Content-Length", fmt.Sprintf("%d", n))
+               }
+               if n > 0 || err == nil {
+                       // success, or there's an error but we can't
+                       // retry because we've already sent some data.
+                       return n, err
+               }
+               if !os.IsNotExist(err) {
+                       // If some volume returns a transient error,
+                       // return it to the caller instead of "Not
+                       // found" so it can retry.
+                       errToCaller = err
+               }
+       }
+       return 0, errToCaller
+}
+
+func (ks *keepstore) blockReadRemote(ctx context.Context, opts arvados.BlockReadOptions) (int, error) {
+       ks.logger.Infof("blockReadRemote(%s)", opts.Locator)
+       token := ctxToken(ctx)
+       if token == "" {
+               return 0, errNoTokenProvided
+       }
+       var remoteClient *keepclient.KeepClient
+       var parts []string
+       var size int
+       for i, part := range strings.Split(opts.Locator, "+") {
+               switch {
+               case i == 0:
+                       // don't try to parse hash part as hint
+               case strings.HasPrefix(part, "A"):
+                       // drop local permission hint
+                       continue
+               case len(part) > 7 && part[0] == 'R' && part[6] == '-':
+                       remoteID := part[1:6]
+                       remote, ok := ks.cluster.RemoteClusters[remoteID]
+                       if !ok {
+                               return 0, httpserver.ErrorWithStatus(errors.New("remote cluster not configured"), http.StatusBadRequest)
+                       }
+                       kc, err := ks.remoteClient(remoteID, remote, token)
+                       if err == auth.ErrObsoleteToken {
+                               return 0, httpserver.ErrorWithStatus(err, http.StatusBadRequest)
+                       } else if err != nil {
+                               return 0, err
+                       }
+                       remoteClient = kc
+                       part = "A" + part[7:]
+               case len(part) > 0 && part[0] >= '0' && part[0] <= '9':
+                       size, _ = strconv.Atoi(part)
+               }
+               parts = append(parts, part)
+       }
+       if remoteClient == nil {
+               return 0, httpserver.ErrorWithStatus(errors.New("invalid remote hint"), http.StatusBadRequest)
+       }
+       locator := strings.Join(parts, "+")
+       if opts.LocalLocator == nil {
+               // Read from remote cluster and stream response back
+               // to caller
+               if rw, ok := opts.WriteTo.(http.ResponseWriter); ok && size > 0 {
+                       rw.Header().Set("Content-Length", fmt.Sprintf("%d", size))
+               }
+               return remoteClient.BlockRead(ctx, arvados.BlockReadOptions{
+                       Locator: locator,
+                       WriteTo: opts.WriteTo,
+               })
+       }
+       // We must call LocalLocator before writing any data to
+       // opts.WriteTo, otherwise the caller can't put the local
+       // locator in a response header.  So we copy into memory,
+       // generate the local signature, then copy from memory to
+       // opts.WriteTo.
+       buf, err := ks.bufferPool.GetContext(ctx)
+       if err != nil {
+               return 0, err
+       }
+       defer ks.bufferPool.Put(buf)
+       writebuf := bytes.NewBuffer(buf[:0])
+       ks.logger.Infof("blockReadRemote(%s): remote read(%s)", opts.Locator, locator)
+       _, err = remoteClient.BlockRead(ctx, arvados.BlockReadOptions{
+               Locator: locator,
+               WriteTo: writebuf,
+       })
+       if err != nil {
+               return 0, err
+       }
+       resp, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+               Hash: locator,
+               Data: writebuf.Bytes(),
+       })
+       if err != nil {
+               return 0, err
+       }
+       opts.LocalLocator(resp.Locator)
+       if rw, ok := opts.WriteTo.(http.ResponseWriter); ok {
+               rw.Header().Set("Content-Length", fmt.Sprintf("%d", writebuf.Len()))
+       }
+       n, err := io.Copy(opts.WriteTo, bytes.NewReader(writebuf.Bytes()))
+       return int(n), err
+}
+
+func (ks *keepstore) remoteClient(remoteID string, remoteCluster arvados.RemoteCluster, token string) (*keepclient.KeepClient, error) {
+       ks.remoteClientsMtx.Lock()
+       kc, ok := ks.remoteClients[remoteID]
+       ks.remoteClientsMtx.Unlock()
+       if !ok {
+               c := &arvados.Client{
+                       APIHost:   remoteCluster.Host,
+                       AuthToken: "xxx",
+                       Insecure:  remoteCluster.Insecure,
+               }
+               ac, err := arvadosclient.New(c)
+               if err != nil {
+                       return nil, err
+               }
+               kc, err = keepclient.MakeKeepClient(ac)
+               if err != nil {
+                       return nil, err
+               }
+               kc.DiskCacheSize = keepclient.DiskCacheDisabled
+
+               ks.remoteClientsMtx.Lock()
+               ks.remoteClients[remoteID] = kc
+               ks.remoteClientsMtx.Unlock()
+       }
+       accopy := *kc.Arvados
+       accopy.ApiToken = token
+       kccopy := kc.Clone()
+       kccopy.Arvados = &accopy
+       token, err := auth.SaltToken(token, remoteID)
+       if err != nil {
+               return nil, err
+       }
+       kccopy.Arvados.ApiToken = token
+       return kccopy, nil
+}
+
+// BlockWrite writes a block to one or more volumes.
+func (ks *keepstore) BlockWrite(ctx context.Context, opts arvados.BlockWriteOptions) (arvados.BlockWriteResponse, error) {
+       var resp arvados.BlockWriteResponse
+       var hash string
+       if opts.Data == nil {
+               buf, err := ks.bufferPool.GetContext(ctx)
+               if err != nil {
+                       return resp, err
+               }
+               defer ks.bufferPool.Put(buf)
+               w := bytes.NewBuffer(buf[:0])
+               h := md5.New()
+               limitedReader := &io.LimitedReader{R: opts.Reader, N: BlockSize}
+               n, err := io.Copy(io.MultiWriter(w, h), limitedReader)
+               if err != nil {
+                       return resp, err
+               }
+               if limitedReader.N == 0 {
+                       // Data size is either exactly BlockSize, or too big.
+                       n, err := opts.Reader.Read(make([]byte, 1))
+                       if n > 0 {
+                               return resp, httpserver.ErrorWithStatus(err, http.StatusRequestEntityTooLarge)
+                       }
+                       if err != io.EOF {
+                               return resp, err
+                       }
+               }
+               opts.Data = buf[:n]
+               if opts.DataSize != 0 && int(n) != opts.DataSize {
+                       return resp, httpserver.ErrorWithStatus(fmt.Errorf("content length %d did not match specified data size %d", n, opts.DataSize), http.StatusBadRequest)
+               }
+               hash = fmt.Sprintf("%x", h.Sum(nil))
+       } else {
+               hash = fmt.Sprintf("%x", md5.Sum(opts.Data))
+       }
+       if opts.Hash != "" && !strings.HasPrefix(opts.Hash, hash) {
+               return resp, httpserver.ErrorWithStatus(fmt.Errorf("content hash %s did not match specified locator %s", hash, opts.Hash), http.StatusBadRequest)
+       }
+       rvzmounts := ks.rendezvous(hash, ks.mountsW)
+       result := newPutProgress(opts.StorageClasses)
+       for _, mnt := range rvzmounts {
+               if !result.Want(mnt) {
+                       continue
+               }
+               cmp := &checkEqual{Expect: opts.Data}
+               if _, err := mnt.BlockRead(ctx, hash, cmp); err == nil {
+                       if !cmp.Equal() {
+                               return resp, errCollision
+                       }
+                       err := mnt.BlockTouch(hash)
+                       if err == nil {
+                               result.Add(mnt)
+                       }
+               }
+       }
+       var allFull atomic.Bool
+       allFull.Store(true)
+       // pending tracks what result will be if all outstanding
+       // writes succeed.
+       pending := result.Copy()
+       cond := sync.NewCond(new(sync.Mutex))
+       cond.L.Lock()
+       var wg sync.WaitGroup
+nextmnt:
+       for _, mnt := range rvzmounts {
+               for {
+                       if result.Done() || ctx.Err() != nil {
+                               break nextmnt
+                       }
+                       if !result.Want(mnt) {
+                               continue nextmnt
+                       }
+                       if pending.Want(mnt) {
+                               break
+                       }
+                       // This mount might not be needed, depending
+                       // on the outcome of pending writes. Wait for
+                       // a pending write to finish, then check
+                       // again.
+                       cond.Wait()
+               }
+               mnt := mnt
+               logger := ks.logger.WithField("mount", mnt.UUID)
+               pending.Add(mnt)
+               wg.Add(1)
+               go func() {
+                       defer wg.Done()
+                       logger.Debug("start write")
+                       err := mnt.BlockWrite(ctx, hash, opts.Data)
+                       cond.L.Lock()
+                       defer cond.L.Unlock()
+                       defer cond.Broadcast()
+                       if err != nil {
+                               logger.Debug("write failed")
+                               pending.Sub(mnt)
+                               if err != errFull {
+                                       allFull.Store(false)
+                               }
+                       } else {
+                               result.Add(mnt)
+                               pending.Sub(mnt)
+                       }
+               }()
+       }
+       cond.L.Unlock()
+       wg.Wait()
+       if ctx.Err() != nil {
+               return resp, ctx.Err()
+       }
+       if result.Done() || result.totalReplication > 0 {
+               resp = arvados.BlockWriteResponse{
+                       Locator:        ks.signLocator(ctxToken(ctx), fmt.Sprintf("%s+%d", hash, len(opts.Data))),
+                       Replicas:       result.totalReplication,
+                       StorageClasses: result.classDone,
+               }
+               return resp, nil
+       }
+       if allFull.Load() {
+               return resp, errFull
+       }
+       return resp, errVolumeUnavailable
+}
+
+// rendezvous sorts the given mounts by descending priority, then by
+// rendezvous order for the given locator.
+func (*keepstore) rendezvous(locator string, mnts []*mount) []*mount {
+       hash := locator
+       if len(hash) > 32 {
+               hash = hash[:32]
+       }
+       // copy the provided []*mount before doing an in-place sort
+       mnts = append([]*mount(nil), mnts...)
+       weight := make(map[*mount]string)
+       for _, mnt := range mnts {
+               uuidpart := mnt.UUID
+               if len(uuidpart) == 27 {
+                       // strip zzzzz-yyyyy- prefixes
+                       uuidpart = uuidpart[12:]
+               }
+               weight[mnt] = fmt.Sprintf("%x", md5.Sum([]byte(hash+uuidpart)))
+       }
+       sort.Slice(mnts, func(i, j int) bool {
+               if p := mnts[i].priority - mnts[j].priority; p != 0 {
+                       return p > 0
+               }
+               return weight[mnts[i]] < weight[mnts[j]]
+       })
+       return mnts
+}
+
+// checkEqual reports whether the data written to it (via io.Writer
+// interface) is equal to the expected data.
+//
+// Expect should not be changed after the first Write.
+type checkEqual struct {
+       Expect     []byte
+       equalUntil int
+}
+
+func (ce *checkEqual) Equal() bool {
+       return ce.equalUntil == len(ce.Expect)
+}
+
+func (ce *checkEqual) Write(p []byte) (int, error) {
+       endpos := ce.equalUntil + len(p)
+       if ce.equalUntil >= 0 && endpos <= len(ce.Expect) && bytes.Equal(p, ce.Expect[ce.equalUntil:endpos]) {
+               ce.equalUntil = endpos
+       } else {
+               ce.equalUntil = -1
+       }
+       return len(p), nil
+}
+
+func (ks *keepstore) BlockUntrash(ctx context.Context, locator string) error {
+       li, err := parseLocator(locator)
+       if err != nil {
+               return err
+       }
+       var errToCaller error = os.ErrNotExist
+       for _, mnt := range ks.mountsW {
+               if ctx.Err() != nil {
+                       return ctx.Err()
+               }
+               err := mnt.BlockUntrash(li.hash)
+               if err == nil {
+                       errToCaller = nil
+               } else if !os.IsNotExist(err) && errToCaller != nil {
+                       errToCaller = err
+               }
+       }
+       return errToCaller
+}
+
+func (ks *keepstore) BlockTouch(ctx context.Context, locator string) error {
+       li, err := parseLocator(locator)
+       if err != nil {
+               return err
+       }
+       var errToCaller error = os.ErrNotExist
+       for _, mnt := range ks.mountsW {
+               if ctx.Err() != nil {
+                       return ctx.Err()
+               }
+               err := mnt.BlockTouch(li.hash)
+               if err == nil {
+                       return nil
+               }
+               if !os.IsNotExist(err) {
+                       errToCaller = err
+               }
+       }
+       return errToCaller
+}
+
+func (ks *keepstore) BlockTrash(ctx context.Context, locator string) error {
+       if !ks.cluster.Collections.BlobTrash {
+               return errMethodNotAllowed
+       }
+       li, err := parseLocator(locator)
+       if err != nil {
+               return err
+       }
+       var errToCaller error = os.ErrNotExist
+       for _, mnt := range ks.mounts {
+               if !mnt.AllowTrash {
+                       continue
+               }
+               if ctx.Err() != nil {
+                       return ctx.Err()
+               }
+               t, err := mnt.Mtime(li.hash)
+               if err == nil && time.Now().Sub(t) > ks.cluster.Collections.BlobSigningTTL.Duration() {
+                       err = mnt.BlockTrash(li.hash)
+               }
+               if os.IsNotExist(errToCaller) || (errToCaller == nil && !os.IsNotExist(err)) {
+                       errToCaller = err
+               }
+       }
+       return errToCaller
+}
+
+func (ks *keepstore) Mounts() []*mount {
+       return ks.mountsR
+}
+
+func (ks *keepstore) Index(ctx context.Context, opts IndexOptions) error {
+       mounts := ks.mountsR
+       if opts.MountUUID != "" {
+               mnt, ok := ks.mounts[opts.MountUUID]
+               if !ok {
+                       return os.ErrNotExist
+               }
+               mounts = []*mount{mnt}
+       }
+       for _, mnt := range mounts {
+               err := mnt.Index(ctx, opts.Prefix, opts.WriteTo)
+               if err != nil {
+                       return err
+               }
+       }
+       return nil
+}
+
+func ctxToken(ctx context.Context) string {
+       if c, ok := auth.FromContext(ctx); ok && len(c.Tokens) > 0 {
+               return c.Tokens[0]
+       } else {
+               return ""
+       }
+}
+
+type locatorInfo struct {
+       hash   string
+       size   int
+       remote bool
+       signed bool
+}
+
+func parseLocator(loc string) (locatorInfo, error) {
+       var li locatorInfo
+       for i, part := range strings.Split(loc, "+") {
+               if i == 0 {
+                       if len(part) != 32 {
+                               return li, errInvalidLocator
+                       }
+                       li.hash = part
+                       continue
+               }
+               if i == 1 {
+                       if size, err := strconv.Atoi(part); err == nil {
+                               li.size = size
+                               continue
+                       }
+               }
+               if len(part) == 0 {
+                       return li, errInvalidLocator
+               }
+               if part[0] == 'A' {
+                       li.signed = true
+               }
+               if part[0] == 'R' {
+                       li.remote = true
+               }
+               if part[0] >= '0' && part[0] <= '9' {
+                       // size, if present at all, must come first
+                       return li, errInvalidLocator
+               }
        }
+       return li, nil
 }
diff --git a/services/keepstore/keepstore_test.go b/services/keepstore/keepstore_test.go
new file mode 100644 (file)
index 0000000..3a01476
--- /dev/null
@@ -0,0 +1,874 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "bytes"
+       "context"
+       "crypto/md5"
+       "errors"
+       "fmt"
+       "io"
+       "net/http"
+       "os"
+       "sort"
+       "strings"
+       "sync"
+       "testing"
+       "time"
+
+       "git.arvados.org/arvados.git/lib/config"
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/arvadostest"
+       "git.arvados.org/arvados.git/sdk/go/auth"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "github.com/prometheus/client_golang/prometheus"
+       . "gopkg.in/check.v1"
+)
+
+func TestGocheck(t *testing.T) {
+       TestingT(t)
+}
+
+const (
+       fooHash = "acbd18db4cc2f85cedef654fccc4a4d8"
+       barHash = "37b51d194a7513e45b56f6524f2d51f2"
+)
+
+var testServiceURL = func() arvados.URL {
+       return arvados.URL{Host: "localhost:12345", Scheme: "http"}
+}()
+
+func authContext(token string) context.Context {
+       return auth.NewContext(context.TODO(), &auth.Credentials{Tokens: []string{token}})
+}
+
+func testCluster(t TB) *arvados.Cluster {
+       cfg, err := config.NewLoader(bytes.NewBufferString("Clusters: {zzzzz: {}}"), ctxlog.TestLogger(t)).Load()
+       if err != nil {
+               t.Fatal(err)
+       }
+       cluster, err := cfg.GetCluster("")
+       if err != nil {
+               t.Fatal(err)
+       }
+       cluster.SystemRootToken = arvadostest.SystemRootToken
+       cluster.ManagementToken = arvadostest.ManagementToken
+       return cluster
+}
+
+func testKeepstore(t TB, cluster *arvados.Cluster, reg *prometheus.Registry) (*keepstore, context.CancelFunc) {
+       if reg == nil {
+               reg = prometheus.NewRegistry()
+       }
+       ctx, cancel := context.WithCancel(context.Background())
+       ctx = ctxlog.Context(ctx, ctxlog.TestLogger(t))
+       ks, err := newKeepstore(ctx, cluster, cluster.SystemRootToken, reg, testServiceURL)
+       if err != nil {
+               t.Fatal(err)
+       }
+       return ks, cancel
+}
+
+var _ = Suite(&keepstoreSuite{})
+
+type keepstoreSuite struct {
+       cluster *arvados.Cluster
+}
+
+func (s *keepstoreSuite) SetUpTest(c *C) {
+       s.cluster = testCluster(c)
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "stub"},
+               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "stub"},
+       }
+}
+
+func (s *keepstoreSuite) TestBlockRead_ChecksumMismatch(c *C) {
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+
+       ctx := authContext(arvadostest.ActiveTokenV2)
+
+       fooHash := fmt.Sprintf("%x", md5.Sum([]byte("foo")))
+       err := ks.mountsW[0].BlockWrite(ctx, fooHash, []byte("bar"))
+       c.Assert(err, IsNil)
+
+       _, err = ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+               Hash: fooHash,
+               Data: []byte("foo"),
+       })
+       c.Check(err, ErrorMatches, "hash collision")
+
+       buf := bytes.NewBuffer(nil)
+       _, err = ks.BlockRead(ctx, arvados.BlockReadOptions{
+               Locator: ks.signLocator(arvadostest.ActiveTokenV2, fooHash+"+3"),
+               WriteTo: buf,
+       })
+       c.Check(err, ErrorMatches, "checksum mismatch in stored data")
+       c.Check(buf.String(), Not(Equals), "foo")
+       c.Check(buf.Len() < 3, Equals, true)
+
+       err = ks.mountsW[1].BlockWrite(ctx, fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+
+       buf = bytes.NewBuffer(nil)
+       _, err = ks.BlockRead(ctx, arvados.BlockReadOptions{
+               Locator: ks.signLocator(arvadostest.ActiveTokenV2, fooHash+"+3"),
+               WriteTo: buf,
+       })
+       c.Check(err, ErrorMatches, "checksum mismatch in stored data")
+       c.Check(buf.Len() < 3, Equals, true)
+}
+
+func (s *keepstoreSuite) TestBlockReadWrite_SigningDisabled(c *C) {
+       origKey := s.cluster.Collections.BlobSigningKey
+       s.cluster.Collections.BlobSigning = false
+       s.cluster.Collections.BlobSigningKey = ""
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+
+       resp, err := ks.BlockWrite(authContext("abcde"), arvados.BlockWriteOptions{
+               Hash: fooHash,
+               Data: []byte("foo"),
+       })
+       c.Assert(err, IsNil)
+       c.Check(resp.Locator, Equals, fooHash+"+3")
+       locUnsigned := resp.Locator
+       ttl := time.Hour
+       locSigned := arvados.SignLocator(locUnsigned, arvadostest.ActiveTokenV2, time.Now().Add(ttl), ttl, []byte(origKey))
+       c.Assert(locSigned, Not(Equals), locUnsigned)
+
+       for _, locator := range []string{locUnsigned, locSigned} {
+               for _, token := range []string{"", "xyzzy", arvadostest.ActiveTokenV2} {
+                       c.Logf("=== locator %q token %q", locator, token)
+                       ctx := authContext(token)
+                       buf := bytes.NewBuffer(nil)
+                       _, err := ks.BlockRead(ctx, arvados.BlockReadOptions{
+                               Locator: locator,
+                               WriteTo: buf,
+                       })
+                       c.Check(err, IsNil)
+                       c.Check(buf.String(), Equals, "foo")
+               }
+       }
+}
+
+func (s *keepstoreSuite) TestBlockRead_OrderedByStorageClassPriority(c *C) {
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-111111111111111": {
+                       Driver:         "stub",
+                       Replication:    1,
+                       StorageClasses: map[string]bool{"class1": true}},
+               "zzzzz-nyw5e-222222222222222": {
+                       Driver:         "stub",
+                       Replication:    1,
+                       StorageClasses: map[string]bool{"class2": true, "class3": true}},
+       }
+
+       // "foobar" is just some data that happens to result in
+       // rendezvous order {111, 222}
+       data := []byte("foobar")
+       hash := fmt.Sprintf("%x", md5.Sum(data))
+
+       for _, trial := range []struct {
+               priority1 int // priority of class1, thus vol1
+               priority2 int // priority of class2
+               priority3 int // priority of class3 (vol2 priority will be max(priority2, priority3))
+               expectLog string
+       }{
+               {100, 50, 50, "111 read 385\n"},              // class1 has higher priority => try vol1 first, no need to try vol2
+               {100, 100, 100, "111 read 385\n"},            // same priority, vol2 is first in rendezvous order => try vol1 first and succeed
+               {66, 99, 33, "222 read 385\n111 read 385\n"}, // class2 has higher priority => try vol2 first, then try vol1
+               {66, 33, 99, "222 read 385\n111 read 385\n"}, // class3 has highest priority => vol2 has highest => try vol2 first, then try vol1
+       } {
+               c.Logf("=== %+v", trial)
+
+               s.cluster.StorageClasses = map[string]arvados.StorageClassConfig{
+                       "class1": {Priority: trial.priority1},
+                       "class2": {Priority: trial.priority2},
+                       "class3": {Priority: trial.priority3},
+               }
+               ks, cancel := testKeepstore(c, s.cluster, nil)
+               defer cancel()
+
+               ctx := authContext(arvadostest.ActiveTokenV2)
+               resp, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+                       Hash:           hash,
+                       Data:           data,
+                       StorageClasses: []string{"class1"},
+               })
+               c.Assert(err, IsNil)
+
+               // Combine logs into one. (We only want the logs from
+               // the BlockRead below, not from BlockWrite above.)
+               stubLog := &stubLog{}
+               for _, mnt := range ks.mounts {
+                       mnt.volume.(*stubVolume).stubLog = stubLog
+               }
+
+               n, err := ks.BlockRead(ctx, arvados.BlockReadOptions{
+                       Locator: resp.Locator,
+                       WriteTo: io.Discard,
+               })
+               c.Assert(n, Equals, len(data))
+               c.Assert(err, IsNil)
+               c.Check(stubLog.String(), Equals, trial.expectLog)
+       }
+}
+
+func (s *keepstoreSuite) TestBlockWrite_NoWritableVolumes(c *C) {
+       for uuid, v := range s.cluster.Volumes {
+               v.ReadOnly = true
+               s.cluster.Volumes[uuid] = v
+       }
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+       for _, mnt := range ks.mounts {
+               mnt.volume.(*stubVolume).blockWrite = func(context.Context, string, []byte) error {
+                       c.Error("volume BlockWrite called")
+                       return errors.New("fail")
+               }
+       }
+       ctx := authContext(arvadostest.ActiveTokenV2)
+
+       _, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+               Hash: fooHash,
+               Data: []byte("foo")})
+       c.Check(err, NotNil)
+       c.Check(err.(interface{ HTTPStatus() int }).HTTPStatus(), Equals, http.StatusInsufficientStorage)
+}
+
+func (s *keepstoreSuite) TestBlockWrite_MultipleStorageClasses(c *C) {
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-111111111111111": {
+                       Driver:         "stub",
+                       Replication:    1,
+                       StorageClasses: map[string]bool{"class1": true}},
+               "zzzzz-nyw5e-121212121212121": {
+                       Driver:         "stub",
+                       Replication:    1,
+                       StorageClasses: map[string]bool{"class1": true, "class2": true}},
+               "zzzzz-nyw5e-222222222222222": {
+                       Driver:         "stub",
+                       Replication:    1,
+                       StorageClasses: map[string]bool{"class2": true}},
+       }
+
+       // testData is a block that happens to have rendezvous order 111, 121, 222
+       testData := []byte("qux")
+       testHash := fmt.Sprintf("%x+%d", md5.Sum(testData), len(testData))
+
+       s.cluster.StorageClasses = map[string]arvados.StorageClassConfig{
+               "class1": {},
+               "class2": {},
+               "class3": {},
+       }
+
+       ctx := authContext(arvadostest.ActiveTokenV2)
+       for idx, trial := range []struct {
+               classes   string // desired classes
+               expectLog string
+       }{
+               {"class1", "" +
+                       "111 read d85\n" +
+                       "121 read d85\n" +
+                       "111 write d85\n" +
+                       "111 read d85\n" +
+                       "111 touch d85\n"},
+               {"class2", "" +
+                       "121 read d85\n" + // write#1
+                       "222 read d85\n" +
+                       "121 write d85\n" +
+                       "121 read d85\n" + // write#2
+                       "121 touch d85\n"},
+               {"class1,class2", "" +
+                       "111 read d85\n" + // write#1
+                       "121 read d85\n" +
+                       "222 read d85\n" +
+                       "121 write d85\n" +
+                       "111 write d85\n" +
+                       "111 read d85\n" + // write#2
+                       "111 touch d85\n" +
+                       "121 read d85\n" +
+                       "121 touch d85\n"},
+               {"class1,class2,class404", "" +
+                       "111 read d85\n" + // write#1
+                       "121 read d85\n" +
+                       "222 read d85\n" +
+                       "121 write d85\n" +
+                       "111 write d85\n" +
+                       "111 read d85\n" + // write#2
+                       "111 touch d85\n" +
+                       "121 read d85\n" +
+                       "121 touch d85\n"},
+       } {
+               c.Logf("=== %d: %+v", idx, trial)
+
+               ks, cancel := testKeepstore(c, s.cluster, nil)
+               defer cancel()
+               stubLog := &stubLog{}
+               for _, mnt := range ks.mounts {
+                       mnt.volume.(*stubVolume).stubLog = stubLog
+               }
+
+               // Check that we chose the right block data
+               rvz := ks.rendezvous(testHash, ks.mountsW)
+               c.Assert(rvz[0].UUID[24:], Equals, "111")
+               c.Assert(rvz[1].UUID[24:], Equals, "121")
+               c.Assert(rvz[2].UUID[24:], Equals, "222")
+
+               for i := 0; i < 2; i++ {
+                       _, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+                               Hash:           testHash,
+                               Data:           testData,
+                               StorageClasses: strings.Split(trial.classes, ","),
+                       })
+                       c.Check(err, IsNil)
+               }
+               c.Check(stubLog.String(), Equals, trial.expectLog)
+       }
+}
+
+func (s *keepstoreSuite) TestBlockTrash(c *C) {
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "stub"},
+               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "stub"},
+               "zzzzz-nyw5e-222222222222222": {Replication: 1, Driver: "stub", ReadOnly: true},
+               "zzzzz-nyw5e-333333333333333": {Replication: 1, Driver: "stub", ReadOnly: true, AllowTrashWhenReadOnly: true},
+       }
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+
+       var vol []*stubVolume
+       for _, mount := range ks.mountsR {
+               vol = append(vol, mount.volume.(*stubVolume))
+       }
+       sort.Slice(vol, func(i, j int) bool {
+               return vol[i].params.UUID < vol[j].params.UUID
+       })
+
+       ctx := context.Background()
+       loc := fooHash + "+3"
+       tOld := time.Now().Add(-s.cluster.Collections.BlobSigningTTL.Duration() - time.Second)
+
+       clear := func() {
+               for _, vol := range vol {
+                       err := vol.BlockTrash(fooHash)
+                       if !os.IsNotExist(err) {
+                               c.Assert(err, IsNil)
+                       }
+               }
+       }
+       writeit := func(volidx int) {
+               err := vol[volidx].BlockWrite(ctx, fooHash, []byte("foo"))
+               c.Assert(err, IsNil)
+               err = vol[volidx].blockTouchWithTime(fooHash, tOld)
+               c.Assert(err, IsNil)
+       }
+       trashit := func() error {
+               return ks.BlockTrash(ctx, loc)
+       }
+       checkexists := func(volidx int) bool {
+               _, err := vol[volidx].BlockRead(ctx, fooHash, io.Discard)
+               if !os.IsNotExist(err) {
+                       c.Check(err, IsNil)
+               }
+               return err == nil
+       }
+
+       clear()
+       c.Check(trashit(), Equals, os.ErrNotExist)
+
+       // one old replica => trash it
+       clear()
+       writeit(0)
+       c.Check(trashit(), IsNil)
+       c.Check(checkexists(0), Equals, false)
+
+       // one old replica + one new replica => keep new, trash old
+       clear()
+       writeit(0)
+       writeit(1)
+       c.Check(vol[1].blockTouchWithTime(fooHash, time.Now()), IsNil)
+       c.Check(trashit(), IsNil)
+       c.Check(checkexists(0), Equals, false)
+       c.Check(checkexists(1), Equals, true)
+
+       // two old replicas => trash both
+       clear()
+       writeit(0)
+       writeit(1)
+       c.Check(trashit(), IsNil)
+       c.Check(checkexists(0), Equals, false)
+       c.Check(checkexists(1), Equals, false)
+
+       // four old replicas => trash all except readonly volume with
+       // AllowTrashWhenReadOnly==false
+       clear()
+       writeit(0)
+       writeit(1)
+       writeit(2)
+       writeit(3)
+       c.Check(trashit(), IsNil)
+       c.Check(checkexists(0), Equals, false)
+       c.Check(checkexists(1), Equals, false)
+       c.Check(checkexists(2), Equals, true)
+       c.Check(checkexists(3), Equals, false)
+
+       // two old replicas but one returns an error => return the
+       // only non-404 backend error
+       clear()
+       vol[0].blockTrash = func(hash string) error {
+               return errors.New("fake error")
+       }
+       writeit(0)
+       writeit(3)
+       c.Check(trashit(), ErrorMatches, "fake error")
+       c.Check(checkexists(0), Equals, true)
+       c.Check(checkexists(1), Equals, false)
+       c.Check(checkexists(2), Equals, false)
+       c.Check(checkexists(3), Equals, false)
+}
+
+func (s *keepstoreSuite) TestBlockWrite_OnlyOneBuffer(c *C) {
+       s.cluster.API.MaxKeepBlobBuffers = 1
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+       ok := make(chan struct{})
+       go func() {
+               defer close(ok)
+               ctx := authContext(arvadostest.ActiveTokenV2)
+               _, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+                       Hash: fooHash,
+                       Data: []byte("foo")})
+               c.Check(err, IsNil)
+       }()
+       select {
+       case <-ok:
+       case <-time.After(time.Second):
+               c.Fatal("PUT deadlocks with MaxKeepBlobBuffers==1")
+       }
+}
+
+func (s *keepstoreSuite) TestBufferPoolLeak(c *C) {
+       s.cluster.API.MaxKeepBlobBuffers = 4
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+
+       ctx := authContext(arvadostest.ActiveTokenV2)
+       var wg sync.WaitGroup
+       for range make([]int, 20) {
+               wg.Add(1)
+               go func() {
+                       defer wg.Done()
+                       resp, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+                               Hash: fooHash,
+                               Data: []byte("foo")})
+                       c.Check(err, IsNil)
+                       _, err = ks.BlockRead(ctx, arvados.BlockReadOptions{
+                               Locator: resp.Locator,
+                               WriteTo: io.Discard})
+                       c.Check(err, IsNil)
+               }()
+       }
+       ok := make(chan struct{})
+       go func() {
+               wg.Wait()
+               close(ok)
+       }()
+       select {
+       case <-ok:
+       case <-time.After(time.Second):
+               c.Fatal("read/write sequence deadlocks, likely buffer pool leak")
+       }
+}
+
+func (s *keepstoreSuite) TestPutStorageClasses(c *C) {
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "stub"}, // "default" is implicit
+               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "stub", StorageClasses: map[string]bool{"special": true, "extra": true}},
+               "zzzzz-nyw5e-222222222222222": {Replication: 1, Driver: "stub", StorageClasses: map[string]bool{"readonly": true}, ReadOnly: true},
+       }
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+       ctx := authContext(arvadostest.ActiveTokenV2)
+
+       for _, trial := range []struct {
+               ask            []string
+               expectReplicas int
+               expectClasses  map[string]int
+       }{
+               {nil,
+                       1,
+                       map[string]int{"default": 1}},
+               {[]string{},
+                       1,
+                       map[string]int{"default": 1}},
+               {[]string{"default"},
+                       1,
+                       map[string]int{"default": 1}},
+               {[]string{"default", "default"},
+                       1,
+                       map[string]int{"default": 1}},
+               {[]string{"special"},
+                       1,
+                       map[string]int{"extra": 1, "special": 1}},
+               {[]string{"special", "readonly"},
+                       1,
+                       map[string]int{"extra": 1, "special": 1}},
+               {[]string{"special", "nonexistent"},
+                       1,
+                       map[string]int{"extra": 1, "special": 1}},
+               {[]string{"extra", "special"},
+                       1,
+                       map[string]int{"extra": 1, "special": 1}},
+               {[]string{"default", "special"},
+                       2,
+                       map[string]int{"default": 1, "extra": 1, "special": 1}},
+       } {
+               c.Logf("success case %#v", trial)
+               resp, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+                       Hash:           fooHash,
+                       Data:           []byte("foo"),
+                       StorageClasses: trial.ask,
+               })
+               if !c.Check(err, IsNil) {
+                       continue
+               }
+               c.Check(resp.Replicas, Equals, trial.expectReplicas)
+               if len(trial.expectClasses) == 0 {
+                       // any non-empty value is correct
+                       c.Check(resp.StorageClasses, Not(HasLen), 0)
+               } else {
+                       c.Check(resp.StorageClasses, DeepEquals, trial.expectClasses)
+               }
+       }
+
+       for _, ask := range [][]string{
+               {"doesnotexist"},
+               {"doesnotexist", "readonly"},
+               {"readonly"},
+       } {
+               c.Logf("failure case %s", ask)
+               _, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{
+                       Hash:           fooHash,
+                       Data:           []byte("foo"),
+                       StorageClasses: ask,
+               })
+               c.Check(err, NotNil)
+       }
+}
+
+func (s *keepstoreSuite) TestUntrashHandlerWithNoWritableVolumes(c *C) {
+       for uuid, v := range s.cluster.Volumes {
+               v.ReadOnly = true
+               s.cluster.Volumes[uuid] = v
+       }
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+
+       for _, mnt := range ks.mounts {
+               err := mnt.BlockWrite(context.Background(), fooHash, []byte("foo"))
+               c.Assert(err, IsNil)
+               _, err = mnt.BlockRead(context.Background(), fooHash, io.Discard)
+               c.Assert(err, IsNil)
+       }
+
+       err := ks.BlockUntrash(context.Background(), fooHash)
+       c.Check(os.IsNotExist(err), Equals, true)
+
+       for _, mnt := range ks.mounts {
+               _, err := mnt.BlockRead(context.Background(), fooHash, io.Discard)
+               c.Assert(err, IsNil)
+       }
+}
+
+func (s *keepstoreSuite) TestBlockWrite_SkipReadOnly(c *C) {
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "stub"},
+               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "stub", ReadOnly: true},
+               "zzzzz-nyw5e-222222222222222": {Replication: 1, Driver: "stub", ReadOnly: true, AllowTrashWhenReadOnly: true},
+       }
+       ks, cancel := testKeepstore(c, s.cluster, nil)
+       defer cancel()
+       ctx := authContext(arvadostest.ActiveTokenV2)
+
+       for i := range make([]byte, 32) {
+               data := []byte(fmt.Sprintf("block %d", i))
+               _, err := ks.BlockWrite(ctx, arvados.BlockWriteOptions{Data: data})
+               c.Assert(err, IsNil)
+       }
+       c.Check(ks.mounts["zzzzz-nyw5e-000000000000000"].volume.(*stubVolume).stubLog.String(), Matches, "(?ms).*write.*")
+       c.Check(ks.mounts["zzzzz-nyw5e-111111111111111"].volume.(*stubVolume).stubLog.String(), HasLen, 0)
+       c.Check(ks.mounts["zzzzz-nyw5e-222222222222222"].volume.(*stubVolume).stubLog.String(), HasLen, 0)
+}
+
+func (s *keepstoreSuite) TestParseLocator(c *C) {
+       for _, trial := range []struct {
+               locator string
+               ok      bool
+               expect  locatorInfo
+       }{
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
+                       ok: true},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa+1234",
+                       ok: true, expect: locatorInfo{size: 1234}},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa+1234+Abcdef@abcdef",
+                       ok: true, expect: locatorInfo{size: 1234, signed: true}},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa+1234+Rzzzzz-abcdef",
+                       ok: true, expect: locatorInfo{size: 1234, remote: true}},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa+12345+Zexample+Rzzzzz-abcdef",
+                       ok: true, expect: locatorInfo{size: 12345, remote: true}},
+               // invalid: hash length != 32
+               {locator: "",
+                       ok: false},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
+                       ok: false},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa+1234",
+                       ok: false},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabb",
+                       ok: false},
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabb+1234",
+                       ok: false},
+               // invalid: first hint is not size
+               {locator: "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa+Abcdef+1234",
+                       ok: false},
+       } {
+               c.Logf("=== %s", trial.locator)
+               li, err := parseLocator(trial.locator)
+               if !trial.ok {
+                       c.Check(err, NotNil)
+                       continue
+               }
+               c.Check(err, IsNil)
+               c.Check(li.hash, Equals, trial.locator[:32])
+               c.Check(li.size, Equals, trial.expect.size)
+               c.Check(li.signed, Equals, trial.expect.signed)
+               c.Check(li.remote, Equals, trial.expect.remote)
+       }
+}
+
+func init() {
+       driver["stub"] = func(params newVolumeParams) (volume, error) {
+               v := &stubVolume{
+                       params:  params,
+                       data:    make(map[string]stubData),
+                       stubLog: &stubLog{},
+               }
+               return v, nil
+       }
+}
+
+type stubLog struct {
+       sync.Mutex
+       bytes.Buffer
+}
+
+func (sl *stubLog) Printf(format string, args ...interface{}) {
+       if sl == nil {
+               return
+       }
+       sl.Lock()
+       defer sl.Unlock()
+       fmt.Fprintf(sl, format+"\n", args...)
+}
+
+type stubData struct {
+       mtime time.Time
+       data  []byte
+       trash time.Time
+}
+
+type stubVolume struct {
+       params  newVolumeParams
+       data    map[string]stubData
+       stubLog *stubLog
+       mtx     sync.Mutex
+
+       // The following funcs enable tests to insert delays and
+       // failures. Each volume operation begins by calling the
+       // corresponding func (if non-nil). If the func returns an
+       // error, that error is returned to caller. Otherwise, the
+       // stub continues normally.
+       blockRead    func(ctx context.Context, hash string, writeTo io.Writer) (int, error)
+       blockWrite   func(ctx context.Context, hash string, data []byte) error
+       deviceID     func() string
+       blockTouch   func(hash string) error
+       blockTrash   func(hash string) error
+       blockUntrash func(hash string) error
+       index        func(ctx context.Context, prefix string, writeTo io.Writer) error
+       mtime        func(hash string) (time.Time, error)
+       emptyTrash   func()
+}
+
+func (v *stubVolume) log(op, hash string) {
+       // Note this intentionally crashes if UUID or hash is short --
+       // if keepstore ever does that, tests should fail.
+       v.stubLog.Printf("%s %s %s", v.params.UUID[24:27], op, hash[:3])
+}
+
+func (v *stubVolume) BlockRead(ctx context.Context, hash string, writeTo io.Writer) (int, error) {
+       v.log("read", hash)
+       if v.blockRead != nil {
+               n, err := v.blockRead(ctx, hash, writeTo)
+               if err != nil {
+                       return n, err
+               }
+       }
+       v.mtx.Lock()
+       ent, ok := v.data[hash]
+       v.mtx.Unlock()
+       if !ok || !ent.trash.IsZero() {
+               return 0, os.ErrNotExist
+       }
+       wrote := 0
+       for writesize := 1000; wrote < len(ent.data); writesize = writesize * 2 {
+               data := ent.data[wrote:]
+               if len(data) > writesize {
+                       data = data[:writesize]
+               }
+               n, err := writeTo.Write(data)
+               wrote += n
+               if err != nil {
+                       return wrote, err
+               }
+       }
+       return wrote, nil
+}
+
+func (v *stubVolume) BlockWrite(ctx context.Context, hash string, data []byte) error {
+       v.log("write", hash)
+       if v.blockWrite != nil {
+               if err := v.blockWrite(ctx, hash, data); err != nil {
+                       return err
+               }
+       }
+       v.mtx.Lock()
+       defer v.mtx.Unlock()
+       v.data[hash] = stubData{
+               mtime: time.Now(),
+               data:  append([]byte(nil), data...),
+       }
+       return nil
+}
+
+func (v *stubVolume) DeviceID() string {
+       return fmt.Sprintf("%p", v)
+}
+
+func (v *stubVolume) BlockTouch(hash string) error {
+       v.log("touch", hash)
+       if v.blockTouch != nil {
+               if err := v.blockTouch(hash); err != nil {
+                       return err
+               }
+       }
+       v.mtx.Lock()
+       defer v.mtx.Unlock()
+       ent, ok := v.data[hash]
+       if !ok || !ent.trash.IsZero() {
+               return os.ErrNotExist
+       }
+       ent.mtime = time.Now()
+       v.data[hash] = ent
+       return nil
+}
+
+// Set mtime to the (presumably old) specified time.
+func (v *stubVolume) blockTouchWithTime(hash string, t time.Time) error {
+       v.log("touchwithtime", hash)
+       v.mtx.Lock()
+       defer v.mtx.Unlock()
+       ent, ok := v.data[hash]
+       if !ok {
+               return os.ErrNotExist
+       }
+       ent.mtime = t
+       v.data[hash] = ent
+       return nil
+}
+
+func (v *stubVolume) BlockTrash(hash string) error {
+       v.log("trash", hash)
+       if v.blockTrash != nil {
+               if err := v.blockTrash(hash); err != nil {
+                       return err
+               }
+       }
+       v.mtx.Lock()
+       defer v.mtx.Unlock()
+       ent, ok := v.data[hash]
+       if !ok || !ent.trash.IsZero() {
+               return os.ErrNotExist
+       }
+       ent.trash = time.Now().Add(v.params.Cluster.Collections.BlobTrashLifetime.Duration())
+       v.data[hash] = ent
+       return nil
+}
+
+func (v *stubVolume) BlockUntrash(hash string) error {
+       v.log("untrash", hash)
+       if v.blockUntrash != nil {
+               if err := v.blockUntrash(hash); err != nil {
+                       return err
+               }
+       }
+       v.mtx.Lock()
+       defer v.mtx.Unlock()
+       ent, ok := v.data[hash]
+       if !ok || ent.trash.IsZero() {
+               return os.ErrNotExist
+       }
+       ent.trash = time.Time{}
+       v.data[hash] = ent
+       return nil
+}
+
+func (v *stubVolume) Index(ctx context.Context, prefix string, writeTo io.Writer) error {
+       v.stubLog.Printf("%s index %s", v.params.UUID, prefix)
+       if v.index != nil {
+               if err := v.index(ctx, prefix, writeTo); err != nil {
+                       return err
+               }
+       }
+       buf := &bytes.Buffer{}
+       v.mtx.Lock()
+       for hash, ent := range v.data {
+               if ent.trash.IsZero() && strings.HasPrefix(hash, prefix) {
+                       fmt.Fprintf(buf, "%s+%d %d\n", hash, len(ent.data), ent.mtime.UnixNano())
+               }
+       }
+       v.mtx.Unlock()
+       _, err := io.Copy(writeTo, buf)
+       return err
+}
+
+func (v *stubVolume) Mtime(hash string) (time.Time, error) {
+       v.log("mtime", hash)
+       if v.mtime != nil {
+               if t, err := v.mtime(hash); err != nil {
+                       return t, err
+               }
+       }
+       v.mtx.Lock()
+       defer v.mtx.Unlock()
+       ent, ok := v.data[hash]
+       if !ok || !ent.trash.IsZero() {
+               return time.Time{}, os.ErrNotExist
+       }
+       return ent.mtime, nil
+}
+
+func (v *stubVolume) EmptyTrash() {
+       v.stubLog.Printf("%s emptytrash", v.params.UUID)
+       v.mtx.Lock()
+       defer v.mtx.Unlock()
+       for hash, ent := range v.data {
+               if !ent.trash.IsZero() && time.Now().After(ent.trash) {
+                       delete(v.data, hash)
+               }
+       }
+}
index d04601fbec84128ff47cf65ea15588aa6212b9c5..4638de544482e18721a6eb9b714f22fdc17a9dba 100644 (file)
@@ -5,66 +5,9 @@
 package keepstore
 
 import (
-       "fmt"
-
        "github.com/prometheus/client_golang/prometheus"
 )
 
-type nodeMetrics struct {
-       reg *prometheus.Registry
-}
-
-func (m *nodeMetrics) setupBufferPoolMetrics(b *bufferPool) {
-       m.reg.MustRegister(prometheus.NewGaugeFunc(
-               prometheus.GaugeOpts{
-                       Namespace: "arvados",
-                       Subsystem: "keepstore",
-                       Name:      "bufferpool_allocated_bytes",
-                       Help:      "Number of bytes allocated to buffers",
-               },
-               func() float64 { return float64(b.Alloc()) },
-       ))
-       m.reg.MustRegister(prometheus.NewGaugeFunc(
-               prometheus.GaugeOpts{
-                       Namespace: "arvados",
-                       Subsystem: "keepstore",
-                       Name:      "bufferpool_max_buffers",
-                       Help:      "Maximum number of buffers allowed",
-               },
-               func() float64 { return float64(b.Cap()) },
-       ))
-       m.reg.MustRegister(prometheus.NewGaugeFunc(
-               prometheus.GaugeOpts{
-                       Namespace: "arvados",
-                       Subsystem: "keepstore",
-                       Name:      "bufferpool_inuse_buffers",
-                       Help:      "Number of buffers in use",
-               },
-               func() float64 { return float64(b.Len()) },
-       ))
-}
-
-func (m *nodeMetrics) setupWorkQueueMetrics(q *WorkQueue, qName string) {
-       m.reg.MustRegister(prometheus.NewGaugeFunc(
-               prometheus.GaugeOpts{
-                       Namespace: "arvados",
-                       Subsystem: "keepstore",
-                       Name:      fmt.Sprintf("%s_queue_inprogress_entries", qName),
-                       Help:      fmt.Sprintf("Number of %s requests in progress", qName),
-               },
-               func() float64 { return float64(getWorkQueueStatus(q).InProgress) },
-       ))
-       m.reg.MustRegister(prometheus.NewGaugeFunc(
-               prometheus.GaugeOpts{
-                       Namespace: "arvados",
-                       Subsystem: "keepstore",
-                       Name:      fmt.Sprintf("%s_queue_pending_entries", qName),
-                       Help:      fmt.Sprintf("Number of queued %s requests", qName),
-               },
-               func() float64 { return float64(getWorkQueueStatus(q).Queued) },
-       ))
-}
-
 type volumeMetricsVecs struct {
        ioBytes     *prometheus.CounterVec
        errCounters *prometheus.CounterVec
diff --git a/services/keepstore/metrics_test.go b/services/keepstore/metrics_test.go
new file mode 100644 (file)
index 0000000..0c8f1e6
--- /dev/null
@@ -0,0 +1,87 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "context"
+       "encoding/json"
+       "net/http"
+
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/arvadostest"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "git.arvados.org/arvados.git/sdk/go/httpserver"
+       "github.com/prometheus/client_golang/prometheus"
+       . "gopkg.in/check.v1"
+)
+
+func (s *routerSuite) TestMetrics(c *C) {
+       reg := prometheus.NewRegistry()
+       router, cancel := testRouter(c, s.cluster, reg)
+       defer cancel()
+       instrumented := httpserver.Instrument(reg, ctxlog.TestLogger(c), router)
+       handler := instrumented.ServeAPI(s.cluster.ManagementToken, instrumented)
+
+       router.keepstore.BlockWrite(context.Background(), arvados.BlockWriteOptions{
+               Hash: fooHash,
+               Data: []byte("foo"),
+       })
+       router.keepstore.BlockWrite(context.Background(), arvados.BlockWriteOptions{
+               Hash: barHash,
+               Data: []byte("bar"),
+       })
+
+       // prime the metrics by doing a no-op request
+       resp := call(handler, "GET", "/", "", nil, nil)
+
+       resp = call(handler, "GET", "/metrics.json", "", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusUnauthorized)
+       resp = call(handler, "GET", "/metrics.json", "foobar", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusForbidden)
+       resp = call(handler, "GET", "/metrics.json", arvadostest.ManagementToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       var j []struct {
+               Name   string
+               Help   string
+               Type   string
+               Metric []struct {
+                       Label []struct {
+                               Name  string
+                               Value string
+                       }
+                       Summary struct {
+                               SampleCount string
+                               SampleSum   float64
+                       }
+               }
+       }
+       json.NewDecoder(resp.Body).Decode(&j)
+       found := make(map[string]bool)
+       names := map[string]bool{}
+       for _, g := range j {
+               names[g.Name] = true
+               for _, m := range g.Metric {
+                       if len(m.Label) == 2 && m.Label[0].Name == "code" && m.Label[0].Value == "200" && m.Label[1].Name == "method" && m.Label[1].Value == "put" {
+                               c.Check(m.Summary.SampleCount, Equals, "2")
+                               found[g.Name] = true
+                       }
+               }
+       }
+
+       metricsNames := []string{
+               "arvados_keepstore_bufferpool_inuse_buffers",
+               "arvados_keepstore_bufferpool_max_buffers",
+               "arvados_keepstore_bufferpool_allocated_bytes",
+               "arvados_keepstore_pull_queue_inprogress_entries",
+               "arvados_keepstore_pull_queue_pending_entries",
+               "arvados_keepstore_trash_queue_inprogress_entries",
+               "arvados_keepstore_trash_queue_pending_entries",
+               "request_duration_seconds",
+       }
+       for _, m := range metricsNames {
+               _, ok := names[m]
+               c.Check(ok, Equals, true, Commentf("checking metric %q", m))
+       }
+}
diff --git a/services/keepstore/mock_mutex_for_test.go b/services/keepstore/mock_mutex_for_test.go
deleted file mode 100644 (file)
index daf0ef0..0000000
+++ /dev/null
@@ -1,27 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-type MockMutex struct {
-       AllowLock   chan struct{}
-       AllowUnlock chan struct{}
-}
-
-func NewMockMutex() *MockMutex {
-       return &MockMutex{
-               AllowLock:   make(chan struct{}),
-               AllowUnlock: make(chan struct{}),
-       }
-}
-
-// Lock waits for someone to send to AllowLock.
-func (m *MockMutex) Lock() {
-       <-m.AllowLock
-}
-
-// Unlock waits for someone to send to AllowUnlock.
-func (m *MockMutex) Unlock() {
-       <-m.AllowUnlock
-}
index e8c248219f77785458110107922983b0917fa51d..d29d5f6dc048e86e76d1498c80e96bb4f9b058e9 100644 (file)
@@ -5,28 +5,24 @@
 package keepstore
 
 import (
-       "bytes"
        "context"
        "encoding/json"
        "net/http"
-       "net/http/httptest"
 
-       "git.arvados.org/arvados.git/sdk/go/arvadostest"
-       "git.arvados.org/arvados.git/sdk/go/ctxlog"
-       "git.arvados.org/arvados.git/sdk/go/httpserver"
-       "github.com/prometheus/client_golang/prometheus"
-       check "gopkg.in/check.v1"
+       . "gopkg.in/check.v1"
 )
 
-func (s *HandlerSuite) TestMounts(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
+func (s *routerSuite) TestMounts(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
 
-       vols := s.handler.volmgr.AllWritable()
-       vols[0].Put(context.Background(), TestHash, TestBlock)
-       vols[1].Put(context.Background(), TestHash2, TestBlock2)
+       router.keepstore.mountsW[0].BlockWrite(context.Background(), fooHash, []byte("foo"))
+       router.keepstore.mountsW[1].BlockWrite(context.Background(), barHash, []byte("bar"))
+
+       resp := call(router, "GET", "/mounts", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Log(resp.Body.String())
 
-       resp := s.call("GET", "/mounts", "", nil)
-       c.Check(resp.Code, check.Equals, http.StatusOK)
        var mntList []struct {
                UUID           string          `json:"uuid"`
                DeviceID       string          `json:"device_id"`
@@ -34,119 +30,56 @@ func (s *HandlerSuite) TestMounts(c *check.C) {
                Replication    int             `json:"replication"`
                StorageClasses map[string]bool `json:"storage_classes"`
        }
-       c.Log(resp.Body.String())
        err := json.Unmarshal(resp.Body.Bytes(), &mntList)
-       c.Assert(err, check.IsNil)
-       c.Assert(len(mntList), check.Equals, 2)
+       c.Assert(err, IsNil)
+       c.Assert(mntList, HasLen, 2)
+
        for _, m := range mntList {
-               c.Check(len(m.UUID), check.Equals, 27)
-               c.Check(m.UUID[:12], check.Equals, "zzzzz-nyw5e-")
-               c.Check(m.DeviceID, check.Equals, "mock-device-id")
-               c.Check(m.ReadOnly, check.Equals, false)
-               c.Check(m.Replication, check.Equals, 1)
-               c.Check(m.StorageClasses, check.DeepEquals, map[string]bool{"default": true})
+               c.Check(len(m.UUID), Equals, 27)
+               c.Check(m.UUID[:12], Equals, "zzzzz-nyw5e-")
+               c.Check(m.DeviceID, Matches, "0x[0-9a-f]+")
+               c.Check(m.ReadOnly, Equals, false)
+               c.Check(m.Replication, Equals, 1)
+               c.Check(m.StorageClasses, HasLen, 1)
+               for k := range m.StorageClasses {
+                       c.Check(k, Matches, "testclass.*")
+               }
        }
-       c.Check(mntList[0].UUID, check.Not(check.Equals), mntList[1].UUID)
+       c.Check(mntList[0].UUID, Not(Equals), mntList[1].UUID)
 
-       // Bad auth
+       c.Logf("=== bad auth")
        for _, tok := range []string{"", "xyzzy"} {
-               resp = s.call("GET", "/mounts/"+mntList[1].UUID+"/blocks", tok, nil)
-               c.Check(resp.Code, check.Equals, http.StatusUnauthorized)
-               c.Check(resp.Body.String(), check.Equals, "Unauthorized\n")
-       }
-
-       tok := arvadostest.SystemRootToken
-
-       // Nonexistent mount UUID
-       resp = s.call("GET", "/mounts/X/blocks", tok, nil)
-       c.Check(resp.Code, check.Equals, http.StatusNotFound)
-       c.Check(resp.Body.String(), check.Equals, "mount not found\n")
-
-       // Complete index of first mount
-       resp = s.call("GET", "/mounts/"+mntList[0].UUID+"/blocks", tok, nil)
-       c.Check(resp.Code, check.Equals, http.StatusOK)
-       c.Check(resp.Body.String(), check.Matches, TestHash+`\+[0-9]+ [0-9]+\n\n`)
-
-       // Partial index of first mount (one block matches prefix)
-       resp = s.call("GET", "/mounts/"+mntList[0].UUID+"/blocks?prefix="+TestHash[:2], tok, nil)
-       c.Check(resp.Code, check.Equals, http.StatusOK)
-       c.Check(resp.Body.String(), check.Matches, TestHash+`\+[0-9]+ [0-9]+\n\n`)
-
-       // Complete index of second mount (note trailing slash)
-       resp = s.call("GET", "/mounts/"+mntList[1].UUID+"/blocks/", tok, nil)
-       c.Check(resp.Code, check.Equals, http.StatusOK)
-       c.Check(resp.Body.String(), check.Matches, TestHash2+`\+[0-9]+ [0-9]+\n\n`)
-
-       // Partial index of second mount (no blocks match prefix)
-       resp = s.call("GET", "/mounts/"+mntList[1].UUID+"/blocks/?prefix="+TestHash[:2], tok, nil)
-       c.Check(resp.Code, check.Equals, http.StatusOK)
-       c.Check(resp.Body.String(), check.Equals, "\n")
-}
-
-func (s *HandlerSuite) TestMetrics(c *check.C) {
-       reg := prometheus.NewRegistry()
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", reg, testServiceURL), check.IsNil)
-       instrumented := httpserver.Instrument(reg, ctxlog.TestLogger(c), s.handler.Handler)
-       s.handler.Handler = instrumented.ServeAPI(s.cluster.ManagementToken, instrumented)
-
-       s.call("PUT", "/"+TestHash, "", TestBlock)
-       s.call("PUT", "/"+TestHash2, "", TestBlock2)
-       resp := s.call("GET", "/metrics.json", "", nil)
-       c.Check(resp.Code, check.Equals, http.StatusUnauthorized)
-       resp = s.call("GET", "/metrics.json", "foobar", nil)
-       c.Check(resp.Code, check.Equals, http.StatusForbidden)
-       resp = s.call("GET", "/metrics.json", arvadostest.ManagementToken, nil)
-       c.Check(resp.Code, check.Equals, http.StatusOK)
-       var j []struct {
-               Name   string
-               Help   string
-               Type   string
-               Metric []struct {
-                       Label []struct {
-                               Name  string
-                               Value string
-                       }
-                       Summary struct {
-                               SampleCount string
-                               SampleSum   float64
-                       }
-               }
-       }
-       json.NewDecoder(resp.Body).Decode(&j)
-       found := make(map[string]bool)
-       names := map[string]bool{}
-       for _, g := range j {
-               names[g.Name] = true
-               for _, m := range g.Metric {
-                       if len(m.Label) == 2 && m.Label[0].Name == "code" && m.Label[0].Value == "200" && m.Label[1].Name == "method" && m.Label[1].Value == "put" {
-                               c.Check(m.Summary.SampleCount, check.Equals, "2")
-                               found[g.Name] = true
-                       }
+               resp = call(router, "GET", "/mounts/"+mntList[1].UUID+"/blocks", tok, nil, nil)
+               if tok == "" {
+                       c.Check(resp.Code, Equals, http.StatusUnauthorized)
+                       c.Check(resp.Body.String(), Equals, "Unauthorized\n")
+               } else {
+                       c.Check(resp.Code, Equals, http.StatusForbidden)
+                       c.Check(resp.Body.String(), Equals, "Forbidden\n")
                }
        }
 
-       metricsNames := []string{
-               "arvados_keepstore_bufferpool_inuse_buffers",
-               "arvados_keepstore_bufferpool_max_buffers",
-               "arvados_keepstore_bufferpool_allocated_bytes",
-               "arvados_keepstore_pull_queue_inprogress_entries",
-               "arvados_keepstore_pull_queue_pending_entries",
-               "arvados_keepstore_trash_queue_inprogress_entries",
-               "arvados_keepstore_trash_queue_pending_entries",
-               "request_duration_seconds",
-       }
-       for _, m := range metricsNames {
-               _, ok := names[m]
-               c.Check(ok, check.Equals, true, check.Commentf("checking metric %q", m))
-       }
-}
-
-func (s *HandlerSuite) call(method, path, tok string, body []byte) *httptest.ResponseRecorder {
-       resp := httptest.NewRecorder()
-       req, _ := http.NewRequest(method, path, bytes.NewReader(body))
-       if tok != "" {
-               req.Header.Set("Authorization", "Bearer "+tok)
-       }
-       s.handler.ServeHTTP(resp, req)
-       return resp
+       c.Logf("=== nonexistent mount UUID")
+       resp = call(router, "GET", "/mounts/X/blocks", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusNotFound)
+
+       c.Logf("=== complete index of first mount")
+       resp = call(router, "GET", "/mounts/"+mntList[0].UUID+"/blocks", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Matches, fooHash+`\+[0-9]+ [0-9]+\n\n`)
+
+       c.Logf("=== partial index of first mount (one block matches prefix)")
+       resp = call(router, "GET", "/mounts/"+mntList[0].UUID+"/blocks?prefix="+fooHash[:2], s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Matches, fooHash+`\+[0-9]+ [0-9]+\n\n`)
+
+       c.Logf("=== complete index of second mount (note trailing slash)")
+       resp = call(router, "GET", "/mounts/"+mntList[1].UUID+"/blocks/", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Matches, barHash+`\+[0-9]+ [0-9]+\n\n`)
+
+       c.Logf("=== partial index of second mount (no blocks match prefix)")
+       resp = call(router, "GET", "/mounts/"+mntList[1].UUID+"/blocks/?prefix="+fooHash[:2], s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Equals, "\n")
 }
diff --git a/services/keepstore/perms.go b/services/keepstore/perms.go
deleted file mode 100644 (file)
index 7205a45..0000000
+++ /dev/null
@@ -1,33 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "time"
-
-       "git.arvados.org/arvados.git/sdk/go/arvados"
-       "git.arvados.org/arvados.git/sdk/go/keepclient"
-)
-
-// SignLocator takes a blobLocator, an apiToken and an expiry time, and
-// returns a signed locator string.
-func SignLocator(cluster *arvados.Cluster, blobLocator, apiToken string, expiry time.Time) string {
-       return keepclient.SignLocator(blobLocator, apiToken, expiry, cluster.Collections.BlobSigningTTL.Duration(), []byte(cluster.Collections.BlobSigningKey))
-}
-
-// VerifySignature returns nil if the signature on the signedLocator
-// can be verified using the given apiToken. Otherwise it returns
-// either ExpiredError (if the timestamp has expired, which is
-// something the client could have figured out independently) or
-// PermissionError.
-func VerifySignature(cluster *arvados.Cluster, signedLocator, apiToken string) error {
-       err := keepclient.VerifySignature(signedLocator, apiToken, cluster.Collections.BlobSigningTTL.Duration(), []byte(cluster.Collections.BlobSigningKey))
-       if err == keepclient.ErrSignatureExpired {
-               return ExpiredError
-       } else if err != nil {
-               return PermissionError
-       }
-       return nil
-}
diff --git a/services/keepstore/perms_test.go b/services/keepstore/perms_test.go
deleted file mode 100644 (file)
index 1322374..0000000
+++ /dev/null
@@ -1,63 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "strconv"
-       "time"
-
-       "git.arvados.org/arvados.git/sdk/go/arvados"
-       check "gopkg.in/check.v1"
-)
-
-const (
-       knownHash    = "acbd18db4cc2f85cedef654fccc4a4d8"
-       knownLocator = knownHash + "+3"
-       knownToken   = "hocfupkn2pjhrpgp2vxv8rsku7tvtx49arbc9s4bvu7p7wxqvk"
-       knownKey     = "13u9fkuccnboeewr0ne3mvapk28epf68a3bhj9q8sb4l6e4e5mkk" +
-               "p6nhj2mmpscgu1zze5h5enydxfe3j215024u16ij4hjaiqs5u4pzsl3nczmaoxnc" +
-               "ljkm4875xqn4xv058koz3vkptmzhyheiy6wzevzjmdvxhvcqsvr5abhl15c2d4o4" +
-               "jhl0s91lojy1mtrzqqvprqcverls0xvy9vai9t1l1lvvazpuadafm71jl4mrwq2y" +
-               "gokee3eamvjy8qq1fvy238838enjmy5wzy2md7yvsitp5vztft6j4q866efym7e6" +
-               "vu5wm9fpnwjyxfldw3vbo01mgjs75rgo7qioh8z8ij7jpyp8508okhgbbex3ceei" +
-               "786u5rw2a9gx743dj3fgq2irk"
-       knownSignatureTTL  = arvados.Duration(24 * 14 * time.Hour)
-       knownSignature     = "89118b78732c33104a4d6231e8b5a5fa1e4301e3"
-       knownTimestamp     = "7fffffff"
-       knownSigHint       = "+A" + knownSignature + "@" + knownTimestamp
-       knownSignedLocator = knownLocator + knownSigHint
-)
-
-func (s *HandlerSuite) TestSignLocator(c *check.C) {
-       tsInt, err := strconv.ParseInt(knownTimestamp, 16, 0)
-       if err != nil {
-               c.Fatal(err)
-       }
-       t0 := time.Unix(tsInt, 0)
-
-       s.cluster.Collections.BlobSigningTTL = knownSignatureTTL
-       s.cluster.Collections.BlobSigningKey = knownKey
-       if x := SignLocator(s.cluster, knownLocator, knownToken, t0); x != knownSignedLocator {
-               c.Fatalf("Got %+q, expected %+q", x, knownSignedLocator)
-       }
-
-       s.cluster.Collections.BlobSigningKey = "arbitrarykey"
-       if x := SignLocator(s.cluster, knownLocator, knownToken, t0); x == knownSignedLocator {
-               c.Fatalf("Got same signature %+q, even though blobSigningKey changed", x)
-       }
-}
-
-func (s *HandlerSuite) TestVerifyLocator(c *check.C) {
-       s.cluster.Collections.BlobSigningTTL = knownSignatureTTL
-       s.cluster.Collections.BlobSigningKey = knownKey
-       if err := VerifySignature(s.cluster, knownSignedLocator, knownToken); err != nil {
-               c.Fatal(err)
-       }
-
-       s.cluster.Collections.BlobSigningKey = "arbitrarykey"
-       if err := VerifySignature(s.cluster, knownSignedLocator, knownToken); err == nil {
-               c.Fatal("Verified signature even with wrong blobSigningKey")
-       }
-}
diff --git a/services/keepstore/pipe_adapters.go b/services/keepstore/pipe_adapters.go
deleted file mode 100644 (file)
index 6b55505..0000000
+++ /dev/null
@@ -1,93 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "bytes"
-       "context"
-       "io"
-       "io/ioutil"
-)
-
-// getWithPipe invokes getter and copies the resulting data into
-// buf. If ctx is done before all data is copied, getWithPipe closes
-// the pipe with an error, and returns early with an error.
-func getWithPipe(ctx context.Context, loc string, buf []byte, br BlockReader) (int, error) {
-       piper, pipew := io.Pipe()
-       go func() {
-               pipew.CloseWithError(br.ReadBlock(ctx, loc, pipew))
-       }()
-       done := make(chan struct{})
-       var size int
-       var err error
-       go func() {
-               size, err = io.ReadFull(piper, buf)
-               if err == io.EOF || err == io.ErrUnexpectedEOF {
-                       err = nil
-               }
-               close(done)
-       }()
-       select {
-       case <-ctx.Done():
-               piper.CloseWithError(ctx.Err())
-               return 0, ctx.Err()
-       case <-done:
-               piper.Close()
-               return size, err
-       }
-}
-
-// putWithPipe invokes putter with a new pipe, and copies data
-// from buf into the pipe. If ctx is done before all data is copied,
-// putWithPipe closes the pipe with an error, and returns early with
-// an error.
-func putWithPipe(ctx context.Context, loc string, buf []byte, bw BlockWriter) error {
-       piper, pipew := io.Pipe()
-       copyErr := make(chan error)
-       go func() {
-               _, err := io.Copy(pipew, bytes.NewReader(buf))
-               copyErr <- err
-               close(copyErr)
-       }()
-
-       putErr := make(chan error, 1)
-       go func() {
-               putErr <- bw.WriteBlock(ctx, loc, piper)
-               close(putErr)
-       }()
-
-       var err error
-       select {
-       case err = <-copyErr:
-       case err = <-putErr:
-       case <-ctx.Done():
-               err = ctx.Err()
-       }
-
-       // Ensure io.Copy goroutine isn't blocked writing to pipew
-       // (otherwise, io.Copy is still using buf so it isn't safe to
-       // return). This can cause pipew to receive corrupt data if
-       // err came from copyErr or ctx.Done() before the copy
-       // finished. That's OK, though: in that case err != nil, and
-       // CloseWithErr(err) ensures putter() will get an error from
-       // piper.Read() before seeing EOF.
-       go pipew.CloseWithError(err)
-       go io.Copy(ioutil.Discard, piper)
-       <-copyErr
-
-       // Note: io.Copy() is finished now, but putter() might still
-       // be running. If we encounter an error before putter()
-       // returns, we return right away without waiting for putter().
-
-       if err != nil {
-               return err
-       }
-       select {
-       case <-ctx.Done():
-               return ctx.Err()
-       case err = <-putErr:
-               return err
-       }
-}
diff --git a/services/keepstore/proxy_remote.go b/services/keepstore/proxy_remote.go
deleted file mode 100644 (file)
index 325f1cf..0000000
+++ /dev/null
@@ -1,212 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "context"
-       "errors"
-       "io"
-       "net/http"
-       "regexp"
-       "strings"
-       "sync"
-       "time"
-
-       "git.arvados.org/arvados.git/sdk/go/arvados"
-       "git.arvados.org/arvados.git/sdk/go/arvadosclient"
-       "git.arvados.org/arvados.git/sdk/go/auth"
-       "git.arvados.org/arvados.git/sdk/go/keepclient"
-)
-
-type remoteProxy struct {
-       clients map[string]*keepclient.KeepClient
-       mtx     sync.Mutex
-}
-
-func (rp *remoteProxy) Get(ctx context.Context, w http.ResponseWriter, r *http.Request, cluster *arvados.Cluster, volmgr *RRVolumeManager) {
-       // Intervening proxies must not return a cached GET response
-       // to a prior request if a X-Keep-Signature request header has
-       // been added or changed.
-       w.Header().Add("Vary", "X-Keep-Signature")
-
-       token := GetAPIToken(r)
-       if token == "" {
-               http.Error(w, "no token provided in Authorization header", http.StatusUnauthorized)
-               return
-       }
-       if strings.SplitN(r.Header.Get("X-Keep-Signature"), ",", 2)[0] == "local" {
-               buf, err := getBufferWithContext(ctx, bufs, BlockSize)
-               if err != nil {
-                       http.Error(w, err.Error(), http.StatusServiceUnavailable)
-                       return
-               }
-               defer bufs.Put(buf)
-               rrc := &remoteResponseCacher{
-                       Locator:        r.URL.Path[1:],
-                       Token:          token,
-                       Buffer:         buf[:0],
-                       ResponseWriter: w,
-                       Context:        ctx,
-                       Cluster:        cluster,
-                       VolumeManager:  volmgr,
-               }
-               defer rrc.Close()
-               w = rrc
-       }
-       var remoteClient *keepclient.KeepClient
-       var parts []string
-       for i, part := range strings.Split(r.URL.Path[1:], "+") {
-               switch {
-               case i == 0:
-                       // don't try to parse hash part as hint
-               case strings.HasPrefix(part, "A"):
-                       // drop local permission hint
-                       continue
-               case len(part) > 7 && part[0] == 'R' && part[6] == '-':
-                       remoteID := part[1:6]
-                       remote, ok := cluster.RemoteClusters[remoteID]
-                       if !ok {
-                               http.Error(w, "remote cluster not configured", http.StatusBadRequest)
-                               return
-                       }
-                       kc, err := rp.remoteClient(remoteID, remote, token)
-                       if err == auth.ErrObsoleteToken {
-                               http.Error(w, err.Error(), http.StatusBadRequest)
-                               return
-                       } else if err != nil {
-                               http.Error(w, err.Error(), http.StatusInternalServerError)
-                               return
-                       }
-                       remoteClient = kc
-                       part = "A" + part[7:]
-               }
-               parts = append(parts, part)
-       }
-       if remoteClient == nil {
-               http.Error(w, "bad request", http.StatusBadRequest)
-               return
-       }
-       locator := strings.Join(parts, "+")
-       rdr, _, _, err := remoteClient.Get(locator)
-       switch err.(type) {
-       case nil:
-               defer rdr.Close()
-               io.Copy(w, rdr)
-       case *keepclient.ErrNotFound:
-               http.Error(w, err.Error(), http.StatusNotFound)
-       default:
-               http.Error(w, err.Error(), http.StatusBadGateway)
-       }
-}
-
-func (rp *remoteProxy) remoteClient(remoteID string, remoteCluster arvados.RemoteCluster, token string) (*keepclient.KeepClient, error) {
-       rp.mtx.Lock()
-       kc, ok := rp.clients[remoteID]
-       rp.mtx.Unlock()
-       if !ok {
-               c := &arvados.Client{
-                       APIHost:   remoteCluster.Host,
-                       AuthToken: "xxx",
-                       Insecure:  remoteCluster.Insecure,
-               }
-               ac, err := arvadosclient.New(c)
-               if err != nil {
-                       return nil, err
-               }
-               kc, err = keepclient.MakeKeepClient(ac)
-               if err != nil {
-                       return nil, err
-               }
-               kc.DiskCacheSize = keepclient.DiskCacheDisabled
-
-               rp.mtx.Lock()
-               if rp.clients == nil {
-                       rp.clients = map[string]*keepclient.KeepClient{remoteID: kc}
-               } else {
-                       rp.clients[remoteID] = kc
-               }
-               rp.mtx.Unlock()
-       }
-       accopy := *kc.Arvados
-       accopy.ApiToken = token
-       kccopy := kc.Clone()
-       kccopy.Arvados = &accopy
-       token, err := auth.SaltToken(token, remoteID)
-       if err != nil {
-               return nil, err
-       }
-       kccopy.Arvados.ApiToken = token
-       return kccopy, nil
-}
-
-var localOrRemoteSignature = regexp.MustCompile(`\+[AR][^\+]*`)
-
-// remoteResponseCacher wraps http.ResponseWriter. It buffers the
-// response data in the provided buffer, writes/touches a copy on a
-// local volume, adds a response header with a locally-signed locator,
-// and finally writes the data through.
-type remoteResponseCacher struct {
-       Locator       string
-       Token         string
-       Buffer        []byte
-       Context       context.Context
-       Cluster       *arvados.Cluster
-       VolumeManager *RRVolumeManager
-       http.ResponseWriter
-       statusCode int
-}
-
-func (rrc *remoteResponseCacher) Write(p []byte) (int, error) {
-       if len(rrc.Buffer)+len(p) > cap(rrc.Buffer) {
-               return 0, errors.New("buffer full")
-       }
-       rrc.Buffer = append(rrc.Buffer, p...)
-       return len(p), nil
-}
-
-func (rrc *remoteResponseCacher) WriteHeader(statusCode int) {
-       rrc.statusCode = statusCode
-}
-
-func (rrc *remoteResponseCacher) Close() error {
-       if rrc.statusCode == 0 {
-               rrc.statusCode = http.StatusOK
-       } else if rrc.statusCode != http.StatusOK {
-               rrc.ResponseWriter.WriteHeader(rrc.statusCode)
-               rrc.ResponseWriter.Write(rrc.Buffer)
-               return nil
-       }
-       _, err := PutBlock(rrc.Context, rrc.VolumeManager, rrc.Buffer, rrc.Locator[:32], nil)
-       if rrc.Context.Err() != nil {
-               // If caller hung up, log that instead of subsequent/misleading errors.
-               http.Error(rrc.ResponseWriter, rrc.Context.Err().Error(), http.StatusGatewayTimeout)
-               return err
-       }
-       if err == RequestHashError {
-               http.Error(rrc.ResponseWriter, "checksum mismatch in remote response", http.StatusBadGateway)
-               return err
-       }
-       if err, ok := err.(*KeepError); ok {
-               http.Error(rrc.ResponseWriter, err.Error(), err.HTTPCode)
-               return err
-       }
-       if err != nil {
-               http.Error(rrc.ResponseWriter, err.Error(), http.StatusBadGateway)
-               return err
-       }
-
-       unsigned := localOrRemoteSignature.ReplaceAllLiteralString(rrc.Locator, "")
-       expiry := time.Now().Add(rrc.Cluster.Collections.BlobSigningTTL.Duration())
-       signed := SignLocator(rrc.Cluster, unsigned, rrc.Token, expiry)
-       if signed == unsigned {
-               err = errors.New("could not sign locator")
-               http.Error(rrc.ResponseWriter, err.Error(), http.StatusInternalServerError)
-               return err
-       }
-       rrc.Header().Set("X-Keep-Locator", signed)
-       rrc.ResponseWriter.WriteHeader(rrc.statusCode)
-       _, err = rrc.ResponseWriter.Write(rrc.Buffer)
-       return err
-}
index 534371cc0ece83ef3a0cead670d1612ec8f57172..886754e14a422d226ccc34c316a608a10bf36f27 100644 (file)
@@ -5,7 +5,6 @@
 package keepstore
 
 import (
-       "context"
        "crypto/md5"
        "encoding/json"
        "fmt"
@@ -20,16 +19,18 @@ import (
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "git.arvados.org/arvados.git/sdk/go/arvadostest"
        "git.arvados.org/arvados.git/sdk/go/auth"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "git.arvados.org/arvados.git/sdk/go/httpserver"
        "git.arvados.org/arvados.git/sdk/go/keepclient"
        "github.com/prometheus/client_golang/prometheus"
        check "gopkg.in/check.v1"
 )
 
-var _ = check.Suite(&ProxyRemoteSuite{})
+var _ = check.Suite(&proxyRemoteSuite{})
 
-type ProxyRemoteSuite struct {
+type proxyRemoteSuite struct {
        cluster *arvados.Cluster
-       handler *handler
+       handler *router
 
        remoteClusterID      string
        remoteBlobSigningKey []byte
@@ -40,7 +41,7 @@ type ProxyRemoteSuite struct {
        remoteAPI            *httptest.Server
 }
 
-func (s *ProxyRemoteSuite) remoteKeepproxyHandler(w http.ResponseWriter, r *http.Request) {
+func (s *proxyRemoteSuite) remoteKeepproxyHandler(w http.ResponseWriter, r *http.Request) {
        expectToken, err := auth.SaltToken(arvadostest.ActiveTokenV2, s.remoteClusterID)
        if err != nil {
                panic(err)
@@ -57,7 +58,7 @@ func (s *ProxyRemoteSuite) remoteKeepproxyHandler(w http.ResponseWriter, r *http
        http.Error(w, "404", 404)
 }
 
-func (s *ProxyRemoteSuite) remoteAPIHandler(w http.ResponseWriter, r *http.Request) {
+func (s *proxyRemoteSuite) remoteAPIHandler(w http.ResponseWriter, r *http.Request) {
        host, port, _ := net.SplitHostPort(strings.Split(s.remoteKeepproxy.URL, "//")[1])
        portnum, _ := strconv.Atoi(port)
        if r.URL.Path == "/arvados/v1/discovery/v1/rest" {
@@ -81,15 +82,13 @@ func (s *ProxyRemoteSuite) remoteAPIHandler(w http.ResponseWriter, r *http.Reque
        http.Error(w, "404", 404)
 }
 
-func (s *ProxyRemoteSuite) SetUpTest(c *check.C) {
+func (s *proxyRemoteSuite) SetUpTest(c *check.C) {
        s.remoteClusterID = "z0000"
        s.remoteBlobSigningKey = []byte("3b6df6fb6518afe12922a5bc8e67bf180a358bc8")
-       s.remoteKeepproxy = httptest.NewServer(http.HandlerFunc(s.remoteKeepproxyHandler))
+       s.remoteKeepproxy = httptest.NewServer(httpserver.LogRequests(http.HandlerFunc(s.remoteKeepproxyHandler)))
        s.remoteAPI = httptest.NewUnstartedServer(http.HandlerFunc(s.remoteAPIHandler))
        s.remoteAPI.StartTLS()
        s.cluster = testCluster(c)
-       s.cluster.Collections.BlobSigningKey = knownKey
-       s.cluster.SystemRootToken = arvadostest.SystemRootToken
        s.cluster.RemoteClusters = map[string]arvados.RemoteCluster{
                s.remoteClusterID: {
                        Host:     strings.Split(s.remoteAPI.URL, "//")[1],
@@ -98,17 +97,21 @@ func (s *ProxyRemoteSuite) SetUpTest(c *check.C) {
                        Insecure: true,
                },
        }
-       s.cluster.Volumes = map[string]arvados.Volume{"zzzzz-nyw5e-000000000000000": {Driver: "mock"}}
-       s.handler = &handler{}
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
+       s.cluster.Volumes = map[string]arvados.Volume{"zzzzz-nyw5e-000000000000000": {Driver: "stub"}}
 }
 
-func (s *ProxyRemoteSuite) TearDownTest(c *check.C) {
+func (s *proxyRemoteSuite) TearDownTest(c *check.C) {
        s.remoteAPI.Close()
        s.remoteKeepproxy.Close()
 }
 
-func (s *ProxyRemoteSuite) TestProxyRemote(c *check.C) {
+func (s *proxyRemoteSuite) TestProxyRemote(c *check.C) {
+       reg := prometheus.NewRegistry()
+       router, cancel := testRouter(c, s.cluster, reg)
+       defer cancel()
+       instrumented := httpserver.Instrument(reg, ctxlog.TestLogger(c), router)
+       handler := httpserver.LogRequests(instrumented.ServeAPI(s.cluster.ManagementToken, instrumented))
+
        data := []byte("foo bar")
        s.remoteKeepData = data
        locator := fmt.Sprintf("%x+%d", md5.Sum(data), len(data))
@@ -172,7 +175,7 @@ func (s *ProxyRemoteSuite) TestProxyRemote(c *check.C) {
                        expectSignature:  true,
                },
        } {
-               c.Logf("trial: %s", trial.label)
+               c.Logf("=== trial: %s", trial.label)
 
                s.remoteKeepRequests = 0
 
@@ -184,11 +187,18 @@ func (s *ProxyRemoteSuite) TestProxyRemote(c *check.C) {
                        req.Header.Set("X-Keep-Signature", trial.xKeepSignature)
                }
                resp = httptest.NewRecorder()
-               s.handler.ServeHTTP(resp, req)
+               handler.ServeHTTP(resp, req)
                c.Check(s.remoteKeepRequests, check.Equals, trial.expectRemoteReqs)
-               c.Check(resp.Code, check.Equals, trial.expectCode)
+               if !c.Check(resp.Code, check.Equals, trial.expectCode) {
+                       c.Logf("resp.Code %d came with resp.Body %q", resp.Code, resp.Body.String())
+               }
                if resp.Code == http.StatusOK {
-                       c.Check(resp.Body.String(), check.Equals, string(data))
+                       if trial.method == "HEAD" {
+                               c.Check(resp.Body.String(), check.Equals, "")
+                               c.Check(resp.Result().ContentLength, check.Equals, int64(len(data)))
+                       } else {
+                               c.Check(resp.Body.String(), check.Equals, string(data))
+                       }
                } else {
                        c.Check(resp.Body.String(), check.Not(check.Equals), string(data))
                }
@@ -203,13 +213,13 @@ func (s *ProxyRemoteSuite) TestProxyRemote(c *check.C) {
 
                c.Check(locHdr, check.Not(check.Equals), "")
                c.Check(locHdr, check.Not(check.Matches), `.*\+R.*`)
-               c.Check(VerifySignature(s.cluster, locHdr, trial.token), check.IsNil)
+               c.Check(arvados.VerifySignature(locHdr, trial.token, s.cluster.Collections.BlobSigningTTL.Duration(), []byte(s.cluster.Collections.BlobSigningKey)), check.IsNil)
 
                // Ensure block can be requested using new signature
                req = httptest.NewRequest("GET", "/"+locHdr, nil)
                req.Header.Set("Authorization", "Bearer "+trial.token)
                resp = httptest.NewRecorder()
-               s.handler.ServeHTTP(resp, req)
+               handler.ServeHTTP(resp, req)
                c.Check(resp.Code, check.Equals, http.StatusOK)
                c.Check(s.remoteKeepRequests, check.Equals, trial.expectRemoteReqs)
        }
index 348bfb4df00087a1726ef36cbd186fe0eb5ea4c7..c131de02cbbfc01230015b59aa1350420da830ae 100644 (file)
 package keepstore
 
 import (
+       "bytes"
        "context"
-       "fmt"
-       "io"
-       "io/ioutil"
-       "time"
+       "sync"
+       "sync/atomic"
 
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/arvadosclient"
        "git.arvados.org/arvados.git/sdk/go/keepclient"
+       "github.com/prometheus/client_golang/prometheus"
 )
 
-// RunPullWorker receives PullRequests from pullq, invokes
-// PullItemAndProcess on each one. After each PR, it logs a message
-// indicating whether the pull was successful.
-func (h *handler) runPullWorker(pullq *WorkQueue) {
-       for item := range pullq.NextItem {
-               pr := item.(PullRequest)
-               err := h.pullItemAndProcess(pr)
-               pullq.DoneItem <- struct{}{}
-               if err == nil {
-                       h.Logger.Printf("Pull %s success", pr)
-               } else {
-                       h.Logger.Printf("Pull %s error: %s", pr, err)
-               }
-       }
+type PullListItem struct {
+       Locator   string   `json:"locator"`
+       Servers   []string `json:"servers"`
+       MountUUID string   `json:"mount_uuid"` // Destination mount, or "" for "anywhere"
 }
 
-// PullItemAndProcess executes a pull request by retrieving the
-// specified block from one of the specified servers, and storing it
-// on a local volume.
-//
-// If the PR specifies a non-blank mount UUID, PullItemAndProcess will
-// only attempt to write the data to the corresponding
-// volume. Otherwise it writes to any local volume, as a PUT request
-// would.
-func (h *handler) pullItemAndProcess(pullRequest PullRequest) error {
-       var vol *VolumeMount
-       if uuid := pullRequest.MountUUID; uuid != "" {
-               vol = h.volmgr.Lookup(pullRequest.MountUUID, true)
-               if vol == nil {
-                       return fmt.Errorf("pull req has nonexistent mount: %v", pullRequest)
-               }
-       }
+type puller struct {
+       keepstore  *keepstore
+       todo       []PullListItem
+       cond       *sync.Cond // lock guards todo accesses; cond broadcasts when todo becomes non-empty
+       inprogress atomic.Int64
+}
 
-       // Make a private copy of keepClient so we can set
-       // ServiceRoots to the source servers specified in the pull
-       // request.
-       keepClient := h.keepClient.Clone()
-       serviceRoots := make(map[string]string)
-       for _, addr := range pullRequest.Servers {
-               serviceRoots[addr] = addr
+func newPuller(ctx context.Context, keepstore *keepstore, reg *prometheus.Registry) *puller {
+       p := &puller{
+               keepstore: keepstore,
+               cond:      sync.NewCond(&sync.Mutex{}),
        }
-       keepClient.SetServiceRoots(serviceRoots, nil, nil)
+       reg.MustRegister(prometheus.NewGaugeFunc(
+               prometheus.GaugeOpts{
+                       Namespace: "arvados",
+                       Subsystem: "keepstore",
+                       Name:      "pull_queue_pending_entries",
+                       Help:      "Number of queued pull requests",
+               },
+               func() float64 {
+                       p.cond.L.Lock()
+                       defer p.cond.L.Unlock()
+                       return float64(len(p.todo))
+               },
+       ))
+       reg.MustRegister(prometheus.NewGaugeFunc(
+               prometheus.GaugeOpts{
+                       Namespace: "arvados",
+                       Subsystem: "keepstore",
+                       Name:      "pull_queue_inprogress_entries",
+                       Help:      "Number of pull requests in progress",
+               },
+               func() float64 {
+                       return float64(p.inprogress.Load())
+               },
+       ))
+       if len(p.keepstore.mountsW) == 0 {
+               keepstore.logger.Infof("not running pull worker because there are no writable volumes")
+               return p
+       }
+       for i := 0; i < 1 || i < keepstore.cluster.Collections.BlobReplicateConcurrency; i++ {
+               go p.runWorker(ctx)
+       }
+       return p
+}
 
-       signedLocator := SignLocator(h.Cluster, pullRequest.Locator, keepClient.Arvados.ApiToken, time.Now().Add(time.Minute))
+func (p *puller) SetPullList(newlist []PullListItem) {
+       p.cond.L.Lock()
+       p.todo = newlist
+       p.cond.L.Unlock()
+       p.cond.Broadcast()
+}
 
-       reader, _, _, err := GetContent(signedLocator, keepClient)
-       if err != nil {
-               return err
+func (p *puller) runWorker(ctx context.Context) {
+       if len(p.keepstore.mountsW) == 0 {
+               p.keepstore.logger.Infof("not running pull worker because there are no writable volumes")
+               return
        }
-       if reader == nil {
-               return fmt.Errorf("No reader found for : %s", signedLocator)
+       c, err := arvados.NewClientFromConfig(p.keepstore.cluster)
+       if err != nil {
+               p.keepstore.logger.Errorf("error setting up pull worker: %s", err)
+               return
        }
-       defer reader.Close()
-
-       readContent, err := ioutil.ReadAll(reader)
+       c.AuthToken = "keepstore-token-used-for-pulling-data-from-same-cluster"
+       ac, err := arvadosclient.New(c)
        if err != nil {
-               return err
+               p.keepstore.logger.Errorf("error setting up pull worker: %s", err)
+               return
        }
-
-       if readContent == nil {
-               return fmt.Errorf("Content not found for: %s", signedLocator)
+       keepClient := &keepclient.KeepClient{
+               Arvados:       ac,
+               Want_replicas: 1,
+               DiskCacheSize: keepclient.DiskCacheDisabled,
        }
+       // Ensure the loop below wakes up and returns when ctx
+       // cancels, even if pull list is empty.
+       go func() {
+               <-ctx.Done()
+               p.cond.Broadcast()
+       }()
+       for {
+               p.cond.L.Lock()
+               for len(p.todo) == 0 && ctx.Err() == nil {
+                       p.cond.Wait()
+               }
+               if ctx.Err() != nil {
+                       return
+               }
+               item := p.todo[0]
+               p.todo = p.todo[1:]
+               p.inprogress.Add(1)
+               p.cond.L.Unlock()
 
-       return writePulledBlock(h.volmgr, vol, readContent, pullRequest.Locator)
-}
+               func() {
+                       defer p.inprogress.Add(-1)
 
-// GetContent fetches the content for the given locator using keepclient.
-var GetContent = func(signedLocator string, keepClient *keepclient.KeepClient) (io.ReadCloser, int64, string, error) {
-       return keepClient.Get(signedLocator)
-}
+                       logger := p.keepstore.logger.WithField("locator", item.Locator)
+
+                       li, err := parseLocator(item.Locator)
+                       if err != nil {
+                               logger.Warn("ignoring pull request for invalid locator")
+                               return
+                       }
+
+                       var dst *mount
+                       if item.MountUUID != "" {
+                               dst = p.keepstore.mounts[item.MountUUID]
+                               if dst == nil {
+                                       logger.Warnf("ignoring pull list entry for nonexistent mount %s", item.MountUUID)
+                                       return
+                               } else if !dst.AllowWrite {
+                                       logger.Warnf("ignoring pull list entry for readonly mount %s", item.MountUUID)
+                                       return
+                               }
+                       } else {
+                               dst = p.keepstore.rendezvous(item.Locator, p.keepstore.mountsW)[0]
+                       }
+
+                       serviceRoots := make(map[string]string)
+                       for _, addr := range item.Servers {
+                               serviceRoots[addr] = addr
+                       }
+                       keepClient.SetServiceRoots(serviceRoots, nil, nil)
+
+                       signedLocator := p.keepstore.signLocator(c.AuthToken, item.Locator)
 
-var writePulledBlock = func(volmgr *RRVolumeManager, volume Volume, data []byte, locator string) error {
-       if volume != nil {
-               return volume.Put(context.Background(), locator, data)
+                       buf := bytes.NewBuffer(nil)
+                       _, err = keepClient.BlockRead(ctx, arvados.BlockReadOptions{
+                               Locator: signedLocator,
+                               WriteTo: buf,
+                       })
+                       if err != nil {
+                               logger.WithError(err).Warnf("error pulling data from remote servers (%s)", item.Servers)
+                               return
+                       }
+                       err = dst.BlockWrite(ctx, li.hash, buf.Bytes())
+                       if err != nil {
+                               logger.WithError(err).Warnf("error writing data to %s", dst.UUID)
+                               return
+                       }
+                       logger.Info("block pulled")
+               }()
        }
-       _, err := PutBlock(context.Background(), volmgr, data, locator, nil)
-       return err
 }
diff --git a/services/keepstore/pull_worker_integration_test.go b/services/keepstore/pull_worker_integration_test.go
deleted file mode 100644 (file)
index 3855b4e..0000000
+++ /dev/null
@@ -1,118 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "bytes"
-       "context"
-       "errors"
-       "io"
-       "io/ioutil"
-       "strings"
-
-       "git.arvados.org/arvados.git/sdk/go/arvadostest"
-       "git.arvados.org/arvados.git/sdk/go/keepclient"
-       "github.com/prometheus/client_golang/prometheus"
-       check "gopkg.in/check.v1"
-)
-
-type PullWorkIntegrationTestData struct {
-       Name     string
-       Locator  string
-       Content  string
-       GetError string
-}
-
-func (s *HandlerSuite) setupPullWorkerIntegrationTest(c *check.C, testData PullWorkIntegrationTestData, wantData bool) PullRequest {
-       arvadostest.StartKeep(2, false)
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       // Put content if the test needs it
-       if wantData {
-               locator, _, err := s.handler.keepClient.PutB([]byte(testData.Content))
-               if err != nil {
-                       c.Errorf("Error putting test data in setup for %s %s %v", testData.Content, locator, err)
-               }
-               if locator == "" {
-                       c.Errorf("No locator found after putting test data")
-               }
-       }
-
-       // Create pullRequest for the test
-       pullRequest := PullRequest{
-               Locator: testData.Locator,
-       }
-       return pullRequest
-}
-
-// Do a get on a block that is not existing in any of the keep servers.
-// Expect "block not found" error.
-func (s *HandlerSuite) TestPullWorkerIntegration_GetNonExistingLocator(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       testData := PullWorkIntegrationTestData{
-               Name:     "TestPullWorkerIntegration_GetLocator",
-               Locator:  "5d41402abc4b2a76b9719d911017c592",
-               Content:  "hello",
-               GetError: "Block not found",
-       }
-
-       pullRequest := s.setupPullWorkerIntegrationTest(c, testData, false)
-       defer arvadostest.StopKeep(2)
-
-       s.performPullWorkerIntegrationTest(testData, pullRequest, c)
-}
-
-// Do a get on a block that exists on one of the keep servers.
-// The setup method will create this block before doing the get.
-func (s *HandlerSuite) TestPullWorkerIntegration_GetExistingLocator(c *check.C) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       testData := PullWorkIntegrationTestData{
-               Name:     "TestPullWorkerIntegration_GetLocator",
-               Locator:  "5d41402abc4b2a76b9719d911017c592",
-               Content:  "hello",
-               GetError: "",
-       }
-
-       pullRequest := s.setupPullWorkerIntegrationTest(c, testData, true)
-       defer arvadostest.StopKeep(2)
-
-       s.performPullWorkerIntegrationTest(testData, pullRequest, c)
-}
-
-// Perform the test.
-// The test directly invokes the "PullItemAndProcess" rather than
-// putting an item on the pullq so that the errors can be verified.
-func (s *HandlerSuite) performPullWorkerIntegrationTest(testData PullWorkIntegrationTestData, pullRequest PullRequest, c *check.C) {
-
-       // Override writePulledBlock to mock PutBlock functionality
-       defer func(orig func(*RRVolumeManager, Volume, []byte, string) error) { writePulledBlock = orig }(writePulledBlock)
-       writePulledBlock = func(_ *RRVolumeManager, _ Volume, content []byte, _ string) error {
-               c.Check(string(content), check.Equals, testData.Content)
-               return nil
-       }
-
-       // Override GetContent to mock keepclient Get functionality
-       defer func(orig func(string, *keepclient.KeepClient) (io.ReadCloser, int64, string, error)) {
-               GetContent = orig
-       }(GetContent)
-       GetContent = func(signedLocator string, keepClient *keepclient.KeepClient) (reader io.ReadCloser, contentLength int64, url string, err error) {
-               if testData.GetError != "" {
-                       return nil, 0, "", errors.New(testData.GetError)
-               }
-               rdr := ioutil.NopCloser(bytes.NewBufferString(testData.Content))
-               return rdr, int64(len(testData.Content)), "", nil
-       }
-
-       err := s.handler.pullItemAndProcess(pullRequest)
-
-       if len(testData.GetError) > 0 {
-               if (err == nil) || (!strings.Contains(err.Error(), testData.GetError)) {
-                       c.Errorf("Got error %v, expected %v", err, testData.GetError)
-               }
-       } else {
-               if err != nil {
-                       c.Errorf("Got error %v, expected nil", err)
-               }
-       }
-}
index 2626e66d8898745b9f29c42d9beda9ee580626a4..d109b56df3cee8e2ac3259ebb784fe4cfdacc20b 100644 (file)
@@ -7,309 +7,130 @@ package keepstore
 import (
        "bytes"
        "context"
+       "crypto/md5"
+       "encoding/json"
        "errors"
+       "fmt"
        "io"
-       "io/ioutil"
        "net/http"
+       "net/http/httptest"
+       "sort"
        "time"
 
        "git.arvados.org/arvados.git/sdk/go/arvados"
-       "git.arvados.org/arvados.git/sdk/go/keepclient"
-       "github.com/prometheus/client_golang/prometheus"
+       "git.arvados.org/arvados.git/sdk/go/arvadostest"
+       "github.com/sirupsen/logrus"
        . "gopkg.in/check.v1"
-       check "gopkg.in/check.v1"
 )
 
-var _ = Suite(&PullWorkerTestSuite{})
-
-type PullWorkerTestSuite struct {
-       cluster *arvados.Cluster
-       handler *handler
-
-       testPullLists map[string]string
-       readContent   string
-       readError     error
-       putContent    []byte
-       putError      error
-}
-
-func (s *PullWorkerTestSuite) SetUpTest(c *C) {
-       s.cluster = testCluster(c)
-       s.cluster.Volumes = map[string]arvados.Volume{
-               "zzzzz-nyw5e-000000000000000": {Driver: "mock"},
-               "zzzzz-nyw5e-111111111111111": {Driver: "mock"},
+func (s *routerSuite) TestPullList_Execute(c *C) {
+       remotecluster := testCluster(c)
+       remotecluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-rrrrrrrrrrrrrrr": {Replication: 1, Driver: "stub"},
        }
-       s.cluster.Collections.BlobReplicateConcurrency = 1
-
-       s.handler = &handler{}
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-
-       s.readContent = ""
-       s.readError = nil
-       s.putContent = []byte{}
-       s.putError = nil
-
-       // When a new pull request arrives, the old one will be overwritten.
-       // This behavior is verified using these two maps in the
-       // "TestPullWorkerPullList_with_two_items_latest_replacing_old"
-       s.testPullLists = make(map[string]string)
-}
-
-var firstPullList = []byte(`[
-               {
-                       "locator":"acbd18db4cc2f85cedef654fccc4a4d8+3",
-                       "servers":[
-                               "server_1",
-                               "server_2"
-                       ]
-               },{
-                       "locator":"37b51d194a7513e45b56f6524f2d51f2+3",
-                       "servers":[
-                               "server_3"
-                       ]
-               }
-       ]`)
-
-var secondPullList = []byte(`[
-               {
-                       "locator":"73feffa4b7f6bb68e44cf984c85f6e88+3",
-                       "servers":[
-                               "server_1",
-                               "server_2"
-                       ]
-               }
-       ]`)
-
-type PullWorkerTestData struct {
-       name         string
-       req          RequestTester
-       responseCode int
-       responseBody string
-       readContent  string
-       readError    bool
-       putError     bool
-}
-
-// Ensure MountUUID in a pull list is correctly translated to a Volume
-// argument passed to writePulledBlock().
-func (s *PullWorkerTestSuite) TestSpecifyMountUUID(c *C) {
-       defer func(f func(*RRVolumeManager, Volume, []byte, string) error) {
-               writePulledBlock = f
-       }(writePulledBlock)
-       pullq := s.handler.Handler.(*router).pullq
-
-       for _, spec := range []struct {
-               sendUUID     string
-               expectVolume Volume
-       }{
-               {
-                       sendUUID:     "",
-                       expectVolume: nil,
-               },
-               {
-                       sendUUID:     s.handler.volmgr.Mounts()[0].UUID,
-                       expectVolume: s.handler.volmgr.Mounts()[0].Volume,
-               },
-       } {
-               writePulledBlock = func(_ *RRVolumeManager, v Volume, _ []byte, _ string) error {
-                       c.Check(v, Equals, spec.expectVolume)
-                       return nil
+       remoterouter, cancel := testRouter(c, remotecluster, nil)
+       defer cancel()
+       remoteserver := httptest.NewServer(remoterouter)
+       defer remoteserver.Close()
+
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       executePullList := func(pullList []PullListItem) string {
+               var logbuf bytes.Buffer
+               logger := logrus.New()
+               logger.Out = &logbuf
+               router.keepstore.logger = logger
+
+               listjson, err := json.Marshal(pullList)
+               c.Assert(err, IsNil)
+               resp := call(router, "PUT", "http://example/pull", s.cluster.SystemRootToken, listjson, nil)
+               c.Check(resp.Code, Equals, http.StatusOK)
+               for {
+                       router.puller.cond.L.Lock()
+                       todolen := len(router.puller.todo)
+                       router.puller.cond.L.Unlock()
+                       if todolen == 0 && router.puller.inprogress.Load() == 0 {
+                               break
+                       }
+                       time.Sleep(time.Millisecond)
                }
-
-               resp := IssueRequest(s.handler, &RequestTester{
-                       uri:      "/pull",
-                       apiToken: s.cluster.SystemRootToken,
-                       method:   "PUT",
-                       requestBody: []byte(`[{
-                               "locator":"acbd18db4cc2f85cedef654fccc4a4d8+3",
-                               "servers":["server_1","server_2"],
-                               "mount_uuid":"` + spec.sendUUID + `"}]`),
-               })
-               c.Assert(resp.Code, Equals, http.StatusOK)
-               expectEqualWithin(c, time.Second, 0, func() interface{} {
-                       st := pullq.Status()
-                       return st.InProgress + st.Queued
-               })
-       }
-}
-
-func (s *PullWorkerTestSuite) TestPullWorkerPullList_with_two_locators(c *C) {
-       testData := PullWorkerTestData{
-               name:         "TestPullWorkerPullList_with_two_locators",
-               req:          RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", firstPullList, ""},
-               responseCode: http.StatusOK,
-               responseBody: "Received 2 pull requests\n",
-               readContent:  "hello",
-               readError:    false,
-               putError:     false,
-       }
-
-       s.performTest(testData, c)
-}
-
-func (s *PullWorkerTestSuite) TestPullWorkerPullList_with_one_locator(c *C) {
-       testData := PullWorkerTestData{
-               name:         "TestPullWorkerPullList_with_one_locator",
-               req:          RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", secondPullList, ""},
-               responseCode: http.StatusOK,
-               responseBody: "Received 1 pull requests\n",
-               readContent:  "hola",
-               readError:    false,
-               putError:     false,
-       }
-
-       s.performTest(testData, c)
-}
-
-func (s *PullWorkerTestSuite) TestPullWorker_error_on_get_one_locator(c *C) {
-       testData := PullWorkerTestData{
-               name:         "TestPullWorker_error_on_get_one_locator",
-               req:          RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", secondPullList, ""},
-               responseCode: http.StatusOK,
-               responseBody: "Received 1 pull requests\n",
-               readContent:  "unused",
-               readError:    true,
-               putError:     false,
+               return logbuf.String()
        }
 
-       s.performTest(testData, c)
-}
-
-func (s *PullWorkerTestSuite) TestPullWorker_error_on_get_two_locators(c *C) {
-       testData := PullWorkerTestData{
-               name:         "TestPullWorker_error_on_get_two_locators",
-               req:          RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", firstPullList, ""},
-               responseCode: http.StatusOK,
-               responseBody: "Received 2 pull requests\n",
-               readContent:  "unused",
-               readError:    true,
-               putError:     false,
-       }
-
-       s.performTest(testData, c)
-}
-
-func (s *PullWorkerTestSuite) TestPullWorker_error_on_put_one_locator(c *C) {
-       testData := PullWorkerTestData{
-               name:         "TestPullWorker_error_on_put_one_locator",
-               req:          RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", secondPullList, ""},
-               responseCode: http.StatusOK,
-               responseBody: "Received 1 pull requests\n",
-               readContent:  "hello hello",
-               readError:    false,
-               putError:     true,
-       }
-
-       s.performTest(testData, c)
-}
-
-func (s *PullWorkerTestSuite) TestPullWorker_error_on_put_two_locators(c *C) {
-       testData := PullWorkerTestData{
-               name:         "TestPullWorker_error_on_put_two_locators",
-               req:          RequestTester{"/pull", s.cluster.SystemRootToken, "PUT", firstPullList, ""},
-               responseCode: http.StatusOK,
-               responseBody: "Received 2 pull requests\n",
-               readContent:  "hello again",
-               readError:    false,
-               putError:     true,
-       }
-
-       s.performTest(testData, c)
-}
-
-// In this case, the item will not be placed on pullq
-func (s *PullWorkerTestSuite) TestPullWorker_invalidToken(c *C) {
-       testData := PullWorkerTestData{
-               name:         "TestPullWorkerPullList_with_two_locators",
-               req:          RequestTester{"/pull", "invalidToken", "PUT", firstPullList, ""},
-               responseCode: http.StatusUnauthorized,
-               responseBody: "Unauthorized\n",
-               readContent:  "hello",
-               readError:    false,
-               putError:     false,
-       }
-
-       s.performTest(testData, c)
-}
-
-func (s *PullWorkerTestSuite) performTest(testData PullWorkerTestData, c *C) {
-       pullq := s.handler.Handler.(*router).pullq
-
-       s.testPullLists[testData.name] = testData.responseBody
-
-       processedPullLists := make(map[string]string)
-
-       // Override GetContent to mock keepclient Get functionality
-       defer func(orig func(string, *keepclient.KeepClient) (io.ReadCloser, int64, string, error)) {
-               GetContent = orig
-       }(GetContent)
-       GetContent = func(signedLocator string, keepClient *keepclient.KeepClient) (reader io.ReadCloser, contentLength int64, url string, err error) {
-               c.Assert(getStatusItem(s.handler, "PullQueue", "InProgress"), Equals, float64(1))
-               processedPullLists[testData.name] = testData.responseBody
-               if testData.readError {
-                       err = errors.New("Error getting data")
-                       s.readError = err
-                       return
-               }
-               s.readContent = testData.readContent
-               reader = ioutil.NopCloser(bytes.NewBufferString(testData.readContent))
-               contentLength = int64(len(testData.readContent))
-               return
+       newRemoteBlock := func(datastring string) string {
+               data := []byte(datastring)
+               hash := fmt.Sprintf("%x", md5.Sum(data))
+               locator := fmt.Sprintf("%s+%d", hash, len(data))
+               _, err := remoterouter.keepstore.BlockWrite(context.Background(), arvados.BlockWriteOptions{
+                       Hash: hash,
+                       Data: data,
+               })
+               c.Assert(err, IsNil)
+               return locator
        }
 
-       // Override writePulledBlock to mock PutBlock functionality
-       defer func(orig func(*RRVolumeManager, Volume, []byte, string) error) { writePulledBlock = orig }(writePulledBlock)
-       writePulledBlock = func(_ *RRVolumeManager, v Volume, content []byte, locator string) error {
-               if testData.putError {
-                       s.putError = errors.New("Error putting data")
-                       return s.putError
-               }
-               s.putContent = content
-               return nil
+       mounts := append([]*mount(nil), router.keepstore.mountsR...)
+       sort.Slice(mounts, func(i, j int) bool { return mounts[i].UUID < mounts[j].UUID })
+       var vols []*stubVolume
+       for _, mount := range mounts {
+               vols = append(vols, mount.volume.(*stubVolume))
        }
 
-       c.Check(getStatusItem(s.handler, "PullQueue", "InProgress"), Equals, float64(0))
-       c.Check(getStatusItem(s.handler, "PullQueue", "Queued"), Equals, float64(0))
-       c.Check(getStatusItem(s.handler, "Version"), Not(Equals), "")
-
-       response := IssueRequest(s.handler, &testData.req)
-       c.Assert(response.Code, Equals, testData.responseCode)
-       c.Assert(response.Body.String(), Equals, testData.responseBody)
+       ctx := authContext(arvadostest.ActiveTokenV2)
 
-       expectEqualWithin(c, time.Second, 0, func() interface{} {
-               st := pullq.Status()
-               return st.InProgress + st.Queued
-       })
+       locator := newRemoteBlock("pull available block to unspecified volume")
+       executePullList([]PullListItem{{
+               Locator: locator,
+               Servers: []string{remoteserver.URL}}})
+       _, err := router.keepstore.BlockRead(ctx, arvados.BlockReadOptions{
+               Locator: router.keepstore.signLocator(arvadostest.ActiveTokenV2, locator),
+               WriteTo: io.Discard})
+       c.Check(err, IsNil)
 
-       if testData.name == "TestPullWorkerPullList_with_two_items_latest_replacing_old" {
-               c.Assert(len(s.testPullLists), Equals, 2)
-               c.Assert(len(processedPullLists), Equals, 1)
-               c.Assert(s.testPullLists["Added_before_actual_test_item"], NotNil)
-               c.Assert(s.testPullLists["TestPullWorkerPullList_with_two_items_latest_replacing_old"], NotNil)
-               c.Assert(processedPullLists["TestPullWorkerPullList_with_two_items_latest_replacing_old"], NotNil)
-       } else {
-               if testData.responseCode == http.StatusOK {
-                       c.Assert(len(s.testPullLists), Equals, 1)
-                       c.Assert(len(processedPullLists), Equals, 1)
-                       c.Assert(s.testPullLists[testData.name], NotNil)
-               } else {
-                       c.Assert(len(s.testPullLists), Equals, 1)
-                       c.Assert(len(processedPullLists), Equals, 0)
-               }
-       }
-
-       if testData.readError {
-               c.Assert(s.readError, NotNil)
-       } else if testData.responseCode == http.StatusOK {
-               c.Assert(s.readError, IsNil)
-               c.Assert(s.readContent, Equals, testData.readContent)
-               if testData.putError {
-                       c.Assert(s.putError, NotNil)
-               } else {
-                       c.Assert(s.putError, IsNil)
-                       c.Assert(string(s.putContent), Equals, testData.readContent)
-               }
-       }
-
-       expectChannelEmpty(c, pullq.NextItem)
+       locator0 := newRemoteBlock("pull available block to specified volume 0")
+       locator1 := newRemoteBlock("pull available block to specified volume 1")
+       executePullList([]PullListItem{
+               {
+                       Locator:   locator0,
+                       Servers:   []string{remoteserver.URL},
+                       MountUUID: vols[0].params.UUID},
+               {
+                       Locator:   locator1,
+                       Servers:   []string{remoteserver.URL},
+                       MountUUID: vols[1].params.UUID}})
+       c.Check(vols[0].data[locator0[:32]].data, NotNil)
+       c.Check(vols[1].data[locator1[:32]].data, NotNil)
+
+       locator = fooHash + "+3"
+       logs := executePullList([]PullListItem{{
+               Locator: locator,
+               Servers: []string{remoteserver.URL}}})
+       c.Check(logs, Matches, ".*error pulling data from remote servers.*Block not found.*locator=acbd.*\n")
+
+       locator = fooHash + "+3"
+       logs = executePullList([]PullListItem{{
+               Locator: locator,
+               Servers: []string{"http://0.0.0.0:9/"}}})
+       c.Check(logs, Matches, ".*error pulling data from remote servers.*connection refused.*locator=acbd.*\n")
+
+       locator = newRemoteBlock("log error writing to local volume")
+       vols[0].blockWrite = func(context.Context, string, []byte) error { return errors.New("test error") }
+       vols[1].blockWrite = vols[0].blockWrite
+       logs = executePullList([]PullListItem{{
+               Locator: locator,
+               Servers: []string{remoteserver.URL}}})
+       c.Check(logs, Matches, ".*error writing data to zzzzz-nyw5e-.*error=\"test error\".*locator=.*\n")
+       vols[0].blockWrite = nil
+       vols[1].blockWrite = nil
+
+       locator = newRemoteBlock("log error when destination mount does not exist")
+       logs = executePullList([]PullListItem{{
+               Locator:   locator,
+               Servers:   []string{remoteserver.URL},
+               MountUUID: "bogus-mount-uuid"}})
+       c.Check(logs, Matches, ".*ignoring pull list entry for nonexistent mount bogus-mount-uuid.*locator=.*\n")
+
+       logs = executePullList([]PullListItem{})
+       c.Logf("%s", logs)
 }
diff --git a/services/keepstore/putprogress.go b/services/keepstore/putprogress.go
new file mode 100644 (file)
index 0000000..e02b2d0
--- /dev/null
@@ -0,0 +1,101 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "github.com/sirupsen/logrus"
+)
+
+type putProgress struct {
+       classNeeded      map[string]bool
+       classTodo        map[string]bool
+       mountUsed        map[*mount]bool
+       totalReplication int
+       classDone        map[string]int
+}
+
+func (pr *putProgress) Add(mnt *mount) {
+       if pr.mountUsed[mnt] {
+               logrus.Warnf("BUG? superfluous extra write to mount %s", mnt.UUID)
+               return
+       }
+       pr.mountUsed[mnt] = true
+       pr.totalReplication += mnt.Replication
+       for class := range mnt.StorageClasses {
+               pr.classDone[class] += mnt.Replication
+               delete(pr.classTodo, class)
+       }
+}
+
+func (pr *putProgress) Sub(mnt *mount) {
+       if !pr.mountUsed[mnt] {
+               logrus.Warnf("BUG? Sub called with no prior matching Add: %s", mnt.UUID)
+               return
+       }
+       pr.mountUsed[mnt] = false
+       pr.totalReplication -= mnt.Replication
+       for class := range mnt.StorageClasses {
+               pr.classDone[class] -= mnt.Replication
+               if pr.classNeeded[class] {
+                       pr.classTodo[class] = true
+               }
+       }
+}
+
+func (pr *putProgress) Done() bool {
+       return len(pr.classTodo) == 0 && pr.totalReplication > 0
+}
+
+func (pr *putProgress) Want(mnt *mount) bool {
+       if pr.Done() || pr.mountUsed[mnt] {
+               return false
+       }
+       if len(pr.classTodo) == 0 {
+               // none specified == "any"
+               return true
+       }
+       for class := range mnt.StorageClasses {
+               if pr.classTodo[class] {
+                       return true
+               }
+       }
+       return false
+}
+
+func (pr *putProgress) Copy() *putProgress {
+       cp := putProgress{
+               classNeeded:      pr.classNeeded,
+               classTodo:        make(map[string]bool, len(pr.classTodo)),
+               classDone:        make(map[string]int, len(pr.classDone)),
+               mountUsed:        make(map[*mount]bool, len(pr.mountUsed)),
+               totalReplication: pr.totalReplication,
+       }
+       for k, v := range pr.classTodo {
+               cp.classTodo[k] = v
+       }
+       for k, v := range pr.classDone {
+               cp.classDone[k] = v
+       }
+       for k, v := range pr.mountUsed {
+               cp.mountUsed[k] = v
+       }
+       return &cp
+}
+
+func newPutProgress(classes []string) putProgress {
+       pr := putProgress{
+               classNeeded: make(map[string]bool, len(classes)),
+               classTodo:   make(map[string]bool, len(classes)),
+               classDone:   map[string]int{},
+               mountUsed:   map[*mount]bool{},
+       }
+       for _, c := range classes {
+               if c != "" {
+                       pr.classNeeded[c] = true
+                       pr.classTodo[c] = true
+               }
+       }
+       return pr
+}
diff --git a/services/keepstore/router.go b/services/keepstore/router.go
new file mode 100644 (file)
index 0000000..7ff82aa
--- /dev/null
@@ -0,0 +1,276 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "encoding/json"
+       "errors"
+       "fmt"
+       "io"
+       "net/http"
+       "os"
+       "strconv"
+       "strings"
+       "sync/atomic"
+
+       "git.arvados.org/arvados.git/lib/service"
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/auth"
+       "git.arvados.org/arvados.git/sdk/go/httpserver"
+       "github.com/gorilla/mux"
+)
+
+type router struct {
+       http.Handler
+       keepstore *keepstore
+       puller    *puller
+       trasher   *trasher
+}
+
+func newRouter(keepstore *keepstore, puller *puller, trasher *trasher) service.Handler {
+       rtr := &router{
+               keepstore: keepstore,
+               puller:    puller,
+               trasher:   trasher,
+       }
+       adminonly := func(h http.HandlerFunc) http.HandlerFunc {
+               return auth.RequireLiteralToken(keepstore.cluster.SystemRootToken, h).ServeHTTP
+       }
+
+       r := mux.NewRouter()
+       locatorPath := `/{locator:[0-9a-f]{32}.*}`
+       get := r.Methods(http.MethodGet, http.MethodHead).Subrouter()
+       get.HandleFunc(locatorPath, rtr.handleBlockRead)
+       get.HandleFunc(`/index`, adminonly(rtr.handleIndex))
+       get.HandleFunc(`/index/{prefix:[0-9a-f]{0,32}}`, adminonly(rtr.handleIndex))
+       get.HandleFunc(`/mounts`, adminonly(rtr.handleMounts))
+       get.HandleFunc(`/mounts/{uuid}/blocks`, adminonly(rtr.handleIndex))
+       get.HandleFunc(`/mounts/{uuid}/blocks/{prefix:[0-9a-f]{0,32}}`, adminonly(rtr.handleIndex))
+       put := r.Methods(http.MethodPut).Subrouter()
+       put.HandleFunc(locatorPath, rtr.handleBlockWrite)
+       put.HandleFunc(`/pull`, adminonly(rtr.handlePullList))
+       put.HandleFunc(`/trash`, adminonly(rtr.handleTrashList))
+       put.HandleFunc(`/untrash`+locatorPath, adminonly(rtr.handleUntrash))
+       touch := r.Methods("TOUCH").Subrouter()
+       touch.HandleFunc(locatorPath, adminonly(rtr.handleBlockTouch))
+       delete := r.Methods(http.MethodDelete).Subrouter()
+       delete.HandleFunc(locatorPath, adminonly(rtr.handleBlockTrash))
+       r.NotFoundHandler = http.HandlerFunc(rtr.handleBadRequest)
+       r.MethodNotAllowedHandler = http.HandlerFunc(rtr.handleBadRequest)
+       rtr.Handler = auth.LoadToken(r)
+       return rtr
+}
+
+func (rtr *router) CheckHealth() error {
+       return nil
+}
+
+func (rtr *router) Done() <-chan struct{} {
+       return nil
+}
+
+func (rtr *router) handleBlockRead(w http.ResponseWriter, req *http.Request) {
+       // Intervening proxies must not return a cached GET response
+       // to a prior request if a X-Keep-Signature request header has
+       // been added or changed.
+       w.Header().Add("Vary", "X-Keep-Signature")
+       var localLocator func(string)
+       if strings.SplitN(req.Header.Get("X-Keep-Signature"), ",", 2)[0] == "local" {
+               localLocator = func(locator string) {
+                       w.Header().Set("X-Keep-Locator", locator)
+               }
+       }
+       out := w
+       if req.Method == http.MethodHead {
+               out = discardWrite{ResponseWriter: w}
+       } else if li, err := parseLocator(mux.Vars(req)["locator"]); err != nil {
+               rtr.handleError(w, req, err)
+               return
+       } else if li.size == 0 && li.hash != "d41d8cd98f00b204e9800998ecf8427e" {
+               // GET {hash} (with no size hint) is not allowed
+               // because we can't report md5 mismatches.
+               rtr.handleError(w, req, errMethodNotAllowed)
+               return
+       }
+       n, err := rtr.keepstore.BlockRead(req.Context(), arvados.BlockReadOptions{
+               Locator:      mux.Vars(req)["locator"],
+               WriteTo:      out,
+               LocalLocator: localLocator,
+       })
+       if err != nil && (n == 0 || req.Method == http.MethodHead) {
+               rtr.handleError(w, req, err)
+               return
+       }
+}
+
+func (rtr *router) handleBlockWrite(w http.ResponseWriter, req *http.Request) {
+       dataSize, _ := strconv.Atoi(req.Header.Get("Content-Length"))
+       replicas, _ := strconv.Atoi(req.Header.Get("X-Arvados-Replicas-Desired"))
+       resp, err := rtr.keepstore.BlockWrite(req.Context(), arvados.BlockWriteOptions{
+               Hash:           mux.Vars(req)["locator"],
+               Reader:         req.Body,
+               DataSize:       dataSize,
+               RequestID:      req.Header.Get("X-Request-Id"),
+               StorageClasses: trimSplit(req.Header.Get("X-Keep-Storage-Classes"), ","),
+               Replicas:       replicas,
+       })
+       if err != nil {
+               rtr.handleError(w, req, err)
+               return
+       }
+       w.Header().Set("X-Keep-Replicas-Stored", fmt.Sprintf("%d", resp.Replicas))
+       scc := ""
+       for k, n := range resp.StorageClasses {
+               if n > 0 {
+                       if scc != "" {
+                               scc += "; "
+                       }
+                       scc += fmt.Sprintf("%s=%d", k, n)
+               }
+       }
+       w.Header().Set("X-Keep-Storage-Classes-Confirmed", scc)
+       w.WriteHeader(http.StatusOK)
+       fmt.Fprintln(w, resp.Locator)
+}
+
+func (rtr *router) handleBlockTouch(w http.ResponseWriter, req *http.Request) {
+       err := rtr.keepstore.BlockTouch(req.Context(), mux.Vars(req)["locator"])
+       rtr.handleError(w, req, err)
+}
+
+func (rtr *router) handleBlockTrash(w http.ResponseWriter, req *http.Request) {
+       err := rtr.keepstore.BlockTrash(req.Context(), mux.Vars(req)["locator"])
+       rtr.handleError(w, req, err)
+}
+
+func (rtr *router) handleMounts(w http.ResponseWriter, req *http.Request) {
+       json.NewEncoder(w).Encode(rtr.keepstore.Mounts())
+}
+
+func (rtr *router) handleIndex(w http.ResponseWriter, req *http.Request) {
+       prefix := req.FormValue("prefix")
+       if prefix == "" {
+               prefix = mux.Vars(req)["prefix"]
+       }
+       cw := &countingWriter{writer: w}
+       err := rtr.keepstore.Index(req.Context(), IndexOptions{
+               MountUUID: mux.Vars(req)["uuid"],
+               Prefix:    prefix,
+               WriteTo:   cw,
+       })
+       if err != nil && cw.n.Load() == 0 {
+               // Nothing was written, so it's not too late to report
+               // an error via http response header. (Otherwise, all
+               // we can do is omit the trailing newline below to
+               // indicate something went wrong.)
+               rtr.handleError(w, req, err)
+               return
+       }
+       if err == nil {
+               // A trailing blank line signals to the caller that
+               // the response is complete.
+               w.Write([]byte("\n"))
+       }
+}
+
+func (rtr *router) handlePullList(w http.ResponseWriter, req *http.Request) {
+       var pl []PullListItem
+       err := json.NewDecoder(req.Body).Decode(&pl)
+       if err != nil {
+               rtr.handleError(w, req, err)
+               return
+       }
+       req.Body.Close()
+       if len(pl) > 0 && len(pl[0].Locator) == 32 {
+               rtr.handleError(w, req, httpserver.ErrorWithStatus(errors.New("rejecting pull list containing a locator without a size hint -- this probably means keep-balance needs to be upgraded"), http.StatusBadRequest))
+               return
+       }
+       rtr.puller.SetPullList(pl)
+}
+
+func (rtr *router) handleTrashList(w http.ResponseWriter, req *http.Request) {
+       var tl []TrashListItem
+       err := json.NewDecoder(req.Body).Decode(&tl)
+       if err != nil {
+               rtr.handleError(w, req, err)
+               return
+       }
+       req.Body.Close()
+       rtr.trasher.SetTrashList(tl)
+}
+
+func (rtr *router) handleUntrash(w http.ResponseWriter, req *http.Request) {
+       err := rtr.keepstore.BlockUntrash(req.Context(), mux.Vars(req)["locator"])
+       rtr.handleError(w, req, err)
+}
+
+func (rtr *router) handleBadRequest(w http.ResponseWriter, req *http.Request) {
+       http.Error(w, "Bad Request", http.StatusBadRequest)
+}
+
+func (rtr *router) handleError(w http.ResponseWriter, req *http.Request, err error) {
+       if req.Context().Err() != nil {
+               w.WriteHeader(499)
+               return
+       }
+       if err == nil {
+               return
+       } else if os.IsNotExist(err) {
+               w.WriteHeader(http.StatusNotFound)
+       } else if statusErr := interface{ HTTPStatus() int }(nil); errors.As(err, &statusErr) {
+               w.WriteHeader(statusErr.HTTPStatus())
+       } else {
+               w.WriteHeader(http.StatusInternalServerError)
+       }
+       fmt.Fprintln(w, err.Error())
+}
+
+type countingWriter struct {
+       writer io.Writer
+       n      atomic.Int64
+}
+
+func (cw *countingWriter) Write(p []byte) (int, error) {
+       n, err := cw.writer.Write(p)
+       cw.n.Add(int64(n))
+       return n, err
+}
+
+// Split s by sep, trim whitespace from each part, and drop empty
+// parts.
+func trimSplit(s, sep string) []string {
+       var r []string
+       for _, part := range strings.Split(s, sep) {
+               part = strings.TrimSpace(part)
+               if part != "" {
+                       r = append(r, part)
+               }
+       }
+       return r
+}
+
+// setSizeOnWrite sets the Content-Length header to the given size on
+// first write.
+type setSizeOnWrite struct {
+       http.ResponseWriter
+       size  int
+       wrote bool
+}
+
+func (ss *setSizeOnWrite) Write(p []byte) (int, error) {
+       if !ss.wrote {
+               ss.Header().Set("Content-Length", fmt.Sprintf("%d", ss.size))
+               ss.wrote = true
+       }
+       return ss.ResponseWriter.Write(p)
+}
+
+type discardWrite struct {
+       http.ResponseWriter
+}
+
+func (discardWrite) Write(p []byte) (int, error) {
+       return len(p), nil
+}
diff --git a/services/keepstore/router_test.go b/services/keepstore/router_test.go
new file mode 100644 (file)
index 0000000..a729ee0
--- /dev/null
@@ -0,0 +1,510 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "bytes"
+       "context"
+       "crypto/md5"
+       "errors"
+       "fmt"
+       "io"
+       "net/http"
+       "net/http/httptest"
+       "os"
+       "sort"
+       "strings"
+       "time"
+
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/arvadostest"
+       "git.arvados.org/arvados.git/sdk/go/httpserver"
+       "github.com/prometheus/client_golang/prometheus"
+       . "gopkg.in/check.v1"
+)
+
+// routerSuite tests that the router correctly translates HTTP
+// requests to the appropriate keepstore functionality, and translates
+// the results to HTTP responses.
+type routerSuite struct {
+       cluster *arvados.Cluster
+}
+
+var _ = Suite(&routerSuite{})
+
+func testRouter(t TB, cluster *arvados.Cluster, reg *prometheus.Registry) (*router, context.CancelFunc) {
+       if reg == nil {
+               reg = prometheus.NewRegistry()
+       }
+       ctx, cancel := context.WithCancel(context.Background())
+       ks, kcancel := testKeepstore(t, cluster, reg)
+       go func() {
+               <-ctx.Done()
+               kcancel()
+       }()
+       puller := newPuller(ctx, ks, reg)
+       trasher := newTrasher(ctx, ks, reg)
+       return newRouter(ks, puller, trasher).(*router), cancel
+}
+
+func (s *routerSuite) SetUpTest(c *C) {
+       s.cluster = testCluster(c)
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "stub", StorageClasses: map[string]bool{"testclass1": true}},
+               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "stub", StorageClasses: map[string]bool{"testclass2": true}},
+       }
+       s.cluster.StorageClasses = map[string]arvados.StorageClassConfig{
+               "testclass1": arvados.StorageClassConfig{
+                       Default: true,
+               },
+               "testclass2": arvados.StorageClassConfig{
+                       Default: true,
+               },
+       }
+}
+
+func (s *routerSuite) TestBlockRead_Token(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       err := router.keepstore.mountsW[0].BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+       locSigned := router.keepstore.signLocator(arvadostest.ActiveTokenV2, fooHash+"+3")
+       c.Assert(locSigned, Not(Equals), fooHash+"+3")
+
+       // No token provided
+       resp := call(router, "GET", "http://example/"+locSigned, "", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusUnauthorized)
+       c.Check(resp.Body.String(), Matches, "no token provided in Authorization header\n")
+
+       // Different token => invalid signature
+       resp = call(router, "GET", "http://example/"+locSigned, "badtoken", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusBadRequest)
+       c.Check(resp.Body.String(), Equals, "invalid signature\n")
+
+       // Correct token
+       resp = call(router, "GET", "http://example/"+locSigned, arvadostest.ActiveTokenV2, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Equals, "foo")
+
+       // HEAD
+       resp = call(router, "HEAD", "http://example/"+locSigned, arvadostest.ActiveTokenV2, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Result().ContentLength, Equals, int64(3))
+       c.Check(resp.Body.String(), Equals, "")
+}
+
+// As a special case we allow HEAD requests that only provide a hash
+// without a size hint. This accommodates uses of keep-block-check
+// where it's inconvenient to attach size hints to known hashes.
+//
+// GET requests must provide a size hint -- otherwise we can't
+// propagate a checksum mismatch error.
+func (s *routerSuite) TestBlockRead_NoSizeHint(c *C) {
+       s.cluster.Collections.BlobSigning = true
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+       err := router.keepstore.mountsW[0].BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+
+       // hash+signature
+       hashSigned := router.keepstore.signLocator(arvadostest.ActiveTokenV2, fooHash)
+       resp := call(router, "GET", "http://example/"+hashSigned, arvadostest.ActiveTokenV2, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusMethodNotAllowed)
+
+       resp = call(router, "HEAD", "http://example/"+fooHash, "", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusUnauthorized)
+       resp = call(router, "HEAD", "http://example/"+fooHash+"+3", "", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusUnauthorized)
+
+       s.cluster.Collections.BlobSigning = false
+       router, cancel = testRouter(c, s.cluster, nil)
+       defer cancel()
+       err = router.keepstore.mountsW[0].BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+
+       resp = call(router, "GET", "http://example/"+fooHash, "", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusMethodNotAllowed)
+
+       resp = call(router, "HEAD", "http://example/"+fooHash, "", nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Equals, "")
+       c.Check(resp.Result().ContentLength, Equals, int64(3))
+       c.Check(resp.Header().Get("Content-Length"), Equals, "3")
+}
+
+// By the time we discover the checksum mismatch, it's too late to
+// change the response code, but the expected block size is given in
+// the Content-Length response header, so a generic http client can
+// detect the problem.
+func (s *routerSuite) TestBlockRead_ChecksumMismatch(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       gooddata := make([]byte, 10_000_000)
+       gooddata[0] = 'a'
+       hash := fmt.Sprintf("%x", md5.Sum(gooddata))
+       locSigned := router.keepstore.signLocator(arvadostest.ActiveTokenV2, fmt.Sprintf("%s+%d", hash, len(gooddata)))
+
+       for _, baddata := range [][]byte{
+               make([]byte, 3),
+               make([]byte, len(gooddata)),
+               make([]byte, len(gooddata)-1),
+               make([]byte, len(gooddata)+1),
+               make([]byte, len(gooddata)*2),
+       } {
+               c.Logf("=== baddata len %d", len(baddata))
+               err := router.keepstore.mountsW[0].BlockWrite(context.Background(), hash, baddata)
+               c.Assert(err, IsNil)
+
+               resp := call(router, "GET", "http://example/"+locSigned, arvadostest.ActiveTokenV2, nil, nil)
+               if !c.Check(resp.Code, Equals, http.StatusOK) {
+                       c.Logf("resp.Body: %s", resp.Body.String())
+               }
+               c.Check(resp.Body.Len(), Not(Equals), len(gooddata))
+               c.Check(resp.Result().ContentLength, Equals, int64(len(gooddata)))
+
+               resp = call(router, "HEAD", "http://example/"+locSigned, arvadostest.ActiveTokenV2, nil, nil)
+               c.Check(resp.Code, Equals, http.StatusBadGateway)
+
+               hashSigned := router.keepstore.signLocator(arvadostest.ActiveTokenV2, hash)
+               resp = call(router, "HEAD", "http://example/"+hashSigned, arvadostest.ActiveTokenV2, nil, nil)
+               c.Check(resp.Code, Equals, http.StatusBadGateway)
+       }
+}
+
+func (s *routerSuite) TestBlockWrite(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       resp := call(router, "PUT", "http://example/"+fooHash, arvadostest.ActiveTokenV2, []byte("foo"), nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       locator := strings.TrimSpace(resp.Body.String())
+
+       resp = call(router, "GET", "http://example/"+locator, arvadostest.ActiveTokenV2, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Equals, "foo")
+}
+
+func (s *routerSuite) TestBlockWrite_Headers(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       resp := call(router, "PUT", "http://example/"+fooHash, arvadostest.ActiveTokenV2, []byte("foo"), http.Header{"X-Arvados-Replicas-Desired": []string{"2"}})
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Header().Get("X-Keep-Replicas-Stored"), Equals, "1")
+       c.Check(sortCommaSeparated(resp.Header().Get("X-Keep-Storage-Classes-Confirmed")), Equals, "testclass1=1")
+
+       resp = call(router, "PUT", "http://example/"+fooHash, arvadostest.ActiveTokenV2, []byte("foo"), http.Header{"X-Keep-Storage-Classes": []string{"testclass1"}})
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Header().Get("X-Keep-Replicas-Stored"), Equals, "1")
+       c.Check(resp.Header().Get("X-Keep-Storage-Classes-Confirmed"), Equals, "testclass1=1")
+
+       resp = call(router, "PUT", "http://example/"+fooHash, arvadostest.ActiveTokenV2, []byte("foo"), http.Header{"X-Keep-Storage-Classes": []string{" , testclass2 , "}})
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Header().Get("X-Keep-Replicas-Stored"), Equals, "1")
+       c.Check(resp.Header().Get("X-Keep-Storage-Classes-Confirmed"), Equals, "testclass2=1")
+}
+
+func sortCommaSeparated(s string) string {
+       slice := strings.Split(s, ", ")
+       sort.Strings(slice)
+       return strings.Join(slice, ", ")
+}
+
+func (s *routerSuite) TestBlockTouch(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       resp := call(router, "TOUCH", "http://example/"+fooHash+"+3", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusNotFound)
+
+       vol0 := router.keepstore.mountsW[0].volume.(*stubVolume)
+       err := vol0.BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+       vol1 := router.keepstore.mountsW[1].volume.(*stubVolume)
+       err = vol1.BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+
+       t1 := time.Now()
+       resp = call(router, "TOUCH", "http://example/"+fooHash+"+3", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       t2 := time.Now()
+
+       // Unauthorized request is a no-op
+       resp = call(router, "TOUCH", "http://example/"+fooHash+"+3", arvadostest.ActiveTokenV2, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusForbidden)
+
+       // Volume 0 mtime should be updated
+       t, err := vol0.Mtime(fooHash)
+       c.Check(err, IsNil)
+       c.Check(t.After(t1), Equals, true)
+       c.Check(t.Before(t2), Equals, true)
+
+       // Volume 1 mtime should not be updated
+       t, err = vol1.Mtime(fooHash)
+       c.Check(err, IsNil)
+       c.Check(t.Before(t1), Equals, true)
+
+       err = vol0.BlockTrash(fooHash)
+       c.Assert(err, IsNil)
+       err = vol1.BlockTrash(fooHash)
+       c.Assert(err, IsNil)
+       resp = call(router, "TOUCH", "http://example/"+fooHash+"+3", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusNotFound)
+}
+
+func (s *routerSuite) TestBlockTrash(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       vol0 := router.keepstore.mountsW[0].volume.(*stubVolume)
+       err := vol0.BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+       err = vol0.blockTouchWithTime(fooHash, time.Now().Add(-s.cluster.Collections.BlobSigningTTL.Duration()))
+       c.Assert(err, IsNil)
+       resp := call(router, "DELETE", "http://example/"+fooHash+"+3", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(vol0.stubLog.String(), Matches, `(?ms).* trash .*`)
+       _, err = vol0.BlockRead(context.Background(), fooHash, io.Discard)
+       c.Assert(err, Equals, os.ErrNotExist)
+}
+
+func (s *routerSuite) TestBlockUntrash(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       vol0 := router.keepstore.mountsW[0].volume.(*stubVolume)
+       err := vol0.BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+       err = vol0.BlockTrash(fooHash)
+       c.Assert(err, IsNil)
+       _, err = vol0.BlockRead(context.Background(), fooHash, io.Discard)
+       c.Assert(err, Equals, os.ErrNotExist)
+       resp := call(router, "PUT", "http://example/untrash/"+fooHash+"+3", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(vol0.stubLog.String(), Matches, `(?ms).* untrash .*`)
+       _, err = vol0.BlockRead(context.Background(), fooHash, io.Discard)
+       c.Check(err, IsNil)
+}
+
+func (s *routerSuite) TestBadRequest(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       for _, trial := range []string{
+               "GET /",
+               "GET /xyz",
+               "GET /aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaabcdefg",
+               "GET /untrash",
+               "GET /mounts/blocks/123",
+               "GET /trash",
+               "GET /pull",
+               "GET /debug.json",
+               "GET /status.json",
+               "POST /",
+               "POST /aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
+               "POST /trash",
+               "PROPFIND /",
+               "MAKE-COFFEE /",
+       } {
+               c.Logf("=== %s", trial)
+               methodpath := strings.Split(trial, " ")
+               req := httptest.NewRequest(methodpath[0], "http://example"+methodpath[1], nil)
+               resp := httptest.NewRecorder()
+               router.ServeHTTP(resp, req)
+               c.Check(resp.Code, Equals, http.StatusBadRequest)
+       }
+}
+
+func (s *routerSuite) TestRequireAdminMgtToken(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       for _, token := range []string{"badtoken", ""} {
+               for _, trial := range []string{
+                       "PUT /pull",
+                       "PUT /trash",
+                       "GET /index",
+                       "GET /index/",
+                       "GET /index/1234",
+                       "PUT /untrash/aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa",
+               } {
+                       c.Logf("=== %s", trial)
+                       methodpath := strings.Split(trial, " ")
+                       req := httptest.NewRequest(methodpath[0], "http://example"+methodpath[1], nil)
+                       if token != "" {
+                               req.Header.Set("Authorization", "Bearer "+token)
+                       }
+                       resp := httptest.NewRecorder()
+                       router.ServeHTTP(resp, req)
+                       if token == "" {
+                               c.Check(resp.Code, Equals, http.StatusUnauthorized)
+                       } else {
+                               c.Check(resp.Code, Equals, http.StatusForbidden)
+                       }
+               }
+       }
+       req := httptest.NewRequest("TOUCH", "http://example/aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa", nil)
+       resp := httptest.NewRecorder()
+       router.ServeHTTP(resp, req)
+       c.Check(resp.Code, Equals, http.StatusUnauthorized)
+}
+
+func (s *routerSuite) TestVolumeErrorStatusCode(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+       router.keepstore.mountsW[0].volume.(*stubVolume).blockRead = func(_ context.Context, hash string, w io.Writer) (int, error) {
+               return 0, httpserver.ErrorWithStatus(errors.New("test error"), http.StatusBadGateway)
+       }
+
+       // To test whether we fall back to volume 1 after volume 0
+       // returns an error, we need to use a block whose rendezvous
+       // order has volume 0 first. Luckily "bar" is such a block.
+       c.Assert(router.keepstore.rendezvous(barHash, router.keepstore.mountsR)[0].UUID, DeepEquals, router.keepstore.mountsR[0].UUID)
+
+       locSigned := router.keepstore.signLocator(arvadostest.ActiveTokenV2, barHash+"+3")
+
+       // Volume 0 fails with an error that specifies an HTTP status
+       // code, so that code should be propagated to caller.
+       resp := call(router, "GET", "http://example/"+locSigned, arvadostest.ActiveTokenV2, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusBadGateway)
+       c.Check(resp.Body.String(), Equals, "test error\n")
+
+       c.Assert(router.keepstore.mountsW[1].volume.BlockWrite(context.Background(), barHash, []byte("bar")), IsNil)
+
+       // If the requested block is available on the second volume,
+       // it doesn't matter that the first volume failed.
+       resp = call(router, "GET", "http://example/"+locSigned, arvadostest.ActiveTokenV2, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Equals, "bar")
+}
+
+func (s *routerSuite) TestIndex(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       resp := call(router, "GET", "http://example/index", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Equals, "\n")
+
+       resp = call(router, "GET", "http://example/index?prefix=fff", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(resp.Body.String(), Equals, "\n")
+
+       t0 := time.Now().Add(-time.Hour)
+       vol0 := router.keepstore.mounts["zzzzz-nyw5e-000000000000000"].volume.(*stubVolume)
+       err := vol0.BlockWrite(context.Background(), fooHash, []byte("foo"))
+       c.Assert(err, IsNil)
+       err = vol0.blockTouchWithTime(fooHash, t0)
+       c.Assert(err, IsNil)
+       err = vol0.BlockWrite(context.Background(), barHash, []byte("bar"))
+       c.Assert(err, IsNil)
+       err = vol0.blockTouchWithTime(barHash, t0)
+       c.Assert(err, IsNil)
+       t1 := time.Now().Add(-time.Minute)
+       vol1 := router.keepstore.mounts["zzzzz-nyw5e-111111111111111"].volume.(*stubVolume)
+       err = vol1.BlockWrite(context.Background(), barHash, []byte("bar"))
+       c.Assert(err, IsNil)
+       err = vol1.blockTouchWithTime(barHash, t1)
+       c.Assert(err, IsNil)
+
+       for _, path := range []string{
+               "/index?prefix=acb",
+               "/index/acb",
+               "/index/?prefix=acb",
+               "/mounts/zzzzz-nyw5e-000000000000000/blocks?prefix=acb",
+               "/mounts/zzzzz-nyw5e-000000000000000/blocks/?prefix=acb",
+               "/mounts/zzzzz-nyw5e-000000000000000/blocks/acb",
+       } {
+               c.Logf("=== %s", path)
+               resp = call(router, "GET", "http://example"+path, s.cluster.SystemRootToken, nil, nil)
+               c.Check(resp.Code, Equals, http.StatusOK)
+               c.Check(resp.Body.String(), Equals, fooHash+"+3 "+fmt.Sprintf("%d", t0.UnixNano())+"\n\n")
+       }
+
+       for _, path := range []string{
+               "/index?prefix=37",
+               "/index/37",
+               "/index/?prefix=37",
+       } {
+               c.Logf("=== %s", path)
+               resp = call(router, "GET", "http://example"+path, s.cluster.SystemRootToken, nil, nil)
+               c.Check(resp.Code, Equals, http.StatusOK)
+               c.Check(resp.Body.String(), Equals, ""+
+                       barHash+"+3 "+fmt.Sprintf("%d", t0.UnixNano())+"\n"+
+                       barHash+"+3 "+fmt.Sprintf("%d", t1.UnixNano())+"\n\n")
+       }
+
+       for _, path := range []string{
+               "/mounts/zzzzz-nyw5e-111111111111111/blocks",
+               "/mounts/zzzzz-nyw5e-111111111111111/blocks/",
+               "/mounts/zzzzz-nyw5e-111111111111111/blocks?prefix=37",
+               "/mounts/zzzzz-nyw5e-111111111111111/blocks/?prefix=37",
+               "/mounts/zzzzz-nyw5e-111111111111111/blocks/37",
+       } {
+               c.Logf("=== %s", path)
+               resp = call(router, "GET", "http://example"+path, s.cluster.SystemRootToken, nil, nil)
+               c.Check(resp.Code, Equals, http.StatusOK)
+               c.Check(resp.Body.String(), Equals, barHash+"+3 "+fmt.Sprintf("%d", t1.UnixNano())+"\n\n")
+       }
+
+       for _, path := range []string{
+               "/index",
+               "/index?prefix=",
+               "/index/",
+               "/index/?prefix=",
+       } {
+               c.Logf("=== %s", path)
+               resp = call(router, "GET", "http://example"+path, s.cluster.SystemRootToken, nil, nil)
+               c.Check(resp.Code, Equals, http.StatusOK)
+               c.Check(strings.Split(resp.Body.String(), "\n"), HasLen, 5)
+       }
+
+}
+
+// Check that the context passed to a volume method gets cancelled
+// when the http client hangs up.
+func (s *routerSuite) TestCancelOnDisconnect(c *C) {
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       unblock := make(chan struct{})
+       router.keepstore.mountsW[0].volume.(*stubVolume).blockRead = func(ctx context.Context, hash string, w io.Writer) (int, error) {
+               <-unblock
+               c.Check(ctx.Err(), NotNil)
+               return 0, ctx.Err()
+       }
+       go func() {
+               time.Sleep(time.Second / 10)
+               cancel()
+               close(unblock)
+       }()
+       locSigned := router.keepstore.signLocator(arvadostest.ActiveTokenV2, fooHash+"+3")
+       ctx, cancel := context.WithCancel(context.Background())
+       defer cancel()
+       req, err := http.NewRequestWithContext(ctx, "GET", "http://example/"+locSigned, nil)
+       c.Assert(err, IsNil)
+       req.Header.Set("Authorization", "Bearer "+arvadostest.ActiveTokenV2)
+       resp := httptest.NewRecorder()
+       router.ServeHTTP(resp, req)
+       c.Check(resp.Code, Equals, 499)
+}
+
+func call(handler http.Handler, method, path, tok string, body []byte, hdr http.Header) *httptest.ResponseRecorder {
+       resp := httptest.NewRecorder()
+       req, err := http.NewRequest(method, path, bytes.NewReader(body))
+       if err != nil {
+               panic(err)
+       }
+       for k := range hdr {
+               req.Header.Set(k, hdr.Get(k))
+       }
+       if tok != "" {
+               req.Header.Set("Authorization", "Bearer "+tok)
+       }
+       handler.ServeHTTP(resp, req)
+       return resp
+}
index 18b30f463806f996639579689987502f354d411a..2417bb81496f918f1114814c3acd7fb5c25e1227 100644 (file)
@@ -38,10 +38,14 @@ func init() {
 }
 
 const (
-       s3DefaultReadTimeout    = arvados.Duration(10 * time.Minute)
-       s3DefaultConnectTimeout = arvados.Duration(time.Minute)
-       maxClockSkew            = 600 * time.Second
-       nearlyRFC1123           = "Mon, 2 Jan 2006 15:04:05 GMT"
+       s3DefaultReadTimeout        = arvados.Duration(10 * time.Minute)
+       s3DefaultConnectTimeout     = arvados.Duration(time.Minute)
+       maxClockSkew                = 600 * time.Second
+       nearlyRFC1123               = "Mon, 2 Jan 2006 15:04:05 GMT"
+       s3downloaderPartSize        = 5 * 1024 * 1024
+       s3downloaderReadConcurrency = 13
+       s3uploaderPartSize          = 5 * 1024 * 1024
+       s3uploaderWriteConcurrency  = 5
 )
 
 var (
@@ -54,13 +58,14 @@ type S3AWSVolume struct {
        AuthToken      string    // populated automatically when IAMRole is used
        AuthExpiration time.Time // populated automatically when IAMRole is used
 
-       cluster   *arvados.Cluster
-       volume    arvados.Volume
-       logger    logrus.FieldLogger
-       metrics   *volumeMetricsVecs
-       bucket    *s3AWSbucket
-       region    string
-       startOnce sync.Once
+       cluster    *arvados.Cluster
+       volume     arvados.Volume
+       logger     logrus.FieldLogger
+       metrics    *volumeMetricsVecs
+       bufferPool *bufferPool
+       bucket     *s3AWSbucket
+       region     string
+       startOnce  sync.Once
 }
 
 // s3bucket wraps s3.bucket and counts I/O and API usage stats. The
@@ -73,11 +78,7 @@ type s3AWSbucket struct {
        mu     sync.Mutex
 }
 
-const (
-       PartSize         = 5 * 1024 * 1024
-       ReadConcurrency  = 13
-       WriteConcurrency = 5
-)
+const ()
 
 var s3AWSKeepBlockRegexp = regexp.MustCompile(`^[0-9a-f]{32}$`)
 var s3AWSZeroTime time.Time
@@ -100,13 +101,18 @@ func (v *S3AWSVolume) key(loc string) string {
        }
 }
 
-func newS3AWSVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) {
-       v := &S3AWSVolume{cluster: cluster, volume: volume, metrics: metrics}
-       err := json.Unmarshal(volume.DriverParameters, v)
+func newS3AWSVolume(params newVolumeParams) (volume, error) {
+       v := &S3AWSVolume{
+               cluster:    params.Cluster,
+               volume:     params.ConfigVolume,
+               metrics:    params.MetricsVecs,
+               bufferPool: params.BufferPool,
+       }
+       err := json.Unmarshal(params.ConfigVolume.DriverParameters, v)
        if err != nil {
                return nil, err
        }
-       v.logger = logger.WithField("Volume", v.String())
+       v.logger = params.Logger.WithField("Volume", v.DeviceID())
        return v, v.check("")
 }
 
@@ -225,73 +231,17 @@ func (v *S3AWSVolume) check(ec2metadataHostname string) error {
        }
 
        // Set up prometheus metrics
-       lbls := prometheus.Labels{"device_id": v.GetDeviceID()}
+       lbls := prometheus.Labels{"device_id": v.DeviceID()}
        v.bucket.stats.opsCounters, v.bucket.stats.errCounters, v.bucket.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
 
        return nil
 }
 
-// String implements fmt.Stringer.
-func (v *S3AWSVolume) String() string {
-       return fmt.Sprintf("s3-bucket:%+q", v.Bucket)
-}
-
-// GetDeviceID returns a globally unique ID for the storage bucket.
-func (v *S3AWSVolume) GetDeviceID() string {
+// DeviceID returns a globally unique ID for the storage bucket.
+func (v *S3AWSVolume) DeviceID() string {
        return "s3://" + v.Endpoint + "/" + v.Bucket
 }
 
-// Compare the given data with the stored data.
-func (v *S3AWSVolume) Compare(ctx context.Context, loc string, expect []byte) error {
-       key := v.key(loc)
-       errChan := make(chan error, 1)
-       go func() {
-               _, err := v.head("recent/" + key)
-               errChan <- err
-       }()
-       var err error
-       select {
-       case <-ctx.Done():
-               return ctx.Err()
-       case err = <-errChan:
-       }
-       if err != nil {
-               // Checking for the key itself here would interfere
-               // with future GET requests.
-               //
-               // On AWS, if X doesn't exist, a HEAD or GET request
-               // for X causes X's non-existence to be cached. Thus,
-               // if we test for X, then create X and return a
-               // signature to our client, the client might still get
-               // 404 from all keepstores when trying to read it.
-               //
-               // To avoid this, we avoid doing HEAD X or GET X until
-               // we know X has been written.
-               //
-               // Note that X might exist even though recent/X
-               // doesn't: for example, the response to HEAD recent/X
-               // might itself come from a stale cache. In such
-               // cases, we will return a false negative and
-               // PutHandler might needlessly create another replica
-               // on a different volume. That's not ideal, but it's
-               // better than passing the eventually-consistent
-               // problem on to our clients.
-               return v.translateError(err)
-       }
-
-       input := &s3.GetObjectInput{
-               Bucket: aws.String(v.bucket.bucket),
-               Key:    aws.String(key),
-       }
-
-       req := v.bucket.svc.GetObjectRequest(input)
-       result, err := req.Send(ctx)
-       if err != nil {
-               return v.translateError(err)
-       }
-       return v.translateError(compareReaderWithBuf(ctx, result.Body, expect, loc[:32]))
-}
-
 // EmptyTrash looks for trashed blocks that exceeded BlobTrashLifetime
 // and deletes them from the volume.
 func (v *S3AWSVolume) EmptyTrash() {
@@ -313,7 +263,7 @@ func (v *S3AWSVolume) EmptyTrash() {
                recent, err := v.head("recent/" + key)
                if err != nil && os.IsNotExist(v.translateError(err)) {
                        v.logger.Warnf("EmptyTrash: found trash marker %q but no %q (%s); calling Untrash", *trash.Key, "recent/"+key, err)
-                       err = v.Untrash(loc)
+                       err = v.BlockUntrash(loc)
                        if err != nil {
                                v.logger.WithError(err).Errorf("EmptyTrash: Untrash(%q) failed", loc)
                        }
@@ -334,7 +284,7 @@ func (v *S3AWSVolume) EmptyTrash() {
                                // necessary to avoid starvation.
                                v.logger.Infof("EmptyTrash: detected old race for %q, calling fixRace + Touch", loc)
                                v.fixRace(key)
-                               v.Touch(loc)
+                               v.BlockTouch(loc)
                                return
                        }
                        _, err := v.head(key)
@@ -401,7 +351,7 @@ func (v *S3AWSVolume) EmptyTrash() {
        if err := trashL.Error(); err != nil {
                v.logger.WithError(err).Error("EmptyTrash: lister failed")
        }
-       v.logger.Infof("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.Infof("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)
 }
 
 // fixRace(X) is called when "recent/X" exists but "X" doesn't
@@ -462,55 +412,60 @@ func (v *S3AWSVolume) head(key string) (result *s3.HeadObjectOutput, err error)
        return
 }
 
-// Get a block: copy the block data into buf, and return the number of
-// bytes copied.
-func (v *S3AWSVolume) Get(ctx context.Context, loc string, buf []byte) (int, error) {
-       // Do not use getWithPipe here: the BlockReader interface does not pass
-       // through 'buf []byte', and we don't want to allocate two buffers for each
-       // read request. Instead, use a version of ReadBlock that accepts 'buf []byte'
-       // as an input.
-       key := v.key(loc)
-       count, err := v.readWorker(ctx, key, buf)
-       if err == nil {
-               return count, err
-       }
-
-       err = v.translateError(err)
-       if !os.IsNotExist(err) {
-               return 0, err
-       }
-
-       _, err = v.head("recent/" + key)
-       err = v.translateError(err)
+// BlockRead reads a Keep block that has been stored as a block blob
+// in the S3 bucket.
+func (v *S3AWSVolume) BlockRead(ctx context.Context, hash string, writeTo io.Writer) (int, error) {
+       key := v.key(hash)
+       buf, err := v.bufferPool.GetContext(ctx)
        if err != nil {
-               // If we can't read recent/X, there's no point in
-               // trying fixRace. Give up.
-               return 0, err
-       }
-       if !v.fixRace(key) {
-               err = os.ErrNotExist
                return 0, err
        }
+       defer v.bufferPool.Put(buf)
 
-       count, err = v.readWorker(ctx, key, buf)
+       streamer := newStreamWriterAt(writeTo, 65536, buf)
+       defer streamer.Close()
+       err = v.readWorker(ctx, key, streamer)
        if err != nil {
-               v.logger.Warnf("reading %s after successful fixRace: %s", loc, err)
                err = v.translateError(err)
-               return 0, err
+               if !os.IsNotExist(err) {
+                       return 0, err
+               }
+               if streamer.WroteAt() > 0 {
+                       return 0, errors.New("bug? readWorker returned ErrNotExist after writing to streamer")
+               }
+
+               _, err = v.head("recent/" + key)
+               err = v.translateError(err)
+               if err != nil {
+                       // If we can't read recent/X, there's no point in
+                       // trying fixRace. Give up.
+                       return 0, err
+               }
+               if !v.fixRace(key) {
+                       err = os.ErrNotExist
+                       return 0, err
+               }
+
+               err = v.readWorker(ctx, key, streamer)
+               if err != nil {
+                       v.logger.Warnf("reading %s after successful fixRace: %s", hash, err)
+                       err = v.translateError(err)
+                       return 0, err
+               }
        }
-       return count, err
+       err = streamer.Close()
+       if err != nil {
+               return 0, v.translateError(err)
+       }
+       return streamer.Wrote(), nil
 }
 
-func (v *S3AWSVolume) readWorker(ctx context.Context, key string, buf []byte) (int, error) {
-       awsBuf := aws.NewWriteAtBuffer(buf)
+func (v *S3AWSVolume) readWorker(ctx context.Context, key string, dst io.WriterAt) error {
        downloader := s3manager.NewDownloaderWithClient(v.bucket.svc, func(u *s3manager.Downloader) {
-               u.PartSize = PartSize
-               u.Concurrency = ReadConcurrency
+               u.PartSize = s3downloaderPartSize
+               u.Concurrency = s3downloaderReadConcurrency
        })
-
-       v.logger.Debugf("Partsize: %d; Concurrency: %d\n", downloader.PartSize, downloader.Concurrency)
-
-       count, err := downloader.DownloadWithContext(ctx, awsBuf, &s3.GetObjectInput{
+       count, err := downloader.DownloadWithContext(ctx, dst, &s3.GetObjectInput{
                Bucket: aws.String(v.bucket.bucket),
                Key:    aws.String(key),
        })
@@ -518,7 +473,7 @@ func (v *S3AWSVolume) readWorker(ctx context.Context, key string, buf []byte) (i
        v.bucket.stats.Tick(&v.bucket.stats.Ops, &v.bucket.stats.GetOps)
        v.bucket.stats.TickErr(err)
        v.bucket.stats.TickInBytes(uint64(count))
-       return int(count), v.translateError(err)
+       return v.translateError(err)
 }
 
 func (v *S3AWSVolume) writeObject(ctx context.Context, key string, r io.Reader) error {
@@ -547,10 +502,10 @@ func (v *S3AWSVolume) writeObject(ctx context.Context, key string, r io.Reader)
        // Experimentation indicated that using concurrency 5 yields the best
        // throughput, better than higher concurrency (10 or 13) by ~5%.
        // Defining u.BufferProvider = s3manager.NewBufferedReadSeekerWriteToPool(64 * 1024 * 1024)
-       // is detrimental to througput (minus ~15%).
+       // is detrimental to throughput (minus ~15%).
        uploader := s3manager.NewUploaderWithClient(v.bucket.svc, func(u *s3manager.Uploader) {
-               u.PartSize = PartSize
-               u.Concurrency = WriteConcurrency
+               u.PartSize = s3uploaderPartSize
+               u.Concurrency = s3uploaderWriteConcurrency
        })
 
        // Unlike the goamz S3 driver, we don't need to precompute ContentSHA256:
@@ -571,16 +526,12 @@ func (v *S3AWSVolume) writeObject(ctx context.Context, key string, r io.Reader)
 }
 
 // Put writes a block.
-func (v *S3AWSVolume) Put(ctx context.Context, loc string, block []byte) error {
+func (v *S3AWSVolume) BlockWrite(ctx context.Context, hash string, data []byte) error {
        // Do not use putWithPipe here; we want to pass an io.ReadSeeker to the S3
        // sdk to avoid memory allocation there. See #17339 for more information.
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-
-       rdr := bytes.NewReader(block)
-       r := NewCountingReaderAtSeeker(rdr, v.bucket.stats.TickOutBytes)
-       key := v.key(loc)
+       rdr := bytes.NewReader(data)
+       r := newCountingReaderAtSeeker(rdr, v.bucket.stats.TickOutBytes)
+       key := v.key(hash)
        err := v.writeObject(ctx, key, r)
        if err != nil {
                return err
@@ -675,9 +626,9 @@ func (lister *s3awsLister) pop() (k *s3.Object) {
        return
 }
 
-// IndexTo writes a complete list of locators with the given prefix
+// Index writes a complete list of locators with the given prefix
 // for which Get() can retrieve data.
-func (v *S3AWSVolume) IndexTo(prefix string, writer io.Writer) error {
+func (v *S3AWSVolume) Index(ctx context.Context, prefix string, writer io.Writer) error {
        prefix = v.key(prefix)
        // Use a merge sort to find matching sets of X and recent/X.
        dataL := s3awsLister{
@@ -695,6 +646,9 @@ func (v *S3AWSVolume) IndexTo(prefix string, writer io.Writer) error {
                Stats:    &v.bucket.stats,
        }
        for data, recent := dataL.First(), recentL.First(); data != nil && dataL.Error() == nil; data = dataL.Next() {
+               if ctx.Err() != nil {
+                       return ctx.Err()
+               }
                if *data.Key >= "g" {
                        // Conveniently, "recent/*" and "trash/*" are
                        // lexically greater than all hex-encoded data
@@ -769,28 +723,14 @@ func (v *S3AWSVolume) Mtime(loc string) (time.Time, error) {
        return *resp.LastModified, err
 }
 
-// Status returns a *VolumeStatus representing the current in-use
-// storage capacity and a fake available capacity that doesn't make
-// the volume seem full or nearly-full.
-func (v *S3AWSVolume) Status() *VolumeStatus {
-       return &VolumeStatus{
-               DeviceNum: 1,
-               BytesFree: BlockSize * 1000,
-               BytesUsed: 1,
-       }
-}
-
 // InternalStats returns bucket I/O and API call counters.
 func (v *S3AWSVolume) InternalStats() interface{} {
        return &v.bucket.stats
 }
 
-// Touch sets the timestamp for the given locator to the current time.
-func (v *S3AWSVolume) Touch(loc string) error {
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-       key := v.key(loc)
+// BlockTouch sets the timestamp for the given locator to the current time.
+func (v *S3AWSVolume) BlockTouch(hash string) error {
+       key := v.key(hash)
        _, err := v.head(key)
        err = v.translateError(err)
        if os.IsNotExist(err) && v.fixRace(key) {
@@ -845,10 +785,7 @@ func (b *s3AWSbucket) Del(path string) error {
 }
 
 // Trash a Keep block.
-func (v *S3AWSVolume) Trash(loc string) error {
-       if v.volume.ReadOnly && !v.volume.AllowTrashWhenReadOnly {
-               return MethodDisabledError
-       }
+func (v *S3AWSVolume) BlockTrash(loc string) error {
        if t, err := v.Mtime(loc); err != nil {
                return err
        } else if time.Since(t) < v.cluster.Collections.BlobSigningTTL.Duration() {
@@ -872,9 +809,9 @@ func (v *S3AWSVolume) Trash(loc string) error {
        return v.translateError(v.bucket.Del(key))
 }
 
-// Untrash moves block from trash back into store
-func (v *S3AWSVolume) Untrash(loc string) error {
-       key := v.key(loc)
+// BlockUntrash moves block from trash back into store
+func (v *S3AWSVolume) BlockUntrash(hash string) error {
+       key := v.key(hash)
        err := v.safeCopy(key, "trash/"+key)
        if err != nil {
                return err
index c7e2d485dfc6f793f107947a3340606c993f63ac..f05cbee848742a436073230eac00b767930dd9b5 100644 (file)
@@ -58,7 +58,6 @@ type StubbedS3AWSSuite struct {
        s3server *httptest.Server
        metadata *httptest.Server
        cluster  *arvados.Cluster
-       handler  *handler
        volumes  []*TestableS3AWSVolume
 }
 
@@ -70,33 +69,37 @@ func (s *StubbedS3AWSSuite) SetUpTest(c *check.C) {
                "zzzzz-nyw5e-000000000000000": {Driver: "S3"},
                "zzzzz-nyw5e-111111111111111": {Driver: "S3"},
        }
-       s.handler = &handler{}
 }
 
 func (s *StubbedS3AWSSuite) TestGeneric(c *check.C) {
-       DoGenericVolumeTests(c, false, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
+       DoGenericVolumeTests(c, false, func(t TB, params newVolumeParams) TestableVolume {
                // Use a negative raceWindow so s3test's 1-second
                // timestamp precision doesn't confuse fixRace.
-               return s.newTestableVolume(c, cluster, volume, metrics, -2*time.Second)
+               return s.newTestableVolume(c, params, -2*time.Second)
        })
 }
 
 func (s *StubbedS3AWSSuite) TestGenericReadOnly(c *check.C) {
-       DoGenericVolumeTests(c, true, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               return s.newTestableVolume(c, cluster, volume, metrics, -2*time.Second)
+       DoGenericVolumeTests(c, true, func(t TB, params newVolumeParams) TestableVolume {
+               return s.newTestableVolume(c, params, -2*time.Second)
        })
 }
 
 func (s *StubbedS3AWSSuite) TestGenericWithPrefix(c *check.C) {
-       DoGenericVolumeTests(c, false, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               v := s.newTestableVolume(c, cluster, volume, metrics, -2*time.Second)
+       DoGenericVolumeTests(c, false, func(t TB, params newVolumeParams) TestableVolume {
+               v := s.newTestableVolume(c, params, -2*time.Second)
                v.PrefixLength = 3
                return v
        })
 }
 
 func (s *StubbedS3AWSSuite) TestIndex(c *check.C) {
-       v := s.newTestableVolume(c, s.cluster, arvados.Volume{Replication: 2}, newVolumeMetricsVecs(prometheus.NewRegistry()), 0)
+       v := s.newTestableVolume(c, newVolumeParams{
+               Cluster:      s.cluster,
+               ConfigVolume: arvados.Volume{Replication: 2},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       }, 0)
        v.IndexPageSize = 3
        for i := 0; i < 256; i++ {
                v.PutRaw(fmt.Sprintf("%02x%030x", i, i), []byte{102, 111, 111})
@@ -111,7 +114,7 @@ func (s *StubbedS3AWSSuite) TestIndex(c *check.C) {
                {"abc", 0},
        } {
                buf := new(bytes.Buffer)
-               err := v.IndexTo(spec.prefix, buf)
+               err := v.Index(context.Background(), spec.prefix, buf)
                c.Check(err, check.IsNil)
 
                idx := bytes.SplitAfter(buf.Bytes(), []byte{10})
@@ -146,7 +149,7 @@ func (s *StubbedS3AWSSuite) TestSignature(c *check.C) {
        vol.bucket.svc.ForcePathStyle = true
 
        c.Check(err, check.IsNil)
-       err = vol.Put(context.Background(), "acbd18db4cc2f85cedef654fccc4a4d8", []byte("foo"))
+       err = vol.BlockWrite(context.Background(), "acbd18db4cc2f85cedef654fccc4a4d8", []byte("foo"))
        c.Check(err, check.IsNil)
        c.Check(header.Get("Authorization"), check.Matches, `AWS4-HMAC-SHA256 .*`)
 }
@@ -202,7 +205,12 @@ func (s *StubbedS3AWSSuite) TestIAMRoleCredentials(c *check.C) {
 }
 
 func (s *StubbedS3AWSSuite) TestStats(c *check.C) {
-       v := s.newTestableVolume(c, s.cluster, arvados.Volume{Replication: 2}, newVolumeMetricsVecs(prometheus.NewRegistry()), 5*time.Minute)
+       v := s.newTestableVolume(c, newVolumeParams{
+               Cluster:      s.cluster,
+               ConfigVolume: arvados.Volume{Replication: 2},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       }, 5*time.Minute)
        stats := func() string {
                buf, err := json.Marshal(v.InternalStats())
                c.Check(err, check.IsNil)
@@ -212,20 +220,20 @@ func (s *StubbedS3AWSSuite) TestStats(c *check.C) {
        c.Check(stats(), check.Matches, `.*"Ops":0,.*`)
 
        loc := "acbd18db4cc2f85cedef654fccc4a4d8"
-       _, err := v.Get(context.Background(), loc, make([]byte, 3))
+       _, err := v.BlockRead(context.Background(), loc, io.Discard)
        c.Check(err, check.NotNil)
        c.Check(stats(), check.Matches, `.*"Ops":[^0],.*`)
        c.Check(stats(), check.Matches, `.*"s3.requestFailure 404 NoSuchKey[^"]*":[^0].*`)
        c.Check(stats(), check.Matches, `.*"InBytes":0,.*`)
 
-       err = v.Put(context.Background(), loc, []byte("foo"))
+       err = v.BlockWrite(context.Background(), loc, []byte("foo"))
        c.Check(err, check.IsNil)
        c.Check(stats(), check.Matches, `.*"OutBytes":3,.*`)
        c.Check(stats(), check.Matches, `.*"PutOps":2,.*`)
 
-       _, err = v.Get(context.Background(), loc, make([]byte, 3))
+       _, err = v.BlockRead(context.Background(), loc, io.Discard)
        c.Check(err, check.IsNil)
-       _, err = v.Get(context.Background(), loc, make([]byte, 3))
+       _, err = v.BlockRead(context.Background(), loc, io.Discard)
        c.Check(err, check.IsNil)
        c.Check(stats(), check.Matches, `.*"InBytes":6,.*`)
 }
@@ -251,30 +259,15 @@ func (h *s3AWSBlockingHandler) ServeHTTP(w http.ResponseWriter, r *http.Request)
 }
 
 func (s *StubbedS3AWSSuite) TestGetContextCancel(c *check.C) {
-       loc := "acbd18db4cc2f85cedef654fccc4a4d8"
-       buf := make([]byte, 3)
-
        s.testContextCancel(c, func(ctx context.Context, v *TestableS3AWSVolume) error {
-               _, err := v.Get(ctx, loc, buf)
+               _, err := v.BlockRead(ctx, fooHash, io.Discard)
                return err
        })
 }
 
-func (s *StubbedS3AWSSuite) TestCompareContextCancel(c *check.C) {
-       loc := "acbd18db4cc2f85cedef654fccc4a4d8"
-       buf := []byte("bar")
-
-       s.testContextCancel(c, func(ctx context.Context, v *TestableS3AWSVolume) error {
-               return v.Compare(ctx, loc, buf)
-       })
-}
-
 func (s *StubbedS3AWSSuite) TestPutContextCancel(c *check.C) {
-       loc := "acbd18db4cc2f85cedef654fccc4a4d8"
-       buf := []byte("foo")
-
        s.testContextCancel(c, func(ctx context.Context, v *TestableS3AWSVolume) error {
-               return v.Put(ctx, loc, buf)
+               return v.BlockWrite(ctx, fooHash, []byte("foo"))
        })
 }
 
@@ -283,7 +276,12 @@ func (s *StubbedS3AWSSuite) testContextCancel(c *check.C, testFunc func(context.
        s.s3server = httptest.NewServer(handler)
        defer s.s3server.Close()
 
-       v := s.newTestableVolume(c, s.cluster, arvados.Volume{Replication: 2}, newVolumeMetricsVecs(prometheus.NewRegistry()), 5*time.Minute)
+       v := s.newTestableVolume(c, newVolumeParams{
+               Cluster:      s.cluster,
+               ConfigVolume: arvados.Volume{Replication: 2},
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       }, 5*time.Minute)
 
        ctx, cancel := context.WithCancel(context.Background())
 
@@ -323,7 +321,13 @@ func (s *StubbedS3AWSSuite) TestBackendStates(c *check.C) {
        s.cluster.Collections.BlobTrashLifetime.Set("1h")
        s.cluster.Collections.BlobSigningTTL.Set("1h")
 
-       v := s.newTestableVolume(c, s.cluster, arvados.Volume{Replication: 2}, newVolumeMetricsVecs(prometheus.NewRegistry()), 5*time.Minute)
+       v := s.newTestableVolume(c, newVolumeParams{
+               Cluster:      s.cluster,
+               ConfigVolume: arvados.Volume{Replication: 2},
+               Logger:       ctxlog.TestLogger(c),
+               MetricsVecs:  newVolumeMetricsVecs(prometheus.NewRegistry()),
+               BufferPool:   newBufferPool(ctxlog.TestLogger(c), 8, prometheus.NewRegistry()),
+       }, 5*time.Minute)
        var none time.Time
 
        putS3Obj := func(t time.Time, key string, data []byte) {
@@ -475,8 +479,7 @@ func (s *StubbedS3AWSSuite) TestBackendStates(c *check.C) {
 
                        // Check canGet
                        loc, blk := setupScenario()
-                       buf := make([]byte, len(blk))
-                       _, err := v.Get(context.Background(), loc, buf)
+                       _, err := v.BlockRead(context.Background(), loc, io.Discard)
                        c.Check(err == nil, check.Equals, scenario.canGet)
                        if err != nil {
                                c.Check(os.IsNotExist(err), check.Equals, true)
@@ -484,9 +487,9 @@ func (s *StubbedS3AWSSuite) TestBackendStates(c *check.C) {
 
                        // Call Trash, then check canTrash and canGetAfterTrash
                        loc, _ = setupScenario()
-                       err = v.Trash(loc)
+                       err = v.BlockTrash(loc)
                        c.Check(err == nil, check.Equals, scenario.canTrash)
-                       _, err = v.Get(context.Background(), loc, buf)
+                       _, err = v.BlockRead(context.Background(), loc, io.Discard)
                        c.Check(err == nil, check.Equals, scenario.canGetAfterTrash)
                        if err != nil {
                                c.Check(os.IsNotExist(err), check.Equals, true)
@@ -494,14 +497,14 @@ func (s *StubbedS3AWSSuite) TestBackendStates(c *check.C) {
 
                        // Call Untrash, then check canUntrash
                        loc, _ = setupScenario()
-                       err = v.Untrash(loc)
+                       err = v.BlockUntrash(loc)
                        c.Check(err == nil, check.Equals, scenario.canUntrash)
                        if scenario.dataT != none || scenario.trashT != none {
                                // In all scenarios where the data exists, we
                                // should be able to Get after Untrash --
                                // regardless of timestamps, errors, race
                                // conditions, etc.
-                               _, err = v.Get(context.Background(), loc, buf)
+                               _, err = v.BlockRead(context.Background(), loc, io.Discard)
                                c.Check(err, check.IsNil)
                        }
 
@@ -522,7 +525,7 @@ func (s *StubbedS3AWSSuite) TestBackendStates(c *check.C) {
                        // Check for current Mtime after Put (applies to all
                        // scenarios)
                        loc, blk = setupScenario()
-                       err = v.Put(context.Background(), loc, blk)
+                       err = v.BlockWrite(context.Background(), loc, blk)
                        c.Check(err, check.IsNil)
                        t, err := v.Mtime(loc)
                        c.Check(err, check.IsNil)
@@ -555,7 +558,7 @@ func (l LogrusLog) Print(level gofakes3.LogLevel, v ...interface{}) {
        }
 }
 
-func (s *StubbedS3AWSSuite) newTestableVolume(c *check.C, cluster *arvados.Cluster, volume arvados.Volume, metrics *volumeMetricsVecs, raceWindow time.Duration) *TestableS3AWSVolume {
+func (s *StubbedS3AWSSuite) newTestableVolume(c *check.C, params newVolumeParams, raceWindow time.Duration) *TestableS3AWSVolume {
 
        clock := &s3AWSFakeClock{}
        // fake s3
@@ -591,10 +594,11 @@ func (s *StubbedS3AWSSuite) newTestableVolume(c *check.C, cluster *arvados.Clust
                                UnsafeDelete:       true,
                                IndexPageSize:      1000,
                        },
-                       cluster: cluster,
-                       volume:  volume,
-                       logger:  ctxlog.TestLogger(c),
-                       metrics: metrics,
+                       cluster:    params.Cluster,
+                       volume:     params.ConfigVolume,
+                       logger:     params.Logger,
+                       metrics:    params.MetricsVecs,
+                       bufferPool: params.BufferPool,
                },
                c:           c,
                server:      srv,
@@ -619,7 +623,7 @@ func (s *StubbedS3AWSSuite) newTestableVolume(c *check.C, cluster *arvados.Clust
 // PutRaw skips the ContentMD5 test
 func (v *TestableS3AWSVolume) PutRaw(loc string, block []byte) {
        key := v.key(loc)
-       r := NewCountingReader(bytes.NewReader(block), v.bucket.stats.TickOutBytes)
+       r := newCountingReader(bytes.NewReader(block), v.bucket.stats.TickOutBytes)
 
        uploader := s3manager.NewUploaderWithClient(v.bucket.svc, func(u *s3manager.Uploader) {
                u.PartSize = 5 * 1024 * 1024
diff --git a/services/keepstore/status_test.go b/services/keepstore/status_test.go
deleted file mode 100644 (file)
index 80f98ad..0000000
+++ /dev/null
@@ -1,25 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "encoding/json"
-)
-
-// We don't have isolated unit tests for /status.json yet, but we do
-// check (e.g., in pull_worker_test.go) that /status.json reports
-// specific statistics correctly at the appropriate times.
-
-// getStatusItem("foo","bar","baz") retrieves /status.json, decodes
-// the response body into resp, and returns resp["foo"]["bar"]["baz"].
-func getStatusItem(h *handler, keys ...string) interface{} {
-       resp := IssueRequest(h, &RequestTester{"/status.json", "", "GET", nil, ""})
-       var s interface{}
-       json.NewDecoder(resp.Body).Decode(&s)
-       for _, k := range keys {
-               s = s.(map[string]interface{})[k]
-       }
-       return s
-}
diff --git a/services/keepstore/streamwriterat.go b/services/keepstore/streamwriterat.go
new file mode 100644 (file)
index 0000000..365b55f
--- /dev/null
@@ -0,0 +1,154 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "errors"
+       "fmt"
+       "io"
+       "sync"
+)
+
+// streamWriterAt translates random-access writes to sequential
+// writes. The caller is expected to use an arbitrary sequence of
+// non-overlapping WriteAt calls covering all positions between 0 and
+// N, for any N < len(buf), then call Close.
+//
+// streamWriterAt writes the data to the provided io.Writer in
+// sequential order.
+//
+// Close returns when all data has been written through.
+type streamWriterAt struct {
+       writer     io.Writer
+       buf        []byte
+       partsize   int         // size of each part written through to writer
+       endpos     int         // portion of buf actually used, judging by WriteAt calls so far
+       partfilled []int       // number of bytes written to each part so far
+       partready  chan []byte // parts of buf fully written / waiting for writer goroutine
+       partnext   int         // index of next part we will send to partready when it's ready
+       wroteAt    int         // bytes we copied to buf in WriteAt
+       wrote      int         // bytes successfully written through to writer
+       errWrite   chan error  // final outcome of writer goroutine
+       closed     bool        // streamWriterAt has been closed
+       mtx        sync.Mutex  // guard internal fields during concurrent calls to WriteAt and Close
+}
+
+// newStreamWriterAt creates a new streamWriterAt.
+func newStreamWriterAt(w io.Writer, partsize int, buf []byte) *streamWriterAt {
+       if partsize == 0 {
+               partsize = 65536
+       }
+       nparts := (len(buf) + partsize - 1) / partsize
+       swa := &streamWriterAt{
+               writer:     w,
+               partsize:   partsize,
+               buf:        buf,
+               partfilled: make([]int, nparts),
+               partready:  make(chan []byte, nparts),
+               errWrite:   make(chan error, 1),
+       }
+       go swa.writeToWriter()
+       return swa
+}
+
+// Wrote returns the number of bytes written through to the
+// io.Writer.
+//
+// Wrote must not be called until after Close.
+func (swa *streamWriterAt) Wrote() int {
+       return swa.wrote
+}
+
+// Wrote returns the number of bytes passed to WriteAt, regardless of
+// whether they were written through to the io.Writer.
+func (swa *streamWriterAt) WroteAt() int {
+       swa.mtx.Lock()
+       defer swa.mtx.Unlock()
+       return swa.wroteAt
+}
+
+func (swa *streamWriterAt) writeToWriter() {
+       defer close(swa.errWrite)
+       for p := range swa.partready {
+               n, err := swa.writer.Write(p)
+               if err != nil {
+                       swa.errWrite <- err
+                       return
+               }
+               swa.wrote += n
+       }
+}
+
+// WriteAt implements io.WriterAt.
+func (swa *streamWriterAt) WriteAt(p []byte, offset int64) (int, error) {
+       pos := int(offset)
+       n := 0
+       if pos <= len(swa.buf) {
+               n = copy(swa.buf[pos:], p)
+       }
+       if n < len(p) {
+               return n, fmt.Errorf("write beyond end of buffer: offset %d len %d buf %d", offset, len(p), len(swa.buf))
+       }
+       endpos := pos + n
+
+       swa.mtx.Lock()
+       defer swa.mtx.Unlock()
+       swa.wroteAt += len(p)
+       if swa.endpos < endpos {
+               swa.endpos = endpos
+       }
+       if swa.closed {
+               return 0, errors.New("invalid use of closed streamWriterAt")
+       }
+       // Track the number of bytes that landed in each of our
+       // (output) parts.
+       for i := pos; i < endpos; {
+               j := i + swa.partsize - (i % swa.partsize)
+               if j > endpos {
+                       j = endpos
+               }
+               pf := swa.partfilled[i/swa.partsize]
+               pf += j - i
+               if pf > swa.partsize {
+                       return 0, errors.New("streamWriterAt: overlapping WriteAt calls")
+               }
+               swa.partfilled[i/swa.partsize] = pf
+               i = j
+       }
+       // Flush filled parts to partready.
+       for swa.partnext < len(swa.partfilled) && swa.partfilled[swa.partnext] == swa.partsize {
+               offset := swa.partnext * swa.partsize
+               swa.partready <- swa.buf[offset : offset+swa.partsize]
+               swa.partnext++
+       }
+       return len(p), nil
+}
+
+// Close flushes all buffered data through to the io.Writer.
+func (swa *streamWriterAt) Close() error {
+       swa.mtx.Lock()
+       defer swa.mtx.Unlock()
+       if swa.closed {
+               return errors.New("invalid use of closed streamWriterAt")
+       }
+       swa.closed = true
+       // Flush last part if needed. If the input doesn't end on a
+       // part boundary, the last part never appears "filled" when we
+       // check in WriteAt.  But here, we know endpos is the end of
+       // the stream, so we can check whether the last part is ready.
+       if offset := swa.partnext * swa.partsize; offset < swa.endpos && offset+swa.partfilled[swa.partnext] == swa.endpos {
+               swa.partready <- swa.buf[offset:swa.endpos]
+               swa.partnext++
+       }
+       close(swa.partready)
+       err := <-swa.errWrite
+       if err != nil {
+               return err
+       }
+       if swa.wrote != swa.wroteAt {
+               return fmt.Errorf("streamWriterAt: detected hole in input: wrote %d but flushed %d", swa.wroteAt, swa.wrote)
+       }
+       return nil
+}
diff --git a/services/keepstore/streamwriterat_test.go b/services/keepstore/streamwriterat_test.go
new file mode 100644 (file)
index 0000000..fe6837e
--- /dev/null
@@ -0,0 +1,83 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package keepstore
+
+import (
+       "bytes"
+       "sync"
+
+       . "gopkg.in/check.v1"
+)
+
+var _ = Suite(&streamWriterAtSuite{})
+
+type streamWriterAtSuite struct{}
+
+func (s *streamWriterAtSuite) TestPartSizes(c *C) {
+       for partsize := 1; partsize < 5; partsize++ {
+               for writesize := 1; writesize < 5; writesize++ {
+                       for datasize := 1; datasize < 100; datasize += 13 {
+                               for bufextra := 0; bufextra < 5; bufextra++ {
+                                       c.Logf("=== partsize %d writesize %d datasize %d bufextra %d", partsize, writesize, datasize, bufextra)
+                                       outbuf := bytes.NewBuffer(nil)
+                                       indata := make([]byte, datasize)
+                                       for i := range indata {
+                                               indata[i] = byte(i)
+                                       }
+                                       swa := newStreamWriterAt(outbuf, partsize, make([]byte, datasize+bufextra))
+                                       var wg sync.WaitGroup
+                                       for pos := 0; pos < datasize; pos += writesize {
+                                               pos := pos
+                                               wg.Add(1)
+                                               go func() {
+                                                       defer wg.Done()
+                                                       endpos := pos + writesize
+                                                       if endpos > datasize {
+                                                               endpos = datasize
+                                                       }
+                                                       swa.WriteAt(indata[pos:endpos], int64(pos))
+                                               }()
+                                       }
+                                       wg.Wait()
+                                       swa.Close()
+                                       c.Check(outbuf.Bytes(), DeepEquals, indata)
+                               }
+                       }
+               }
+       }
+}
+
+func (s *streamWriterAtSuite) TestOverflow(c *C) {
+       for offset := -1; offset < 2; offset++ {
+               buf := make([]byte, 50)
+               swa := newStreamWriterAt(bytes.NewBuffer(nil), 20, buf)
+               _, err := swa.WriteAt([]byte("foo"), int64(len(buf)+offset))
+               c.Check(err, NotNil)
+               err = swa.Close()
+               c.Check(err, IsNil)
+       }
+}
+
+func (s *streamWriterAtSuite) TestIncompleteWrite(c *C) {
+       for _, partsize := range []int{20, 25} {
+               for _, bufsize := range []int{50, 55, 60} {
+                       for offset := 0; offset < 3; offset++ {
+                               swa := newStreamWriterAt(bytes.NewBuffer(nil), partsize, make([]byte, bufsize))
+                               _, err := swa.WriteAt(make([]byte, 1), 49)
+                               c.Check(err, IsNil)
+                               _, err = swa.WriteAt(make([]byte, 46), int64(offset))
+                               c.Check(err, IsNil)
+                               err = swa.Close()
+                               c.Check(err, NotNil)
+                               c.Check(swa.WroteAt(), Equals, 47)
+                               if offset == 0 {
+                                       c.Check(swa.Wrote(), Equals, 40/partsize*partsize)
+                               } else {
+                                       c.Check(swa.Wrote(), Equals, 0)
+                               }
+                       }
+               }
+       }
+}
index 5e8a5a963ceaad37527e652ca19460dce349fcd4..d704c3a7d5d821713a14d75b403b67326966caa7 100644 (file)
 package keepstore
 
 import (
-       "errors"
+       "context"
+       "sync"
+       "sync/atomic"
        "time"
 
        "git.arvados.org/arvados.git/sdk/go/arvados"
-       "github.com/sirupsen/logrus"
+       "github.com/prometheus/client_golang/prometheus"
 )
 
-// RunTrashWorker processes the trash request queue.
-func RunTrashWorker(volmgr *RRVolumeManager, logger logrus.FieldLogger, cluster *arvados.Cluster, trashq *WorkQueue) {
-       for item := range trashq.NextItem {
-               trashRequest := item.(TrashRequest)
-               TrashItem(volmgr, logger, cluster, trashRequest)
-               trashq.DoneItem <- struct{}{}
-       }
+type TrashListItem struct {
+       Locator    string `json:"locator"`
+       BlockMtime int64  `json:"block_mtime"`
+       MountUUID  string `json:"mount_uuid"` // Target mount, or "" for "everywhere"
+}
+
+type trasher struct {
+       keepstore  *keepstore
+       todo       []TrashListItem
+       cond       *sync.Cond // lock guards todo accesses; cond broadcasts when todo becomes non-empty
+       inprogress atomic.Int64
 }
 
-// TrashItem deletes the indicated block from every writable volume.
-func TrashItem(volmgr *RRVolumeManager, logger logrus.FieldLogger, cluster *arvados.Cluster, trashRequest TrashRequest) {
-       reqMtime := time.Unix(0, trashRequest.BlockMtime)
-       if time.Since(reqMtime) < cluster.Collections.BlobSigningTTL.Duration() {
-               logger.Warnf("client asked to delete a %v old block %v (BlockMtime %d = %v), but my blobSignatureTTL is %v! Skipping.",
-                       arvados.Duration(time.Since(reqMtime)),
-                       trashRequest.Locator,
-                       trashRequest.BlockMtime,
-                       reqMtime,
-                       cluster.Collections.BlobSigningTTL)
-               return
+func newTrasher(ctx context.Context, keepstore *keepstore, reg *prometheus.Registry) *trasher {
+       t := &trasher{
+               keepstore: keepstore,
+               cond:      sync.NewCond(&sync.Mutex{}),
+       }
+       reg.MustRegister(prometheus.NewGaugeFunc(
+               prometheus.GaugeOpts{
+                       Namespace: "arvados",
+                       Subsystem: "keepstore",
+                       Name:      "trash_queue_pending_entries",
+                       Help:      "Number of queued trash requests",
+               },
+               func() float64 {
+                       t.cond.L.Lock()
+                       defer t.cond.L.Unlock()
+                       return float64(len(t.todo))
+               },
+       ))
+       reg.MustRegister(prometheus.NewGaugeFunc(
+               prometheus.GaugeOpts{
+                       Namespace: "arvados",
+                       Subsystem: "keepstore",
+                       Name:      "trash_queue_inprogress_entries",
+                       Help:      "Number of trash requests in progress",
+               },
+               func() float64 {
+                       return float64(t.inprogress.Load())
+               },
+       ))
+       if !keepstore.cluster.Collections.BlobTrash {
+               keepstore.logger.Info("not running trash worker because Collections.BlobTrash == false")
+               return t
        }
 
-       var volumes []*VolumeMount
-       if uuid := trashRequest.MountUUID; uuid == "" {
-               volumes = volmgr.Mounts()
-       } else if mnt := volmgr.Lookup(uuid, false); mnt == nil {
-               logger.Warnf("trash request for nonexistent mount: %v", trashRequest)
-               return
-       } else if !mnt.KeepMount.AllowTrash {
-               logger.Warnf("trash request for mount with ReadOnly=true, AllowTrashWhenReadOnly=false: %v", trashRequest)
+       var mntsAllowTrash []*mount
+       for _, mnt := range t.keepstore.mounts {
+               if mnt.AllowTrash {
+                       mntsAllowTrash = append(mntsAllowTrash, mnt)
+               }
+       }
+       if len(mntsAllowTrash) == 0 {
+               t.keepstore.logger.Info("not running trash worker because there are no writable or trashable volumes")
        } else {
-               volumes = []*VolumeMount{mnt}
+               for i := 0; i < keepstore.cluster.Collections.BlobTrashConcurrency; i++ {
+                       go t.runWorker(ctx, mntsAllowTrash)
+               }
        }
+       return t
+}
+
+func (t *trasher) SetTrashList(newlist []TrashListItem) {
+       t.cond.L.Lock()
+       t.todo = newlist
+       t.cond.L.Unlock()
+       t.cond.Broadcast()
+}
 
-       for _, volume := range volumes {
-               mtime, err := volume.Mtime(trashRequest.Locator)
-               if err != nil {
-                       logger.WithError(err).Errorf("%v Trash(%v)", volume, trashRequest.Locator)
-                       continue
+func (t *trasher) runWorker(ctx context.Context, mntsAllowTrash []*mount) {
+       go func() {
+               <-ctx.Done()
+               t.cond.Broadcast()
+       }()
+       for {
+               t.cond.L.Lock()
+               for len(t.todo) == 0 && ctx.Err() == nil {
+                       t.cond.Wait()
                }
-               if trashRequest.BlockMtime != mtime.UnixNano() {
-                       logger.Infof("%v Trash(%v): stored mtime %v does not match trash list value %v; skipping", volume, trashRequest.Locator, mtime.UnixNano(), trashRequest.BlockMtime)
-                       continue
+               if ctx.Err() != nil {
+                       t.cond.L.Unlock()
+                       return
                }
+               item := t.todo[0]
+               t.todo = t.todo[1:]
+               t.inprogress.Add(1)
+               t.cond.L.Unlock()
 
-               if !cluster.Collections.BlobTrash {
-                       err = errors.New("skipping because Collections.BlobTrash is false")
-               } else {
-                       err = volume.Trash(trashRequest.Locator)
-               }
+               func() {
+                       defer t.inprogress.Add(-1)
+                       logger := t.keepstore.logger.WithField("locator", item.Locator)
 
-               if err != nil {
-                       logger.WithError(err).Errorf("%v Trash(%v)", volume, trashRequest.Locator)
-               } else {
-                       logger.Infof("%v Trash(%v) OK", volume, trashRequest.Locator)
-               }
+                       li, err := parseLocator(item.Locator)
+                       if err != nil {
+                               logger.Warn("ignoring trash request for invalid locator")
+                               return
+                       }
+
+                       reqMtime := time.Unix(0, item.BlockMtime)
+                       if time.Since(reqMtime) < t.keepstore.cluster.Collections.BlobSigningTTL.Duration() {
+                               logger.Warnf("client asked to delete a %v old block (BlockMtime %d = %v), but my blobSignatureTTL is %v! Skipping.",
+                                       arvados.Duration(time.Since(reqMtime)),
+                                       item.BlockMtime,
+                                       reqMtime,
+                                       t.keepstore.cluster.Collections.BlobSigningTTL)
+                               return
+                       }
+
+                       var mnts []*mount
+                       if item.MountUUID == "" {
+                               mnts = mntsAllowTrash
+                       } else if mnt := t.keepstore.mounts[item.MountUUID]; mnt == nil {
+                               logger.Warnf("ignoring trash request for nonexistent mount %s", item.MountUUID)
+                               return
+                       } else if !mnt.AllowTrash {
+                               logger.Warnf("ignoring trash request for readonly mount %s with AllowTrashWhenReadOnly==false", item.MountUUID)
+                               return
+                       } else {
+                               mnts = []*mount{mnt}
+                       }
+
+                       for _, mnt := range mnts {
+                               logger := logger.WithField("mount", mnt.UUID)
+                               mtime, err := mnt.Mtime(li.hash)
+                               if err != nil {
+                                       logger.WithError(err).Error("error getting stored mtime")
+                                       continue
+                               }
+                               if !mtime.Equal(reqMtime) {
+                                       logger.Infof("stored mtime (%v) does not match trash list mtime (%v); skipping", mtime, reqMtime)
+                                       continue
+                               }
+                               err = mnt.BlockTrash(li.hash)
+                               if err != nil {
+                                       logger.WithError(err).Info("error trashing block")
+                                       continue
+                               }
+                               logger.Info("block trashed")
+                       }
+               }()
        }
 }
+
+type trashEmptier struct{}
+
+func newTrashEmptier(ctx context.Context, ks *keepstore, reg *prometheus.Registry) *trashEmptier {
+       d := ks.cluster.Collections.BlobTrashCheckInterval.Duration()
+       if d <= 0 ||
+               !ks.cluster.Collections.BlobTrash ||
+               ks.cluster.Collections.BlobDeleteConcurrency <= 0 {
+               ks.logger.Infof("not running trash emptier because disabled by config (enabled=%t, interval=%v, concurrency=%d)", ks.cluster.Collections.BlobTrash, d, ks.cluster.Collections.BlobDeleteConcurrency)
+               return &trashEmptier{}
+       }
+       go func() {
+               ticker := time.NewTicker(d)
+               for {
+                       select {
+                       case <-ctx.Done():
+                               return
+                       case <-ticker.C:
+                       }
+                       for _, mnt := range ks.mounts {
+                               if mnt.KeepMount.AllowTrash {
+                                       mnt.volume.EmptyTrash()
+                               }
+                       }
+               }
+       }()
+       return &trashEmptier{}
+}
index a1648c52cc9312b65339a348a94c306a9d5c1c29..0c304dbadec5498d8f736bb83cfeab88cbda6de4 100644 (file)
 package keepstore
 
 import (
-       "container/list"
        "context"
+       "crypto/md5"
+       "encoding/json"
+       "fmt"
+       "net/http"
+       "sort"
        "time"
 
-       "git.arvados.org/arvados.git/sdk/go/ctxlog"
-       "github.com/prometheus/client_golang/prometheus"
-       check "gopkg.in/check.v1"
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       . "gopkg.in/check.v1"
 )
 
-type TrashWorkerTestData struct {
-       Locator1    string
-       Block1      []byte
-       BlockMtime1 int64
-
-       Locator2    string
-       Block2      []byte
-       BlockMtime2 int64
-
-       CreateData      bool
-       CreateInVolume1 bool
-
-       UseTrashLifeTime bool
-       DifferentMtimes  bool
-
-       DeleteLocator    string
-       SpecifyMountUUID bool
-
-       ExpectLocator1 bool
-       ExpectLocator2 bool
-}
-
-// Delete block that does not exist in any of the keep volumes.
-// Expect no errors.
-func (s *HandlerSuite) TestTrashWorkerIntegration_GetNonExistingLocator(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: "5d41402abc4b2a76b9719d911017c592",
-               Block1:   []byte("hello"),
-
-               Locator2: "5d41402abc4b2a76b9719d911017c592",
-               Block2:   []byte("hello"),
-
-               CreateData: false,
-
-               DeleteLocator: "5d41402abc4b2a76b9719d911017c592",
-
-               ExpectLocator1: false,
-               ExpectLocator2: false,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Delete a block that exists on volume 1 of the keep servers. Expect
-// the second locator in volume 2 to be unaffected.
-func (s *HandlerSuite) TestTrashWorkerIntegration_LocatorInVolume1(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash2,
-               Block2:   TestBlock2,
-
-               CreateData: true,
-
-               DeleteLocator: TestHash, // first locator
-
-               ExpectLocator1: false,
-               ExpectLocator2: true,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Delete a block that exists on volume 2 of the keep servers. Expect
-// the first locator in volume 1 to be unaffected.
-func (s *HandlerSuite) TestTrashWorkerIntegration_LocatorInVolume2(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash2,
-               Block2:   TestBlock2,
-
-               CreateData: true,
-
-               DeleteLocator: TestHash2, // locator 2
-
-               ExpectLocator1: true,
-               ExpectLocator2: false,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Delete a block with matching mtime for locator in both
-// volumes. Expect locator to be deleted from both volumes.
-func (s *HandlerSuite) TestTrashWorkerIntegration_LocatorInBothVolumes(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash,
-               Block2:   TestBlock,
-
-               CreateData: true,
-
-               DeleteLocator: TestHash,
-
-               ExpectLocator1: false,
-               ExpectLocator2: false,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Same locator with different Mtimes exists in both volumes. Delete
-// the second and expect the first to be still around.
-func (s *HandlerSuite) TestTrashWorkerIntegration_MtimeMatchesForLocator1ButNotForLocator2(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash,
-               Block2:   TestBlock,
-
-               CreateData:      true,
-               DifferentMtimes: true,
-
-               DeleteLocator: TestHash,
-
-               ExpectLocator1: true,
-               ExpectLocator2: false,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Delete a block that exists on both volumes with matching mtimes,
-// but specify a MountUUID in the request so it only gets deleted from
-// the first volume.
-func (s *HandlerSuite) TestTrashWorkerIntegration_SpecifyMountUUID(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash,
-               Block2:   TestBlock,
-
-               CreateData: true,
-
-               DeleteLocator:    TestHash,
-               SpecifyMountUUID: true,
-
-               ExpectLocator1: true,
-               ExpectLocator2: true,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Two different locators in volume 1. Delete one of them. Expect the
-// other unaffected.
-func (s *HandlerSuite) TestTrashWorkerIntegration_TwoDifferentLocatorsInVolume1(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash2,
-               Block2:   TestBlock2,
-
-               CreateData:      true,
-               CreateInVolume1: true,
-
-               DeleteLocator: TestHash, // locator 1
-
-               ExpectLocator1: false,
-               ExpectLocator2: true,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Allow default Trash Life time to be used. Thus, the newly created
-// block will not be deleted because its Mtime is within the trash
-// life time.
-func (s *HandlerSuite) TestTrashWorkerIntegration_SameLocatorInTwoVolumesWithDefaultTrashLifeTime(c *check.C) {
-       s.cluster.Collections.BlobTrash = true
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash2,
-               Block2:   TestBlock2,
-
-               CreateData:      true,
-               CreateInVolume1: true,
-
-               UseTrashLifeTime: true,
-
-               DeleteLocator: TestHash, // locator 1
-
-               // Since trash life time is in effect, block won't be deleted.
-               ExpectLocator1: true,
-               ExpectLocator2: true,
-       }
-       s.performTrashWorkerTest(c, testData)
-}
-
-// Delete a block with matching mtime for locator in both volumes, but
-// EnableDelete is false, so block won't be deleted.
-func (s *HandlerSuite) TestTrashWorkerIntegration_DisabledDelete(c *check.C) {
+func (s *routerSuite) TestTrashList_Clear(c *C) {
        s.cluster.Collections.BlobTrash = false
-       testData := TrashWorkerTestData{
-               Locator1: TestHash,
-               Block1:   TestBlock,
-
-               Locator2: TestHash,
-               Block2:   TestBlock,
-
-               CreateData: true,
-
-               DeleteLocator: TestHash,
-
-               ExpectLocator1: true,
-               ExpectLocator2: true,
-       }
-       s.performTrashWorkerTest(c, testData)
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       resp := call(router, "PUT", "http://example/trash", s.cluster.SystemRootToken, []byte(`
+               [
+                {
+                 "locator":"acbd18db4cc2f85cedef654fccc4a4d8+3",
+                 "block_mtime":1707249451308502672,
+                 "mount_uuid":"zzzzz-nyw5e-000000000000000"
+                }
+               ]
+               `), nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(router.trasher.todo, DeepEquals, []TrashListItem{{
+               Locator:    "acbd18db4cc2f85cedef654fccc4a4d8+3",
+               BlockMtime: 1707249451308502672,
+               MountUUID:  "zzzzz-nyw5e-000000000000000",
+       }})
+
+       resp = call(router, "PUT", "http://example/trash", s.cluster.SystemRootToken, []byte("[]"), nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       c.Check(router.trasher.todo, HasLen, 0)
 }
 
-func (s *HandlerSuite) performTrashWorkerTest(c *check.C, testData TrashWorkerTestData) {
-       c.Assert(s.handler.setup(context.Background(), s.cluster, "", prometheus.NewRegistry(), testServiceURL), check.IsNil)
-       // Replace the router's trashq -- which the worker goroutines
-       // started by setup() are now receiving from -- with a new
-       // one, so we can see what the handler sends to it.
-       trashq := NewWorkQueue()
-       s.handler.Handler.(*router).trashq = trashq
-
-       // Put test content
-       mounts := s.handler.volmgr.AllWritable()
-       if testData.CreateData {
-               mounts[0].Put(context.Background(), testData.Locator1, testData.Block1)
-               mounts[0].Put(context.Background(), testData.Locator1+".meta", []byte("metadata"))
-
-               if testData.CreateInVolume1 {
-                       mounts[0].Put(context.Background(), testData.Locator2, testData.Block2)
-                       mounts[0].Put(context.Background(), testData.Locator2+".meta", []byte("metadata"))
-               } else {
-                       mounts[1].Put(context.Background(), testData.Locator2, testData.Block2)
-                       mounts[1].Put(context.Background(), testData.Locator2+".meta", []byte("metadata"))
-               }
-       }
-
-       oldBlockTime := time.Now().Add(-s.cluster.Collections.BlobSigningTTL.Duration() - time.Minute)
-
-       // Create TrashRequest for the test
-       trashRequest := TrashRequest{
-               Locator:    testData.DeleteLocator,
-               BlockMtime: oldBlockTime.UnixNano(),
-       }
-       if testData.SpecifyMountUUID {
-               trashRequest.MountUUID = s.handler.volmgr.Mounts()[0].UUID
-       }
-
-       // Run trash worker and put the trashRequest on trashq
-       trashList := list.New()
-       trashList.PushBack(trashRequest)
-
-       if !testData.UseTrashLifeTime {
-               // Trash worker would not delete block if its Mtime is
-               // within trash life time. Back-date the block to
-               // allow the deletion to succeed.
-               for _, mnt := range mounts {
-                       mnt.Volume.(*MockVolume).Timestamps[testData.DeleteLocator] = oldBlockTime
-                       if testData.DifferentMtimes {
-                               oldBlockTime = oldBlockTime.Add(time.Second)
+func (s *routerSuite) TestTrashList_Execute(c *C) {
+       s.cluster.Collections.BlobTrashConcurrency = 1
+       s.cluster.Volumes = map[string]arvados.Volume{
+               "zzzzz-nyw5e-000000000000000": {Replication: 1, Driver: "stub"},
+               "zzzzz-nyw5e-111111111111111": {Replication: 1, Driver: "stub"},
+               "zzzzz-nyw5e-222222222222222": {Replication: 1, Driver: "stub", ReadOnly: true},
+               "zzzzz-nyw5e-333333333333333": {Replication: 1, Driver: "stub", ReadOnly: true, AllowTrashWhenReadOnly: true},
+       }
+       router, cancel := testRouter(c, s.cluster, nil)
+       defer cancel()
+
+       var mounts []struct {
+               UUID     string
+               DeviceID string `json:"device_id"`
+       }
+       resp := call(router, "GET", "http://example/mounts", s.cluster.SystemRootToken, nil, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+       err := json.Unmarshal(resp.Body.Bytes(), &mounts)
+       c.Assert(err, IsNil)
+       c.Assert(mounts, HasLen, 4)
+
+       // Sort mounts by UUID
+       sort.Slice(mounts, func(i, j int) bool {
+               return mounts[i].UUID < mounts[j].UUID
+       })
+
+       // Make vols (stub volumes) in same order as mounts
+       var vols []*stubVolume
+       for _, mount := range mounts {
+               vols = append(vols, router.keepstore.mounts[mount.UUID].volume.(*stubVolume))
+       }
+
+       // The "trial" loop below will construct the trashList which
+       // we'll send to trasher via router, plus a slice of checks
+       // which we'll run after the trasher has finished executing
+       // the list.
+       var trashList []TrashListItem
+       var checks []func()
+
+       tNew := time.Now().Add(-s.cluster.Collections.BlobSigningTTL.Duration() / 2)
+       tOld := time.Now().Add(-s.cluster.Collections.BlobSigningTTL.Duration() - time.Second)
+
+       for _, trial := range []struct {
+               comment        string
+               storeMtime     []time.Time
+               trashListItems []TrashListItem
+               expectData     []bool
+       }{
+               {
+                       comment:    "timestamp matches, but is not old enough to trash => skip",
+                       storeMtime: []time.Time{tNew},
+                       trashListItems: []TrashListItem{
+                               {
+                                       BlockMtime: tNew.UnixNano(),
+                                       MountUUID:  mounts[0].UUID,
+                               },
+                       },
+                       expectData: []bool{true},
+               },
+               {
+                       comment:    "timestamp matches, and is old enough => trash",
+                       storeMtime: []time.Time{tOld},
+                       trashListItems: []TrashListItem{
+                               {
+                                       BlockMtime: tOld.UnixNano(),
+                                       MountUUID:  mounts[0].UUID,
+                               },
+                       },
+                       expectData: []bool{false},
+               },
+               {
+                       comment:    "timestamp matches and is old enough on mount 0, but the request specifies mount 1, where timestamp does not match => skip",
+                       storeMtime: []time.Time{tOld, tOld.Add(-time.Second)},
+                       trashListItems: []TrashListItem{
+                               {
+                                       BlockMtime: tOld.UnixNano(),
+                                       MountUUID:  mounts[1].UUID,
+                               },
+                       },
+                       expectData: []bool{true, true},
+               },
+               {
+                       comment:    "MountUUID unspecified => trash from any mount where timestamp matches, leave alone elsewhere",
+                       storeMtime: []time.Time{tOld, tOld.Add(-time.Second)},
+                       trashListItems: []TrashListItem{
+                               {
+                                       BlockMtime: tOld.UnixNano(),
+                               },
+                       },
+                       expectData: []bool{false, true},
+               },
+               {
+                       comment:    "MountUUID unspecified => trash from multiple mounts if timestamp matches, but skip readonly volumes unless AllowTrashWhenReadOnly",
+                       storeMtime: []time.Time{tOld, tOld, tOld, tOld},
+                       trashListItems: []TrashListItem{
+                               {
+                                       BlockMtime: tOld.UnixNano(),
+                               },
+                       },
+                       expectData: []bool{false, false, true, false},
+               },
+               {
+                       comment:    "readonly MountUUID specified => skip",
+                       storeMtime: []time.Time{tOld, tOld, tOld},
+                       trashListItems: []TrashListItem{
+                               {
+                                       BlockMtime: tOld.UnixNano(),
+                                       MountUUID:  mounts[2].UUID,
+                               },
+                       },
+                       expectData: []bool{true, true, true},
+               },
+       } {
+               trial := trial
+               data := []byte(fmt.Sprintf("trial %+v", trial))
+               hash := fmt.Sprintf("%x", md5.Sum(data))
+               for i, t := range trial.storeMtime {
+                       if t.IsZero() {
+                               continue
                        }
+                       err := vols[i].BlockWrite(context.Background(), hash, data)
+                       c.Assert(err, IsNil)
+                       err = vols[i].blockTouchWithTime(hash, t)
+                       c.Assert(err, IsNil)
                }
-       }
-       go RunTrashWorker(s.handler.volmgr, ctxlog.TestLogger(c), s.cluster, trashq)
-
-       // Install gate so all local operations block until we say go
-       gate := make(chan struct{})
-       for _, mnt := range mounts {
-               mnt.Volume.(*MockVolume).Gate = gate
-       }
-
-       assertStatusItem := func(k string, expect float64) {
-               if v := getStatusItem(s.handler, "TrashQueue", k); v != expect {
-                       c.Errorf("Got %s %v, expected %v", k, v, expect)
-               }
-       }
-
-       assertStatusItem("InProgress", 0)
-       assertStatusItem("Queued", 0)
-
-       listLen := trashList.Len()
-       trashq.ReplaceQueue(trashList)
-
-       // Wait for worker to take request(s)
-       expectEqualWithin(c, time.Second, listLen, func() interface{} { return trashq.Status().InProgress })
-
-       // Ensure status.json also reports work is happening
-       assertStatusItem("InProgress", float64(1))
-       assertStatusItem("Queued", float64(listLen-1))
-
-       // Let worker proceed
-       close(gate)
-
-       // Wait for worker to finish
-       expectEqualWithin(c, time.Second, 0, func() interface{} { return trashq.Status().InProgress })
-
-       // Verify Locator1 to be un/deleted as expected
-       buf := make([]byte, BlockSize)
-       size, err := GetBlock(context.Background(), s.handler.volmgr, testData.Locator1, buf, nil)
-       if testData.ExpectLocator1 {
-               if size == 0 || err != nil {
-                       c.Errorf("Expected Locator1 to be still present: %s", testData.Locator1)
+               for _, item := range trial.trashListItems {
+                       item.Locator = fmt.Sprintf("%s+%d", hash, len(data))
+                       trashList = append(trashList, item)
                }
-       } else {
-               if size > 0 || err == nil {
-                       c.Errorf("Expected Locator1 to be deleted: %s", testData.Locator1)
+               for i, expect := range trial.expectData {
+                       i, expect := i, expect
+                       checks = append(checks, func() {
+                               ent := vols[i].data[hash]
+                               dataPresent := ent.data != nil && ent.trash.IsZero()
+                               c.Check(dataPresent, Equals, expect, Commentf("%s mount %d (%s) expect present=%v but got len(ent.data)=%d ent.trash=%v // %s\nlog:\n%s", hash, i, vols[i].params.UUID, expect, len(ent.data), !ent.trash.IsZero(), trial.comment, vols[i].stubLog.String()))
+                       })
                }
        }
 
-       // Verify Locator2 to be un/deleted as expected
-       if testData.Locator1 != testData.Locator2 {
-               size, err = GetBlock(context.Background(), s.handler.volmgr, testData.Locator2, buf, nil)
-               if testData.ExpectLocator2 {
-                       if size == 0 || err != nil {
-                               c.Errorf("Expected Locator2 to be still present: %s", testData.Locator2)
-                       }
-               } else {
-                       if size > 0 || err == nil {
-                               c.Errorf("Expected Locator2 to be deleted: %s", testData.Locator2)
-                       }
+       listjson, err := json.Marshal(trashList)
+       resp = call(router, "PUT", "http://example/trash", s.cluster.SystemRootToken, listjson, nil)
+       c.Check(resp.Code, Equals, http.StatusOK)
+
+       for {
+               router.trasher.cond.L.Lock()
+               todolen := len(router.trasher.todo)
+               router.trasher.cond.L.Unlock()
+               if todolen == 0 && router.trasher.inprogress.Load() == 0 {
+                       break
                }
+               time.Sleep(time.Millisecond)
        }
 
-       // The DifferentMtimes test puts the same locator in two
-       // different volumes, but only one copy has an Mtime matching
-       // the trash request.
-       if testData.DifferentMtimes {
-               locatorFoundIn := 0
-               for _, volume := range s.handler.volmgr.AllReadable() {
-                       buf := make([]byte, BlockSize)
-                       if _, err := volume.Get(context.Background(), testData.Locator1, buf); err == nil {
-                               locatorFoundIn = locatorFoundIn + 1
-                       }
-               }
-               c.Check(locatorFoundIn, check.Equals, 1)
+       for _, check := range checks {
+               check()
        }
 }
index dee4bdc1c1ed1d59badb076dedac2615eef3f2d7..98edfae14d7e602d79e055a77d698dc8a6b466d2 100644 (file)
@@ -28,16 +28,22 @@ import (
 )
 
 func init() {
-       driver["Directory"] = newDirectoryVolume
+       driver["Directory"] = newUnixVolume
 }
 
-func newDirectoryVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) {
-       v := &UnixVolume{cluster: cluster, volume: volume, logger: logger, metrics: metrics}
-       err := json.Unmarshal(volume.DriverParameters, &v)
+func newUnixVolume(params newVolumeParams) (volume, error) {
+       v := &UnixVolume{
+               uuid:    params.UUID,
+               cluster: params.Cluster,
+               volume:  params.ConfigVolume,
+               logger:  params.Logger,
+               metrics: params.MetricsVecs,
+       }
+       err := json.Unmarshal(params.ConfigVolume.DriverParameters, &v)
        if err != nil {
                return nil, err
        }
-       v.logger = v.logger.WithField("Volume", v.String())
+       v.logger = v.logger.WithField("Volume", v.DeviceID())
        return v, v.check()
 }
 
@@ -53,7 +59,7 @@ func (v *UnixVolume) check() error {
        }
 
        // Set up prometheus metrics
-       lbls := prometheus.Labels{"device_id": v.GetDeviceID()}
+       lbls := prometheus.Labels{"device_id": v.DeviceID()}
        v.os.stats.opsCounters, v.os.stats.errCounters, v.os.stats.ioBytes = v.metrics.getCounterVecsFor(lbls)
 
        _, err := v.os.Stat(v.Root)
@@ -65,6 +71,7 @@ type UnixVolume struct {
        Root      string // path to the volume's root directory
        Serialize bool
 
+       uuid    string
        cluster *arvados.Cluster
        volume  arvados.Volume
        logger  logrus.FieldLogger
@@ -77,15 +84,16 @@ type UnixVolume struct {
        os osWithStats
 }
 
-// GetDeviceID returns a globally unique ID for the volume's root
+// DeviceID returns a globally unique ID for the volume's root
 // directory, consisting of the filesystem's UUID and the path from
 // filesystem root to storage directory, joined by "/". For example,
 // the device ID for a local directory "/mnt/xvda1/keep" might be
 // "fa0b6166-3b55-4994-bd3f-92f4e00a1bb0/keep".
-func (v *UnixVolume) GetDeviceID() string {
+func (v *UnixVolume) DeviceID() string {
        giveup := func(f string, args ...interface{}) string {
-               v.logger.Infof(f+"; using blank DeviceID for volume %s", append(args, v)...)
-               return ""
+               v.logger.Infof(f+"; using hostname:path for volume %s", append(args, v.uuid)...)
+               host, _ := os.Hostname()
+               return host + ":" + v.Root
        }
        buf, err := exec.Command("findmnt", "--noheadings", "--target", v.Root).CombinedOutput()
        if err != nil {
@@ -154,12 +162,9 @@ func (v *UnixVolume) GetDeviceID() string {
        return giveup("could not find entry in %q matching %q", udir, dev)
 }
 
-// Touch sets the timestamp for the given locator to the current time
-func (v *UnixVolume) Touch(loc string) error {
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-       p := v.blockPath(loc)
+// BlockTouch sets the timestamp for the given locator to the current time
+func (v *UnixVolume) BlockTouch(hash string) error {
+       p := v.blockPath(hash)
        f, err := v.os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644)
        if err != nil {
                return err
@@ -203,7 +208,7 @@ func (v *UnixVolume) getFunc(ctx context.Context, path string, fn func(io.Reader
                return err
        }
        defer f.Close()
-       return fn(NewCountingReader(ioutil.NopCloser(f), v.os.stats.TickInBytes))
+       return fn(newCountingReader(ioutil.NopCloser(f), v.os.stats.TickInBytes))
 }
 
 // stat is os.Stat() with some extra sanity checks.
@@ -213,72 +218,45 @@ func (v *UnixVolume) stat(path string) (os.FileInfo, error) {
                if stat.Size() < 0 {
                        err = os.ErrInvalid
                } else if stat.Size() > BlockSize {
-                       err = TooLongError
+                       err = errTooLarge
                }
        }
        return stat, err
 }
 
-// Get retrieves a block, copies it to the given slice, and returns
-// the number of bytes copied.
-func (v *UnixVolume) Get(ctx context.Context, loc string, buf []byte) (int, error) {
-       return getWithPipe(ctx, loc, buf, v)
-}
-
-// ReadBlock implements BlockReader.
-func (v *UnixVolume) ReadBlock(ctx context.Context, loc string, w io.Writer) error {
-       path := v.blockPath(loc)
+// BlockRead reads a block from the volume.
+func (v *UnixVolume) BlockRead(ctx context.Context, hash string, w io.Writer) (int, error) {
+       path := v.blockPath(hash)
        stat, err := v.stat(path)
        if err != nil {
-               return v.translateError(err)
+               return 0, v.translateError(err)
        }
-       return v.getFunc(ctx, path, func(rdr io.Reader) error {
-               n, err := io.Copy(w, rdr)
+       var n int64
+       err = v.getFunc(ctx, path, func(rdr io.Reader) error {
+               n, err = io.Copy(w, rdr)
                if err == nil && n != stat.Size() {
                        err = io.ErrUnexpectedEOF
                }
                return err
        })
+       return int(n), err
 }
 
-// Compare returns nil if Get(loc) would return the same content as
-// expect. It is functionally equivalent to Get() followed by
-// bytes.Compare(), but uses less memory.
-func (v *UnixVolume) Compare(ctx context.Context, loc string, expect []byte) error {
-       path := v.blockPath(loc)
-       if _, err := v.stat(path); err != nil {
-               return v.translateError(err)
-       }
-       return v.getFunc(ctx, path, func(rdr io.Reader) error {
-               return compareReaderWithBuf(ctx, rdr, expect, loc[:32])
-       })
-}
-
-// Put stores a block of data identified by the locator string
-// "loc".  It returns nil on success.  If the volume is full, it
-// returns a FullError.  If the write fails due to some other error,
-// that error is returned.
-func (v *UnixVolume) Put(ctx context.Context, loc string, block []byte) error {
-       return putWithPipe(ctx, loc, block, v)
-}
-
-// WriteBlock implements BlockWriter.
-func (v *UnixVolume) WriteBlock(ctx context.Context, loc string, rdr io.Reader) error {
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-       if v.IsFull() {
-               return FullError
+// BlockWrite stores a block on the volume. If it already exists, its
+// timestamp is updated.
+func (v *UnixVolume) BlockWrite(ctx context.Context, hash string, data []byte) error {
+       if v.isFull() {
+               return errFull
        }
-       bdir := v.blockDir(loc)
+       bdir := v.blockDir(hash)
        if err := os.MkdirAll(bdir, 0755); err != nil {
                return fmt.Errorf("error creating directory %s: %s", bdir, err)
        }
 
-       bpath := v.blockPath(loc)
-       tmpfile, err := v.os.TempFile(bdir, "tmp"+loc)
+       bpath := v.blockPath(hash)
+       tmpfile, err := v.os.TempFile(bdir, "tmp"+hash)
        if err != nil {
-               return fmt.Errorf("TempFile(%s, tmp%s) failed: %s", bdir, loc, err)
+               return fmt.Errorf("TempFile(%s, tmp%s) failed: %s", bdir, hash, err)
        }
        defer v.os.Remove(tmpfile.Name())
        defer tmpfile.Close()
@@ -287,7 +265,7 @@ func (v *UnixVolume) WriteBlock(ctx context.Context, loc string, rdr io.Reader)
                return err
        }
        defer v.unlock()
-       n, err := io.Copy(tmpfile, rdr)
+       n, err := tmpfile.Write(data)
        v.os.stats.TickOutBytes(uint64(n))
        if err != nil {
                return fmt.Errorf("error writing %s: %s", bpath, err)
@@ -312,56 +290,10 @@ func (v *UnixVolume) WriteBlock(ctx context.Context, loc string, rdr io.Reader)
        return nil
 }
 
-// Status returns a VolumeStatus struct describing the volume's
-// current state, or nil if an error occurs.
-func (v *UnixVolume) Status() *VolumeStatus {
-       fi, err := v.os.Stat(v.Root)
-       if err != nil {
-               v.logger.WithError(err).Error("stat failed")
-               return nil
-       }
-       // uint64() cast here supports GOOS=darwin where Dev is
-       // int32. If the device number is negative, the unsigned
-       // devnum won't be the real device number any more, but that's
-       // fine -- all we care about is getting the same number each
-       // time.
-       devnum := uint64(fi.Sys().(*syscall.Stat_t).Dev)
-
-       var fs syscall.Statfs_t
-       if err := syscall.Statfs(v.Root, &fs); err != nil {
-               v.logger.WithError(err).Error("statfs failed")
-               return nil
-       }
-       // These calculations match the way df calculates disk usage:
-       // "free" space is measured by fs.Bavail, but "used" space
-       // uses fs.Blocks - fs.Bfree.
-       free := fs.Bavail * uint64(fs.Bsize)
-       used := (fs.Blocks - fs.Bfree) * uint64(fs.Bsize)
-       return &VolumeStatus{
-               MountPoint: v.Root,
-               DeviceNum:  devnum,
-               BytesFree:  free,
-               BytesUsed:  used,
-       }
-}
-
 var blockDirRe = regexp.MustCompile(`^[0-9a-f]+$`)
 var blockFileRe = regexp.MustCompile(`^[0-9a-f]{32}$`)
 
-// IndexTo writes (to the given Writer) a list of blocks found on this
-// volume which begin with the specified prefix. If the prefix is an
-// empty string, IndexTo writes a complete list of blocks.
-//
-// Each block is given in the format
-//
-//     locator+size modification-time {newline}
-//
-// e.g.:
-//
-//     e4df392f86be161ca6ed3773a962b8f3+67108864 1388894303
-//     e4d41e6fd68460e0e3fc18cc746959d2+67108864 1377796043
-//     e4de7a2810f5554cd39b36d8ddb132ff+67108864 1388701136
-func (v *UnixVolume) IndexTo(prefix string, w io.Writer) error {
+func (v *UnixVolume) Index(ctx context.Context, prefix string, w io.Writer) error {
        rootdir, err := v.os.Open(v.Root)
        if err != nil {
                return err
@@ -374,6 +306,9 @@ func (v *UnixVolume) IndexTo(prefix string, w io.Writer) error {
                return err
        }
        for _, subdir := range subdirs {
+               if ctx.Err() != nil {
+                       return ctx.Err()
+               }
                if !strings.HasPrefix(subdir, prefix) && !strings.HasPrefix(prefix, subdir) {
                        // prefix excludes all blocks stored in this dir
                        continue
@@ -388,7 +323,9 @@ func (v *UnixVolume) IndexTo(prefix string, w io.Writer) error {
                        v.os.stats.TickOps("readdir")
                        v.os.stats.Tick(&v.os.stats.ReaddirOps)
                        dirents, err = os.ReadDir(blockdirpath)
-                       if err == nil {
+                       if ctx.Err() != nil {
+                               return ctx.Err()
+                       } else if err == nil {
                                break
                        } else if attempt < 5 && strings.Contains(err.Error(), "errno 523") {
                                // EBADCOOKIE (NFS stopped accepting
@@ -402,6 +339,9 @@ func (v *UnixVolume) IndexTo(prefix string, w io.Writer) error {
                }
 
                for _, dirent := range dirents {
+                       if ctx.Err() != nil {
+                               return ctx.Err()
+                       }
                        fileInfo, err := dirent.Info()
                        if os.IsNotExist(err) {
                                // File disappeared between ReadDir() and now
@@ -430,11 +370,11 @@ func (v *UnixVolume) IndexTo(prefix string, w io.Writer) error {
        return nil
 }
 
-// Trash trashes the block data from the unix storage
-// If BlobTrashLifetime == 0, the block is deleted
-// Else, the block is renamed as path/{loc}.trash.{deadline},
-// where deadline = now + BlobTrashLifetime
-func (v *UnixVolume) Trash(loc string) error {
+// BlockTrash trashes the block data from the unix storage.  If
+// BlobTrashLifetime == 0, the block is deleted; otherwise, the block
+// is renamed as path/{loc}.trash.{deadline}, where deadline = now +
+// BlobTrashLifetime.
+func (v *UnixVolume) BlockTrash(loc string) error {
        // Touch() must be called before calling Write() on a block.  Touch()
        // also uses lockfile().  This avoids a race condition between Write()
        // and Trash() because either (a) the file will be trashed and Touch()
@@ -442,9 +382,6 @@ func (v *UnixVolume) Trash(loc string) error {
        // be re-written), or (b) Touch() will update the file's timestamp and
        // Trash() will read the correct up-to-date timestamp and choose not to
        // trash the file.
-       if v.volume.ReadOnly && !v.volume.AllowTrashWhenReadOnly {
-               return MethodDisabledError
-       }
        if err := v.lock(context.TODO()); err != nil {
                return err
        }
@@ -477,17 +414,13 @@ func (v *UnixVolume) Trash(loc string) error {
        return v.os.Rename(p, fmt.Sprintf("%v.trash.%d", p, time.Now().Add(v.cluster.Collections.BlobTrashLifetime.Duration()).Unix()))
 }
 
-// Untrash moves block from trash back into store
+// BlockUntrash moves block from trash back into store
 // Look for path/{loc}.trash.{deadline} in storage,
 // and rename the first such file as path/{loc}
-func (v *UnixVolume) Untrash(loc string) (err error) {
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-
+func (v *UnixVolume) BlockUntrash(hash string) error {
        v.os.stats.TickOps("readdir")
        v.os.stats.Tick(&v.os.stats.ReaddirOps)
-       files, err := ioutil.ReadDir(v.blockDir(loc))
+       files, err := ioutil.ReadDir(v.blockDir(hash))
        if err != nil {
                return err
        }
@@ -497,11 +430,11 @@ func (v *UnixVolume) Untrash(loc string) (err error) {
        }
 
        foundTrash := false
-       prefix := fmt.Sprintf("%v.trash.", loc)
+       prefix := fmt.Sprintf("%v.trash.", hash)
        for _, f := range files {
                if strings.HasPrefix(f.Name(), prefix) {
                        foundTrash = true
-                       err = v.os.Rename(v.blockPath(f.Name()), v.blockPath(loc))
+                       err = v.os.Rename(v.blockPath(f.Name()), v.blockPath(hash))
                        if err == nil {
                                break
                        }
@@ -512,7 +445,7 @@ func (v *UnixVolume) Untrash(loc string) (err error) {
                return os.ErrNotExist
        }
 
-       return
+       return nil
 }
 
 // blockDir returns the fully qualified directory name for the directory
@@ -527,9 +460,9 @@ func (v *UnixVolume) blockPath(loc string) string {
        return filepath.Join(v.blockDir(loc), loc)
 }
 
-// IsFull returns true if the free space on the volume is less than
+// isFull returns true if the free space on the volume is less than
 // MinFreeKilobytes.
-func (v *UnixVolume) IsFull() (isFull bool) {
+func (v *UnixVolume) isFull() (isFull bool) {
        fullSymlink := v.Root + "/full"
 
        // Check if the volume has been marked as full in the last hour.
@@ -543,9 +476,9 @@ func (v *UnixVolume) IsFull() (isFull bool) {
        }
 
        if avail, err := v.FreeDiskSpace(); err == nil {
-               isFull = avail < MinFreeKilobytes
+               isFull = avail < BlockSize
        } else {
-               v.logger.WithError(err).Errorf("%s: FreeDiskSpace failed", v)
+               v.logger.WithError(err).Errorf("%s: FreeDiskSpace failed", v.DeviceID())
                isFull = false
        }
 
@@ -565,15 +498,11 @@ func (v *UnixVolume) FreeDiskSpace() (free uint64, err error) {
        if err == nil {
                // Statfs output is not guaranteed to measure free
                // space in terms of 1K blocks.
-               free = fs.Bavail * uint64(fs.Bsize) / 1024
+               free = fs.Bavail * uint64(fs.Bsize)
        }
        return
 }
 
-func (v *UnixVolume) String() string {
-       return fmt.Sprintf("[UnixVolume %s]", v.Root)
-}
-
 // InternalStats returns I/O and filesystem ops counters.
 func (v *UnixVolume) InternalStats() interface{} {
        return &v.os.stats
index 75d9b22de55604cc01a2d1f6f4ffaad7b9b585a7..a8dc4e809a863c92bb1459967cc165115bfb4d6b 100644 (file)
@@ -17,82 +17,74 @@ import (
        "syscall"
        "time"
 
-       "git.arvados.org/arvados.git/sdk/go/arvados"
        "git.arvados.org/arvados.git/sdk/go/ctxlog"
        "github.com/prometheus/client_golang/prometheus"
-       "github.com/sirupsen/logrus"
        check "gopkg.in/check.v1"
 )
 
-type TestableUnixVolume struct {
+type testableUnixVolume struct {
        UnixVolume
        t TB
 }
 
-// PutRaw writes a Keep block directly into a UnixVolume, even if
-// the volume is readonly.
-func (v *TestableUnixVolume) PutRaw(locator string, data []byte) {
-       defer func(orig bool) {
-               v.volume.ReadOnly = orig
-       }(v.volume.ReadOnly)
-       v.volume.ReadOnly = false
-       err := v.Put(context.Background(), locator, data)
+func (v *testableUnixVolume) TouchWithDate(locator string, lastPut time.Time) {
+       err := syscall.Utime(v.blockPath(locator), &syscall.Utimbuf{Actime: lastPut.Unix(), Modtime: lastPut.Unix()})
        if err != nil {
                v.t.Fatal(err)
        }
 }
 
-func (v *TestableUnixVolume) TouchWithDate(locator string, lastPut time.Time) {
-       err := syscall.Utime(v.blockPath(locator), &syscall.Utimbuf{lastPut.Unix(), lastPut.Unix()})
-       if err != nil {
-               v.t.Fatal(err)
-       }
-}
-
-func (v *TestableUnixVolume) Teardown() {
+func (v *testableUnixVolume) Teardown() {
        if err := os.RemoveAll(v.Root); err != nil {
                v.t.Error(err)
        }
 }
 
-func (v *TestableUnixVolume) ReadWriteOperationLabelValues() (r, w string) {
+func (v *testableUnixVolume) ReadWriteOperationLabelValues() (r, w string) {
        return "open", "create"
 }
 
-var _ = check.Suite(&UnixVolumeSuite{})
+var _ = check.Suite(&unixVolumeSuite{})
 
-type UnixVolumeSuite struct {
-       cluster *arvados.Cluster
-       volumes []*TestableUnixVolume
-       metrics *volumeMetricsVecs
+type unixVolumeSuite struct {
+       params  newVolumeParams
+       volumes []*testableUnixVolume
 }
 
-func (s *UnixVolumeSuite) SetUpTest(c *check.C) {
-       s.cluster = testCluster(c)
-       s.metrics = newVolumeMetricsVecs(prometheus.NewRegistry())
+func (s *unixVolumeSuite) SetUpTest(c *check.C) {
+       logger := ctxlog.TestLogger(c)
+       reg := prometheus.NewRegistry()
+       s.params = newVolumeParams{
+               UUID:        "zzzzz-nyw5e-999999999999999",
+               Cluster:     testCluster(c),
+               Logger:      logger,
+               MetricsVecs: newVolumeMetricsVecs(reg),
+               BufferPool:  newBufferPool(logger, 8, reg),
+       }
 }
 
-func (s *UnixVolumeSuite) TearDownTest(c *check.C) {
+func (s *unixVolumeSuite) TearDownTest(c *check.C) {
        for _, v := range s.volumes {
                v.Teardown()
        }
 }
 
-func (s *UnixVolumeSuite) newTestableUnixVolume(c *check.C, cluster *arvados.Cluster, volume arvados.Volume, metrics *volumeMetricsVecs, serialize bool) *TestableUnixVolume {
+func (s *unixVolumeSuite) newTestableUnixVolume(c *check.C, params newVolumeParams, serialize bool) *testableUnixVolume {
        d, err := ioutil.TempDir("", "volume_test")
        c.Check(err, check.IsNil)
        var locker sync.Locker
        if serialize {
                locker = &sync.Mutex{}
        }
-       v := &TestableUnixVolume{
+       v := &testableUnixVolume{
                UnixVolume: UnixVolume{
                        Root:    d,
                        locker:  locker,
-                       cluster: cluster,
-                       logger:  ctxlog.TestLogger(c),
-                       volume:  volume,
-                       metrics: metrics,
+                       uuid:    params.UUID,
+                       cluster: params.Cluster,
+                       logger:  params.Logger,
+                       volume:  params.ConfigVolume,
+                       metrics: params.MetricsVecs,
                },
                t: c,
        }
@@ -101,56 +93,52 @@ func (s *UnixVolumeSuite) newTestableUnixVolume(c *check.C, cluster *arvados.Clu
        return v
 }
 
-// serialize = false; readonly = false
-func (s *UnixVolumeSuite) TestUnixVolumeWithGenericTests(c *check.C) {
-       DoGenericVolumeTests(c, false, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               return s.newTestableUnixVolume(c, cluster, volume, metrics, false)
+func (s *unixVolumeSuite) TestUnixVolumeWithGenericTests(c *check.C) {
+       DoGenericVolumeTests(c, false, func(t TB, params newVolumeParams) TestableVolume {
+               return s.newTestableUnixVolume(c, params, false)
        })
 }
 
-// serialize = false; readonly = true
-func (s *UnixVolumeSuite) TestUnixVolumeWithGenericTestsReadOnly(c *check.C) {
-       DoGenericVolumeTests(c, true, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               return s.newTestableUnixVolume(c, cluster, volume, metrics, true)
+func (s *unixVolumeSuite) TestUnixVolumeWithGenericTests_ReadOnly(c *check.C) {
+       DoGenericVolumeTests(c, true, func(t TB, params newVolumeParams) TestableVolume {
+               return s.newTestableUnixVolume(c, params, false)
        })
 }
 
-// serialize = true; readonly = false
-func (s *UnixVolumeSuite) TestUnixVolumeWithGenericTestsSerialized(c *check.C) {
-       DoGenericVolumeTests(c, false, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               return s.newTestableUnixVolume(c, cluster, volume, metrics, false)
+func (s *unixVolumeSuite) TestUnixVolumeWithGenericTests_Serialized(c *check.C) {
+       DoGenericVolumeTests(c, false, func(t TB, params newVolumeParams) TestableVolume {
+               return s.newTestableUnixVolume(c, params, true)
        })
 }
 
-// serialize = true; readonly = true
-func (s *UnixVolumeSuite) TestUnixVolumeHandlersWithGenericVolumeTests(c *check.C) {
-       DoGenericVolumeTests(c, true, func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume {
-               return s.newTestableUnixVolume(c, cluster, volume, metrics, true)
+func (s *unixVolumeSuite) TestUnixVolumeWithGenericTests_Readonly_Serialized(c *check.C) {
+       DoGenericVolumeTests(c, true, func(t TB, params newVolumeParams) TestableVolume {
+               return s.newTestableUnixVolume(c, params, true)
        })
 }
 
-func (s *UnixVolumeSuite) TestGetNotFound(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestGetNotFound(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, true)
        defer v.Teardown()
-       v.Put(context.Background(), TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
 
-       buf := make([]byte, BlockSize)
-       n, err := v.Get(context.Background(), TestHash2, buf)
+       buf := bytes.NewBuffer(nil)
+       _, err := v.BlockRead(context.Background(), TestHash2, buf)
        switch {
        case os.IsNotExist(err):
                break
        case err == nil:
-               c.Errorf("Read should have failed, returned %+q", buf[:n])
+               c.Errorf("Read should have failed, returned %+q", buf.Bytes())
        default:
                c.Errorf("Read expected ErrNotExist, got: %s", err)
        }
 }
 
-func (s *UnixVolumeSuite) TestPut(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestPut(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, false)
        defer v.Teardown()
 
-       err := v.Put(context.Background(), TestHash, TestBlock)
+       err := v.BlockWrite(context.Background(), TestHash, TestBlock)
        if err != nil {
                c.Error(err)
        }
@@ -163,89 +151,41 @@ func (s *UnixVolumeSuite) TestPut(c *check.C) {
        }
 }
 
-func (s *UnixVolumeSuite) TestPutBadVolume(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestPutBadVolume(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, false)
        defer v.Teardown()
 
        err := os.RemoveAll(v.Root)
        c.Assert(err, check.IsNil)
-       err = v.Put(context.Background(), TestHash, TestBlock)
+       err = v.BlockWrite(context.Background(), TestHash, TestBlock)
        c.Check(err, check.IsNil)
 }
 
-func (s *UnixVolumeSuite) TestUnixVolumeReadonly(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{ReadOnly: true, Replication: 1}, s.metrics, false)
-       defer v.Teardown()
-
-       v.PutRaw(TestHash, TestBlock)
-
-       buf := make([]byte, BlockSize)
-       _, err := v.Get(context.Background(), TestHash, buf)
-       if err != nil {
-               c.Errorf("got err %v, expected nil", err)
-       }
-
-       err = v.Put(context.Background(), TestHash, TestBlock)
-       if err != MethodDisabledError {
-               c.Errorf("got err %v, expected MethodDisabledError", err)
-       }
-
-       err = v.Touch(TestHash)
-       if err != MethodDisabledError {
-               c.Errorf("got err %v, expected MethodDisabledError", err)
-       }
-
-       err = v.Trash(TestHash)
-       if err != MethodDisabledError {
-               c.Errorf("got err %v, expected MethodDisabledError", err)
-       }
-}
-
-func (s *UnixVolumeSuite) TestIsFull(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestIsFull(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, false)
        defer v.Teardown()
 
        fullPath := v.Root + "/full"
        now := fmt.Sprintf("%d", time.Now().Unix())
        os.Symlink(now, fullPath)
-       if !v.IsFull() {
-               c.Errorf("%s: claims not to be full", v)
+       if !v.isFull() {
+               c.Error("volume claims not to be full")
        }
        os.Remove(fullPath)
 
        // Test with an expired /full link.
        expired := fmt.Sprintf("%d", time.Now().Unix()-3605)
        os.Symlink(expired, fullPath)
-       if v.IsFull() {
-               c.Errorf("%s: should no longer be full", v)
+       if v.isFull() {
+               c.Error("volume should no longer be full")
        }
 }
 
-func (s *UnixVolumeSuite) TestNodeStatus(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestUnixVolumeGetFuncWorkerError(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, false)
        defer v.Teardown()
 
-       // Get node status and make a basic sanity check.
-       volinfo := v.Status()
-       if volinfo.MountPoint != v.Root {
-               c.Errorf("GetNodeStatus mount_point %s, expected %s", volinfo.MountPoint, v.Root)
-       }
-       if volinfo.DeviceNum == 0 {
-               c.Errorf("uninitialized device_num in %v", volinfo)
-       }
-       if volinfo.BytesFree == 0 {
-               c.Errorf("uninitialized bytes_free in %v", volinfo)
-       }
-       if volinfo.BytesUsed == 0 {
-               c.Errorf("uninitialized bytes_used in %v", volinfo)
-       }
-}
-
-func (s *UnixVolumeSuite) TestUnixVolumeGetFuncWorkerError(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
-       defer v.Teardown()
-
-       v.Put(context.Background(), TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        mockErr := errors.New("Mock error")
        err := v.getFunc(context.Background(), v.blockPath(TestHash), func(rdr io.Reader) error {
                return mockErr
@@ -255,8 +195,8 @@ func (s *UnixVolumeSuite) TestUnixVolumeGetFuncWorkerError(c *check.C) {
        }
 }
 
-func (s *UnixVolumeSuite) TestUnixVolumeGetFuncFileError(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestUnixVolumeGetFuncFileError(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, false)
        defer v.Teardown()
 
        funcCalled := false
@@ -272,11 +212,11 @@ func (s *UnixVolumeSuite) TestUnixVolumeGetFuncFileError(c *check.C) {
        }
 }
 
-func (s *UnixVolumeSuite) TestUnixVolumeGetFuncWorkerWaitsOnMutex(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestUnixVolumeGetFuncWorkerWaitsOnMutex(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, false)
        defer v.Teardown()
 
-       v.Put(context.Background(), TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
 
        mtx := NewMockMutex()
        v.locker = mtx
@@ -307,80 +247,66 @@ func (s *UnixVolumeSuite) TestUnixVolumeGetFuncWorkerWaitsOnMutex(c *check.C) {
        }
 }
 
-func (s *UnixVolumeSuite) TestUnixVolumeCompare(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
-       defer v.Teardown()
-
-       v.Put(context.Background(), TestHash, TestBlock)
-       err := v.Compare(context.Background(), TestHash, TestBlock)
-       if err != nil {
-               c.Errorf("Got err %q, expected nil", err)
-       }
+type MockMutex struct {
+       AllowLock   chan struct{}
+       AllowUnlock chan struct{}
+}
 
-       err = v.Compare(context.Background(), TestHash, []byte("baddata"))
-       if err != CollisionError {
-               c.Errorf("Got err %q, expected %q", err, CollisionError)
+func NewMockMutex() *MockMutex {
+       return &MockMutex{
+               AllowLock:   make(chan struct{}),
+               AllowUnlock: make(chan struct{}),
        }
+}
 
-       v.Put(context.Background(), TestHash, []byte("baddata"))
-       err = v.Compare(context.Background(), TestHash, TestBlock)
-       if err != DiskHashError {
-               c.Errorf("Got err %q, expected %q", err, DiskHashError)
-       }
+// Lock waits for someone to send to AllowLock.
+func (m *MockMutex) Lock() {
+       <-m.AllowLock
+}
 
-       if os.Getuid() == 0 {
-               c.Log("skipping 'permission denied' check when running as root")
-       } else {
-               p := fmt.Sprintf("%s/%s/%s", v.Root, TestHash[:3], TestHash)
-               err = os.Chmod(p, 000)
-               c.Assert(err, check.IsNil)
-               err = v.Compare(context.Background(), TestHash, TestBlock)
-               c.Check(err, check.ErrorMatches, ".*permission denied.*")
-       }
+// Unlock waits for someone to send to AllowUnlock.
+func (m *MockMutex) Unlock() {
+       <-m.AllowUnlock
 }
 
-func (s *UnixVolumeSuite) TestUnixVolumeContextCancelPut(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, true)
+func (s *unixVolumeSuite) TestUnixVolumeContextCancelBlockWrite(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, true)
        defer v.Teardown()
        v.locker.Lock()
+       defer v.locker.Unlock()
        ctx, cancel := context.WithCancel(context.Background())
        go func() {
                time.Sleep(50 * time.Millisecond)
                cancel()
-               time.Sleep(50 * time.Millisecond)
-               v.locker.Unlock()
        }()
-       err := v.Put(ctx, TestHash, TestBlock)
+       err := v.BlockWrite(ctx, TestHash, TestBlock)
        if err != context.Canceled {
-               c.Errorf("Put() returned %s -- expected short read / canceled", err)
+               c.Errorf("BlockWrite() returned %s -- expected short read / canceled", err)
        }
 }
 
-func (s *UnixVolumeSuite) TestUnixVolumeContextCancelGet(c *check.C) {
-       v := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestUnixVolumeContextCancelBlockRead(c *check.C) {
+       v := s.newTestableUnixVolume(c, s.params, true)
        defer v.Teardown()
-       bpath := v.blockPath(TestHash)
-       v.PutRaw(TestHash, TestBlock)
-       os.Remove(bpath)
-       err := syscall.Mkfifo(bpath, 0600)
+       err := v.BlockWrite(context.Background(), TestHash, TestBlock)
        if err != nil {
-               c.Fatalf("Mkfifo %s: %s", bpath, err)
+               c.Fatal(err)
        }
-       defer os.Remove(bpath)
        ctx, cancel := context.WithCancel(context.Background())
+       v.locker.Lock()
+       defer v.locker.Unlock()
        go func() {
                time.Sleep(50 * time.Millisecond)
                cancel()
        }()
-       buf := make([]byte, len(TestBlock))
-       n, err := v.Get(ctx, TestHash, buf)
-       if n == len(TestBlock) || err != context.Canceled {
-               c.Errorf("Get() returned %d, %s -- expected short read / canceled", n, err)
+       n, err := v.BlockRead(ctx, TestHash, io.Discard)
+       if n > 0 || err != context.Canceled {
+               c.Errorf("BlockRead() returned %d, %s -- expected short read / canceled", n, err)
        }
 }
 
-func (s *UnixVolumeSuite) TestStats(c *check.C) {
-       vol := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestStats(c *check.C) {
+       vol := s.newTestableUnixVolume(c, s.params, false)
        stats := func() string {
                buf, err := json.Marshal(vol.InternalStats())
                c.Check(err, check.IsNil)
@@ -390,8 +316,7 @@ func (s *UnixVolumeSuite) TestStats(c *check.C) {
        c.Check(stats(), check.Matches, `.*"StatOps":1,.*`) // (*UnixVolume)check() calls Stat() once
        c.Check(stats(), check.Matches, `.*"Errors":0,.*`)
 
-       loc := "acbd18db4cc2f85cedef654fccc4a4d8"
-       _, err := vol.Get(context.Background(), loc, make([]byte, 3))
+       _, err := vol.BlockRead(context.Background(), fooHash, io.Discard)
        c.Check(err, check.NotNil)
        c.Check(stats(), check.Matches, `.*"StatOps":[^0],.*`)
        c.Check(stats(), check.Matches, `.*"Errors":[^0],.*`)
@@ -400,33 +325,33 @@ func (s *UnixVolumeSuite) TestStats(c *check.C) {
        c.Check(stats(), check.Matches, `.*"OpenOps":0,.*`)
        c.Check(stats(), check.Matches, `.*"CreateOps":0,.*`)
 
-       err = vol.Put(context.Background(), loc, []byte("foo"))
+       err = vol.BlockWrite(context.Background(), fooHash, []byte("foo"))
        c.Check(err, check.IsNil)
        c.Check(stats(), check.Matches, `.*"OutBytes":3,.*`)
        c.Check(stats(), check.Matches, `.*"CreateOps":1,.*`)
        c.Check(stats(), check.Matches, `.*"OpenOps":0,.*`)
        c.Check(stats(), check.Matches, `.*"UtimesOps":1,.*`)
 
-       err = vol.Touch(loc)
+       err = vol.BlockTouch(fooHash)
        c.Check(err, check.IsNil)
        c.Check(stats(), check.Matches, `.*"FlockOps":1,.*`)
        c.Check(stats(), check.Matches, `.*"OpenOps":1,.*`)
        c.Check(stats(), check.Matches, `.*"UtimesOps":2,.*`)
 
-       _, err = vol.Get(context.Background(), loc, make([]byte, 3))
-       c.Check(err, check.IsNil)
-       err = vol.Compare(context.Background(), loc, []byte("foo"))
+       buf := bytes.NewBuffer(nil)
+       _, err = vol.BlockRead(context.Background(), fooHash, buf)
        c.Check(err, check.IsNil)
-       c.Check(stats(), check.Matches, `.*"InBytes":6,.*`)
-       c.Check(stats(), check.Matches, `.*"OpenOps":3,.*`)
+       c.Check(buf.String(), check.Equals, "foo")
+       c.Check(stats(), check.Matches, `.*"InBytes":3,.*`)
+       c.Check(stats(), check.Matches, `.*"OpenOps":2,.*`)
 
-       err = vol.Trash(loc)
+       err = vol.BlockTrash(fooHash)
        c.Check(err, check.IsNil)
        c.Check(stats(), check.Matches, `.*"FlockOps":2,.*`)
 }
 
-func (s *UnixVolumeSuite) TestSkipUnusedDirs(c *check.C) {
-       vol := s.newTestableUnixVolume(c, s.cluster, arvados.Volume{Replication: 1}, s.metrics, false)
+func (s *unixVolumeSuite) TestSkipUnusedDirs(c *check.C) {
+       vol := s.newTestableUnixVolume(c, s.params, false)
 
        err := os.Mkdir(vol.UnixVolume.Root+"/aaa", 0777)
        c.Assert(err, check.IsNil)
index f597ff578106544c54763c9847a3190b53154130..41a0eba86f5c281b265e0a605ab4f34e7ff20a5d 100644 (file)
@@ -6,426 +6,35 @@ package keepstore
 
 import (
        "context"
-       "crypto/rand"
-       "fmt"
        "io"
-       "math/big"
-       "sort"
-       "sync/atomic"
        "time"
 
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "github.com/sirupsen/logrus"
 )
 
-type BlockWriter interface {
-       // WriteBlock reads all data from r, writes it to a backing
-       // store as "loc", and returns the number of bytes written.
-       WriteBlock(ctx context.Context, loc string, r io.Reader) error
-}
-
-type BlockReader interface {
-       // ReadBlock retrieves data previously stored as "loc" and
-       // writes it to w.
-       ReadBlock(ctx context.Context, loc string, w io.Writer) error
-}
-
-var driver = map[string]func(*arvados.Cluster, arvados.Volume, logrus.FieldLogger, *volumeMetricsVecs) (Volume, error){}
-
-// A Volume is an interface representing a Keep back-end storage unit:
-// for example, a single mounted disk, a RAID array, an Amazon S3 volume,
-// etc.
-type Volume interface {
-       // Get a block: copy the block data into buf, and return the
-       // number of bytes copied.
-       //
-       // loc is guaranteed to consist of 32 or more lowercase hex
-       // digits.
-       //
-       // Get should not verify the integrity of the data: it should
-       // just return whatever was found in its backing
-       // store. (Integrity checking is the caller's responsibility.)
-       //
-       // If an error is encountered that prevents it from
-       // retrieving the data, that error should be returned so the
-       // caller can log (and send to the client) a more useful
-       // message.
-       //
-       // If the error is "not found", and there's no particular
-       // reason to expect the block to be found (other than that a
-       // caller is asking for it), the returned error should satisfy
-       // os.IsNotExist(err): this is a normal condition and will not
-       // be logged as an error (except that a 404 will appear in the
-       // access log if the block is not found on any other volumes
-       // either).
-       //
-       // If the data in the backing store is bigger than len(buf),
-       // then Get is permitted to return an error without reading
-       // any of the data.
-       //
-       // len(buf) will not exceed BlockSize.
-       Get(ctx context.Context, loc string, buf []byte) (int, error)
-
-       // Compare the given data with the stored data (i.e., what Get
-       // would return). If equal, return nil. If not, return
-       // CollisionError or DiskHashError (depending on whether the
-       // data on disk matches the expected hash), or whatever error
-       // was encountered opening/reading the stored data.
-       Compare(ctx context.Context, loc string, data []byte) error
-
-       // Put writes a block to an underlying storage device.
-       //
-       // loc is as described in Get.
-       //
-       // len(block) is guaranteed to be between 0 and BlockSize.
-       //
-       // If a block is already stored under the same name (loc) with
-       // different content, Put must either overwrite the existing
-       // data with the new data or return a non-nil error. When
-       // overwriting existing data, it must never leave the storage
-       // device in an inconsistent state: a subsequent call to Get
-       // must return either the entire old block, the entire new
-       // block, or an error. (An implementation that cannot peform
-       // atomic updates must leave the old data alone and return an
-       // error.)
-       //
-       // Put also sets the timestamp for the given locator to the
-       // current time.
-       //
-       // Put must return a non-nil error unless it can guarantee
-       // that the entire block has been written and flushed to
-       // persistent storage, and that its timestamp is current. Of
-       // course, this guarantee is only as good as the underlying
-       // storage device, but it is Put's responsibility to at least
-       // get whatever guarantee is offered by the storage device.
-       //
-       // Put should not verify that loc==hash(block): this is the
-       // caller's responsibility.
-       Put(ctx context.Context, loc string, block []byte) error
-
-       // Touch sets the timestamp for the given locator to the
-       // current time.
-       //
-       // loc is as described in Get.
-       //
-       // If invoked at time t0, Touch must guarantee that a
-       // subsequent call to Mtime will return a timestamp no older
-       // than {t0 minus one second}. For example, if Touch is called
-       // at 2015-07-07T01:23:45.67890123Z, it is acceptable for a
-       // subsequent Mtime to return any of the following:
-       //
-       //   - 2015-07-07T01:23:45.00000000Z
-       //   - 2015-07-07T01:23:45.67890123Z
-       //   - 2015-07-07T01:23:46.67890123Z
-       //   - 2015-07-08T00:00:00.00000000Z
-       //
-       // It is not acceptable for a subsequente Mtime to return
-       // either of the following:
-       //
-       //   - 2015-07-07T00:00:00.00000000Z -- ERROR
-       //   - 2015-07-07T01:23:44.00000000Z -- ERROR
-       //
-       // Touch must return a non-nil error if the timestamp cannot
-       // be updated.
-       Touch(loc string) error
-
-       // Mtime returns the stored timestamp for the given locator.
-       //
-       // loc is as described in Get.
-       //
-       // Mtime must return a non-nil error if the given block is not
-       // found or the timestamp could not be retrieved.
-       Mtime(loc string) (time.Time, error)
-
-       // IndexTo writes a complete list of locators with the given
-       // prefix for which Get() can retrieve data.
-       //
-       // prefix consists of zero or more lowercase hexadecimal
-       // digits.
-       //
-       // Each locator must be written to the given writer using the
-       // following format:
-       //
-       //   loc "+" size " " timestamp "\n"
-       //
-       // where:
-       //
-       //   - size is the number of bytes of content, given as a
-       //     decimal number with one or more digits
-       //
-       //   - timestamp is the timestamp stored for the locator,
-       //     given as a decimal number of seconds after January 1,
-       //     1970 UTC.
-       //
-       // IndexTo must not write any other data to writer: for
-       // example, it must not write any blank lines.
-       //
-       // If an error makes it impossible to provide a complete
-       // index, IndexTo must return a non-nil error. It is
-       // acceptable to return a non-nil error after writing a
-       // partial index to writer.
-       //
-       // The resulting index is not expected to be sorted in any
-       // particular order.
-       IndexTo(prefix string, writer io.Writer) error
-
-       // Trash moves the block data from the underlying storage
-       // device to trash area. The block then stays in trash for
-       // BlobTrashLifetime before it is actually deleted.
-       //
-       // loc is as described in Get.
-       //
-       // If the timestamp for the given locator is newer than
-       // BlobSigningTTL, Trash must not trash the data.
-       //
-       // If a Trash operation overlaps with any Touch or Put
-       // operations on the same locator, the implementation must
-       // ensure one of the following outcomes:
-       //
-       //   - Touch and Put return a non-nil error, or
-       //   - Trash does not trash the block, or
-       //   - Both of the above.
-       //
-       // If it is possible for the storage device to be accessed by
-       // a different process or host, the synchronization mechanism
-       // should also guard against races with other processes and
-       // hosts. If such a mechanism is not available, there must be
-       // a mechanism for detecting unsafe configurations, alerting
-       // the operator, and aborting or falling back to a read-only
-       // state. In other words, running multiple keepstore processes
-       // with the same underlying storage device must either work
-       // reliably or fail outright.
-       //
-       // Corollary: A successful Touch or Put guarantees a block
-       // will not be trashed for at least BlobSigningTTL seconds.
-       Trash(loc string) error
-
-       // Untrash moves block from trash back into store
-       Untrash(loc string) error
-
-       // Status returns a *VolumeStatus representing the current
-       // in-use and available storage capacity and an
-       // implementation-specific volume identifier (e.g., "mount
-       // point" for a UnixVolume).
-       Status() *VolumeStatus
-
-       // String returns an identifying label for this volume,
-       // suitable for including in log messages. It should contain
-       // enough information to uniquely identify the underlying
-       // storage device, but should not contain any credentials or
-       // secrets.
-       String() string
-
-       // EmptyTrash looks for trashed blocks that exceeded
-       // BlobTrashLifetime and deletes them from the volume.
+// volume is the interface to a back-end storage device.
+type volume interface {
+       BlockRead(ctx context.Context, hash string, writeTo io.Writer) (int, error)
+       BlockWrite(ctx context.Context, hash string, data []byte) error
+       DeviceID() string
+       BlockTouch(hash string) error
+       BlockTrash(hash string) error
+       BlockUntrash(hash string) error
+       Index(ctx context.Context, prefix string, writeTo io.Writer) error
+       Mtime(hash string) (time.Time, error)
        EmptyTrash()
-
-       // Return a globally unique ID of the underlying storage
-       // device if possible, otherwise "".
-       GetDeviceID() string
-}
-
-// A VolumeWithExamples provides example configs to display in the
-// -help message.
-type VolumeWithExamples interface {
-       Volume
-       Examples() []Volume
 }
 
-// A VolumeManager tells callers which volumes can read, which volumes
-// can write, and on which volume the next write should be attempted.
-type VolumeManager interface {
-       // Mounts returns all mounts (volume attachments).
-       Mounts() []*VolumeMount
-
-       // Lookup returns the mount with the given UUID. Returns nil
-       // if the mount does not exist. If write==true, returns nil if
-       // the mount is not writable.
-       Lookup(uuid string, write bool) *VolumeMount
-
-       // AllReadable returns all mounts.
-       AllReadable() []*VolumeMount
-
-       // AllWritable returns all mounts that aren't known to be in
-       // a read-only state. (There is no guarantee that a write to
-       // one will succeed, though.)
-       AllWritable() []*VolumeMount
-
-       // NextWritable returns the volume where the next new block
-       // should be written. A VolumeManager can select a volume in
-       // order to distribute activity across spindles, fill up disks
-       // with more free space, etc.
-       NextWritable() *VolumeMount
-
-       // VolumeStats returns the ioStats used for tracking stats for
-       // the given Volume.
-       VolumeStats(Volume) *ioStats
-
-       // Close shuts down the volume manager cleanly.
-       Close()
-}
-
-// A VolumeMount is an attachment of a Volume to a VolumeManager.
-type VolumeMount struct {
-       arvados.KeepMount
-       Volume
-}
-
-// Generate a UUID the way API server would for a "KeepVolumeMount"
-// object.
-func (*VolumeMount) generateUUID() string {
-       var max big.Int
-       _, ok := max.SetString("zzzzzzzzzzzzzzz", 36)
-       if !ok {
-               panic("big.Int parse failed")
-       }
-       r, err := rand.Int(rand.Reader, &max)
-       if err != nil {
-               panic(err)
-       }
-       return fmt.Sprintf("zzzzz-ivpuk-%015s", r.Text(36))
-}
-
-// RRVolumeManager is a round-robin VolumeManager: the Nth call to
-// NextWritable returns the (N % len(writables))th writable Volume
-// (where writables are all Volumes v where v.Writable()==true).
-type RRVolumeManager struct {
-       mounts    []*VolumeMount
-       mountMap  map[string]*VolumeMount
-       readables []*VolumeMount
-       writables []*VolumeMount
-       counter   uint32
-       iostats   map[Volume]*ioStats
-}
-
-func makeRRVolumeManager(logger logrus.FieldLogger, cluster *arvados.Cluster, myURL arvados.URL, metrics *volumeMetricsVecs) (*RRVolumeManager, error) {
-       vm := &RRVolumeManager{
-               iostats: make(map[Volume]*ioStats),
-       }
-       vm.mountMap = make(map[string]*VolumeMount)
-       for uuid, cfgvol := range cluster.Volumes {
-               va, ok := cfgvol.AccessViaHosts[myURL]
-               if !ok && len(cfgvol.AccessViaHosts) > 0 {
-                       continue
-               }
-               dri, ok := driver[cfgvol.Driver]
-               if !ok {
-                       return nil, fmt.Errorf("volume %s: invalid driver %q", uuid, cfgvol.Driver)
-               }
-               vol, err := dri(cluster, cfgvol, logger, metrics)
-               if err != nil {
-                       return nil, fmt.Errorf("error initializing volume %s: %s", uuid, err)
-               }
-               sc := cfgvol.StorageClasses
-               if len(sc) == 0 {
-                       sc = map[string]bool{"default": true}
-               }
-               repl := cfgvol.Replication
-               if repl < 1 {
-                       repl = 1
-               }
-               mnt := &VolumeMount{
-                       KeepMount: arvados.KeepMount{
-                               UUID:           uuid,
-                               DeviceID:       vol.GetDeviceID(),
-                               AllowWrite:     !va.ReadOnly && !cfgvol.ReadOnly,
-                               AllowTrash:     !va.ReadOnly && (!cfgvol.ReadOnly || cfgvol.AllowTrashWhenReadOnly),
-                               Replication:    repl,
-                               StorageClasses: sc,
-                       },
-                       Volume: vol,
-               }
-               vm.iostats[vol] = &ioStats{}
-               vm.mounts = append(vm.mounts, mnt)
-               vm.mountMap[uuid] = mnt
-               vm.readables = append(vm.readables, mnt)
-               if mnt.KeepMount.AllowWrite {
-                       vm.writables = append(vm.writables, mnt)
-               }
-               logger.Printf("started volume %s (%s), AllowWrite=%v, AllowTrash=%v", uuid, vol, mnt.AllowWrite, mnt.AllowTrash)
-       }
-       // pri(mnt): return highest priority of any storage class
-       // offered by mnt
-       pri := func(mnt *VolumeMount) int {
-               any, best := false, 0
-               for class := range mnt.KeepMount.StorageClasses {
-                       if p := cluster.StorageClasses[class].Priority; !any || best < p {
-                               best = p
-                               any = true
-                       }
-               }
-               return best
-       }
-       // less(a,b): sort first by highest priority of any offered
-       // storage class (highest->lowest), then by volume UUID
-       less := func(a, b *VolumeMount) bool {
-               if pa, pb := pri(a), pri(b); pa != pb {
-                       return pa > pb
-               } else {
-                       return a.KeepMount.UUID < b.KeepMount.UUID
-               }
-       }
-       sort.Slice(vm.readables, func(i, j int) bool {
-               return less(vm.readables[i], vm.readables[j])
-       })
-       sort.Slice(vm.writables, func(i, j int) bool {
-               return less(vm.writables[i], vm.writables[j])
-       })
-       sort.Slice(vm.mounts, func(i, j int) bool {
-               return less(vm.mounts[i], vm.mounts[j])
-       })
-       return vm, nil
-}
-
-func (vm *RRVolumeManager) Mounts() []*VolumeMount {
-       return vm.mounts
-}
-
-func (vm *RRVolumeManager) Lookup(uuid string, needWrite bool) *VolumeMount {
-       if mnt, ok := vm.mountMap[uuid]; ok && (!needWrite || mnt.AllowWrite) {
-               return mnt
-       }
-       return nil
-}
-
-// AllReadable returns an array of all readable volumes
-func (vm *RRVolumeManager) AllReadable() []*VolumeMount {
-       return vm.readables
-}
-
-// AllWritable returns writable volumes, sorted by priority/uuid. Used
-// by CompareAndTouch to ensure higher-priority volumes are checked
-// first.
-func (vm *RRVolumeManager) AllWritable() []*VolumeMount {
-       return vm.writables
-}
-
-// NextWritable returns writable volumes, rotated by vm.counter so
-// each volume gets a turn to be first. Used by PutBlock to distribute
-// new data across available volumes.
-func (vm *RRVolumeManager) NextWritable() []*VolumeMount {
-       if len(vm.writables) == 0 {
-               return nil
-       }
-       offset := (int(atomic.AddUint32(&vm.counter, 1)) - 1) % len(vm.writables)
-       return append(append([]*VolumeMount(nil), vm.writables[offset:]...), vm.writables[:offset]...)
-}
-
-// VolumeStats returns an ioStats for the given volume.
-func (vm *RRVolumeManager) VolumeStats(v Volume) *ioStats {
-       return vm.iostats[v]
-}
-
-// Close the RRVolumeManager
-func (vm *RRVolumeManager) Close() {
-}
+type volumeDriver func(newVolumeParams) (volume, error)
 
-// VolumeStatus describes the current condition of a volume
-type VolumeStatus struct {
-       MountPoint string
-       DeviceNum  uint64
-       BytesFree  uint64
-       BytesUsed  uint64
+type newVolumeParams struct {
+       UUID         string
+       Cluster      *arvados.Cluster
+       ConfigVolume arvados.Volume
+       Logger       logrus.FieldLogger
+       MetricsVecs  *volumeMetricsVecs
+       BufferPool   *bufferPool
 }
 
 // ioStats tracks I/O statistics for a volume or server
index 21804124316fe2ea9421c25eeeec14c9aab190aa..22667743ddc2acc0a80d2034e9b3c5b2eb362085 100644 (file)
@@ -9,11 +9,13 @@ import (
        "context"
        "crypto/md5"
        "fmt"
+       "io"
        "os"
        "regexp"
        "sort"
        "strconv"
        "strings"
+       "sync"
        "time"
 
        "git.arvados.org/arvados.git/sdk/go/arvados"
@@ -39,7 +41,7 @@ type TB interface {
 // A TestableVolumeFactory returns a new TestableVolume. The factory
 // function, and the TestableVolume it returns, can use "t" to write
 // logs, fail the current test, etc.
-type TestableVolumeFactory func(t TB, cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) TestableVolume
+type TestableVolumeFactory func(t TB, params newVolumeParams) TestableVolume
 
 // DoGenericVolumeTests runs a set of tests that every TestableVolume
 // is expected to pass. It calls factory to create a new TestableVolume
@@ -51,16 +53,6 @@ func DoGenericVolumeTests(t TB, readonly bool, factory TestableVolumeFactory) {
        s.testGet(t, factory)
        s.testGetNoSuchBlock(t, factory)
 
-       s.testCompareNonexistent(t, factory)
-       s.testCompareSameContent(t, factory, TestHash, TestBlock)
-       s.testCompareSameContent(t, factory, EmptyHash, EmptyBlock)
-       s.testCompareWithCollision(t, factory, TestHash, TestBlock, []byte("baddata"))
-       s.testCompareWithCollision(t, factory, TestHash, TestBlock, EmptyBlock)
-       s.testCompareWithCollision(t, factory, EmptyHash, EmptyBlock, TestBlock)
-       s.testCompareWithCorruptStoredData(t, factory, TestHash, TestBlock, []byte("baddata"))
-       s.testCompareWithCorruptStoredData(t, factory, TestHash, TestBlock, EmptyBlock)
-       s.testCompareWithCorruptStoredData(t, factory, EmptyHash, EmptyBlock, []byte("baddata"))
-
        if !readonly {
                s.testPutBlockWithSameContent(t, factory, TestHash, TestBlock)
                s.testPutBlockWithSameContent(t, factory, EmptyHash, EmptyBlock)
@@ -76,7 +68,7 @@ func DoGenericVolumeTests(t TB, readonly bool, factory TestableVolumeFactory) {
 
        s.testMtimeNoSuchBlock(t, factory)
 
-       s.testIndexTo(t, factory)
+       s.testIndex(t, factory)
 
        if !readonly {
                s.testDeleteNewBlock(t, factory)
@@ -84,33 +76,24 @@ func DoGenericVolumeTests(t TB, readonly bool, factory TestableVolumeFactory) {
        }
        s.testDeleteNoSuchBlock(t, factory)
 
-       s.testStatus(t, factory)
-
        s.testMetrics(t, readonly, factory)
 
-       s.testString(t, factory)
-
-       if readonly {
-               s.testUpdateReadOnly(t, factory)
-       }
-
        s.testGetConcurrent(t, factory)
        if !readonly {
                s.testPutConcurrent(t, factory)
-
                s.testPutFullBlock(t, factory)
+               s.testTrashUntrash(t, readonly, factory)
+               s.testTrashEmptyTrashUntrash(t, factory)
        }
-
-       s.testTrashUntrash(t, readonly, factory)
-       s.testTrashEmptyTrashUntrash(t, factory)
 }
 
 type genericVolumeSuite struct {
-       cluster  *arvados.Cluster
-       volume   arvados.Volume
-       logger   logrus.FieldLogger
-       metrics  *volumeMetricsVecs
-       registry *prometheus.Registry
+       cluster    *arvados.Cluster
+       volume     arvados.Volume
+       logger     logrus.FieldLogger
+       metrics    *volumeMetricsVecs
+       registry   *prometheus.Registry
+       bufferPool *bufferPool
 }
 
 func (s *genericVolumeSuite) setup(t TB) {
@@ -118,10 +101,18 @@ func (s *genericVolumeSuite) setup(t TB) {
        s.logger = ctxlog.TestLogger(t)
        s.registry = prometheus.NewRegistry()
        s.metrics = newVolumeMetricsVecs(s.registry)
+       s.bufferPool = newBufferPool(s.logger, 8, s.registry)
 }
 
 func (s *genericVolumeSuite) newVolume(t TB, factory TestableVolumeFactory) TestableVolume {
-       return factory(t, s.cluster, s.volume, s.logger, s.metrics)
+       return factory(t, newVolumeParams{
+               UUID:         "zzzzz-nyw5e-999999999999999",
+               Cluster:      s.cluster,
+               ConfigVolume: s.volume,
+               Logger:       s.logger,
+               MetricsVecs:  s.metrics,
+               BufferPool:   s.bufferPool,
+       })
 }
 
 // Put a test block, get it and verify content
@@ -131,95 +122,30 @@ func (s *genericVolumeSuite) testGet(t TB, factory TestableVolumeFactory) {
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       v.PutRaw(TestHash, TestBlock)
-
-       buf := make([]byte, BlockSize)
-       n, err := v.Get(context.Background(), TestHash, buf)
+       err := v.BlockWrite(context.Background(), TestHash, TestBlock)
        if err != nil {
-               t.Fatal(err)
-       }
-
-       if bytes.Compare(buf[:n], TestBlock) != 0 {
-               t.Errorf("expected %s, got %s", string(TestBlock), string(buf))
-       }
-}
-
-// Invoke get on a block that does not exist in volume; should result in error
-// Test should pass for both writable and read-only volumes
-func (s *genericVolumeSuite) testGetNoSuchBlock(t TB, factory TestableVolumeFactory) {
-       s.setup(t)
-       v := s.newVolume(t, factory)
-       defer v.Teardown()
-
-       buf := make([]byte, BlockSize)
-       if _, err := v.Get(context.Background(), TestHash2, buf); err == nil {
-               t.Errorf("Expected error while getting non-existing block %v", TestHash2)
-       }
-}
-
-// Compare() should return os.ErrNotExist if the block does not exist.
-// Otherwise, writing new data causes CompareAndTouch() to generate
-// error logs even though everything is working fine.
-func (s *genericVolumeSuite) testCompareNonexistent(t TB, factory TestableVolumeFactory) {
-       s.setup(t)
-       v := s.newVolume(t, factory)
-       defer v.Teardown()
-
-       err := v.Compare(context.Background(), TestHash, TestBlock)
-       if err != os.ErrNotExist {
-               t.Errorf("Got err %T %q, expected os.ErrNotExist", err, err)
+               t.Error(err)
        }
-}
 
-// Put a test block and compare the locator with same content
-// Test should pass for both writable and read-only volumes
-func (s *genericVolumeSuite) testCompareSameContent(t TB, factory TestableVolumeFactory, testHash string, testData []byte) {
-       s.setup(t)
-       v := s.newVolume(t, factory)
-       defer v.Teardown()
-
-       v.PutRaw(testHash, testData)
-
-       // Compare the block locator with same content
-       err := v.Compare(context.Background(), testHash, testData)
+       buf := bytes.NewBuffer(nil)
+       _, err = v.BlockRead(context.Background(), TestHash, buf)
        if err != nil {
-               t.Errorf("Got err %q, expected nil", err)
+               t.Error(err)
        }
-}
-
-// Test behavior of Compare() when stored data matches expected
-// checksum but differs from new data we need to store. Requires
-// testHash = md5(testDataA).
-//
-// Test should pass for both writable and read-only volumes
-func (s *genericVolumeSuite) testCompareWithCollision(t TB, factory TestableVolumeFactory, testHash string, testDataA, testDataB []byte) {
-       s.setup(t)
-       v := s.newVolume(t, factory)
-       defer v.Teardown()
-
-       v.PutRaw(testHash, testDataA)
-
-       // Compare the block locator with different content; collision
-       err := v.Compare(context.Background(), TestHash, testDataB)
-       if err == nil {
-               t.Errorf("Got err nil, expected error due to collision")
+       if bytes.Compare(buf.Bytes(), TestBlock) != 0 {
+               t.Errorf("expected %s, got %s", "foo", buf.String())
        }
 }
 
-// Test behavior of Compare() when stored data has become
-// corrupted. Requires testHash = md5(testDataA) != md5(testDataB).
-//
+// Invoke get on a block that does not exist in volume; should result in error
 // Test should pass for both writable and read-only volumes
-func (s *genericVolumeSuite) testCompareWithCorruptStoredData(t TB, factory TestableVolumeFactory, testHash string, testDataA, testDataB []byte) {
+func (s *genericVolumeSuite) testGetNoSuchBlock(t TB, factory TestableVolumeFactory) {
        s.setup(t)
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       v.PutRaw(TestHash, testDataB)
-
-       err := v.Compare(context.Background(), testHash, testDataA)
-       if err == nil || err == CollisionError {
-               t.Errorf("Got err %+v, expected non-collision error", err)
+       if _, err := v.BlockRead(context.Background(), barHash, io.Discard); err == nil {
+               t.Errorf("Expected error while getting non-existing block %v", barHash)
        }
 }
 
@@ -230,12 +156,12 @@ func (s *genericVolumeSuite) testPutBlockWithSameContent(t TB, factory TestableV
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       err := v.Put(context.Background(), testHash, testData)
+       err := v.BlockWrite(context.Background(), testHash, testData)
        if err != nil {
                t.Errorf("Got err putting block %q: %q, expected nil", TestBlock, err)
        }
 
-       err = v.Put(context.Background(), testHash, testData)
+       err = v.BlockWrite(context.Background(), testHash, testData)
        if err != nil {
                t.Errorf("Got err putting block second time %q: %q, expected nil", TestBlock, err)
        }
@@ -248,23 +174,23 @@ func (s *genericVolumeSuite) testPutBlockWithDifferentContent(t TB, factory Test
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       v.PutRaw(testHash, testDataA)
+       v.BlockWrite(context.Background(), testHash, testDataA)
 
-       putErr := v.Put(context.Background(), testHash, testDataB)
-       buf := make([]byte, BlockSize)
-       n, getErr := v.Get(context.Background(), testHash, buf)
+       putErr := v.BlockWrite(context.Background(), testHash, testDataB)
+       buf := bytes.NewBuffer(nil)
+       _, getErr := v.BlockRead(context.Background(), testHash, buf)
        if putErr == nil {
                // Put must not return a nil error unless it has
                // overwritten the existing data.
-               if bytes.Compare(buf[:n], testDataB) != 0 {
-                       t.Errorf("Put succeeded but Get returned %+q, expected %+q", buf[:n], testDataB)
+               if buf.String() != string(testDataB) {
+                       t.Errorf("Put succeeded but Get returned %+q, expected %+q", buf, testDataB)
                }
        } else {
                // It is permissible for Put to fail, but it must
                // leave us with either the original data, the new
                // data, or nothing at all.
-               if getErr == nil && bytes.Compare(buf[:n], testDataA) != 0 && bytes.Compare(buf[:n], testDataB) != 0 {
-                       t.Errorf("Put failed but Get returned %+q, which is neither %+q nor %+q", buf[:n], testDataA, testDataB)
+               if getErr == nil && buf.String() != string(testDataA) && buf.String() != string(testDataB) {
+                       t.Errorf("Put failed but Get returned %+q, which is neither %+q nor %+q", buf, testDataA, testDataB)
                }
        }
 }
@@ -276,46 +202,48 @@ func (s *genericVolumeSuite) testPutMultipleBlocks(t TB, factory TestableVolumeF
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       err := v.Put(context.Background(), TestHash, TestBlock)
+       err := v.BlockWrite(context.Background(), TestHash, TestBlock)
        if err != nil {
                t.Errorf("Got err putting block %q: %q, expected nil", TestBlock, err)
        }
 
-       err = v.Put(context.Background(), TestHash2, TestBlock2)
+       err = v.BlockWrite(context.Background(), TestHash2, TestBlock2)
        if err != nil {
                t.Errorf("Got err putting block %q: %q, expected nil", TestBlock2, err)
        }
 
-       err = v.Put(context.Background(), TestHash3, TestBlock3)
+       err = v.BlockWrite(context.Background(), TestHash3, TestBlock3)
        if err != nil {
                t.Errorf("Got err putting block %q: %q, expected nil", TestBlock3, err)
        }
 
-       data := make([]byte, BlockSize)
-       n, err := v.Get(context.Background(), TestHash, data)
+       buf := bytes.NewBuffer(nil)
+       _, err = v.BlockRead(context.Background(), TestHash, buf)
        if err != nil {
                t.Error(err)
        } else {
-               if bytes.Compare(data[:n], TestBlock) != 0 {
-                       t.Errorf("Block present, but got %+q, expected %+q", data[:n], TestBlock)
+               if bytes.Compare(buf.Bytes(), TestBlock) != 0 {
+                       t.Errorf("Block present, but got %+q, expected %+q", buf, TestBlock)
                }
        }
 
-       n, err = v.Get(context.Background(), TestHash2, data)
+       buf.Reset()
+       _, err = v.BlockRead(context.Background(), TestHash2, buf)
        if err != nil {
                t.Error(err)
        } else {
-               if bytes.Compare(data[:n], TestBlock2) != 0 {
-                       t.Errorf("Block present, but got %+q, expected %+q", data[:n], TestBlock2)
+               if bytes.Compare(buf.Bytes(), TestBlock2) != 0 {
+                       t.Errorf("Block present, but got %+q, expected %+q", buf, TestBlock2)
                }
        }
 
-       n, err = v.Get(context.Background(), TestHash3, data)
+       buf.Reset()
+       _, err = v.BlockRead(context.Background(), TestHash3, buf)
        if err != nil {
                t.Error(err)
        } else {
-               if bytes.Compare(data[:n], TestBlock3) != 0 {
-                       t.Errorf("Block present, but to %+q, expected %+q", data[:n], TestBlock3)
+               if bytes.Compare(buf.Bytes(), TestBlock3) != 0 {
+                       t.Errorf("Block present, but to %+q, expected %+q", buf, TestBlock3)
                }
        }
 }
@@ -328,13 +256,13 @@ func (s *genericVolumeSuite) testPutAndTouch(t TB, factory TestableVolumeFactory
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       if err := v.Put(context.Background(), TestHash, TestBlock); err != nil {
+       if err := v.BlockWrite(context.Background(), TestHash, TestBlock); err != nil {
                t.Error(err)
        }
 
        // We'll verify { t0 < threshold < t1 }, where t0 is the
-       // existing block's timestamp on disk before Put() and t1 is
-       // its timestamp after Put().
+       // existing block's timestamp on disk before BlockWrite() and t1 is
+       // its timestamp after BlockWrite().
        threshold := time.Now().Add(-time.Second)
 
        // Set the stored block's mtime far enough in the past that we
@@ -348,7 +276,7 @@ func (s *genericVolumeSuite) testPutAndTouch(t TB, factory TestableVolumeFactory
        }
 
        // Write the same block again.
-       if err := v.Put(context.Background(), TestHash, TestBlock); err != nil {
+       if err := v.BlockWrite(context.Background(), TestHash, TestBlock); err != nil {
                t.Error(err)
        }
 
@@ -367,7 +295,7 @@ func (s *genericVolumeSuite) testTouchNoSuchBlock(t TB, factory TestableVolumeFa
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       if err := v.Touch(TestHash); err == nil {
+       if err := v.BlockTouch(TestHash); err == nil {
                t.Error("Expected error when attempted to touch a non-existing block")
        }
 }
@@ -384,12 +312,12 @@ func (s *genericVolumeSuite) testMtimeNoSuchBlock(t TB, factory TestableVolumeFa
        }
 }
 
-// Put a few blocks and invoke IndexTo with:
+// Put a few blocks and invoke Index with:
 // * no prefix
 // * with a prefix
 // * with no such prefix
 // Test should pass for both writable and read-only volumes
-func (s *genericVolumeSuite) testIndexTo(t TB, factory TestableVolumeFactory) {
+func (s *genericVolumeSuite) testIndex(t TB, factory TestableVolumeFactory) {
        s.setup(t)
        v := s.newVolume(t, factory)
        defer v.Teardown()
@@ -400,9 +328,9 @@ func (s *genericVolumeSuite) testIndexTo(t TB, factory TestableVolumeFactory) {
        minMtime := time.Now().UTC().UnixNano()
        minMtime -= minMtime % 1e9
 
-       v.PutRaw(TestHash, TestBlock)
-       v.PutRaw(TestHash2, TestBlock2)
-       v.PutRaw(TestHash3, TestBlock3)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash2, TestBlock2)
+       v.BlockWrite(context.Background(), TestHash3, TestBlock3)
 
        maxMtime := time.Now().UTC().UnixNano()
        if maxMtime%1e9 > 0 {
@@ -412,13 +340,13 @@ func (s *genericVolumeSuite) testIndexTo(t TB, factory TestableVolumeFactory) {
 
        // Blocks whose names aren't Keep hashes should be omitted from
        // index
-       v.PutRaw("fffffffffnotreallyahashfffffffff", nil)
-       v.PutRaw("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF", nil)
-       v.PutRaw("f0000000000000000000000000000000f", nil)
-       v.PutRaw("f00", nil)
+       v.BlockWrite(context.Background(), "fffffffffnotreallyahashfffffffff", nil)
+       v.BlockWrite(context.Background(), "FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF", nil)
+       v.BlockWrite(context.Background(), "f0000000000000000000000000000000f", nil)
+       v.BlockWrite(context.Background(), "f00", nil)
 
        buf := new(bytes.Buffer)
-       v.IndexTo("", buf)
+       v.Index(context.Background(), "", buf)
        indexRows := strings.Split(string(buf.Bytes()), "\n")
        sort.Strings(indexRows)
        sortedIndex := strings.Join(indexRows, "\n")
@@ -441,7 +369,7 @@ func (s *genericVolumeSuite) testIndexTo(t TB, factory TestableVolumeFactory) {
 
        for _, prefix := range []string{"f", "f15", "f15ac"} {
                buf = new(bytes.Buffer)
-               v.IndexTo(prefix, buf)
+               v.Index(context.Background(), prefix, buf)
 
                m, err := regexp.MatchString(`^`+TestHash2+`\+\d+ \d+\n$`, string(buf.Bytes()))
                if err != nil {
@@ -453,11 +381,11 @@ func (s *genericVolumeSuite) testIndexTo(t TB, factory TestableVolumeFactory) {
 
        for _, prefix := range []string{"zero", "zip", "zilch"} {
                buf = new(bytes.Buffer)
-               err := v.IndexTo(prefix, buf)
+               err := v.Index(context.Background(), prefix, buf)
                if err != nil {
-                       t.Errorf("Got error on IndexTo with no such prefix %v", err.Error())
+                       t.Errorf("Got error on Index with no such prefix %v", err.Error())
                } else if buf.Len() != 0 {
-                       t.Errorf("Expected empty list for IndexTo with no such prefix %s", prefix)
+                       t.Errorf("Expected empty list for Index with no such prefix %s", prefix)
                }
        }
 }
@@ -471,17 +399,17 @@ func (s *genericVolumeSuite) testDeleteNewBlock(t TB, factory TestableVolumeFact
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       v.Put(context.Background(), TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
 
-       if err := v.Trash(TestHash); err != nil {
+       if err := v.BlockTrash(TestHash); err != nil {
                t.Error(err)
        }
-       data := make([]byte, BlockSize)
-       n, err := v.Get(context.Background(), TestHash, data)
+       buf := bytes.NewBuffer(nil)
+       _, err := v.BlockRead(context.Background(), TestHash, buf)
        if err != nil {
                t.Error(err)
-       } else if bytes.Compare(data[:n], TestBlock) != 0 {
-               t.Errorf("Got data %+q, expected %+q", data[:n], TestBlock)
+       } else if buf.String() != string(TestBlock) {
+               t.Errorf("Got data %+q, expected %+q", buf.String(), TestBlock)
        }
 }
 
@@ -494,36 +422,30 @@ func (s *genericVolumeSuite) testDeleteOldBlock(t TB, factory TestableVolumeFact
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       v.Put(context.Background(), TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
 
-       if err := v.Trash(TestHash); err != nil {
+       if err := v.BlockTrash(TestHash); err != nil {
                t.Error(err)
        }
-       data := make([]byte, BlockSize)
-       if _, err := v.Get(context.Background(), TestHash, data); err == nil || !os.IsNotExist(err) {
+       if _, err := v.BlockRead(context.Background(), TestHash, io.Discard); err == nil || !os.IsNotExist(err) {
                t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
        _, err := v.Mtime(TestHash)
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
-       }
-
-       err = v.Compare(context.Background(), TestHash, TestBlock)
-       if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
        indexBuf := new(bytes.Buffer)
-       v.IndexTo("", indexBuf)
+       v.Index(context.Background(), "", indexBuf)
        if strings.Contains(string(indexBuf.Bytes()), TestHash) {
-               t.Fatalf("Found trashed block in IndexTo")
+               t.Errorf("Found trashed block in Index")
        }
 
-       err = v.Touch(TestHash)
+       err = v.BlockTouch(TestHash)
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 }
 
@@ -534,33 +456,11 @@ func (s *genericVolumeSuite) testDeleteNoSuchBlock(t TB, factory TestableVolumeF
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       if err := v.Trash(TestHash2); err == nil {
+       if err := v.BlockTrash(TestHash2); err == nil {
                t.Errorf("Expected error when attempting to delete a non-existing block")
        }
 }
 
-// Invoke Status and verify that VolumeStatus is returned
-// Test should pass for both writable and read-only volumes
-func (s *genericVolumeSuite) testStatus(t TB, factory TestableVolumeFactory) {
-       s.setup(t)
-       v := s.newVolume(t, factory)
-       defer v.Teardown()
-
-       // Get node status and make a basic sanity check.
-       status := v.Status()
-       if status.DeviceNum == 0 {
-               t.Errorf("uninitialized device_num in %v", status)
-       }
-
-       if status.BytesFree == 0 {
-               t.Errorf("uninitialized bytes_free in %v", status)
-       }
-
-       if status.BytesUsed == 0 {
-               t.Errorf("uninitialized bytes_used in %v", status)
-       }
-}
-
 func getValueFrom(cv *prometheus.CounterVec, lbls prometheus.Labels) float64 {
        c, _ := cv.GetMetricWith(lbls)
        pb := &dto.Metric{}
@@ -575,7 +475,7 @@ func (s *genericVolumeSuite) testMetrics(t TB, readonly bool, factory TestableVo
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       opsC, _, ioC := s.metrics.getCounterVecsFor(prometheus.Labels{"device_id": v.GetDeviceID()})
+       opsC, _, ioC := s.metrics.getCounterVecsFor(prometheus.Labels{"device_id": v.DeviceID()})
 
        if ioC == nil {
                t.Error("ioBytes CounterVec is nil")
@@ -600,7 +500,7 @@ func (s *genericVolumeSuite) testMetrics(t TB, readonly bool, factory TestableVo
 
        // Test Put if volume is writable
        if !readonly {
-               err = v.Put(context.Background(), TestHash, TestBlock)
+               err = v.BlockWrite(context.Background(), TestHash, TestBlock)
                if err != nil {
                        t.Errorf("Got err putting block %q: %q, expected nil", TestBlock, err)
                }
@@ -614,13 +514,12 @@ func (s *genericVolumeSuite) testMetrics(t TB, readonly bool, factory TestableVo
                        t.Error("ioBytes{direction=out} counter shouldn't be zero")
                }
        } else {
-               v.PutRaw(TestHash, TestBlock)
+               v.BlockWrite(context.Background(), TestHash, TestBlock)
        }
 
-       buf := make([]byte, BlockSize)
-       _, err = v.Get(context.Background(), TestHash, buf)
+       _, err = v.BlockRead(context.Background(), TestHash, io.Discard)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        // Check that the operations counter increased
@@ -634,63 +533,6 @@ func (s *genericVolumeSuite) testMetrics(t TB, readonly bool, factory TestableVo
        }
 }
 
-// Invoke String for the volume; expect non-empty result
-// Test should pass for both writable and read-only volumes
-func (s *genericVolumeSuite) testString(t TB, factory TestableVolumeFactory) {
-       s.setup(t)
-       v := s.newVolume(t, factory)
-       defer v.Teardown()
-
-       if id := v.String(); len(id) == 0 {
-               t.Error("Got empty string for v.String()")
-       }
-}
-
-// Putting, updating, touching, and deleting blocks from a read-only volume result in error.
-// Test is intended for only read-only volumes
-func (s *genericVolumeSuite) testUpdateReadOnly(t TB, factory TestableVolumeFactory) {
-       s.setup(t)
-       v := s.newVolume(t, factory)
-       defer v.Teardown()
-
-       v.PutRaw(TestHash, TestBlock)
-       buf := make([]byte, BlockSize)
-
-       // Get from read-only volume should succeed
-       _, err := v.Get(context.Background(), TestHash, buf)
-       if err != nil {
-               t.Errorf("got err %v, expected nil", err)
-       }
-
-       // Put a new block to read-only volume should result in error
-       err = v.Put(context.Background(), TestHash2, TestBlock2)
-       if err == nil {
-               t.Errorf("Expected error when putting block in a read-only volume")
-       }
-       _, err = v.Get(context.Background(), TestHash2, buf)
-       if err == nil {
-               t.Errorf("Expected error when getting block whose put in read-only volume failed")
-       }
-
-       // Touch a block in read-only volume should result in error
-       err = v.Touch(TestHash)
-       if err == nil {
-               t.Errorf("Expected error when touching block in a read-only volume")
-       }
-
-       // Delete a block from a read-only volume should result in error
-       err = v.Trash(TestHash)
-       if err == nil {
-               t.Errorf("Expected error when deleting block from a read-only volume")
-       }
-
-       // Overwriting an existing block in read-only volume should result in error
-       err = v.Put(context.Background(), TestHash, TestBlock)
-       if err == nil {
-               t.Errorf("Expected error when putting block in a read-only volume")
-       }
-}
-
 // Launch concurrent Gets
 // Test should pass for both writable and read-only volumes
 func (s *genericVolumeSuite) testGetConcurrent(t TB, factory TestableVolumeFactory) {
@@ -698,43 +540,43 @@ func (s *genericVolumeSuite) testGetConcurrent(t TB, factory TestableVolumeFacto
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       v.PutRaw(TestHash, TestBlock)
-       v.PutRaw(TestHash2, TestBlock2)
-       v.PutRaw(TestHash3, TestBlock3)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash2, TestBlock2)
+       v.BlockWrite(context.Background(), TestHash3, TestBlock3)
 
        sem := make(chan int)
        go func() {
-               buf := make([]byte, BlockSize)
-               n, err := v.Get(context.Background(), TestHash, buf)
+               buf := bytes.NewBuffer(nil)
+               _, err := v.BlockRead(context.Background(), TestHash, buf)
                if err != nil {
                        t.Errorf("err1: %v", err)
                }
-               if bytes.Compare(buf[:n], TestBlock) != 0 {
-                       t.Errorf("buf should be %s, is %s", string(TestBlock), string(buf[:n]))
+               if buf.String() != string(TestBlock) {
+                       t.Errorf("buf should be %s, is %s", TestBlock, buf)
                }
                sem <- 1
        }()
 
        go func() {
-               buf := make([]byte, BlockSize)
-               n, err := v.Get(context.Background(), TestHash2, buf)
+               buf := bytes.NewBuffer(nil)
+               _, err := v.BlockRead(context.Background(), TestHash2, buf)
                if err != nil {
                        t.Errorf("err2: %v", err)
                }
-               if bytes.Compare(buf[:n], TestBlock2) != 0 {
-                       t.Errorf("buf should be %s, is %s", string(TestBlock2), string(buf[:n]))
+               if buf.String() != string(TestBlock2) {
+                       t.Errorf("buf should be %s, is %s", TestBlock2, buf)
                }
                sem <- 1
        }()
 
        go func() {
-               buf := make([]byte, BlockSize)
-               n, err := v.Get(context.Background(), TestHash3, buf)
+               buf := bytes.NewBuffer(nil)
+               _, err := v.BlockRead(context.Background(), TestHash3, buf)
                if err != nil {
                        t.Errorf("err3: %v", err)
                }
-               if bytes.Compare(buf[:n], TestBlock3) != 0 {
-                       t.Errorf("buf should be %s, is %s", string(TestBlock3), string(buf[:n]))
+               if buf.String() != string(TestBlock3) {
+                       t.Errorf("buf should be %s, is %s", TestBlock3, buf)
                }
                sem <- 1
        }()
@@ -752,60 +594,38 @@ func (s *genericVolumeSuite) testPutConcurrent(t TB, factory TestableVolumeFacto
        v := s.newVolume(t, factory)
        defer v.Teardown()
 
-       sem := make(chan int)
-       go func(sem chan int) {
-               err := v.Put(context.Background(), TestHash, TestBlock)
+       blks := []struct {
+               hash string
+               data []byte
+       }{
+               {hash: TestHash, data: TestBlock},
+               {hash: TestHash2, data: TestBlock2},
+               {hash: TestHash3, data: TestBlock3},
+       }
+
+       var wg sync.WaitGroup
+       for _, blk := range blks {
+               blk := blk
+               wg.Add(1)
+               go func() {
+                       defer wg.Done()
+                       err := v.BlockWrite(context.Background(), blk.hash, blk.data)
+                       if err != nil {
+                               t.Errorf("%s: %v", blk.hash, err)
+                       }
+               }()
+       }
+       wg.Wait()
+
+       // Check that we actually wrote the blocks.
+       for _, blk := range blks {
+               buf := bytes.NewBuffer(nil)
+               _, err := v.BlockRead(context.Background(), blk.hash, buf)
                if err != nil {
-                       t.Errorf("err1: %v", err)
+                       t.Errorf("get %s: %v", blk.hash, err)
+               } else if buf.String() != string(blk.data) {
+                       t.Errorf("get %s: expected %s, got %s", blk.hash, blk.data, buf)
                }
-               sem <- 1
-       }(sem)
-
-       go func(sem chan int) {
-               err := v.Put(context.Background(), TestHash2, TestBlock2)
-               if err != nil {
-                       t.Errorf("err2: %v", err)
-               }
-               sem <- 1
-       }(sem)
-
-       go func(sem chan int) {
-               err := v.Put(context.Background(), TestHash3, TestBlock3)
-               if err != nil {
-                       t.Errorf("err3: %v", err)
-               }
-               sem <- 1
-       }(sem)
-
-       // Wait for all goroutines to finish
-       for done := 0; done < 3; done++ {
-               <-sem
-       }
-
-       // Double check that we actually wrote the blocks we expected to write.
-       buf := make([]byte, BlockSize)
-       n, err := v.Get(context.Background(), TestHash, buf)
-       if err != nil {
-               t.Errorf("Get #1: %v", err)
-       }
-       if bytes.Compare(buf[:n], TestBlock) != 0 {
-               t.Errorf("Get #1: expected %s, got %s", string(TestBlock), string(buf[:n]))
-       }
-
-       n, err = v.Get(context.Background(), TestHash2, buf)
-       if err != nil {
-               t.Errorf("Get #2: %v", err)
-       }
-       if bytes.Compare(buf[:n], TestBlock2) != 0 {
-               t.Errorf("Get #2: expected %s, got %s", string(TestBlock2), string(buf[:n]))
-       }
-
-       n, err = v.Get(context.Background(), TestHash3, buf)
-       if err != nil {
-               t.Errorf("Get #3: %v", err)
-       }
-       if bytes.Compare(buf[:n], TestBlock3) != 0 {
-               t.Errorf("Get #3: expected %s, got %s", string(TestBlock3), string(buf[:n]))
        }
 }
 
@@ -819,17 +639,18 @@ func (s *genericVolumeSuite) testPutFullBlock(t TB, factory TestableVolumeFactor
        wdata[0] = 'a'
        wdata[BlockSize-1] = 'z'
        hash := fmt.Sprintf("%x", md5.Sum(wdata))
-       err := v.Put(context.Background(), hash, wdata)
+       err := v.BlockWrite(context.Background(), hash, wdata)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
-       buf := make([]byte, BlockSize)
-       n, err := v.Get(context.Background(), hash, buf)
+
+       buf := bytes.NewBuffer(nil)
+       _, err = v.BlockRead(context.Background(), hash, buf)
        if err != nil {
                t.Error(err)
        }
-       if bytes.Compare(buf[:n], wdata) != 0 {
-               t.Error("buf %+q != wdata %+q", buf[:n], wdata)
+       if buf.String() != string(wdata) {
+               t.Error("buf %+q != wdata %+q", buf, wdata)
        }
 }
 
@@ -844,48 +665,44 @@ func (s *genericVolumeSuite) testTrashUntrash(t TB, readonly bool, factory Testa
        defer v.Teardown()
 
        // put block and backdate it
-       v.PutRaw(TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
 
-       buf := make([]byte, BlockSize)
-       n, err := v.Get(context.Background(), TestHash, buf)
+       buf := bytes.NewBuffer(nil)
+       _, err := v.BlockRead(context.Background(), TestHash, buf)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
-       if bytes.Compare(buf[:n], TestBlock) != 0 {
-               t.Errorf("Got data %+q, expected %+q", buf[:n], TestBlock)
+       if buf.String() != string(TestBlock) {
+               t.Errorf("Got data %+q, expected %+q", buf, TestBlock)
        }
 
        // Trash
-       err = v.Trash(TestHash)
-       if readonly {
-               if err != MethodDisabledError {
-                       t.Fatal(err)
-               }
-       } else if err != nil {
-               if err != ErrNotImplemented {
-                       t.Fatal(err)
-               }
-       } else {
-               _, err = v.Get(context.Background(), TestHash, buf)
-               if err == nil || !os.IsNotExist(err) {
-                       t.Errorf("os.IsNotExist(%v) should have been true", err)
-               }
+       err = v.BlockTrash(TestHash)
+       if err != nil {
+               t.Error(err)
+               return
+       }
+       buf.Reset()
+       _, err = v.BlockRead(context.Background(), TestHash, buf)
+       if err == nil || !os.IsNotExist(err) {
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
+       }
 
-               // Untrash
-               err = v.Untrash(TestHash)
-               if err != nil {
-                       t.Fatal(err)
-               }
+       // Untrash
+       err = v.BlockUntrash(TestHash)
+       if err != nil {
+               t.Error(err)
        }
 
        // Get the block - after trash and untrash sequence
-       n, err = v.Get(context.Background(), TestHash, buf)
+       buf.Reset()
+       _, err = v.BlockRead(context.Background(), TestHash, buf)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
-       if bytes.Compare(buf[:n], TestBlock) != 0 {
-               t.Errorf("Got data %+q, expected %+q", buf[:n], TestBlock)
+       if buf.String() != string(TestBlock) {
+               t.Errorf("Got data %+q, expected %+q", buf, TestBlock)
        }
 }
 
@@ -895,13 +712,13 @@ func (s *genericVolumeSuite) testTrashEmptyTrashUntrash(t TB, factory TestableVo
        defer v.Teardown()
 
        checkGet := func() error {
-               buf := make([]byte, BlockSize)
-               n, err := v.Get(context.Background(), TestHash, buf)
+               buf := bytes.NewBuffer(nil)
+               _, err := v.BlockRead(context.Background(), TestHash, buf)
                if err != nil {
                        return err
                }
-               if bytes.Compare(buf[:n], TestBlock) != 0 {
-                       t.Fatalf("Got data %+q, expected %+q", buf[:n], TestBlock)
+               if buf.String() != string(TestBlock) {
+                       t.Errorf("Got data %+q, expected %+q", buf, TestBlock)
                }
 
                _, err = v.Mtime(TestHash)
@@ -909,13 +726,8 @@ func (s *genericVolumeSuite) testTrashEmptyTrashUntrash(t TB, factory TestableVo
                        return err
                }
 
-               err = v.Compare(context.Background(), TestHash, TestBlock)
-               if err != nil {
-                       return err
-               }
-
                indexBuf := new(bytes.Buffer)
-               v.IndexTo("", indexBuf)
+               v.Index(context.Background(), "", indexBuf)
                if !strings.Contains(string(indexBuf.Bytes()), TestHash) {
                        return os.ErrNotExist
                }
@@ -927,50 +739,47 @@ func (s *genericVolumeSuite) testTrashEmptyTrashUntrash(t TB, factory TestableVo
 
        s.cluster.Collections.BlobTrashLifetime.Set("1h")
 
-       v.PutRaw(TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
 
        err := checkGet()
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        // Trash the block
-       err = v.Trash(TestHash)
-       if err == MethodDisabledError || err == ErrNotImplemented {
-               // Skip the trash tests for read-only volumes, and
-               // volume types that don't support
-               // BlobTrashLifetime>0.
-               return
+       err = v.BlockTrash(TestHash)
+       if err != nil {
+               t.Error(err)
        }
 
        err = checkGet()
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
-       err = v.Touch(TestHash)
+       err = v.BlockTouch(TestHash)
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
        v.EmptyTrash()
 
        // Even after emptying the trash, we can untrash our block
        // because the deadline hasn't been reached.
-       err = v.Untrash(TestHash)
+       err = v.BlockUntrash(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        err = checkGet()
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
-       err = v.Touch(TestHash)
+       err = v.BlockTouch(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        // Because we Touch'ed, need to backdate again for next set of tests
@@ -979,16 +788,16 @@ func (s *genericVolumeSuite) testTrashEmptyTrashUntrash(t TB, factory TestableVo
        // If the only block in the trash has already been untrashed,
        // most volumes will fail a subsequent Untrash with a 404, but
        // it's also acceptable for Untrash to succeed.
-       err = v.Untrash(TestHash)
+       err = v.BlockUntrash(TestHash)
        if err != nil && !os.IsNotExist(err) {
-               t.Fatalf("Expected success or os.IsNotExist(), but got: %v", err)
+               t.Errorf("Expected success or os.IsNotExist(), but got: %v", err)
        }
 
        // The additional Untrash should not interfere with our
        // already-untrashed copy.
        err = checkGet()
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        // Untrash might have updated the timestamp, so backdate again
@@ -998,74 +807,74 @@ func (s *genericVolumeSuite) testTrashEmptyTrashUntrash(t TB, factory TestableVo
 
        s.cluster.Collections.BlobTrashLifetime.Set("1ns")
 
-       err = v.Trash(TestHash)
+       err = v.BlockTrash(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
        err = checkGet()
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
        // Even though 1ns has passed, we can untrash because we
        // haven't called EmptyTrash yet.
-       err = v.Untrash(TestHash)
+       err = v.BlockUntrash(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
        err = checkGet()
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        // Trash it again, and this time call EmptyTrash so it really
        // goes away.
        // (In Azure volumes, un/trash changes Mtime, so first backdate again)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
-       _ = v.Trash(TestHash)
+       _ = v.BlockTrash(TestHash)
        err = checkGet()
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
        v.EmptyTrash()
 
        // Untrash won't find it
-       err = v.Untrash(TestHash)
+       err = v.BlockUntrash(TestHash)
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
        // Get block won't find it
        err = checkGet()
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
        // Third set: If the same data block gets written again after
        // being trashed, and then the trash gets emptied, the newer
        // un-trashed copy doesn't get deleted along with it.
 
-       v.PutRaw(TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
 
        s.cluster.Collections.BlobTrashLifetime.Set("1ns")
-       err = v.Trash(TestHash)
+       err = v.BlockTrash(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
        err = checkGet()
        if err == nil || !os.IsNotExist(err) {
-               t.Fatalf("os.IsNotExist(%v) should have been true", err)
+               t.Errorf("os.IsNotExist(%v) should have been true", err)
        }
 
-       v.PutRaw(TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
 
        // EmptyTrash should not delete the untrashed copy.
        v.EmptyTrash()
        err = checkGet()
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        // Fourth set: If the same data block gets trashed twice with
@@ -1073,33 +882,33 @@ func (s *genericVolumeSuite) testTrashEmptyTrashUntrash(t TB, factory TestableVo
        // at intermediate time B (A < B < C), it is still possible to
        // untrash the block whose deadline is "C".
 
-       v.PutRaw(TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
 
        s.cluster.Collections.BlobTrashLifetime.Set("1ns")
-       err = v.Trash(TestHash)
+       err = v.BlockTrash(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
-       v.PutRaw(TestHash, TestBlock)
+       v.BlockWrite(context.Background(), TestHash, TestBlock)
        v.TouchWithDate(TestHash, time.Now().Add(-2*s.cluster.Collections.BlobSigningTTL.Duration()))
 
        s.cluster.Collections.BlobTrashLifetime.Set("1h")
-       err = v.Trash(TestHash)
+       err = v.BlockTrash(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 
        // EmptyTrash should not prevent us from recovering the
        // time.Hour ("C") trash
        v.EmptyTrash()
-       err = v.Untrash(TestHash)
+       err = v.BlockUntrash(TestHash)
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
        err = checkGet()
        if err != nil {
-               t.Fatal(err)
+               t.Error(err)
        }
 }
index 950b3989aa0f6a72e20553f8505f6575a91b39c4..5a17b3a7dc103355273ceb75f13fc3e320addcd8 100644 (file)
@@ -5,25 +5,12 @@
 package keepstore
 
 import (
-       "bytes"
-       "context"
-       "crypto/md5"
-       "errors"
-       "fmt"
-       "io"
-       "os"
-       "strings"
-       "sync"
        "time"
-
-       "git.arvados.org/arvados.git/sdk/go/arvados"
-       "github.com/sirupsen/logrus"
 )
 
 var (
-       TestBlock       = []byte("The quick brown fox jumps over the lazy dog.")
-       TestHash        = "e4d909c290d0fb1ca068ffaddf22cbd0"
-       TestHashPutResp = "e4d909c290d0fb1ca068ffaddf22cbd0+44\n"
+       TestBlock = []byte("The quick brown fox jumps over the lazy dog.")
+       TestHash  = "e4d909c290d0fb1ca068ffaddf22cbd0"
 
        TestBlock2 = []byte("Pack my box with five dozen liquor jugs.")
        TestHash2  = "f15ac516f788aec4f30932ffb6395c39"
@@ -31,10 +18,6 @@ var (
        TestBlock3 = []byte("Now is the time for all good men to come to the aid of their country.")
        TestHash3  = "eed29bbffbc2dbe5e5ee0bb71888e61f"
 
-       // BadBlock is used to test collisions and corruption.
-       // It must not match any test hashes.
-       BadBlock = []byte("The magic words are squeamish ossifrage.")
-
        EmptyHash  = "d41d8cd98f00b204e9800998ecf8427e"
        EmptyBlock = []byte("")
 )
@@ -43,230 +26,15 @@ var (
 // underlying Volume, in order to test behavior in cases that are
 // impractical to achieve with a sequence of normal Volume operations.
 type TestableVolume interface {
-       Volume
-
-       // [Over]write content for a locator with the given data,
-       // bypassing all constraints like readonly and serialize.
-       PutRaw(locator string, data []byte)
+       volume
 
        // Returns the strings that a driver uses to record read/write operations.
        ReadWriteOperationLabelValues() (r, w string)
 
        // Specify the value Mtime() should return, until the next
-       // call to Touch, TouchWithDate, or Put.
-       TouchWithDate(locator string, lastPut time.Time)
+       // call to Touch, TouchWithDate, or BlockWrite.
+       TouchWithDate(locator string, lastBlockWrite time.Time)
 
        // Clean up, delete temporary files.
        Teardown()
 }
-
-func init() {
-       driver["mock"] = newMockVolume
-}
-
-// MockVolumes are test doubles for Volumes, used to test handlers.
-type MockVolume struct {
-       Store      map[string][]byte
-       Timestamps map[string]time.Time
-
-       // Bad volumes return an error for every operation.
-       Bad            bool
-       BadVolumeError error
-
-       // Touchable volumes' Touch() method succeeds for a locator
-       // that has been Put().
-       Touchable bool
-
-       // Gate is a "starting gate", allowing test cases to pause
-       // volume operations long enough to inspect state. Every
-       // operation (except Status) starts by receiving from
-       // Gate. Sending one value unblocks one operation; closing the
-       // channel unblocks all operations. By default, Gate is a
-       // closed channel, so all operations proceed without
-       // blocking. See trash_worker_test.go for an example.
-       Gate chan struct{} `json:"-"`
-
-       cluster *arvados.Cluster
-       volume  arvados.Volume
-       logger  logrus.FieldLogger
-       metrics *volumeMetricsVecs
-       called  map[string]int
-       mutex   sync.Mutex
-}
-
-// newMockVolume returns a non-Bad, non-Readonly, Touchable mock
-// volume.
-func newMockVolume(cluster *arvados.Cluster, volume arvados.Volume, logger logrus.FieldLogger, metrics *volumeMetricsVecs) (Volume, error) {
-       gate := make(chan struct{})
-       close(gate)
-       return &MockVolume{
-               Store:      make(map[string][]byte),
-               Timestamps: make(map[string]time.Time),
-               Bad:        false,
-               Touchable:  true,
-               called:     map[string]int{},
-               Gate:       gate,
-               cluster:    cluster,
-               volume:     volume,
-               logger:     logger,
-               metrics:    metrics,
-       }, nil
-}
-
-// CallCount returns how many times the named method has been called.
-func (v *MockVolume) CallCount(method string) int {
-       v.mutex.Lock()
-       defer v.mutex.Unlock()
-       c, ok := v.called[method]
-       if !ok {
-               return 0
-       }
-       return c
-}
-
-func (v *MockVolume) gotCall(method string) {
-       v.mutex.Lock()
-       defer v.mutex.Unlock()
-       if _, ok := v.called[method]; !ok {
-               v.called[method] = 1
-       } else {
-               v.called[method]++
-       }
-}
-
-func (v *MockVolume) Compare(ctx context.Context, loc string, buf []byte) error {
-       v.gotCall("Compare")
-       <-v.Gate
-       if v.Bad {
-               return v.BadVolumeError
-       } else if block, ok := v.Store[loc]; ok {
-               if fmt.Sprintf("%x", md5.Sum(block)) != loc {
-                       return DiskHashError
-               }
-               if bytes.Compare(buf, block) != 0 {
-                       return CollisionError
-               }
-               return nil
-       } else {
-               return os.ErrNotExist
-       }
-}
-
-func (v *MockVolume) Get(ctx context.Context, loc string, buf []byte) (int, error) {
-       v.gotCall("Get")
-       <-v.Gate
-       if v.Bad {
-               return 0, v.BadVolumeError
-       } else if block, ok := v.Store[loc]; ok {
-               copy(buf[:len(block)], block)
-               return len(block), nil
-       }
-       return 0, os.ErrNotExist
-}
-
-func (v *MockVolume) Put(ctx context.Context, loc string, block []byte) error {
-       v.gotCall("Put")
-       <-v.Gate
-       if v.Bad {
-               return v.BadVolumeError
-       }
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-       v.Store[loc] = block
-       return v.Touch(loc)
-}
-
-func (v *MockVolume) Touch(loc string) error {
-       return v.TouchWithDate(loc, time.Now())
-}
-
-func (v *MockVolume) TouchWithDate(loc string, t time.Time) error {
-       v.gotCall("Touch")
-       <-v.Gate
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-       if _, exists := v.Store[loc]; !exists {
-               return os.ErrNotExist
-       }
-       if v.Touchable {
-               v.Timestamps[loc] = t
-               return nil
-       }
-       return errors.New("Touch failed")
-}
-
-func (v *MockVolume) Mtime(loc string) (time.Time, error) {
-       v.gotCall("Mtime")
-       <-v.Gate
-       var mtime time.Time
-       var err error
-       if v.Bad {
-               err = v.BadVolumeError
-       } else if t, ok := v.Timestamps[loc]; ok {
-               mtime = t
-       } else {
-               err = os.ErrNotExist
-       }
-       return mtime, err
-}
-
-func (v *MockVolume) IndexTo(prefix string, w io.Writer) error {
-       v.gotCall("IndexTo")
-       <-v.Gate
-       for loc, block := range v.Store {
-               if !IsValidLocator(loc) || !strings.HasPrefix(loc, prefix) {
-                       continue
-               }
-               _, err := fmt.Fprintf(w, "%s+%d %d\n",
-                       loc, len(block), 123456789)
-               if err != nil {
-                       return err
-               }
-       }
-       return nil
-}
-
-func (v *MockVolume) Trash(loc string) error {
-       v.gotCall("Delete")
-       <-v.Gate
-       if v.volume.ReadOnly {
-               return MethodDisabledError
-       }
-       if _, ok := v.Store[loc]; ok {
-               if time.Since(v.Timestamps[loc]) < time.Duration(v.cluster.Collections.BlobSigningTTL) {
-                       return nil
-               }
-               delete(v.Store, loc)
-               return nil
-       }
-       return os.ErrNotExist
-}
-
-func (v *MockVolume) GetDeviceID() string {
-       return "mock-device-id"
-}
-
-func (v *MockVolume) Untrash(loc string) error {
-       return nil
-}
-
-func (v *MockVolume) Status() *VolumeStatus {
-       var used uint64
-       for _, block := range v.Store {
-               used = used + uint64(len(block))
-       }
-       return &VolumeStatus{"/bogo", 123, 1000000 - used, used}
-}
-
-func (v *MockVolume) String() string {
-       return "[MockVolume]"
-}
-
-func (v *MockVolume) EmptyTrash() {
-}
-
-func (v *MockVolume) GetStorageClasses() []string {
-       return nil
-}
diff --git a/services/keepstore/work_queue.go b/services/keepstore/work_queue.go
deleted file mode 100644 (file)
index be3d118..0000000
+++ /dev/null
@@ -1,208 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-/* A WorkQueue is an asynchronous thread-safe queue manager.  It
-   provides a channel from which items can be read off the queue, and
-   permits replacing the contents of the queue at any time.
-
-   The overall work flow for a WorkQueue is as follows:
-
-     1. A WorkQueue is created with NewWorkQueue().  This
-        function instantiates a new WorkQueue and starts a manager
-        goroutine.  The manager listens on an input channel
-        (manager.newlist) and an output channel (manager.NextItem).
-
-     2. The manager first waits for a new list of requests on the
-        newlist channel.  When another goroutine calls
-        manager.ReplaceQueue(lst), it sends lst over the newlist
-        channel to the manager.  The manager goroutine now has
-        ownership of the list.
-
-     3. Once the manager has this initial list, it listens on both the
-        input and output channels for one of the following to happen:
-
-          a. A worker attempts to read an item from the NextItem
-             channel.  The manager sends the next item from the list
-             over this channel to the worker, and loops.
-
-          b. New data is sent to the manager on the newlist channel.
-             This happens when another goroutine calls
-             manager.ReplaceItem() with a new list.  The manager
-             discards the current list, replaces it with the new one,
-             and begins looping again.
-
-          c. The input channel is closed.  The manager closes its
-             output channel (signalling any workers to quit) and
-             terminates.
-
-   Tasks currently handled by WorkQueue:
-     * the pull list
-     * the trash list
-
-   Example usage:
-
-        // Any kind of user-defined type can be used with the
-        // WorkQueue.
-               type FrobRequest struct {
-                       frob string
-               }
-
-               // Make a work list.
-               froblist := NewWorkQueue()
-
-               // Start a concurrent worker to read items from the NextItem
-               // channel until it is closed, deleting each one.
-               go func(list WorkQueue) {
-                       for i := range list.NextItem {
-                               req := i.(FrobRequest)
-                               frob.Run(req)
-                       }
-               }(froblist)
-
-               // Set up a HTTP handler for PUT /frob
-               router.HandleFunc(`/frob`,
-                       func(w http.ResponseWriter, req *http.Request) {
-                               // Parse the request body into a list.List
-                               // of FrobRequests, and give this list to the
-                               // frob manager.
-                               newfrobs := parseBody(req.Body)
-                               froblist.ReplaceQueue(newfrobs)
-                       }).Methods("PUT")
-
-   Methods available on a WorkQueue:
-
-               ReplaceQueue(list)
-                       Replaces the current item list with a new one.  The list
-            manager discards any unprocessed items on the existing
-            list and replaces it with the new one. If the worker is
-            processing a list item when ReplaceQueue is called, it
-            finishes processing before receiving items from the new
-            list.
-               Close()
-                       Shuts down the manager goroutine. When Close is called,
-                       the manager closes the NextItem channel.
-*/
-
-import "container/list"
-
-// WorkQueue definition
-type WorkQueue struct {
-       getStatus chan WorkQueueStatus
-       newlist   chan *list.List
-       // Workers get work items by reading from this channel.
-       NextItem <-chan interface{}
-       // Each worker must send struct{}{} to DoneItem exactly once
-       // for each work item received from NextItem, when it stops
-       // working on that item (regardless of whether the work was
-       // successful).
-       DoneItem chan<- struct{}
-}
-
-// WorkQueueStatus reflects the queue status.
-type WorkQueueStatus struct {
-       InProgress int
-       Queued     int
-}
-
-// NewWorkQueue returns a new empty WorkQueue.
-func NewWorkQueue() *WorkQueue {
-       nextItem := make(chan interface{})
-       reportDone := make(chan struct{})
-       newList := make(chan *list.List)
-       b := WorkQueue{
-               getStatus: make(chan WorkQueueStatus),
-               newlist:   newList,
-               NextItem:  nextItem,
-               DoneItem:  reportDone,
-       }
-       go func() {
-               // Read new work lists from the newlist channel.
-               // Reply to "status" and "get next item" queries by
-               // sending to the getStatus and nextItem channels
-               // respectively. Return when the newlist channel
-               // closes.
-
-               todo := &list.List{}
-               status := WorkQueueStatus{}
-
-               // When we're done, close the output channel; workers will
-               // shut down next time they ask for new work.
-               defer close(nextItem)
-               defer close(b.getStatus)
-
-               // nextChan and nextVal are both nil when we have
-               // nothing to send; otherwise they are, respectively,
-               // the nextItem channel and the next work item to send
-               // to it.
-               var nextChan chan interface{}
-               var nextVal interface{}
-
-               for newList != nil || status.InProgress > 0 {
-                       select {
-                       case p, ok := <-newList:
-                               if !ok {
-                                       // Closed, stop receiving
-                                       newList = nil
-                               }
-                               todo = p
-                               if todo == nil {
-                                       todo = &list.List{}
-                               }
-                               status.Queued = todo.Len()
-                               if status.Queued == 0 {
-                                       // Stop sending work
-                                       nextChan = nil
-                                       nextVal = nil
-                               } else {
-                                       nextChan = nextItem
-                                       nextVal = todo.Front().Value
-                               }
-                       case nextChan <- nextVal:
-                               todo.Remove(todo.Front())
-                               status.InProgress++
-                               status.Queued--
-                               if status.Queued == 0 {
-                                       // Stop sending work
-                                       nextChan = nil
-                                       nextVal = nil
-                               } else {
-                                       nextVal = todo.Front().Value
-                               }
-                       case <-reportDone:
-                               status.InProgress--
-                       case b.getStatus <- status:
-                       }
-               }
-       }()
-       return &b
-}
-
-// ReplaceQueue abandons any work items left in the existing queue,
-// and starts giving workers items from the given list. After giving
-// it to ReplaceQueue, the caller must not read or write the given
-// list.
-func (b *WorkQueue) ReplaceQueue(list *list.List) {
-       b.newlist <- list
-}
-
-// Close shuts down the manager and terminates the goroutine, which
-// abandons any pending requests, but allows any pull request already
-// in progress to continue.
-//
-// After Close, Status will return correct values, NextItem will be
-// closed, and ReplaceQueue will panic.
-func (b *WorkQueue) Close() {
-       close(b.newlist)
-}
-
-// Status returns an up-to-date WorkQueueStatus reflecting the current
-// queue status.
-func (b *WorkQueue) Status() WorkQueueStatus {
-       // If the channel is closed, we get the nil value of
-       // WorkQueueStatus, which is an accurate description of a
-       // finished queue.
-       return <-b.getStatus
-}
diff --git a/services/keepstore/work_queue_test.go b/services/keepstore/work_queue_test.go
deleted file mode 100644 (file)
index 254f96c..0000000
+++ /dev/null
@@ -1,244 +0,0 @@
-// Copyright (C) The Arvados Authors. All rights reserved.
-//
-// SPDX-License-Identifier: AGPL-3.0
-
-package keepstore
-
-import (
-       "container/list"
-       "runtime"
-       "testing"
-       "time"
-)
-
-type fatalfer interface {
-       Fatalf(string, ...interface{})
-}
-
-func makeTestWorkList(ary []interface{}) *list.List {
-       l := list.New()
-       for _, n := range ary {
-               l.PushBack(n)
-       }
-       return l
-}
-
-func expectChannelEmpty(t fatalfer, c <-chan interface{}) {
-       select {
-       case item, ok := <-c:
-               if ok {
-                       t.Fatalf("Received value (%+v) from channel that we expected to be empty", item)
-               }
-       default:
-       }
-}
-
-func expectChannelNotEmpty(t fatalfer, c <-chan interface{}) interface{} {
-       select {
-       case item, ok := <-c:
-               if !ok {
-                       t.Fatalf("expected data on a closed channel")
-               }
-               return item
-       case <-time.After(time.Second):
-               t.Fatalf("expected data on an empty channel")
-               return nil
-       }
-}
-
-func expectChannelClosedWithin(t fatalfer, timeout time.Duration, c <-chan interface{}) {
-       select {
-       case received, ok := <-c:
-               if ok {
-                       t.Fatalf("Expected channel to be closed, but received %+v instead", received)
-               }
-       case <-time.After(timeout):
-               t.Fatalf("Expected channel to be closed, but it is still open after %v", timeout)
-       }
-}
-
-func doWorkItems(t fatalfer, q *WorkQueue, expected []interface{}) {
-       for i := range expected {
-               actual, ok := <-q.NextItem
-               if !ok {
-                       t.Fatalf("Expected %+v but channel was closed after receiving %+v as expected.", expected, expected[:i])
-               }
-               q.DoneItem <- struct{}{}
-               if actual.(int) != expected[i] {
-                       t.Fatalf("Expected %+v but received %+v after receiving %+v as expected.", expected[i], actual, expected[:i])
-               }
-       }
-}
-
-func expectEqualWithin(t fatalfer, timeout time.Duration, expect interface{}, f func() interface{}) {
-       ok := make(chan struct{})
-       giveup := false
-       go func() {
-               for f() != expect && !giveup {
-                       time.Sleep(time.Millisecond)
-               }
-               close(ok)
-       }()
-       select {
-       case <-ok:
-       case <-time.After(timeout):
-               giveup = true
-               _, file, line, _ := runtime.Caller(1)
-               t.Fatalf("Still getting %+v, timed out waiting for %+v\n%s:%d", f(), expect, file, line)
-       }
-}
-
-func expectQueued(t fatalfer, b *WorkQueue, expectQueued int) {
-       if l := b.Status().Queued; l != expectQueued {
-               t.Fatalf("Got Queued==%d, expected %d", l, expectQueued)
-       }
-}
-
-func TestWorkQueueDoneness(t *testing.T) {
-       b := NewWorkQueue()
-       defer b.Close()
-       b.ReplaceQueue(makeTestWorkList([]interface{}{1, 2, 3}))
-       expectQueued(t, b, 3)
-       gate := make(chan struct{})
-       go func() {
-               <-gate
-               for range b.NextItem {
-                       <-gate
-                       time.Sleep(time.Millisecond)
-                       b.DoneItem <- struct{}{}
-               }
-       }()
-       expectEqualWithin(t, time.Second, 0, func() interface{} { return b.Status().InProgress })
-       b.ReplaceQueue(makeTestWorkList([]interface{}{4, 5, 6}))
-       for i := 1; i <= 3; i++ {
-               gate <- struct{}{}
-               expectEqualWithin(t, time.Second, 3-i, func() interface{} { return b.Status().Queued })
-               expectEqualWithin(t, time.Second, 1, func() interface{} { return b.Status().InProgress })
-       }
-       close(gate)
-       expectEqualWithin(t, time.Second, 0, func() interface{} { return b.Status().InProgress })
-       expectChannelEmpty(t, b.NextItem)
-}
-
-// Create a WorkQueue, generate a list for it, and instantiate a worker.
-func TestWorkQueueReadWrite(t *testing.T) {
-       var input = []interface{}{1, 1, 2, 3, 5, 8, 13, 21, 34}
-
-       b := NewWorkQueue()
-       expectQueued(t, b, 0)
-
-       b.ReplaceQueue(makeTestWorkList(input))
-       expectQueued(t, b, len(input))
-
-       doWorkItems(t, b, input)
-       expectChannelEmpty(t, b.NextItem)
-       b.Close()
-}
-
-// Start a worker before the list has any input.
-func TestWorkQueueEarlyRead(t *testing.T) {
-       var input = []interface{}{1, 1, 2, 3, 5, 8, 13, 21, 34}
-
-       b := NewWorkQueue()
-       defer b.Close()
-
-       // First, demonstrate that nothing is available on the NextItem
-       // channel.
-       expectChannelEmpty(t, b.NextItem)
-
-       // Start a reader in a goroutine. The reader will block until the
-       // block work list has been initialized.
-       //
-       done := make(chan int)
-       go func() {
-               doWorkItems(t, b, input)
-               done <- 1
-       }()
-
-       // Feed the blocklist a new worklist, and wait for the worker to
-       // finish.
-       b.ReplaceQueue(makeTestWorkList(input))
-       <-done
-       expectQueued(t, b, 0)
-}
-
-// After Close(), NextItem closes, work finishes, then stats return zero.
-func TestWorkQueueClose(t *testing.T) {
-       b := NewWorkQueue()
-       input := []interface{}{1, 2, 3, 4, 5, 6, 7, 8}
-       mark := make(chan struct{})
-       go func() {
-               <-b.NextItem
-               mark <- struct{}{}
-               <-mark
-               b.DoneItem <- struct{}{}
-       }()
-       b.ReplaceQueue(makeTestWorkList(input))
-       // Wait for worker to take item 1
-       <-mark
-       b.Close()
-       expectEqualWithin(t, time.Second, 1, func() interface{} { return b.Status().InProgress })
-       // Tell worker to report done
-       mark <- struct{}{}
-       expectEqualWithin(t, time.Second, 0, func() interface{} { return b.Status().InProgress })
-       expectChannelClosedWithin(t, time.Second, b.NextItem)
-}
-
-// Show that a reader may block when the manager's list is exhausted,
-// and that the reader resumes automatically when new data is
-// available.
-func TestWorkQueueReaderBlocks(t *testing.T) {
-       var (
-               inputBeforeBlock = []interface{}{1, 2, 3, 4, 5}
-               inputAfterBlock  = []interface{}{6, 7, 8, 9, 10}
-       )
-
-       b := NewWorkQueue()
-       defer b.Close()
-       sendmore := make(chan int)
-       done := make(chan int)
-       go func() {
-               doWorkItems(t, b, inputBeforeBlock)
-
-               // Confirm that the channel is empty, so a subsequent read
-               // on it will block.
-               expectChannelEmpty(t, b.NextItem)
-
-               // Signal that we're ready for more input.
-               sendmore <- 1
-               doWorkItems(t, b, inputAfterBlock)
-               done <- 1
-       }()
-
-       // Write a slice of the first five elements and wait for the
-       // reader to signal that it's ready for us to send more input.
-       b.ReplaceQueue(makeTestWorkList(inputBeforeBlock))
-       <-sendmore
-
-       b.ReplaceQueue(makeTestWorkList(inputAfterBlock))
-
-       // Wait for the reader to complete.
-       <-done
-}
-
-// Replace one active work list with another.
-func TestWorkQueueReplaceQueue(t *testing.T) {
-       var firstInput = []interface{}{1, 1, 2, 3, 5, 8, 13, 21, 34}
-       var replaceInput = []interface{}{1, 4, 9, 16, 25, 36, 49, 64, 81}
-
-       b := NewWorkQueue()
-       b.ReplaceQueue(makeTestWorkList(firstInput))
-
-       // Read just the first five elements from the work list.
-       // Confirm that the channel is not empty.
-       doWorkItems(t, b, firstInput[0:5])
-       expectChannelNotEmpty(t, b.NextItem)
-
-       // Replace the work list and read five more elements.
-       // The old list should have been discarded and all new
-       // elements come from the new list.
-       b.ReplaceQueue(makeTestWorkList(replaceInput))
-       doWorkItems(t, b, replaceInput[0:5])
-
-       b.Close()
-}
index 4dcb47a8da02e3eea9edddf5e612dff660076147..5bd7136eaa8d060d4d78a83a492917258b887e4e 100644 (file)
@@ -48,6 +48,7 @@ func (s *ServerRequiredSuite) TearDownSuite(c *C) {
 }
 
 func (s *ServerRequiredSuite) SetUpTest(c *C) {
+       logBuffer.Reset()
        logOutput := io.MultiWriter(&logBuffer)
        log.SetOutput(logOutput)
 }
@@ -55,7 +56,7 @@ func (s *ServerRequiredSuite) SetUpTest(c *C) {
 func (s *ServerRequiredSuite) TearDownTest(c *C) {
        arvadostest.StopKeep(2)
        log.SetOutput(os.Stdout)
-       log.Printf("%v", logBuffer.String())
+       c.Log(logBuffer.String())
 }
 
 func (s *DoMainTestSuite) SetUpSuite(c *C) {
@@ -226,7 +227,9 @@ func (s *ServerRequiredSuite) TestBlockCheck_BadSignature(c *C) {
        setupTestData(c)
        err := performKeepBlockCheck(kc, blobSignatureTTL, "badblobsigningkey", []string{TestHash, TestHash2}, false)
        c.Assert(err.Error(), Equals, "Block verification failed for 2 out of 2 blocks with matching prefix")
-       checkErrorLog(c, []string{TestHash, TestHash2}, "Error verifying block", "HTTP 403")
+       // older versions of keepstore return 403 Forbidden for
+       // invalid signatures, newer versions return 400 Bad Request.
+       checkErrorLog(c, []string{TestHash, TestHash2}, "Error verifying block", "HTTP 40[03]")
        // verbose logging not requested
        c.Assert(strings.Contains(logBuffer.String(), "Verifying block 1 of 2"), Equals, false)
 }
index dc5b957125c731d13ae51969b7a47ec765ffcf33..1d2d6b5c1917115e39775bb3066273aa9de62106 100644 (file)
@@ -161,7 +161,7 @@ func testNoCrosstalk(c *C, testData string, kc1, kc2 *keepclient.KeepClient) {
        locator, _, err := kc1.PutB([]byte(testData))
        c.Assert(err, Equals, nil)
 
-       locator = strings.Split(locator, "+")[0]
+       locator = strings.Join(strings.Split(locator, "+")[:2], "+")
        _, _, _, err = kc2.Get(keepclient.SignLocator(locator, kc2.Arvados.ApiToken, time.Now().AddDate(0, 0, 1), blobSignatureTTL, []byte(blobSigningKey)))
        c.Assert(err, NotNil)
        c.Check(err.Error(), Equals, "Block not found")
@@ -330,7 +330,7 @@ func (s *ServerRequiredSuite) TestErrorDuringRsync_ErrorGettingBlockFromSrc(c *C
 
        err := performKeepRsync(kcSrc, kcDst, blobSignatureTTL, blobSigningKey, "")
        c.Assert(err, NotNil)
-       c.Check(err.Error(), Matches, ".*HTTP 403 \"Forbidden\".*")
+       c.Check(err.Error(), Matches, ".*HTTP 400 \"invalid signature\".*")
 }
 
 // Test rsync with error during Put to src.