Aleksey Savchuk
f0c43c8d80
All checks were successful
Vulncheck / Vulncheck (pull_request) Successful in 3m1s
Pre-commit hooks / Pre-commit (pull_request) Successful in 3m29s
Tests and linters / gopls check (pull_request) Successful in 3m50s
Tests and linters / Lint (pull_request) Successful in 4m35s
DCO action / DCO (pull_request) Successful in 5m12s
Tests and linters / Run gofumpt (pull_request) Successful in 5m33s
Build / Build Components (pull_request) Successful in 5m45s
Tests and linters / Tests with -race (pull_request) Successful in 6m37s
Tests and linters / Tests (pull_request) Successful in 7m17s
Tests and linters / Staticcheck (pull_request) Successful in 7m36s
Tests and linters / Run gofumpt (push) Successful in 1m22s
Tests and linters / Staticcheck (push) Successful in 3m19s
Tests and linters / Lint (push) Successful in 4m35s
Vulncheck / Vulncheck (push) Successful in 5m20s
Build / Build Components (push) Successful in 6m16s
Pre-commit hooks / Pre-commit (push) Successful in 6m37s
Tests and linters / Tests (push) Successful in 6m48s
Tests and linters / Tests with -race (push) Successful in 7m15s
Tests and linters / gopls check (push) Successful in 7m27s
Use `zap.Error` instead of `zap.String` for logging errors: change all expressions like `zap.String("error", err.Error())` or `zap.String("err", err.Error())` to `zap.Error(err)`. Leave similar expressions with other messages unchanged, for example, `zap.String("last_error", lastErr.Error())` or `zap.String("reason", ctx.Err().Error())`. This change was made by applying the following patch: ```diff @@ var err expression @@ -zap.String("error", err.Error()) +zap.Error(err) @@ var err expression @@ -zap.String("err", err.Error()) +zap.Error(err) ``` Signed-off-by: Aleksey Savchuk <a.savchuk@yadro.com>
200 lines
5.6 KiB
Go
200 lines
5.6 KiB
Go
package engine
|
|
|
|
import (
|
|
"context"
|
|
"errors"
|
|
|
|
"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"
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/blobstor/common"
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
|
tracingPkg "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/tracing"
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util"
|
|
"git.frostfs.info/TrueCloudLab/frostfs-observability/tracing"
|
|
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/client"
|
|
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"
|
|
)
|
|
|
|
// PutPrm groups the parameters of Put operation.
|
|
type PutPrm struct {
|
|
Object *objectSDK.Object
|
|
IsIndexedContainer bool
|
|
}
|
|
|
|
var errPutShard = errors.New("could not put object to any shard")
|
|
|
|
type putToShardStatus byte
|
|
|
|
const (
|
|
putToShardUnknown putToShardStatus = iota
|
|
putToShardSuccess
|
|
putToShardExists
|
|
putToShardRemoved
|
|
)
|
|
|
|
type putToShardRes struct {
|
|
status putToShardStatus
|
|
err error
|
|
}
|
|
|
|
// Put saves the object to local storage.
|
|
//
|
|
// Returns any error encountered that
|
|
// did not allow to completely save the 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(ctx context.Context, prm PutPrm) (err error) {
|
|
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Put",
|
|
trace.WithAttributes(
|
|
attribute.String("address", object.AddressOf(prm.Object).EncodeToString()),
|
|
))
|
|
defer span.End()
|
|
defer elapsed("Put", e.metrics.AddMethodDuration)()
|
|
|
|
err = e.execIfNotBlocked(func() error {
|
|
err = e.put(ctx, prm)
|
|
return err
|
|
})
|
|
|
|
return
|
|
}
|
|
|
|
func (e *StorageEngine) put(ctx context.Context, prm PutPrm) error {
|
|
addr := object.AddressOf(prm.Object)
|
|
|
|
// In #1146 this check was parallelized, however, it became
|
|
// much slower on fast machines for 4 shards.
|
|
var ecParent oid.Address
|
|
if prm.Object.ECHeader() != nil {
|
|
ecParent.SetObject(prm.Object.ECHeader().Parent())
|
|
ecParent.SetContainer(addr.Container())
|
|
}
|
|
var shPrm shard.ExistsPrm
|
|
shPrm.Address = addr
|
|
shPrm.ECParentAddress = ecParent
|
|
existed, locked, err := e.exists(ctx, shPrm)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if !existed && locked {
|
|
lockers, err := e.GetLocks(ctx, ecParent)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
for _, locker := range lockers {
|
|
err = e.lock(ctx, addr.Container(), locker, []oid.ID{addr.Object()})
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
}
|
|
|
|
var shRes putToShardRes
|
|
e.iterateOverSortedShards(addr, func(_ int, sh hashedShard) (stop bool) {
|
|
e.mtx.RLock()
|
|
pool, ok := e.shardPools[sh.ID().String()]
|
|
e.mtx.RUnlock()
|
|
if !ok {
|
|
// Shard was concurrently removed, skip.
|
|
return false
|
|
}
|
|
shRes = e.putToShard(ctx, sh, pool, addr, prm.Object, prm.IsIndexedContainer)
|
|
return shRes.status != putToShardUnknown
|
|
})
|
|
switch shRes.status {
|
|
case putToShardUnknown:
|
|
return errPutShard
|
|
case putToShardRemoved:
|
|
return shRes.err
|
|
case putToShardExists, putToShardSuccess:
|
|
return nil
|
|
default:
|
|
return errPutShard
|
|
}
|
|
}
|
|
|
|
// putToShard puts object to sh.
|
|
// Return putToShardStatus and error if it is necessary to propagate an error upper.
|
|
func (e *StorageEngine) putToShard(ctx context.Context, sh hashedShard, pool util.WorkerPool,
|
|
addr oid.Address, obj *objectSDK.Object, isIndexedContainer bool,
|
|
) (res putToShardRes) {
|
|
exitCh := make(chan struct{})
|
|
|
|
if err := pool.Submit(func() {
|
|
defer close(exitCh)
|
|
|
|
var existPrm shard.ExistsPrm
|
|
existPrm.Address = addr
|
|
|
|
exists, err := sh.Exists(ctx, existPrm)
|
|
if err != nil {
|
|
if shard.IsErrObjectExpired(err) {
|
|
// object is already found but
|
|
// expired => do nothing with it
|
|
res.status = putToShardExists
|
|
} else {
|
|
e.log.Warn(ctx, logs.EngineCouldNotCheckObjectExistence,
|
|
zap.Stringer("shard_id", sh.ID()),
|
|
zap.Error(err),
|
|
zap.String("trace_id", tracingPkg.GetTraceID(ctx)))
|
|
}
|
|
|
|
return // this is not ErrAlreadyRemoved error so we can go to the next shard
|
|
}
|
|
|
|
if exists.Exists() {
|
|
res.status = putToShardExists
|
|
return
|
|
}
|
|
|
|
var putPrm shard.PutPrm
|
|
putPrm.SetObject(obj)
|
|
putPrm.SetIndexAttributes(isIndexedContainer)
|
|
|
|
_, 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) {
|
|
e.log.Warn(ctx, logs.EngineCouldNotPutObjectToShard,
|
|
zap.Stringer("shard_id", sh.ID()),
|
|
zap.Error(err),
|
|
zap.String("trace_id", tracingPkg.GetTraceID(ctx)))
|
|
return
|
|
}
|
|
if client.IsErrObjectAlreadyRemoved(err) {
|
|
e.log.Warn(ctx, logs.EngineCouldNotPutObjectToShard,
|
|
zap.Stringer("shard_id", sh.ID()),
|
|
zap.Error(err),
|
|
zap.String("trace_id", tracingPkg.GetTraceID(ctx)))
|
|
res.status = putToShardRemoved
|
|
res.err = err
|
|
return
|
|
}
|
|
|
|
e.reportShardError(ctx, sh, "could not put object to shard", err, zap.Stringer("address", addr))
|
|
return
|
|
}
|
|
|
|
res.status = putToShardSuccess
|
|
}); err != nil {
|
|
e.log.Warn(ctx, logs.EngineCouldNotPutObjectToShard, zap.Error(err))
|
|
close(exitCh)
|
|
}
|
|
|
|
<-exitCh
|
|
|
|
return
|
|
}
|
|
|
|
// Put writes provided object to local storage.
|
|
func Put(ctx context.Context, storage *StorageEngine, obj *objectSDK.Object, indexedContainer bool) error {
|
|
return storage.Put(ctx, PutPrm{Object: obj, IsIndexedContainer: indexedContainer})
|
|
}
|