[#1402] ir: Filter expired SGs in the audit process

Signed-off-by: Pavel Karpy <carpawell@nspcc.ru>
This commit is contained in:
Pavel Karpy 2022-05-19 22:40:22 +03:00 committed by fyrchik
parent 7864959d0c
commit 27304455bf
6 changed files with 83 additions and 42 deletions

View file

@ -589,6 +589,7 @@ func New(ctx context.Context, log *zap.Logger, cfg *viper.Viper, errChan chan<-
NetmapClient: server.netmapClient, NetmapClient: server.netmapClient,
ContainerClient: cnrClient, ContainerClient: cnrClient,
IRList: server, IRList: server,
EpochSource: server,
SGSource: clientCache, SGSource: clientCache,
Key: &server.key.PrivateKey, Key: &server.key.PrivateKey,
RPCSearchTimeout: cfg.GetDuration("audit.timeout.search"), RPCSearchTimeout: cfg.GetDuration("audit.timeout.search"),

View file

@ -12,7 +12,9 @@ import (
"github.com/nspcc-dev/neofs-node/pkg/services/object_manager/placement" "github.com/nspcc-dev/neofs-node/pkg/services/object_manager/placement"
"github.com/nspcc-dev/neofs-node/pkg/util/rand" "github.com/nspcc-dev/neofs-node/pkg/util/rand"
cid "github.com/nspcc-dev/neofs-sdk-go/container/id" cid "github.com/nspcc-dev/neofs-sdk-go/container/id"
"github.com/nspcc-dev/neofs-sdk-go/netmap"
oid "github.com/nspcc-dev/neofs-sdk-go/object/id" oid "github.com/nspcc-dev/neofs-sdk-go/object/id"
storagegroupSDK "github.com/nspcc-dev/neofs-sdk-go/storagegroup"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -80,14 +82,20 @@ func (ap *Processor) processStartAudit(epoch uint64) {
}) })
// search storage groups // search storage groups
storageGroups := ap.findStorageGroups(containers[i], n) storageGroupsIDs := ap.findStorageGroups(containers[i], n)
log.Info("select storage groups for audit", log.Info("select storage groups for audit",
zap.Stringer("cid", containers[i]),
zap.Int("amount", len(storageGroupsIDs)))
// filter expired storage groups
storageGroups := ap.filterExpiredSG(containers[i], storageGroupsIDs, nodes, *nm)
log.Info("filter expired storage groups for audit",
zap.Stringer("cid", containers[i]), zap.Stringer("cid", containers[i]),
zap.Int("amount", len(storageGroups))) zap.Int("amount", len(storageGroups)))
// skip audit for containers // skip audit for containers without
// without storage groups // non-expired storage groups
if len(storageGroups) == 0 { if len(storageGroupsIDs) == 0 {
continue continue
} }
@ -161,3 +169,41 @@ func (ap *Processor) findStorageGroups(cnr cid.ID, shuffled netmapcore.Nodes) []
return sg return sg
} }
func (ap *Processor) filterExpiredSG(cid cid.ID, sgIDs []oid.ID,
cnr [][]netmap.NodeInfo, nm netmap.NetMap) map[oid.ID]storagegroupSDK.StorageGroup {
sgs := make(map[oid.ID]storagegroupSDK.StorageGroup, len(sgIDs))
var getSGPrm GetSGPrm
getSGPrm.CID = cid
getSGPrm.Container = cnr
getSGPrm.NetMap = nm
for _, sgID := range sgIDs {
ctx, cancel := context.WithTimeout(context.Background(), ap.searchTimeout)
getSGPrm.OID = sgID
getSGPrm.Context = ctx
sg, err := ap.sgSrc.GetSG(getSGPrm)
cancel()
if err != nil {
ap.log.Error(
"could not get storage group object for audit, skipping",
zap.Stringer("cid", cid),
zap.Stringer("oid", sgID),
zap.Error(err),
)
continue
}
// filter expired epochs
if sg.ExpirationEpoch() > ap.epochSrc.EpochCounter() {
sgs[sgID] = *sg
}
}
return sgs
}

View file

