satellite/audit: create the audit queue, chore, and worker (#2888)
This commit is contained in:
parent
1fc0c63a1d
commit
6d363fb756
@ -166,6 +166,11 @@ func (planet *Planet) newSatellites(count int) ([]*satellite.Peer, error) {
|
||||
Interval: 30 * time.Second,
|
||||
MinBytesPerSecond: 1 * memory.KB,
|
||||
MinDownloadTimeout: 5 * time.Second,
|
||||
MaxReverifyCount: 3,
|
||||
ChoreInterval: 30 * time.Second,
|
||||
QueueInterval: 1 * time.Hour,
|
||||
Slots: 3,
|
||||
WorkerConcurrency: 1,
|
||||
},
|
||||
GarbageCollection: gc.Config{
|
||||
Interval: 1 * time.Minute,
|
||||
|
85
satellite/audit/chore.go
Normal file
85
satellite/audit/chore.go
Normal file
@ -0,0 +1,85 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package audit
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"storj.io/storj/internal/sync2"
|
||||
"storj.io/storj/pkg/storj"
|
||||
"storj.io/storj/satellite/metainfo"
|
||||
)
|
||||
|
||||
// Chore populates reservoirs and the audit queue.
|
||||
type Chore struct {
|
||||
log *zap.Logger
|
||||
rand *rand.Rand
|
||||
queue *Queue
|
||||
Loop sync2.Cycle
|
||||
|
||||
metainfoLoop *metainfo.Loop
|
||||
config Config
|
||||
}
|
||||
|
||||
// NewChore instantiates Chore.
|
||||
func NewChore(log *zap.Logger, queue *Queue, metaLoop *metainfo.Loop, config Config) *Chore {
|
||||
return &Chore{
|
||||
log: log,
|
||||
rand: rand.New(rand.NewSource(time.Now().Unix())),
|
||||
queue: queue,
|
||||
Loop: *sync2.NewCycle(config.ChoreInterval),
|
||||
|
||||
metainfoLoop: metaLoop,
|
||||
config: config,
|
||||
}
|
||||
}
|
||||
|
||||
// Run starts the chore.
|
||||
func (chore *Chore) Run(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
return chore.Loop.Run(ctx, func(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
pathCollector := NewPathCollector(chore.config.Slots, chore.rand)
|
||||
err = chore.metainfoLoop.Join(ctx, pathCollector)
|
||||
if err != nil {
|
||||
chore.log.Error("error joining metainfoloop", zap.Error(err))
|
||||
return nil
|
||||
}
|
||||
|
||||
var newQueue []storj.Path
|
||||
queuePaths := make(map[storj.Path]struct{})
|
||||
|
||||
// Add reservoir paths to queue in pseudorandom order.
|
||||
for i := 0; i < chore.config.Slots; i++ {
|
||||
for _, res := range pathCollector.Reservoirs {
|
||||
// Skip reservoir if no path at this index.
|
||||
if len(res.Paths) <= i {
|
||||
continue
|
||||
}
|
||||
path := res.Paths[i]
|
||||
if path == "" {
|
||||
continue
|
||||
}
|
||||
if _, ok := queuePaths[path]; !ok {
|
||||
newQueue = append(newQueue, path)
|
||||
queuePaths[path] = struct{}{}
|
||||
}
|
||||
}
|
||||
}
|
||||
chore.queue.Swap(newQueue)
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
// Close closes chore.
|
||||
func (chore *Chore) Close() error {
|
||||
chore.Loop.Close()
|
||||
return nil
|
||||
}
|
68
satellite/audit/integration_test.go
Normal file
68
satellite/audit/integration_test.go
Normal file
@ -0,0 +1,68 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package audit_test
|
||||
|
||||
import (
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"storj.io/storj/internal/memory"
|
||||
"storj.io/storj/internal/testcontext"
|
||||
"storj.io/storj/internal/testplanet"
|
||||
"storj.io/storj/internal/testrand"
|
||||
"storj.io/storj/pkg/storj"
|
||||
"storj.io/storj/satellite/audit"
|
||||
)
|
||||
|
||||
func TestChoreAndWorkerIntegration(t *testing.T) {
|
||||
testplanet.Run(t, testplanet.Config{
|
||||
SatelliteCount: 1, StorageNodeCount: 5, UplinkCount: 1,
|
||||
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
|
||||
satellite := planet.Satellites[0]
|
||||
satellite.Audit.Worker.Loop.Pause()
|
||||
satellite.Audit.Chore.Loop.Pause()
|
||||
|
||||
ul := planet.Uplinks[0]
|
||||
|
||||
// Upload 2 remote files with 1 segment.
|
||||
for i := 0; i < 2; i++ {
|
||||
testData := testrand.Bytes(8 * memory.KiB)
|
||||
path := "/some/remote/path/" + strconv.Itoa(i)
|
||||
err := ul.Upload(ctx, satellite, "testbucket", path, testData)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
satellite.Audit.Chore.Loop.TriggerWait()
|
||||
require.EqualValues(t, 2, satellite.Audit.Queue.Size(), "audit queue")
|
||||
|
||||
uniquePaths := make(map[storj.Path]struct{})
|
||||
var err error
|
||||
var path storj.Path
|
||||
var pathCount int
|
||||
for {
|
||||
path, err = satellite.Audit.Queue.Next()
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
pathCount++
|
||||
_, ok := uniquePaths[path]
|
||||
require.False(t, ok, "expected unique path in chore queue")
|
||||
|
||||
uniquePaths[path] = struct{}{}
|
||||
}
|
||||
require.True(t, audit.ErrEmptyQueue.Has(err))
|
||||
require.Equal(t, 2, pathCount)
|
||||
require.Equal(t, 0, satellite.Audit.Queue.Size())
|
||||
|
||||
// Repopulate the queue for the worker.
|
||||
satellite.Audit.Chore.Loop.TriggerWait()
|
||||
require.EqualValues(t, 2, satellite.Audit.Queue.Size(), "audit queue")
|
||||
|
||||
// Make sure the worker processes all the items in the audit queue.
|
||||
satellite.Audit.Worker.Loop.TriggerWait()
|
||||
require.EqualValues(t, 0, satellite.Audit.Queue.Size(), "audit queue")
|
||||
})
|
||||
}
|
@ -55,8 +55,8 @@ func TestAuditPathCollector(t *testing.T) {
|
||||
}
|
||||
|
||||
r := rand.New(rand.NewSource(time.Now().Unix()))
|
||||
observer := audit.NewPathCollector(3, r)
|
||||
err = audits.ReservoirService.MetainfoLoop.Join(ctx, observer)
|
||||
observer := audit.NewPathCollector(4, r)
|
||||
err = satellite.Metainfo.Loop.Join(ctx, observer)
|
||||
require.NoError(t, err)
|
||||
|
||||
for _, node := range planet.StorageNodes {
|
||||
@ -64,9 +64,9 @@ func TestAuditPathCollector(t *testing.T) {
|
||||
require.NotNil(t, observer.Reservoirs[node.ID()])
|
||||
require.True(t, len(observer.Reservoirs[node.ID()].Paths) > 1)
|
||||
|
||||
// Require that len paths are <= 2 even though the PathCollector was instantiated with 3
|
||||
// because the maxReservoirSize is currently 2.
|
||||
require.True(t, len(observer.Reservoirs[node.ID()].Paths) <= 2)
|
||||
// Require that len paths are <= 3 even though the PathCollector was instantiated with 4
|
||||
// because the maxReservoirSize is currently 3.
|
||||
require.True(t, len(observer.Reservoirs[node.ID()].Paths) <= 3)
|
||||
|
||||
repeats := make(map[storj.Path]bool)
|
||||
for _, path := range observer.Reservoirs[node.ID()].Paths {
|
||||
|
53
satellite/audit/queue.go
Normal file
53
satellite/audit/queue.go
Normal file
@ -0,0 +1,53 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package audit
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/zeebo/errs"
|
||||
|
||||
"storj.io/storj/pkg/storj"
|
||||
)
|
||||
|
||||
// ErrEmptyQueue is used to indicate that the queue is empty.
|
||||
var ErrEmptyQueue = errs.Class("empty audit queue")
|
||||
|
||||
// Queue is a list of paths to audit, shared between the reservoir chore and audit workers.
|
||||
type Queue struct {
|
||||
mu sync.Mutex
|
||||
queue []storj.Path
|
||||
}
|
||||
|
||||
// Swap switches the backing queue slice with a new queue slice.
|
||||
func (q *Queue) Swap(newQueue []storj.Path) {
|
||||
q.mu.Lock()
|
||||
defer q.mu.Unlock()
|
||||
|
||||
q.queue = newQueue
|
||||
}
|
||||
|
||||
// Next gets the next item in the queue.
|
||||
func (q *Queue) Next() (storj.Path, error) {
|
||||
q.mu.Lock()
|
||||
defer q.mu.Unlock()
|
||||
|
||||
// return error if queue is empty
|
||||
if len(q.queue) == 0 {
|
||||
return "", ErrEmptyQueue.New("")
|
||||
}
|
||||
|
||||
next := q.queue[0]
|
||||
q.queue = q.queue[1:]
|
||||
|
||||
return next, nil
|
||||
}
|
||||
|
||||
// Size returns the size of the queue.
|
||||
func (q *Queue) Size() int {
|
||||
q.mu.Lock()
|
||||
defer q.mu.Unlock()
|
||||
|
||||
return len(q.queue)
|
||||
}
|
42
satellite/audit/queue_test.go
Normal file
42
satellite/audit/queue_test.go
Normal file
@ -0,0 +1,42 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package audit_test
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"storj.io/storj/pkg/storj"
|
||||
"storj.io/storj/satellite/audit"
|
||||
)
|
||||
|
||||
func TestQueue(t *testing.T) {
|
||||
q := &audit.Queue{}
|
||||
|
||||
_, err := q.Next()
|
||||
require.True(t, audit.ErrEmptyQueue.Has(err), "required ErrEmptyQueue error")
|
||||
|
||||
testQueue1 := []storj.Path{"a", "b", "c"}
|
||||
q.Swap(testQueue1)
|
||||
path, err := q.Next()
|
||||
require.NoError(t, err)
|
||||
require.EqualValues(t, testQueue1[0], path)
|
||||
|
||||
path, err = q.Next()
|
||||
require.NoError(t, err)
|
||||
require.EqualValues(t, testQueue1[1], path)
|
||||
|
||||
testQueue2 := []storj.Path{"0", "1", "2"}
|
||||
q.Swap(testQueue2)
|
||||
|
||||
for _, expected := range testQueue2 {
|
||||
path, err := q.Next()
|
||||
require.NoError(t, err)
|
||||
require.EqualValues(t, expected, path)
|
||||
}
|
||||
|
||||
_, err = q.Next()
|
||||
require.True(t, audit.ErrEmptyQueue.Has(err), "required ErrEmptyQueue error")
|
||||
}
|
@ -9,7 +9,7 @@ import (
|
||||
"storj.io/storj/pkg/storj"
|
||||
)
|
||||
|
||||
const maxReservoirSize = 2
|
||||
const maxReservoirSize = 3
|
||||
|
||||
// Reservoir holds a certain number of segments to reflect a random sample
|
||||
type Reservoir struct {
|
||||
|
@ -1,66 +0,0 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package audit
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"storj.io/storj/internal/sync2"
|
||||
"storj.io/storj/pkg/storj"
|
||||
"storj.io/storj/satellite/metainfo"
|
||||
)
|
||||
|
||||
// ReservoirService is a temp name for the service struct during the audit 2.0 refactor.
|
||||
// Once V3-2363 and V3-2364 are implemented, ReservoirService will replace the existing Service struct.
|
||||
type ReservoirService struct {
|
||||
log *zap.Logger
|
||||
|
||||
reservoirSlots int
|
||||
Reservoirs map[storj.NodeID]*Reservoir
|
||||
rand *rand.Rand
|
||||
|
||||
MetainfoLoop *metainfo.Loop
|
||||
Loop sync2.Cycle
|
||||
}
|
||||
|
||||
// NewReservoirService instantiates ReservoirService
|
||||
func NewReservoirService(log *zap.Logger, metaLoop *metainfo.Loop, config Config) *ReservoirService {
|
||||
return &ReservoirService{
|
||||
log: log,
|
||||
|
||||
reservoirSlots: config.Slots,
|
||||
rand: rand.New(rand.NewSource(time.Now().Unix())),
|
||||
|
||||
MetainfoLoop: metaLoop,
|
||||
Loop: *sync2.NewCycle(config.Interval),
|
||||
}
|
||||
}
|
||||
|
||||
// Run runs auditing service 2.0
|
||||
func (service *ReservoirService) Run(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
service.log.Info("audit 2.0 is starting up")
|
||||
|
||||
return service.Loop.Run(ctx, func(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
pathCollector := NewPathCollector(service.reservoirSlots, service.rand)
|
||||
err = service.MetainfoLoop.Join(ctx, pathCollector)
|
||||
if err != nil {
|
||||
service.log.Error("error joining metainfoloop", zap.Error(err))
|
||||
return nil
|
||||
}
|
||||
service.Reservoirs = pathCollector.Reservoirs
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
// Close halts the reservoir service loop
|
||||
func (service *ReservoirService) Close() error {
|
||||
service.Loop.Close()
|
||||
return nil
|
||||
}
|
@ -31,7 +31,10 @@ type Config struct {
|
||||
MinDownloadTimeout time.Duration `help:"the minimum duration for downloading a share from storage nodes before timing out" default:"25s"`
|
||||
MaxReverifyCount int `help:"limit above which we consider an audit is failed" default:"3"`
|
||||
|
||||
Slots int `help:"number of reservoir slots allotted for nodes, currently capped at 2" default:"1"`
|
||||
ChoreInterval time.Duration `help:"how often to run the reservoir chore" default:"24h"`
|
||||
QueueInterval time.Duration `help:"how often to recheck an empty audit queue" default:"1h"`
|
||||
Slots int `help:"number of reservoir slots allotted for nodes, currently capped at 3" default:"3"`
|
||||
WorkerConcurrency int `help:"number of workers to run audits on paths" default:"1"`
|
||||
}
|
||||
|
||||
// Service helps coordinate Cursor and Verifier to run the audit process continuously
|
||||
|
83
satellite/audit/worker.go
Normal file
83
satellite/audit/worker.go
Normal file
@ -0,0 +1,83 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package audit
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"storj.io/storj/internal/sync2"
|
||||
"storj.io/storj/pkg/storj"
|
||||
)
|
||||
|
||||
// Worker contains information for populating audit queue and processing audits.
|
||||
type Worker struct {
|
||||
log *zap.Logger
|
||||
queue *Queue
|
||||
Loop sync2.Cycle
|
||||
limiter sync2.Limiter
|
||||
}
|
||||
|
||||
// NewWorker instantiates Worker.
|
||||
func NewWorker(log *zap.Logger, queue *Queue, config Config) (*Worker, error) {
|
||||
return &Worker{
|
||||
log: log,
|
||||
|
||||
queue: queue,
|
||||
Loop: *sync2.NewCycle(config.QueueInterval),
|
||||
limiter: *sync2.NewLimiter(config.WorkerConcurrency),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Run runs audit service 2.0.
|
||||
func (worker *Worker) Run(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
worker.log.Debug("starting")
|
||||
|
||||
// Wait for all audits to run.
|
||||
defer worker.limiter.Wait()
|
||||
|
||||
return worker.Loop.Run(ctx, func(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
err = worker.process(ctx)
|
||||
if err != nil {
|
||||
worker.log.Error("process", zap.Error(Error.Wrap(err)))
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
// Close halts the worker.
|
||||
func (worker *Worker) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// process repeatedly removes an item from the queue and runs an audit.
|
||||
func (worker *Worker) process(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
worker.limiter.Wait()
|
||||
for {
|
||||
path, err := worker.queue.Next()
|
||||
if err != nil {
|
||||
if ErrEmptyQueue.Has(err) {
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
worker.limiter.Go(ctx, func() {
|
||||
err := worker.work(ctx, path)
|
||||
if err != nil {
|
||||
worker.log.Error("audit failed", zap.Error(err))
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func (worker *Worker) work(ctx context.Context, path storj.Path) error {
|
||||
// TODO: handle work.
|
||||
return nil
|
||||
}
|
@ -196,8 +196,10 @@ type Peer struct {
|
||||
Inspector *irreparable.Inspector
|
||||
}
|
||||
Audit struct {
|
||||
Service *audit.Service
|
||||
ReservoirService *audit.ReservoirService
|
||||
Service *audit.Service
|
||||
Queue *audit.Queue
|
||||
Worker *audit.Worker
|
||||
Chore *audit.Chore
|
||||
}
|
||||
|
||||
GarbageCollection struct {
|
||||
@ -492,7 +494,18 @@ func New(log *zap.Logger, full *identity.FullIdentity, db DB, revocationDB exten
|
||||
}
|
||||
|
||||
// setup audit 2.0
|
||||
peer.Audit.ReservoirService = audit.NewReservoirService(peer.Log.Named("reservoir service"),
|
||||
peer.Audit.Queue = &audit.Queue{}
|
||||
|
||||
peer.Audit.Worker, err = audit.NewWorker(peer.Log.Named("audit worker"),
|
||||
peer.Audit.Queue,
|
||||
config,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, errs.Combine(err, peer.Close())
|
||||
}
|
||||
|
||||
peer.Audit.Chore = audit.NewChore(peer.Log.Named("audit reservoir chore"),
|
||||
peer.Audit.Queue,
|
||||
peer.Metainfo.Loop,
|
||||
config,
|
||||
)
|
||||
@ -714,7 +727,10 @@ func (peer *Peer) Run(ctx context.Context) (err error) {
|
||||
return errs2.IgnoreCanceled(peer.Audit.Service.Run(ctx))
|
||||
})
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.Audit.ReservoirService.Run(ctx))
|
||||
return errs2.IgnoreCanceled(peer.Audit.Worker.Run(ctx))
|
||||
})
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.Audit.Chore.Run(ctx))
|
||||
})
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.GarbageCollection.Service.Run(ctx))
|
||||
@ -767,6 +783,13 @@ func (peer *Peer) Close() error {
|
||||
errlist.Add(peer.Marketing.Listener.Close())
|
||||
}
|
||||
|
||||
if peer.Audit.Chore != nil {
|
||||
errlist.Add(peer.Audit.Chore.Close())
|
||||
}
|
||||
if peer.Audit.Worker != nil {
|
||||
errlist.Add(peer.Audit.Worker.Close())
|
||||
}
|
||||
|
||||
// close services in reverse initialization order
|
||||
if peer.DBCleanup.Chore != nil {
|
||||
errlist.Add(peer.DBCleanup.Chore.Close())
|
||||
|
13
scripts/testdata/satellite-config.yaml.lock
vendored
13
scripts/testdata/satellite-config.yaml.lock
vendored
@ -1,3 +1,6 @@
|
||||
# how often to run the reservoir chore
|
||||
# audit.chore-interval: 24h0m0s
|
||||
|
||||
# how frequently segments are audited
|
||||
# audit.interval: 30s
|
||||
|
||||
@ -13,8 +16,14 @@
|
||||
# the minimum duration for downloading a share from storage nodes before timing out
|
||||
# audit.min-download-timeout: 25s
|
||||
|
||||
# number of reservoir slots allotted for nodes, currently capped at 2
|
||||
# audit.slots: 1
|
||||
# how often to recheck an empty audit queue
|
||||
# audit.queue-interval: 1h0m0s
|
||||
|
||||
# number of reservoir slots allotted for nodes, currently capped at 3
|
||||
# audit.slots: 3
|
||||
|
||||
# number of workers to run audits on paths
|
||||
# audit.worker-concurrency: 1
|
||||
|
||||
# how frequently checker should check for bad segments
|
||||
# checker.interval: 30s
|
||||
|
Loading…
Reference in New Issue
Block a user