satellite/audit: create the audit queue, chore, and worker (#2888)

This commit is contained in:
Natalie Villasana 2019-09-05 11:40:52 -04:00 committed by GitHub
parent 1fc0c63a1d
commit 6d363fb756
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 384 additions and 79 deletions

View File

@ -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
View 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
}

View 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")
})
}

View File

@ -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
View 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)
}

View 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")
}

View File

@ -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 {

View File

@ -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
}

View File

@ -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
View 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
}

View File

@ -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())

View File

@ -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