[arvados] created: 2.7.0-5885-gbf2113bc6c

git repository hosting git at public.arvados.org
Fri Jan 26 23:00:52 UTC 2024


        at  bf2113bc6cd3302e326a0903a55b0222b7215a10 (commit)


commit bf2113bc6cd3302e326a0903a55b0222b7215a10
Author: Tom Clegg <tom at curii.com>
Date:   Fri Jan 26 18:00:37 2024 -0500

    2960: Refactor keepstore into a streaming server. (incomplete)
    
    Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tom at curii.com>

diff --git a/sdk/go/arvados/api.go b/sdk/go/arvados/api.go
index a6b240e214..e7310818f7 100644
--- a/sdk/go/arvados/api.go
+++ b/sdk/go/arvados/api.go
@@ -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 {
diff --git a/services/keepstore/azure_blob_volume.go b/services/keepstore/azure_blob_volume.go
index 56a52c913a..d8bb0f06ba 100644
--- a/services/keepstore/azure_blob_volume.go
+++ b/services/keepstore/azure_blob_volume.go
@@ -32,17 +32,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 +82,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 +109,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 +126,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 +143,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 +181,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 +190,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 +213,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 +260,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 +290,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 +303,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 +318,26 @@ 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 {
+// 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 {
 	if v.volume.ReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
 	// 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))
+		io.Copy(bufw, bytes.NewReader(data))
 		bufw.Close()
 	}()
 	errChan := make(chan error)
 	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,7 +346,7 @@ 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():
@@ -390,12 +366,12 @@ 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 {
+// BlockTouch updates the last-modified property of a block blob.
+func (v *AzureBlobVolume) BlockTouch(hash string) error {
 	if v.volume.ReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
-	trashed, metadata, err := v.checkTrashed(loc)
+	trashed, metadata, err := v.checkTrashed(hash)
 	if err != nil {
 		return err
 	}
@@ -404,12 +380,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 +393,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 +447,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,9 +459,9 @@ func (v *AzureBlobVolume) listBlobs(page int, params storage.ListBlobsParameters
 }
 
 // Trash a Keep block.
-func (v *AzureBlobVolume) Trash(loc string) error {
+func (v *AzureBlobVolume) BlockTrash(loc string) error {
 	if v.volume.ReadOnly && !v.volume.AllowTrashWhenReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
 	// Ideally we would use If-Unmodified-Since, but that
 	// particular condition seems to be ignored by Azure. Instead,
@@ -513,11 +493,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,7 +507,7 @@ 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)
 }
 
@@ -553,7 +533,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
diff --git a/services/keepstore/bufferpool.go b/services/keepstore/bufferpool.go
index b4cc5d38e1..811715b191 100644
--- a/services/keepstore/bufferpool.go
+++ b/services/keepstore/bufferpool.go
@@ -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
 }
 
diff --git a/services/keepstore/bufferpool_test.go b/services/keepstore/bufferpool_test.go
index 13e1cb4f33..c348c7765c 100644
--- a/services/keepstore/bufferpool_test.go
+++ b/services/keepstore/bufferpool_test.go
@@ -5,7 +5,6 @@
 package keepstore
 
 import (
-	"context"
 	"time"
 
 	"git.arvados.org/arvados.git/sdk/go/ctxlog"
@@ -14,46 +13,45 @@ import (
 
 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) TearDownTest(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)
+	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)
+	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)
+	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 +66,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)
+	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 +79,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
index 16f2d09232..0000000000
--- a/services/keepstore/collision.go
+++ /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
index aa8f0cbaa1..0000000000
--- a/services/keepstore/collision_test.go
+++ /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)
-}
diff --git a/services/keepstore/command.go b/services/keepstore/command.go
index 48c8256a3c..c0b2f56692 100644
--- a/services/keepstore/command.go
+++ b/services/keepstore/command.go
@@ -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(ks, reg)
+	trasher := newTrasher(ks, reg)
+	_ = newTrashEmptier(ks, reg)
+	return newRouter(ks, puller, trasher)
 }
