2020-11-17 12:26:03 +00:00
|
|
|
package engine
|
|
|
|
|
|
|
|
import (
|
2023-08-31 16:26:47 +00:00
|
|
|
"context"
|
2020-11-17 12:26:03 +00:00
|
|
|
"fmt"
|
2023-05-19 15:06:20 +00:00
|
|
|
"sync/atomic"
|
2020-11-17 12:26:03 +00:00
|
|
|
|
2023-04-12 14:35:10 +00:00
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
|
2023-03-07 13:38:26 +00:00
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
|
|
|
|
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/util/logicerr"
|
|
|
|
oid "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/object/id"
|
|
|
|
"git.frostfs.info/TrueCloudLab/hrw"
|
2020-11-17 12:26:03 +00:00
|
|
|
"github.com/google/uuid"
|
2021-10-08 12:25:45 +00:00
|
|
|
"github.com/panjf2000/ants/v2"
|
2022-09-26 21:39:34 +00:00
|
|
|
"go.uber.org/zap"
|
2020-11-17 12:26:03 +00:00
|
|
|
)
|
|
|
|
|
2022-10-31 07:02:30 +00:00
|
|
|
var errShardNotFound = logicerr.New("shard not found")
|
2020-11-19 13:04:04 +00:00
|
|
|
|
2023-02-27 13:16:37 +00:00
|
|
|
type hashedShard struct {
|
|
|
|
shardWrapper
|
|
|
|
hash uint64
|
|
|
|
}
|
2020-11-30 14:58:44 +00:00
|
|
|
|
2022-08-19 16:49:09 +00:00
|
|
|
type metricsWithID struct {
|
|
|
|
id string
|
|
|
|
mw MetricRegister
|
|
|
|
}
|
|
|
|
|
2022-10-12 17:55:35 +00:00
|
|
|
func (m *metricsWithID) SetShardID(id string) {
|
|
|
|
// concurrent settings are not expected =>
|
|
|
|
// no mutex protection
|
|
|
|
m.id = id
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *metricsWithID) SetObjectCounter(objectType string, v uint64) {
|
2022-09-09 11:37:35 +00:00
|
|
|
m.mw.SetObjectCounter(m.id, objectType, v)
|
2022-08-19 17:24:05 +00:00
|
|
|
}
|
|
|
|
|
2022-10-12 17:55:35 +00:00
|
|
|
func (m *metricsWithID) AddToObjectCounter(objectType string, delta int) {
|
2022-09-09 11:37:35 +00:00
|
|
|
m.mw.AddToObjectCounter(m.id, objectType, delta)
|
2022-08-19 16:49:09 +00:00
|
|
|
}
|
|
|
|
|
2022-10-12 17:55:35 +00:00
|
|
|
func (m *metricsWithID) IncObjectCounter(objectType string) {
|
2022-09-09 11:37:35 +00:00
|
|
|
m.mw.AddToObjectCounter(m.id, objectType, +1)
|
2022-08-19 16:49:09 +00:00
|
|
|
}
|
|
|
|
|
2023-06-14 08:00:44 +00:00
|
|
|
func (m *metricsWithID) SetMode(mode mode.Mode) {
|
|
|
|
m.mw.SetMode(m.id, mode)
|
2022-12-09 13:52:13 +00:00
|
|
|
}
|
|
|
|
|
2022-12-01 11:59:22 +00:00
|
|
|
func (m *metricsWithID) AddToContainerSize(cnr string, size int64) {
|
|
|
|
m.mw.AddToContainerSize(cnr, size)
|
|
|
|
}
|
|
|
|
|
2023-01-25 14:01:25 +00:00
|
|
|
func (m *metricsWithID) AddToPayloadSize(size int64) {
|
|
|
|
m.mw.AddToPayloadCounter(m.id, size)
|
|
|
|
}
|
|
|
|
|
2023-06-01 14:28:04 +00:00
|
|
|
func (m *metricsWithID) IncErrorCounter() {
|
|
|
|
m.mw.IncErrorCounter(m.id)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *metricsWithID) ClearErrorCounter() {
|
|
|
|
m.mw.ClearErrorCounter(m.id)
|
|
|
|
}
|
|
|
|
|
2023-06-13 16:48:15 +00:00
|
|
|
func (m *metricsWithID) DeleteShardMetrics() {
|
|
|
|
m.mw.DeleteShardMetrics(m.id)
|
2023-06-01 14:28:04 +00:00
|
|
|
}
|
|
|
|
|
2023-11-02 10:50:52 +00:00
|
|
|
func (m *metricsWithID) SetContainerObjectsCount(cnrID string, objectType string, value uint64) {
|
|
|
|
m.mw.SetContainerObjectCounter(m.id, cnrID, objectType, value)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *metricsWithID) IncContainerObjectsCount(cnrID string, objectType string) {
|
|
|
|
m.mw.IncContainerObjectCounter(m.id, cnrID, objectType)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (m *metricsWithID) SubContainerObjectsCount(cnrID string, objectType string, value uint64) {
|
|
|
|
m.mw.SubContainerObjectCounter(m.id, cnrID, objectType, value)
|
|
|
|
}
|
|
|
|
|
2020-11-17 12:26:03 +00:00
|
|
|
// AddShard adds a new shard to the storage engine.
|
|
|
|
//
|
|
|
|
// Returns any error encountered that did not allow adding a shard.
|
|
|
|
// Otherwise returns the ID of the added shard.
|
2023-08-31 16:26:47 +00:00
|
|
|
func (e *StorageEngine) AddShard(ctx context.Context, opts ...shard.Option) (*shard.ID, error) {
|
|
|
|
sh, err := e.createShard(ctx, opts)
|
2022-09-27 21:43:38 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, fmt.Errorf("could not create a shard: %w", err)
|
|
|
|
}
|
2022-09-26 21:39:34 +00:00
|
|
|
|
2022-09-27 21:43:38 +00:00
|
|
|
err = e.addShard(sh)
|
2022-03-01 08:59:05 +00:00
|
|
|
if err != nil {
|
2022-09-27 21:43:38 +00:00
|
|
|
return nil, fmt.Errorf("could not add %s shard: %w", sh.ID().String(), err)
|
2022-03-11 07:13:33 +00:00
|
|
|
}
|
|
|
|
|
2022-12-09 13:52:13 +00:00
|
|
|
if e.cfg.metrics != nil {
|
2023-06-14 08:00:44 +00:00
|
|
|
e.cfg.metrics.SetMode(sh.ID().String(), sh.GetMode())
|
2022-12-09 13:52:13 +00:00
|
|
|
}
|
|
|
|
|
2022-09-27 21:43:38 +00:00
|
|
|
return sh.ID(), nil
|
|
|
|
}
|
|
|
|
|
2023-08-31 16:26:47 +00:00
|
|
|
func (e *StorageEngine) createShard(ctx context.Context, opts []shard.Option) (*shard.Shard, error) {
|
2022-03-11 07:13:33 +00:00
|
|
|
id, err := generateShardID()
|
|
|
|
if err != nil {
|
|
|
|
return nil, fmt.Errorf("could not generate shard ID: %w", err)
|
2020-11-17 12:26:03 +00:00
|
|
|
}
|
|
|
|
|
2023-06-20 08:23:17 +00:00
|
|
|
opts = e.appendMetrics(id, opts)
|
|
|
|
|
|
|
|
sh := shard.New(append(opts,
|
|
|
|
shard.WithID(id),
|
|
|
|
shard.WithExpiredTombstonesCallback(e.processExpiredTombstones),
|
|
|
|
shard.WithExpiredLocksCallback(e.processExpiredLocks),
|
|
|
|
shard.WithDeletedLockCallback(e.processDeletedLocks),
|
|
|
|
shard.WithReportErrorFunc(e.reportShardErrorBackground),
|
2023-10-03 08:58:35 +00:00
|
|
|
shard.WithRebuildWorkerLimiter(e.rebuildLimiter),
|
2023-12-27 15:58:36 +00:00
|
|
|
shard.WithZeroSizeCallback(e.processZeroSizeContainers),
|
2023-06-20 08:23:17 +00:00
|
|
|
)...)
|
|
|
|
|
2023-08-31 16:26:47 +00:00
|
|
|
if err := sh.UpdateID(ctx); err != nil {
|
2023-06-20 08:23:17 +00:00
|
|
|
return nil, fmt.Errorf("could not update shard ID: %w", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
return sh, err
|
|
|
|
}
|
|
|
|
|
|
|
|
func (e *StorageEngine) appendMetrics(id *shard.ID, opts []shard.Option) []shard.Option {
|
2022-09-27 21:43:38 +00:00
|
|
|
e.mtx.RLock()
|
2023-06-20 08:23:17 +00:00
|
|
|
defer e.mtx.RUnlock()
|
2022-09-27 21:43:38 +00:00
|
|
|
|
2022-08-19 16:49:09 +00:00
|
|
|
if e.metrics != nil {
|
2023-06-20 08:23:17 +00:00
|
|
|
opts = append(opts,
|
|
|
|
shard.WithMetricsWriter(
|
|
|
|
&metricsWithID{
|
|
|
|
id: id.String(),
|
|
|
|
mw: e.metrics,
|
|
|
|
},
|
|
|
|
),
|
2023-06-22 11:55:30 +00:00
|
|
|
shard.WithWriteCacheMetrics(
|
2023-05-29 14:32:13 +00:00
|
|
|
&writeCacheMetrics{
|
|
|
|
shardID: id.String(),
|
|
|
|
metrics: e.metrics.WriteCache(),
|
2023-06-22 11:55:30 +00:00
|
|
|
},
|
2023-05-29 14:32:13 +00:00
|
|
|
),
|
|
|
|
shard.WithGCMetrics(
|
|
|
|
&gcMetrics{
|
|
|
|
storage: e.metrics.GC(),
|
|
|
|
shardID: id.String(),
|
|
|
|
},
|
|
|
|
),
|
|
|
|
)
|
2022-08-19 16:49:09 +00:00
|
|
|
}
|
|
|
|
|
2023-06-20 08:23:17 +00:00
|
|
|
return opts
|
2022-09-27 21:43:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (e *StorageEngine) addShard(sh *shard.Shard) error {
|
|
|
|
e.mtx.Lock()
|
|
|
|
defer e.mtx.Unlock()
|
|
|
|
|
|
|
|
pool, err := ants.NewPool(int(e.shardPoolSize), ants.WithNonblocking(true))
|
|
|
|
if err != nil {
|
|
|
|
return fmt.Errorf("could not create pool: %w", err)
|
|
|
|
}
|
|
|
|
|
2022-03-01 08:59:05 +00:00
|
|
|
strID := sh.ID().String()
|
|
|
|
if _, ok := e.shards[strID]; ok {
|
2022-09-27 21:43:38 +00:00
|
|
|
return fmt.Errorf("shard with id %s was already added", strID)
|
2022-03-01 08:59:05 +00:00
|
|
|
}
|
2021-10-08 12:25:45 +00:00
|
|
|
|
2023-02-27 13:16:37 +00:00
|
|
|
e.shards[strID] = hashedShard{
|
|
|
|
shardWrapper: shardWrapper{
|
2023-05-19 15:06:20 +00:00
|
|
|
errorCount: new(atomic.Uint32),
|
2023-02-27 13:16:37 +00:00
|
|
|
Shard: sh,
|
|
|
|
},
|
2023-06-02 12:39:16 +00:00
|
|
|
hash: hrw.StringHash(strID),
|
2022-01-31 14:58:32 +00:00
|
|
|
}
|
2020-11-17 12:26:03 +00:00
|
|
|
|
2021-10-08 12:25:45 +00:00
|
|
|
e.shardPools[strID] = pool
|
|
|
|
|
2022-09-27 21:43:38 +00:00
|
|
|
return nil
|
2020-11-17 12:26:03 +00:00
|
|
|
}
|
|
|
|
|
2022-09-26 21:39:34 +00:00
|
|
|
// removeShards removes specified shards. Skips non-existent shards.
|
2022-09-30 14:01:48 +00:00
|
|
|
// Logs errors about shards that it could not Close after the removal.
|
|
|
|
func (e *StorageEngine) removeShards(ids ...string) {
|
|
|
|
if len(ids) == 0 {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2023-02-27 13:16:37 +00:00
|
|
|
ss := make([]hashedShard, 0, len(ids))
|
2022-09-26 21:39:34 +00:00
|
|
|
|
2022-09-30 14:01:48 +00:00
|
|
|
e.mtx.Lock()
|
2022-09-26 21:39:34 +00:00
|
|
|
for _, id := range ids {
|
|
|
|
sh, found := e.shards[id]
|
|
|
|
if !found {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2023-06-13 16:48:15 +00:00
|
|
|
sh.DeleteShardMetrics()
|
2023-06-01 14:28:04 +00:00
|
|
|
|
2022-09-30 14:01:48 +00:00
|
|
|
ss = append(ss, sh)
|
2022-09-26 21:39:34 +00:00
|
|
|
delete(e.shards, id)
|
|
|
|
|
|
|
|
pool, ok := e.shardPools[id]
|
|
|
|
if ok {
|
|
|
|
pool.Release()
|
|
|
|
delete(e.shardPools, id)
|
|
|
|
}
|
|
|
|
|
2023-04-12 14:35:10 +00:00
|
|
|
e.log.Info(logs.EngineShardHasBeenRemoved,
|
2022-09-26 21:39:34 +00:00
|
|
|
zap.String("id", id))
|
|
|
|
}
|
2022-09-30 14:01:48 +00:00
|
|
|
e.mtx.Unlock()
|
2022-09-26 21:39:34 +00:00
|
|
|
|
2022-09-30 14:01:48 +00:00
|
|
|
for _, sh := range ss {
|
2023-06-20 08:59:18 +00:00
|
|
|
err := sh.SetMode(mode.Disabled)
|
|
|
|
if err != nil {
|
|
|
|
e.log.Error(logs.EngineCouldNotChangeShardModeToDisabled,
|
|
|
|
zap.Stringer("id", sh.ID()),
|
|
|
|
zap.Error(err),
|
|
|
|
)
|
|
|
|
}
|
|
|
|
err = sh.Close()
|
2022-09-30 14:01:48 +00:00
|
|
|
if err != nil {
|
2023-04-12 14:35:10 +00:00
|
|
|
e.log.Error(logs.EngineCouldNotCloseRemovedShard,
|
2022-09-30 14:01:48 +00:00
|
|
|
zap.Stringer("id", sh.ID()),
|
|
|
|
zap.Error(err),
|
|
|
|
)
|
|
|
|
}
|
|
|
|
}
|
2022-09-26 21:39:34 +00:00
|
|
|
}
|
|
|
|
|
2021-04-14 08:47:42 +00:00
|
|
|
func generateShardID() (*shard.ID, error) {
|
2020-11-17 12:26:03 +00:00
|
|
|
uid, err := uuid.NewRandom()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
bin, err := uid.MarshalBinary()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return shard.NewIDFromBytes(bin), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (e *StorageEngine) shardWeight(sh *shard.Shard) float64 {
|
|
|
|
weightValues := sh.WeightValues()
|
|
|
|
|
|
|
|
return float64(weightValues.FreeSpace)
|
|
|
|
}
|
|
|
|
|
2022-05-31 17:00:41 +00:00
|
|
|
func (e *StorageEngine) sortShardsByWeight(objAddr interface{ EncodeToString() string }) []hashedShard {
|
2020-11-18 12:06:47 +00:00
|
|
|
e.mtx.RLock()
|
|
|
|
defer e.mtx.RUnlock()
|
|
|
|
|
2023-06-02 12:39:16 +00:00
|
|
|
h := hrw.StringHash(objAddr.EncodeToString())
|
2020-11-30 14:58:44 +00:00
|
|
|
shards := make([]hashedShard, 0, len(e.shards))
|
2020-11-17 12:26:03 +00:00
|
|
|
for _, sh := range e.shards {
|
2022-01-31 14:58:32 +00:00
|
|
|
shards = append(shards, hashedShard(sh))
|
2020-11-17 12:26:03 +00:00
|
|
|
}
|
2023-03-30 11:49:15 +00:00
|
|
|
return sortShardsByWeight(shards, h)
|
|
|
|
}
|
2020-11-17 12:26:03 +00:00
|
|
|
|
2023-03-30 11:49:15 +00:00
|
|
|
func sortShardsByWeight(shards []hashedShard, h uint64) []hashedShard {
|
|
|
|
weights := make([]float64, 0, len(shards))
|
|
|
|
for _, sh := range shards {
|
|
|
|
weights = append(weights, float64(sh.Shard.WeightValues().FreeSpace))
|
|
|
|
}
|
2020-11-17 12:26:03 +00:00
|
|
|
|
2023-03-30 11:49:15 +00:00
|
|
|
hrw.SortHasherSliceByWeightValue(shards, weights, h)
|
2020-11-17 12:26:03 +00:00
|
|
|
return shards
|
|
|
|
}
|
|
|
|
|
2020-12-01 10:18:25 +00:00
|
|
|
func (e *StorageEngine) unsortedShards() []hashedShard {
|
|
|
|
e.mtx.RLock()
|
|
|
|
defer e.mtx.RUnlock()
|
|
|
|
|
|
|
|
shards := make([]hashedShard, 0, len(e.shards))
|
|
|
|
|
|
|
|
for _, sh := range e.shards {
|
2022-01-31 14:58:32 +00:00
|
|
|
shards = append(shards, hashedShard(sh))
|
2020-12-01 10:18:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return shards
|
|
|
|
}
|
|
|
|
|
2022-05-31 17:00:41 +00:00
|
|
|
func (e *StorageEngine) iterateOverSortedShards(addr oid.Address, handler func(int, hashedShard) (stop bool)) {
|
2020-12-01 10:18:25 +00:00
|
|
|
for i, sh := range e.sortShardsByWeight(addr) {
|
2022-01-31 14:58:32 +00:00
|
|
|
if handler(i, sh) {
|
2020-12-01 10:18:25 +00:00
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-01-31 14:58:32 +00:00
|
|
|
func (e *StorageEngine) iterateOverUnsortedShards(handler func(hashedShard) (stop bool)) {
|
2020-12-01 10:18:25 +00:00
|
|
|
for _, sh := range e.unsortedShards() {
|
2022-01-31 14:58:32 +00:00
|
|
|
if handler(sh) {
|
2020-11-17 12:26:03 +00:00
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-11-19 13:04:04 +00:00
|
|
|
|
|
|
|
// SetShardMode sets mode of the shard with provided identifier.
|
|
|
|
//
|
|
|
|
// Returns an error if shard mode was not set, or shard was not found in storage engine.
|
2022-06-28 14:05:08 +00:00
|
|
|
func (e *StorageEngine) SetShardMode(id *shard.ID, m mode.Mode, resetErrorCounter bool) error {
|
2020-11-19 13:04:04 +00:00
|
|
|
e.mtx.RLock()
|
|
|
|
defer e.mtx.RUnlock()
|
|
|
|
|
|
|
|
for shID, sh := range e.shards {
|
|
|
|
if id.String() == shID {
|
2022-01-31 15:15:33 +00:00
|
|
|
if resetErrorCounter {
|
|
|
|
sh.errorCount.Store(0)
|
2023-06-01 14:28:04 +00:00
|
|
|
sh.Shard.ClearErrorCounter()
|
2022-01-31 15:15:33 +00:00
|
|
|
}
|
2020-11-19 13:04:04 +00:00
|
|
|
return sh.SetMode(m)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return errShardNotFound
|
|
|
|
}
|
2020-11-30 14:58:44 +00:00
|
|
|
|
2022-09-26 06:30:41 +00:00
|
|
|
// HandleNewEpoch notifies every shard about NewEpoch event.
|
|
|
|
func (e *StorageEngine) HandleNewEpoch(epoch uint64) {
|
|
|
|
ev := shard.EventNewEpoch(epoch)
|
|
|
|
|
|
|
|
e.mtx.RLock()
|
|
|
|
defer e.mtx.RUnlock()
|
|
|
|
|
|
|
|
for _, sh := range e.shards {
|
|
|
|
sh.NotificationChannel() <- ev
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-30 14:58:44 +00:00
|
|
|
func (s hashedShard) Hash() uint64 {
|
2023-02-27 13:16:37 +00:00
|
|
|
return s.hash
|
2020-11-30 14:58:44 +00:00
|
|
|
}
|