Implement garbage collection on satellite (#2577)
* Added a gc package at satellite/gc, which contains the gc.Service, which runs garbage collection integrated with the metainfoloop, and the gc PieceTracker, which implements the metainfo loop Observer interface and stores all of the filters (about which pieces are good) for each node. * Added a gc config located at satellite/gc/service.go (loop disabled by default in release) * Creates bloom filters with pieces to be retained inside the metainfo loop * Sends RetainRequests (or filters with good piece ids) to all storage nodes.
This commit is contained in:
parent
b8fe349816
commit
f11413bc8e
@ -25,6 +25,7 @@ import (
|
||||
"storj.io/storj/satellite"
|
||||
"storj.io/storj/satellite/console"
|
||||
"storj.io/storj/satellite/console/consoleweb"
|
||||
"storj.io/storj/satellite/gc"
|
||||
"storj.io/storj/satellite/mailservice"
|
||||
"storj.io/storj/satellite/marketingweb"
|
||||
"storj.io/storj/satellite/metainfo"
|
||||
@ -169,6 +170,13 @@ func (planet *Planet) newSatellites(count int) ([]*satellite.Peer, error) {
|
||||
MinBytesPerSecond: 1 * memory.KB,
|
||||
MinDownloadTimeout: 5 * time.Second,
|
||||
},
|
||||
GarbageCollection: gc.Config{
|
||||
Interval: 1 * time.Minute,
|
||||
Enabled: true,
|
||||
InitialPieces: 10,
|
||||
FalsePositiveRate: 0.1,
|
||||
ConcurrentSends: 1,
|
||||
},
|
||||
Tally: tally.Config{
|
||||
Interval: 30 * time.Second,
|
||||
},
|
||||
|
@ -10,6 +10,7 @@ import (
|
||||
|
||||
"github.com/zeebo/errs"
|
||||
|
||||
"storj.io/storj/internal/memory"
|
||||
"storj.io/storj/pkg/storj"
|
||||
)
|
||||
|
||||
@ -39,18 +40,35 @@ func newExplicit(seed, hashCount byte, sizeInBytes int) *Filter {
|
||||
|
||||
// NewOptimal returns a filter based on expected element count and false positive rate.
|
||||
func NewOptimal(expectedElements int, falsePositiveRate float64) *Filter {
|
||||
hashCount, sizeInBytes := getHashCountAndSize(expectedElements, falsePositiveRate)
|
||||
seed := byte(rand.Intn(255))
|
||||
|
||||
return newExplicit(seed, byte(hashCount), sizeInBytes)
|
||||
}
|
||||
|
||||
// NewOptimalMaxSize returns a filter based on expected element count and false positive rate, capped at a maximum size in bytes
|
||||
func NewOptimalMaxSize(expectedElements int, falsePositiveRate float64, maxSize memory.Size) *Filter {
|
||||
hashCount, sizeInBytes := getHashCountAndSize(expectedElements, falsePositiveRate)
|
||||
seed := byte(rand.Intn(255))
|
||||
|
||||
if sizeInBytes > maxSize.Int() {
|
||||
sizeInBytes = maxSize.Int()
|
||||
}
|
||||
|
||||
return newExplicit(seed, byte(hashCount), sizeInBytes)
|
||||
}
|
||||
|
||||
func getHashCountAndSize(expectedElements int, falsePositiveRate float64) (hashCount, size int) {
|
||||
// calculation based on https://en.wikipedia.org/wiki/Bloom_filter#Optimal_number_of_hash_functions
|
||||
bitsPerElement := -1.44 * math.Log2(falsePositiveRate)
|
||||
hashCount := math.Ceil(bitsPerElement * math.Log(2))
|
||||
hashCount = int(math.Ceil(bitsPerElement * math.Log(2)))
|
||||
if hashCount > 32 {
|
||||
// it will never be larger, but just in case to avoid overflow
|
||||
hashCount = 32
|
||||
}
|
||||
sizeInBytes := int(math.Ceil(float64(expectedElements) * bitsPerElement / 8))
|
||||
size = int(math.Ceil(float64(expectedElements) * bitsPerElement / 8))
|
||||
|
||||
return newExplicit(seed, byte(hashCount), sizeInBytes)
|
||||
return hashCount, size
|
||||
}
|
||||
|
||||
// Parameters returns filter parameters.
|
||||
@ -144,3 +162,9 @@ func (filter *Filter) Bytes() []byte {
|
||||
copy(bytes[3:], filter.table)
|
||||
return bytes
|
||||
}
|
||||
|
||||
// Size returns the size of Bytes call.
|
||||
func (filter *Filter) Size() int64 {
|
||||
// the first three bytes represent the version, seed, and hash count
|
||||
return int64(1 + 1 + 1 + len(filter.table))
|
||||
}
|
||||
|
21
satellite/gc/doc.go
Normal file
21
satellite/gc/doc.go
Normal file
@ -0,0 +1,21 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
/*
|
||||
Package gc contains the functions needed to run garbage collection.
|
||||
|
||||
The gc.PieceTracker implements the metainfo loop Observer interface
|
||||
allowing us to subscribe to the loop to get information for every segment
|
||||
in the metainfo database.
|
||||
|
||||
The gc.PieceTracker handling functions are used by the gc.Service to periodically
|
||||
account for all existing pieces on storage nodes and create "retain requests"
|
||||
which contain a bloom filter of all pieces that possibly exist on a storage node.
|
||||
|
||||
The gc.Service will send that request to the storagenode after a full metaloop
|
||||
iteration, and the storage node will use that request to delete the "garbage" pieces
|
||||
that are not in the bloom filter.
|
||||
|
||||
See storj/docs/design/garbage-collection.md for more info.
|
||||
*/
|
||||
package gc
|
122
satellite/gc/gc_test.go
Normal file
122
satellite/gc/gc_test.go
Normal file
@ -0,0 +1,122 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package gc_test
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"storj.io/storj/internal/memory"
|
||||
"storj.io/storj/internal/testcontext"
|
||||
"storj.io/storj/internal/testplanet"
|
||||
"storj.io/storj/internal/testrand"
|
||||
"storj.io/storj/pkg/encryption"
|
||||
"storj.io/storj/pkg/paths"
|
||||
"storj.io/storj/pkg/pb"
|
||||
"storj.io/storj/pkg/storj"
|
||||
"storj.io/storj/satellite"
|
||||
)
|
||||
|
||||
// TestGarbageCollection does the following:
|
||||
// * Set up a network with one storagenode
|
||||
// * Upload two objects
|
||||
// * Delete one object from the metainfo service on the satellite
|
||||
// * Wait for bloom filter generation
|
||||
// * Check that pieces of the deleted object are deleted on the storagenode
|
||||
// * Check that pieces of the kept object are not deleted on the storagenode
|
||||
func TestGarbageCollection(t *testing.T) {
|
||||
testplanet.Run(t, testplanet.Config{
|
||||
SatelliteCount: 1, StorageNodeCount: 1, UplinkCount: 1,
|
||||
Reconfigure: testplanet.Reconfigure{
|
||||
Satellite: func(log *zap.Logger, index int, config *satellite.Config) {
|
||||
config.GarbageCollection.FalsePositiveRate = 0.000000001
|
||||
config.GarbageCollection.Interval = 500 * time.Millisecond
|
||||
},
|
||||
},
|
||||
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
|
||||
satellite := planet.Satellites[0]
|
||||
upl := planet.Uplinks[0]
|
||||
targetNode := planet.StorageNodes[0]
|
||||
gcService := satellite.GarbageCollection.Service
|
||||
|
||||
// Upload two objects
|
||||
testData1 := testrand.Bytes(8 * memory.KiB)
|
||||
testData2 := testrand.Bytes(8 * memory.KiB)
|
||||
|
||||
err := upl.Upload(ctx, satellite, "testbucket", "test/path/1", testData1)
|
||||
require.NoError(t, err)
|
||||
deletedEncPath, pointerToDelete := getPointer(ctx, t, satellite, upl, "testbucket", "test/path/1")
|
||||
var deletedPieceID storj.PieceID
|
||||
for _, p := range pointerToDelete.GetRemote().GetRemotePieces() {
|
||||
if p.NodeId == targetNode.ID() {
|
||||
deletedPieceID = pointerToDelete.GetRemote().RootPieceId.Derive(p.NodeId, p.PieceNum)
|
||||
break
|
||||
}
|
||||
}
|
||||
require.NotZero(t, deletedPieceID)
|
||||
|
||||
err = upl.Upload(ctx, satellite, "testbucket", "test/path/2", testData2)
|
||||
require.NoError(t, err)
|
||||
_, pointerToKeep := getPointer(ctx, t, satellite, upl, "testbucket", "test/path/2")
|
||||
var keptPieceID storj.PieceID
|
||||
for _, p := range pointerToKeep.GetRemote().GetRemotePieces() {
|
||||
if p.NodeId == targetNode.ID() {
|
||||
keptPieceID = pointerToKeep.GetRemote().RootPieceId.Derive(p.NodeId, p.PieceNum)
|
||||
break
|
||||
}
|
||||
}
|
||||
require.NotZero(t, keptPieceID)
|
||||
|
||||
// Delete one object from metainfo service on satellite
|
||||
err = satellite.Metainfo.Service.Delete(ctx, deletedEncPath)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Check that piece of the deleted object is on the storagenode
|
||||
pieceInfo, err := targetNode.DB.PieceInfo().Get(ctx, satellite.ID(), deletedPieceID)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, pieceInfo)
|
||||
|
||||
// The pieceInfo.GetPieceIDs query converts piece creation and the filter creation timestamps
|
||||
// to datetime in sql. This chops off all precision beyond seconds.
|
||||
// In this test, the amount of time that elapses between piece uploads and the gc loop is
|
||||
// less than a second, meaning datetime(piece_creation) < datetime(filter_creation) is false unless we sleep
|
||||
// for a second.
|
||||
time.Sleep(1 * time.Second)
|
||||
|
||||
// Wait for next iteration of garbage collection to finish
|
||||
gcService.Loop.TriggerWait()
|
||||
|
||||
// Check that piece of the deleted object is not on the storagenode
|
||||
pieceInfo, err = targetNode.DB.PieceInfo().Get(ctx, satellite.ID(), deletedPieceID)
|
||||
require.Error(t, err)
|
||||
require.Nil(t, pieceInfo)
|
||||
|
||||
// Check that piece of the kept object is on the storagenode
|
||||
pieceInfo, err = targetNode.DB.PieceInfo().Get(ctx, satellite.ID(), keptPieceID)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, pieceInfo)
|
||||
})
|
||||
}
|
||||
|
||||
func getPointer(ctx *testcontext.Context, t *testing.T, satellite *satellite.Peer, upl *testplanet.Uplink, bucket, path string) (lastSegPath string, pointer *pb.Pointer) {
|
||||
projects, err := satellite.DB.Console().Projects().GetAll(ctx)
|
||||
require.NoError(t, err)
|
||||
require.Len(t, projects, 1)
|
||||
|
||||
encParameters := upl.GetConfig(satellite).GetEncryptionParameters()
|
||||
cipherSuite := encParameters.CipherSuite
|
||||
store := encryption.NewStore()
|
||||
store.SetDefaultKey(new(storj.Key))
|
||||
encryptedPath, err := encryption.EncryptPath(bucket, paths.NewUnencrypted(path), cipherSuite, store)
|
||||
require.NoError(t, err)
|
||||
|
||||
lastSegPath = storj.JoinPaths(projects[0].ID.String(), "l", bucket, encryptedPath.Raw())
|
||||
pointer, err = satellite.Metainfo.Service.Get(ctx, lastSegPath)
|
||||
require.NoError(t, err)
|
||||
|
||||
return lastSegPath, pointer
|
||||
}
|
82
satellite/gc/piecetracker.go
Normal file
82
satellite/gc/piecetracker.go
Normal file
@ -0,0 +1,82 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package gc
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"storj.io/storj/internal/memory"
|
||||
"storj.io/storj/pkg/bloomfilter"
|
||||
"storj.io/storj/pkg/pb"
|
||||
"storj.io/storj/pkg/storj"
|
||||
)
|
||||
|
||||
// PieceTracker implements the metainfo loop observer interface for garbage collection
|
||||
type PieceTracker struct {
|
||||
log *zap.Logger
|
||||
config Config
|
||||
creationDate time.Time
|
||||
pieceCounts map[storj.NodeID]int
|
||||
|
||||
retainInfos map[storj.NodeID]*RetainInfo
|
||||
}
|
||||
|
||||
// NewPieceTracker instantiates a new gc piece tracker to be subscribed to the metainfo loop
|
||||
func NewPieceTracker(log *zap.Logger, config Config, pieceCounts map[storj.NodeID]int) *PieceTracker {
|
||||
return &PieceTracker{
|
||||
log: log,
|
||||
config: config,
|
||||
creationDate: time.Now().UTC(),
|
||||
pieceCounts: pieceCounts,
|
||||
|
||||
retainInfos: make(map[storj.NodeID]*RetainInfo),
|
||||
}
|
||||
}
|
||||
|
||||
// RemoteSegment takes a remote segment found in metainfo and adds pieces to bloom filters
|
||||
func (pieceTracker *PieceTracker) RemoteSegment(ctx context.Context, path storj.Path, pointer *pb.Pointer) (err error) {
|
||||
defer mon.Task()(&ctx, path)(&err)
|
||||
|
||||
remote := pointer.GetRemote()
|
||||
pieces := remote.GetRemotePieces()
|
||||
|
||||
for _, piece := range pieces {
|
||||
pieceID := remote.RootPieceId.Derive(piece.NodeId, piece.PieceNum)
|
||||
pieceTracker.add(piece.NodeId, pieceID)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// RemoteObject returns nil because gc does not interact with remote objects
|
||||
func (pieceTracker *PieceTracker) RemoteObject(ctx context.Context, path storj.Path, pointer *pb.Pointer) (err error) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// InlineSegment returns nil because we're only doing gc for storage nodes for now
|
||||
func (pieceTracker *PieceTracker) InlineSegment(ctx context.Context, path storj.Path, pointer *pb.Pointer) (err error) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// adds a pieceID to the relevant node's RetainInfo
|
||||
func (pieceTracker *PieceTracker) add(nodeID storj.NodeID, pieceID storj.PieceID) {
|
||||
if _, ok := pieceTracker.retainInfos[nodeID]; !ok {
|
||||
// If we know how many pieces a node should be storing, use that number. Otherwise use default.
|
||||
numPieces := pieceTracker.config.InitialPieces
|
||||
if pieceTracker.pieceCounts[nodeID] > 0 {
|
||||
numPieces = pieceTracker.pieceCounts[nodeID]
|
||||
}
|
||||
// limit size of bloom filter to ensure we are under the limit for GRPC
|
||||
filter := bloomfilter.NewOptimalMaxSize(numPieces, pieceTracker.config.FalsePositiveRate, 2*memory.MiB)
|
||||
pieceTracker.retainInfos[nodeID] = &RetainInfo{
|
||||
Filter: filter,
|
||||
CreationDate: pieceTracker.creationDate,
|
||||
}
|
||||
}
|
||||
|
||||
pieceTracker.retainInfos[nodeID].Filter.Add(pieceID)
|
||||
pieceTracker.retainInfos[nodeID].Count++
|
||||
}
|
147
satellite/gc/service.go
Normal file
147
satellite/gc/service.go
Normal file
@ -0,0 +1,147 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package gc
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/zeebo/errs"
|
||||
"go.uber.org/zap"
|
||||
monkit "gopkg.in/spacemonkeygo/monkit.v2"
|
||||
|
||||
"storj.io/storj/internal/sync2"
|
||||
"storj.io/storj/pkg/bloomfilter"
|
||||
"storj.io/storj/pkg/overlay"
|
||||
"storj.io/storj/pkg/pb"
|
||||
"storj.io/storj/pkg/storj"
|
||||
"storj.io/storj/pkg/transport"
|
||||
"storj.io/storj/satellite/metainfo"
|
||||
"storj.io/storj/uplink/piecestore"
|
||||
)
|
||||
|
||||
var (
|
||||
// Error defines the gc service errors class
|
||||
Error = errs.Class("gc service error")
|
||||
mon = monkit.Package()
|
||||
)
|
||||
|
||||
// Config contains configurable values for garbage collection
|
||||
type Config struct {
|
||||
Interval time.Duration `help:"the time between each send of garbage collection filters to storage nodes" releaseDefault:"168h" devDefault:"10m"`
|
||||
Enabled bool `help:"set if garbage collection is enabled or not" releaseDefault:"false" devDefault:"true"`
|
||||
// value for InitialPieces currently based on average pieces per node
|
||||
InitialPieces int `help:"the initial number of pieces expected for a storage node to have, used for creating a filter" releaseDefault:"400000" devDefault:"10"`
|
||||
FalsePositiveRate float64 `help:"the false positive rate used for creating a garbage collection bloom filter" releaseDefault:"0.1" devDefault:"0.1"`
|
||||
ConcurrentSends int `help:"the number of nodes to concurrently send garbage collection bloom filters to" releaseDefault:"1" devDefault:"1"`
|
||||
}
|
||||
|
||||
// Service implements the garbage collection service
|
||||
type Service struct {
|
||||
log *zap.Logger
|
||||
config Config
|
||||
Loop sync2.Cycle
|
||||
|
||||
transport transport.Client
|
||||
overlay overlay.DB
|
||||
metainfoLoop *metainfo.Loop
|
||||
}
|
||||
|
||||
// RetainInfo contains info needed for a storage node to retain important data and delete garbage data
|
||||
type RetainInfo struct {
|
||||
Filter *bloomfilter.Filter
|
||||
CreationDate time.Time
|
||||
Count int
|
||||
}
|
||||
|
||||
// NewService creates a new instance of the gc service
|
||||
func NewService(log *zap.Logger, config Config, transport transport.Client, overlay overlay.DB, loop *metainfo.Loop) *Service {
|
||||
return &Service{
|
||||
log: log,
|
||||
config: config,
|
||||
Loop: *sync2.NewCycle(config.Interval),
|
||||
|
||||
transport: transport,
|
||||
overlay: overlay,
|
||||
metainfoLoop: loop,
|
||||
}
|
||||
}
|
||||
|
||||
// Run starts the gc loop service
|
||||
func (service *Service) Run(ctx context.Context) (err error) {
|
||||
|
||||
if !service.config.Enabled {
|
||||
return nil
|
||||
}
|
||||
|
||||
// TODO retrieve piece counts from overlay (when there is a column for them)
|
||||
lastPieceCounts := make(map[storj.NodeID]int)
|
||||
|
||||
return service.Loop.Run(ctx, func(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
pieceTracker := NewPieceTracker(service.log.Named("gc observer"), service.config, lastPieceCounts)
|
||||
|
||||
// collect things to retain
|
||||
err = service.metainfoLoop.Join(ctx, pieceTracker)
|
||||
if err != nil {
|
||||
service.log.Error("error joining metainfoloop", zap.Error(err))
|
||||
return nil
|
||||
}
|
||||
|
||||
// save piece counts for next iteration
|
||||
for id := range lastPieceCounts {
|
||||
delete(lastPieceCounts, id)
|
||||
}
|
||||
for id, info := range pieceTracker.retainInfos {
|
||||
lastPieceCounts[id] = info.Count
|
||||
}
|
||||
|
||||
// monitor information
|
||||
for _, info := range pieceTracker.retainInfos {
|
||||
mon.IntVal("node_piece_count").Observe(int64(info.Count))
|
||||
mon.IntVal("retain_filter_size_bytes").Observe(info.Filter.Size())
|
||||
}
|
||||
|
||||
// send retain requests
|
||||
limiter := sync2.NewLimiter(service.config.ConcurrentSends)
|
||||
for id, info := range pieceTracker.retainInfos {
|
||||
id, info := id, info
|
||||
limiter.Go(ctx, func() {
|
||||
err := service.sendRetainRequest(ctx, id, info)
|
||||
if err != nil {
|
||||
service.log.Error("error sending retain info to node", zap.Stringer("node ID", id), zap.Error(err))
|
||||
}
|
||||
})
|
||||
}
|
||||
limiter.Wait()
|
||||
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
func (service *Service) sendRetainRequest(ctx context.Context, id storj.NodeID, info *RetainInfo) (err error) {
|
||||
defer mon.Task()(&ctx, id.String())(&err)
|
||||
|
||||
log := service.log.Named(id.String())
|
||||
|
||||
dossier, err := service.overlay.Get(ctx, id)
|
||||
if err != nil {
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
|
||||
client, err := piecestore.Dial(ctx, service.transport, &dossier.Node, log, piecestore.DefaultConfig)
|
||||
if err != nil {
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
defer func() {
|
||||
err = errs.Combine(err, Error.Wrap(client.Close()))
|
||||
}()
|
||||
|
||||
err = client.Retain(ctx, &pb.RetainRequest{
|
||||
CreationDate: info.CreationDate,
|
||||
Filter: info.Filter.Bytes(),
|
||||
})
|
||||
return Error.Wrap(err)
|
||||
}
|
@ -46,6 +46,7 @@ import (
|
||||
"storj.io/storj/satellite/console"
|
||||
"storj.io/storj/satellite/console/consoleauth"
|
||||
"storj.io/storj/satellite/console/consoleweb"
|
||||
"storj.io/storj/satellite/gc"
|
||||
"storj.io/storj/satellite/inspector"
|
||||
"storj.io/storj/satellite/mailservice"
|
||||
"storj.io/storj/satellite/mailservice/simulate"
|
||||
@ -118,6 +119,8 @@ type Config struct {
|
||||
Repairer repairer.Config
|
||||
Audit audit.Config
|
||||
|
||||
GarbageCollection gc.Config
|
||||
|
||||
Tally tally.Config
|
||||
Rollup rollup.Config
|
||||
LiveAccounting live.Config
|
||||
@ -188,6 +191,10 @@ type Peer struct {
|
||||
Service *audit.Service
|
||||
}
|
||||
|
||||
GarbageCollection struct {
|
||||
Service *gc.Service
|
||||
}
|
||||
|
||||
Accounting struct {
|
||||
Tally *tally.Service
|
||||
Rollup *rollup.Service
|
||||
@ -470,6 +477,18 @@ func New(log *zap.Logger, full *identity.FullIdentity, db DB, config *Config, ve
|
||||
}
|
||||
}
|
||||
|
||||
{ // setup garbage collection
|
||||
log.Debug("Setting up garbage collection")
|
||||
|
||||
peer.GarbageCollection.Service = gc.NewService(
|
||||
peer.Log.Named("garbage collection"),
|
||||
config.GarbageCollection,
|
||||
peer.Transport,
|
||||
peer.DB.OverlayCache(),
|
||||
peer.Metainfo.Loop,
|
||||
)
|
||||
}
|
||||
|
||||
{ // setup accounting
|
||||
log.Debug("Setting up accounting")
|
||||
peer.Accounting.Tally = tally.New(peer.Log.Named("tally"), peer.DB.StoragenodeAccounting(), peer.DB.ProjectAccounting(), peer.LiveAccounting.Service, peer.Metainfo.Service, peer.Overlay.Service, 0, config.Tally.Interval)
|
||||
@ -668,6 +687,9 @@ func (peer *Peer) Run(ctx context.Context) (err error) {
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.Audit.Service.Run(ctx))
|
||||
})
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.GarbageCollection.Service.Run(ctx))
|
||||
})
|
||||
group.Go(func() error {
|
||||
// TODO: move the message into Server instead
|
||||
// Don't change the format of this comment, it is used to figure out the node id.
|
||||
|
15
scripts/testdata/satellite-config.yaml.lock
vendored
15
scripts/testdata/satellite-config.yaml.lock
vendored
@ -73,6 +73,21 @@ defaults: "release"
|
||||
# the amount of nodes read from the overlay cache in a single pagination call
|
||||
# discovery.refresh-limit: 100
|
||||
|
||||
# the number of nodes to concurrently send garbage collection bloom filters to
|
||||
# garbage-collection.concurrent-sends: 1
|
||||
|
||||
# set if garbage collection is enabled or not
|
||||
# garbage-collection.enabled: false
|
||||
|
||||
# the false positive rate used for creating a garbage collection bloom filter
|
||||
# garbage-collection.false-positive-rate: 0.1
|
||||
|
||||
# the initial number of pieces expected for a storage node to have, used for creating a filter
|
||||
# garbage-collection.initial-pieces: 400000
|
||||
|
||||
# the time between each send of garbage collection filters to storage nodes
|
||||
# garbage-collection.interval: 168h0m0s
|
||||
|
||||
# help for setup
|
||||
# help: false
|
||||
|
||||
|
@ -101,16 +101,21 @@ func (db *pieceinfo) Get(ctx context.Context, satelliteID storj.NodeID, pieceID
|
||||
|
||||
var orderLimit []byte
|
||||
var uplinkPieceHash []byte
|
||||
var pieceExpiration *time.Time
|
||||
|
||||
err = db.db.QueryRowContext(ctx, db.Rebind(`
|
||||
SELECT piece_size, piece_creation, piece_expiration, order_limit, uplink_piece_hash
|
||||
FROM pieceinfo_
|
||||
WHERE satellite_id = ? AND piece_id = ?
|
||||
`), satelliteID, pieceID).Scan(&info.PieceSize, &info.PieceCreation, &info.PieceExpiration, &orderLimit, &uplinkPieceHash)
|
||||
`), satelliteID, pieceID).Scan(&info.PieceSize, &info.PieceCreation, &pieceExpiration, &orderLimit, &uplinkPieceHash)
|
||||
if err != nil {
|
||||
return nil, ErrInfo.Wrap(err)
|
||||
}
|
||||
|
||||
if pieceExpiration != nil {
|
||||
info.PieceExpiration = *pieceExpiration
|
||||
}
|
||||
|
||||
info.OrderLimit = &pb.OrderLimit{}
|
||||
err = proto.Unmarshal(orderLimit, info.OrderLimit)
|
||||
if err != nil {
|
||||
|
Loading…
Reference in New Issue
Block a user