diff --git a/services/keepstore/handlers.go b/services/keepstore/handlers.go
deleted file mode 100644
index abeb20fe86..0000000000
--- a/services/keepstore/handlers.go
+++ /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/keepstore.go b/services/keepstore/keepstore.go
index 953aa047cb..0eab7e0adb 100644
--- a/services/keepstore/keepstore.go
+++ b/services/keepstore/keepstore.go
@@ -5,53 +5,534 @@
 package keepstore
 
 import (
+	"bytes"
+	"context"
+	"crypto/md5"
+	"errors"
+	"fmt"
+	"io"
+	"net/http"
+	"os"
+	"sort"
+	"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.StatusUnauthorized)
+	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
 }
 
-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{
+			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
+		}
+		mnt := &mount{
+			volume: vol,
+			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, 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)
+		}
+	}
+	return nil
+}
+
+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
+}
+
+func (ks *keepstore) signLocator(token, locator string) string {
+	if token == "" {
+		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) {
+	if strings.Contains(opts.Locator, "+R") && !strings.Contains(opts.Locator, "+A") {
+		return ks.blockReadRemote(ctx, opts)
+	}
+	if err := ks.checkLocatorSignature(ctx, opts.Locator); err != nil {
+		return 0, err
+	}
+	var errToCaller error = os.ErrNotExist
+	for _, mnt := range ks.mountsR {
+		if ctx.Err() != nil {
+			return 0, ctx.Err()
+		}
+		n, err = mnt.BlockRead(ctx, strings.SplitN(opts.Locator, "+", 2)[0], opts.WriteTo)
+		if n > 0 || err == nil {
+			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) {
+	token := ctxToken(ctx)
+	if token == "" {
+		return 0, errNoTokenProvided
+	}
+	var remoteClient *keepclient.KeepClient
+	var parts []string
+	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:]
+		}
+		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
+		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])
+	_, 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)
+	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)
+		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)
+	}
+	result := newPutProgress(opts.StorageClasses)
+	for _, mnt := range ks.mountsW {
+		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 := result.Copy()
+	cond := sync.NewCond(new(sync.Mutex))
+	cond.L.Lock()
+	var wg sync.WaitGroup
+	for _, mnt := range ks.rendezvous(hash, ks.mountsW) {
+		for result.Want(mnt) && !pending.Want(mnt) && ctx.Err() == nil {
+			cond.Wait()
+		}
+		if !result.Want(mnt) || ctx.Err() != nil {
+			continue
+		}
+		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()
+			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
+	}
+	return resp, errVolumeUnavailable
+}
+
+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 { 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, hash string) error {
+	var errToCaller error = os.ErrNotExist
+	for _, mnt := range ks.mountsW {
+		if ctx.Err() != nil {
+			return ctx.Err()
+		}
+		err := mnt.BlockUntrash(hash)
+		if err == nil {
+			errToCaller = nil
+		} else if !os.IsNotExist(err) && errToCaller != nil {
+			errToCaller = err
+		}
+	}
+	return errToCaller
+}
+
+func (ks *keepstore) BlockTouch(ctx context.Context, hash string) error {
+	var errToCaller error = os.ErrNotExist
+	for _, mnt := range ks.mountsW {
+		if ctx.Err() != nil {
+			return ctx.Err()
+		}
+		err := mnt.BlockTouch(hash)
+		if err == nil {
+			errToCaller = nil
+		} else if !os.IsNotExist(err) && errToCaller != nil {
+			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 ""
 	}
 }
diff --git a/services/keepstore/metrics.go b/services/keepstore/metrics.go
index d04601fbec..4638de5444 100644
--- a/services/keepstore/metrics.go
+++ b/services/keepstore/metrics.go
@@ -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/perms.go b/services/keepstore/perms.go
deleted file mode 100644
index 7205a4594d..0000000000
--- a/services/keepstore/perms.go
+++ /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
index 1322374706..0000000000
--- a/services/keepstore/perms_test.go
+++ /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
index 6b555054b6..0000000000
--- a/services/keepstore/pipe_adapters.go
+++ /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
index 325f1cf485..0000000000
--- a/services/keepstore/proxy_remote.go
+++ /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
-}
diff --git a/services/keepstore/pull_worker.go b/services/keepstore/pull_worker.go
index 348bfb4df0..0207943bfa 100644
--- a/services/keepstore/pull_worker.go
+++ b/services/keepstore/pull_worker.go
@@ -5,90 +5,127 @@
 package keepstore
 
 import (
+	"bytes"
 	"context"
-	"fmt"
-	"io"
-	"io/ioutil"
-	"time"
+	"sync"
 
+	"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
+}
 
