forked from TrueCloudLab/frostfs-node
[#242] node: Add tracing spans
Add tracing spans for PUT requests. Add tracing spans for DELETE requests. Add tracing spans for SELECT requests. Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
This commit is contained in:
parent
200fc8b882
commit
d62c6e4ce6
122 changed files with 863 additions and 417 deletions
|
@ -308,7 +308,7 @@ loop:
|
|||
e.removeShards(shardsToRemove...)
|
||||
|
||||
for _, p := range shardsToReload {
|
||||
err := p.sh.Reload(p.opts...)
|
||||
err := p.sh.Reload(ctx, p.opts...)
|
||||
if err != nil {
|
||||
e.log.Error(logs.EngineCouldNotReloadAShard,
|
||||
zap.Stringer("shard id", p.sh.ID()),
|
||||
|
|
|
@ -204,7 +204,7 @@ func TestExecBlocks(t *testing.T) {
|
|||
|
||||
addr := object.AddressOf(obj)
|
||||
|
||||
require.NoError(t, Put(e, obj))
|
||||
require.NoError(t, Put(context.Background(), e, obj))
|
||||
|
||||
// block executions
|
||||
errBlock := errors.New("block exec err")
|
||||
|
|
|
@ -4,11 +4,14 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/pkg/tracing"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
||||
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
|
||||
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
||||
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
@ -47,6 +50,13 @@ func (p *DeletePrm) WithForceRemoval() {
|
|||
// on operations with that object) if WithForceRemoval option has
|
||||
// been provided.
|
||||
func (e *StorageEngine) Delete(ctx context.Context, prm DeletePrm) (res DeleteRes, err error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Delete",
|
||||
trace.WithAttributes(
|
||||
attribute.String("address", prm.addr.EncodeToString()),
|
||||
attribute.Bool("force_removal", prm.forceRemoval),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
err = e.execIfNotBlocked(func() error {
|
||||
res, err = e.delete(ctx, prm)
|
||||
return err
|
||||
|
@ -135,7 +145,7 @@ func (e *StorageEngine) deleteChildren(ctx context.Context, addr oid.Address, fo
|
|||
}
|
||||
|
||||
e.iterateOverSortedShards(addr, func(_ int, sh hashedShard) (stop bool) {
|
||||
res, err := sh.Select(selectPrm)
|
||||
res, err := sh.Select(ctx, selectPrm)
|
||||
if err != nil {
|
||||
e.log.Warn(logs.EngineErrorDuringSearchingForObjectChildren,
|
||||
zap.Stringer("addr", addr),
|
||||
|
|
|
@ -59,9 +59,9 @@ func TestDeleteBigObject(t *testing.T) {
|
|||
defer e.Close()
|
||||
|
||||
for i := range children {
|
||||
require.NoError(t, Put(e, children[i]))
|
||||
require.NoError(t, Put(context.Background(), e, children[i]))
|
||||
}
|
||||
require.NoError(t, Put(e, link))
|
||||
require.NoError(t, Put(context.Background(), e, link))
|
||||
|
||||
var splitErr *objectSDK.SplitInfoError
|
||||
|
||||
|
|
|
@ -60,7 +60,7 @@ func benchmarkExists(b *testing.B, shardNum int) {
|
|||
addr := oidtest.Address()
|
||||
for i := 0; i < 100; i++ {
|
||||
obj := testutil.GenerateObjectWithCID(cidtest.ID())
|
||||
err := Put(e, obj)
|
||||
err := Put(context.Background(), e, obj)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
|
@ -69,7 +69,7 @@ func benchmarkExists(b *testing.B, shardNum int) {
|
|||
b.ReportAllocs()
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
ok, err := e.exists(addr)
|
||||
ok, err := e.exists(context.Background(), addr)
|
||||
if err != nil || ok {
|
||||
b.Fatalf("%t %v", ok, err)
|
||||
}
|
||||
|
|
|
@ -98,7 +98,7 @@ func TestErrorReporting(t *testing.T) {
|
|||
var prm shard.PutPrm
|
||||
prm.SetObject(obj)
|
||||
te.ng.mtx.RLock()
|
||||
_, err := te.ng.shards[te.shards[0].id.String()].Shard.Put(prm)
|
||||
_, err := te.ng.shards[te.shards[0].id.String()].Shard.Put(context.Background(), prm)
|
||||
te.ng.mtx.RUnlock()
|
||||
require.NoError(t, err)
|
||||
|
||||
|
@ -132,7 +132,7 @@ func TestErrorReporting(t *testing.T) {
|
|||
var prm shard.PutPrm
|
||||
prm.SetObject(obj)
|
||||
te.ng.mtx.RLock()
|
||||
_, err := te.ng.shards[te.shards[0].id.String()].Put(prm)
|
||||
_, err := te.ng.shards[te.shards[0].id.String()].Put(context.Background(), prm)
|
||||
te.ng.mtx.RUnlock()
|
||||
require.NoError(t, err)
|
||||
|
||||
|
@ -185,7 +185,7 @@ func TestBlobstorFailback(t *testing.T) {
|
|||
var prm shard.PutPrm
|
||||
prm.SetObject(obj)
|
||||
te.ng.mtx.RLock()
|
||||
_, err = te.ng.shards[te.shards[0].id.String()].Shard.Put(prm)
|
||||
_, err = te.ng.shards[te.shards[0].id.String()].Shard.Put(context.Background(), prm)
|
||||
te.ng.mtx.RUnlock()
|
||||
require.NoError(t, err)
|
||||
objs = append(objs, obj)
|
||||
|
|
|
@ -57,7 +57,7 @@ func newEngineEvacuate(t *testing.T, shardNum int, objPerShard int) (*StorageEng
|
|||
|
||||
var putPrm shard.PutPrm
|
||||
putPrm.SetObject(obj)
|
||||
_, err := e.shards[sh.String()].Put(putPrm)
|
||||
_, err := e.shards[sh.String()].Put(context.Background(), putPrm)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
|
@ -67,7 +67,7 @@ func newEngineEvacuate(t *testing.T, shardNum int, objPerShard int) (*StorageEng
|
|||
var putPrm PutPrm
|
||||
putPrm.WithObject(objects[len(objects)-1])
|
||||
|
||||
_, err := e.Put(putPrm)
|
||||
err := e.Put(context.Background(), putPrm)
|
||||
require.NoError(t, err)
|
||||
|
||||
res, err := e.shards[ids[len(ids)-1].String()].List()
|
||||
|
|
|
@ -10,14 +10,14 @@ import (
|
|||
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
||||
)
|
||||
|
||||
func (e *StorageEngine) exists(addr oid.Address) (bool, error) {
|
||||
func (e *StorageEngine) exists(ctx context.Context, addr oid.Address) (bool, error) {
|
||||
var shPrm shard.ExistsPrm
|
||||
shPrm.SetAddress(addr)
|
||||
alreadyRemoved := false
|
||||
exists := false
|
||||
|
||||
e.iterateOverSortedShards(addr, func(_ int, sh hashedShard) (stop bool) {
|
||||
res, err := sh.Exists(context.TODO(), shPrm)
|
||||
res, err := sh.Exists(ctx, shPrm)
|
||||
if err != nil {
|
||||
if shard.IsErrRemoved(err) {
|
||||
alreadyRemoved = true
|
||||
|
|
|
@ -48,6 +48,12 @@ func (r GetRes) Object() *objectSDK.Object {
|
|||
//
|
||||
// Returns an error if executions are blocked (see BlockExecution).
|
||||
func (e *StorageEngine) Get(ctx context.Context, prm GetPrm) (res GetRes, err error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Get",
|
||||
trace.WithAttributes(
|
||||
attribute.String("address", prm.addr.EncodeToString()),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
err = e.execIfNotBlocked(func() error {
|
||||
res, err = e.get(ctx, prm)
|
||||
return err
|
||||
|
@ -57,12 +63,6 @@ func (e *StorageEngine) Get(ctx context.Context, prm GetPrm) (res GetRes, err er
|
|||
}
|
||||
|
||||
func (e *StorageEngine) get(ctx context.Context, prm GetPrm) (GetRes, error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.get",
|
||||
trace.WithAttributes(
|
||||
attribute.String("address", prm.addr.EncodeToString()),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
if e.metrics != nil {
|
||||
defer elapsed(e.metrics.AddGetDuration)()
|
||||
}
|
||||
|
|
|
@ -55,11 +55,11 @@ func TestHeadRaw(t *testing.T) {
|
|||
putPrmLink.SetObject(link)
|
||||
|
||||
// put most left object in one shard
|
||||
_, err := s1.Put(putPrmLeft)
|
||||
_, err := s1.Put(context.Background(), putPrmLeft)
|
||||
require.NoError(t, err)
|
||||
|
||||
// put link object in another shard
|
||||
_, err = s2.Put(putPrmLink)
|
||||
_, err = s2.Put(context.Background(), putPrmLink)
|
||||
require.NoError(t, err)
|
||||
|
||||
// head with raw flag should return SplitInfoError
|
||||
|
|
|
@ -4,12 +4,15 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/pkg/tracing"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
|
||||
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
||||
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
|
||||
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
||||
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
@ -62,6 +65,9 @@ var errInhumeFailure = errors.New("inhume operation failed")
|
|||
//
|
||||
// Returns an error if executions are blocked (see BlockExecution).
|
||||
func (e *StorageEngine) Inhume(ctx context.Context, prm InhumePrm) (res InhumeRes, err error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Inhume")
|
||||
defer span.End()
|
||||
|
||||
err = e.execIfNotBlocked(func() error {
|
||||
res, err = e.inhume(ctx, prm)
|
||||
return err
|
||||
|
@ -82,7 +88,7 @@ func (e *StorageEngine) inhume(ctx context.Context, prm InhumePrm) (InhumeRes, e
|
|||
|
||||
for i := range prm.addrs {
|
||||
if !prm.forceRemoval {
|
||||
locked, err := e.IsLocked(prm.addrs[i])
|
||||
locked, err := e.IsLocked(ctx, prm.addrs[i])
|
||||
if err != nil {
|
||||
e.log.Warn(logs.EngineRemovingAnObjectWithoutFullLockingCheck,
|
||||
zap.Error(err),
|
||||
|
@ -181,13 +187,19 @@ func (e *StorageEngine) inhumeAddr(ctx context.Context, addr oid.Address, prm sh
|
|||
}
|
||||
|
||||
// IsLocked checks whether an object is locked according to StorageEngine's state.
|
||||
func (e *StorageEngine) IsLocked(addr oid.Address) (bool, error) {
|
||||
func (e *StorageEngine) IsLocked(ctx context.Context, addr oid.Address) (bool, error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.IsLocked",
|
||||
trace.WithAttributes(
|
||||
attribute.String("address", addr.EncodeToString()),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
var locked bool
|
||||
var err error
|
||||
var outErr error
|
||||
|
||||
e.iterateOverUnsortedShards(func(h hashedShard) (stop bool) {
|
||||
locked, err = h.Shard.IsLocked(addr)
|
||||
locked, err = h.Shard.IsLocked(ctx, addr)
|
||||
if err != nil {
|
||||
e.reportShardError(h, "can't check object's lockers", err, zap.Stringer("addr", addr))
|
||||
outErr = err
|
||||
|
@ -206,7 +218,7 @@ func (e *StorageEngine) IsLocked(addr oid.Address) (bool, error) {
|
|||
|
||||
func (e *StorageEngine) processExpiredTombstones(ctx context.Context, addrs []meta.TombstonedObject) {
|
||||
e.iterateOverUnsortedShards(func(sh hashedShard) (stop bool) {
|
||||
sh.HandleExpiredTombstones(addrs)
|
||||
sh.HandleExpiredTombstones(ctx, addrs)
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
|
|
|
@ -42,7 +42,7 @@ func TestStorageEngine_Inhume(t *testing.T) {
|
|||
e := testNewEngine(t).setShardsNum(t, 1).engine
|
||||
defer e.Close()
|
||||
|
||||
err := Put(e, parent)
|
||||
err := Put(context.Background(), e, parent)
|
||||
require.NoError(t, err)
|
||||
|
||||
var inhumePrm InhumePrm
|
||||
|
@ -51,7 +51,7 @@ func TestStorageEngine_Inhume(t *testing.T) {
|
|||
_, err = e.Inhume(context.Background(), inhumePrm)
|
||||
require.NoError(t, err)
|
||||
|
||||
addrs, err := Select(e, cnr, fs)
|
||||
addrs, err := Select(context.Background(), e, cnr, fs)
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, addrs)
|
||||
})
|
||||
|
@ -65,12 +65,12 @@ func TestStorageEngine_Inhume(t *testing.T) {
|
|||
|
||||
var putChild shard.PutPrm
|
||||
putChild.SetObject(child)
|
||||
_, err := s1.Put(putChild)
|
||||
_, err := s1.Put(context.Background(), putChild)
|
||||
require.NoError(t, err)
|
||||
|
||||
var putLink shard.PutPrm
|
||||
putLink.SetObject(link)
|
||||
_, err = s2.Put(putLink)
|
||||
_, err = s2.Put(context.Background(), putLink)
|
||||
require.NoError(t, err)
|
||||
|
||||
var inhumePrm InhumePrm
|
||||
|
@ -79,7 +79,7 @@ func TestStorageEngine_Inhume(t *testing.T) {
|
|||
_, err = e.Inhume(context.Background(), inhumePrm)
|
||||
require.NoError(t, err)
|
||||
|
||||
addrs, err := Select(e, cnr, fs)
|
||||
addrs, err := Select(context.Background(), e, cnr, fs)
|
||||
require.NoError(t, err)
|
||||
require.Empty(t, addrs)
|
||||
})
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package engine
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"os"
|
||||
"sort"
|
||||
|
@ -35,7 +36,7 @@ func TestListWithCursor(t *testing.T) {
|
|||
var prm PutPrm
|
||||
prm.WithObject(obj)
|
||||
|
||||
_, err := e.Put(prm)
|
||||
err := e.Put(context.Background(), prm)
|
||||
require.NoError(t, err)
|
||||
expected = append(expected, object.AddressWithType{Type: objectSDK.TypeRegular, Address: object.AddressOf(obj)})
|
||||
}
|
||||
|
|
|
@ -4,12 +4,15 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/pkg/tracing"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
|
||||
apistatus "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client/status"
|
||||
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
|
||||
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
||||
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
)
|
||||
|
||||
var errLockFailed = errors.New("lock operation failed")
|
||||
|
@ -20,19 +23,27 @@ var errLockFailed = errors.New("lock operation failed")
|
|||
// Allows locking regular objects only (otherwise returns apistatus.LockNonRegularObject).
|
||||
//
|
||||
// Locked list should be unique. Panics if it is empty.
|
||||
func (e *StorageEngine) Lock(idCnr cid.ID, locker oid.ID, locked []oid.ID) error {
|
||||
func (e *StorageEngine) Lock(ctx context.Context, idCnr cid.ID, locker oid.ID, locked []oid.ID) error {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Lock",
|
||||
trace.WithAttributes(
|
||||
attribute.String("container_id", idCnr.EncodeToString()),
|
||||
attribute.String("locker", locker.EncodeToString()),
|
||||
attribute.Int("locked_count", len(locked)),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
return e.execIfNotBlocked(func() error {
|
||||
return e.lock(idCnr, locker, locked)
|
||||
return e.lock(ctx, idCnr, locker, locked)
|
||||
})
|
||||
}
|
||||
|
||||
func (e *StorageEngine) lock(idCnr cid.ID, locker oid.ID, locked []oid.ID) error {
|
||||
func (e *StorageEngine) lock(ctx context.Context, idCnr cid.ID, locker oid.ID, locked []oid.ID) error {
|
||||
for i := range locked {
|
||||
switch e.lockSingle(idCnr, locker, locked[i], true) {
|
||||
switch e.lockSingle(ctx, idCnr, locker, locked[i], true) {
|
||||
case 1:
|
||||
return logicerr.Wrap(apistatus.LockNonRegularObject{})
|
||||
case 0:
|
||||
switch e.lockSingle(idCnr, locker, locked[i], false) {
|
||||
switch e.lockSingle(ctx, idCnr, locker, locked[i], false) {
|
||||
case 1:
|
||||
return logicerr.Wrap(apistatus.LockNonRegularObject{})
|
||||
case 0:
|
||||
|
@ -48,7 +59,7 @@ func (e *StorageEngine) lock(idCnr cid.ID, locker oid.ID, locked []oid.ID) error
|
|||
// - 0: fail
|
||||
// - 1: locking irregular object
|
||||
// - 2: ok
|
||||
func (e *StorageEngine) lockSingle(idCnr cid.ID, locker, locked oid.ID, checkExists bool) (status uint8) {
|
||||
func (e *StorageEngine) lockSingle(ctx context.Context, idCnr cid.ID, locker, locked oid.ID, checkExists bool) (status uint8) {
|
||||
// code is pretty similar to inhumeAddr, maybe unify?
|
||||
root := false
|
||||
var errIrregular apistatus.LockNonRegularObject
|
||||
|
@ -70,7 +81,7 @@ func (e *StorageEngine) lockSingle(idCnr cid.ID, locker, locked oid.ID, checkExi
|
|||
var existsPrm shard.ExistsPrm
|
||||
existsPrm.SetAddress(addrLocked)
|
||||
|
||||
exRes, err := sh.Exists(context.TODO(), existsPrm)
|
||||
exRes, err := sh.Exists(ctx, existsPrm)
|
||||
if err != nil {
|
||||
var siErr *objectSDK.SplitInfoError
|
||||
if !errors.As(err, &siErr) {
|
||||
|
@ -90,7 +101,7 @@ func (e *StorageEngine) lockSingle(idCnr cid.ID, locker, locked oid.ID, checkExi
|
|||
}
|
||||
}
|
||||
|
||||
err := sh.Lock(idCnr, locker, []oid.ID{locked})
|
||||
err := sh.Lock(ctx, idCnr, locker, []oid.ID{locked})
|
||||
if err != nil {
|
||||
e.reportShardError(sh, "could not lock object in shard", err)
|
||||
|
||||
|
|
|
@ -99,7 +99,7 @@ func TestLockUserScenario(t *testing.T) {
|
|||
id, _ := obj.ID()
|
||||
objAddr.SetObject(id)
|
||||
|
||||
err = Put(e, obj)
|
||||
err = Put(context.Background(), e, obj)
|
||||
require.NoError(t, err)
|
||||
|
||||
// 2.
|
||||
|
@ -107,10 +107,10 @@ func TestLockUserScenario(t *testing.T) {
|
|||
locker.WriteMembers([]oid.ID{id})
|
||||
object.WriteLock(lockerObj, locker)
|
||||
|
||||
err = Put(e, lockerObj)
|
||||
err = Put(context.Background(), e, lockerObj)
|
||||
require.NoError(t, err)
|
||||
|
||||
err = e.Lock(cnr, lockerID, []oid.ID{id})
|
||||
err = e.Lock(context.Background(), cnr, lockerID, []oid.ID{id})
|
||||
require.NoError(t, err)
|
||||
|
||||
// 3.
|
||||
|
@ -125,7 +125,7 @@ func TestLockUserScenario(t *testing.T) {
|
|||
tombObj.SetID(tombForLockID)
|
||||
tombObj.SetAttributes(a)
|
||||
|
||||
err = Put(e, tombObj)
|
||||
err = Put(context.Background(), e, tombObj)
|
||||
require.NoError(t, err)
|
||||
|
||||
inhumePrm.WithTarget(tombForLockAddr, lockerAddr)
|
||||
|
@ -180,7 +180,7 @@ func TestLockExpiration(t *testing.T) {
|
|||
// 1.
|
||||
obj := testutil.GenerateObjectWithCID(cnr)
|
||||
|
||||
err = Put(e, obj)
|
||||
err = Put(context.Background(), e, obj)
|
||||
require.NoError(t, err)
|
||||
|
||||
// 2.
|
||||
|
@ -192,13 +192,13 @@ func TestLockExpiration(t *testing.T) {
|
|||
lock.SetType(object.TypeLock)
|
||||
lock.SetAttributes(a)
|
||||
|
||||
err = Put(e, lock)
|
||||
err = Put(context.Background(), e, lock)
|
||||
require.NoError(t, err)
|
||||
|
||||
id, _ := obj.ID()
|
||||
idLock, _ := lock.ID()
|
||||
|
||||
err = e.Lock(cnr, idLock, []oid.ID{id})
|
||||
err = e.Lock(context.Background(), cnr, idLock, []oid.ID{id})
|
||||
require.NoError(t, err)
|
||||
|
||||
var inhumePrm InhumePrm
|
||||
|
@ -255,20 +255,20 @@ func TestLockForceRemoval(t *testing.T) {
|
|||
// 1.
|
||||
obj := testutil.GenerateObjectWithCID(cnr)
|
||||
|
||||
err = Put(e, obj)
|
||||
err = Put(context.Background(), e, obj)
|
||||
require.NoError(t, err)
|
||||
|
||||
// 2.
|
||||
lock := testutil.GenerateObjectWithCID(cnr)
|
||||
lock.SetType(object.TypeLock)
|
||||
|
||||
err = Put(e, lock)
|
||||
err = Put(context.Background(), e, lock)
|
||||
require.NoError(t, err)
|
||||
|
||||
id, _ := obj.ID()
|
||||
idLock, _ := lock.ID()
|
||||
|
||||
err = e.Lock(cnr, idLock, []oid.ID{id})
|
||||
err = e.Lock(context.Background(), cnr, idLock, []oid.ID{id})
|
||||
require.NoError(t, err)
|
||||
|
||||
// 3.
|
||||
|
|
|
@ -4,6 +4,7 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/pkg/tracing"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor"
|
||||
|
@ -12,6 +13,8 @@ import (
|
|||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util"
|
||||
objectSDK "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
||||
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
@ -20,9 +23,6 @@ type PutPrm struct {
|
|||
obj *objectSDK.Object
|
||||
}
|
||||
|
||||
// PutRes groups the resulting values of Put operation.
|
||||
type PutRes struct{}
|
||||
|
||||
var errPutShard = errors.New("could not put object to any shard")
|
||||
|
||||
// WithObject is a Put option to set object to save.
|
||||
|
@ -40,16 +40,22 @@ func (p *PutPrm) WithObject(obj *objectSDK.Object) {
|
|||
// Returns an error if executions are blocked (see BlockExecution).
|
||||
//
|
||||
// Returns an error of type apistatus.ObjectAlreadyRemoved if the object has been marked as removed.
|
||||
func (e *StorageEngine) Put(prm PutPrm) (res PutRes, err error) {
|
||||
func (e *StorageEngine) Put(ctx context.Context, prm PutPrm) (err error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Put",
|
||||
trace.WithAttributes(
|
||||
attribute.String("address", object.AddressOf(prm.obj).EncodeToString()),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
err = e.execIfNotBlocked(func() error {
|
||||
res, err = e.put(prm)
|
||||
err = e.put(ctx, prm)
|
||||
return err
|
||||
})
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (e *StorageEngine) put(prm PutPrm) (PutRes, error) {
|
||||
func (e *StorageEngine) put(ctx context.Context, prm PutPrm) error {
|
||||
if e.metrics != nil {
|
||||
defer elapsed(e.metrics.AddPutDuration)()
|
||||
}
|
||||
|
@ -58,9 +64,9 @@ func (e *StorageEngine) put(prm PutPrm) (PutRes, error) {
|
|||
|
||||
// In #1146 this check was parallelized, however, it became
|
||||
// much slower on fast machines for 4 shards.
|
||||
_, err := e.exists(addr)
|
||||
_, err := e.exists(ctx, addr)
|
||||
if err != nil {
|
||||
return PutRes{}, err
|
||||
return err
|
||||
}
|
||||
|
||||
finished := false
|
||||
|
@ -74,7 +80,7 @@ func (e *StorageEngine) put(prm PutPrm) (PutRes, error) {
|
|||
return false
|
||||
}
|
||||
|
||||
putDone, exists := e.putToShard(context.TODO(), sh, ind, pool, addr, prm.obj)
|
||||
putDone, exists := e.putToShard(ctx, sh, ind, pool, addr, prm.obj)
|
||||
finished = putDone || exists
|
||||
return finished
|
||||
})
|
||||
|
@ -83,7 +89,7 @@ func (e *StorageEngine) put(prm PutPrm) (PutRes, error) {
|
|||
err = errPutShard
|
||||
}
|
||||
|
||||
return PutRes{}, err
|
||||
return err
|
||||
}
|
||||
|
||||
// putToShard puts object to sh.
|
||||
|
@ -117,7 +123,7 @@ func (e *StorageEngine) putToShard(ctx context.Context, sh hashedShard, ind int,
|
|||
var toMoveItPrm shard.ToMoveItPrm
|
||||
toMoveItPrm.SetAddress(addr)
|
||||
|
||||
_, err = sh.ToMoveIt(toMoveItPrm)
|
||||
_, err = sh.ToMoveIt(ctx, toMoveItPrm)
|
||||
if err != nil {
|
||||
e.log.Warn(logs.EngineCouldNotMarkObjectForShardRelocation,
|
||||
zap.Stringer("shard", sh.ID()),
|
||||
|
@ -132,7 +138,7 @@ func (e *StorageEngine) putToShard(ctx context.Context, sh hashedShard, ind int,
|
|||
var putPrm shard.PutPrm
|
||||
putPrm.SetObject(obj)
|
||||
|
||||
_, err = sh.Put(putPrm)
|
||||
_, err = sh.Put(ctx, putPrm)
|
||||
if err != nil {
|
||||
if errors.Is(err, shard.ErrReadOnlyMode) || errors.Is(err, blobstor.ErrNoPlaceFound) ||
|
||||
errors.Is(err, common.ErrReadOnly) || errors.Is(err, common.ErrNoSpace) {
|
||||
|
@ -157,11 +163,9 @@ func (e *StorageEngine) putToShard(ctx context.Context, sh hashedShard, ind int,
|
|||
}
|
||||
|
||||
// Put writes provided object to local storage.
|
||||
func Put(storage *StorageEngine, obj *objectSDK.Object) error {
|
||||
func Put(ctx context.Context, storage *StorageEngine, obj *objectSDK.Object) error {
|
||||
var putPrm PutPrm
|
||||
putPrm.WithObject(obj)
|
||||
|
||||
_, err := storage.Put(putPrm)
|
||||
|
||||
return err
|
||||
return storage.Put(ctx, putPrm)
|
||||
}
|
||||
|
|
|
@ -129,7 +129,7 @@ func (e *StorageEngine) removeObjects(ctx context.Context, ch <-chan oid.Address
|
|||
|
||||
var deletePrm shard.DeletePrm
|
||||
deletePrm.SetAddresses(addr)
|
||||
_, err = shards[i].Delete(deletePrm)
|
||||
_, err = shards[i].Delete(ctx, deletePrm)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -49,10 +49,10 @@ func TestRebalance(t *testing.T) {
|
|||
te.ng.mtx.RLock()
|
||||
// Every 3rd object (i%3 == 0) is put to both shards, others are distributed.
|
||||
if i%3 != 1 {
|
||||
_, err1 = te.ng.shards[te.shards[0].id.String()].Shard.Put(prm)
|
||||
_, err1 = te.ng.shards[te.shards[0].id.String()].Shard.Put(context.Background(), prm)
|
||||
}
|
||||
if i%3 != 2 {
|
||||
_, err2 = te.ng.shards[te.shards[1].id.String()].Shard.Put(prm)
|
||||
_, err2 = te.ng.shards[te.shards[1].id.String()].Shard.Put(context.Background(), prm)
|
||||
}
|
||||
te.ng.mtx.RUnlock()
|
||||
|
||||
|
@ -109,8 +109,8 @@ func TestRebalanceSingleThread(t *testing.T) {
|
|||
var prm shard.PutPrm
|
||||
prm.SetObject(obj)
|
||||
te.ng.mtx.RLock()
|
||||
_, err1 := te.ng.shards[te.shards[0].id.String()].Shard.Put(prm)
|
||||
_, err2 := te.ng.shards[te.shards[1].id.String()].Shard.Put(prm)
|
||||
_, err1 := te.ng.shards[te.shards[0].id.String()].Shard.Put(context.Background(), prm)
|
||||
_, err2 := te.ng.shards[te.shards[1].id.String()].Shard.Put(context.Background(), prm)
|
||||
te.ng.mtx.RUnlock()
|
||||
require.NoError(t, err1)
|
||||
require.NoError(t, err2)
|
||||
|
@ -162,8 +162,8 @@ func TestRebalanceExitByContext(t *testing.T) {
|
|||
prm.SetObject(objects[i])
|
||||
|
||||
te.ng.mtx.RLock()
|
||||
_, err1 := te.ng.shards[te.shards[0].id.String()].Shard.Put(prm)
|
||||
_, err2 := te.ng.shards[te.shards[1].id.String()].Shard.Put(prm)
|
||||
_, err1 := te.ng.shards[te.shards[0].id.String()].Shard.Put(context.Background(), prm)
|
||||
_, err2 := te.ng.shards[te.shards[1].id.String()].Shard.Put(context.Background(), prm)
|
||||
te.ng.mtx.RUnlock()
|
||||
|
||||
require.NoError(t, err1)
|
||||
|
|
|
@ -1,11 +1,24 @@
|
|||
package engine
|
||||
|
||||
import "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
||||
import (
|
||||
"context"
|
||||
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/pkg/tracing"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
)
|
||||
|
||||
// RestoreShard restores objects from dump to the shard with provided identifier.
|
||||
//
|
||||
// Returns an error if shard is not read-only.
|
||||
func (e *StorageEngine) RestoreShard(id *shard.ID, prm shard.RestorePrm) error {
|
||||
func (e *StorageEngine) RestoreShard(ctx context.Context, id *shard.ID, prm shard.RestorePrm) error {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.RestoreShard",
|
||||
trace.WithAttributes(
|
||||
attribute.String("shard_id", id.String()),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
e.mtx.RLock()
|
||||
defer e.mtx.RUnlock()
|
||||
|
||||
|
@ -14,6 +27,6 @@ func (e *StorageEngine) RestoreShard(id *shard.ID, prm shard.RestorePrm) error {
|
|||
return errShardNotFound
|
||||
}
|
||||
|
||||
_, err := sh.Restore(prm)
|
||||
_, err := sh.Restore(ctx, prm)
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -1,10 +1,15 @@
|
|||
package engine
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/pkg/tracing"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
||||
cid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/container/id"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object"
|
||||
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
)
|
||||
|
||||
// SelectPrm groups the parameters of Select operation.
|
||||
|
@ -38,16 +43,22 @@ func (r SelectRes) AddressList() []oid.Address {
|
|||
// Returns any error encountered that did not allow to completely select the objects.
|
||||
//
|
||||
// Returns an error if executions are blocked (see BlockExecution).
|
||||
func (e *StorageEngine) Select(prm SelectPrm) (res SelectRes, err error) {
|
||||
func (e *StorageEngine) Select(ctx context.Context, prm SelectPrm) (res SelectRes, err error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Select",
|
||||
trace.WithAttributes(
|
||||
attribute.String("container_id", prm.cnr.EncodeToString()),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
err = e.execIfNotBlocked(func() error {
|
||||
res, err = e._select(prm)
|
||||
res, err = e._select(ctx, prm)
|
||||
return err
|
||||
})
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (e *StorageEngine) _select(prm SelectPrm) (SelectRes, error) {
|
||||
func (e *StorageEngine) _select(ctx context.Context, prm SelectPrm) (SelectRes, error) {
|
||||
if e.metrics != nil {
|
||||
defer elapsed(e.metrics.AddSearchDuration)()
|
||||
}
|
||||
|
@ -62,7 +73,7 @@ func (e *StorageEngine) _select(prm SelectPrm) (SelectRes, error) {
|
|||
shPrm.SetFilters(prm.filters)
|
||||
|
||||
e.iterateOverUnsortedShards(func(sh hashedShard) (stop bool) {
|
||||
res, err := sh.Select(shPrm)
|
||||
res, err := sh.Select(ctx, shPrm)
|
||||
if err != nil {
|
||||
e.reportShardError(sh, "could not select objects from shard", err)
|
||||
return false
|
||||
|
@ -133,12 +144,12 @@ func (e *StorageEngine) list(limit uint64) (SelectRes, error) {
|
|||
}
|
||||
|
||||
// Select selects objects from local storage using provided filters.
|
||||
func Select(storage *StorageEngine, cnr cid.ID, fs object.SearchFilters) ([]oid.Address, error) {
|
||||
func Select(ctx context.Context, storage *StorageEngine, cnr cid.ID, fs object.SearchFilters) ([]oid.Address, error) {
|
||||
var selectPrm SelectPrm
|
||||
selectPrm.WithContainerID(cnr)
|
||||
selectPrm.WithFilters(fs)
|
||||
|
||||
res, err := storage.Select(selectPrm)
|
||||
res, err := storage.Select(ctx, selectPrm)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package engine
|
||||
|
||||
import (
|
||||
"context"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
|
@ -31,7 +32,7 @@ func benchmarkTreeVsSearch(b *testing.B, objCount int) {
|
|||
for i := 0; i < objCount; i++ {
|
||||
obj := testutil.GenerateObjectWithCID(cid)
|
||||
testutil.AddAttribute(obj, pilorama.AttributeFilename, strconv.Itoa(i))
|
||||
err := Put(te.ng, obj)
|
||||
err := Put(context.Background(), te.ng, obj)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
|
@ -51,7 +52,7 @@ func benchmarkTreeVsSearch(b *testing.B, objCount int) {
|
|||
prm.WithFilters(fs)
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
res, err := te.ng.Select(prm)
|
||||
res, err := te.ng.Select(context.Background(), prm)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
|
|
|
@ -1,7 +1,12 @@
|
|||
package engine
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/pkg/tracing"
|
||||
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
||||
"go.opentelemetry.io/otel/attribute"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
)
|
||||
|
||||
// FlushWriteCachePrm groups the parameters of FlushWriteCache operation.
|
||||
|
@ -26,7 +31,14 @@ func (p *FlushWriteCachePrm) SetIgnoreErrors(ignore bool) {
|
|||
type FlushWriteCacheRes struct{}
|
||||
|
||||
// FlushWriteCache flushes write-cache on a single shard.
|
||||
func (e *StorageEngine) FlushWriteCache(p FlushWriteCachePrm) (FlushWriteCacheRes, error) {
|
||||
func (e *StorageEngine) FlushWriteCache(ctx context.Context, p FlushWriteCachePrm) (FlushWriteCacheRes, error) {
|
||||
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.FlushWriteCache",
|
||||
trace.WithAttributes(
|
||||
attribute.String("shard)id", p.shardID.String()),
|
||||
attribute.Bool("ignore_errors", p.ignoreErrors),
|
||||
))
|
||||
defer span.End()
|
||||
|
||||
e.mtx.RLock()
|
||||
sh, ok := e.shards[p.shardID.String()]
|
||||
e.mtx.RUnlock()
|
||||
|
@ -38,5 +50,5 @@ func (e *StorageEngine) FlushWriteCache(p FlushWriteCachePrm) (FlushWriteCacheRe
|
|||
var prm shard.FlushWriteCachePrm
|
||||
prm.SetIgnoreErrors(p.ignoreErrors)
|
||||
|
||||
return FlushWriteCacheRes{}, sh.FlushWriteCache(prm)
|
||||
return FlushWriteCacheRes{}, sh.FlushWriteCache(ctx, prm)
|
||||
}
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue