[#947] engine: Evacuate trees to local shards

Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
This commit is contained in:
Dmitrii Stepanov 2024-02-06 13:59:50 +03:00
parent e4064c4394
commit 728150d1d2
6 changed files with 463 additions and 35 deletions

View file

@ -5,11 +5,13 @@ import (
"errors"
"fmt"
"strings"
"sync"
"sync/atomic"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/core/object"
meta "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/metabase"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/pilorama"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
tracingPkg "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/tracing"
@ -56,6 +58,18 @@ func (s EvacuateScope) String() string {
return sb.String()
}
func (s EvacuateScope) WithObjects() bool {
return s&EvacuateScopeObjects == EvacuateScopeObjects
}
func (s EvacuateScope) WithTrees() bool {
return s&EvacuateScopeTrees == EvacuateScopeTrees
}
func (s EvacuateScope) TreesOnly() bool {
return s == EvacuateScopeTrees
}
// EvacuateShardPrm represents parameters for the EvacuateShard operation.
type EvacuateShardPrm struct {
ShardID []*shard.ID
@ -264,7 +278,7 @@ func (e *StorageEngine) evacuateShards(ctx context.Context, shardIDs []string, p
e.log.Info(logs.EngineStartedShardsEvacuation, zap.Strings("shard_ids", shardIDs), evacuationOperationLogField,
zap.String("trace_id", tracingPkg.GetTraceID(ctx)), zap.Stringer("scope", prm.Scope))
err = e.getTotalObjectsCount(ctx, shardsToEvacuate, res)
err = e.getTotals(ctx, prm, shardsToEvacuate, res)
if err != nil {
e.log.Error(logs.EngineShardsEvacuationFailedToCount, zap.Strings("shard_ids", shardIDs), zap.Error(err), evacuationOperationLogField,
zap.String("trace_id", tracingPkg.GetTraceID(ctx)), zap.Stringer("scope", prm.Scope))
@ -293,19 +307,28 @@ func (e *StorageEngine) evacuateShards(ctx context.Context, shardIDs []string, p
return nil
}
func (e *StorageEngine) getTotalObjectsCount(ctx context.Context, shardsToEvacuate map[string]*shard.Shard, res *EvacuateShardRes) error {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.getTotalObjectsCount")
func (e *StorageEngine) getTotals(ctx context.Context, prm EvacuateShardPrm, shardsToEvacuate map[string]*shard.Shard, res *EvacuateShardRes) error {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.getTotals")
defer span.End()
for _, sh := range shardsToEvacuate {
cnt, err := sh.LogicalObjectsCount(ctx)
if err != nil {
if errors.Is(err, shard.ErrDegradedMode) {
continue
if prm.Scope.WithObjects() {
cnt, err := sh.LogicalObjectsCount(ctx)
if err != nil {
if errors.Is(err, shard.ErrDegradedMode) {
continue
}
return err
}
return err
res.objTotal.Add(cnt)
}
if prm.Scope.WithTrees() && sh.PiloramaEnabled() {
cnt, err := pilorama.TreeCountAll(ctx, sh)
if err != nil {
return err
}
res.trTotal.Add(cnt)
}
res.objTotal.Add(cnt)
}
return nil
}
@ -319,6 +342,24 @@ func (e *StorageEngine) evacuateShard(ctx context.Context, shardID string, prm E
))
defer span.End()
if prm.Scope.WithObjects() {
if err := e.evacuateShardObjects(ctx, shardID, prm, res, shards, weights, shardsToEvacuate); err != nil {
return err
}
}
if prm.Scope.WithTrees() && shardsToEvacuate[shardID].PiloramaEnabled() {
if err := e.evacuateShardTrees(ctx, shardID, prm, res, shards, weights, shardsToEvacuate); err != nil {
return err
}
}
return nil
}
func (e *StorageEngine) evacuateShardObjects(ctx context.Context, shardID string, prm EvacuateShardPrm, res *EvacuateShardRes,
shards []pooledShard, weights []float64, shardsToEvacuate map[string]*shard.Shard,
) error {
var listPrm shard.ListWithCursorPrm
listPrm.WithCount(defaultEvacuateBatchSize)
@ -349,6 +390,172 @@ func (e *StorageEngine) evacuateShard(ctx context.Context, shardID string, prm E
return nil
}
func (e *StorageEngine) evacuateShardTrees(ctx context.Context, shardID string, prm EvacuateShardPrm, res *EvacuateShardRes,
shards []pooledShard, weights []float64, shardsToEvacuate map[string]*shard.Shard,
) error {
sh := shardsToEvacuate[shardID]
var listPrm pilorama.TreeListTreesPrm
first := true
for len(listPrm.NextPageToken) > 0 || first {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
first = false
listRes, err := sh.TreeListTrees(ctx, listPrm)
if err != nil {
return err
}
listPrm.NextPageToken = listRes.NextPageToken
if err := e.evacuateTrees(ctx, sh, listRes.Items, prm, res, shards, weights, shardsToEvacuate); err != nil {
return err
}
}
return nil
}
func (e *StorageEngine) evacuateTrees(ctx context.Context, sh *shard.Shard, trees []pilorama.ContainerIDTreeID,
prm EvacuateShardPrm, res *EvacuateShardRes, shards []pooledShard, weights []float64,
shardsToEvacuate map[string]*shard.Shard,
) error {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.evacuateTrees",
trace.WithAttributes(
attribute.Int("trees_count", len(trees)),
))
defer span.End()
for _, contTree := range trees {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
success, _, err := e.tryEvacuateTreeLocal(ctx, sh, contTree, prm, shards, weights, shardsToEvacuate)
if err != nil {
return err
}
if success {
res.trEvacuated.Add(1)
} else {
res.trFailed.Add(1)
}
}
return nil
}
func (e *StorageEngine) tryEvacuateTreeLocal(ctx context.Context, sh *shard.Shard, tree pilorama.ContainerIDTreeID,
prm EvacuateShardPrm, shards []pooledShard, weights []float64, shardsToEvacuate map[string]*shard.Shard,
) (bool, string, error) {
target, found, err := e.findShardToEvacuateTree(ctx, tree, shards, weights, shardsToEvacuate)
if err != nil {
return false, "", err
}
if !found {
return false, "", nil
}
const readBatchSize = 1000
source := make(chan *pilorama.Move, readBatchSize)
ctx, cancel := context.WithCancel(ctx)
defer cancel()
var wg sync.WaitGroup
wg.Add(1)
var applyErr error
go func() {
defer wg.Done()
applyErr = target.TreeApplyStream(ctx, tree.CID, tree.TreeID, source)
if applyErr != nil {
cancel()
}
}()
var height uint64
for {
op, err := sh.TreeGetOpLog(ctx, tree.CID, tree.TreeID, height)
if err != nil {
cancel()
wg.Wait()
close(source) // close after cancel to ctx.Done() hits first
if prm.IgnoreErrors {
return false, "", nil
}
return false, "", err
}
if op.Time == 0 { // completed get op log
close(source)
wg.Wait()
if applyErr == nil {
return true, target.ID().String(), nil
}
if prm.IgnoreErrors {
return false, "", nil
}
return false, "", applyErr
}
select {
case <-ctx.Done(): // apply stream failed or operation cancelled
wg.Wait()
if prm.IgnoreErrors {
return false, "", nil
}
if applyErr != nil {
return false, "", applyErr
}
return false, "", ctx.Err()
case source <- &op:
}
height = op.Time + 1
}
}
// findShardToEvacuateTree returns first shard according HRW or first shard with tree exists.
func (e *StorageEngine) findShardToEvacuateTree(ctx context.Context, tree pilorama.ContainerIDTreeID,
shards []pooledShard, weights []float64, shardsToEvacuate map[string]*shard.Shard,
) (pooledShard, bool, error) {
hrw.SortHasherSliceByWeightValue(shards, weights, hrw.StringHash(tree.CID.EncodeToString()))
var result pooledShard
var found bool
for _, target := range shards {
select {
case <-ctx.Done():
return pooledShard{}, false, ctx.Err()
default:
}
if _, ok := shardsToEvacuate[target.ID().String()]; ok {
continue
}
if !target.PiloramaEnabled() || target.GetMode().ReadOnly() {
continue
}
if !found {
result = target
found = true
}
exists, err := target.TreeExists(ctx, tree.CID, tree.TreeID)
if err != nil {
continue
}
if exists {
return target, true, nil
}
}
return result, found, nil
}
func (e *StorageEngine) getActualShards(shardIDs []string, prm EvacuateShardPrm) ([]pooledShard, []float64, error) {
e.mtx.RLock()
defer e.mtx.RUnlock()
@ -362,9 +569,13 @@ func (e *StorageEngine) getActualShards(shardIDs []string, prm EvacuateShardPrm)
if !sh.GetMode().ReadOnly() {
return nil, nil, ErrMustBeReadOnly
}
if prm.Scope.TreesOnly() && !sh.PiloramaEnabled() {
return nil, nil, fmt.Errorf("shard %s doesn't have pilorama enabled", sh.ID())
}
}
if len(e.shards)-len(shardIDs) < 1 && prm.ObjectsHandler == nil {
if len(e.shards)-len(shardIDs) < 1 && prm.ObjectsHandler == nil && prm.Scope.WithObjects() {
return nil, nil, errMustHaveTwoShards
}