-	// 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(keepstore *keepstore, reg *prometheus.Registry) *puller {
+	p := &puller{
+		keepstore: keepstore,
+		cond:      sync.NewCond(&sync.Mutex{}),
+	}
+	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))
+		},
+	))
+	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()
 	}
-	keepClient.SetServiceRoots(serviceRoots, nil, nil)
+	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() {
+	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)
+	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,
 	}
+	for {
+		p.cond.L.Lock()
+		for len(p.todo) == 0 {
+			p.cond.Wait()
+		}
+		item := p.todo[0]
+		p.todo = p.todo[1:]
+		p.cond.L.Unlock()
 
-	return writePulledBlock(h.volmgr, vol, readContent, pullRequest.Locator)
-}
+		var dst *mount
+		if item.MountUUID != "" {
+			dst = p.keepstore.mounts[item.MountUUID]
+			if dst == nil {
+				p.keepstore.logger.Warnf("ignoring pull list entry for nonexistent mount: %v", item)
+				continue
+			} else if !dst.AllowWrite {
+				p.keepstore.logger.Warnf("ignoring pull list entry for readonly mount: %v", item)
+				continue
+			}
+		} else {
+			dst = p.keepstore.rendezvous(item.Locator, p.keepstore.mountsW)[0]
+		}
 
-// 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)
-}
+		serviceRoots := make(map[string]string)
+		for _, addr := range item.Servers {
+			serviceRoots[addr] = addr
+		}
+		keepClient.SetServiceRoots(serviceRoots, nil, nil)
 
-var writePulledBlock = func(volmgr *RRVolumeManager, volume Volume, data []byte, locator string) error {
-	if volume != nil {
-		return volume.Put(context.Background(), locator, data)
+		signedLocator := p.keepstore.signLocator(c.AuthToken, item.Locator)
+
+		buf := bytes.NewBuffer(nil)
+		_, err := keepClient.BlockRead(context.Background(), arvados.BlockReadOptions{
+			Locator: signedLocator,
+			WriteTo: buf,
+		})
+		if err != nil {
+			p.keepstore.logger.Warnf("error pulling data for pull list entry (%v): %s", item, err)
+			continue
+		}
+		err = dst.BlockWrite(context.Background(), item.Locator, buf.Bytes())
+		if err != nil {
+			p.keepstore.logger.Warnf("error writing data for pull list entry (%v): %s", item, err)
+			continue
+		}
 	}
-	_, err := PutBlock(context.Background(), volmgr, data, locator, nil)
-	return err
 }
