WIP: Change metabase engine to pebble #1221

Closed
dstepanov-yadro wants to merge 28 commits from dstepanov-yadro/frostfs-node:feat/pebble_metabase into master
55 changed files with 3277 additions and 2930 deletions

View file

@ -1,13 +1,10 @@
package meta
import (
"time"
common "git.frostfs.info/TrueCloudLab/frostfs-node/cmd/frostfs-lens/internal"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"github.com/spf13/cobra"
"go.etcd.io/bbolt"
)
var (
@ -38,10 +35,6 @@ func init() {
func openMeta(cmd *cobra.Command) *meta.DB {
db := meta.New(
meta.WithPath(vPath),
meta.WithBoltDBOptions(&bbolt.Options{
ReadOnly: true,
Timeout: 100 * time.Millisecond,
}),
meta.WithEpochState(epochState{}),
)
common.ExitOnErr(cmd, common.Errf("could not open metabase: %w", db.Open(cmd.Context(), mode.ReadOnly)))

View file

@ -76,7 +76,6 @@ import (
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
neogoutil "github.com/nspcc-dev/neo-go/pkg/util"
"github.com/panjf2000/ants/v2"
"go.etcd.io/bbolt"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
"google.golang.org/grpc"
@ -974,11 +973,6 @@ func (c *cfg) getShardOpts(ctx context.Context, shCfg shardCfg) shardOptsWithID
mbOptions := []meta.Option{
meta.WithPath(shCfg.metaCfg.path),
meta.WithPermissions(shCfg.metaCfg.perm),
meta.WithMaxBatchSize(shCfg.metaCfg.maxBatchSize),
meta.WithMaxBatchDelay(shCfg.metaCfg.maxBatchDelay),
meta.WithBoltDBOptions(&bbolt.Options{
Timeout: 100 * time.Millisecond,
}),
meta.WithLogger(c.log),
meta.WithEpochState(c.cfgNetmap.state),
}

13
go.mod
View file

@ -16,6 +16,7 @@ require (
git.frostfs.info/TrueCloudLab/zapjournald v0.0.0-20240124114243-cb2e66427d02
github.com/cheggaaa/pb v1.0.29
github.com/chzyer/readline v1.5.1
github.com/cockroachdb/pebble v1.1.1
github.com/flynn-archive/go-shlex v0.0.0-20150515145356-3f9db97f8568
github.com/go-pkgz/expirable-cache/v3 v3.0.0
github.com/google/uuid v1.6.0
@ -55,11 +56,17 @@ require (
require (
git.frostfs.info/TrueCloudLab/rfc6979 v0.4.0 // indirect
github.com/DataDog/zstd v1.4.5 // indirect
github.com/antlr4-go/antlr/v4 v4.13.0 // indirect
github.com/beorn7/perks v1.0.1 // indirect
github.com/bits-and-blooms/bitset v1.13.0 // indirect
github.com/cenkalti/backoff/v4 v4.2.1 // indirect
github.com/cespare/xxhash/v2 v2.2.0 // indirect
github.com/cockroachdb/errors v1.11.3 // indirect
github.com/cockroachdb/fifo v0.0.0-20240606204812-0bbfbd93a7ce // indirect
github.com/cockroachdb/logtags v0.0.0-20230118201751-21c54148d20b // indirect
github.com/cockroachdb/redact v1.1.5 // indirect
github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect
github.com/consensys/bavard v0.1.13 // indirect
github.com/consensys/gnark-crypto v0.12.2-0.20231222162921-eb75782795d2 // indirect
github.com/cpuguy83/go-md2man/v2 v2.0.4 // indirect
@ -67,9 +74,11 @@ require (
github.com/davidmz/go-pageant v1.0.2 // indirect
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect
github.com/fsnotify/fsnotify v1.7.0 // indirect
github.com/getsentry/sentry-go v0.27.0 // indirect
github.com/go-fed/httpsig v1.1.0 // indirect
github.com/go-logr/logr v1.4.1 // indirect
github.com/go-logr/stdr v1.2.2 // indirect
github.com/gogo/protobuf v1.3.2 // indirect
github.com/golang/snappy v0.0.4 // indirect
github.com/gorilla/websocket v1.5.1 // indirect
github.com/grpc-ecosystem/go-grpc-middleware/providers/prometheus v1.0.0 // indirect
@ -83,6 +92,8 @@ require (
github.com/josharian/intern v1.0.0 // indirect
github.com/klauspost/cpuid/v2 v2.2.6 // indirect
github.com/klauspost/reedsolomon v1.12.1 // indirect
github.com/kr/pretty v0.3.1 // indirect
github.com/kr/text v0.2.0 // indirect
github.com/magiconair/properties v1.8.7 // indirect
github.com/mailru/easyjson v0.7.7 // indirect
github.com/mattn/go-runewidth v0.0.15 // indirect
@ -98,11 +109,13 @@ require (
github.com/nspcc-dev/neo-go/pkg/interop v0.0.0-20240521091047-78685785716d // indirect
github.com/nspcc-dev/rfc6979 v0.2.1 // indirect
github.com/pelletier/go-toml/v2 v2.2.2 // indirect
github.com/pkg/errors v0.9.1 // indirect
github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect
github.com/prometheus/client_model v0.5.0 // indirect
github.com/prometheus/common v0.48.0 // indirect
github.com/prometheus/procfs v0.12.0 // indirect
github.com/rivo/uniseg v0.4.4 // indirect
github.com/rogpeppe/go-internal v1.11.0 // indirect
github.com/russross/blackfriday/v2 v2.1.0 // indirect
github.com/spaolacci/murmur3 v1.1.0 // indirect
github.com/spf13/afero v1.11.0 // indirect

BIN
go.sum

Binary file not shown.

View file

@ -539,4 +539,10 @@ const (
PolicerCouldNotGetChunk = "could not get EC chunk"
PolicerCouldNotGetChunks = "could not get EC chunks"
AuditEventLogRecord = "audit event log record"
MetabaseCouldNotIterateOverThePrefix = "could not iterate over the prefix"
FailedToParseAddressFromKey = "failed to parse address from key"
FailedToParseOwnerFromKey = "failed to parse owner from key"
FailedToParsePayloadHashFromKey = "failed to parse payload hash from key"
FailedToParseSplitIDFromKey = "failed to parse splitID from key"
FailedToParseAttributeValueFromKey = "failed to parse attribute value from key"
)

View file

@ -44,9 +44,9 @@ func (r ListContainersRes) Containers() []cid.ID {
// ContainerSize returns the sum of estimation container sizes among all shards.
//
// Returns an error if executions are blocked (see BlockExecution).
func (e *StorageEngine) ContainerSize(prm ContainerSizePrm) (res ContainerSizeRes, err error) {
func (e *StorageEngine) ContainerSize(ctx context.Context, prm ContainerSizePrm) (res ContainerSizeRes, err error) {
err = e.execIfNotBlocked(func() error {
res, err = e.containerSize(prm)
res, err = e.containerSize(ctx, prm)
return err
})
@ -54,12 +54,12 @@ func (e *StorageEngine) ContainerSize(prm ContainerSizePrm) (res ContainerSizeRe
}
// ContainerSize calls ContainerSize method on engine to calculate sum of estimation container sizes among all shards.
func ContainerSize(e *StorageEngine, id cid.ID) (uint64, error) {
func ContainerSize(ctx context.Context, e *StorageEngine, id cid.ID) (uint64, error) {
var prm ContainerSizePrm
prm.SetContainerID(id)
res, err := e.ContainerSize(prm)
res, err := e.ContainerSize(ctx, prm)
if err != nil {
return 0, err
}
@ -67,7 +67,7 @@ func ContainerSize(e *StorageEngine, id cid.ID) (uint64, error) {
return res.Size(), nil
}
func (e *StorageEngine) containerSize(prm ContainerSizePrm) (res ContainerSizeRes, err error) {
func (e *StorageEngine) containerSize(ctx context.Context, prm ContainerSizePrm) (res ContainerSizeRes, err error) {
if e.metrics != nil {
defer elapsed("EstimateContainerSize", e.metrics.AddMethodDuration)()
}
@ -76,7 +76,7 @@ func (e *StorageEngine) containerSize(prm ContainerSizePrm) (res ContainerSizeRe
var csPrm shard.ContainerSizePrm
csPrm.SetContainerID(prm.cnr)
csRes, err := sh.Shard.ContainerSize(csPrm)
csRes, err := sh.Shard.ContainerSize(ctx, csPrm)
if err != nil {
e.reportShardError(sh, "can't get container size", err,
zap.Stringer("container_id", prm.cnr))

View file

@ -10,7 +10,6 @@ import (
"strconv"
"sync/atomic"
"testing"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor"
@ -24,7 +23,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger/test"
cidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id/test"
"github.com/stretchr/testify/require"
"go.etcd.io/bbolt"
)
// TestInitializationFailure checks that shard is initialized and closed even if media
@ -53,10 +51,6 @@ func TestInitializationFailure(t *testing.T) {
shard.WithBlobStorOptions(
blobstor.WithStorages(storages)),
shard.WithMetaBaseOptions(
meta.WithBoltDBOptions(&bbolt.Options{
Timeout: 100 * time.Millisecond,
OpenFile: opts.openFileMetabase,
}),
meta.WithPath(filepath.Join(t.TempDir(), "metabase")),
meta.WithPermissions(0o700),
meta.WithEpochState(epochState{})),
@ -83,6 +77,7 @@ func TestInitializationFailure(t *testing.T) {
testEngineFailInitAndReload(t, false, shardOpts, beforeReload)
})
t.Run("metabase", func(t *testing.T) {
t.Skip("will be implemented correctly later")
var openFileMetabaseSucceed atomic.Bool
openFileMetabase := func(p string, f int, mode fs.FileMode) (*os.File, error) {
if openFileMetabaseSucceed.Load() {

View file

@ -277,7 +277,7 @@ func (e *StorageEngine) processExpiredLocks(ctx context.Context, epoch uint64, l
func (e *StorageEngine) processDeletedLocks(ctx context.Context, lockers []oid.Address) {
e.iterateOverUnsortedShards(func(sh hashedShard) (stop bool) {
sh.HandleDeletedLocks(lockers)
sh.HandleDeletedLocks(ctx, lockers)
select {
case <-ctx.Done():
@ -317,7 +317,7 @@ func (e *StorageEngine) processZeroSizeContainers(ctx context.Context, ids []cid
var drop []cid.ID
for id := range idMap {
prm.SetContainerID(id)
s, err := sh.ContainerSize(prm)
s, err := sh.ContainerSize(ctx, prm)
if err != nil {
e.log.Warn(logs.EngineFailedToGetContainerSize, zap.Stringer("container_id", id), zap.Error(err))
failed = true

View file

@ -7,7 +7,7 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
@ -37,34 +37,30 @@ func (db *DB) GetChildren(ctx context.Context, addresses []oid.Address) (map[oid
}
result := make(map[oid.Address][]oid.Address, len(addresses))
buffer := make([]byte, bucketKeySize)
err := db.boltDB.View(func(tx *bbolt.Tx) error {
err := db.snapshot(func(s *pebble.Snapshot) error {
for _, addr := range addresses {
if _, found := result[addr]; found {
continue
}
result[addr] = []oid.Address{}
bkt := tx.Bucket(parentBucketName(addr.Container(), buffer))
if bkt == nil {
continue
}
binObjIDs, err := decodeList(bkt.Get(objectKey(addr.Object(), buffer)))
if err != nil {
return err
}
for _, binObjID := range binObjIDs {
var id oid.ID
if err = id.Decode(binObjID); err != nil {
for {
keys, err := selectByPrefixBatch(ctx, s, parentKeyLongPrefix(addr.Container(), addr.Object()), batchSize)
if err != nil {
return err
}
var resultAddress oid.Address
resultAddress.SetContainer(addr.Container())
resultAddress.SetObject(id)
result[addr] = append(result[addr], resultAddress)
for _, key := range keys {
resultAddress, err := addressOfTargetFromParentKey(key)
if err != nil {
return err
}
result[addr] = append(result[addr], resultAddress)
}
if len(keys) < batchSize {
break
}
}
}
return nil

View file

@ -0,0 +1,51 @@
package meta
import (
"bytes"
"slices"
utilSync "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/sync"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
)
type concurrency struct {
containerLocks *utilSync.KeyLocker[cid.ID]
}
func newConcurrency() *concurrency {
return &concurrency{
containerLocks: utilSync.NewKeyLocker[cid.ID](),
}
}
func (c *concurrency) LockContainerID(id cid.ID) func() {
c.containerLocks.Lock(id)
return func() {
c.containerLocks.Unlock(id)
}
}
func (c *concurrency) LockContainerIDs(ids []cid.ID) func() {
var containerIDs []cid.ID
m := make(map[cid.ID]struct{})
for _, id := range ids {
m[id] = struct{}{}
}
for id := range m {
containerIDs = append(containerIDs, id)
}
slices.SortFunc(containerIDs, func(lhs, rhs cid.ID) int {
return bytes.Compare(lhs[:], rhs[:])
})
for _, id := range containerIDs {
c.containerLocks.Lock(id)
}
return func() {
for idx := range containerIDs {
c.containerLocks.Unlock(containerIDs[len(containerIDs)-idx-1])
}
}
}

View file

@ -1,14 +1,23 @@
package meta
import (
"bytes"
"context"
"encoding/binary"
"errors"
"fmt"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
const (
containerSizeKeySize = 1 + cidSize
)
func (db *DB) Containers(ctx context.Context) (list []cid.ID, err error) {
@ -30,8 +39,8 @@ func (db *DB) Containers(ctx context.Context) (list []cid.ID, err error) {
return nil, ErrDegradedMode
}
err = db.boltDB.View(func(tx *bbolt.Tx) error {
list, err = db.containers(tx)
err = db.snapshot(func(s *pebble.Snapshot) error {
list, err = containers(ctx, s)
return err
})
@ -39,50 +48,34 @@ func (db *DB) Containers(ctx context.Context) (list []cid.ID, err error) {
return list, metaerr.Wrap(err)
}
func (db *DB) containers(tx *bbolt.Tx) ([]cid.ID, error) {
func containers(ctx context.Context, r pebble.Reader) ([]cid.ID, error) {
result := make([]cid.ID, 0)
unique := make(map[string]struct{})
var cnr cid.ID
err := tx.ForEach(func(name []byte, _ *bbolt.Bucket) error {
if parseContainerID(&cnr, name, unique) {
result = append(result, cnr)
unique[string(name[1:bucketKeySize])] = struct{}{}
}
return nil
})
return result, err
}
func (db *DB) ContainerSize(id cid.ID) (size uint64, err error) {
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return 0, ErrDegradedMode
it, err := r.NewIter(&pebble.IterOptions{})
if err != nil {
return nil, err
}
err = db.boltDB.View(func(tx *bbolt.Tx) error {
size, err = db.containerSize(tx, id)
for v := it.First(); v; v = it.Next() {
select {
case <-ctx.Done():
return nil, errors.Join(ctx.Err(), it.Close())
default:
}
return err
})
if parseContainerIDWithIgnore(&cnr, it.Key(), unique) {
result = append(result, cnr)
unique[string(it.Key()[1:containerSizeKeySize])] = struct{}{}
}
}
return size, metaerr.Wrap(err)
return result, it.Close()
}
func (db *DB) containerSize(tx *bbolt.Tx, id cid.ID) (uint64, error) {
containerVolume := tx.Bucket(containerVolumeBucketName)
key := make([]byte, cidSize)
id.Encode(key)
return parseContainerSize(containerVolume.Get(key)), nil
}
func parseContainerID(dst *cid.ID, name []byte, ignore map[string]struct{}) bool {
if len(name) != bucketKeySize {
func parseContainerIDWithIgnore(dst *cid.ID, name []byte, ignore map[string]struct{}) bool {
if len(name) < bucketKeySize {
return false
}
if _, ok := ignore[string(name[1:bucketKeySize])]; ok {
@ -91,31 +84,200 @@ func parseContainerID(dst *cid.ID, name []byte, ignore map[string]struct{}) bool
return dst.Decode(name[1:bucketKeySize]) == nil
}
func parseContainerSize(v []byte) uint64 {
func (db *DB) ContainerSize(ctx context.Context, id cid.ID) (size uint64, err error) {
_, span := tracing.StartSpanFromContext(ctx, "metabase.ContainerSize")
defer span.End()
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return 0, ErrDegradedMode
}
err = db.snapshot(func(s *pebble.Snapshot) error {
val, err := valueSafe(s, containerSizeKey(id))
if err != nil {
return err
}
if len(val) == 0 {
return nil
}
value, ok := parseSize(val)
if !ok || value < 0 {
return fmt.Errorf("invalid container size value for container %s", id)
}
size = uint64(value)
return nil
})
if err != nil {
return 0, metaerr.Wrap(err)
}
return size, nil
}
func (db *DB) ContainerSizes(ctx context.Context) (map[cid.ID]uint64, error) {
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return nil, ErrDegradedMode
}
return db.containerSizesInternal(ctx)
}
// ZeroSizeContainers returns containers with size = 0.
func (db *DB) ZeroSizeContainers(ctx context.Context) ([]cid.ID, error) {
var (
startedAt = time.Now()
success = false
)
defer func() {
db.metrics.AddMethodDuration("ZeroSizeContainers", time.Since(startedAt), success)
}()
ctx, span := tracing.StartSpanFromContext(ctx, "metabase.ZeroSizeContainers")
defer span.End()
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
sizes, err := db.containerSizesInternal(ctx)
if err != nil {
return nil, err
}
var result []cid.ID
for id, size := range sizes {
if size == 0 {
result = append(result, id)
}
}
return result, nil
}
func (db *DB) DeleteContainerSize(ctx context.Context, id cid.ID) error {
var (
startedAt = time.Now()
success = false
)
defer func() {
db.metrics.AddMethodDuration("DeleteContainerSize", time.Since(startedAt), success)
}()
_, span := tracing.StartSpanFromContext(ctx, "metabase.DeleteContainerSize",
trace.WithAttributes(
attribute.Stringer("container_id", id),
))
defer span.End()
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return ErrDegradedMode
}
if db.mode.ReadOnly() {
return ErrReadOnlyMode
}
defer db.guard.LockContainerID(id)()
return metaerr.Wrap(db.batch(
func(b *pebble.Batch) error {
return b.Delete(containerSizeKey(id), pebble.Sync)
}))
}
func (db *DB) containerSizesInternal(ctx context.Context) (map[cid.ID]uint64, error) {
prefix := []byte{containerSizePrefix}
result := make(map[cid.ID]int64)
err := db.snapshot(func(s *pebble.Snapshot) error {
it, err := s.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return err
}
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return errors.Join(ctx.Err(), it.Close())
default:
}
key := it.Key()
var cnr cid.ID
if err := cnr.Decode(key[1:containerSizeKeySize]); err != nil {
return errors.Join(fmt.Errorf("invalid container size key: %w", err), it.Close())
}
value, ok := parseSize(it.Value())
if !ok {
return errors.Join(fmt.Errorf("invalid container size value for container %s", cnr), it.Close())
}
result[cnr] += value
}
return it.Close()
})
if err != nil {
return nil, metaerr.Wrap(err)
}
return normilizeContainerSizes(result)
}
func normilizeContainerSizes(sizes map[cid.ID]int64) (map[cid.ID]uint64, error) {
result := make(map[cid.ID]uint64, len(sizes))
for k, v := range sizes {
if v < 0 {
return nil, fmt.Errorf("invalid cumulative size for container %s", k)
}
result[k] = uint64(v)
}
return result, nil
}
func changeContainerSize(b *pebble.Batch, id cid.ID, delta int64) error {
key := containerSizeKey(id)
v, err := valueSafe(b, key)
if err != nil {
return err
}
size, ok := parseSize(v)
if !ok {
return fmt.Errorf("invalid container size value for container %s", id)
}
size += delta
value := marshalSize(size)
return b.Set(key, value, pebble.Sync)
}
// containerSizeKey returns containerVolumePrefix_CID_bucketID key.
func containerSizeKey(cnr cid.ID) []byte {
result := make([]byte, containerSizeKeySize)
result[0] = containerSizePrefix
cnr.Encode(result[1:])
return result
}
func parseSize(v []byte) (int64, bool) {
if len(v) == 0 {
return 0
return 0, true
}
return binary.LittleEndian.Uint64(v)
if len(v) != 8 {
return 0, false
}
return int64(binary.LittleEndian.Uint64(v)), true
}
func changeContainerSize(tx *bbolt.Tx, id cid.ID, delta uint64, increase bool) error {
containerVolume := tx.Bucket(containerVolumeBucketName)
key := make([]byte, cidSize)
id.Encode(key)
size := parseContainerSize(containerVolume.Get(key))
if increase {
size += delta
} else if size > delta {
size -= delta
} else {
size = 0
}
buf := make([]byte, 8) // consider using sync.Pool to decrease allocations
binary.LittleEndian.PutUint64(buf, size)
return containerVolume.Put(key, buf)
func marshalSize(v int64) []byte {
buf := make([]byte, 8)
binary.LittleEndian.PutUint64(buf, uint64(v))
return buf
}

View file

@ -151,7 +151,7 @@ func TestDB_ContainerSize(t *testing.T) {
}
for cnr, volume := range cids {
n, err := db.ContainerSize(cnr)
n, err := db.ContainerSize(context.Background(), cnr)
require.NoError(t, err)
require.Equal(t, volume, int(n))
}
@ -169,7 +169,7 @@ func TestDB_ContainerSize(t *testing.T) {
volume -= int(obj.PayloadSize())
n, err := db.ContainerSize(cnr)
n, err := db.ContainerSize(context.Background(), cnr)
require.NoError(t, err)
require.Equal(t, volume, int(n))
}

View file

@ -2,8 +2,8 @@ package meta
import (
"context"
"errors"
"fmt"
"os"
"path/filepath"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
@ -11,7 +11,7 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.uber.org/zap"
)
@ -21,23 +21,7 @@ var ErrDegradedMode = logicerr.New("metabase is in a degraded mode")
// ErrReadOnlyMode is returned when metabase is in a read-only mode.
var ErrReadOnlyMode = logicerr.New("metabase is in a read-only mode")
var (
mStaticBuckets = map[string]struct{}{
string(containerVolumeBucketName): {},
string(containerCounterBucketName): {},
string(graveyardBucketName): {},
string(garbageBucketName): {},
string(shardInfoBucket): {},
string(bucketNameLocked): {},
}
// deprecatedBuckets buckets that are not used anymore.
deprecatedBuckets = [][]byte{
toMoveItBucketName,
}
)
// Open boltDB instance for metabase.
// Open metabase.
func (db *DB) Open(_ context.Context, m mode.Mode) error {
db.modeMtx.Lock()
defer db.modeMtx.Unlock()
@ -47,147 +31,123 @@ func (db *DB) Open(_ context.Context, m mode.Mode) error {
if m.NoMetabase() {
return nil
}
return db.openDB(m)
}
func (db *DB) openDB(mode mode.Mode) error {
err := util.MkdirAllX(filepath.Dir(db.info.Path), db.info.Permission)
err := util.MkdirAllX(db.info.Path, db.info.Permission)
if err != nil {
return fmt.Errorf("can't create dir %s for metabase: %w", db.info.Path, err)
}
db.log.Debug(logs.MetabaseCreatedDirectoryForMetabase, zap.String("path", db.info.Path))
if db.boltOptions == nil {
opts := *bbolt.DefaultOptions
db.boltOptions = &opts
}
db.boltOptions.ReadOnly = mode.ReadOnly()
return metaerr.Wrap(db.openBolt())
return metaerr.Wrap(db.openDatabase(mode.ReadOnly()))
}
func (db *DB) openBolt() error {
var err error
db.boltDB, err = bbolt.Open(db.info.Path, db.info.Permission, db.boltOptions)
if err != nil {
return fmt.Errorf("can't open boltDB database: %w", err)
func (db *DB) pebbleOptions(readOnly bool) *pebble.Options {
opts := &pebble.Options{
ReadOnly: readOnly,
}
db.boltDB.MaxBatchDelay = db.boltBatchDelay
db.boltDB.MaxBatchSize = db.boltBatchSize
return opts
}
db.log.Debug(logs.MetabaseOpenedBoltDBInstanceForMetabase)
func (db *DB) openDatabase(readOnly bool) error {
opts := db.pebbleOptions(readOnly)
db.log.Debug(logs.MetabaseCheckingMetabaseVersion)
return db.boltDB.View(func(tx *bbolt.Tx) error {
// The safest way to check if the metabase is fresh is to check if it has no buckets.
// However, shard info can be present. So here we check that the number of buckets is
// at most 1.
// Another thing to consider is that tests do not persist shard ID, we want to support
// this case too.
var n int
err := tx.ForEach(func([]byte, *bbolt.Bucket) error {
if n++; n >= 2 { // do not iterate a lot
return errBreakBucketForEach
}
return nil
})
var err error
db.database, err = pebble.Open(db.info.Path, opts)
if err != nil {
return fmt.Errorf("can't open badger database: %w", err)
}
if err == errBreakBucketForEach {
db.initialized = true
err = nil
return db.snapshot(func(s *pebble.Snapshot) error {
data, err := valueSafe(s, shardInfoKey(versionKey))
if err != nil {
return err
}
return err
db.initialized = len(data) > 0
return nil
})
}
// Init initializes metabase. It creates static (CID-independent) buckets in underlying BoltDB instance.
// Init initializes metabase.
//
// Returns ErrOutdatedVersion if a database at the provided path is outdated.
//
// Does nothing if metabase has already been initialized and filled. To roll back the database to its initial state,
// use Reset.
func (db *DB) Init() error {
return metaerr.Wrap(db.init(false))
db.modeMtx.Lock()
defer db.modeMtx.Unlock()
return metaerr.Wrap(db.init(context.TODO(), false))
}
// Reset resets metabase. Works similar to Init but cleans up all static buckets and
// removes all dynamic (CID-dependent) ones in non-blank BoltDB instances.
func (db *DB) Reset() error {
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
func (db *DB) Init2(ctx context.Context) error {
db.modeMtx.Lock()
defer db.modeMtx.Unlock()
return metaerr.Wrap(db.init(ctx, false))
}
// Reset resets metabase. Works similar to Init but cleans up all data records.
func (db *DB) Reset(ctx context.Context) error {
db.modeMtx.Lock()
defer db.modeMtx.Unlock()
if db.mode.NoMetabase() {
return ErrDegradedMode
}
return metaerr.Wrap(db.init(true))
return metaerr.Wrap(db.init(ctx, true))
}
func (db *DB) init(reset bool) error {
func (db *DB) init(ctx context.Context, reset bool) error {
if db.mode.NoMetabase() || db.mode.ReadOnly() {
return nil
}
return db.boltDB.Update(func(tx *bbolt.Tx) error {
var err error
if !reset {
// Normal open, check version and update if not initialized.
err := checkVersion(tx, db.initialized)
if err != nil {
return err
}
if reset {
if err := db.reset(); err != nil {
return err
}
for k := range mStaticBuckets {
name := []byte(k)
if reset {
err := tx.DeleteBucket(name)
if err != nil && !errors.Is(err, bbolt.ErrBucketNotFound) {
return fmt.Errorf("could not delete static bucket %s: %w", k, err)
}
}
}
_, err := tx.CreateBucketIfNotExists(name)
if err != nil {
return fmt.Errorf("could not create static bucket %s: %w", k, err)
}
return db.batch(func(b *pebble.Batch) error {
err := checkVersion(b, db.initialized)
if err != nil {
return err
}
err = syncCounter(ctx, b, false)
if err != nil {
return fmt.Errorf("could not sync object counter: %w", err)
}
for _, b := range deprecatedBuckets {
err := tx.DeleteBucket(b)
if err != nil && !errors.Is(err, bbolt.ErrBucketNotFound) {
return fmt.Errorf("could not delete deprecated bucket %s: %w", string(b), err)
}
}
return nil
})
}
if !reset { // counters will be recalculated by refill metabase
err = syncCounter(tx, false)
if err != nil {
return fmt.Errorf("could not sync object counter: %w", err)
}
return nil
}
bucketCursor := tx.Cursor()
name, _ := bucketCursor.First()
for name != nil {
if _, ok := mStaticBuckets[string(name)]; !ok {
if err := tx.DeleteBucket(name); err != nil {
return err
}
name, _ = bucketCursor.Seek(name)
continue
}
name, _ = bucketCursor.Next()
}
return updateVersion(tx, version)
func (db *DB) reset() error {
if err := db.database.Close(); err != nil {
return err
}
db.database = nil
if err := os.RemoveAll(db.info.Path); err != nil {
return err
}
var err error
db.database, err = pebble.Open(db.info.Path, db.pebbleOptions(false))
if err != nil {
return fmt.Errorf("can't open badger database: %w", err)
}
return db.batch(func(b *pebble.Batch) error {
return updateVersion(b, version)
})
}
// SyncCounters forces to synchronize the object counters.
func (db *DB) SyncCounters() error {
func (db *DB) SyncCounters(ctx context.Context) error {
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
@ -197,26 +157,29 @@ func (db *DB) SyncCounters() error {
return ErrReadOnlyMode
}
return metaerr.Wrap(db.boltDB.Update(func(tx *bbolt.Tx) error {
return syncCounter(tx, true)
return metaerr.Wrap(db.batch(func(b *pebble.Batch) error {
return syncCounter(ctx, b, true)
}))
}
// Close closes boltDB instance
// and reports metabase metric.
// Close closes metabase.
func (db *DB) Close() error {
var err error
if db.boltDB != nil {
err = db.close()
}
if err == nil {
db.metrics.Close()
}
return err
db.modeMtx.Lock()
defer db.modeMtx.Unlock()
return db.close()
}
func (db *DB) close() error {
return metaerr.Wrap(db.boltDB.Close())
var err error
if db.database != nil {
err = metaerr.Wrap(db.database.Close())
}
if err == nil {
db.database = nil
db.metrics.Close()
}
return err
}
// Reload reloads part of the configuration.
@ -235,14 +198,14 @@ func (db *DB) Reload(opts ...Option) (bool, error) {
defer db.modeMtx.Unlock()
if db.mode.NoMetabase() || c.info.Path != "" && filepath.Clean(db.info.Path) != filepath.Clean(c.info.Path) {
if err := db.Close(); err != nil {
if err := db.close(); err != nil {
return false, err
}
db.mode = mode.Disabled
db.metrics.SetMode(mode.ComponentDisabled)
db.info.Path = c.info.Path
if err := db.openBolt(); err != nil {
if err := db.openDatabase(false); err != nil {
return false, metaerr.Wrap(fmt.Errorf("%w: %v", ErrDegradedMode, err))
}

View file

@ -17,7 +17,7 @@ func TestReset(t *testing.T) {
db := newDB(t)
defer func() { require.NoError(t, db.Close()) }()
err := db.Reset()
err := db.Reset(context.Background())
require.NoError(t, err)
obj := testutil.GenerateObject()
@ -47,7 +47,7 @@ func TestReset(t *testing.T) {
assertExists(addr, true, nil)
assertExists(addrToInhume, false, client.IsErrObjectAlreadyRemoved)
err = db.Reset()
err = db.Reset(context.Background())
require.NoError(t, err)
assertExists(addr, false, nil)

View file

@ -13,29 +13,20 @@ import (
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.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
var (
objectPhyCounterKey = []byte("phy_counter")
objectLogicCounterKey = []byte("logic_counter")
objectUserCounterKey = []byte("user_counter")
errInvalidKeyLenght = errors.New("invalid key length")
errInvalidKeyPrefix = errors.New("invalid key prefix")
errInvalidValueLenght = errors.New("invalid value length")
errInvalidContainerIDValue = errors.New("invalid container ID value")
)
var (
errInvalidKeyLenght = errors.New("invalid key length")
errInvalidValueLenght = errors.New("invalid value length")
)
type objectType uint8
const (
_ objectType = iota
phy
logical
user
containerObjectCountKeySize = 1 + cidSize
)
// ObjectCounters groups object counter
@ -50,12 +41,18 @@ func (o ObjectCounters) IsZero() bool {
return o.Phy == 0 && o.Logic == 0 && o.User == 0
}
type objectCounterValue struct {
Logic int64
Phy int64
User int64
}
// ObjectCounters returns object counters that metabase has
// tracked since it was opened and initialized.
//
// Returns only the errors that do not allow reading counter
// in Bolt database.
func (db *DB) ObjectCounters() (cc ObjectCounters, err error) {
// in badger database.
func (db *DB) ObjectCounters(ctx context.Context) (ObjectCounters, error) {
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
@ -63,29 +60,22 @@ func (db *DB) ObjectCounters() (cc ObjectCounters, err error) {
return ObjectCounters{}, ErrDegradedMode
}
err = db.boltDB.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(shardInfoBucket)
if b != nil {
data := b.Get(objectPhyCounterKey)
if len(data) == 8 {
cc.Phy = binary.LittleEndian.Uint64(data)
}
data = b.Get(objectLogicCounterKey)
if len(data) == 8 {
cc.Logic = binary.LittleEndian.Uint64(data)
}
data = b.Get(objectUserCounterKey)
if len(data) == 8 {
cc.User = binary.LittleEndian.Uint64(data)
}
}
return nil
var cc map[cid.ID]ObjectCounters
err := db.snapshot(func(s *pebble.Snapshot) error {
var err error
cc, err = containerObjectCounters(ctx, s)
return err
})
return cc, metaerr.Wrap(err)
if err != nil {
return ObjectCounters{}, metaerr.Wrap(err)
}
var result ObjectCounters
for _, v := range cc {
result.Logic += v.Logic
result.Phy += v.Phy
result.User += v.User
}
return result, nil
}
type ContainerCounters struct {
@ -96,7 +86,7 @@ type ContainerCounters struct {
// that metabase has tracked since it was opened and initialized.
//
// Returns only the errors that do not allow reading counter
// in Bolt database.
// in badger database.
//
// It is guaranteed that the ContainerCounters fields are not nil.
func (db *DB) ContainerCounters(ctx context.Context) (ContainerCounters, error) {
@ -114,84 +104,16 @@ func (db *DB) ContainerCounters(ctx context.Context) (ContainerCounters, error)
cc := ContainerCounters{
Counts: make(map[cid.ID]ObjectCounters),
}
lastKey := make([]byte, cidSize)
// there is no limit for containers count, so use batching with cancellation
for {
select {
case <-ctx.Done():
return cc, ctx.Err()
default:
}
completed, err := db.containerCountersNextBatch(lastKey, func(id cid.ID, entity ObjectCounters) {
cc.Counts[id] = entity
})
if err != nil {
return cc, err
}
if completed {
break
}
}
success = true
return cc, nil
}
func (db *DB) containerCountersNextBatch(lastKey []byte, f func(id cid.ID, entity ObjectCounters)) (bool, error) {
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return false, ErrDegradedMode
}
counter := 0
const batchSize = 1000
err := db.boltDB.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(containerCounterBucketName)
if b == nil {
return ErrInterruptIterator
}
c := b.Cursor()
var key, value []byte
for key, value = c.Seek(lastKey); key != nil; key, value = c.Next() {
if bytes.Equal(lastKey, key) {
continue
}
copy(lastKey, key)
cnrID, err := parseContainerCounterKey(key)
if err != nil {
return err
}
ent, err := parseContainerCounterValue(value)
if err != nil {
return err
}
f(cnrID, ent)
counter++
if counter == batchSize {
break
}
}
if counter < batchSize { // last batch
return ErrInterruptIterator
}
return nil
err := db.snapshot(func(s *pebble.Snapshot) error {
var err error
cc.Counts, err = containerObjectCounters(ctx, s)
return err
})
if err != nil {
if errors.Is(err, ErrInterruptIterator) {
return true, nil
}
return false, metaerr.Wrap(err)
return ContainerCounters{}, metaerr.Wrap(err)
}
return false, nil
success = true
return cc, nil
}
func (db *DB) ContainerCount(ctx context.Context, id cid.ID) (ObjectCounters, error) {
@ -213,144 +135,78 @@ func (db *DB) ContainerCount(ctx context.Context, id cid.ID) (ObjectCounters, er
return ObjectCounters{}, ErrDegradedMode
}
var result ObjectCounters
err := db.boltDB.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(containerCounterBucketName)
key := make([]byte, cidSize)
id.Encode(key)
v := b.Get(key)
if v == nil {
var res ObjectCounters
err := db.snapshot(func(s *pebble.Snapshot) error {
val, err := valueSafe(s, containerCounterKey(id))
if err != nil {
return err
}
if len(val) == 0 {
return nil
}
var err error
result, err = parseContainerCounterValue(v)
return err
})
oc, err := parseContainerCounterValue(val)
if err != nil {
return err
}
return result, metaerr.Wrap(err)
if oc.Logic < 0 || oc.Phy < 0 || oc.User < 0 {
return fmt.Errorf("invalid container object counter for container ID %s", id.EncodeToString())
}
res.Logic = uint64(oc.Logic)
res.Phy = uint64(oc.Phy)
res.User = uint64(oc.User)
return nil
})
if err != nil {
return ObjectCounters{}, metaerr.Wrap(err)
}
return res, nil
}
func (db *DB) incCounters(tx *bbolt.Tx, cnrID cid.ID, isUserObject bool) error {
b := tx.Bucket(shardInfoBucket)
if b == nil {
return db.incContainerObjectCounter(tx, cnrID, isUserObject)
}
func containerCounterKey(cnrID cid.ID) []byte {
result := make([]byte, containerObjectCountKeySize)
result[0] = containerCountersPrefix
cnrID.Encode(result[1:])
return result
}
if err := db.updateShardObjectCounterBucket(b, phy, 1, true); err != nil {
return fmt.Errorf("could not increase phy object counter: %w", err)
}
if err := db.updateShardObjectCounterBucket(b, logical, 1, true); err != nil {
return fmt.Errorf("could not increase logical object counter: %w", err)
func incCounters(b *pebble.Batch, cnrID cid.ID, isUserObject bool) error {
delta := objectCounterValue{
Logic: 1,
Phy: 1,
}
if isUserObject {
if err := db.updateShardObjectCounterBucket(b, user, 1, true); err != nil {
return fmt.Errorf("could not increase user object counter: %w", err)
}
delta.User = 1
}
return db.incContainerObjectCounter(tx, cnrID, isUserObject)
return editContainerCounterValue(b, cnrID, delta)
}
func (db *DB) updateShardObjectCounter(tx *bbolt.Tx, typ objectType, delta uint64, inc bool) error {
b := tx.Bucket(shardInfoBucket)
if b == nil {
return nil
}
return db.updateShardObjectCounterBucket(b, typ, delta, inc)
}
func (*DB) updateShardObjectCounterBucket(b *bbolt.Bucket, typ objectType, delta uint64, inc bool) error {
var counter uint64
var counterKey []byte
switch typ {
case phy:
counterKey = objectPhyCounterKey
case logical:
counterKey = objectLogicCounterKey
case user:
counterKey = objectUserCounterKey
default:
panic("unknown object type counter")
}
data := b.Get(counterKey)
if len(data) == 8 {
counter = binary.LittleEndian.Uint64(data)
}
if inc {
counter += delta
} else if counter <= delta {
counter = 0
} else {
counter -= delta
}
newCounter := make([]byte, 8)
binary.LittleEndian.PutUint64(newCounter, counter)
return b.Put(counterKey, newCounter)
}
func (db *DB) updateContainerCounter(tx *bbolt.Tx, delta map[cid.ID]ObjectCounters, inc bool) error {
b := tx.Bucket(containerCounterBucketName)
if b == nil {
return nil
}
key := make([]byte, cidSize)
func updateContainerCounter(b *pebble.Batch, delta map[cid.ID]objectCounterValue) error {
for cnrID, cnrDelta := range delta {
cnrID.Encode(key)
if err := db.editContainerCounterValue(b, key, cnrDelta, inc); err != nil {
if err := editContainerCounterValue(b, cnrID, cnrDelta); err != nil {
return err
}
}
return nil
}
func (*DB) editContainerCounterValue(b *bbolt.Bucket, key []byte, delta ObjectCounters, inc bool) error {
var entity ObjectCounters
var err error
data := b.Get(key)
if len(data) > 0 {
entity, err = parseContainerCounterValue(data)
func editContainerCounterValue(b *pebble.Batch, cnrID cid.ID, delta objectCounterValue) error {
key := containerCounterKey(cnrID)
val, err := valueSafe(b, key)
if err != nil {
return err
}
setValue := delta
if val != nil {
exited, err := parseContainerCounterValue(val)
if err != nil {
return err
}
setValue = mergeObjectCounterValues(setValue, exited)
}
entity.Phy = nextValue(entity.Phy, delta.Phy, inc)
entity.Logic = nextValue(entity.Logic, delta.Logic, inc)
entity.User = nextValue(entity.User, delta.User, inc)
value := containerCounterValue(entity)
return b.Put(key, value)
}
func nextValue(existed, delta uint64, inc bool) uint64 {
if inc {
existed += delta
} else if existed <= delta {
existed = 0
} else {
existed -= delta
}
return existed
}
func (db *DB) incContainerObjectCounter(tx *bbolt.Tx, cnrID cid.ID, isUserObject bool) error {
b := tx.Bucket(containerCounterBucketName)
if b == nil {
return nil
}
key := make([]byte, cidSize)
cnrID.Encode(key)
c := ObjectCounters{Logic: 1, Phy: 1}
if isUserObject {
c.User = 1
}
return db.editContainerCounterValue(b, key, c, true)
return b.Set(key, marshalContainerCounterValue(setValue), pebble.Sync)
}
// syncCounter updates object counters according to metabase state:
@ -359,34 +215,31 @@ func (db *DB) incContainerObjectCounter(tx *bbolt.Tx, cnrID cid.ID, isUserObject
//
// Does nothing if counters are not empty and force is false. If force is
// true, updates the counters anyway.
func syncCounter(tx *bbolt.Tx, force bool) error {
shardInfoB, err := createBucketLikelyExists(tx, shardInfoBucket)
if err != nil {
return fmt.Errorf("could not get shard info bucket: %w", err)
}
shardObjectCounterInitialized := len(shardInfoB.Get(objectPhyCounterKey)) == 8 &&
len(shardInfoB.Get(objectLogicCounterKey)) == 8 &&
len(shardInfoB.Get(objectUserCounterKey)) == 8
containerObjectCounterInitialized := containerObjectCounterInitialized(tx)
if !force && shardObjectCounterInitialized && containerObjectCounterInitialized {
// the counters are already inited
func syncCounter(ctx context.Context, b *pebble.Batch, force bool) error {
if !force && containerObjectCounterInitialized(ctx, b) {
return nil
}
containerCounterB, err := createBucketLikelyExists(tx, containerCounterBucketName)
// drop existed counters
err := deleteByPrefix(ctx, b, []byte{containerCountersPrefix})
if err != nil {
return fmt.Errorf("could not get container counter bucket: %w", err)
return err
}
counters, err := getActualObjectCounters(b)
if err != nil {
return err
}
return setObjectCounters(b, counters)
}
func getActualObjectCounters(r pebble.Reader) (map[cid.ID]ObjectCounters, error) {
var addr oid.Address
var isAvailable bool
counters := make(map[cid.ID]ObjectCounters)
graveyardBKT := tx.Bucket(graveyardBucketName)
garbageBKT := tx.Bucket(garbageBucketName)
key := make([]byte, addressKeySize)
var isAvailable bool
err = iteratePhyObjects(tx, func(cnr cid.ID, objID oid.ID, obj *objectSDK.Object) error {
err := iteratePhyObjects(r, func(cnr cid.ID, objID oid.ID, obj *objectSDK.Object) error {
if v, ok := counters[cnr]; ok {
v.Phy++
counters[cnr] = v
@ -400,9 +253,12 @@ func syncCounter(tx *bbolt.Tx, force bool) error {
addr.SetObject(objID)
isAvailable = false
// check if an object is available: not with GCMark
// and not covered with a tombstone
if inGraveyardWithKey(addressKey(addr, key), graveyardBKT, garbageBKT) == 0 {
st, err := inGraveyardWithKey(r, addr)
if err != nil {
return err
}
if st == 0 {
if v, ok := counters[cnr]; ok {
v.Logic++
counters[cnr] = v
@ -428,104 +284,25 @@ func syncCounter(tx *bbolt.Tx, force bool) error {
return nil
})
if err != nil {
return fmt.Errorf("could not iterate objects: %w", err)
return nil, fmt.Errorf("could not iterate objects: %w", err)
}
return setObjectCounters(counters, shardInfoB, containerCounterB)
return counters, nil
}
func setObjectCounters(counters map[cid.ID]ObjectCounters, shardInfoB, containerCounterB *bbolt.Bucket) error {
var phyTotal uint64
var logicTotal uint64
var userTotal uint64
key := make([]byte, cidSize)
func setObjectCounters(b *pebble.Batch, counters map[cid.ID]ObjectCounters) error {
for cnrID, count := range counters {
phyTotal += count.Phy
logicTotal += count.Logic
userTotal += count.User
cnrID.Encode(key)
value := containerCounterValue(count)
err := containerCounterB.Put(key, value)
if err != nil {
return fmt.Errorf("could not update phy container object counter: %w", err)
delta := objectCounterValue{
Logic: int64(count.Logic),
Phy: int64(count.Phy),
User: int64(count.User),
}
if err := editContainerCounterValue(b, cnrID, delta); err != nil {
return err
}
}
phyData := make([]byte, 8)
binary.LittleEndian.PutUint64(phyData, phyTotal)
err := shardInfoB.Put(objectPhyCounterKey, phyData)
if err != nil {
return fmt.Errorf("could not update phy object counter: %w", err)
}
logData := make([]byte, 8)
binary.LittleEndian.PutUint64(logData, logicTotal)
err = shardInfoB.Put(objectLogicCounterKey, logData)
if err != nil {
return fmt.Errorf("could not update logic object counter: %w", err)
}
userData := make([]byte, 8)
binary.LittleEndian.PutUint64(userData, userTotal)
err = shardInfoB.Put(objectUserCounterKey, userData)
if err != nil {
return fmt.Errorf("could not update user object counter: %w", err)
}
return nil
}
func containerCounterValue(entity ObjectCounters) []byte {
res := make([]byte, 24)
binary.LittleEndian.PutUint64(res, entity.Phy)
binary.LittleEndian.PutUint64(res[8:], entity.Logic)
binary.LittleEndian.PutUint64(res[16:], entity.User)
return res
}
func parseContainerCounterKey(buf []byte) (cid.ID, error) {
if len(buf) != cidSize {
return cid.ID{}, errInvalidKeyLenght
}
var cnrID cid.ID
if err := cnrID.Decode(buf); err != nil {
return cid.ID{}, fmt.Errorf("failed to decode container ID: %w", err)
}
return cnrID, nil
}
// parseContainerCounterValue return phy, logic values.
func parseContainerCounterValue(buf []byte) (ObjectCounters, error) {
if len(buf) != 24 {
return ObjectCounters{}, errInvalidValueLenght
}
return ObjectCounters{
Phy: binary.LittleEndian.Uint64(buf),
Logic: binary.LittleEndian.Uint64(buf[8:16]),
User: binary.LittleEndian.Uint64(buf[16:]),
}, nil
}
func containerObjectCounterInitialized(tx *bbolt.Tx) bool {
b := tx.Bucket(containerCounterBucketName)
if b == nil {
return false
}
k, v := b.Cursor().First()
if k == nil && v == nil {
return true
}
_, err := parseContainerCounterKey(k)
if err != nil {
return false
}
_, err = parseContainerCounterValue(v)
return err == nil
}
func IsUserObject(obj *objectSDK.Object) bool {
ech := obj.ECHeader()
if ech == nil {
@ -537,134 +314,6 @@ func IsUserObject(obj *objectSDK.Object) bool {
return ech.Index() == 0 && (ech.ParentSplitID() == nil || ech.ParentSplitParentID() != nil)
}
// ZeroSizeContainers returns containers with size = 0.
func (db *DB) ZeroSizeContainers(ctx context.Context) ([]cid.ID, error) {
var (
startedAt = time.Now()
success = false
)
defer func() {
db.metrics.AddMethodDuration("ZeroSizeContainers", time.Since(startedAt), success)
}()
ctx, span := tracing.StartSpanFromContext(ctx, "metabase.ZeroSizeContainers")
defer span.End()
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
var result []cid.ID
lastKey := make([]byte, cidSize)
for {
select {
case <-ctx.Done():
return nil, ctx.Err()
default:
}
completed, err := db.containerSizesNextBatch(lastKey, func(contID cid.ID, size uint64) {
if size == 0 {
result = append(result, contID)
}
})
if err != nil {
return nil, err
}
if completed {
break
}
}
success = true
return result, nil
}
func (db *DB) containerSizesNextBatch(lastKey []byte, f func(cid.ID, uint64)) (bool, error) {
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return false, ErrDegradedMode
}
counter := 0
const batchSize = 1000
err := db.boltDB.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(containerVolumeBucketName)
c := b.Cursor()
var key, value []byte
for key, value = c.Seek(lastKey); key != nil; key, value = c.Next() {
if bytes.Equal(lastKey, key) {
continue
}
copy(lastKey, key)
size := parseContainerSize(value)
var id cid.ID
if err := id.Decode(key); err != nil {
return err
}
f(id, size)
counter++
if counter == batchSize {
break
}
}
if counter < batchSize {
return ErrInterruptIterator
}
return nil
})
if err != nil {
if errors.Is(err, ErrInterruptIterator) {
return true, nil
}
return false, metaerr.Wrap(err)
}
return false, nil
}
func (db *DB) DeleteContainerSize(ctx context.Context, id cid.ID) error {
var (
startedAt = time.Now()
success = false
)
defer func() {
db.metrics.AddMethodDuration("DeleteContainerSize", time.Since(startedAt), success)
}()
_, span := tracing.StartSpanFromContext(ctx, "metabase.DeleteContainerSize",
trace.WithAttributes(
attribute.Stringer("container_id", id),
))
defer span.End()
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return ErrDegradedMode
}
if db.mode.ReadOnly() {
return ErrReadOnlyMode
}
err := db.boltDB.Update(func(tx *bbolt.Tx) error {
b := tx.Bucket(containerVolumeBucketName)
key := make([]byte, cidSize)
id.Encode(key)
return b.Delete(key)
})
success = err == nil
return metaerr.Wrap(err)
}
// ZeroCountContainers returns containers with objects count = 0 in metabase.
func (db *DB) ZeroCountContainers(ctx context.Context) ([]cid.ID, error) {
var (
@ -687,24 +336,18 @@ func (db *DB) ZeroCountContainers(ctx context.Context) ([]cid.ID, error) {
var result []cid.ID
lastKey := make([]byte, cidSize)
for {
select {
case <-ctx.Done():
return nil, ctx.Err()
default:
}
completed, err := db.containerCountersNextBatch(lastKey, func(id cid.ID, entity ObjectCounters) {
if entity.IsZero() {
result = append(result, id)
}
})
if err != nil {
return nil, metaerr.Wrap(err)
}
if completed {
break
var cc map[cid.ID]ObjectCounters
err := db.snapshot(func(s *pebble.Snapshot) error {
var err error
cc, err = containerObjectCounters(ctx, s)
return err
})
if err != nil {
return nil, metaerr.Wrap(err)
}
for cnrID, c := range cc {
if c.IsZero() {
result = append(result, cnrID)
}
}
success = true
@ -737,13 +380,104 @@ func (db *DB) DeleteContainerCount(ctx context.Context, id cid.ID) error {
return ErrReadOnlyMode
}
err := db.boltDB.Update(func(tx *bbolt.Tx) error {
b := tx.Bucket(containerCounterBucketName)
defer db.guard.LockContainerID(id)()
key := make([]byte, cidSize)
id.Encode(key)
return b.Delete(key)
err := db.batch(func(b *pebble.Batch) error {
return b.Delete(containerCounterKey(id), pebble.Sync)
})
success = err == nil
return metaerr.Wrap(err)
if err != nil {
return metaerr.Wrap(err)
}
success = true
return nil
}
func containerObjectCounterInitialized(ctx context.Context, r pebble.Reader) bool {
_, err := containerObjectCounters(ctx, r)
return err == nil
}
func containerObjectCounters(ctx context.Context, r pebble.Reader) (map[cid.ID]ObjectCounters, error) {
prefix := []byte{containerCountersPrefix}
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return nil, err
}
counters := make(map[cid.ID]objectCounterValue)
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return nil, errors.Join(ctx.Err(), it.Close())
default:
}
var cnrID cid.ID
if !parseContainerID(&cnrID, it.Key()) {
return nil, errors.Join(errInvalidContainerIDValue, it.Close())
}
oc, err := parseContainerCounterValue(it.Value())
if err != nil {
return nil, errors.Join(err, it.Close())
}
counters[cnrID] = oc
}
if err := it.Close(); err != nil {
return nil, err
}
return normilizeObjectCounters(counters)
}
// parseContainerCounterValue return phy, logic values.
func parseContainerCounterValue(buf []byte) (objectCounterValue, error) {
if len(buf) != 24 {
return objectCounterValue{}, errInvalidValueLenght
}
return objectCounterValue{
Phy: int64(binary.LittleEndian.Uint64(buf[:8])),
Logic: int64(binary.LittleEndian.Uint64(buf[8:16])),
User: int64(binary.LittleEndian.Uint64(buf[16:])),
}, nil
}
func marshalContainerCounterValue(v objectCounterValue) []byte {
buf := make([]byte, 24)
binary.LittleEndian.PutUint64(buf[:8], uint64(v.Phy))
binary.LittleEndian.PutUint64(buf[8:16], uint64(v.Logic))
binary.LittleEndian.PutUint64(buf[16:], uint64(v.User))
return buf
}
func mergeObjectCounterValues(lhs, rhs objectCounterValue) objectCounterValue {
lhs.Logic += rhs.Logic
lhs.Phy += rhs.Phy
lhs.User += rhs.User
return lhs
}
func normilizeObjectCounters(values map[cid.ID]objectCounterValue) (map[cid.ID]ObjectCounters, error) {
result := make(map[cid.ID]ObjectCounters, len(values))
for k, v := range values {
if v.Logic < 0 || v.Phy < 0 || v.User < 0 {
return nil, fmt.Errorf("invalid container object counter for container ID %s", k.EncodeToString())
}
var oc ObjectCounters
oc.Logic = uint64(v.Logic)
oc.Phy = uint64(v.Phy)
oc.User = uint64(v.User)
result[k] = oc
}
return result, nil
}
func parseContainerID(dst *cid.ID, name []byte) bool {
if len(name) < bucketKeySize {
return false
}
return dst.Decode(name[1:bucketKeySize]) == nil
}

View file

@ -23,7 +23,7 @@ func TestCounters(t *testing.T) {
t.Parallel()
db := newDB(t)
defer func() { require.NoError(t, db.Close()) }()
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Zero(t, c.Phy)
require.Zero(t, c.Logic)
@ -59,7 +59,7 @@ func TestCounters(t *testing.T) {
_, err := db.Put(context.Background(), prm)
require.NoError(t, err)
c, err = db.ObjectCounters()
c, err = db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(i+1), c.Phy)
@ -90,13 +90,13 @@ func TestCounters(t *testing.T) {
var prm meta.DeletePrm
for i := objCount - 1; i >= 0; i-- {
prm.SetAddresses(objectcore.AddressOf(oo[i]))
prm.Address = objectcore.AddressOf(oo[i])
res, err := db.Delete(context.Background(), prm)
require.NoError(t, err)
require.Equal(t, uint64(1), res.LogicCount())
require.Equal(t, uint64(1), res.LogicCount)
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(i), c.Phy)
@ -164,7 +164,7 @@ func TestCounters(t *testing.T) {
require.Equal(t, uint64(len(inhumedObjs)), res.LogicInhumed())
require.Equal(t, uint64(len(inhumedObjs)), res.UserInhumed())
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(objCount), c.Phy)
@ -203,7 +203,7 @@ func TestCounters(t *testing.T) {
require.NoError(t, putBig(db, o))
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(i+1), c.Phy)
require.Equal(t, uint64(i+1), c.Logic)
@ -238,7 +238,7 @@ func TestCounters(t *testing.T) {
addr := objectcore.AddressOf(o)
require.NoError(t, metaDelete(db, addr))
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(objCount-i-1), c.Phy)
require.Equal(t, uint64(objCount-i-1), c.Logic)
@ -302,7 +302,7 @@ func TestCounters(t *testing.T) {
_, err := db.Inhume(context.Background(), prm)
require.NoError(t, err)
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(objCount), c.Phy)
@ -336,7 +336,7 @@ func TestDoublePut(t *testing.T) {
require.NoError(t, err)
require.True(t, pr.Inserted)
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(1), c.Phy)
@ -352,7 +352,7 @@ func TestDoublePut(t *testing.T) {
require.NoError(t, err)
require.False(t, pr.Inserted)
c, err = db.ObjectCounters()
c, err = db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(1), c.Phy)
@ -395,7 +395,7 @@ func TestCounters_Expired(t *testing.T) {
// 1. objects are available and counters are correct
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(objCount), c.Phy)
require.Equal(t, uint64(objCount), c.Logic)
@ -416,7 +416,7 @@ func TestCounters_Expired(t *testing.T) {
es.e = epoch + 2
c, err = db.ObjectCounters()
c, err = db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(objCount), c.Phy)
require.Equal(t, uint64(objCount), c.Logic)
@ -445,7 +445,7 @@ func TestCounters_Expired(t *testing.T) {
require.Equal(t, uint64(1), inhumeRes.LogicInhumed())
require.Equal(t, uint64(1), inhumeRes.UserInhumed())
c, err = db.ObjectCounters()
c, err = db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(len(oo)), c.Phy)
@ -472,12 +472,12 @@ func TestCounters_Expired(t *testing.T) {
// that step they should be equal)
var deletePrm meta.DeletePrm
deletePrm.SetAddresses(oo[0])
deletePrm.Address = oo[0]
deleteRes, err := db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
require.Zero(t, deleteRes.LogicCount())
require.Zero(t, deleteRes.UserCount())
require.Zero(t, deleteRes.LogicCount)
require.Zero(t, deleteRes.UserCount)
if v, ok := exp[oo[0].Container()]; ok {
v.Phy--
@ -486,7 +486,7 @@ func TestCounters_Expired(t *testing.T) {
oo = oo[1:]
c, err = db.ObjectCounters()
c, err = db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(len(oo)), c.Phy)
require.Equal(t, uint64(len(oo)), c.Logic)
@ -501,12 +501,12 @@ func TestCounters_Expired(t *testing.T) {
// service do) should decrease both counters despite the
// expiration fact
deletePrm.SetAddresses(oo[0])
deletePrm.Address = oo[0]
deleteRes, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
require.Equal(t, uint64(1), deleteRes.LogicCount())
require.Equal(t, uint64(1), deleteRes.UserCount())
require.Equal(t, uint64(1), deleteRes.LogicCount)
require.Equal(t, uint64(1), deleteRes.UserCount)
if v, ok := exp[oo[0].Container()]; ok {
v.Phy--
@ -517,7 +517,7 @@ func TestCounters_Expired(t *testing.T) {
oo = oo[1:]
c, err = db.ObjectCounters()
c, err = db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(len(oo)), c.Phy)
require.Equal(t, uint64(len(oo)), c.Logic)
@ -548,7 +548,7 @@ func putObjs(t *testing.T, db *meta.DB, count int, withParent bool) []*objectSDK
_, err = db.Put(context.Background(), prm)
require.NoError(t, err)
c, err := db.ObjectCounters()
c, err := db.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, uint64(i+1), c.Phy)

View file

@ -1,7 +1,6 @@
package meta
import (
"bytes"
"encoding/binary"
"encoding/hex"
"io/fs"
@ -9,22 +8,16 @@ import (
"strconv"
"strings"
"sync"
"time"
v2object "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
"github.com/cockroachdb/pebble"
"github.com/mr-tron/base58"
"go.etcd.io/bbolt"
"go.uber.org/zap"
)
type matcher struct {
matchSlow func(string, []byte, string) bool
matchBucket func(*bbolt.Bucket, string, string, func([]byte, []byte) error) error
}
// EpochState is an interface that provides access to the
// current epoch number.
type EpochState interface {
@ -41,7 +34,7 @@ type DB struct {
matchers map[objectSDK.SearchMatchType]matcher
boltDB *bbolt.DB
database *pebble.DB
initialized bool
}
@ -50,17 +43,14 @@ type DB struct {
type Option func(*cfg)
type cfg struct {
boltOptions *bbolt.Options // optional
boltBatchSize int
boltBatchDelay time.Duration
info Info
log *logger.Logger
epochState EpochState
metrics Metrics
guard *concurrency
}
func defaultCfg() *cfg {
@ -68,10 +58,9 @@ func defaultCfg() *cfg {
info: Info{
Permission: os.ModePerm, // 0777
},
boltBatchDelay: bbolt.DefaultMaxBatchDelay,
boltBatchSize: bbolt.DefaultMaxBatchSize,
log: &logger.Logger{Logger: zap.L()},
metrics: &noopMetrics{},
log: &logger.Logger{Logger: zap.L()},
metrics: &noopMetrics{},
guard: newConcurrency(),
}
}
@ -91,20 +80,16 @@ func New(opts ...Option) *DB {
cfg: c,
matchers: map[objectSDK.SearchMatchType]matcher{
objectSDK.MatchUnknown: {
matchSlow: unknownMatcher,
matchBucket: unknownMatcherBucket,
matchSlow: unknownMatcher,
},
objectSDK.MatchStringEqual: {
matchSlow: stringEqualMatcher,
matchBucket: stringEqualMatcherBucket,
matchSlow: stringEqualMatcher,
},
objectSDK.MatchStringNotEqual: {
matchSlow: stringNotEqualMatcher,
matchBucket: stringNotEqualMatcherBucket,
matchSlow: stringNotEqualMatcher,
},
objectSDK.MatchCommonPrefix: {
matchSlow: stringCommonPrefixMatcher,
matchBucket: stringCommonPrefixMatcherBucket,
matchSlow: stringCommonPrefixMatcher,
},
},
mode: mode.Disabled,
@ -186,110 +171,18 @@ func stringEqualMatcher(key string, objVal []byte, filterVal string) bool {
return stringifyValue(key, objVal) == filterVal
}
func stringEqualMatcherBucket(b *bbolt.Bucket, fKey string, fValue string, f func([]byte, []byte) error) error {
// Ignore the second return value because we check for strict equality.
val, _, ok := destringifyValue(fKey, fValue, false)
if !ok {
return nil
}
if data := b.Get(val); data != nil {
return f(val, data)
}
if b.Bucket(val) != nil {
return f(val, nil)
}
return nil
}
func stringNotEqualMatcher(key string, objVal []byte, filterVal string) bool {
return stringifyValue(key, objVal) != filterVal
}
func stringNotEqualMatcherBucket(b *bbolt.Bucket, fKey string, fValue string, f func([]byte, []byte) error) error {
// Ignore the second return value because we check for strict inequality.
val, _, ok := destringifyValue(fKey, fValue, false)
return b.ForEach(func(k, v []byte) error {
if !ok || !bytes.Equal(val, k) {
return f(k, v)
}
return nil
})
}
func stringCommonPrefixMatcher(key string, objVal []byte, filterVal string) bool {
return strings.HasPrefix(stringifyValue(key, objVal), filterVal)
}
func stringCommonPrefixMatcherBucket(b *bbolt.Bucket, fKey string, fVal string, f func([]byte, []byte) error) error {
val, checkLast, ok := destringifyValue(fKey, fVal, true)
if !ok {
return nil
}
prefix := val
if checkLast {
prefix = val[:len(val)-1]
}
if len(val) == 0 {
// empty common prefix, all the objects
// satisfy that filter
return b.ForEach(f)
}
c := b.Cursor()
for k, v := c.Seek(val); bytes.HasPrefix(k, prefix); k, v = c.Next() {
if checkLast && (len(k) == len(prefix) || k[len(prefix)]>>4 != val[len(val)-1]) {
// If the last byte doesn't match, this means the prefix does no longer match,
// so we need to break here.
break
}
if err := f(k, v); err != nil {
return err
}
}
return nil
}
func unknownMatcher(_ string, _ []byte, _ string) bool {
return false
}
func unknownMatcherBucket(_ *bbolt.Bucket, _ string, _ string, _ func([]byte, []byte) error) error {
return nil
}
// bucketKeyHelper returns byte representation of val that is used as a key
// in boltDB. Useful for getting filter values from unique and list indexes.
func bucketKeyHelper(hdr string, val string) []byte {
switch hdr {
case v2object.FilterHeaderParent, v2object.FilterHeaderECParent:
v, err := base58.Decode(val)
if err != nil {
return nil
}
return v
case v2object.FilterHeaderPayloadHash:
v, err := hex.DecodeString(val)
if err != nil {
return nil
}
return v
case v2object.FilterHeaderSplitID:
s := objectSDK.NewSplitID()
err := s.Parse(val)
if err != nil {
return nil
}
return s.ToV2()
default:
return []byte(val)
}
}
// SetLogger sets logger. It is used after the shard ID was generated to use it in logs.
func (db *DB) SetLogger(l *logger.Logger) {
db.log = l
@ -307,13 +200,6 @@ func WithLogger(l *logger.Logger) Option {
}
}
// WithBoltDBOptions returns option to specify BoltDB options.
func WithBoltDBOptions(opts *bbolt.Options) Option {
return func(c *cfg) {
c.boltOptions = opts
}
}
// WithPath returns option to set system path to Metabase.
func WithPath(path string) Option {
return func(c *cfg) {
@ -329,28 +215,6 @@ func WithPermissions(perm fs.FileMode) Option {
}
}
// WithMaxBatchSize returns option to specify maximum concurrent operations
// to be processed in a single transactions.
// This option is missing from `bbolt.Options` but is set right after DB is open.
func WithMaxBatchSize(s int) Option {
return func(c *cfg) {
if s != 0 {
c.boltBatchSize = s
}
}
}
// WithMaxBatchDelay returns option to specify maximum time to wait before
// the batch of concurrent transactions is processed.
// This option is missing from `bbolt.Options` but is set right after DB is open.
func WithMaxBatchDelay(d time.Duration) Option {
return func(c *cfg) {
if d != 0 {
c.boltBatchDelay = d
}
}
}
// WithEpochState return option to specify a source of current epoch height.
func WithEpochState(s EpochState) Option {
return func(c *cfg) {

View file

@ -1,7 +1,6 @@
package meta
import (
"bytes"
"context"
"errors"
"fmt"
@ -15,63 +14,23 @@ import (
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.etcd.io/bbolt"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
"github.com/cockroachdb/pebble"
)
var errFailedToRemoveUniqueIndexes = errors.New("can't remove unique indexes")
// DeletePrm groups the parameters of Delete operation.
type DeletePrm struct {
addrs []oid.Address
Address oid.Address
}
// DeleteRes groups the resulting values of Delete operation.
type DeleteRes struct {
phyCount uint64
logicCount uint64
userCount uint64
phySize uint64
logicSize uint64
removedByCnrID map[cid.ID]ObjectCounters
}
// LogicCount returns the number of removed logic
// objects.
func (d DeleteRes) LogicCount() uint64 {
return d.logicCount
}
func (d DeleteRes) UserCount() uint64 {
return d.userCount
}
// RemovedByCnrID returns the number of removed objects by container ID.
func (d DeleteRes) RemovedByCnrID() map[cid.ID]ObjectCounters {
return d.removedByCnrID
}
// PhyCount returns the number of removed physical objects.
func (d DeleteRes) PhyCount() uint64 {
return d.phyCount
}
// PhySize returns the size of removed physical objects.
func (d DeleteRes) PhySize() uint64 {
return d.phySize
}
// LogicSize returns the size of removed logical objects.
func (d DeleteRes) LogicSize() uint64 {
return d.logicSize
}
// SetAddresses is a Delete option to set the addresses of the objects to delete.
//
// Option is required.
func (p *DeletePrm) SetAddresses(addrs ...oid.Address) {
p.addrs = addrs
PhyCount uint64
LogicCount uint64
UserCount uint64
PhySize uint64
LogicSize uint64
}
type referenceNumber struct {
@ -82,8 +41,6 @@ type referenceNumber struct {
obj *objectSDK.Object
}
type referenceCounter map[string]*referenceNumber
// Delete removed object records from metabase indexes.
func (db *DB) Delete(ctx context.Context, prm DeletePrm) (DeleteRes, error) {
var (
@ -94,10 +51,7 @@ func (db *DB) Delete(ctx context.Context, prm DeletePrm) (DeleteRes, error) {
db.metrics.AddMethodDuration("Delete", time.Since(startedAt), deleted)
}()
_, span := tracing.StartSpanFromContext(ctx, "metabase.Delete",
trace.WithAttributes(
attribute.Int("addr_count", len(prm.addrs)),
))
_, span := tracing.StartSpanFromContext(ctx, "metabase.Delete")
defer span.End()
db.modeMtx.RLock()
@ -109,403 +63,260 @@ func (db *DB) Delete(ctx context.Context, prm DeletePrm) (DeleteRes, error) {
return DeleteRes{}, ErrReadOnlyMode
}
defer db.guard.LockContainerID(prm.Address.Container())()
var err error
var res DeleteRes
err = db.boltDB.Update(func(tx *bbolt.Tx) error {
res, err = db.deleteGroup(tx, prm.addrs)
err = db.batch(func(b *pebble.Batch) error {
res, err = db.deleteByAddress(ctx, b, prm.Address)
return err
})
if err == nil {
deleted = true
for i := range prm.addrs {
storagelog.Write(db.log,
storagelog.AddressField(prm.addrs[i]),
storagelog.OpField("metabase DELETE"))
}
storagelog.Write(db.log,
storagelog.AddressField(prm.Address),
storagelog.OpField("metabase DELETE"))
}
return res, metaerr.Wrap(err)
}
// deleteGroup deletes object from the metabase. Handles removal of the
// references of the split objects.
func (db *DB) deleteGroup(tx *bbolt.Tx, addrs []oid.Address) (DeleteRes, error) {
res := DeleteRes{
removedByCnrID: make(map[cid.ID]ObjectCounters),
}
refCounter := make(referenceCounter, len(addrs))
func (db *DB) deleteByAddress(ctx context.Context, b *pebble.Batch, addr oid.Address) (DeleteRes, error) {
refCounter := &referenceNumber{}
currEpoch := db.epochState.CurrentEpoch()
for i := range addrs {
r, err := db.delete(tx, addrs[i], refCounter, currEpoch)
if err != nil {
return DeleteRes{}, err
}
applyDeleteSingleResult(r, &res, addrs, i)
}
if err := db.updateCountersDelete(tx, res); err != nil {
res, err := db.delete(ctx, b, addr, refCounter, currEpoch)
if err != nil {
return DeleteRes{}, err
}
for _, refNum := range refCounter {
if refNum.cur == refNum.all {
err := db.deleteObject(tx, refNum.obj, true)
if err != nil {
return DeleteRes{}, err
}
}
if err := db.updateCountersDelete(b, addr.Container(), res); err != nil {
return DeleteRes{}, err
}
if refCounter.cur == refCounter.all {
err := db.deleteObject(ctx, b, refCounter.obj, true)
if err != nil {
return DeleteRes{}, err
}
}
return res, nil
}
func (db *DB) updateCountersDelete(tx *bbolt.Tx, res DeleteRes) error {
if res.phyCount > 0 {
err := db.updateShardObjectCounter(tx, phy, res.phyCount, false)
if err != nil {
return fmt.Errorf("could not decrease phy object counter: %w", err)
}
}
if res.logicCount > 0 {
err := db.updateShardObjectCounter(tx, logical, res.logicCount, false)
if err != nil {
return fmt.Errorf("could not decrease logical object counter: %w", err)
}
}
if res.userCount > 0 {
err := db.updateShardObjectCounter(tx, user, res.userCount, false)
if err != nil {
return fmt.Errorf("could not decrease user object counter: %w", err)
}
}
if err := db.updateContainerCounter(tx, res.removedByCnrID, false); err != nil {
func (db *DB) updateCountersDelete(b *pebble.Batch, cnrID cid.ID, res DeleteRes) error {
if err := editContainerCounterValue(b, cnrID, objectCounterValue{
Logic: -1 * int64(res.LogicCount),
Phy: -1 * int64(res.PhyCount),
User: -1 * int64(res.UserCount),
}); err != nil {
return fmt.Errorf("could not decrease container object counter: %w", err)
}
return nil
}
func applyDeleteSingleResult(r deleteSingleResult, res *DeleteRes, addrs []oid.Address, i int) {
if r.Phy {
if v, ok := res.removedByCnrID[addrs[i].Container()]; ok {
v.Phy++
res.removedByCnrID[addrs[i].Container()] = v
} else {
res.removedByCnrID[addrs[i].Container()] = ObjectCounters{
Phy: 1,
}
}
res.phyCount++
res.phySize += r.Size
}
if r.Logic {
if v, ok := res.removedByCnrID[addrs[i].Container()]; ok {
v.Logic++
res.removedByCnrID[addrs[i].Container()] = v
} else {
res.removedByCnrID[addrs[i].Container()] = ObjectCounters{
Logic: 1,
}
}
res.logicCount++
res.logicSize += r.Size
}
if r.User {
if v, ok := res.removedByCnrID[addrs[i].Container()]; ok {
v.User++
res.removedByCnrID[addrs[i].Container()] = v
} else {
res.removedByCnrID[addrs[i].Container()] = ObjectCounters{
User: 1,
}
}
res.userCount++
}
}
type deleteSingleResult struct {
Phy bool
Logic bool
User bool
Size uint64
}
// delete removes object indexes from the metabase. Counts the references
// of the object that is being removed.
// The first return value indicates if an object has been removed. (removing a
// non-exist object is error-free). The second return value indicates if an
// object was available before the removal (for calculating the logical object
// counter). The third return value The fourth return value is removed object payload size.
func (db *DB) delete(tx *bbolt.Tx, addr oid.Address, refCounter referenceCounter, currEpoch uint64) (deleteSingleResult, error) {
key := make([]byte, addressKeySize)
addrKey := addressKey(addr, key)
garbageBKT := tx.Bucket(garbageBucketName)
graveyardBKT := tx.Bucket(graveyardBucketName)
removeAvailableObject := inGraveyardWithKey(addrKey, graveyardBKT, garbageBKT) == 0
func (db *DB) delete(ctx context.Context, b *pebble.Batch, addr oid.Address, refCounter *referenceNumber, currEpoch uint64) (DeleteRes, error) {
status, err := inGraveyardWithKey(b, addr)
if err != nil {
return DeleteRes{}, err
}
removeAvailableObject := status == 0
// unmarshal object, work only with physically stored (raw == true) objects
obj, err := db.get(tx, addr, key, false, true, currEpoch)
obj, err := get(ctx, b, addr, false, true, currEpoch)
if err != nil {
if client.IsErrObjectNotFound(err) {
addrKey = addressKey(addr, key)
if garbageBKT != nil {
err := garbageBKT.Delete(addrKey)
if err != nil {
return deleteSingleResult{}, fmt.Errorf("could not remove from garbage bucket: %w", err)
}
err := deleteFromGarbage(b, addr)
if err != nil {
return DeleteRes{}, fmt.Errorf("could not remove from garbage bucket: %w", err)
}
return deleteSingleResult{}, nil
return DeleteRes{}, nil
}
var siErr *objectSDK.SplitInfoError
var ecErr *objectSDK.ECInfoError
if errors.As(err, &siErr) || errors.As(err, &ecErr) {
// if object is virtual (parent) then do nothing, it will be deleted with last child
// if object is erasure-coded it will be deleted with the last chunk presented on the shard
return deleteSingleResult{}, nil
return DeleteRes{}, nil
}
return deleteSingleResult{}, err
return DeleteRes{}, err
}
addrKey = addressKey(addr, key)
// remove record from the garbage bucket
if garbageBKT != nil {
err := garbageBKT.Delete(addrKey)
if err != nil {
return deleteSingleResult{}, fmt.Errorf("could not remove from garbage bucket: %w", err)
}
err = deleteFromGarbage(b, addr)
if err != nil {
return DeleteRes{}, fmt.Errorf("could not remove from garbage bucket: %w", err)
}
// if object is an only link to a parent, then remove parent
if parent := obj.Parent(); parent != nil {
parAddr := object.AddressOf(parent)
sParAddr := addressKey(parAddr, key)
k := string(sParAddr)
nRef, ok := refCounter[k]
if !ok {
nRef = &referenceNumber{
all: parentLength(tx, parAddr),
addr: parAddr,
obj: parent,
}
refCounter[k] = nRef
parentLen, err := parentLength(ctx, b, parAddr)
if err != nil {
return DeleteRes{}, fmt.Errorf("failed to get parent count for object %s: %w", parAddr, err)
}
nRef.cur++
refCounter.addr = parAddr
refCounter.all = parentLen
refCounter.obj = parent
refCounter.cur = 1
}
isUserObject := IsUserObject(obj)
// remove object
err = db.deleteObject(tx, obj, false)
err = db.deleteObject(ctx, b, obj, false)
if err != nil {
return deleteSingleResult{}, fmt.Errorf("could not remove object: %w", err)
return DeleteRes{}, fmt.Errorf("could not remove object: %w", err)
}
if err := deleteECRelatedInfo(tx, garbageBKT, obj, addr.Container(), refCounter); err != nil {
return deleteSingleResult{}, err
if err := deleteECRelatedInfo(ctx, b, obj, addr.Container(), refCounter); err != nil {
return DeleteRes{}, fmt.Errorf("could not remove EC related info object: %w", err)
}
return deleteSingleResult{
Phy: true,
Logic: removeAvailableObject,
User: isUserObject && removeAvailableObject,
Size: obj.PayloadSize(),
}, nil
var result DeleteRes
result.PhyCount = 1
result.PhySize = obj.PayloadSize()
if removeAvailableObject {
result.LogicCount = 1
result.LogicSize = obj.PayloadSize()
}
if removeAvailableObject && isUserObject {
result.UserCount = 1
}
return result, nil
}
func (db *DB) deleteObject(
tx *bbolt.Tx,
ctx context.Context,
b *pebble.Batch,
obj *objectSDK.Object,
isParent bool,
) error {
err := delUniqueIndexes(tx, obj, isParent)
err := delUniqueIndexes(ctx, b, obj, isParent)
if err != nil {
return errFailedToRemoveUniqueIndexes
}
err = updateListIndexes(tx, obj, delListIndexItem)
err = updateListIndexes(b, obj, deleteByKey)
if err != nil {
return fmt.Errorf("can't remove list indexes: %w", err)
}
err = updateFKBTIndexes(tx, obj, delFKBTIndexItem)
err = updateFKBTIndexes(b, obj, deleteByKey)
if err != nil {
return fmt.Errorf("can't remove fake bucket tree indexes: %w", err)
}
if isParent {
// remove record from the garbage bucket, because regular object deletion does nothing for virtual object
garbageBKT := tx.Bucket(garbageBucketName)
if garbageBKT != nil {
key := make([]byte, addressKeySize)
addrKey := addressKey(object.AddressOf(obj), key)
err := garbageBKT.Delete(addrKey)
if err != nil {
return fmt.Errorf("could not remove from garbage bucket: %w", err)
}
err := deleteFromGarbage(b, object.AddressOf(obj))
if err != nil {
return fmt.Errorf("could not remove from garbage bucket: %w", err)
}
}
return nil
}
func deleteByKey(b *pebble.Batch, key []byte) error {
return b.Delete(key, pebble.Sync)
}
func deleteFromGarbage(b *pebble.Batch, addr oid.Address) error {
return b.Delete(garbageKey(addr.Container(), addr.Object()), pebble.Sync)
}
// parentLength returns amount of available children from parentid index.
func parentLength(tx *bbolt.Tx, addr oid.Address) int {
bucketName := make([]byte, bucketKeySize)
bkt := tx.Bucket(parentBucketName(addr.Container(), bucketName[:]))
if bkt == nil {
return 0
}
lst, err := decodeList(bkt.Get(objectKey(addr.Object(), bucketName[:])))
if err != nil {
return 0
}
return len(lst)
}
func delUniqueIndexItem(tx *bbolt.Tx, item namedBucketItem) {
bkt := tx.Bucket(item.name)
if bkt != nil {
_ = bkt.Delete(item.key) // ignore error, best effort there
}
}
func delFKBTIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
bkt := tx.Bucket(item.name)
if bkt == nil {
return nil
}
fkbtRoot := bkt.Bucket(item.key)
if fkbtRoot == nil {
return nil
}
_ = fkbtRoot.Delete(item.val) // ignore error, best effort there
return nil
}
func delListIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
bkt := tx.Bucket(item.name)
if bkt == nil {
return nil
}
lst, err := decodeList(bkt.Get(item.key))
if err != nil || len(lst) == 0 {
return nil
}
// remove element from the list
for i := range lst {
if bytes.Equal(item.val, lst[i]) {
copy(lst[i:], lst[i+1:])
lst = lst[:len(lst)-1]
func parentLength(ctx context.Context, r pebble.Reader, addr oid.Address) (int, error) {
var result int
prefix := parentKeyLongPrefix(addr.Container(), addr.Object())
for {
ids, err := selectByPrefixBatch(ctx, r, prefix, batchSize)
if err != nil {
return 0, err
}
result += len(ids)
if len(ids) < batchSize {
break
}
}
// if list empty, remove the key from <list> bucket
if len(lst) == 0 {
_ = bkt.Delete(item.key) // ignore error, best effort there
return nil
}
// if list is not empty, then update it
encodedLst, err := encodeList(lst)
if err != nil {
return nil // ignore error, best effort there
}
_ = bkt.Put(item.key, encodedLst) // ignore error, best effort there
return nil
return result, nil
}
func delUniqueIndexes(tx *bbolt.Tx, obj *objectSDK.Object, isParent bool) error {
addr := object.AddressOf(obj)
func delParent(ctx context.Context, b *pebble.Batch, addr oid.Address) error {
prefix := parentKeyLongPrefix(addr.Container(), addr.Object())
return deleteByPrefix(ctx, b, prefix)
}
objKey := objectKey(addr.Object(), make([]byte, objectKeySize))
cnr := addr.Container()
bucketName := make([]byte, bucketKeySize)
func delUniqueIndexes(ctx context.Context, b *pebble.Batch, obj *objectSDK.Object, isParent bool) error {
addr := object.AddressOf(obj)
// add value to primary unique bucket
if !isParent {
var key []byte
switch obj.Type() {
case objectSDK.TypeRegular:
bucketName = primaryBucketName(cnr, bucketName)
key = primaryKey(addr.Container(), addr.Object())
case objectSDK.TypeTombstone:
bucketName = tombstoneBucketName(cnr, bucketName)
key = tombstoneKey(addr.Container(), addr.Object())
case objectSDK.TypeLock:
bucketName = bucketNameLockers(cnr, bucketName)
key = lockersKey(addr.Container(), addr.Object())
default:
return ErrUnknownObjectType
}
delUniqueIndexItem(tx, namedBucketItem{
name: bucketName,
key: objKey,
})
if err := b.Delete(key, pebble.Sync); err != nil {
return err
}
} else {
delUniqueIndexItem(tx, namedBucketItem{
name: parentBucketName(cnr, bucketName),
key: objKey,
})
if err := delParent(ctx, b, addr); err != nil {
return err
}
}
delUniqueIndexItem(tx, namedBucketItem{ // remove from storage id index
name: smallBucketName(cnr, bucketName),
key: objKey,
})
delUniqueIndexItem(tx, namedBucketItem{ // remove from root index
name: rootBucketName(cnr, bucketName),
key: objKey,
})
return nil
if err := b.Delete(smallKey(addr.Container(), addr.Object()), pebble.Sync); err != nil {
return err
}
if ecHead := obj.ECHeader(); ecHead != nil {
if err := b.Delete(ecInfoKey(addr.Container(), ecHead.Parent(), addr.Object()), pebble.Sync); err != nil {
return err
}
}
return b.Delete(rootKey(addr.Container(), addr.Object()), pebble.Sync)
}
func deleteECRelatedInfo(tx *bbolt.Tx, garbageBKT *bbolt.Bucket, obj *objectSDK.Object, cnr cid.ID, refCounter referenceCounter) error {
func deleteECRelatedInfo(ctx context.Context, b *pebble.Batch, obj *objectSDK.Object, cnr cid.ID, refCounter *referenceNumber) error {
ech := obj.ECHeader()
if ech == nil {
return nil
}
hasAnyChunks := hasAnyECChunks(tx, ech, cnr)
hasAnyChunks, err := hasAnyECChunks(ctx, b, ech, cnr)
if err != nil {
return err
}
// drop EC parent GC mark if current EC chunk is the last one
if !hasAnyChunks && garbageBKT != nil {
if !hasAnyChunks {
var ecParentAddress oid.Address
ecParentAddress.SetContainer(cnr)
ecParentAddress.SetObject(ech.Parent())
addrKey := addressKey(ecParentAddress, make([]byte, addressKeySize))
err := garbageBKT.Delete(addrKey)
err := deleteFromGarbage(b, ecParentAddress)
if err != nil {
return fmt.Errorf("could not remove EC parent from garbage bucket: %w", err)
}
}
// also drop EC parent root info if current EC chunk is the last one
if !hasAnyChunks {
delUniqueIndexItem(tx, namedBucketItem{
name: rootBucketName(cnr, make([]byte, bucketKeySize)),
key: objectKey(ech.Parent(), make([]byte, objectKeySize)),
})
// also drop EC parent root info if current EC chunk is the last one
err = b.Delete(rootKey(ecParentAddress.Container(), ecParentAddress.Object()), pebble.Sync)
if err != nil {
return fmt.Errorf("could not remove EC parent from root bucket: %w", err)
}
}
if ech.ParentSplitParentID() == nil {
@ -516,38 +327,37 @@ func deleteECRelatedInfo(tx *bbolt.Tx, garbageBKT *bbolt.Bucket, obj *objectSDK.
splitParentAddress.SetContainer(cnr)
splitParentAddress.SetObject(*ech.ParentSplitParentID())
if ref, ok := refCounter[string(addressKey(splitParentAddress, make([]byte, addressKeySize)))]; ok {
if refCounter.all > 0 {
// linking object is already processing
// so just inform that one more reference was deleted
// split info and gc marks will be deleted after linking object delete
ref.cur++
refCounter.cur++
return nil
}
if parentLength(tx, splitParentAddress) > 0 {
parLen, err := parentLength(ctx, b, splitParentAddress)
if err != nil {
return err
}
if parLen > 0 {
// linking object still exists, so leave split info and gc mark deletion for linking object processing
return nil
}
// drop split parent gc mark
if garbageBKT != nil {
addrKey := addressKey(splitParentAddress, make([]byte, addressKeySize))
err := garbageBKT.Delete(addrKey)
if err != nil {
return fmt.Errorf("could not remove EC parent from garbage bucket: %w", err)
}
err = deleteFromGarbage(b, splitParentAddress)
if err != nil {
return fmt.Errorf("could not remove EC split parent from garbage bucket: %w", err)
}
// drop split info
delUniqueIndexItem(tx, namedBucketItem{
name: rootBucketName(cnr, make([]byte, bucketKeySize)),
key: objectKey(*ech.ParentSplitParentID(), make([]byte, objectKeySize)),
})
return nil
return b.Delete(rootKey(splitParentAddress.Container(), splitParentAddress.Object()), pebble.Sync)
}
func hasAnyECChunks(tx *bbolt.Tx, ech *objectSDK.ECHeader, cnr cid.ID) bool {
data := getFromBucket(tx, ecInfoBucketName(cnr, make([]byte, bucketKeySize)),
objectKey(ech.Parent(), make([]byte, objectKeySize)))
return len(data) > 0
func hasAnyECChunks(ctx context.Context, r pebble.Reader, ech *objectSDK.ECHeader, cnr cid.ID) (bool, error) {
data, err := selectByPrefixBatch(ctx, r, ecInfoLongKeyPrefix(cnr, ech.Parent()), 1)
if err != nil {
return false, err
}
return len(data) > 0, nil
}

View file

@ -16,6 +16,7 @@ import (
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
oidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id/test"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/require"
"go.etcd.io/bbolt"
)
@ -107,10 +108,11 @@ func TestDeleteECObject_WithoutSplit(t *testing.T) {
require.True(t, slices.Contains(garbageAddresses, ecParentAddress))
require.True(t, slices.Contains(garbageAddresses, ecChunkAddress))
var deletePrm DeletePrm
deletePrm.SetAddresses(garbageAddresses...)
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
for _, a := range garbageAddresses {
deletePrm := DeletePrm{Address: a}
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
}
garbageAddresses = nil
itPrm.SetHandler(func(g GarbageObject) error {
@ -153,9 +155,11 @@ func TestDeleteECObject_WithoutSplit(t *testing.T) {
require.Equal(t, 1, len(garbageAddresses))
require.Equal(t, tombstoneID, garbageAddresses[0].Object())
deletePrm.SetAddresses(garbageAddresses...)
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
for _, a := range garbageAddresses {
deletePrm := DeletePrm{Address: a}
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
}
// no more objects should left as garbage
@ -165,7 +169,7 @@ func TestDeleteECObject_WithoutSplit(t *testing.T) {
})
require.NoError(t, db.IterateOverGarbage(context.Background(), itPrm))
require.NoError(t, db.boltDB.View(testVerifyNoObjectDataLeft))
require.NoError(t, testVerifyNoObjectDataLeft(db.database.NewSnapshot()))
require.NoError(t, testCountersAreZero(db, cnr))
}
@ -351,10 +355,11 @@ func testDeleteECObjectWithSplit(t *testing.T, chunksCount int, withLinking bool
require.True(t, slices.Contains(garbageAddresses, ecChunkAddress))
}
var deletePrm DeletePrm
deletePrm.SetAddresses(garbageAddresses...)
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
for _, a := range garbageAddresses {
deletePrm := DeletePrm{Address: a}
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
}
var garbageStub []oid.Address
itPrm.SetHandler(func(g GarbageObject) error {
@ -397,9 +402,11 @@ func testDeleteECObjectWithSplit(t *testing.T, chunksCount int, withLinking bool
require.Equal(t, 1, len(garbageAddresses))
require.Equal(t, tombstoneID, garbageAddresses[0].Object())
deletePrm.SetAddresses(garbageAddresses...)
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
for _, a := range garbageAddresses {
deletePrm := DeletePrm{Address: a}
_, err = db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
}
// no more objects should left as garbage
@ -409,20 +416,32 @@ func testDeleteECObjectWithSplit(t *testing.T, chunksCount int, withLinking bool
})
require.NoError(t, db.IterateOverGarbage(context.Background(), itPrm))
require.NoError(t, db.boltDB.View(testVerifyNoObjectDataLeft))
require.NoError(t, testVerifyNoObjectDataLeft(db.database.NewSnapshot()))
require.NoError(t, testCountersAreZero(db, cnr))
}
func testVerifyNoObjectDataLeft(tx *bbolt.Tx) error {
return tx.ForEach(func(name []byte, b *bbolt.Bucket) error {
if bytes.Equal(name, shardInfoBucket) ||
bytes.Equal(name, containerCounterBucketName) ||
bytes.Equal(name, containerVolumeBucketName) {
return nil
func testVerifyNoObjectDataLeft(s *pebble.Snapshot) error {
defer s.Close()
it, err := s.NewIter(nil)
if err != nil {
return err
}
for v := it.First(); v; v = it.Next() {
if bytes.Equal(it.Key(), shardInfoKey(versionKey)) ||
bytes.HasPrefix(it.Key(), []byte{containerCountersPrefix}) ||
bytes.HasPrefix(it.Key(), []byte{containerSizePrefix}) {
continue
}
return testBucketEmpty(name, b)
})
err := it.Close()
if err != nil {
return err
}
return fmt.Errorf("database is not empty, key %s found", string(it.Key()))
}
return it.Close()
}
func testBucketEmpty(name []byte, b *bbolt.Bucket) error {
@ -448,7 +467,7 @@ func testCountersAreZero(db *DB, cnr cid.ID) error {
if !c.IsZero() {
return fmt.Errorf("container %s has non zero counters", cnr.EncodeToString())
}
s, err := db.ContainerSize(cnr)
s, err := db.ContainerSize(context.Background(), cnr)
if err != nil {
return err
}

View file

@ -155,9 +155,11 @@ func TestDelete(t *testing.T) {
require.NoError(t, db.IterateOverGarbage(context.Background(), iprm))
require.Equal(t, 10, len(addrs))
var deletePrm meta.DeletePrm
deletePrm.SetAddresses(addrs...)
_, err := db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
for _, addr := range addrs {
deletePrm.Address = addr
_, err := db.Delete(context.Background(), deletePrm)
require.NoError(t, err)
}
addrs = nil
iprm.SetHandler(func(o meta.GarbageObject) error {
@ -190,7 +192,7 @@ func TestDeleteDropsGCMarkIfObjectNotFound(t *testing.T) {
require.Equal(t, 1, garbageCount)
var delPrm meta.DeletePrm
delPrm.SetAddresses(addr)
delPrm.Address = addr
_, err = db.Delete(context.Background(), delPrm)
require.NoError(t, err)
@ -201,8 +203,13 @@ func TestDeleteDropsGCMarkIfObjectNotFound(t *testing.T) {
func metaDelete(db *meta.DB, addrs ...oid.Address) error {
var deletePrm meta.DeletePrm
deletePrm.SetAddresses(addrs...)
for _, addr := range addrs {
deletePrm.Address = addr
_, err := db.Delete(context.Background(), deletePrm)
return err
_, err := db.Delete(context.Background(), deletePrm)
if err != nil {
return err
}
}
return nil
}

View file

@ -1,11 +1,11 @@
package meta
import (
"bytes"
"context"
"fmt"
"time"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
@ -13,7 +13,7 @@ import (
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.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
@ -81,8 +81,8 @@ func (db *DB) Exists(ctx context.Context, prm ExistsPrm) (res ExistsRes, err err
currEpoch := db.epochState.CurrentEpoch()
err = db.boltDB.View(func(tx *bbolt.Tx) error {
res.exists, res.locked, err = db.exists(tx, prm.addr, prm.paddr, currEpoch)
err = db.snapshot(func(s *pebble.Snapshot) error {
res.exists, res.locked, err = db.exists(ctx, s, prm.addr, prm.paddr, currEpoch)
return err
})
@ -90,13 +90,21 @@ func (db *DB) Exists(ctx context.Context, prm ExistsPrm) (res ExistsRes, err err
return res, metaerr.Wrap(err)
}
func (db *DB) exists(tx *bbolt.Tx, addr oid.Address, parent oid.Address, currEpoch uint64) (bool, bool, error) {
func (db *DB) exists(ctx context.Context, r pebble.Reader, addr oid.Address, parent oid.Address, currEpoch uint64) (bool, bool, error) {
var locked bool
var err error
if !parent.Equals(oid.Address{}) {
locked = objectLocked(tx, parent.Container(), parent.Object())
locked, err = objectLocked(ctx, r, parent.Container(), parent.Object())
if err != nil {
return false, false, err
}
}
// check graveyard and object expiration first
switch objectStatus(tx, addr, currEpoch) {
st, err := objectStatus(ctx, r, addr, currEpoch)
if err != nil {
return false, false, err
}
switch st {
case 1:
return false, locked, logicerr.Wrap(new(apistatus.ObjectNotFound))
case 2:
@ -105,32 +113,41 @@ func (db *DB) exists(tx *bbolt.Tx, addr oid.Address, parent oid.Address, currEpo
return false, locked, ErrObjectIsExpired
}
objKey := objectKey(addr.Object(), make([]byte, objectKeySize))
cnr := addr.Container()
key := make([]byte, bucketKeySize)
// if graveyard is empty, then check if object exists in primary bucket
if inBucket(tx, primaryBucketName(cnr, key), objKey) {
v, err := valueSafe(r, primaryKey(addr.Container(), addr.Object()))
if err != nil {
return false, false, err
}
if v != nil {
return true, locked, nil
}
// if primary bucket is empty, then check if object exists in parent bucket
if inBucket(tx, parentBucketName(cnr, key), objKey) {
splitInfo, err := getSplitInfo(tx, cnr, objKey)
children, err := selectByPrefixBatch(ctx, r, parentKeyLongPrefix(addr.Container(), addr.Object()), 1) // try to found any child
if err != nil {
return false, false, err
}
if len(children) > 0 {
splitInfo, err := getSplitInfo(r, addr)
if err != nil {
return false, locked, err
return false, false, err
}
return false, locked, logicerr.Wrap(objectSDK.NewSplitInfoError(splitInfo))
}
// if parent bucket is empty, then check if object exists in ec bucket
if data := getFromBucket(tx, ecInfoBucketName(cnr, key), objKey); len(data) != 0 {
return false, locked, getECInfoError(tx, cnr, data)
// if parent bucket is empty, then check if object exists with ec prefix
children, err = selectByPrefixBatch(ctx, r, ecInfoLongKeyPrefix(addr.Container(), addr.Object()), 1) // try to found any child
if err != nil {
return false, locked, err
}
if len(children) > 0 {
return false, locked, getECInfoError(ctx, r, addr)
}
// if parent bucket is empty, then check if object exists in typed buckets
return firstIrregularObjectType(tx, cnr, objKey) != objectSDK.TypeRegular, locked, nil
t, err := firstIrregularObjectType(r, addr.Container(), addr.Object())
if err != nil {
return false, false, err
}
return t != objectSDK.TypeRegular, locked, nil
}
// objectStatus returns:
@ -138,89 +155,95 @@ func (db *DB) exists(tx *bbolt.Tx, addr oid.Address, parent oid.Address, currEpo
// - 1 if object with GC mark;
// - 2 if object is covered with tombstone;
// - 3 if object is expired.
func objectStatus(tx *bbolt.Tx, addr oid.Address, currEpoch uint64) uint8 {
func objectStatus(ctx context.Context, r pebble.Reader, addr oid.Address, currEpoch uint64) (uint8, error) {
// locked object could not be removed/marked with GC/expired
if objectLocked(tx, addr.Container(), addr.Object()) {
return 0
locked, err := objectLocked(ctx, r, addr.Container(), addr.Object())
if err != nil {
return 0, err
}
if locked {
return 0, nil
}
// we check only if the object is expired in the current
// epoch since it is considered the only corner case: the
// GC is expected to collect all the objects that have
// expired previously for less than the one epoch duration
st, err := inGraveyardWithKey(r, addr)
if err != nil {
return 0, err
}
if st > 0 {
return st, nil
}
expired := isExpiredWithAttribute(tx, objectV2.SysAttributeExpEpoch, addr, currEpoch)
if !expired {
expired = isExpiredWithAttribute(tx, objectV2.SysAttributeExpEpochNeoFS, addr, currEpoch)
expired, err := isExpired(ctx, r, addr, currEpoch)
if err != nil {
return 0, err
}
if expired {
return 3
return 3, nil
}
graveyardBkt := tx.Bucket(graveyardBucketName)
garbageBkt := tx.Bucket(garbageBucketName)
addrKey := addressKey(addr, make([]byte, addressKeySize))
return inGraveyardWithKey(addrKey, graveyardBkt, garbageBkt)
return 0, nil
}
func inGraveyardWithKey(addrKey []byte, graveyard, garbageBCK *bbolt.Bucket) uint8 {
if graveyard == nil {
// incorrect metabase state, does not make
// sense to check garbage bucket
return 0
func inGraveyardWithKey(r pebble.Reader, addr oid.Address) (uint8, error) {
v, err := valueSafe(r, graveyardKey(addr.Container(), addr.Object()))
if err != nil {
return 0, err
}
val := graveyard.Get(addrKey)
if val == nil {
if garbageBCK == nil {
// incorrect node state
return 0
}
val = garbageBCK.Get(addrKey)
if val != nil {
// object has been marked with GC
return 1
}
// neither in the graveyard
// nor was marked with GC mark
return 0
if v != nil {
return 2, nil
}
// object in the graveyard
return 2
}
// inBucket checks if key <key> is present in bucket <name>.
func inBucket(tx *bbolt.Tx, name, key []byte) bool {
bkt := tx.Bucket(name)
if bkt == nil {
return false
v, err = valueSafe(r, garbageKey(addr.Container(), addr.Object()))
if err != nil {
return 0, err
}
// using `get` as `exists`: https://github.com/boltdb/bolt/issues/321
val := bkt.Get(key)
return len(val) != 0
if v != nil {
return 1, nil
}
return 0, nil
}
// getSplitInfo returns SplitInfo structure from root index. Returns error
// if there is no `key` record in root index.
func getSplitInfo(tx *bbolt.Tx, cnr cid.ID, key []byte) (*objectSDK.SplitInfo, error) {
bucketName := rootBucketName(cnr, make([]byte, bucketKeySize))
rawSplitInfo := getFromBucket(tx, bucketName, key)
if len(rawSplitInfo) == 0 {
func getSplitInfo(r pebble.Reader, addr oid.Address) (*objectSDK.SplitInfo, error) {
rawSplitInfo, err := valueSafe(r, rootKey(addr.Container(), addr.Object()))
if err != nil {
return nil, err
}
if len(rawSplitInfo) == 0 || bytes.Equal(zeroValue, rawSplitInfo) {
return nil, ErrLackSplitInfo
}
splitInfo := objectSDK.NewSplitInfo()
err := splitInfo.Unmarshal(rawSplitInfo)
err = splitInfo.Unmarshal(rawSplitInfo)
if err != nil {
return nil, fmt.Errorf("can't unmarshal split info from root index: %w", err)
}
return splitInfo, nil
}
// if meets irregular object container in objs - returns its type, otherwise returns object.TypeRegular.
//
// firstIrregularObjectType(tx, cnr, obj) usage allows getting object type.
func firstIrregularObjectType(r pebble.Reader, idCnr cid.ID, objs ...oid.ID) (objectSDK.Type, error) {
for _, objID := range objs {
key := tombstoneKey(idCnr, objID)
v, err := valueSafe(r, key)
if err != nil {
return objectSDK.TypeRegular, err
}
if v != nil {
return objectSDK.TypeTombstone, nil
}
key = lockersKey(idCnr, objID)
v, err = valueSafe(r, key)
if err != nil {
return objectSDK.TypeRegular, err
}
if v != nil {
return objectSDK.TypeLock, nil
}
}
return objectSDK.TypeRegular, nil
}

View file

@ -1,18 +1,16 @@
package meta
import (
"bytes"
"context"
"errors"
"fmt"
"strconv"
"time"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
@ -44,44 +42,10 @@ func (db *DB) FilterExpired(ctx context.Context, epoch uint64, addresses []oid.A
}
result := make([]oid.Address, 0, len(addresses))
containerIDToObjectIDs := make(map[cid.ID][]oid.ID)
for _, addr := range addresses {
containerIDToObjectIDs[addr.Container()] = append(containerIDToObjectIDs[addr.Container()], addr.Object())
}
err := db.boltDB.View(func(tx *bbolt.Tx) error {
for containerID, objectIDs := range containerIDToObjectIDs {
select {
case <-ctx.Done():
return ErrInterruptIterator
default:
}
expiredNeoFS, err := selectExpiredObjectIDs(tx, objectV2.SysAttributeExpEpochNeoFS, epoch, containerID, objectIDs)
if err != nil {
return err
}
expiredSys, err := selectExpiredObjectIDs(tx, objectV2.SysAttributeExpEpoch, epoch, containerID, objectIDs)
if err != nil {
return err
}
for _, o := range expiredNeoFS {
var a oid.Address
a.SetContainer(containerID)
a.SetObject(o)
result = append(result, a)
}
for _, o := range expiredSys {
var a oid.Address
a.SetContainer(containerID)
a.SetObject(o)
result = append(result, a)
}
}
return nil
err := db.snapshot(func(s *pebble.Snapshot) error {
var e error
result, e = selectExpiredObjects(ctx, s, epoch, addresses)
return e
})
if err != nil {
return nil, metaerr.Wrap(err)
@ -90,76 +54,179 @@ func (db *DB) FilterExpired(ctx context.Context, epoch uint64, addresses []oid.A
return result, nil
}
func isExpiredWithAttribute(tx *bbolt.Tx, attr string, addr oid.Address, currEpoch uint64) bool {
// bucket with objects that have expiration attr
attrKey := make([]byte, bucketKeySize+len(attr))
expirationBucket := tx.Bucket(attributeBucketName(addr.Container(), attr, attrKey))
if expirationBucket != nil {
// bucket that contains objects that expire in the current epoch
prevEpochBkt := expirationBucket.Bucket([]byte(strconv.FormatUint(currEpoch-1, 10)))
if prevEpochBkt != nil {
rawOID := objectKey(addr.Object(), make([]byte, objectKeySize))
if prevEpochBkt.Get(rawOID) != nil {
return true
}
}
func isExpired(ctx context.Context, r pebble.Reader, addr oid.Address, currEpoch uint64) (bool, error) {
prefix := []byte{expiredPrefix}
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return false, err
}
return false
// iteration does in ascending order by expiration epoch.
// gc does expired objects collect every epoch, so here should be not so much items.
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return false, errors.Join(ctx.Err(), it.Close())
default:
}
expEpoch, err := expirationEpochFromExpiredKey(it.Key())
if err != nil {
return false, errors.Join(err, it.Close())
}
if expEpoch >= currEpoch {
return false, it.Close() // keys are ordered by epoch, so next items will be discarded anyway.
}
curAddr, err := addressFromExpiredKey(it.Key())
if err != nil {
return false, errors.Join(err, it.Close())
}
if curAddr == addr {
return true, it.Close()
}
}
return false, it.Close()
}
func selectExpiredObjectIDs(tx *bbolt.Tx, attr string, epoch uint64, containerID cid.ID, objectIDs []oid.ID) ([]oid.ID, error) {
result := make([]oid.ID, 0)
notResolved := make(map[oid.ID]struct{})
for _, oid := range objectIDs {
notResolved[oid] = struct{}{}
func selectExpiredObjects(ctx context.Context, r pebble.Reader, epoch uint64, objects []oid.Address) ([]oid.Address, error) {
result := make([]oid.Address, 0)
objMap := make(map[oid.Address]struct{})
for _, obj := range objects {
objMap[obj] = struct{}{}
}
expiredBuffer := make([]oid.ID, 0)
objectKeyBuffer := make([]byte, objectKeySize)
expirationBucketKey := make([]byte, bucketKeySize+len(attr))
expirationBucket := tx.Bucket(attributeBucketName(containerID, attr, expirationBucketKey))
if expirationBucket == nil {
return result, nil // all not expired
}
err := expirationBucket.ForEach(func(epochExpBucketKey, _ []byte) error {
bucketExpiresAfter, err := strconv.ParseUint(string(epochExpBucketKey), 10, 64)
if err != nil {
return fmt.Errorf("could not parse expiration epoch: %w", err)
} else if bucketExpiresAfter >= epoch {
return nil
}
epochExpirationBucket := expirationBucket.Bucket(epochExpBucketKey)
if epochExpirationBucket == nil {
return nil
}
expiredBuffer = expiredBuffer[:0]
for oid := range notResolved {
key := objectKey(oid, objectKeyBuffer)
if epochExpirationBucket.Get(key) != nil {
expiredBuffer = append(expiredBuffer, oid)
}
}
for _, oid := range expiredBuffer {
delete(notResolved, oid)
result = append(result, oid)
}
if len(notResolved) == 0 {
return errBreakBucketForEach
}
return nil
prefix := []byte{expiredPrefix}
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil && !errors.Is(err, errBreakBucketForEach) {
if err != nil {
return nil, err
}
return result, nil
// iteration does in ascending order by expiration epoch.
// gc does expired objects collect every epoch, so here should be not so much items.
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return nil, errors.Join(ctx.Err(), it.Close())
default:
}
expEpoch, err := expirationEpochFromExpiredKey(it.Key())
if err != nil {
return nil, errors.Join(err, it.Close())
}
if expEpoch >= epoch {
return result, it.Close() // keys are ordered by epoch, so next items will be discarded anyway.
}
addr, err := addressFromExpiredKey(it.Key())
if err != nil {
return nil, errors.Join(err, it.Close())
}
if _, ok := objMap[addr]; ok {
result = append(result, addr)
}
}
return result, it.Close()
}
// IterateExpired iterates over all objects in DB which are out of date
// relative to epoch. Locked objects are not included (do not confuse
// with objects of type LOCK).
//
// If h returns ErrInterruptIterator, nil returns immediately.
// Returns other errors of h directly.
func (db *DB) IterateExpired(ctx context.Context, epoch uint64, h ExpiredObjectHandler) error {
var (
startedAt = time.Now()
success = false
)
defer func() {
db.metrics.AddMethodDuration("IterateExpired", time.Since(startedAt), success)
}()
_, span := tracing.StartSpanFromContext(ctx, "metabase.IterateExpired",
trace.WithAttributes(
attribute.String("epoch", strconv.FormatUint(epoch, 10)),
))
defer span.End()
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return ErrDegradedMode
}
err := metaerr.Wrap(db.snapshot(func(s *pebble.Snapshot) error {
return iterateExpired(ctx, s, epoch, h)
}))
success = err == nil
return err
}
func iterateExpired(ctx context.Context, r pebble.Reader, epoch uint64, h ExpiredObjectHandler) error {
prefix := []byte{expiredPrefix}
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return err
}
// iteration does in ascending order by expiration epoch.
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return errors.Join(ctx.Err(), it.Close())
default:
}
expEpoch, err := expirationEpochFromExpiredKey(it.Key())
if err != nil {
return errors.Join(err, it.Close())
}
if expEpoch >= epoch {
return it.Close() // keys are ordered by epoch, so next items will be discarded anyway.
}
addr, err := addressFromExpiredKey(it.Key())
if err != nil {
return errors.Join(err, it.Close())
}
// Ignore locked objects.
//
// To slightly optimize performance we can check only REGULAR objects
// (only they can be locked), but it's more reliable.
isLocked, err := objectLocked(ctx, r, addr.Container(), addr.Object())
if err != nil {
return errors.Join(err, it.Close())
}
if isLocked {
continue
}
objType, err := firstIrregularObjectType(r, addr.Container(), addr.Object())
if err != nil {
return errors.Join(err, it.Close())
}
if err := h(&ExpiredObject{
typ: objType,
addr: addr,
}); err != nil {
if errors.Is(err, ErrInterruptIterator) {
return it.Close()
}
return errors.Join(err, it.Close())
}
}
return it.Close()
}

View file

@ -9,10 +9,9 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
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.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
@ -77,9 +76,8 @@ func (db *DB) Get(ctx context.Context, prm GetPrm) (res GetRes, err error) {
currEpoch := db.epochState.CurrentEpoch()
err = db.boltDB.View(func(tx *bbolt.Tx) error {
key := make([]byte, addressKeySize)
res.hdr, err = db.get(tx, prm.addr, key, true, prm.raw, currEpoch)
err = db.snapshot(func(s *pebble.Snapshot) error {
res.hdr, err = get(ctx, s, prm.addr, true, prm.raw, currEpoch)
return err
})
@ -87,9 +85,13 @@ func (db *DB) Get(ctx context.Context, prm GetPrm) (res GetRes, err error) {
return res, metaerr.Wrap(err)
}
func (db *DB) get(tx *bbolt.Tx, addr oid.Address, key []byte, checkStatus, raw bool, currEpoch uint64) (*objectSDK.Object, error) {
func get(ctx context.Context, r pebble.Reader, addr oid.Address, checkStatus, raw bool, currEpoch uint64) (*objectSDK.Object, error) {
if checkStatus {
switch objectStatus(tx, addr, currEpoch) {
st, err := objectStatus(ctx, r, addr, currEpoch)
if err != nil {
return nil, err
}
switch st {
case 1:
return nil, logicerr.Wrap(new(apistatus.ObjectNotFound))
case 2:
@ -98,78 +100,76 @@ func (db *DB) get(tx *bbolt.Tx, addr oid.Address, key []byte, checkStatus, raw b
return nil, ErrObjectIsExpired
}
}
key = objectKey(addr.Object(), key)
cnr := addr.Container()
obj := objectSDK.New()
bucketName := make([]byte, bucketKeySize)
// check in primary index
data := getFromBucket(tx, primaryBucketName(cnr, bucketName), key)
if len(data) != 0 {
data, err := valueSafe(r, primaryKey(addr.Container(), addr.Object()))
if err != nil {
return nil, err
}
if data != nil {
return obj, obj.Unmarshal(data)
}
data = getFromBucket(tx, ecInfoBucketName(cnr, bucketName), key)
if len(data) != 0 {
return nil, getECInfoError(tx, cnr, data)
children, err := selectByPrefixBatch(ctx, r, ecInfoLongKeyPrefix(addr.Container(), addr.Object()), 1) // try to found any child
if err != nil {
return nil, err
}
if len(children) > 0 {
return nil, getECInfoError(ctx, r, addr)
}
// if not found then check in tombstone index
data = getFromBucket(tx, tombstoneBucketName(cnr, bucketName), key)
if len(data) != 0 {
data, err = valueSafe(r, tombstoneKey(addr.Container(), addr.Object()))
if err != nil {
return nil, err
}
if data != nil {
return obj, obj.Unmarshal(data)
}
// if not found then check in locker index
data = getFromBucket(tx, bucketNameLockers(cnr, bucketName), key)
if len(data) != 0 {
data, err = valueSafe(r, lockersKey(addr.Container(), addr.Object()))
if err != nil {
return nil, err
}
if data != nil {
return obj, obj.Unmarshal(data)
}
// if not found then check if object is a virtual
return getVirtualObject(tx, cnr, key, raw)
return getVirtualObject(ctx, r, addr, raw)
}
func getFromBucket(tx *bbolt.Tx, name, key []byte) []byte {
bkt := tx.Bucket(name)
if bkt == nil {
return nil
}
return bkt.Get(key)
}
func getVirtualObject(tx *bbolt.Tx, cnr cid.ID, key []byte, raw bool) (*objectSDK.Object, error) {
func getVirtualObject(ctx context.Context, r pebble.Reader, addr oid.Address, raw bool) (*objectSDK.Object, error) {
if raw {
return nil, getSplitInfoError(tx, cnr, key)
return nil, getSplitInfoError(r, addr)
}
bucketName := make([]byte, bucketKeySize)
parentBucket := tx.Bucket(parentBucketName(cnr, bucketName))
if parentBucket == nil {
return nil, logicerr.Wrap(new(apistatus.ObjectNotFound))
}
relativeLst, err := decodeList(parentBucket.Get(key))
binObjIDs, err := selectByPrefixBatch(ctx, r, parentKeyLongPrefix(addr.Container(), addr.Object()), 1)
if err != nil {
return nil, err
}
if len(relativeLst) == 0 { // this should never happen though
if len(binObjIDs) == 0 { // this should never happen though
return nil, logicerr.Wrap(new(apistatus.ObjectNotFound))
}
// pick last item, for now there is not difference which address to pick
// but later list might be sorted so first or last value can be more
// prioritized to choose
virtualOID := relativeLst[len(relativeLst)-1]
data := getFromBucket(tx, primaryBucketName(cnr, bucketName), virtualOID)
child := objectSDK.New()
err = child.Unmarshal(data)
phyObjAddr, err := addressOfTargetFromParentKey(binObjIDs[0])
if err != nil {
return nil, err
}
data, err := valueSafe(r, primaryKey(phyObjAddr.Container(), phyObjAddr.Object()))
if err != nil {
return nil, err
}
if data == nil { // this should never happen though #2
return nil, logicerr.Wrap(new(apistatus.ObjectNotFound))
}
child := objectSDK.New()
if err := child.Unmarshal(data); err != nil {
return nil, fmt.Errorf("can't unmarshal child with parent: %w", err)
}
@ -182,8 +182,8 @@ func getVirtualObject(tx *bbolt.Tx, cnr cid.ID, key []byte, raw bool) (*objectSD
return par, nil
}
func getSplitInfoError(tx *bbolt.Tx, cnr cid.ID, key []byte) error {
splitInfo, err := getSplitInfo(tx, cnr, key)
func getSplitInfoError(r pebble.Reader, addr oid.Address) error {
splitInfo, err := getSplitInfo(r, addr)
if err == nil {
return logicerr.Wrap(objectSDK.NewSplitInfoError(splitInfo))
}
@ -191,27 +191,50 @@ func getSplitInfoError(tx *bbolt.Tx, cnr cid.ID, key []byte) error {
return logicerr.Wrap(new(apistatus.ObjectNotFound))
}
func getECInfoError(tx *bbolt.Tx, cnr cid.ID, data []byte) error {
keys, err := decodeList(data)
if err != nil {
return err
}
ecInfo := objectSDK.NewECInfo()
for _, key := range keys {
// check in primary index
ojbData := getFromBucket(tx, primaryBucketName(cnr, make([]byte, bucketKeySize)), key)
if len(ojbData) != 0 {
obj := objectSDK.New()
if err := obj.Unmarshal(ojbData); err != nil {
func getECInfoError(ctx context.Context, r pebble.Reader, addr oid.Address) error {
var chunkAddresses []oid.Address
for {
keys, err := selectByPrefixBatch(ctx, r, ecInfoLongKeyPrefix(addr.Container(), addr.Object()), batchSize)
if err != nil {
return err
}
for _, key := range keys {
chunkAddress, err := addressOfChunkFromECInfoKey(key)
if err != nil {
return err
}
chunk := objectSDK.ECChunk{}
id, _ := obj.ID()
chunk.SetID(id)
chunk.Index = obj.ECHeader().Index()
chunk.Total = obj.ECHeader().Total()
ecInfo.AddChunk(chunk)
chunkAddresses = append(chunkAddresses, chunkAddress)
}
if len(keys) < batchSize {
break
}
}
ecInfo := objectSDK.NewECInfo()
for _, chunkAddress := range chunkAddresses {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
objData, err := valueSafe(r, primaryKey(chunkAddress.Container(), chunkAddress.Object()))
if err != nil {
return err
}
if len(objData) == 0 {
continue
}
obj := objectSDK.New()
if err := obj.Unmarshal(objData); err != nil {
return err
}
chunk := objectSDK.ECChunk{}
id, _ := obj.ID()
chunk.SetID(id)
chunk.Index = obj.ECHeader().Index()
chunk.Total = obj.ECHeader().Total()
ecInfo.AddChunk(chunk)
}
return logicerr.Wrap(objectSDK.NewECInfoError(ecInfo))
}

View file

@ -8,7 +8,6 @@ import (
"os"
"runtime"
"testing"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
@ -216,10 +215,7 @@ func BenchmarkGet(b *testing.B) {
func benchmarkGet(b *testing.B, numOfObj int) {
prepareDb := func(batchSize int) (*meta.DB, []oid.Address) {
db := newDB(b,
meta.WithMaxBatchSize(batchSize),
meta.WithMaxBatchDelay(10*time.Millisecond),
)
db := newDB(b)
defer func() { require.NoError(b, db.Close()) }()
addrs := make([]oid.Address, 0, numOfObj)

View file

@ -9,8 +9,9 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
)
// GarbageObject represents descriptor of the
@ -80,8 +81,8 @@ func (db *DB) IterateOverGarbage(ctx context.Context, p GarbageIterationPrm) err
return ErrDegradedMode
}
err := metaerr.Wrap(db.boltDB.View(func(tx *bbolt.Tx) error {
return db.iterateDeletedObj(tx, gcHandler{p.h}, p.offset)
err := metaerr.Wrap(db.snapshot(func(s *pebble.Snapshot) error {
return db.iterateDeletedObj(ctx, s, gcHandler{p.h}, p.offset)
}))
success = err == nil
return err
@ -160,8 +161,8 @@ func (db *DB) IterateOverGraveyard(ctx context.Context, p GraveyardIterationPrm)
return ErrDegradedMode
}
return metaerr.Wrap(db.boltDB.View(func(tx *bbolt.Tx) error {
return db.iterateDeletedObj(tx, graveyardHandler{p.h}, p.offset)
return metaerr.Wrap(db.snapshot(func(s *pebble.Snapshot) error {
return db.iterateDeletedObj(ctx, s, graveyardHandler{p.h}, p.offset)
}))
}
@ -195,49 +196,66 @@ func (g graveyardHandler) handleKV(k, v []byte) error {
return g.h(o)
}
func (db *DB) iterateDeletedObj(tx *bbolt.Tx, h kvHandler, offset *oid.Address) error {
var bkt *bbolt.Bucket
func (db *DB) iterateDeletedObj(ctx context.Context, r pebble.Reader, h kvHandler, offset *oid.Address) error {
var prefix []byte
switch t := h.(type) {
case graveyardHandler:
bkt = tx.Bucket(graveyardBucketName)
prefix = []byte{graveyardPrefix}
case gcHandler:
bkt = tx.Bucket(garbageBucketName)
prefix = []byte{garbagePrefix}
default:
panic(fmt.Sprintf("metabase: unknown iteration object hadler: %T", t))
}
var seekKey []byte
if offset != nil {
cidBytes := make([]byte, cidSize)
offset.Container().Encode(cidBytes)
oidBytes := make([]byte, objectKeySize)
offset.Object().Encode(oidBytes)
seekKey = append(prefix, cidBytes...)
seekKey = append(seekKey, oidBytes...)
}
c := bkt.Cursor()
var k, v []byte
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return err
}
if offset == nil {
k, v = c.First()
var v bool
if len(seekKey) > 0 {
v = it.SeekGE(seekKey)
} else {
rawAddr := addressKey(*offset, make([]byte, addressKeySize))
k, v = c.Seek(rawAddr)
if bytes.Equal(k, rawAddr) {
// offset was found, move
// cursor to the next element
k, v = c.Next()
}
v = it.First()
}
for ; k != nil; k, v = c.Next() {
err := h.handleKV(k, v)
if err != nil {
for ; v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return errors.Join(ctx.Err(), it.Close())
default:
}
if bytes.Equal(it.Key(), seekKey) {
continue
}
key := bytes.Clone(it.Key())
value := bytes.Clone(it.Value())
if err = h.handleKV(key, value); err != nil {
if errors.Is(err, ErrInterruptIterator) {
return nil
return it.Close()
}
return err
return errors.Join(err, it.Close())
}
}
return nil
return it.Close()
}
func garbageFromKV(k []byte) (res GarbageObject, err error) {
err = decodeAddressFromKey(&res.addr, k)
res.addr, err = addressFromGarbageKey(k)
if err != nil {
err = fmt.Errorf("could not parse address: %w", err)
}
@ -246,15 +264,44 @@ func garbageFromKV(k []byte) (res GarbageObject, err error) {
}
func graveFromKV(k, v []byte) (res TombstonedObject, err error) {
if err = decodeAddressFromKey(&res.addr, k); err != nil {
res.addr, err = addressFromGraveyardKey(k)
if err != nil {
err = fmt.Errorf("decode tombstone target from key: %w", err)
} else if err = decodeAddressFromKey(&res.tomb, v); err != nil {
err = fmt.Errorf("decode tombstone address from value: %w", err)
return
}
res.tomb, err = decodeAddressFromGrave(v)
if err != nil {
err = fmt.Errorf("decode tombstone address from value: %w", err)
return
}
return
}
func encodeAddressToGrave(addr oid.Address) []byte {
value := make([]byte, cidSize+objectKeySize)
addr.Container().Encode(value)
addr.Object().Encode(value[cidSize:])
return value
}
func decodeAddressFromGrave(v []byte) (oid.Address, error) {
if len(v) != cidSize+objectKeySize {
return oid.Address{}, errInvalidValueLenght
}
var cont cid.ID
if err := cont.Decode(v[:cidSize]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode container ID: %w", err)
}
var obj oid.ID
if err := obj.Decode(v[cidSize:]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode object ID: %w", err)
}
var result oid.Address
result.SetContainer(cont)
result.SetObject(obj)
return result, nil
}
// DropGraves deletes tombstoned objects from the
// graveyard bucket.
//
@ -280,16 +327,15 @@ func (db *DB) DropGraves(ctx context.Context, tss []TombstonedObject) error {
return ErrReadOnlyMode
}
buf := make([]byte, addressKeySize)
return db.boltDB.Update(func(tx *bbolt.Tx) error {
bkt := tx.Bucket(graveyardBucketName)
if bkt == nil {
return nil
}
var contIDs []cid.ID
for _, to := range tss {
contIDs = append(contIDs, to.tomb.Container())
}
defer db.guard.LockContainerIDs(contIDs)()
return db.batch(func(b *pebble.Batch) error {
for _, ts := range tss {
err := bkt.Delete(addressKey(ts.Address(), buf))
err := b.Delete(graveyardKey(ts.Address().Container(), ts.Address().Object()), pebble.Sync)
if err != nil {
return err
}

View file

@ -1,65 +0,0 @@
package meta
import (
"crypto/rand"
"math"
mrand "math/rand"
"testing"
"time"
"github.com/nspcc-dev/neo-go/pkg/io"
"github.com/stretchr/testify/require"
)
func Test_getVarUint(t *testing.T) {
data := make([]byte, 10)
for _, val := range []uint64{0, 0xfc, 0xfd, 0xfffe, 0xffff, 0xfffffffe, 0xffffffff, math.MaxUint64} {
expSize := io.PutVarUint(data, val)
actual, actSize, err := getVarUint(data)
require.NoError(t, err)
require.Equal(t, val, actual)
require.Equal(t, expSize, actSize, "value: %x", val)
_, _, err = getVarUint(data[:expSize-1])
require.Error(t, err)
}
}
func Test_decodeList(t *testing.T) {
t.Run("empty", func(t *testing.T) {
lst, err := decodeList(nil)
require.NoError(t, err)
require.True(t, len(lst) == 0)
})
t.Run("empty, 0 len", func(t *testing.T) {
lst, err := decodeList([]byte{0})
require.NoError(t, err)
require.True(t, len(lst) == 0)
})
t.Run("bad len", func(t *testing.T) {
_, err := decodeList([]byte{0xfe})
require.Error(t, err)
})
t.Run("random", func(t *testing.T) {
r := mrand.New(mrand.NewSource(time.Now().Unix()))
expected := make([][]byte, 20)
for i := range expected {
expected[i] = make([]byte, r.Uint32()%10)
rand.Read(expected[i])
}
data, err := encodeList(expected)
require.NoError(t, err)
actual, err := decodeList(data)
require.NoError(t, err)
require.Equal(t, expected, actual)
t.Run("unexpected EOF", func(t *testing.T) {
for i := 1; i < len(data)-1; i++ {
_, err := decodeList(data[:i])
require.Error(t, err)
}
})
})
}

View file

@ -15,7 +15,7 @@ import (
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.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
)
// InhumePrm encapsulates parameters for Inhume operation.
@ -143,8 +143,6 @@ func (p *InhumePrm) SetForceGCMark() {
p.forceRemoval = true
}
var errBreakBucketForEach = errors.New("bucket ForEach break")
// ErrLockObjectRemoval is returned when inhume operation is being
// performed on lock object, and it is not a forced object removal.
var ErrLockObjectRemoval = logicerr.New("lock object removal")
@ -180,9 +178,19 @@ func (db *DB) Inhume(ctx context.Context, prm InhumePrm) (InhumeRes, error) {
res := InhumeRes{
inhumedByCnrID: make(map[cid.ID]ObjectCounters),
}
var containerIDs []cid.ID
if prm.tomb != nil {
containerIDs = append(containerIDs, prm.tomb.Container())
}
for _, a := range prm.target {
containerIDs = append(containerIDs, a.Container())
}
defer db.guard.LockContainerIDs(containerIDs)()
currEpoch := db.epochState.CurrentEpoch()
err := db.boltDB.Update(func(tx *bbolt.Tx) error {
return db.inhumeTx(tx, currEpoch, prm, &res)
err := db.batch(func(b *pebble.Batch) error {
return db.inhumeTx(ctx, b, currEpoch, prm, &res)
})
success = err == nil
if success {
@ -195,48 +203,31 @@ func (db *DB) Inhume(ctx context.Context, prm InhumePrm) (InhumeRes, error) {
return res, metaerr.Wrap(err)
}
func (db *DB) inhumeTx(tx *bbolt.Tx, epoch uint64, prm InhumePrm, res *InhumeRes) error {
garbageBKT := tx.Bucket(garbageBucketName)
graveyardBKT := tx.Bucket(graveyardBucketName)
bkt, value, err := db.getInhumeTargetBucketAndValue(garbageBKT, graveyardBKT, &prm)
func (db *DB) inhumeTx(ctx context.Context, b *pebble.Batch, epoch uint64, prm InhumePrm, res *InhumeRes) error {
keyer, value, err := getInhumeTargetBucketAndValue(b, prm)
if err != nil {
return err
}
buf := make([]byte, addressKeySize)
for i := range prm.target {
id := prm.target[i].Object()
cnr := prm.target[i].Container()
// prevent locked objects to be inhumed
if !prm.forceRemoval && objectLocked(tx, cnr, id) {
return new(apistatus.ObjectLocked)
}
var lockWasChecked bool
// prevent lock objects to be inhumed
// if `Inhume` was called not with the
// `WithForceGCMark` option
var ecErr *objectSDK.ECInfoError
if !prm.forceRemoval {
if isLockObject(tx, cnr, id) {
return ErrLockObjectRemoval
if err := checkNotLockerOrLocked(ctx, b, cnr, id); err != nil {
return err
}
lockWasChecked = true
}
obj, err := db.get(tx, prm.target[i], buf, false, true, epoch)
targetKey := addressKey(prm.target[i], buf)
var ecErr *objectSDK.ECInfoError
obj, err := get(ctx, b, prm.target[i], false, true, epoch)
if err == nil {
err = db.updateDeleteInfo(tx, garbageBKT, graveyardBKT, targetKey, cnr, obj, res)
err = db.updateDeleteInfo(b, prm.target[i], obj, res)
if err != nil {
return err
}
} else if errors.As(err, &ecErr) {
err = db.inhumeECInfo(tx, epoch, prm.tomb, res, garbageBKT, graveyardBKT, ecErr.ECInfo(), cnr, bkt, value, targetKey)
err = db.inhumeECInfo(ctx, b, epoch, prm.tomb, keyer, value, res, ecErr.ECInfo(), cnr)
if err != nil {
return err
}
@ -244,18 +235,18 @@ func (db *DB) inhumeTx(tx *bbolt.Tx, epoch uint64, prm InhumePrm, res *InhumeRes
if prm.tomb != nil {
var isTomb bool
isTomb, err = db.markAsGC(graveyardBKT, garbageBKT, targetKey)
isTomb, err = markAsGC(b, prm.target[i])
if err != nil {
return err
}
if isTomb {
continue
}
}
// consider checking if target is already in graveyard?
err = bkt.Put(targetKey, value)
key := keyer(prm.target[i])
err = b.Set(key, value, pebble.Sync)
if err != nil {
return err
}
@ -268,22 +259,24 @@ func (db *DB) inhumeTx(tx *bbolt.Tx, epoch uint64, prm InhumePrm, res *InhumeRes
// the LOCK type
continue
}
if isLockObject(tx, cnr, id) {
isLock, err := isLockObject(b, cnr, id)
if err != nil {
return err
}
if isLock {
res.deletedLockObj = append(res.deletedLockObj, prm.target[i])
}
}
}
return db.applyInhumeResToCounters(tx, res)
return db.applyInhumeResToCounters(b, res)
}
func (db *DB) inhumeECInfo(tx *bbolt.Tx, epoch uint64, tomb *oid.Address, res *InhumeRes,
garbageBKT *bbolt.Bucket, graveyardBKT *bbolt.Bucket,
ecInfo *objectSDK.ECInfo, cnr cid.ID, targetBucket *bbolt.Bucket, value []byte, targetKey []byte,
func (db *DB) inhumeECInfo(ctx context.Context, b *pebble.Batch, epoch uint64, tomb *oid.Address,
keyer func(addr oid.Address) []byte, value []byte,
res *InhumeRes, ecInfo *objectSDK.ECInfo, cnr cid.ID,
) error {
for _, chunk := range ecInfo.Chunks {
chunkBuf := make([]byte, addressKeySize)
var chunkAddr oid.Address
chunkAddr.SetContainer(cnr)
var chunkID oid.ID
@ -292,22 +285,22 @@ func (db *DB) inhumeECInfo(tx *bbolt.Tx, epoch uint64, tomb *oid.Address, res *I
return err
}
chunkAddr.SetObject(chunkID)
chunkObj, err := db.get(tx, chunkAddr, chunkBuf, false, true, epoch)
chunkObj, err := get(ctx, b, chunkAddr, false, true, epoch)
if err != nil {
return err
}
err = db.updateDeleteInfo(tx, garbageBKT, graveyardBKT, targetKey, cnr, chunkObj, res)
err = db.updateDeleteInfo(b, chunkAddr, chunkObj, res)
if err != nil {
return err
}
chunkKey := addressKey(chunkAddr, chunkBuf)
if tomb != nil {
_, err = db.markAsGC(graveyardBKT, garbageBKT, chunkKey)
_, err = markAsGC(b, chunkAddr)
if err != nil {
return err
}
}
err = targetBucket.Put(chunkKey, value)
key := keyer(chunkAddr)
err = b.Set(key, value, pebble.Sync)
if err != nil {
return err
}
@ -315,15 +308,38 @@ func (db *DB) inhumeECInfo(tx *bbolt.Tx, epoch uint64, tomb *oid.Address, res *I
return nil
}
func (db *DB) applyInhumeResToCounters(tx *bbolt.Tx, res *InhumeRes) error {
if err := db.updateShardObjectCounter(tx, logical, res.LogicInhumed(), false); err != nil {
func checkNotLockerOrLocked(ctx context.Context, r pebble.Reader, cnr cid.ID, id oid.ID) error {
// prevent locked objects to be inhumed
locked, err := objectLocked(ctx, r, cnr, id)
if err != nil {
return err
}
if err := db.updateShardObjectCounter(tx, user, res.UserInhumed(), false); err != nil {
if locked {
return new(apistatus.ObjectLocked)
}
// prevent lock objects to be inhumed
// if `Inhume` was called not with the
// `WithForceGCMark` option
isLock, err := isLockObject(r, cnr, id)
if err != nil {
return err
}
if isLock {
return ErrLockObjectRemoval
}
return nil
}
return db.updateContainerCounter(tx, res.inhumedByCnrID, false)
func (db *DB) applyInhumeResToCounters(b *pebble.Batch, res *InhumeRes) error {
counters := make(map[cid.ID]objectCounterValue, len(res.inhumedByCnrID))
for contID, inhumed := range res.inhumedByCnrID {
counters[contID] = objectCounterValue{
Logic: -1 * int64(inhumed.Logic),
Phy: -1 * int64(inhumed.Phy),
User: -1 * int64(inhumed.User),
}
}
return updateContainerCounter(b, counters)
}
// getInhumeTargetBucketAndValue return target bucket to store inhume result and value that will be put in the bucket.
@ -336,35 +352,36 @@ func (db *DB) applyInhumeResToCounters(tx *bbolt.Tx, res *InhumeRes) error {
// 1. tombstone address if Inhume was called with
// a Tombstone
// 2. zeroValue if Inhume was called with a GC mark
func (db *DB) getInhumeTargetBucketAndValue(garbageBKT, graveyardBKT *bbolt.Bucket, prm *InhumePrm) (targetBucket *bbolt.Bucket, value []byte, err error) {
func getInhumeTargetBucketAndValue(b *pebble.Batch, prm InhumePrm) (key func(addr oid.Address) []byte, value []byte, err error) {
if prm.tomb != nil {
targetBucket = graveyardBKT
tombKey := addressKey(*prm.tomb, make([]byte, addressKeySize))
// it is forbidden to have a tomb-on-tomb in FrostFS,
// so graveyard keys must not be addresses of tombstones
data := targetBucket.Get(tombKey)
if data != nil {
err := targetBucket.Delete(tombKey)
tombKey := graveyardKey(prm.tomb.Container(), prm.tomb.Object())
v, err := valueSafe(b, tombKey)
if err != nil {
return nil, nil, err
}
if v != nil {
err := b.Delete(tombKey, pebble.Sync)
if err != nil {
return nil, nil, fmt.Errorf("could not remove grave with tombstone key: %w", err)
}
}
value = tombKey
} else {
targetBucket = garbageBKT
value = zeroValue
return func(addr oid.Address) []byte {
return graveyardKey(addr.Container(), addr.Object())
}, encodeAddressToGrave(*prm.tomb), nil
}
return targetBucket, value, nil
return func(addr oid.Address) []byte {
return garbageKey(addr.Container(), addr.Object())
}, zeroValue, nil
}
func (db *DB) markAsGC(graveyardBKT, garbageBKT *bbolt.Bucket, key []byte) (bool, error) {
targetIsTomb, err := isTomb(graveyardBKT, key)
func markAsGC(b *pebble.Batch, addr oid.Address) (bool, error) {
targetIsTomb, err := isTomb(b, addr)
if err != nil {
return false, err
}
// do not add grave if target is a tombstone
if targetIsTomb {
return true, nil
@ -372,19 +389,23 @@ func (db *DB) markAsGC(graveyardBKT, garbageBKT *bbolt.Bucket, key []byte) (bool
// if tombstone appears object must be
// additionally marked with GC
return false, garbageBKT.Put(key, zeroValue)
key := garbageKey(addr.Container(), addr.Object())
return false, b.Set(key, zeroValue, pebble.Sync)
}
func (db *DB) updateDeleteInfo(tx *bbolt.Tx, garbageBKT, graveyardBKT *bbolt.Bucket, targetKey []byte, cnr cid.ID, obj *objectSDK.Object, res *InhumeRes) error {
containerID, _ := obj.ContainerID()
if inGraveyardWithKey(targetKey, graveyardBKT, garbageBKT) == 0 {
res.storeDeletionInfo(containerID, obj.PayloadSize(), IsUserObject(obj))
func (db *DB) updateDeleteInfo(b *pebble.Batch, addr oid.Address, obj *objectSDK.Object, res *InhumeRes) error {
st, err := inGraveyardWithKey(b, addr)
if err != nil {
return err
}
if st == 0 {
res.storeDeletionInfo(addr.Container(), obj.PayloadSize(), IsUserObject(obj))
}
// if object is stored, and it is regular object then update bucket
// with container size estimations
if obj.Type() == objectSDK.TypeRegular {
err := changeContainerSize(tx, cnr, obj.PayloadSize(), false)
err := changeContainerSize(b, addr.Container(), -1*int64(obj.PayloadSize()))
if err != nil {
return err
}
@ -392,25 +413,30 @@ func (db *DB) updateDeleteInfo(tx *bbolt.Tx, garbageBKT, graveyardBKT *bbolt.Buc
return nil
}
func isTomb(graveyardBucket *bbolt.Bucket, key []byte) (bool, error) {
func isTomb(r pebble.Reader, addr oid.Address) (bool, error) {
targetIsTomb := false
expectedValue := make([]byte, cidSize+objectKeySize)
addr.Container().Encode(expectedValue)
addr.Object().Encode(expectedValue[cidSize:])
prefix := []byte{graveyardPrefix}
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return false, err
}
// iterate over graveyard and check if target address
// is the address of tombstone in graveyard.
err := graveyardBucket.ForEach(func(_, v []byte) error {
// check if graveyard has record with key corresponding
// to tombstone address (at least one)
targetIsTomb = bytes.Equal(v, key)
// check if graveyard has record with key corresponding
// to tombstone address (at least one)
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
targetIsTomb = bytes.Equal(expectedValue, it.Value())
if targetIsTomb {
// break bucket iterator
return errBreakBucketForEach
return true, it.Close()
}
return nil
})
if err != nil && !errors.Is(err, errBreakBucketForEach) {
return false, err
}
return targetIsTomb, nil
return false, it.Close()
}

View file

@ -1,23 +1,18 @@
package meta
import (
"bytes"
"context"
"errors"
"fmt"
"strconv"
"time"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
"github.com/cockroachdb/pebble"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.etcd.io/bbolt"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
// ExpiredObject is a descriptor of expired object from DB.
@ -44,99 +39,7 @@ type ExpiredObjectHandler func(*ExpiredObject) error
// as a "break" keyword.
var ErrInterruptIterator = logicerr.New("iterator is interrupted")
// IterateExpired iterates over all objects in DB which are out of date
// relative to epoch. Locked objects are not included (do not confuse
// with objects of type LOCK).
//
// If h returns ErrInterruptIterator, nil returns immediately.
// Returns other errors of h directly.
func (db *DB) IterateExpired(ctx context.Context, epoch uint64, h ExpiredObjectHandler) error {
var (
startedAt = time.Now()
success = false
)
defer func() {
db.metrics.AddMethodDuration("IterateExpired", time.Since(startedAt), success)
}()
_, span := tracing.StartSpanFromContext(ctx, "metabase.IterateExpired",
trace.WithAttributes(
attribute.String("epoch", strconv.FormatUint(epoch, 10)),
))
defer span.End()
db.modeMtx.RLock()
defer db.modeMtx.RUnlock()
if db.mode.NoMetabase() {
return ErrDegradedMode
}
err := metaerr.Wrap(db.boltDB.View(func(tx *bbolt.Tx) error {
return db.iterateExpired(tx, epoch, h)
}))
success = err == nil
return err
}
func (db *DB) iterateExpired(tx *bbolt.Tx, epoch uint64, h ExpiredObjectHandler) error {
err := tx.ForEach(func(name []byte, b *bbolt.Bucket) error {
cidBytes := cidFromAttributeBucket(name, objectV2.SysAttributeExpEpoch)
if cidBytes == nil {
cidBytes = cidFromAttributeBucket(name, objectV2.SysAttributeExpEpochNeoFS)
if cidBytes == nil {
return nil
}
}
var cnrID cid.ID
err := cnrID.Decode(cidBytes)
if err != nil {
return fmt.Errorf("could not parse container ID of expired bucket: %w", err)
}
return b.ForEachBucket(func(expKey []byte) error {
bktExpired := b.Bucket(expKey)
expiresAfter, err := strconv.ParseUint(string(expKey), 10, 64)
if err != nil {
return fmt.Errorf("could not parse expiration epoch: %w", err)
} else if expiresAfter >= epoch {
return nil
}
return bktExpired.ForEach(func(idKey, _ []byte) error {
var id oid.ID
err = id.Decode(idKey)
if err != nil {
return fmt.Errorf("could not parse ID of expired object: %w", err)
}
// Ignore locked objects.
//
// To slightly optimize performance we can check only REGULAR objects
// (only they can be locked), but it's more reliable.
if objectLocked(tx, cnrID, id) {
return nil
}
var addr oid.Address
addr.SetContainer(cnrID)
addr.SetObject(id)
return h(&ExpiredObject{
typ: firstIrregularObjectType(tx, cnrID, idKey),
addr: addr,
})
})
})
})
if errors.Is(err, ErrInterruptIterator) {
err = nil
}
return err
}
var errInvalidAttributeKey = errors.New("invalid userr attribute key")
// IterateCoveredByTombstones iterates over all objects in DB which are covered
// by tombstone with string address from tss. Locked objects are not included
@ -164,69 +67,91 @@ func (db *DB) IterateCoveredByTombstones(ctx context.Context, tss map[string]oid
return ErrDegradedMode
}
return db.boltDB.View(func(tx *bbolt.Tx) error {
return db.iterateCoveredByTombstones(tx, tss, h)
return db.snapshot(func(s *pebble.Snapshot) error {
return db.iterateCoveredByTombstones(ctx, s, tss, h)
})
}
func (db *DB) iterateCoveredByTombstones(tx *bbolt.Tx, tss map[string]oid.Address, h func(oid.Address) error) error {
bktGraveyard := tx.Bucket(graveyardBucketName)
err := bktGraveyard.ForEach(func(k, v []byte) error {
var addr oid.Address
if err := decodeAddressFromKey(&addr, v); err != nil {
return err
}
if _, ok := tss[addr.EncodeToString()]; ok {
var addr oid.Address
err := decodeAddressFromKey(&addr, k)
if err != nil {
return fmt.Errorf("could not parse address of the object under tombstone: %w", err)
}
if objectLocked(tx, addr.Container(), addr.Object()) {
return nil
}
return h(addr)
}
return nil
func (db *DB) iterateCoveredByTombstones(ctx context.Context, r pebble.Reader, tss map[string]oid.Address, h func(oid.Address) error) error {
prefix := []byte{graveyardPrefix}
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if errors.Is(err, ErrInterruptIterator) {
err = nil
if err != nil {
return err
}
return err
}
func iteratePhyObjects(tx *bbolt.Tx, f func(cid.ID, oid.ID, *objectSDK.Object) error) error {
var cid cid.ID
var oid oid.ID
obj := objectSDK.New()
return tx.ForEach(func(name []byte, b *bbolt.Bucket) error {
b58CID, postfix := parseContainerIDWithPrefix(&cid, name)
if len(b58CID) == 0 {
return nil
}
switch postfix {
case primaryPrefix,
lockersPrefix,
tombstonePrefix:
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return errors.Join(ctx.Err(), it.Close())
default:
return nil
}
return b.ForEach(func(k, v []byte) error {
if oid.Decode(k) == nil && obj.Unmarshal(v) == nil {
return f(cid, oid, obj)
}
tombstoneAddress, err := decodeAddressFromGrave(it.Value())
if err != nil {
return errors.Join(err, it.Close())
}
if _, ok := tss[tombstoneAddress.EncodeToString()]; !ok {
continue
}
return nil
})
})
objectAddress, err := addressFromGraveyardKey(it.Key())
if err != nil {
return errors.Join(err, it.Close())
}
isLocked, err := objectLocked(ctx, r, objectAddress.Container(), objectAddress.Object())
if err != nil {
return errors.Join(err, it.Close())
}
if isLocked {
continue
}
if err := h(objectAddress); err != nil {
if errors.Is(err, ErrInterruptIterator) {
return it.Close()
}
return errors.Join(err, it.Close())
}
}
return it.Close()
}
func iteratePhyObjects(r pebble.Reader, f func(cid.ID, oid.ID, *objectSDK.Object) error) error {
if err := iteratePhyObjectsWithPrefix(r, primaryPrefix, f); err != nil {
return err
}
if err := iteratePhyObjectsWithPrefix(r, lockersPrefix, f); err != nil {
return err
}
if err := iteratePhyObjectsWithPrefix(r, tombstonePrefix, f); err != nil {
return err
}
return nil
}
func iteratePhyObjectsWithPrefix(r pebble.Reader, typePrefix byte, f func(cid.ID, oid.ID, *objectSDK.Object) error) error {
prefix := []byte{typePrefix}
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return err
}
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
addr, err := addressFromKey(typePrefix, it.Key())
if err != nil {
return errors.Join(err, it.Close())
}
obj := objectSDK.New()
if err := obj.Unmarshal(it.Value()); err != nil {
return errors.Join(err, it.Close())
}
if err := f(addr.Container(), addr.Object(), obj); err != nil {
return errors.Join(err, it.Close())
}
}
return it.Close()
}

View file

@ -2,16 +2,17 @@ package meta
import (
"context"
"fmt"
"slices"
"time"
objectcore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
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.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
@ -21,10 +22,33 @@ import (
// cursor. Use nil cursor object to start listing again.
var ErrEndOfListing = logicerr.New("end of object listing")
type listPrefix struct {
prefix []byte
keyParser func(k []byte) (oid.Address, error)
objectType objectSDK.Type
}
var listPrefixes = []listPrefix{
{
prefix: []byte{primaryPrefix},
keyParser: addressFromPrimaryKey,
objectType: objectSDK.TypeRegular,
},
{
prefix: []byte{lockersPrefix},
keyParser: addressFromLockersKey,
objectType: objectSDK.TypeLock,
},
{
prefix: []byte{tombstonePrefix},
keyParser: addressFromTombstoneKey,
objectType: objectSDK.TypeTombstone,
},
}
// Cursor is a type for continuous object listing.
type Cursor struct {
bucketName []byte
inBucketOffset []byte
lastKey []byte
}
// ListPrm contains parameters for ListWithCursor operation.
@ -89,173 +113,109 @@ func (db *DB) ListWithCursor(ctx context.Context, prm ListPrm) (res ListRes, err
return res, ErrDegradedMode
}
result := make([]objectcore.Info, 0, prm.count)
if prm.count == 0 {
return ListRes{}, ErrEndOfListing
}
err = db.boltDB.View(func(tx *bbolt.Tx) error {
res.addrList, res.cursor, err = db.listWithCursor(tx, result, prm.count, prm.cursor)
err = db.snapshot(func(s *pebble.Snapshot) error {
res.addrList, res.cursor, err = db.listWithCursor(ctx, s, prm.count, prm.cursor)
return err
})
success = err == nil
return res, metaerr.Wrap(err)
}
func (db *DB) listWithCursor(tx *bbolt.Tx, result []objectcore.Info, count int, cursor *Cursor) ([]objectcore.Info, *Cursor, error) {
threshold := cursor == nil // threshold is a flag to ignore cursor
var bucketName []byte
var err error
c := tx.Cursor()
name, _ := c.First()
if !threshold {
name, _ = c.Seek(cursor.bucketName)
func (db *DB) listWithCursor(ctx context.Context, r pebble.Reader, count int, cursor *Cursor) ([]objectcore.Info, *Cursor, error) {
var prefix []byte
var lastSeen []byte
if cursor != nil {
prefix = []byte{cursor.lastKey[0]}
lastSeen = cursor.lastKey
} else {
prefix = listPrefixes[0].prefix
}
var containerID cid.ID
var offset []byte
graveyardBkt := tx.Bucket(graveyardBucketName)
garbageBkt := tx.Bucket(garbageBucketName)
rawAddr := make([]byte, cidSize, addressKeySize)
loop:
for ; name != nil; name, _ = c.Next() {
cidRaw, prefix := parseContainerIDWithPrefix(&containerID, name)
if cidRaw == nil {
continue
}
var objType objectSDK.Type
switch prefix {
case primaryPrefix:
objType = objectSDK.TypeRegular
case lockersPrefix:
objType = objectSDK.TypeLock
case tombstonePrefix:
objType = objectSDK.TypeTombstone
default:
continue
}
bkt := tx.Bucket(name)
if bkt != nil {
copy(rawAddr, cidRaw)
result, offset, cursor, err = selectNFromBucket(bkt, objType, graveyardBkt, garbageBkt, rawAddr, containerID,
result, count, cursor, threshold)
if err != nil {
return nil, nil, err
}
}
bucketName = name
if len(result) >= count {
break loop
}
// set threshold flag after first `selectNFromBucket` invocation
// first invocation must look for cursor object
threshold = true
idx := slices.IndexFunc(listPrefixes, func(e listPrefix) bool {
return e.prefix[0] == prefix[0]
})
if idx < 0 {
return nil, nil, fmt.Errorf("invalid prefix value %d", prefix[0])
}
if offset != nil {
// new slice is much faster but less memory efficient
// we need to copy, because offset exists during bbolt tx
cursor.inBucketOffset = make([]byte, len(offset))
copy(cursor.inBucketOffset, offset)
var next Cursor
result := make([]objectcore.Info, 0, count)
for ; idx < len(listPrefixes); idx++ {
indexResult, lastIndexSeen, err := listByPrefix(ctx, r, lastSeen, idx, count-len(result))
if err != nil {
return nil, nil, err
}
result = append(result, indexResult...)
if len(lastIndexSeen) > 0 {
next.lastKey = lastIndexSeen
}
if len(result) == count {
return result, &next, nil
}
lastSeen = nil
}
if len(result) == 0 {
return nil, nil, ErrEndOfListing
}
// new slice is much faster but less memory efficient
// we need to copy, because bucketName exists during bbolt tx
cursor.bucketName = make([]byte, len(bucketName))
copy(cursor.bucketName, bucketName)
return result, cursor, nil
return result, &next, nil
}
// selectNFromBucket similar to selectAllFromBucket but uses cursor to find
// object to start selecting from. Ignores inhumed objects.
func selectNFromBucket(bkt *bbolt.Bucket, // main bucket
objType objectSDK.Type, // type of the objects stored in the main bucket
graveyardBkt, garbageBkt *bbolt.Bucket, // cached graveyard buckets
cidRaw []byte, // container ID prefix, optimization
cnt cid.ID, // container ID
to []objectcore.Info, // listing result
limit int, // stop listing at `limit` items in result
cursor *Cursor, // start from cursor object
threshold bool, // ignore cursor and start immediately
) ([]objectcore.Info, []byte, *Cursor, error) {
if cursor == nil {
cursor = new(Cursor)
}
count := len(to)
c := bkt.Cursor()
k, v := c.First()
offset := cursor.inBucketOffset
if !threshold {
c.Seek(offset)
k, v = c.Next() // we are looking for objects _after_ the cursor
}
for ; k != nil; k, v = c.Next() {
if count >= limit {
break
func listByPrefix(ctx context.Context, r pebble.Reader, lastSeen []byte, idx int, count int) ([]objectcore.Info, []byte, error) {
var result []objectcore.Info
for {
kvs, err := selectByPrefixAndSeek(ctx, r, listPrefixes[idx].prefix, lastSeen, count-len(result))
if err != nil {
return nil, nil, err
}
var obj oid.ID
if err := obj.Decode(k); err != nil {
break
if len(kvs) == 0 {
return result, lastSeen, nil
}
offset = k
if inGraveyardWithKey(append(cidRaw, k...), graveyardBkt, garbageBkt) > 0 {
continue
}
var isLinkingObj bool
var ecInfo *objectcore.ECInfo
if objType == objectSDK.TypeRegular {
var o objectSDK.Object
if err := o.Unmarshal(v); err != nil {
return nil, nil, nil, err
for _, kv := range kvs {
lastSeen = kv.Key
addr, err := listPrefixes[idx].keyParser(kv.Key)
if err != nil {
return nil, nil, err
}
isLinkingObj = isLinkObject(&o)
ecHeader := o.ECHeader()
if ecHeader != nil {
ecInfo = &objectcore.ECInfo{
ParentID: ecHeader.Parent(),
Index: ecHeader.Index(),
Total: ecHeader.Total(),
st, err := inGraveyardWithKey(r, addr)
if err != nil {
return nil, nil, err
}
if st > 0 {
continue
}
var isLinkingObj bool
var ecInfo *objectcore.ECInfo
if listPrefixes[idx].objectType == objectSDK.TypeRegular {
var o objectSDK.Object
if err := o.Unmarshal(kv.Value); err != nil {
return nil, nil, err
}
isLinkingObj = isLinkObject(&o)
ecHeader := o.ECHeader()
if ecHeader != nil {
ecInfo = &objectcore.ECInfo{
ParentID: ecHeader.Parent(),
Index: ecHeader.Index(),
Total: ecHeader.Total(),
}
}
}
result = append(result, objectcore.Info{
Address: addr,
Type: listPrefixes[idx].objectType,
IsLinkingObject: isLinkingObj,
ECInfo: ecInfo,
})
if len(result) == count {
return result, lastSeen, nil
}
}
var a oid.Address
a.SetContainer(cnt)
a.SetObject(obj)
to = append(to, objectcore.Info{Address: a, Type: objType, IsLinkingObject: isLinkingObj, ECInfo: ecInfo})
count++
}
return to, offset, cursor, nil
}
func parseContainerIDWithPrefix(containerID *cid.ID, name []byte) ([]byte, byte) {
if len(name) < bucketKeySize {
return nil, 0
}
rawID := name[1:bucketKeySize]
if err := containerID.Decode(rawID); err != nil {
return nil, 0
}
return rawID, name[0]
}

View file

@ -12,7 +12,6 @@ import (
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
oidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id/test"
"github.com/stretchr/testify/require"
"go.etcd.io/bbolt"
)
func BenchmarkListWithCursor(b *testing.B) {
@ -29,9 +28,7 @@ func BenchmarkListWithCursor(b *testing.B) {
}
func listWithCursorPrepareDB(b *testing.B) *meta.DB {
db := newDB(b, meta.WithMaxBatchSize(1), meta.WithBoltDBOptions(&bbolt.Options{
NoSync: true,
})) // faster single-thread generation
db := newDB(b) // faster single-thread generation
defer func() { require.NoError(b, db.Close()) }()
obj := testutil.GenerateObject()

View file

@ -3,7 +3,7 @@ package meta
import (
"bytes"
"context"
"fmt"
"errors"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
@ -13,23 +13,16 @@ import (
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.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
var bucketNameLocked = []byte{lockedPrefix}
type keyValue struct {
Key []byte
Value []byte
}
// returns name of the bucket with objects of type LOCK for specified container.
func bucketNameLockers(idCnr cid.ID, key []byte) []byte {
return bucketName(idCnr, lockersPrefix, key)
}
// Lock marks objects as locked with another object. All objects are from the
// specified container.
//
@ -66,66 +59,45 @@ func (db *DB) Lock(ctx context.Context, cnr cid.ID, locker oid.ID, locked []oid.
panic("empty locked list")
}
err := db.lockInternal(locked, cnr, locker)
defer db.guard.LockContainerID(cnr)()
err := db.batch(func(b *pebble.Batch) error {
return lockInternal(b, locked, cnr, locker)
})
success = err == nil
return err
}
func (db *DB) lockInternal(locked []oid.ID, cnr cid.ID, locker oid.ID) error {
bucketKeysLocked := make([][]byte, len(locked))
for i := range locked {
bucketKeysLocked[i] = objectKey(locked[i], make([]byte, objectKeySize))
func lockInternal(b *pebble.Batch, locked []oid.ID, cnr cid.ID, locker oid.ID) error {
t, err := firstIrregularObjectType(b, cnr, locked...)
if err != nil {
return err
}
if t != objectSDK.TypeRegular {
return logicerr.Wrap(new(apistatus.LockNonRegularObject))
}
key := make([]byte, cidSize)
return metaerr.Wrap(db.boltDB.Update(func(tx *bbolt.Tx) error {
if firstIrregularObjectType(tx, cnr, bucketKeysLocked...) != objectSDK.TypeRegular {
return logicerr.Wrap(new(apistatus.LockNonRegularObject))
}
bucketLocked := tx.Bucket(bucketNameLocked)
cnr.Encode(key)
bucketLockedContainer, err := bucketLocked.CreateBucketIfNotExists(key)
for _, objID := range locked {
key := lockedKey(cnr, objID, locker)
v, err := valueSafe(b, key)
if err != nil {
return fmt.Errorf("create container bucket for locked objects %v: %w", cnr, err)
return err
}
if v != nil {
// already locked by locker
continue
}
keyLocker := objectKey(locker, key)
var exLockers [][]byte
var updLockers []byte
loop:
for i := range bucketKeysLocked {
exLockers, err = decodeList(bucketLockedContainer.Get(bucketKeysLocked[i]))
if err != nil {
return fmt.Errorf("decode list of object lockers: %w", err)
}
for i := range exLockers {
if bytes.Equal(exLockers[i], keyLocker) {
continue loop
}
}
updLockers, err = encodeList(append(exLockers, keyLocker))
if err != nil {
return fmt.Errorf("encode list of object lockers: %w", err)
}
err = bucketLockedContainer.Put(bucketKeysLocked[i], updLockers)
if err != nil {
return fmt.Errorf("update list of object lockers: %w", err)
}
if err := b.Set(key, zeroValue, pebble.Sync); err != nil {
return err
}
return nil
}))
}
return nil
}
// FreeLockedBy unlocks all objects in DB which are locked by lockers.
// Returns slice of unlocked object ID's or an error.
func (db *DB) FreeLockedBy(lockers []oid.Address) ([]oid.Address, error) {
func (db *DB) FreeLockedBy(ctx context.Context, lockers []oid.Address) ([]oid.Address, error) {
var (
startedAt = time.Now()
success = false
@ -141,11 +113,17 @@ func (db *DB) FreeLockedBy(lockers []oid.Address) ([]oid.Address, error) {
return nil, ErrDegradedMode
}
var containerIDs []cid.ID
for _, a := range lockers {
containerIDs = append(containerIDs, a.Container())
}
defer db.guard.LockContainerIDs(containerIDs)()
var unlockedObjects []oid.Address
if err := db.boltDB.Update(func(tx *bbolt.Tx) error {
if err := db.batch(func(b *pebble.Batch) error {
for i := range lockers {
unlocked, err := freePotentialLocks(tx, lockers[i].Container(), lockers[i].Object())
unlocked, err := freePotentialLocks(ctx, b, lockers[i])
if err != nil {
return err
}
@ -161,42 +139,38 @@ func (db *DB) FreeLockedBy(lockers []oid.Address) ([]oid.Address, error) {
}
// checks if specified object is locked in the specified container.
func objectLocked(tx *bbolt.Tx, idCnr cid.ID, idObj oid.ID) bool {
bucketLocked := tx.Bucket(bucketNameLocked)
if bucketLocked != nil {
key := make([]byte, cidSize)
idCnr.Encode(key)
bucketLockedContainer := bucketLocked.Bucket(key)
if bucketLockedContainer != nil {
return bucketLockedContainer.Get(objectKey(idObj, key)) != nil
}
}
func objectLocked(ctx context.Context, r pebble.Reader, idCnr cid.ID, idObj oid.ID) (bool, error) {
prefix := lockedKeyLongPrefix(idCnr, idObj)
return false
items, err := selectByPrefixBatch(ctx, r, prefix, 1)
if err != nil {
return false, err
}
return len(items) > 0, nil
}
// return `LOCK` id's if specified object is locked in the specified container.
func getLocked(tx *bbolt.Tx, idCnr cid.ID, idObj oid.ID) ([]oid.ID, error) {
func getLocked(ctx context.Context, r pebble.Reader, idCnr cid.ID, idObj oid.ID) ([]oid.ID, error) {
prefix := lockedKeyLongPrefix(idCnr, idObj)
var lockers []oid.ID
bucketLocked := tx.Bucket(bucketNameLocked)
if bucketLocked != nil {
key := make([]byte, cidSize)
idCnr.Encode(key)
bucketLockedContainer := bucketLocked.Bucket(key)
if bucketLockedContainer != nil {
binObjIDs, err := decodeList(bucketLockedContainer.Get(objectKey(idObj, key)))
for {
items, err := selectByPrefixBatch(ctx, r, prefix, batchSize)
if err != nil {
return nil, err
}
for _, it := range items {
id, err := lockerObjectIDFromLockedKey(it)
if err != nil {
return nil, fmt.Errorf("decode list of object lockers: %w", err)
}
for _, binObjID := range binObjIDs {
var id oid.ID
if err = id.Decode(binObjID); err != nil {
return nil, err
}
lockers = append(lockers, id)
return nil, err
}
lockers = append(lockers, id)
}
if len(items) < batchSize {
break
}
}
return lockers, nil
}
@ -206,95 +180,65 @@ func getLocked(tx *bbolt.Tx, idCnr cid.ID, idObj oid.ID) ([]oid.ID, error) {
// Operation is very resource-intensive, which is caused by the admissibility
// of multiple locks. Also, if we knew what objects are locked, it would be
// possible to speed up the execution.
func freePotentialLocks(tx *bbolt.Tx, idCnr cid.ID, locker oid.ID) ([]oid.Address, error) {
func freePotentialLocks(ctx context.Context, b *pebble.Batch, locker oid.Address) ([]oid.Address, error) {
var unlockedObjects []oid.Address
bucketLocked := tx.Bucket(bucketNameLocked)
if bucketLocked == nil {
return unlockedObjects, nil
}
key := make([]byte, cidSize)
idCnr.Encode(key)
bucketLockedContainer := bucketLocked.Bucket(key)
if bucketLockedContainer == nil {
return unlockedObjects, nil
}
keyLocker := objectKey(locker, key)
updates := make([]keyValue, 0)
err := bucketLockedContainer.ForEach(func(k, v []byte) error {
keyLockers, err := decodeList(v)
if err != nil {
return fmt.Errorf("decode list of lockers in locked bucket: %w", err)
}
for i := range keyLockers {
if bytes.Equal(keyLockers[i], keyLocker) {
if len(keyLockers) == 1 {
updates = append(updates, keyValue{
Key: k,
Value: nil,
})
var id oid.ID
err = id.Decode(k)
if err != nil {
return fmt.Errorf("decode unlocked object id error: %w", err)
}
var addr oid.Address
addr.SetContainer(idCnr)
addr.SetObject(id)
unlockedObjects = append(unlockedObjects, addr)
} else {
// exclude locker
keyLockers = append(keyLockers[:i], keyLockers[i+1:]...)
v, err = encodeList(keyLockers)
if err != nil {
return fmt.Errorf("encode updated list of lockers: %w", err)
}
updates = append(updates, keyValue{
Key: k,
Value: v,
})
}
return nil
}
}
return nil
})
locked, err := lockedObjects(b, locker)
if err != nil {
return nil, err
}
if err = applyBucketUpdates(bucketLockedContainer, updates); err != nil {
return nil, err
for _, lockedObject := range locked {
select {
case <-ctx.Done():
return nil, ctx.Err()
default:
}
if err := b.Delete(lockedKey(locker.Container(), lockedObject, locker.Object()), pebble.Sync); err != nil {
return nil, err
}
isLocked, err := objectLocked(ctx, b, locker.Container(), lockedObject)
if err != nil {
return nil, err
}
if !isLocked { // deleted locker was the last one
var addr oid.Address
addr.SetContainer(locker.Container())
addr.SetObject(lockedObject)
unlockedObjects = append(unlockedObjects, addr)
}
}
return unlockedObjects, nil
}
func applyBucketUpdates(bucket *bbolt.Bucket, updates []keyValue) error {
for _, update := range updates {
if update.Value == nil {
err := bucket.Delete(update.Key)
if err != nil {
return fmt.Errorf("delete locked object record from locked bucket: %w", err)
}
} else {
err := bucket.Put(update.Key, update.Value)
if err != nil {
return fmt.Errorf("update list of lockers: %w", err)
}
}
func lockedObjects(r pebble.Reader, locker oid.Address) ([]oid.ID, error) {
var lockedByLocker []oid.ID
prefix := lockedKeyShortPrefix(locker.Container())
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return nil, err
}
return nil
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
currentLockerObjID, err := lockerObjectIDFromLockedKey(it.Key())
if err != nil {
return nil, errors.Join(err, it.Close())
}
if !currentLockerObjID.Equals(locker.Object()) {
continue
}
currentObjectID, err := objectIDFromLockedKey(it.Key())
if err != nil {
return nil, errors.Join(err, it.Close())
}
lockedByLocker = append(lockedByLocker, currentObjectID)
}
return lockedByLocker, it.Close()
}
// IsLockedPrm groups the parameters of IsLocked operation.
@ -343,9 +287,10 @@ func (db *DB) IsLocked(ctx context.Context, prm IsLockedPrm) (res IsLockedRes, e
if db.mode.NoMetabase() {
return res, ErrDegradedMode
}
err = metaerr.Wrap(db.boltDB.View(func(tx *bbolt.Tx) error {
res.locked = objectLocked(tx, prm.addr.Container(), prm.addr.Object())
return nil
err = metaerr.Wrap(db.snapshot(func(s *pebble.Snapshot) error {
var e error
res.locked, e = objectLocked(ctx, s, prm.addr.Container(), prm.addr.Object())
return e
}))
success = err == nil
return res, err
@ -376,10 +321,19 @@ func (db *DB) GetLocked(ctx context.Context, addr oid.Address) (res []oid.ID, er
if db.mode.NoMetabase() {
return res, ErrDegradedMode
}
err = metaerr.Wrap(db.boltDB.View(func(tx *bbolt.Tx) error {
res, err = getLocked(tx, addr.Container(), addr.Object())
err = metaerr.Wrap(db.snapshot(func(s *pebble.Snapshot) error {
res, err = getLocked(ctx, s, addr.Container(), addr.Object())
return nil
}))
success = err == nil
return res, err
}
func isLockObject(r pebble.Reader, idCnr cid.ID, obj oid.ID) (bool, error) {
key := lockersKey(idCnr, obj)
v, err := valueSafe(r, key)
if err != nil {
return false, err
}
return v != nil, nil
}

View file

@ -117,7 +117,7 @@ func TestDB_Lock(t *testing.T) {
require.Len(t, res.DeletedLockObjects(), 1)
require.Equal(t, objectcore.AddressOf(lockObj), res.DeletedLockObjects()[0])
_, err = db.FreeLockedBy([]oid.Address{lockAddr})
_, err = db.FreeLockedBy(context.Background(), []oid.Address{lockAddr})
require.NoError(t, err)
inhumePrm.SetAddresses(objAddr)
@ -148,7 +148,7 @@ func TestDB_Lock(t *testing.T) {
// unlock just objects that were locked by
// just removed locker
_, err = db.FreeLockedBy([]oid.Address{res.DeletedLockObjects()[0]})
_, err = db.FreeLockedBy(context.Background(), []oid.Address{res.DeletedLockObjects()[0]})
require.NoError(t, err)
// removing objects after unlock

View file

@ -1,8 +1,10 @@
package meta
import (
"context"
"fmt"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
)
@ -17,17 +19,17 @@ func (db *DB) SetMode(m mode.Mode) error {
}
if !db.mode.NoMetabase() {
if err := db.Close(); err != nil {
if err := db.close(); err != nil {
return fmt.Errorf("can't set metabase mode (old=%s, new=%s): %w", db.mode, m, err)
}
}
if m.NoMetabase() {
db.boltDB = nil
db.database = nil
} else {
err := db.openDB(m)
if err == nil && !m.ReadOnly() {
err = db.Init()
err = metaerr.Wrap(db.init(context.TODO(), false))
}
if err != nil {
return fmt.Errorf("can't set metabase mode (old=%s, new=%s): %w", db.mode, m, err)

View file

@ -24,14 +24,14 @@ func Test_Mode(t *testing.T) {
}...)
require.NoError(t, bdb.Open(context.Background(), mode.DegradedReadOnly))
require.Nil(t, bdb.boltDB)
require.Nil(t, bdb.database)
require.NoError(t, bdb.Init())
require.Nil(t, bdb.boltDB)
require.Nil(t, bdb.database)
require.NoError(t, bdb.Close())
require.NoError(t, bdb.Open(context.Background(), mode.Degraded))
require.Nil(t, bdb.boltDB)
require.Nil(t, bdb.database)
require.NoError(t, bdb.Init())
require.Nil(t, bdb.boltDB)
require.Nil(t, bdb.database)
require.NoError(t, bdb.Close())
}

View file

@ -0,0 +1,126 @@
package meta
import (
"bytes"
"context"
"errors"
"github.com/cockroachdb/pebble"
)
const batchSize = 1000
func valueSafe(r pebble.Reader, key []byte) ([]byte, error) {
data, closer, err := r.Get(key)
if err != nil {
if errors.Is(err, pebble.ErrNotFound) {
return nil, nil
}
return nil, err
}
result := bytes.Clone(data)
if err := closer.Close(); err != nil {
return nil, err
}
return result, nil
}
func (db *DB) batch(f func(batch *pebble.Batch) error) error {
b := db.database.NewIndexedBatch()
err := f(b)
if err != nil {
return errors.Join(err, b.Close())
}
return errors.Join(b.Commit(pebble.Sync), b.Close())
}
func (db *DB) snapshot(f func(*pebble.Snapshot) error) error {
s := db.database.NewSnapshot()
err := f(s)
if err != nil {
return errors.Join(err, s.Close())
}
return s.Close()
}
func selectByPrefixBatch(ctx context.Context, r pebble.Reader, prefix []byte, batchSize int) ([][]byte, error) {
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return nil, err
}
var result [][]byte
for v := it.First(); v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return nil, errors.Join(ctx.Err(), it.Close())
default:
}
result = append(result, bytes.Clone(it.Key()))
if len(result) == batchSize {
return result, it.Close()
}
}
return result, it.Close()
}
func deleteByPrefix(ctx context.Context, b *pebble.Batch, prefix []byte) error {
for {
batch, err := selectByPrefixBatch(ctx, b, prefix, batchSize)
if err != nil {
return err
}
for _, key := range batch {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
if err := b.Delete(key, pebble.Sync); err != nil {
return err
}
}
if len(batch) < batchSize {
return nil
}
}
}
func selectByPrefixAndSeek(ctx context.Context, r pebble.Reader, prefix, lastSeen []byte, batchSize int) ([]keyValue, error) {
it, err := r.NewIter(&pebble.IterOptions{
LowerBound: prefix,
})
if err != nil {
return nil, err
}
var result []keyValue
var v bool
if len(lastSeen) == 0 {
v = it.First()
} else {
v = it.SeekGE(lastSeen)
}
for ; v && bytes.HasPrefix(it.Key(), prefix); v = it.Next() {
select {
case <-ctx.Done():
return nil, errors.Join(ctx.Err(), it.Close())
default:
}
if bytes.Equal(lastSeen, it.Key()) {
continue
}
var current keyValue
current.Key = bytes.Clone(it.Key())
current.Value = bytes.Clone(it.Value())
result = append(result, current)
if len(result) == batchSize {
return result, it.Close()
}
}
return result, it.Close()
}

View file

@ -1,13 +1,14 @@
package meta
import (
"bytes"
"context"
"encoding/binary"
"errors"
"fmt"
gio "io"
"strconv"
"time"
objectV2 "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/object"
objectCore "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
storagelog "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/log"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
@ -16,18 +17,11 @@ import (
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"
"github.com/nspcc-dev/neo-go/pkg/io"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
type (
namedBucketItem struct {
name, key, val []byte
}
)
// PutPrm groups the parameters of Put operation.
type PutPrm struct {
obj *objectSDK.Object
@ -54,6 +48,8 @@ var (
ErrUnknownObjectType = errors.New("unknown object type")
ErrIncorrectSplitInfoUpdate = errors.New("updating split info on object without it")
ErrIncorrectRootObject = errors.New("invalid root object")
errInvalidUserAttributeKeyFormat = errors.New("invalid user attribute key format")
)
// Put saves object header in metabase. Object payload expected to be cut.
@ -85,10 +81,16 @@ func (db *DB) Put(ctx context.Context, prm PutPrm) (res PutRes, err error) {
}
currEpoch := db.epochState.CurrentEpoch()
cnr, ok := prm.obj.ContainerID()
if !ok {
return PutRes{}, errors.New("missing container in object")
}
err = db.boltDB.Batch(func(tx *bbolt.Tx) error {
defer db.guard.LockContainerID(cnr)()
err = db.batch(func(b *pebble.Batch) error {
var e error
res, e = db.put(tx, prm.obj, prm.id, nil, currEpoch)
res, e = db.put(ctx, b, prm.obj, prm.id, nil, currEpoch)
return e
})
if err == nil {
@ -101,7 +103,9 @@ func (db *DB) Put(ctx context.Context, prm PutPrm) (res PutRes, err error) {
return res, metaerr.Wrap(err)
}
func (db *DB) put(tx *bbolt.Tx,
func (db *DB) put(
ctx context.Context,
b *pebble.Batch,
obj *objectSDK.Object,
id []byte,
si *objectSDK.SplitInfo,
@ -114,7 +118,7 @@ func (db *DB) put(tx *bbolt.Tx,
isParent := si != nil
exists, _, err := db.exists(tx, objectCore.AddressOf(obj), oid.Address{}, currEpoch)
exists, _, err := db.exists(ctx, b, objectCore.AddressOf(obj), oid.Address{}, currEpoch)
var splitInfoError *objectSDK.SplitInfoError
if errors.As(err, &splitInfoError) {
@ -124,70 +128,71 @@ func (db *DB) put(tx *bbolt.Tx,
}
if exists {
return PutRes{}, db.updateObj(tx, obj, id, si, isParent)
return PutRes{}, db.updateObj(b, obj, id, si, isParent)
}
return PutRes{Inserted: true}, db.insertObject(tx, obj, id, si, isParent, cnr, currEpoch)
return PutRes{Inserted: true}, db.insertObject(ctx, b, obj, id, si, isParent, cnr, currEpoch)
}
func (db *DB) updateObj(tx *bbolt.Tx, obj *objectSDK.Object, id []byte, si *objectSDK.SplitInfo, isParent bool) error {
func (db *DB) updateObj(b *pebble.Batch, obj *objectSDK.Object, id []byte, si *objectSDK.SplitInfo, isParent bool) error {
addr := objectCore.AddressOf(obj)
// most right child and split header overlap parent so we have to
// check if object exists to not overwrite it twice
// When storage engine moves objects between different sub-storages,
// it calls metabase.Put method with new storage ID, thus triggering this code.
if !isParent && id != nil {
return setStorageID(tx, objectCore.AddressOf(obj), id, true)
return setStorageID(b, addr, id, true)
}
// when storage already has last object in split hierarchy and there is
// a linking object to put (or vice versa), we should update split info
// with object ids of these objects
if isParent {
return updateSplitInfo(tx, objectCore.AddressOf(obj), si)
return updateSplitInfo(b, addr.Container(), addr.Object(), si)
}
return nil
}
func (db *DB) insertObject(tx *bbolt.Tx, obj *objectSDK.Object, id []byte, si *objectSDK.SplitInfo, isParent bool, cnr cid.ID, currEpoch uint64) error {
func (db *DB) insertObject(ctx context.Context, b *pebble.Batch, obj *objectSDK.Object, id []byte, si *objectSDK.SplitInfo, isParent bool, cnr cid.ID, currEpoch uint64) error {
if par := obj.Parent(); par != nil && !isParent { // limit depth by two
parentSI, err := splitInfoFromObject(obj)
if err != nil {
return err
}
_, err = db.put(tx, par, id, parentSI, currEpoch)
_, err = db.put(ctx, b, par, id, parentSI, currEpoch)
if err != nil {
return err
}
}
err := putUniqueIndexes(tx, obj, si, id)
err := putUniqueIndexes(b, obj, si, id)
if err != nil {
return fmt.Errorf("can't put unique indexes: %w", err)
}
err = updateListIndexes(tx, obj, putListIndexItem)
err = updateListIndexes(b, obj, putListIndexItem)
if err != nil {
return fmt.Errorf("can't put list indexes: %w", err)
}
err = updateFKBTIndexes(tx, obj, putFKBTIndexItem)
err = updateFKBTIndexes(b, obj, putListIndexItem)
if err != nil {
return fmt.Errorf("can't put fake bucket tree indexes: %w", err)
}
// update container volume size estimation
if obj.Type() == objectSDK.TypeRegular && !isParent {
err = changeContainerSize(tx, cnr, obj.PayloadSize(), true)
err = changeContainerSize(b, cnr, int64(obj.PayloadSize()))
if err != nil {
return err
}
}
if !isParent {
if err = db.incCounters(tx, cnr, IsUserObject(obj)); err != nil {
if err = incCounters(b, cnr, IsUserObject(obj)); err != nil {
return err
}
}
@ -196,26 +201,24 @@ func (db *DB) insertObject(tx *bbolt.Tx, obj *objectSDK.Object, id []byte, si *o
}
func putUniqueIndexes(
tx *bbolt.Tx,
b *pebble.Batch,
obj *objectSDK.Object,
si *objectSDK.SplitInfo,
id []byte,
) error {
isParent := si != nil
addr := objectCore.AddressOf(obj)
cnr := addr.Container()
objKey := objectKey(addr.Object(), make([]byte, objectKeySize))
bucketName := make([]byte, bucketKeySize)
// add value to primary unique bucket
if !isParent {
var key []byte
switch obj.Type() {
case objectSDK.TypeRegular:
bucketName = primaryBucketName(cnr, bucketName)
key = primaryKey(addr.Container(), addr.Object())
case objectSDK.TypeTombstone:
bucketName = tombstoneBucketName(cnr, bucketName)
key = tombstoneKey(addr.Container(), addr.Object())
case objectSDK.TypeLock:
bucketName = bucketNameLockers(cnr, bucketName)
key = lockersKey(addr.Container(), addr.Object())
default:
return ErrUnknownObjectType
}
@ -225,18 +228,14 @@ func putUniqueIndexes(
return fmt.Errorf("can't marshal object header: %w", err)
}
err = putUniqueIndexItem(tx, namedBucketItem{
name: bucketName,
key: objKey,
val: rawObject,
})
err = b.Set(key, rawObject, pebble.Sync)
if err != nil {
return err
}
// index storageID if it is present
if id != nil {
if err = setStorageID(tx, objectCore.AddressOf(obj), id, false); err != nil {
if err = setStorageID(b, objectCore.AddressOf(obj), id, false); err != nil {
return err
}
}
@ -244,6 +243,7 @@ func putUniqueIndexes(
// index root object
if obj.Type() == objectSDK.TypeRegular && !obj.HasParent() {
objID := addr.Object()
if ecHead := obj.ECHeader(); ecHead != nil {
parentID := ecHead.Parent()
if ecHead.ParentSplitID() != nil {
@ -258,53 +258,23 @@ func putUniqueIndexes(
parentID = *parentSplitParentID
}
objKey = objectKey(parentID, objKey)
objID = parentID
}
return updateSplitInfoIndex(tx, objKey, cnr, bucketName, si)
return updateSplitInfo(b, addr.Container(), objID, si)
}
return nil
}
func updateSplitInfoIndex(tx *bbolt.Tx, objKey []byte, cnr cid.ID, bucketName []byte, si *objectSDK.SplitInfo) error {
return updateUniqueIndexItem(tx, namedBucketItem{
name: rootBucketName(cnr, bucketName),
key: objKey,
}, func(old, _ []byte) ([]byte, error) {
switch {
case si == nil && old == nil:
return []byte{}, nil
case si == nil:
return old, nil
case old == nil:
return si.Marshal()
default:
oldSI := objectSDK.NewSplitInfo()
if err := oldSI.Unmarshal(old); err != nil {
return nil, err
}
si = util.MergeSplitInfo(si, oldSI)
return si.Marshal()
}
})
}
type updateIndexItemFunc = func(b *pebble.Batch, key []byte) error
type updateIndexItemFunc = func(tx *bbolt.Tx, item namedBucketItem) error
func updateListIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFunc) error {
func updateListIndexes(b *pebble.Batch, obj *objectSDK.Object, f updateIndexItemFunc) error {
idObj, _ := obj.ID()
cnr, _ := obj.ContainerID()
objKey := objectKey(idObj, make([]byte, objectKeySize))
bucketName := make([]byte, bucketKeySize)
cs, _ := obj.PayloadChecksum()
// index payload hashes
err := f(tx, namedBucketItem{
name: payloadHashBucketName(cnr, bucketName),
key: cs.Value(),
val: objKey,
})
err := f(b, payloadHashKey(cnr, idObj, cs.Value()))
if err != nil {
return err
}
@ -313,11 +283,7 @@ func updateListIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFun
// index parent ids
if ok {
err := f(tx, namedBucketItem{
name: parentBucketName(cnr, bucketName),
key: objectKey(idParent, make([]byte, objectKeySize)),
val: objKey,
})
err := f(b, parentKey(cnr, idParent, idObj))
if err != nil {
return err
}
@ -325,45 +291,42 @@ func updateListIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFun
// index split ids
if obj.SplitID() != nil {
err := f(tx, namedBucketItem{
name: splitBucketName(cnr, bucketName),
key: obj.SplitID().ToV2(),
val: objKey,
})
err := f(b, splitKey(cnr, idObj, obj.SplitID().ToV2()))
if err != nil {
return err
}
}
for _, attr := range obj.Attributes() {
if attr.Key() != objectV2.SysAttributeExpEpochNeoFS && attr.Key() != objectV2.SysAttributeExpEpoch {
continue
}
expEpoch, err := strconv.ParseUint(attr.Value(), 10, 64)
if err != nil {
return errInvalidUserAttributeKeyFormat
}
err = f(b, expiredKey(cnr, idObj, expEpoch))
if err != nil {
return err
}
break
}
if ech := obj.ECHeader(); ech != nil {
err := f(tx, namedBucketItem{
name: ecInfoBucketName(cnr, bucketName),
key: objectKey(ech.Parent(), make([]byte, objectKeySize)),
val: objKey,
})
err := f(b, ecInfoKey(cnr, ech.Parent(), idObj))
if err != nil {
return err
}
if ech.ParentSplitID() != nil {
objKey := objectKey(ech.Parent(), make([]byte, objectKeySize))
err := f(tx, namedBucketItem{
name: splitBucketName(cnr, bucketName),
key: ech.ParentSplitID().ToV2(),
val: objKey,
})
err := f(b, splitKey(cnr, ech.Parent(), ech.ParentSplitID().ToV2()))
if err != nil {
return err
}
}
if parentSplitParentID := ech.ParentSplitParentID(); parentSplitParentID != nil {
objKey := objectKey(ech.Parent(), make([]byte, objectKeySize))
err := f(tx, namedBucketItem{
name: parentBucketName(cnr, bucketName),
key: objectKey(*parentSplitParentID, make([]byte, objectKeySize)),
val: objKey,
})
err := f(b, parentKey(cnr, *parentSplitParentID, ech.Parent()))
if err != nil {
return err
}
@ -373,17 +336,10 @@ func updateListIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFun
return nil
}
func updateFKBTIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFunc) error {
func updateFKBTIndexes(b *pebble.Batch, obj *objectSDK.Object, f updateIndexItemFunc) error {
id, _ := obj.ID()
cnr, _ := obj.ContainerID()
objKey := objectKey(id, make([]byte, objectKeySize))
key := make([]byte, bucketKeySize)
err := f(tx, namedBucketItem{
name: ownerBucketName(cnr, key),
key: []byte(obj.OwnerID().EncodeToString()),
val: objKey,
})
err := f(b, ownerKey(cnr, id, []byte(obj.OwnerID().EncodeToString())))
if err != nil {
return err
}
@ -391,19 +347,14 @@ func updateFKBTIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFun
var attrs []objectSDK.Attribute
if obj.ECHeader() != nil {
attrs = obj.ECHeader().ParentAttributes()
objKey = objectKey(obj.ECHeader().Parent(), make([]byte, objectKeySize))
id = obj.ECHeader().Parent()
} else {
attrs = obj.Attributes()
}
// user specified attributes
for i := range attrs {
key = attributeBucketName(cnr, attrs[i].Key(), key)
err := f(tx, namedBucketItem{
name: key,
key: []byte(attrs[i].Value()),
val: objKey,
})
err := f(b, attributeKey(cnr, id, attrs[i].Key(), attrs[i].Value()))
if err != nil {
return err
}
@ -412,161 +363,42 @@ func updateFKBTIndexes(tx *bbolt.Tx, obj *objectSDK.Object, f updateIndexItemFun
return nil
}
type bucketContainer interface {
Bucket([]byte) *bbolt.Bucket
CreateBucket([]byte) (*bbolt.Bucket, error)
CreateBucketIfNotExists([]byte) (*bbolt.Bucket, error)
}
func createBucketLikelyExists[T bucketContainer](tx T, name []byte) (*bbolt.Bucket, error) {
if bkt := tx.Bucket(name); bkt != nil {
return bkt, nil
}
return tx.CreateBucket(name)
}
func updateUniqueIndexItem(tx *bbolt.Tx, item namedBucketItem, update func(oldData, newData []byte) ([]byte, error)) error {
bkt, err := createBucketLikelyExists(tx, item.name)
if err != nil {
return fmt.Errorf("can't create index %v: %w", item.name, err)
}
data, err := update(bkt.Get(item.key), item.val)
if err != nil {
return err
}
return bkt.Put(item.key, data)
}
func putUniqueIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
return updateUniqueIndexItem(tx, item, func(_, val []byte) ([]byte, error) { return val, nil })
}
func putFKBTIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
bkt, err := createBucketLikelyExists(tx, item.name)
if err != nil {
return fmt.Errorf("can't create index %v: %w", item.name, err)
}
fkbtRoot, err := createBucketLikelyExists(bkt, item.key)
if err != nil {
return fmt.Errorf("can't create fake bucket tree index %v: %w", item.key, err)
}
return fkbtRoot.Put(item.val, zeroValue)
}
func putListIndexItem(tx *bbolt.Tx, item namedBucketItem) error {
bkt, err := createBucketLikelyExists(tx, item.name)
if err != nil {
return fmt.Errorf("can't create index %v: %w", item.name, err)
}
lst, err := decodeList(bkt.Get(item.key))
if err != nil {
return fmt.Errorf("can't decode leaf list %v: %w", item.key, err)
}
lst = append(lst, item.val)
encodedLst, err := encodeList(lst)
if err != nil {
return fmt.Errorf("can't encode leaf list %v: %w", item.key, err)
}
return bkt.Put(item.key, encodedLst)
}
// encodeList decodes list of bytes into a single blog for list bucket indexes.
func encodeList(lst [][]byte) ([]byte, error) {
w := io.NewBufBinWriter()
w.WriteVarUint(uint64(len(lst)))
for i := range lst {
w.WriteVarBytes(lst[i])
}
if w.Err != nil {
return nil, w.Err
}
return w.Bytes(), nil
}
// decodeList decodes blob into the list of bytes from list bucket index.
func decodeList(data []byte) (lst [][]byte, err error) {
if len(data) == 0 {
return nil, nil
}
var offset uint64
size, n, err := getVarUint(data)
if err != nil {
return nil, err
}
offset += uint64(n)
lst = make([][]byte, size, size+1)
for i := range lst {
sz, n, err := getVarUint(data[offset:])
if err != nil {
return nil, err
}
offset += uint64(n)
next := offset + sz
if uint64(len(data)) < next {
return nil, gio.ErrUnexpectedEOF
}
lst[i] = data[offset:next]
offset = next
}
return lst, nil
}
func getVarUint(data []byte) (uint64, int, error) {
if len(data) == 0 {
return 0, 0, gio.ErrUnexpectedEOF
}
switch b := data[0]; b {
case 0xfd:
if len(data) < 3 {
return 0, 1, gio.ErrUnexpectedEOF
}
return uint64(binary.LittleEndian.Uint16(data[1:])), 3, nil
case 0xfe:
if len(data) < 5 {
return 0, 1, gio.ErrUnexpectedEOF
}
return uint64(binary.LittleEndian.Uint32(data[1:])), 5, nil
case 0xff:
if len(data) < 9 {
return 0, 1, gio.ErrUnexpectedEOF
}
return binary.LittleEndian.Uint64(data[1:]), 9, nil
default:
return uint64(b), 1, nil
}
}
// setStorageID for existing objects if they were moved from one
// storage location to another.
func setStorageID(tx *bbolt.Tx, addr oid.Address, id []byte, override bool) error {
key := make([]byte, bucketKeySize)
bkt, err := createBucketLikelyExists(tx, smallBucketName(addr.Container(), key))
if err != nil {
return err
}
key = objectKey(addr.Object(), key)
if override || bkt.Get(key) == nil {
return bkt.Put(key, id)
}
return nil
func putListIndexItem(b *pebble.Batch, key []byte) error {
return b.Set(key, zeroValue, pebble.Sync)
}
// updateSpliInfo for existing objects if storage filled with extra information
// about last object in split hierarchy or linking object.
func updateSplitInfo(tx *bbolt.Tx, addr oid.Address, from *objectSDK.SplitInfo) error {
objKey := objectKey(addr.Object(), make([]byte, bucketKeySize))
return updateSplitInfoIndex(tx, objKey, addr.Container(), make([]byte, bucketKeySize), from)
func updateSplitInfo(b *pebble.Batch, cnr cid.ID, obj oid.ID, si *objectSDK.SplitInfo) error {
key := rootKey(cnr, obj)
existed, err := valueSafe(b, key)
if err != nil {
return nil
}
switch {
case si == nil && existed == nil:
return b.Set(key, zeroValue, pebble.Sync)
case si == nil:
return nil
case existed == nil || bytes.Equal(existed, zeroValue):
siBytes, err := si.Marshal()
if err != nil {
return nil
}
return b.Set(key, siBytes, pebble.Sync)
default:
existedSI := objectSDK.NewSplitInfo()
if err := existedSI.Unmarshal(existed); err != nil {
return err
}
si = util.MergeSplitInfo(si, existedSI)
siBytes, err := si.Marshal()
if err != nil {
return nil
}
return b.Set(key, siBytes, pebble.Sync)
}
}
// splitInfoFromObject returns split info based on last or linkin object.

View file

@ -2,11 +2,9 @@ package meta_test
import (
"context"
"runtime"
"strconv"
"sync/atomic"
"testing"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
@ -43,9 +41,7 @@ func prepareObjects(n int) []*objectSDK.Object {
func BenchmarkPut(b *testing.B) {
b.Run("parallel", func(b *testing.B) {
db := newDB(b,
meta.WithMaxBatchDelay(time.Millisecond*10),
meta.WithMaxBatchSize(runtime.NumCPU()))
db := newDB(b)
defer func() { require.NoError(b, db.Close()) }()
// Ensure the benchmark is bound by CPU and not waiting batch-delay time.
b.SetParallelism(1)
@ -65,9 +61,7 @@ func BenchmarkPut(b *testing.B) {
})
})
b.Run("sequential", func(b *testing.B) {
db := newDB(b,
meta.WithMaxBatchDelay(time.Millisecond*10),
meta.WithMaxBatchSize(1))
db := newDB(b)
defer func() { require.NoError(b, db.Close()) }()
var index atomic.Int64
index.Store(-1)

View file

@ -8,8 +8,8 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/testutil"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/require"
"go.etcd.io/bbolt"
)
type epochState struct{ e uint64 }
@ -42,16 +42,19 @@ func TestResetDropsContainerBuckets(t *testing.T) {
require.NoError(t, err)
}
require.NoError(t, db.Reset())
require.NoError(t, db.Reset(context.Background()))
var bucketCount int
require.NoError(t, db.boltDB.Update(func(tx *bbolt.Tx) error {
return tx.ForEach(func(name []byte, b *bbolt.Bucket) error {
_, exists := mStaticBuckets[string(name)]
require.True(t, exists, "unexpected bucket:"+string(name))
bucketCount++
return nil
})
}))
require.Equal(t, len(mStaticBuckets), bucketCount)
var cnt int
err := db.snapshot(func(s *pebble.Snapshot) error {
it, err := s.NewIter(nil)
if err != nil {
return err
}
for v := it.First(); v; v = it.Next() {
cnt++
}
return it.Close()
})
require.NoError(t, err)
require.Equal(t, 1, cnt)
}

File diff suppressed because it is too large Load diff

View file

@ -8,13 +8,10 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
metamode "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
)
var (
shardInfoBucket = []byte{shardInfoPrefix}
shardIDKey = []byte("id")
)
var shardIDKey = []byte("id")
// GetShardID sets metabase operation mode
// and reads shard id from db.
@ -47,11 +44,12 @@ func (db *DB) GetShardID(mode metamode.Mode) ([]byte, error) {
// If id is missing, returns nil, nil.
func (db *DB) readShardID() ([]byte, error) {
var id []byte
err := db.boltDB.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(shardInfoBucket)
if b != nil {
id = bytes.Clone(b.Get(shardIDKey))
err := db.snapshot(func(s *pebble.Snapshot) error {
v, err := valueSafe(s, shardInfoKey(shardIDKey))
if err != nil {
return err
}
id = bytes.Clone(v)
return nil
})
return id, metaerr.Wrap(err)
@ -86,11 +84,7 @@ func (db *DB) SetShardID(id []byte, mode metamode.Mode) error {
// writeShardID writes shard id to db.
func (db *DB) writeShardID(id []byte) error {
return metaerr.Wrap(db.boltDB.Update(func(tx *bbolt.Tx) error {
b, err := tx.CreateBucketIfNotExists(shardInfoBucket)
if err != nil {
return err
}
return b.Put(shardIDKey, id)
return metaerr.Wrap(db.batch(func(b *pebble.Batch) error {
return b.Set(shardInfoKey(shardIDKey), id, pebble.Sync)
}))
}

View file

@ -8,7 +8,7 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/internal/metaerr"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
@ -57,8 +57,8 @@ func (db *DB) StorageID(ctx context.Context, prm StorageIDPrm) (res StorageIDRes
return res, ErrDegradedMode
}
err = db.boltDB.View(func(tx *bbolt.Tx) error {
res.id, err = db.storageID(tx, prm.addr)
err = db.snapshot(func(s *pebble.Snapshot) error {
res.id, err = db.storageID(s, prm.addr)
return err
})
@ -66,14 +66,11 @@ func (db *DB) StorageID(ctx context.Context, prm StorageIDPrm) (res StorageIDRes
return res, metaerr.Wrap(err)
}
func (db *DB) storageID(tx *bbolt.Tx, addr oid.Address) ([]byte, error) {
key := make([]byte, bucketKeySize)
smallBucket := tx.Bucket(smallBucketName(addr.Container(), key))
if smallBucket == nil {
return nil, nil
func (db *DB) storageID(r pebble.Reader, addr oid.Address) ([]byte, error) {
storageID, err := valueSafe(r, smallKey(addr.Container(), addr.Object()))
if err != nil {
return nil, err
}
storageID := smallBucket.Get(objectKey(addr.Object(), key))
if storageID == nil {
return nil, nil
}
@ -126,9 +123,28 @@ func (db *DB) UpdateStorageID(ctx context.Context, prm UpdateStorageIDPrm) (res
return res, ErrReadOnlyMode
}
err = db.boltDB.Batch(func(tx *bbolt.Tx) error {
return setStorageID(tx, prm.addr, prm.id, true)
defer db.guard.LockContainerID(prm.addr.Container())()
err = db.batch(func(b *pebble.Batch) error {
return setStorageID(b, prm.addr, prm.id, true)
})
success = err == nil
return res, metaerr.Wrap(err)
}
// setStorageID for existing objects if they were moved from one
// storage location to another.
func setStorageID(b *pebble.Batch, addr oid.Address, id []byte, override bool) error {
key := smallKey(addr.Container(), addr.Object())
if override {
return b.Set(key, id, pebble.Sync)
}
v, err := valueSafe(b, key)
if err != nil {
return err
}
if v == nil {
return b.Set(key, id, pebble.Sync)
}
return nil
}

View file

@ -3,31 +3,14 @@ package meta
import (
"bytes"
"crypto/sha256"
"errors"
"encoding/binary"
"fmt"
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.etcd.io/bbolt"
)
var (
// graveyardBucketName stores rows with the objects that have been
// covered with Tombstone objects. That objects should not be returned
// from the node and should not be accepted by the node from other
// nodes.
graveyardBucketName = []byte{graveyardPrefix}
// garbageBucketName stores rows with the objects that should be physically
// deleted by the node (Garbage Collector routine).
garbageBucketName = []byte{garbagePrefix}
toMoveItBucketName = []byte{toMoveItPrefix}
containerVolumeBucketName = []byte{containerVolumePrefix}
containerCounterBucketName = []byte{containerCountersPrefix}
zeroValue = []byte{0xFF}
errInvalidLength = errors.New("invalid length")
)
var zeroValue = []byte{0xFF}
// Prefix bytes for database keys. All ids and addresses are encoded in binary
// unless specified otherwise.
@ -42,13 +25,13 @@ const (
// Key: object address
// Value: dummy value
garbagePrefix
// toMoveItPrefix is used for bucket containing IDs of objects that are candidates for moving
// _ Previous usage was for for bucket containing IDs of objects that are candidates for moving
// to another shard.
toMoveItPrefix
// containerVolumePrefix is used for storing container size estimations.
_
// containerSizePrefix is used for storing container size estimations.
// Key: container ID
// Value: container size in bytes as little-endian uint64
containerVolumePrefix
containerSizePrefix
// lockedPrefix is used for storing locked objects information.
// Key: container ID
// Value: bucket mapping objects locked to the list of corresponding LOCK objects.
@ -124,6 +107,9 @@ const (
// Key: container ID + type
// Value: Object id
ecInfoPrefix
// expiredPrefix used to store expiration info.
expiredPrefix
)
const (
@ -133,139 +119,560 @@ const (
addressKeySize = cidSize + objectKeySize
)
func bucketName(cnr cid.ID, prefix byte, key []byte) []byte {
key[0] = prefix
cnr.Encode(key[1:])
return key[:bucketKeySize]
func keyPrefix(cnr cid.ID, prefix byte) []byte {
result := make([]byte, 1+cidSize)
result[0] = prefix
cnr.Encode(result[1:])
return result
}
// primaryBucketName returns <CID>.
func primaryBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, primaryPrefix, key)
func keyObject(prefix byte, cnr cid.ID, objID oid.ID) []byte {
result := make([]byte, 1+cidSize+objectKeySize)
result[0] = prefix
cnr.Encode(result[1:])
objID.Encode(result[1+cidSize:])
return result
}
// tombstoneBucketName returns <CID>_TS.
func tombstoneBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, tombstonePrefix, key)
func addressFromKey(prefix byte, key []byte) (oid.Address, error) {
if len(key) != 1+cidSize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if key[0] != prefix {
return oid.Address{}, errInvalidKeyPrefix
}
var cont cid.ID
if err := cont.Decode(key[1 : 1+cidSize]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode container ID: %w", err)
}
var obj oid.ID
if err := obj.Decode(key[1+cidSize:]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode object ID: %w", err)
}
var result oid.Address
result.SetContainer(cont)
result.SetObject(obj)
return result, nil
}
// smallBucketName returns <CID>_small.
func smallBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, smallPrefix, key)
// primaryKeyPrefix returns primaryPrefix_<CID>.
func primaryKeyPrefix(cnr cid.ID) []byte {
return keyPrefix(cnr, primaryPrefix)
}
// attributeBucketName returns <CID>_attr_<attributeKey>.
func attributeBucketName(cnr cid.ID, attributeKey string, key []byte) []byte {
key[0] = userAttributePrefix
cnr.Encode(key[1:])
return append(key[:bucketKeySize], attributeKey...)
func primaryKey(cnr cid.ID, objID oid.ID) []byte {
return keyObject(primaryPrefix, cnr, objID)
}
// returns <CID> from attributeBucketName result, nil otherwise.
func cidFromAttributeBucket(val []byte, attributeKey string) []byte {
if len(val) < bucketKeySize || val[0] != userAttributePrefix || !bytes.Equal(val[bucketKeySize:], []byte(attributeKey)) {
return nil
func addressFromPrimaryKey(v []byte) (oid.Address, error) {
return addressFromKey(primaryPrefix, v)
}
// tombstoneKeyPrefix returns tombstonePrefix_<CID>.
func tombstoneKeyPrefix(cnr cid.ID) []byte {
return keyPrefix(cnr, tombstonePrefix)
}
func tombstoneKey(cnr cid.ID, objID oid.ID) []byte {
return keyObject(tombstonePrefix, cnr, objID)
}
func addressFromTombstoneKey(v []byte) (oid.Address, error) {
return addressFromKey(tombstonePrefix, v)
}
func garbageKey(cnr cid.ID, objID oid.ID) []byte {
return keyObject(garbagePrefix, cnr, objID)
}
func addressFromGarbageKey(v []byte) (oid.Address, error) {
return addressFromKey(garbagePrefix, v)
}
func graveyardKey(cnr cid.ID, objID oid.ID) []byte {
return keyObject(graveyardPrefix, cnr, objID)
}
func addressFromGraveyardKey(v []byte) (oid.Address, error) {
return addressFromKey(graveyardPrefix, v)
}
func smallKey(cnr cid.ID, obj oid.ID) []byte {
return keyObject(smallPrefix, cnr, obj)
}
// attributeKeyPrefix returns userAttributePrefix_<attributeKey>_<CID>_<attributeValue>.
func attributeKeyPrefix(cnr cid.ID, attributeKey, attributeValue string) []byte {
result := make([]byte, 1+len(attributeKey)+cidSize+len(attributeValue))
result[0] = userAttributePrefix
copy(result[1:], []byte(attributeKey))
cnr.Encode(result[1+len(attributeKey):])
copy(result[1+len(attributeKey)+cidSize:], []byte(attributeValue))
return result
}
// userAttributePrefix+attributeKey+<CID>+attributeValue+<OID>.
func attributeKey(cnr cid.ID, objID oid.ID, attributeKey, attributeValue string) []byte {
result := make([]byte, 1+len(attributeKey)+cidSize+len(attributeValue)+objectKeySize)
result[0] = userAttributePrefix
copy(result[1:], []byte(attributeKey))
cnr.Encode(result[1+len(attributeKey):])
copy(result[1+len(attributeKey)+cidSize:], []byte(attributeValue))
objID.Encode(result[1+cidSize+len(attributeKey)+len(attributeValue):])
return result
}
// returns attributeValue from attributeKey result, nil otherwise.
func attributeValueFromAttributeKey(key []byte, attributeKey string) ([]byte, error) {
if len(key) < 1+len(attributeKey)+cidSize+objectKeySize {
return nil, errInvalidKeyLenght
}
if key[0] != userAttributePrefix {
return nil, errInvalidKeyPrefix
}
if !bytes.Equal(key[1:1+len(attributeKey)], []byte(attributeKey)) {
return nil, errInvalidAttributeKey
}
return val[1:bucketKeySize]
return key[1+len(attributeKey)+cidSize : len(key)-objectKeySize], nil
}
// payloadHashBucketName returns <CID>_payloadhash.
func payloadHashBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, payloadHashPrefix, key)
}
// rootBucketName returns <CID>_root.
func rootBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, rootPrefix, key)
}
// ownerBucketName returns <CID>_ownerid.
func ownerBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, ownerPrefix, key)
}
// parentBucketName returns <CID>_parent.
func parentBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, parentPrefix, key)
}
// splitBucketName returns <CID>_splitid.
func splitBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, splitPrefix, key)
}
// ecInfoBucketName returns <CID>_ecinfo.
func ecInfoBucketName(cnr cid.ID, key []byte) []byte {
return bucketName(cnr, ecInfoPrefix, key)
}
// addressKey returns key for K-V tables when key is a whole address.
func addressKey(addr oid.Address, key []byte) []byte {
addr.Container().Encode(key)
addr.Object().Encode(key[cidSize:])
return key[:addressKeySize]
}
// parses object address formed by addressKey.
func decodeAddressFromKey(dst *oid.Address, k []byte) error {
if len(k) != addressKeySize {
return errInvalidLength
func addressFromAttributeKey(key []byte, attributeKey string) (oid.Address, error) {
if len(key) < 1+len(attributeKey)+cidSize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if key[0] != userAttributePrefix {
return oid.Address{}, errInvalidKeyPrefix
}
if !bytes.Equal(key[1:1+len(attributeKey)], []byte(attributeKey)) {
return oid.Address{}, errInvalidAttributeKey
}
var cnrID cid.ID
if err := cnrID.Decode(key[1+len(attributeKey) : 1+len(attributeKey)+cidSize]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode container ID: %w", err)
}
var objID oid.ID
if err := objID.Decode(key[len(key)-objectKeySize:]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode object ID: %w", err)
}
var result oid.Address
result.SetContainer(cnrID)
result.SetObject(objID)
return result, nil
}
// payloadHashKeyLongPrefix returns payloadHashPrefix_<CID>_hash.
func payloadHashKeyLongPrefix(cnr cid.ID, hash []byte) []byte {
result := make([]byte, 1+cidSize+len(hash))
result[0] = payloadHashPrefix
cnr.Encode(result[1:])
copy(result[1+cidSize:], hash)
return result
}
// payloadHashKeyShortPrefix returns payloadHashPrefix_<CID>.
func payloadHashKeyShortPrefix(cnr cid.ID) []byte {
result := make([]byte, 1+cidSize)
result[0] = payloadHashPrefix
cnr.Encode(result[1:])
return result
}
// payloadHashKey returns payloadHashPrefix_<CID>_hash_<OID>.
func payloadHashKey(cnr cid.ID, obj oid.ID, hash []byte) []byte {
result := make([]byte, 1+cidSize+len(hash)+objectKeySize)
result[0] = payloadHashPrefix
cnr.Encode(result[1:])
copy(result[1+cidSize:], hash)
obj.Encode(result[1+cidSize+len(hash):])
return result
}
func addressFromPayloadHashKey(k []byte) (oid.Address, error) {
if len(k) < 1+cidSize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if k[0] != payloadHashPrefix {
return oid.Address{}, errInvalidKeyPrefix
}
var cnr cid.ID
if err := cnr.Decode(k[:cidSize]); err != nil {
return err
if err := cnr.Decode(k[1 : 1+cidSize]); err != nil {
return oid.Address{}, err
}
var obj oid.ID
if err := obj.Decode(k[cidSize:]); err != nil {
return err
if err := obj.Decode(k[len(k)-objectKeySize:]); err != nil {
return oid.Address{}, err
}
dst.SetObject(obj)
dst.SetContainer(cnr)
return nil
var result oid.Address
result.SetObject(obj)
result.SetContainer(cnr)
return result, nil
}
// objectKey returns key for K-V tables when key is an object id.
func objectKey(obj oid.ID, key []byte) []byte {
obj.Encode(key)
return key[:objectKeySize]
}
// if meets irregular object container in objs - returns its type, otherwise returns object.TypeRegular.
//
// firstIrregularObjectType(tx, cnr, obj) usage allows getting object type.
func firstIrregularObjectType(tx *bbolt.Tx, idCnr cid.ID, objs ...[]byte) objectSDK.Type {
if len(objs) == 0 {
panic("empty object list in firstIrregularObjectType")
func payloadHashFromPayloadHashKey(k []byte) ([]byte, error) {
if len(k) < 1+cidSize+objectKeySize {
return nil, errInvalidKeyLenght
}
if k[0] != payloadHashPrefix {
return nil, errInvalidKeyPrefix
}
var keys [2][1 + cidSize]byte
irregularTypeBuckets := [...]struct {
typ objectSDK.Type
name []byte
}{
{objectSDK.TypeTombstone, tombstoneBucketName(idCnr, keys[0][:])},
{objectSDK.TypeLock, bucketNameLockers(idCnr, keys[1][:])},
}
for i := range objs {
for j := range irregularTypeBuckets {
if inBucket(tx, irregularTypeBuckets[j].name, objs[i]) {
return irregularTypeBuckets[j].typ
}
}
}
return objectSDK.TypeRegular
return bytes.Clone(k[1+cidSize : len(k)-objectKeySize]), nil
}
// return true if provided object is of LOCK type.
func isLockObject(tx *bbolt.Tx, idCnr cid.ID, obj oid.ID) bool {
return inBucket(tx,
bucketNameLockers(idCnr, make([]byte, bucketKeySize)),
objectKey(obj, make([]byte, objectKeySize)))
// rootBucketName returns rootPrefix_<CID>.
func rootKeyPrefix(cnr cid.ID) []byte {
return keyPrefix(cnr, rootPrefix)
}
func rootKey(cnr cid.ID, objID oid.ID) []byte {
return keyObject(rootPrefix, cnr, objID)
}
func addressFromRootKey(key []byte) (oid.Address, error) {
return addressFromKey(rootPrefix, key)
}
// ownerKey returns ownerPrefix_<CID>_owner_<OID>.
func ownerKey(cnr cid.ID, obj oid.ID, owner []byte) []byte {
result := make([]byte, 1+cidSize+len(owner)+objectKeySize)
result[0] = ownerPrefix
cnr.Encode(result[1:])
copy(result[1+cidSize:], owner)
obj.Encode(result[1+cidSize+len(owner):])
return result
}
// ownerKeyShortPrefix returns ownerPrefix_<CID>.
func ownerKeyShortPrefix(cnr cid.ID) []byte {
result := make([]byte, 1+cidSize)
result[0] = ownerPrefix
cnr.Encode(result[1:])
return result
}
// ownerKeyLongPrefix returns ownerPrefix_<CID>_owner.
func ownerKeyLongPrefix(cnr cid.ID, owner []byte) []byte {
result := make([]byte, 1+cidSize+len(owner))
result[0] = ownerPrefix
cnr.Encode(result[1:])
copy(result[1+cidSize:], owner)
return result
}
func addressFromOwnerKey(k []byte) (oid.Address, error) {
if len(k) < 1+cidSize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if k[0] != ownerPrefix {
return oid.Address{}, errInvalidKeyPrefix
}
var cnr cid.ID
if err := cnr.Decode(k[1 : 1+cidSize]); err != nil {
return oid.Address{}, err
}
var obj oid.ID
if err := obj.Decode(k[len(k)-objectKeySize:]); err != nil {
return oid.Address{}, err
}
var result oid.Address
result.SetObject(obj)
result.SetContainer(cnr)
return result, nil
}
func ownerFromOwnerKey(k []byte) ([]byte, error) {
if len(k) < 1+cidSize+objectKeySize {
return nil, errInvalidKeyLenght
}
if k[0] != ownerPrefix {
return nil, errInvalidKeyPrefix
}
return bytes.Clone(k[1+cidSize : len(k)-objectKeySize]), nil
}
// ecInfoLongKeyPrefix returns ecInfoPrefix_<CID>_<parent_OID>.
func ecInfoLongKeyPrefix(cnr cid.ID, parent oid.ID) []byte {
result := make([]byte, 1+cidSize+objectKeySize)
result[0] = ecInfoPrefix
cnr.Encode(result[1:])
parent.Encode(result[1+cidSize:])
return result
}
// ecInfoShortKeyPrefix returns ecInfoPrefix_<CID>.
func ecInfoShortKeyPrefix(cnr cid.ID) []byte {
result := make([]byte, 1+cidSize)
result[0] = ecInfoPrefix
cnr.Encode(result[1:])
return result
}
func ecInfoKey(cnr cid.ID, parent, chunk oid.ID) []byte {
result := make([]byte, 1+cidSize+objectKeySize+objectKeySize)
result[0] = ecInfoPrefix
cnr.Encode(result[1:])
parent.Encode(result[1+cidSize:])
chunk.Encode(result[1+cidSize+objectKeySize:])
return result
}
func addressOfParentFromECInfoKey(key []byte) (oid.Address, error) {
return addressFromKey(ecInfoPrefix, key[:1+cidSize+objectKeySize])
}
func addressOfChunkFromECInfoKey(key []byte) (oid.Address, error) {
if len(key) != 1+cidSize+objectKeySize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if key[0] != ecInfoPrefix {
return oid.Address{}, errInvalidKeyPrefix
}
var cont cid.ID
if err := cont.Decode(key[1 : 1+cidSize]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode container ID: %w", err)
}
var obj oid.ID
if err := obj.Decode(key[1+cidSize+objectKeySize:]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode object ID: %w", err)
}
var result oid.Address
result.SetContainer(cont)
result.SetObject(obj)
return result, nil
}
// parentKeyShortPrefix returns parentPrefix_<CID>.
func parentKeyShortPrefix(cnr cid.ID) []byte {
result := make([]byte, 1+cidSize)
result[0] = parentPrefix
cnr.Encode(result[1:])
return result
}
func addressOfParentFromParentKey(key []byte) (oid.Address, error) {
return addressFromKey(parentPrefix, key[:1+cidSize+objectKeySize])
}
func addressOfTargetFromParentKey(key []byte) (oid.Address, error) {
if len(key) != 1+cidSize+objectKeySize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if key[0] != parentPrefix {
return oid.Address{}, errInvalidKeyPrefix
}
var cont cid.ID
if err := cont.Decode(key[1 : 1+cidSize]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode container ID: %w", err)
}
var obj oid.ID
if err := obj.Decode(key[1+cidSize+objectKeySize:]); err != nil {
return oid.Address{}, fmt.Errorf("failed to decode object ID: %w", err)
}
var result oid.Address
result.SetContainer(cont)
result.SetObject(obj)
return result, nil
}
// parentKeyLongPrefix returns parentPrefix_<CID>_<parent_OID>.
func parentKeyLongPrefix(cnr cid.ID, parentObj oid.ID) []byte {
result := make([]byte, 1+cidSize+objectKeySize)
result[0] = parentPrefix
cnr.Encode(result[1:])
parentObj.Encode(result[bucketKeySize:])
return result
}
func parentKey(cnr cid.ID, parentObj, obj oid.ID) []byte {
result := make([]byte, 1+cidSize+objectKeySize+objectKeySize)
result[0] = parentPrefix
cnr.Encode(result[1:])
parentObj.Encode(result[1+cidSize:])
obj.Encode(result[1+cidSize+objectKeySize:])
return result
}
// splitKeyLongPrefix returns splitPrefix_<CID>_splitID.
func splitKeyLongPrefix(cnr cid.ID, splitID []byte) []byte {
result := make([]byte, 1+cidSize+len(splitID))
result[0] = splitPrefix
cnr.Encode(result[1:])
copy(result[1+cidSize:], splitID)
return result
}
// splitKeyShortPrefix returns splitPrefix_<CID>.
func splitKeyShortPrefix(cnr cid.ID) []byte {
result := make([]byte, 1+cidSize)
result[0] = splitPrefix
cnr.Encode(result[1:])
return result
}
// splitKey returns splitPrefix_<CID>_splitID_<OID>.
func splitKey(cnr cid.ID, obj oid.ID, splitID []byte) []byte {
result := make([]byte, 1+cidSize+len(splitID)+objectKeySize)
result[0] = splitPrefix
cnr.Encode(result[1:])
copy(result[1+cidSize:], splitID)
obj.Encode(result[1+cidSize+len(splitID):])
return result
}
func addressFromSplitKey(key []byte) (oid.Address, error) {
if len(key) < 1+cidSize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if key[0] != splitPrefix {
return oid.Address{}, errInvalidKeyPrefix
}
var cnr cid.ID
if err := cnr.Decode(key[1 : 1+cidSize]); err != nil {
return oid.Address{}, err
}
var obj oid.ID
if err := obj.Decode(key[len(key)-objectKeySize:]); err != nil {
return oid.Address{}, err
}
var result oid.Address
result.SetObject(obj)
result.SetContainer(cnr)
return result, nil
}
func splitIDFromSplitKey(key []byte) ([]byte, error) {
if len(key) < 1+cidSize+objectKeySize {
return nil, errInvalidKeyLenght
}
if key[0] != splitPrefix {
return nil, errInvalidKeyPrefix
}
return bytes.Clone(key[1+cidSize : len(key)-objectKeySize]), nil
}
// returns prefix of the keys with objects of type LOCK for specified container.
func lockersKeyPrefix(idCnr cid.ID) []byte {
return keyPrefix(idCnr, lockersPrefix)
}
func lockersKey(cnrID cid.ID, objID oid.ID) []byte {
return keyObject(lockersPrefix, cnrID, objID)
}
func addressFromLockersKey(v []byte) (oid.Address, error) {
return addressFromKey(lockersPrefix, v)
}
// returns lockedPrefix_<CID>_<OID>.
func lockedKeyLongPrefix(cnrID cid.ID, objID oid.ID) []byte {
prefix := make([]byte, 1+cidSize+objectKeySize)
prefix[0] = lockedPrefix
cnrID.Encode(prefix[1:])
objID.Encode(prefix[1+cidSize:])
return prefix
}
// returns lockedPrefix_<CID>.
func lockedKeyShortPrefix(cnrID cid.ID) []byte {
prefix := make([]byte, 1+cidSize)
prefix[0] = lockedPrefix
cnrID.Encode(prefix[1:])
return prefix
}
// returns lockedPrefix_<CID>_<OID>_<LOCKER_OID>.
func lockedKey(cnrID cid.ID, objID, lockerObjID oid.ID) []byte {
result := make([]byte, 1+cidSize+objectKeySize+objectKeySize)
result[0] = lockedPrefix
cnrID.Encode(result[1:])
objID.Encode(result[1+cidSize:])
lockerObjID.Encode(result[1+cidSize+objectKeySize:])
return result
}
func lockerObjectIDFromLockedKey(k []byte) (oid.ID, error) {
if len(k) != 1+cidSize+objectKeySize+objectKeySize {
return oid.ID{}, errInvalidKeyLenght
}
if k[0] != lockedPrefix {
return oid.ID{}, errInvalidKeyPrefix
}
var result oid.ID
if err := result.Decode(k[1+cidSize+objectKeySize:]); err != nil {
return oid.ID{}, fmt.Errorf("failed to decode lockers object ID: %w", err)
}
return result, nil
}
func objectIDFromLockedKey(k []byte) (oid.ID, error) {
if len(k) != 1+cidSize+objectKeySize+objectKeySize {
return oid.ID{}, errInvalidKeyLenght
}
if k[0] != lockedPrefix {
return oid.ID{}, errInvalidKeyPrefix
}
var result oid.ID
if err := result.Decode(k[1+cidSize : 1+cidSize+objectKeySize]); err != nil {
return oid.ID{}, fmt.Errorf("failed to decode locked object ID: %w", err)
}
return result, nil
}
func expiredKey(cnr cid.ID, obj oid.ID, epoch uint64) []byte {
result := make([]byte, 1+8+cidSize+objectKeySize)
result[0] = expiredPrefix
// BigEndian is important for iteration order
binary.BigEndian.PutUint64(result[1:1+8], epoch)
cnr.Encode(result[1+8 : 1+8+cidSize])
obj.Encode(result[1+8+cidSize:])
return result
}
func expirationEpochFromExpiredKey(key []byte) (uint64, error) {
if len(key) != 1+8+cidSize+objectKeySize {
return 0, errInvalidKeyLenght
}
if key[0] != expiredPrefix {
return 0, errInvalidKeyPrefix
}
// BigEndian is important for iteration order
return binary.BigEndian.Uint64(key[1 : 1+8]), nil
}
func addressFromExpiredKey(key []byte) (oid.Address, error) {
if len(key) != 1+8+cidSize+objectKeySize {
return oid.Address{}, errInvalidKeyLenght
}
if key[0] != expiredPrefix {
return oid.Address{}, errInvalidKeyPrefix
}
var cnr cid.ID
if err := cnr.Decode(key[1+8 : 1+8+cidSize]); err != nil {
return oid.Address{}, err
}
var obj oid.ID
if err := obj.Decode(key[len(key)-objectKeySize:]); err != nil {
return oid.Address{}, err
}
var result oid.Address
result.SetObject(obj)
result.SetContainer(cnr)
return result, nil
}
func shardInfoKey(key []byte) []byte {
result := make([]byte, len(key))
result[0] = shardInfoPrefix
copy(result[1:], key)
return result
}

View file

@ -5,7 +5,7 @@ import (
"fmt"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
"go.etcd.io/bbolt"
"github.com/cockroachdb/pebble"
)
// version contains current metabase version.
@ -18,25 +18,25 @@ var versionKey = []byte("version")
// the current code version.
var ErrOutdatedVersion = logicerr.New("invalid version, resynchronization is required")
func checkVersion(tx *bbolt.Tx, initialized bool) error {
func checkVersion(b *pebble.Batch, initialized bool) error {
var knownVersion bool
b := tx.Bucket(shardInfoBucket)
if b != nil {
data := b.Get(versionKey)
if len(data) == 8 {
knownVersion = true
data, err := valueSafe(b, shardInfoKey(versionKey))
if err != nil {
return err
}
if len(data) == 8 {
knownVersion = true
stored := binary.LittleEndian.Uint64(data)
if stored != version {
return fmt.Errorf("%w: expected=%d, stored=%d", ErrOutdatedVersion, version, stored)
}
stored := binary.LittleEndian.Uint64(data)
if stored != version {
return fmt.Errorf("%w: expected=%d, stored=%d", ErrOutdatedVersion, version, stored)
}
}
if !initialized {
// new database, write version
return updateVersion(tx, version)
return updateVersion(b, version)
} else if !knownVersion {
// db is initialized but no version
// has been found; that could happen
@ -49,13 +49,8 @@ func checkVersion(tx *bbolt.Tx, initialized bool) error {
return nil
}
func updateVersion(tx *bbolt.Tx, version uint64) error {
func updateVersion(b *pebble.Batch, version uint64) error {
data := make([]byte, 8)
binary.LittleEndian.PutUint64(data, version)
b, err := tx.CreateBucketIfNotExists(shardInfoBucket)
if err != nil {
return fmt.Errorf("can't create auxiliary bucket: %w", err)
}
return b.Put(versionKey, data)
return b.Set(shardInfoKey(versionKey), data, pebble.Sync)
}

View file

@ -9,8 +9,8 @@ import (
"testing"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
"github.com/cockroachdb/pebble"
"github.com/stretchr/testify/require"
"go.etcd.io/bbolt"
)
type epochStateImpl struct{}
@ -27,12 +27,9 @@ func TestVersion(t *testing.T) {
WithPermissions(0o600), WithEpochState(epochStateImpl{}))
}
check := func(t *testing.T, db *DB) {
require.NoError(t, db.boltDB.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(shardInfoBucket)
if b == nil {
return errors.New("shard info bucket not found")
}
data := b.Get(versionKey)
require.NoError(t, db.snapshot(func(s *pebble.Snapshot) error {
data, err := valueSafe(s, shardInfoKey(versionKey))
require.NoError(t, err)
if len(data) != 8 {
return errors.New("invalid version data")
}
@ -68,8 +65,8 @@ func TestVersion(t *testing.T) {
t.Run("invalid version", func(t *testing.T) {
db := newDB(t)
require.NoError(t, db.Open(context.Background(), mode.ReadWrite))
require.NoError(t, db.boltDB.Update(func(tx *bbolt.Tx) error {
return updateVersion(tx, version+1)
require.NoError(t, db.batch(func(b *pebble.Batch) error {
return updateVersion(b, version+1)
}))
require.NoError(t, db.Close())
@ -79,7 +76,7 @@ func TestVersion(t *testing.T) {
t.Run("reset", func(t *testing.T) {
require.NoError(t, db.Open(context.Background(), mode.ReadWrite))
require.NoError(t, db.Reset())
require.NoError(t, db.Reset(context.Background()))
check(t, db)
require.NoError(t, db.Close())
})

View file

@ -26,7 +26,7 @@ func (r ContainerSizeRes) Size() uint64 {
return r.size
}
func (s *Shard) ContainerSize(prm ContainerSizePrm) (ContainerSizeRes, error) {
func (s *Shard) ContainerSize(ctx context.Context, prm ContainerSizePrm) (ContainerSizeRes, error) {
s.m.RLock()
defer s.m.RUnlock()
@ -34,7 +34,7 @@ func (s *Shard) ContainerSize(prm ContainerSizePrm) (ContainerSizeRes, error) {
return ContainerSizeRes{}, ErrDegradedMode
}
size, err := s.metaBase.ContainerSize(prm.cnr)
size, err := s.metaBase.ContainerSize(ctx, prm.cnr)
if err != nil {
return ContainerSizeRes{}, fmt.Errorf("could not get container size: %w", err)
}

View file

@ -194,7 +194,7 @@ func (s *Shard) refillMetabase(ctx context.Context) error {
}
}()
err := s.metaBase.Reset()
err := s.metaBase.Reset(ctx)
if err != nil {
return fmt.Errorf("could not reset metabase: %w", err)
}
@ -248,7 +248,7 @@ func (s *Shard) refillMetabase(ctx context.Context) error {
return fmt.Errorf("could not put objects to the meta: %w", err)
}
err = s.metaBase.SyncCounters()
err = s.metaBase.SyncCounters(ctx)
if err != nil {
return fmt.Errorf("could not sync object counters: %w", err)
}
@ -410,7 +410,7 @@ func (s *Shard) Reload(ctx context.Context, opts ...Option) error {
// config after the node was updated.
err = s.refillMetabase(ctx)
} else {
err = s.metaBase.Init()
err = s.metaBase.Init2(ctx)
}
if err != nil {
s.log.Error(logs.ShardCantInitializeMetabaseMoveToADegradedreadonlyMode, zap.Error(err))

View file

@ -3,7 +3,6 @@ package shard
import (
"context"
"fmt"
"io/fs"
"math"
"os"
"path/filepath"
@ -28,7 +27,6 @@ import (
oidtest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id/test"
objecttest "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/test"
"github.com/stretchr/testify/require"
"go.etcd.io/bbolt"
)
type objAddr struct {
@ -37,6 +35,7 @@ type objAddr struct {
}
func TestShardOpen(t *testing.T) {
t.Skip("not implemented for pebble")
t.Parallel()
dir := t.TempDir()
@ -49,13 +48,6 @@ func TestShardOpen(t *testing.T) {
))
var allowedMode atomic.Int64
openFileMetabase := func(p string, f int, perm fs.FileMode) (*os.File, error) {
const modeMask = os.O_RDONLY | os.O_RDWR | os.O_WRONLY
if int64(f&modeMask) == allowedMode.Load() {
return os.OpenFile(p, f, perm)
}
return nil, fs.ErrPermission
}
wcOpts := []writecache.Option{
writecache.WithPath(filepath.Join(dir, "wc")),
@ -72,7 +64,6 @@ func TestShardOpen(t *testing.T) {
WithMetaBaseOptions(
meta.WithPath(metaPath),
meta.WithEpochState(epochState{}),
meta.WithBoltDBOptions(&bbolt.Options{OpenFile: openFileMetabase}),
),
WithPiloramaOptions(
pilorama.WithPath(filepath.Join(dir, "pilorama"))),
@ -352,7 +343,7 @@ func TestRefillMetabase(t *testing.T) {
checkTombMembers(true)
checkLocked(t, cnrLocked, locked)
c, err := sh.metaBase.ObjectCounters()
c, err := sh.metaBase.ObjectCounters(context.Background())
require.NoError(t, err)
phyBefore := c.Phy
@ -388,7 +379,7 @@ func TestRefillMetabase(t *testing.T) {
err = sh.refillMetabase(context.Background())
require.NoError(t, err)
c, err = sh.metaBase.ObjectCounters()
c, err = sh.metaBase.ObjectCounters(context.Background())
require.NoError(t, err)
require.Equal(t, phyBefore, c.Phy)

View file

@ -23,7 +23,7 @@ func (s *Shard) LogicalObjectsCount(ctx context.Context) (uint64, error) {
return 0, ErrDegradedMode
}
cc, err := s.metaBase.ObjectCounters()
cc, err := s.metaBase.ObjectCounters(ctx)
if err != nil {
return 0, err
}

View file

@ -10,6 +10,7 @@ import (
tracingPkg "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/tracing"
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
@ -141,18 +142,25 @@ func (s *Shard) deleteFromBlobstor(ctx context.Context, addr oid.Address) error
func (s *Shard) deleteFromMetabase(ctx context.Context, addr oid.Address) error {
var delPrm meta.DeletePrm
delPrm.SetAddresses(addr)
delPrm.Address = addr
res, err := s.metaBase.Delete(ctx, delPrm)
if err != nil {
return err
}
s.decObjectCounterBy(physical, res.PhyCount())
s.decObjectCounterBy(logical, res.LogicCount())
s.decObjectCounterBy(user, res.UserCount())
s.decContainerObjectCounter(res.RemovedByCnrID())
s.addToContainerSize(addr.Container().EncodeToString(), -int64(res.LogicSize()))
s.addToPayloadSize(-int64(res.PhySize()))
s.decObjectCounterBy(physical, res.PhyCount)
s.decObjectCounterBy(logical, res.LogicCount)
s.decObjectCounterBy(user, res.UserCount)
containerCounter := map[cid.ID]meta.ObjectCounters{
addr.Container(): {
Logic: res.LogicCount,
Phy: res.PhyCount,
User: res.UserCount,
},
}
s.decContainerObjectCounter(containerCounter)
s.addToContainerSize(addr.Container().EncodeToString(), -int64(res.LogicSize))
s.addToPayloadSize(-int64(res.PhySize))
return nil
}

View file

@ -678,7 +678,7 @@ func (s *Shard) HandleExpiredLocks(ctx context.Context, epoch uint64, lockers []
if s.GetMode().NoMetabase() {
return
}
unlocked, err := s.metaBase.FreeLockedBy(lockers)
unlocked, err := s.metaBase.FreeLockedBy(ctx, lockers)
if err != nil {
s.log.Warn(logs.ShardFailureToUnlockObjects,
zap.String("error", err.Error()),
@ -730,12 +730,12 @@ func (s *Shard) inhumeUnlockedIfExpired(ctx context.Context, epoch uint64, unloc
}
// HandleDeletedLocks unlocks all objects which were locked by lockers.
func (s *Shard) HandleDeletedLocks(lockers []oid.Address) {
func (s *Shard) HandleDeletedLocks(ctx context.Context, lockers []oid.Address) {
if s.GetMode().NoMetabase() {
return
}
_, err := s.metaBase.FreeLockedBy(lockers)
_, err := s.metaBase.FreeLockedBy(ctx, lockers)
if err != nil {
s.log.Warn(logs.ShardFailureToUnlockObjects,
zap.String("error", err.Error()),

View file

@ -61,8 +61,8 @@ func Test_ObjectNotFoundIfNotDeletedFromMetabase(t *testing.T) {
meta.WithEpochState(epochState{}),
),
WithPiloramaOptions(pilorama.WithPath(filepath.Join(rootPath, "pilorama"))),
WithDeletedLockCallback(func(_ context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(addresses)
WithDeletedLockCallback(func(ctx context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(ctx, addresses)
}),
WithExpiredLocksCallback(func(ctx context.Context, epoch uint64, a []oid.Address) {
sh.HandleExpiredLocks(ctx, epoch, a)

View file

@ -53,8 +53,8 @@ func TestShard_Lock(t *testing.T) {
meta.WithPath(filepath.Join(rootPath, "meta")),
meta.WithEpochState(epochState{}),
),
WithDeletedLockCallback(func(_ context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(addresses)
WithDeletedLockCallback(func(ctx context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(ctx, addresses)
}),
}

View file

@ -111,6 +111,7 @@ func TestShardReload(t *testing.T) {
t.Run("reload failed", func(t *testing.T) {
badPath := filepath.Join(p, "meta3")
require.NoError(t, os.RemoveAll(badPath))
require.NoError(t, os.WriteFile(badPath, []byte{1}, 0))
newOpts = newShardOpts(badPath, true)

View file

@ -439,7 +439,7 @@ func (s *Shard) updateMetrics(ctx context.Context) {
return
}
cc, err := s.metaBase.ObjectCounters()
cc, err := s.metaBase.ObjectCounters(ctx)
if err != nil {
s.log.Warn(logs.ShardMetaObjectCounterRead,
zap.Error(err),
@ -461,7 +461,7 @@ func (s *Shard) updateMetrics(ctx context.Context) {
var totalPayload uint64
for i := range cnrList {
size, err := s.metaBase.ContainerSize(cnrList[i])
size, err := s.metaBase.ContainerSize(ctx, cnrList[i])
if err != nil {
s.log.Warn(logs.ShardMetaCantReadContainerSize,
zap.String("cid", cnrList[i].EncodeToString()),

View file

@ -89,8 +89,8 @@ func newCustomShard(t testing.TB, enableWriteCache bool, o shardOptions) *Shard
WithPiloramaOptions(pilorama.WithPath(filepath.Join(o.rootPath, "pilorama"))),
WithWriteCache(enableWriteCache),
WithWriteCacheOptions(o.wcOpts),
WithDeletedLockCallback(func(_ context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(addresses)
WithDeletedLockCallback(func(ctx context.Context, addresses []oid.Address) {
sh.HandleDeletedLocks(ctx, addresses)
}),
WithExpiredLocksCallback(func(ctx context.Context, epoch uint64, a []oid.Address) {
sh.HandleExpiredLocks(ctx, epoch, a)