[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