diff --git a/services/keepstore/putprogress.go b/services/keepstore/putprogress.go
new file mode 100644
index 0000000000..e02b2d09e9
--- /dev/null
+++ b/services/keepstore/putprogress.go
@@ -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
index 0000000000..2ad39716b9
--- /dev/null
+++ b/services/keepstore/router.go
@@ -0,0 +1,223 @@
+// 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"
+	"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}
+	adminonly := func(h http.HandlerFunc) http.HandlerFunc {
+		return auth.RequireLiteralToken(keepstore.cluster.SystemRootToken, h).ServeHTTP
+	}
+	mgtonly := func(h http.HandlerFunc) http.HandlerFunc {
+		return auth.RequireLiteralToken(keepstore.cluster.ManagementToken, h).ServeHTTP
+	}
+
+	r := mux.NewRouter()
+	locatorPath := `/{locator:[0-9a-f]{32}(\+[0-9]+(\+.+)?)?}`
+	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(`/debug.json`, mgtonly(rtr.handleDebug))
+	get.HandleFunc(`/status.json`, mgtonly(rtr.handleStatus))
+	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))
+	r.NotFoundHandler = 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)
+		}
+	}
+	n, err := rtr.keepstore.BlockRead(req.Context(), arvados.BlockReadOptions{
+		Locator:      mux.Vars(req)["locator"],
+		WriteTo:      w,
+		LocalLocator: localLocator,
+	})
+	if n == 0 && err != nil {
+		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: strings.Split(",", 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"])
+	if err != nil {
+		rtr.handleError(w, req, err)
+		return
+	}
+}
+
+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) {
+	cw := &countingWriter{writer: w}
+	err := rtr.keepstore.Index(req.Context(), IndexOptions{
+		MountUUID: mux.Vars(req)["uuid"],
+		Prefix:    mux.Vars(req)["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) handleDebug(w http.ResponseWriter, req *http.Request) {
+	rtr.handleError(w, req, errors.New("unimplemented"))
+}
+
+func (rtr *router) handleStatus(w http.ResponseWriter, req *http.Request) {
+	rtr.handleError(w, req, errors.New("unimplemented"))
+}
+
+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()
+	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"])
+	if err != nil {
+		rtr.handleError(w, req, err)
+		return
+	}
+}
+
+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 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
+}
diff --git a/services/keepstore/s3aws_volume.go b/services/keepstore/s3aws_volume.go
index 18b30f4638..89fe060552 100644
--- a/services/keepstore/s3aws_volume.go
+++ b/services/keepstore/s3aws_volume.go
@@ -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.String())
 	return v, v.check("")
 }
 
@@ -225,7 +231,7 @@ 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
@@ -236,62 +242,11 @@ 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 +268,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 +289,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)
@@ -462,55 +417,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
+		}
+	}
+	err = streamer.Close()
+	if err != nil {
+		return 0, v.translateError(err)
 	}
-	return count, 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 +478,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 +507,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 +531,16 @@ 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
+		return errMethodNotAllowed
 	}
 
-	rdr := bytes.NewReader(block)
+	rdr := bytes.NewReader(data)
 	r := NewCountingReaderAtSeeker(rdr, v.bucket.stats.TickOutBytes)
-	key := v.key(loc)
+	key := v.key(hash)
 	err := v.writeObject(ctx, key, r)
 	if err != nil {
 		return err
@@ -675,9 +635,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 +655,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
@@ -785,12 +748,12 @@ 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 {
+// BlockTouch sets the timestamp for the given locator to the current time.
+func (v *S3AWSVolume) BlockTouch(hash string) error {
 	if v.volume.ReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
-	key := v.key(loc)
+	key := v.key(hash)
 	_, err := v.head(key)
 	err = v.translateError(err)
 	if os.IsNotExist(err) && v.fixRace(key) {
@@ -845,9 +808,9 @@ func (b *s3AWSbucket) Del(path string) error {
 }
 
 // Trash a Keep block.
-func (v *S3AWSVolume) Trash(loc string) error {
+func (v *S3AWSVolume) BlockTrash(loc string) error {
 	if v.volume.ReadOnly && !v.volume.AllowTrashWhenReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
 	if t, err := v.Mtime(loc); err != nil {
 		return err
@@ -872,9 +835,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
diff --git a/services/keepstore/streamwriterat.go b/services/keepstore/streamwriterat.go
new file mode 100644
index 0000000000..a5ccf1ffae
--- /dev/null
+++ b/services/keepstore/streamwriterat.go
@@ -0,0 +1,148 @@
+// 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 {
+	nparts := (len(buf) + partsize - 1) / partsize
+	swa := &streamWriterAt{
+		writer:     w,
+		partsize:   partsize,
+		buf:        buf[:0],
+		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 := copy(swa.buf[pos:], p)
+	if n < len(p) {
+		return n, errors.New("write beyond end of buffer")
+	}
+	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
index 0000000000..c9db3b6eaf
--- /dev/null
+++ b/services/keepstore/streamwriterat_test.go
@@ -0,0 +1,101 @@
+// 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) TestInvalidUsage(c *C) {
+	p := []byte("foo")
+
+	swa := streamWriterAt{}
+	_, err := swa.WriteAt(p, 0)
+	c.Check(err, NotNil)
+	err = swa.Close()
+	c.Check(err, NotNil)
+
+	swa = streamWriterAt{buf: make([]byte, 3)}
+	_, err = swa.WriteAt(p, 0)
+	c.Check(err, NotNil)
+	err = swa.Close()
+	c.Check(err, NotNil)
+}
+
+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++ {
+					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], pos)
+						}()
+					}
+					go func() {
+						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(), 20, buf)
+		_, err := swa.WriteAt([]byte("foo"), len(buf)+offset)
+		c.Check(err, NotNil)
+		err = swa.Close()
+		c.Check(err, NotNil)
+	}
+}
+
+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(), partsize, make([]byte, bufsize))
+				_, err := swa.WriteAt(make([]byte, 1), 49)
+				c.Check(err, IsNil)
+				_, err = swa.WriteAt(make([]byte, 46), 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)
+				}
+			}
+		}
+	}
+}
diff --git a/services/keepstore/trash_worker.go b/services/keepstore/trash_worker.go
index 5e8a5a963c..7ab0019b0d 100644
--- a/services/keepstore/trash_worker.go
+++ b/services/keepstore/trash_worker.go
@@ -5,68 +5,134 @@
 package keepstore
 
 import (
-	"errors"
+	"sync"
 	"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"
 }
 
-// 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
-	}
+type trasher struct {
+	keepstore *keepstore
+	todo      []trashListItem
+	cond      *sync.Cond // lock guards todo accesses; cond broadcasts when todo becomes non-empty
+}
 
-	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)
-	} else {
-		volumes = []*VolumeMount{mnt}
+func newTrasher(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))
+		},
+	))
+	if !keepstore.cluster.Collections.BlobTrash {
+		keepstore.logger.Info("not running trash worker because Collections.BlobTrash == false")
+		return t
+	}
+	for i := 0; i < keepstore.cluster.Collections.BlobTrashConcurrency; i++ {
+		go t.runWorker()
 	}
+	return t
+}
 
-	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) SetTrashList(newlist []trashListItem) {
+	t.cond.L.Lock()
+	t.todo = newlist
+	t.cond.L.Unlock()
+	t.cond.Broadcast()
+}
+
+func (t *trasher) runWorker() {
+	for {
+		t.cond.L.Lock()
+		for len(t.todo) == 0 {
+			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)
+		item := t.todo[0]
+		t.todo = t.todo[1:]
+		t.cond.L.Unlock()
+
+		logger := t.keepstore.logger.WithField("locator", item.Locator)
+
+		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)
 			continue
 		}
 