@ -35,12 +35,20 @@ type (
Reset() int Reset() int
} }
// EpochSource is an interface that provides actual
// epoch information.
EpochSource interface {
// EpochCounter must return current epoch number.
EpochCounter() uint64
}
// Processor of events related to data audit. // Processor of events related to data audit.
Processor struct { Processor struct {
log *zap.Logger log *zap.Logger
pool *ants.Pool pool *ants.Pool
irList Indexer irList Indexer
sgSrc SGSource sgSrc SGSource
epochSrc EpochSource
searchTimeout time.Duration searchTimeout time.Duration
containerClient *cntClient.Client containerClient *cntClient.Client
@ -62,6 +70,7 @@ type (
TaskManager TaskManager TaskManager TaskManager
Reporter audit.Reporter Reporter audit.Reporter
Key *ecdsa.PrivateKey Key *ecdsa.PrivateKey
EpochSource EpochSource
} }
) )
@ -148,6 +157,8 @@ func New(p *Params) (*Processor, error) {
return nil, errors.New("ir/audit: audit result reporter is not set") return nil, errors.New("ir/audit: audit result reporter is not set")
case p.Key == nil: case p.Key == nil:
return nil, errors.New("ir/audit: signing key is not set") return nil, errors.New("ir/audit: signing key is not set")
case p.EpochSource == nil:
return nil, errors.New("ir/audit: epoch source is not set")
} }
pool, err := ants.NewPool(ProcessorPoolSize, ants.WithNonblocking(true)) pool, err := ants.NewPool(ProcessorPoolSize, ants.WithNonblocking(true))
@ -161,6 +172,7 @@ func New(p *Params) (*Processor, error) {
containerClient: p.ContainerClient, containerClient: p.ContainerClient,
irList: p.IRList, irList: p.IRList,
sgSrc: p.SGSource, sgSrc: p.SGSource,
epochSrc: p.EpochSource,
searchTimeout: p.RPCSearchTimeout, searchTimeout: p.RPCSearchTimeout,
netmapClient: p.NetmapClient, netmapClient: p.NetmapClient,
taskManager: p.TaskManager, taskManager: p.TaskManager,

View file

@ -26,7 +26,7 @@ type Context struct {
report *audit.Report report *audit.Report
sgMembersMtx sync.RWMutex sgMembersMtx sync.RWMutex
sgMembersCache map[int][]oid.ID sgMembersCache map[oid.ID][]oid.ID
placementMtx sync.Mutex placementMtx sync.Mutex
placementCache map[string][][]netmap.NodeInfo placementCache map[string][][]netmap.NodeInfo
@ -178,7 +178,7 @@ func (c *Context) containerID() cid.ID {
func (c *Context) init() { func (c *Context) init() {
c.report = audit.NewReport(c.containerID()) c.report = audit.NewReport(c.containerID())
c.sgMembersCache = make(map[int][]oid.ID) c.sgMembersCache = make(map[oid.ID][]oid.ID)
c.placementCache = make(map[string][][]netmap.NodeInfo) c.placementCache = make(map[string][][]netmap.NodeInfo)
@ -293,9 +293,9 @@ func (c *Context) updateHeadResponses(hdr *object.Object) {
} }
} }
func (c *Context) updateSGInfo(ind int, members []oid.ID) { func (c *Context) updateSGInfo(id oid.ID, members []oid.ID) {
c.sgMembersMtx.Lock() c.sgMembersMtx.Lock()
defer c.sgMembersMtx.Unlock() defer c.sgMembersMtx.Unlock()
c.sgMembersCache[ind] = members c.sgMembersCache[id] = members
} }

View file

@ -5,10 +5,10 @@ import (
"encoding/hex" "encoding/hex"
"sync" "sync"
"github.com/nspcc-dev/neofs-node/pkg/innerring/processors/audit"
"github.com/nspcc-dev/neofs-node/pkg/services/object_manager/placement" "github.com/nspcc-dev/neofs-node/pkg/services/object_manager/placement"
"github.com/nspcc-dev/neofs-node/pkg/util/rand" "github.com/nspcc-dev/neofs-node/pkg/util/rand"
oid "github.com/nspcc-dev/neofs-sdk-go/object/id" oid "github.com/nspcc-dev/neofs-sdk-go/object/id"
storagegroupSDK "github.com/nspcc-dev/neofs-sdk-go/storagegroup"
"github.com/nspcc-dev/tzhash/tz" "github.com/nspcc-dev/tzhash/tz"
"go.uber.org/zap" "go.uber.org/zap"
) )
@ -17,13 +17,11 @@ func (c *Context) executePoR() {
wg := new(sync.WaitGroup) wg := new(sync.WaitGroup)
sgs := c.task.StorageGroupList() sgs := c.task.StorageGroupList()
for i := range sgs { for id, sg := range sgs {
wg.Add(1) wg.Add(1)
sg := sgs[i]
if err := c.porWorkerPool.Submit(func() { if err := c.porWorkerPool.Submit(func() {
c.checkStorageGroupPoR(i, sg) c.checkStorageGroupPoR(id, sg)
wg.Done() wg.Done()
}); err != nil { }); err != nil {
wg.Done() wg.Done()
@ -36,26 +34,9 @@ func (c *Context) executePoR() {
c.report.SetPoRCounters(c.porRequests.Load(), c.porRetries.Load()) c.report.SetPoRCounters(c.porRequests.Load(), c.porRetries.Load())
} }
func (c *Context) checkStorageGroupPoR(ind int, sg oid.ID) { func (c *Context) checkStorageGroupPoR(sgID oid.ID, sg storagegroupSDK.StorageGroup) {
var getSgPrm audit.GetSGPrm members := sg.Members()
c.updateSGInfo(sgID, members)
getSgPrm.Context = c.task.AuditContext()
getSgPrm.CID = c.task.ContainerID()
getSgPrm.OID = sg
getSgPrm.NetMap = *c.task.NetworkMap()
getSgPrm.Container = c.task.ContainerNodes()
storageGroup, err := c.cnrCom.GetSG(getSgPrm) // get storage group
if err != nil {
c.log.Warn("can't get storage group",
zap.Stringer("sgid", sg),
zap.String("error", err.Error()))
return
}
members := storageGroup.Members()
c.updateSGInfo(ind, members)
var ( var (
tzHash []byte tzHash []byte
@ -73,7 +54,7 @@ func (c *Context) checkStorageGroupPoR(ind int, sg oid.ID) {
objectPlacement, err := c.buildPlacement(members[i]) objectPlacement, err := c.buildPlacement(members[i])
if err != nil { if err != nil {
c.log.Info("can't build placement for storage group member", c.log.Info("can't build placement for storage group member",
zap.Stringer("sg", sg), zap.Stringer("sg", sgID),
zap.String("member_id", members[i].String()), zap.String("member_id", members[i].String()),
) )
@ -133,16 +114,16 @@ func (c *Context) checkStorageGroupPoR(ind int, sg oid.ID) {
c.porRequests.Add(accRequests) c.porRequests.Add(accRequests)
c.porRetries.Add(accRetries) c.porRetries.Add(accRetries)
sizeCheck := storageGroup.ValidationDataSize() == totalSize sizeCheck := sg.ValidationDataSize() == totalSize
cs, _ := storageGroup.ValidationDataHash() cs, _ := sg.ValidationDataHash()
tzCheck := bytes.Equal(tzHash, cs.Value()) tzCheck := bytes.Equal(tzHash, cs.Value())
if sizeCheck && tzCheck { if sizeCheck && tzCheck {
c.report.PassedPoR(sg) // write report c.report.PassedPoR(sgID) // write report
} else { } else {
if !sizeCheck { if !sizeCheck {
c.log.Debug("storage group size check failed", c.log.Debug("storage group size check failed",
zap.Uint64("expected", storageGroup.ValidationDataSize()), zap.Uint64("expected", sg.ValidationDataSize()),
zap.Uint64("got", totalSize)) zap.Uint64("got", totalSize))
} }
@ -150,6 +131,6 @@ func (c *Context) checkStorageGroupPoR(ind int, sg oid.ID) {
c.log.Debug("storage group tz hash check failed") c.log.Debug("storage group tz hash check failed")
} }
c.report.FailedPoR(sg) // write report c.report.FailedPoR(sgID) // write report
} }
} }

View file

@ -7,6 +7,7 @@ import (
cid "github.com/nspcc-dev/neofs-sdk-go/container/id" cid "github.com/nspcc-dev/neofs-sdk-go/container/id"
"github.com/nspcc-dev/neofs-sdk-go/netmap" "github.com/nspcc-dev/neofs-sdk-go/netmap"
oid "github.com/nspcc-dev/neofs-sdk-go/object/id" oid "github.com/nspcc-dev/neofs-sdk-go/object/id"
storagegroupSDK "github.com/nspcc-dev/neofs-sdk-go/storagegroup"
) )
// Task groups groups the container audit parameters. // Task groups groups the container audit parameters.
@ -23,7 +24,7 @@ type Task struct {
cnrNodes [][]netmap.NodeInfo cnrNodes [][]netmap.NodeInfo
sgList []oid.ID sgList map[oid.ID]storagegroupSDK.StorageGroup
} }
// WithReporter sets audit report writer. // WithReporter sets audit report writer.
@ -111,7 +112,7 @@ func (t *Task) ContainerNodes() [][]netmap.NodeInfo {
} }
// WithStorageGroupList sets a list of storage groups from container under audit. // WithStorageGroupList sets a list of storage groups from container under audit.
func (t *Task) WithStorageGroupList(sgList []oid.ID) *Task { func (t *Task) WithStorageGroupList(sgList map[oid.ID]storagegroupSDK.StorageGroup) *Task {
if t != nil { if t != nil {
t.sgList = sgList t.sgList = sgList
} }
@ -120,6 +121,6 @@ func (t *Task) WithStorageGroupList(sgList []oid.ID) *Task {
} }
// StorageGroupList returns list of storage groups from container under audit. // StorageGroupList returns list of storage groups from container under audit.
func (t *Task) StorageGroupList() []oid.ID { func (t *Task) StorageGroupList() map[oid.ID]storagegroupSDK.StorageGroup {
return t.sgList return t.sgList
} }