frostfs-node/pkg/local_object_storage/engine/inhume.go
Aleksey Savchuk 77b8545601
[#1689] engine: Fix removal of objects not found on node
Ensured correct object status if the object is not found on a node.
Fixed regression introduced in #1450.

Besides an object not being found on any shard, it also important to
remove it anyway in order to populate the metabase indexes because
they are responsible for the correct object status, i.e., the status
will be `object not found` without the indexes, the status will be
`object is already removed` with the indexes.

Change-Id: I6237fbc0f8bb0c4f2a51ada3a68f52950050e660
Signed-off-by: Aleksey Savchuk <a.savchuk@yadro.com>
2025-04-22 10:04:42 +03:00

568 lines
15 KiB
Go

package engine
import (
"context"
"errors"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
"go.uber.org/zap"
)
// InhumePrm encapsulates parameters for inhume operation.
type InhumePrm struct {
tombstone *oid.Address
addrs []oid.Address
forceRemoval bool
}
// WithTarget sets a list of objects that should be inhumed and tombstone address
// as the reason for inhume operation.
//
// tombstone should not be nil, addr should not be empty.
// Should not be called along with MarkAsGarbage.
func (p *InhumePrm) WithTarget(tombstone oid.Address, addrs ...oid.Address) {
p.addrs = addrs
p.tombstone = &tombstone
}
// MarkAsGarbage marks an object to be physically removed from local storage.
//
// Should not be called along with WithTarget.
func (p *InhumePrm) MarkAsGarbage(addrs ...oid.Address) {
p.addrs = addrs
p.tombstone = nil
}
// WithForceRemoval inhumes objects specified via MarkAsGarbage with GC mark
// without any object restrictions checks.
func (p *InhumePrm) WithForceRemoval() {
p.forceRemoval = true
p.tombstone = nil
}
var errInhumeFailure = errors.New("inhume operation failed")
// Inhume calls metabase. Inhume method to mark an object as removed. It won't be
// removed physically from the shard until `Delete` operation.
//
// Allows inhuming non-locked objects only. Returns apistatus.ObjectLocked
// if at least one object is locked.
//
// NOTE: Marks any object as removed (despite any prohibitions on operations
// with that object) if WithForceRemoval option has been provided.
//
// Returns an error if executions are blocked (see BlockExecution).
func (e *StorageEngine) Inhume(ctx context.Context, prm InhumePrm) error {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Inhume")
defer span.End()
defer elapsed("Inhume", e.metrics.AddMethodDuration)()
return e.execIfNotBlocked(func() error {
return e.inhume(ctx, prm)
})
}
func (e *StorageEngine) inhume(ctx context.Context, prm InhumePrm) error {
addrsPerShard, notFoundObjects, err := e.groupObjectsByShard(ctx, prm.addrs, !prm.forceRemoval)
if err != nil {
return err
}
var shPrm shard.InhumePrm
if prm.forceRemoval {
shPrm.ForceRemoval()
}
for shardID, addrs := range addrsPerShard {
if prm.tombstone != nil {
shPrm.SetTarget(*prm.tombstone, addrs...)
} else {
shPrm.MarkAsGarbage(addrs...)
}
sh, exists := e.shards[shardID]
if !exists {
e.log.Warn(ctx, logs.EngineCouldNotInhumeObjectInShard,
zap.Error(errors.New("this shard was expected to exist")),
zap.String("shard_id", shardID),
)
return errInhumeFailure
}
if _, err := sh.Inhume(ctx, shPrm); err != nil {
e.reportInhumeError(ctx, err, sh)
return err
}
}
return e.inhumeNotFoundObjects(ctx, notFoundObjects, prm)
}
func (e *StorageEngine) reportInhumeError(ctx context.Context, err error, hs hashedShard) {
if err == nil {
return
}
var errLocked *apistatus.ObjectLocked
switch {
case errors.As(err, &errLocked):
case errors.Is(err, shard.ErrLockObjectRemoval):
case errors.Is(err, shard.ErrReadOnlyMode):
case errors.Is(err, shard.ErrDegradedMode):
default:
e.reportShardError(ctx, hs, "couldn't inhume object in shard", err)
}
}
// inhumeNotFoundObjects removes object which are not found on any shard.
//
// Besides an object not being found on any shard, it is also important to
// remove it anyway in order to populate the metabase indexes because they are
// responsible for the correct object status, i.e., the status will be `object
// not found` without the indexes, the status will be `object is already
// removed` with the indexes.
//
// It is suggested to evenly remove those objects on each shard with the batch
// size equal to 1 + floor(number of objects / number of shards).
func (e *StorageEngine) inhumeNotFoundObjects(ctx context.Context, addrs []oid.Address, prm InhumePrm) error {
if len(addrs) == 0 {
return nil
}
var shPrm shard.InhumePrm
if prm.forceRemoval {
shPrm.ForceRemoval()
}
numObjectsPerShard := 1 + len(addrs)/len(e.shards)
var inhumeErr error
itErr := e.iterateOverUnsortedShards(ctx, func(hs hashedShard) (stop bool) {
numObjects := min(numObjectsPerShard, len(addrs))
if numObjects == 0 {
return true
}
if prm.tombstone != nil {
shPrm.SetTarget(*prm.tombstone, addrs[:numObjects]...)
} else {
shPrm.MarkAsGarbage(addrs[:numObjects]...)
}
addrs = addrs[numObjects:]
_, inhumeErr = hs.Inhume(ctx, shPrm)
e.reportInhumeError(ctx, inhumeErr, hs)
return inhumeErr != nil
})
if inhumeErr != nil {
return inhumeErr
}
return itErr
}
// groupObjectsByShard groups objects based on the shard(s) they are stored on.
//
// If checkLocked is set, [apistatus.ObjectLocked] will be returned if any of
// the objects are locked.
//
// Returns two sets of objects: found objects which are grouped per shard and
// not found object. Not found objects are objects which are not found on any
// shard. This can happen if a node is a container node but doesn't participate
// in a replica group of the object.
func (e *StorageEngine) groupObjectsByShard(ctx context.Context, addrs []oid.Address, checkLocked bool) (groups map[string][]oid.Address, notFoundObjects []oid.Address, err error) {
groups = make(map[string][]oid.Address)
var ids []string
for _, addr := range addrs {
ids, err = e.findShards(ctx, addr, checkLocked)
if err != nil {
return
}
if len(ids) == 0 {
notFoundObjects = append(notFoundObjects, addr)
continue
}
for _, id := range ids {
groups[id] = append(groups[id], addr)
}
}
return
}
// findShards determines the shard(s) where the object is stored.
//
// If the object is a root object, multiple shards will be returned.
//
// If checkLocked is set, [apistatus.ObjectLocked] will be returned if any of
// the objects are locked.
func (e *StorageEngine) findShards(ctx context.Context, addr oid.Address, checkLocked bool) ([]string, error) {
var (
ids []string
retErr error
prm shard.ExistsPrm
siErr *objectSDK.SplitInfoError
ecErr *objectSDK.ECInfoError
isRootObject bool
objectExists bool
)
if err := e.iterateOverSortedShards(ctx, addr, func(_ int, sh hashedShard) (stop bool) {
objectExists = false
prm.Address = addr
switch res, err := sh.Exists(ctx, prm); {
case client.IsErrObjectAlreadyRemoved(err) || shard.IsErrObjectExpired(err):
// NOTE(@a-savchuk): there were some considerations that we can stop
// immediately if the object is already removed or expired. However,
// the previous method behavior was:
// - keep iterating if it's a root object and already removed,
// - stop iterating if it's not a root object and removed.
//
// Since my task was only improving method speed, let's keep the
// previous method behavior. Continue if it's a root object.
return !isRootObject
case errors.As(err, &siErr) || errors.As(err, &ecErr):
isRootObject = true
objectExists = true
case err != nil:
e.reportShardError(
ctx, sh, "couldn't check for presence in shard",
err, zap.Stringer("address", addr),
)
case res.Exists():
objectExists = true
default:
}
if checkLocked {
if isLocked, err := sh.IsLocked(ctx, addr); err != nil {
e.log.Warn(ctx, logs.EngineRemovingAnObjectWithoutFullLockingCheck,
zap.Error(err),
zap.Stringer("address", addr),
)
} else if isLocked {
retErr = new(apistatus.ObjectLocked)
return true
}
}
// This exit point must come after checking if the object is locked,
// since the locked index may be populated even if the object doesn't
// exist.
if !objectExists {
return
}
ids = append(ids, sh.ID().String())
// Continue if it's a root object.
return !isRootObject
}); err != nil {
return nil, err
}
if retErr != nil {
return nil, retErr
}
return ids, nil
}
// IsLocked checks whether an object is locked according to StorageEngine's state.
func (e *StorageEngine) IsLocked(ctx context.Context, addr oid.Address) (bool, error) {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.IsLocked",
trace.WithAttributes(
attribute.String("address", addr.EncodeToString()),
))
defer span.End()
var locked bool
var err error
var outErr error
if err := e.iterateOverUnsortedShards(ctx, func(h hashedShard) (stop bool) {
locked, err = h.IsLocked(ctx, addr)
if err != nil {
e.reportShardError(ctx, h, "can't check object's lockers", err, zap.Stringer("address", addr))
outErr = err
return false
}
return locked
}); err != nil {
return false, err
}
if locked {
return locked, nil
}
return locked, outErr
}
// GetLocks return lock id's if object is locked according to StorageEngine's state.
func (e *StorageEngine) GetLocks(ctx context.Context, addr oid.Address) ([]oid.ID, error) {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.GetLocks",
trace.WithAttributes(
attribute.String("address", addr.EncodeToString()),
))
defer span.End()
var allLocks []oid.ID
var outErr error
if err := e.iterateOverUnsortedShards(ctx, func(h hashedShard) (stop bool) {
locks, err := h.GetLocks(ctx, addr)
if err != nil {
e.reportShardError(ctx, h, logs.EngineInterruptGettingLockers, err, zap.Stringer("address", addr))
outErr = err
}
allLocks = append(allLocks, locks...)
return false
}); err != nil {
return nil, err
}
if len(allLocks) > 0 {
return allLocks, nil
}
return allLocks, outErr
}
func (e *StorageEngine) processExpiredTombstones(ctx context.Context, addrs []meta.TombstonedObject) {
if err := e.iterateOverUnsortedShards(ctx, func(sh hashedShard) (stop bool) {
sh.HandleExpiredTombstones(ctx, addrs)
select {
case <-ctx.Done():
e.log.Info(ctx, logs.EngineInterruptProcessingTheExpiredTombstones, zap.Error(ctx.Err()))
return true
default:
return false
}
}); err != nil {
e.log.Info(ctx, logs.EngineInterruptProcessingTheExpiredTombstones, zap.Error(err))
}
}
func (e *StorageEngine) processExpiredLocks(ctx context.Context, epoch uint64, lockers []oid.Address) {
if err := e.iterateOverUnsortedShards(ctx, func(sh hashedShard) (stop bool) {
sh.HandleExpiredLocks(ctx, epoch, lockers)
select {
case <-ctx.Done():
e.log.Info(ctx, logs.EngineInterruptProcessingTheExpiredLocks, zap.Error(ctx.Err()))
return true
default:
return false
}
}); err != nil {
e.log.Info(ctx, logs.EngineInterruptProcessingTheExpiredLocks, zap.Error(err))
}
}
func (e *StorageEngine) processDeletedLocks(ctx context.Context, lockers []oid.Address) {
if err := e.iterateOverUnsortedShards(ctx, func(sh hashedShard) (stop bool) {
sh.HandleDeletedLocks(ctx, lockers)
select {
case <-ctx.Done():
e.log.Info(ctx, logs.EngineInterruptProcessingTheDeletedLocks, zap.Error(ctx.Err()))
return true
default:
return false
}
}); err != nil {
e.log.Info(ctx, logs.EngineInterruptProcessingTheDeletedLocks, zap.Error(err))
}
}
func (e *StorageEngine) processZeroSizeContainers(ctx context.Context, ids []cid.ID) {
if len(ids) == 0 {
return
}
idMap, err := e.selectNonExistentIDs(ctx, ids)
if err != nil {
return
}
if len(idMap) == 0 {
return
}
var failed bool
var prm shard.ContainerSizePrm
if err := e.iterateOverUnsortedShards(ctx, func(sh hashedShard) bool {
select {
case <-ctx.Done():
e.log.Info(ctx, logs.EngineInterruptProcessingZeroSizeContainers, zap.Error(ctx.Err()))
failed = true
return true
default:
}
var drop []cid.ID
for id := range idMap {
prm.SetContainerID(id)
s, err := sh.ContainerSize(ctx, prm)
if err != nil {
e.log.Warn(ctx, logs.EngineFailedToGetContainerSize, zap.Stringer("container_id", id), zap.Error(err))
failed = true
return true
}
if s.Size() > 0 {
drop = append(drop, id)
}
}
for _, id := range drop {
delete(idMap, id)
}
return len(idMap) == 0
}); err != nil {
e.log.Info(ctx, logs.EngineInterruptProcessingZeroSizeContainers, zap.Error(err))
return
}
if failed || len(idMap) == 0 {
return
}
if err := e.iterateOverUnsortedShards(ctx, func(sh hashedShard) bool {
select {
case <-ctx.Done():
e.log.Info(ctx, logs.EngineInterruptProcessingZeroSizeContainers, zap.Error(ctx.Err()))
failed = true
return true
default:
}
for id := range idMap {
if err := sh.DeleteContainerSize(ctx, id); err != nil {
e.log.Warn(ctx, logs.EngineFailedToDeleteContainerSize, zap.Stringer("container_id", id), zap.Error(err))
failed = true
return true
}
}
return false
}); err != nil {
e.log.Info(ctx, logs.EngineInterruptProcessingZeroSizeContainers, zap.Error(err))
return
}
if failed {
return
}
for id := range idMap {
e.metrics.DeleteContainerSize(id.EncodeToString())
}
}
func (e *StorageEngine) processZeroCountContainers(ctx context.Context, ids []cid.ID) {
if len(ids) == 0 {
return
}
idMap, err := e.selectNonExistentIDs(ctx, ids)
if err != nil {
return
}
if len(idMap) == 0 {
return
}
var failed bool
var prm shard.ContainerCountPrm
if err := e.iterateOverUnsortedShards(ctx, func(sh hashedShard) bool {
select {
case <-ctx.Done():
e.log.Info(ctx, logs.EngineInterruptProcessingZeroCountContainers, zap.Error(ctx.Err()))
failed = true
return true
default:
}
var drop []cid.ID
for id := range idMap {
prm.ContainerID = id
s, err := sh.ContainerCount(ctx, prm)
if err != nil {
e.log.Warn(ctx, logs.EngineFailedToGetContainerCounters, zap.Stringer("container_id", id), zap.Error(err))
failed = true
return true
}
if s.User > 0 || s.Logic > 0 || s.Phy > 0 {
drop = append(drop, id)
}
}
for _, id := range drop {
delete(idMap, id)
}
return len(idMap) == 0
}); err != nil {
e.log.Info(ctx, logs.EngineInterruptProcessingZeroCountContainers, zap.Error(err))
return
}
if failed || len(idMap) == 0 {
return
}
if err := e.iterateOverUnsortedShards(ctx, func(sh hashedShard) bool {
select {
case <-ctx.Done():
e.log.Info(ctx, logs.EngineInterruptProcessingZeroCountContainers, zap.Error(ctx.Err()))
failed = true
return true
default:
}
for id := range idMap {
if err := sh.DeleteContainerCount(ctx, id); err != nil {
e.log.Warn(ctx, logs.EngineFailedToDeleteContainerSize, zap.Stringer("container_id", id), zap.Error(err))
failed = true
return true
}
}
return false
}); err != nil {
e.log.Info(ctx, logs.EngineInterruptProcessingZeroCountContainers, zap.Error(err))
return
}
if failed {
return
}
for id := range idMap {
e.metrics.DeleteContainerCount(id.EncodeToString())
}
}
func (e *StorageEngine) selectNonExistentIDs(ctx context.Context, ids []cid.ID) (map[cid.ID]struct{}, error) {
cs := e.containerSource.Load()
idMap := make(map[cid.ID]struct{})
for _, id := range ids {
isAvailable, err := cs.IsContainerAvailable(ctx, id)
if err != nil {
e.log.Warn(ctx, logs.EngineFailedToCheckContainerAvailability, zap.Stringer("container_id", id), zap.Error(err))
return nil, err
}
if isAvailable {
continue
}
idMap[id] = struct{}{}
}
return idMap, nil
}