-		if !cluster.Collections.BlobTrash {
-			err = errors.New("skipping because Collections.BlobTrash is false")
+		var mnts []*mount
+		if item.MountUUID == "" {
+			mnts = t.keepstore.mountsW
+		} else if mnt := t.keepstore.mounts[item.MountUUID]; mnt == nil {
+			logger.Warnf("ignoring trash request for nonexistent mount %s", item.MountUUID)
+			continue
+		} else if !mnt.AllowTrash {
+			logger.Warnf("ignoring trash request for readonly mount %s with AllowTrashWhenReadOnly==false", item.MountUUID)
+			continue
 		} else {
-			err = volume.Trash(trashRequest.Locator)
+			mnts = []*mount{mnt}
 		}
 
-		if err != nil {
-			logger.WithError(err).Errorf("%v Trash(%v)", volume, trashRequest.Locator)
-		} else {
-			logger.Infof("%v Trash(%v) OK", volume, trashRequest.Locator)
+		for _, mnt := range mnts {
+			logger := logger.WithField("mount", mnt.UUID)
+			mtime, err := mnt.Mtime(item.Locator)
+			if err != nil {
+				logger.WithError(err).Error("error getting stored mtime")
+				continue
+			}
+			if item.BlockMtime != mtime.UnixNano() {
+				logger.Infof("stored mtime %v does not match trash list value %v; skipping", mtime.UnixNano(), item.BlockMtime)
+				continue
+			}
+			err = mnt.BlockTrash(item.Locator)
+			if err != nil {
+				logger.WithError(err).Info("error trashing block")
+				continue
+			}
+			logger.Info("block trashed")
 		}
 	}
 }
+
+type trashEmptier struct{}
+
+func newTrashEmptier(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() {
+		for range time.NewTicker(d).C {
+			for _, mnt := range ks.mounts {
+				if mnt.KeepMount.AllowTrash {
+					mnt.volume.EmptyTrash()
+				}
+			}
+		}
+	}()
+	return &trashEmptier{}
+}
diff --git a/services/keepstore/unix_volume.go b/services/keepstore/unix_volume.go
index dee4bdc1c1..c04e53cad4 100644
--- a/services/keepstore/unix_volume.go
+++ b/services/keepstore/unix_volume.go
@@ -31,9 +31,14 @@ func init() {
 	driver["Directory"] = newDirectoryVolume
 }
 
-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 newDirectoryVolume(params newVolumeParams) (volume, error) {
+	v := &UnixVolume{
+		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
 	}
@@ -53,7 +58,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)
@@ -77,12 +82,12 @@ 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 ""
@@ -154,12 +159,12 @@ 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 {
+// BlockTouch sets the timestamp for the given locator to the current time
+func (v *UnixVolume) BlockTouch(hash string) error {
 	if v.volume.ReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
-	p := v.blockPath(loc)
+	p := v.blockPath(hash)
 	f, err := v.os.OpenFile(p, os.O_RDWR|os.O_APPEND, 0644)
 	if err != nil {
 		return err
@@ -213,72 +218,48 @@ 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 {
+// 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.volume.ReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
 	if v.IsFull() {
-		return FullError
+		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 +268,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)
@@ -361,7 +342,7 @@ var blockFileRe = regexp.MustCompile(`^[0-9a-f]{32}$`)
 //	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 +355,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 +372,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 +388,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 +419,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()
@@ -443,7 +432,7 @@ func (v *UnixVolume) Trash(loc string) error {
 	// 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
+		return errMethodNotAllowed
 	}
 	if err := v.lock(context.TODO()); err != nil {
 		return err
@@ -477,17 +466,17 @@ 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) {
+func (v *UnixVolume) BlockUntrash(hash string) error {
 	if v.volume.ReadOnly {
-		return MethodDisabledError
+		return errMethodNotAllowed
 	}
 
 	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 +486,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 +501,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
@@ -543,7 +532,7 @@ 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)
 		isFull = false
diff --git a/services/keepstore/volume.go b/services/keepstore/volume.go
index f597ff5781..16fd1207c2 100644
--- a/services/keepstore/volume.go
+++ b/services/keepstore/volume.go
@@ -6,418 +6,34 @@ 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]
-}
+type volumeDriver func(newVolumeParams) (volume, error)
 
-// Close the RRVolumeManager
-func (vm *RRVolumeManager) Close() {
+type newVolumeParams struct {
+	Cluster      *arvados.Cluster
+	ConfigVolume arvados.Volume
+	Logger       logrus.FieldLogger
+	MetricsVecs  *volumeMetricsVecs
+	BufferPool   *bufferPool
 }
 
 // VolumeStatus describes the current condition of a volume

-----------------------------------------------------------------------


hooks/post-receive
-- 




More information about the arvados-commits mailing list