[#947] engine: Evacuate trees to local shards
Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
This commit is contained in:
parent
e4064c4394
commit
728150d1d2
6 changed files with 463 additions and 35 deletions
|
@ -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
|
||||
}
|
||||
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue