Signed-off-by: Anton Nikiforov <an.nikiforov@yadro.com>
This commit is contained in:
Anton Nikiforov 2024-12-02 22:47:26 +03:00
parent 635a292ae4
commit 16afbf1369
23 changed files with 107 additions and 85 deletions

View file

@ -884,7 +884,10 @@ func (c *cfg) engineOpts() []engine.Option {
) )
if c.metricsCollector != nil { if c.metricsCollector != nil {
opts = append(opts, engine.WithMetrics(c.metricsCollector.Engine())) opts = append(opts,
engine.WithEngineMetrics(c.metricsCollector.Engine()),
engine.WithWritecacheMetrics(c.metricsCollector.Writecachce()),
engine.WithGCMetrics(c.metricsCollector.GC()))
} }
return opts return opts

View file

@ -12,7 +12,7 @@ import (
type BlobobvnizcaMetrics interface { type BlobobvnizcaMetrics interface {
SetBlobobvnizcaTreeMode(shardID, path string, mode mode.ComponentMode) SetBlobobvnizcaTreeMode(shardID, path string, mode mode.ComponentMode)
CloseBlobobvnizcaTree(shardID, path string) CloseBlobobvnizcaTree(shardID, path string)
BlobobvnizcaTreeMethodDuration(shardID, path string, method string, d time.Duration, success bool, withStorageID NullBool) BlobobvnizcaTreeMethodDuration(shardID, path string, method string, d time.Duration, success bool, withStorageID string)
AddBlobobvnizcaTreePut(shardID, path string, size int) AddBlobobvnizcaTreePut(shardID, path string, size int)
AddBlobobvnizcaTreeGet(shardID, path string, size int) AddBlobobvnizcaTreeGet(shardID, path string, size int)
@ -128,13 +128,13 @@ func (b *blobovnicza) CloseBlobobvnizcaTree(shardID, path string) {
b.treeRebuildStatus.SetMode(shardID, path, undefinedStatus) b.treeRebuildStatus.SetMode(shardID, path, undefinedStatus)
} }
func (b *blobovnicza) BlobobvnizcaTreeMethodDuration(shardID, path string, method string, d time.Duration, success bool, withStorageID NullBool) { func (b *blobovnicza) BlobobvnizcaTreeMethodDuration(shardID, path string, method string, d time.Duration, success bool, withStorageID string) {
b.treeReqDuration.With(prometheus.Labels{ b.treeReqDuration.With(prometheus.Labels{
shardIDLabel: shardID, shardIDLabel: shardID,
pathLabel: path, pathLabel: path,
successLabel: strconv.FormatBool(success), successLabel: strconv.FormatBool(success),
methodLabel: method, methodLabel: method,
withStorageIDLabel: withStorageID.String(), withStorageIDLabel: withStorageID,
}).Observe(d.Seconds()) }).Observe(d.Seconds())
} }

View file

@ -12,7 +12,7 @@ type BlobstoreMetrics interface {
SetMode(shardID string, readOnly bool) SetMode(shardID string, readOnly bool)
Close(shardID string) Close(shardID string)
MethodDuration(shardID string, method string, d time.Duration, success bool, withStorageID NullBool) MethodDuration(shardID string, method string, d time.Duration, success bool, withStorageID string)
AddPut(shardID string, size int) AddPut(shardID string, size int)
AddGet(shardID string, size int) AddGet(shardID string, size int)
} }
@ -65,12 +65,12 @@ func (m *blobstoreMetrics) Close(shardID string) {
}) })
} }
func (m *blobstoreMetrics) MethodDuration(shardID string, method string, d time.Duration, success bool, withStorageID NullBool) { func (m *blobstoreMetrics) MethodDuration(shardID string, method string, d time.Duration, success bool, withStorageID string) {
m.reqDuration.With(prometheus.Labels{ m.reqDuration.With(prometheus.Labels{
shardIDLabel: shardID, shardIDLabel: shardID,
successLabel: strconv.FormatBool(success), successLabel: strconv.FormatBool(success),
methodLabel: method, methodLabel: method,
withStorageIDLabel: withStorageID.String(), withStorageIDLabel: withStorageID,
}).Observe(d.Seconds()) }).Observe(d.Seconds())
} }

View file

@ -28,9 +28,6 @@ type EngineMetrics interface {
SetRefillPercent(shardID, path string, percent uint32) SetRefillPercent(shardID, path string, percent uint32)
SetRefillStatus(shardID, path, status string) SetRefillStatus(shardID, path, status string)
SetEvacuationInProgress(shardID string, value bool) SetEvacuationInProgress(shardID string, value bool)
WriteCache() WriteCacheMetrics
GC() GCMetrics
} }
type engineMetrics struct { type engineMetrics struct {
@ -182,11 +179,11 @@ func (m *engineMetrics) SetMode(shardID string, mode mode.Mode) {
m.mode.SetMode(shardID, mode.String()) m.mode.SetMode(shardID, mode.String())
} }
func (m *engineMetrics) WriteCache() WriteCacheMetrics { func (m *engineMetrics) WriteCache() any {
return m.writeCache return m.writeCache
} }
func (m *engineMetrics) GC() GCMetrics { func (m *engineMetrics) GC() any {
return m.gc return m.gc
} }

View file

@ -79,6 +79,14 @@ func (m *NodeMetrics) Engine() EngineMetrics {
return m.engine return m.engine
} }
func (m *NodeMetrics) Writecachce() WriteCacheMetrics {
return m.engine.writeCache
}
func (m *NodeMetrics) GC() GCMetrics {
return m.engine.gc
}
func (m *NodeMetrics) State() StateMetrics { func (m *NodeMetrics) State() StateMetrics {
return m.state return m.state
} }

View file

@ -1,17 +0,0 @@
package metrics
import (
"strconv"
)
type NullBool struct {
Bool bool
Valid bool // Valid is true if Bool is not NULL
}
func (v NullBool) String() string {
if !v.Valid {
return ""
}
return strconv.FormatBool(v.Bool)
}

View file

@ -13,7 +13,7 @@ type WriteCacheMetrics interface {
SetActualCount(shardID, path, storageType string, count uint64) SetActualCount(shardID, path, storageType string, count uint64)
SetEstimateSize(shardID, path, storageType string, size uint64) SetEstimateSize(shardID, path, storageType string, size uint64)
SetMode(shardID, mode string) SetMode(shardID, mode string)
IncOperationCounter(shardID, path, storageType, operation string, success NullBool) IncOperationCounter(shardID, path, storageType, operation string, success string)
Close(shardID, path string) Close(shardID, path string)
} }
@ -80,12 +80,12 @@ func (m *writeCacheMetrics) SetMode(shardID string, mode string) {
m.mode.SetMode(shardID, mode) m.mode.SetMode(shardID, mode)
} }
func (m *writeCacheMetrics) IncOperationCounter(shardID, path, storageType, operation string, success NullBool) { func (m *writeCacheMetrics) IncOperationCounter(shardID, path, storageType, operation string, success string) {
m.operationCounter.With(prometheus.Labels{ m.operationCounter.With(prometheus.Labels{
shardIDLabel: shardID, shardIDLabel: shardID,
storageLabel: storageType, storageLabel: storageType,
operationLabel: operation, operationLabel: operation,
successLabel: success.String(), successLabel: success,
pathLabel: path, pathLabel: path,
}).Inc() }).Inc()
} }

View file

@ -45,7 +45,7 @@ func (r ListContainersRes) Containers() []cid.ID {
// //
// Returns an error if executions are blocked (see BlockExecution). // Returns an error if executions are blocked (see BlockExecution).
func (e *StorageEngine) ContainerSize(ctx context.Context, prm ContainerSizePrm) (res ContainerSizeRes, err error) { func (e *StorageEngine) ContainerSize(ctx context.Context, prm ContainerSizePrm) (res ContainerSizeRes, err error) {
defer elapsed("ContainerSize", e.metrics.AddMethodDuration)() defer elapsed("ContainerSize", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e.containerSize(ctx, prm) res, err = e.containerSize(ctx, prm)
@ -93,7 +93,7 @@ func (e *StorageEngine) containerSize(ctx context.Context, prm ContainerSizePrm)
// //
// Returns an error if executions are blocked (see BlockExecution). // Returns an error if executions are blocked (see BlockExecution).
func (e *StorageEngine) ListContainers(ctx context.Context, _ ListContainersPrm) (res ListContainersRes, err error) { func (e *StorageEngine) ListContainers(ctx context.Context, _ ListContainersPrm) (res ListContainersRes, err error) {
defer elapsed("ListContainers", e.metrics.AddMethodDuration)() defer elapsed("ListContainers", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e.listContainers(ctx) res, err = e.listContainers(ctx)

View file

@ -58,7 +58,7 @@ func (e *StorageEngine) Delete(ctx context.Context, prm DeletePrm) (res DeleteRe
attribute.Bool("force_removal", prm.forceRemoval), attribute.Bool("force_removal", prm.forceRemoval),
)) ))
defer span.End() defer span.End()
defer elapsed("Delete", e.metrics.AddMethodDuration)() defer elapsed("Delete", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e.delete(ctx, prm) res, err = e.delete(ctx, prm)

View file

@ -145,7 +145,7 @@ func (e *StorageEngine) reportShardError(
} }
errCount := sh.errorCount.Add(1) errCount := sh.errorCount.Add(1)
e.metrics.IncErrorCounter(sh.ID().String()) e.engineMetrics.IncErrorCounter(sh.ID().String())
sid := sh.ID() sid := sh.ID()
e.log.Warn(ctx, msg, append([]zap.Field{ e.log.Warn(ctx, msg, append([]zap.Field{
@ -187,7 +187,9 @@ type cfg struct {
errorsThreshold uint32 errorsThreshold uint32
metrics MetricRegister engineMetrics EngineMetrics
writeCacheMetrics WriteCacheMetrics
gcMetrics GCMetrics
shardPoolSize uint32 shardPoolSize uint32
@ -200,7 +202,10 @@ func defaultCfg() *cfg {
res := &cfg{ res := &cfg{
log: logger.NewLoggerWrapper(zap.L()), log: logger.NewLoggerWrapper(zap.L()),
shardPoolSize: 20, shardPoolSize: 20,
metrics: noopMetrics{},
engineMetrics: noopMetrics{},
writeCacheMetrics: noopWriteCacheMetrics{},
gcMetrics: noopGCMetrics{},
} }
res.containerSource.Store(&containerSource{}) res.containerSource.Store(&containerSource{})
return res return res
@ -231,9 +236,21 @@ func WithLogger(l *logger.Logger) Option {
} }
} }
func WithMetrics(v MetricRegister) Option { func WithEngineMetrics(v EngineMetrics) Option {
return func(c *cfg) { return func(c *cfg) {
c.metrics = v c.engineMetrics = v
}
}
func WithWritecacheMetrics(v WriteCacheMetrics) Option {
return func(c *cfg) {
c.writeCacheMetrics = v
}
}
func WithGCMetrics(v GCMetrics) Option {
return func(c *cfg) {
c.gcMetrics = v
} }
} }

View file

@ -56,7 +56,7 @@ func (e *StorageEngine) Get(ctx context.Context, prm GetPrm) (res GetRes, err er
attribute.String("address", prm.addr.EncodeToString()), attribute.String("address", prm.addr.EncodeToString()),
)) ))
defer span.End() defer span.End()
defer elapsed("Get", e.metrics.AddMethodDuration)() defer elapsed("Get", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e.get(ctx, prm) res, err = e.get(ctx, prm)

View file

@ -68,7 +68,7 @@ func (e *StorageEngine) Head(ctx context.Context, prm HeadPrm) (res HeadRes, err
func (e *StorageEngine) head(ctx context.Context, prm HeadPrm) (HeadRes, error) { func (e *StorageEngine) head(ctx context.Context, prm HeadPrm) (HeadRes, error) {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.head") ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.head")
defer span.End() defer span.End()
defer elapsed("Head", e.metrics.AddMethodDuration)() defer elapsed("Head", e.engineMetrics.AddMethodDuration)()
var ( var (
head *objectSDK.Object head *objectSDK.Object

View file

@ -70,7 +70,7 @@ var errInhumeFailure = errors.New("inhume operation failed")
func (e *StorageEngine) Inhume(ctx context.Context, prm InhumePrm) (res InhumeRes, err error) { func (e *StorageEngine) Inhume(ctx context.Context, prm InhumePrm) (res InhumeRes, err error) {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Inhume") ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.Inhume")
defer span.End() defer span.End()
defer elapsed("Inhume", e.metrics.AddMethodDuration)() defer elapsed("Inhume", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e.inhume(ctx, prm) res, err = e.inhume(ctx, prm)
@ -360,7 +360,7 @@ func (e *StorageEngine) processZeroSizeContainers(ctx context.Context, ids []cid
} }
for id := range idMap { for id := range idMap {
e.metrics.DeleteContainerSize(id.EncodeToString()) e.engineMetrics.DeleteContainerSize(id.EncodeToString())
} }
} }
@ -438,7 +438,7 @@ func (e *StorageEngine) processZeroCountContainers(ctx context.Context, ids []ci
} }
for id := range idMap { for id := range idMap {
e.metrics.DeleteContainerCount(id.EncodeToString()) e.engineMetrics.DeleteContainerCount(id.EncodeToString())
} }
} }

View file

@ -101,7 +101,7 @@ func (l ListWithCursorRes) Cursor() *Cursor {
func (e *StorageEngine) ListWithCursor(ctx context.Context, prm ListWithCursorPrm) (ListWithCursorRes, error) { func (e *StorageEngine) ListWithCursor(ctx context.Context, prm ListWithCursorPrm) (ListWithCursorRes, error) {
ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.ListWithCursor") ctx, span := tracing.StartSpanFromContext(ctx, "StorageEngine.ListWithCursor")
defer span.End() defer span.End()
defer elapsed("ListWithCursor", e.metrics.AddMethodDuration)() defer elapsed("ListWithCursor", e.engineMetrics.AddMethodDuration)()
result := make([]objectcore.Info, 0, prm.count) result := make([]objectcore.Info, 0, prm.count)

View file

@ -32,7 +32,7 @@ func (e *StorageEngine) Lock(ctx context.Context, idCnr cid.ID, locker oid.ID, l
attribute.Int("locked_count", len(locked)), attribute.Int("locked_count", len(locked)),
)) ))
defer span.End() defer span.End()
defer elapsed("Lock", e.metrics.AddMethodDuration)() defer elapsed("Lock", e.engineMetrics.AddMethodDuration)()
return e.execIfNotBlocked(func() error { return e.execIfNotBlocked(func() error {
return e.lock(ctx, idCnr, locker, locked) return e.lock(ctx, idCnr, locker, locked)

View file

@ -7,7 +7,23 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/local_object_storage/shard/mode"
) )
type MetricRegister interface { type WriteCacheMetrics interface {
AddMethodDuration(shardID, path, storageType, method string, success bool, d time.Duration)
SetActualCount(shardID, path, storageType string, count uint64)
SetEstimateSize(shardID, path, storageType string, size uint64)
SetMode(shardID, mode string)
IncOperationCounter(shardID, path, storageType, operation string, success string)
Close(shardID, path string)
}
type GCMetrics interface {
AddRunDuration(shardID string, d time.Duration, success bool)
AddDeletedCount(shardID string, deleted, failed uint64)
AddExpiredObjectCollectionDuration(shardID string, d time.Duration, success bool, objectType string)
AddInhumedObjectCount(shardID string, count uint64, objectType string)
}
type EngineMetrics interface {
AddMethodDuration(method string, d time.Duration) AddMethodDuration(method string, d time.Duration)
SetObjectCounter(shardID, objectType string, v uint64) SetObjectCounter(shardID, objectType string, v uint64)
@ -31,9 +47,6 @@ type MetricRegister interface {
SetRefillPercent(shardID, path string, percent uint32) SetRefillPercent(shardID, path string, percent uint32)
SetRefillStatus(shardID, path, status string) SetRefillStatus(shardID, path, status string)
SetEvacuationInProgress(shardID string, value bool) SetEvacuationInProgress(shardID string, value bool)
WriteCache() metrics.WriteCacheMetrics
GC() metrics.GCMetrics
} }
func elapsed(method string, addFunc func(method string, d time.Duration)) func() { func elapsed(method string, addFunc func(method string, d time.Duration)) func() {
@ -76,9 +89,9 @@ type (
) )
var ( var (
_ MetricRegister = noopMetrics{} _ EngineMetrics = noopMetrics{}
_ metrics.WriteCacheMetrics = noopWriteCacheMetrics{} _ WriteCacheMetrics = noopWriteCacheMetrics{}
_ metrics.GCMetrics = noopGCMetrics{} _ GCMetrics = noopGCMetrics{}
) )
func (noopMetrics) AddMethodDuration(string, time.Duration) {} func (noopMetrics) AddMethodDuration(string, time.Duration) {}
@ -99,14 +112,12 @@ func (noopMetrics) IncRefillObjectsCount(string, string, int, bool) {}
func (noopMetrics) SetRefillPercent(string, string, uint32) {} func (noopMetrics) SetRefillPercent(string, string, uint32) {}
func (noopMetrics) SetRefillStatus(string, string, string) {} func (noopMetrics) SetRefillStatus(string, string, string) {}
func (noopMetrics) SetEvacuationInProgress(string, bool) {} func (noopMetrics) SetEvacuationInProgress(string, bool) {}
func (noopMetrics) WriteCache() metrics.WriteCacheMetrics { return noopWriteCacheMetrics{} }
func (noopMetrics) GC() metrics.GCMetrics { return noopGCMetrics{} }
func (noopWriteCacheMetrics) AddMethodDuration(string, string, string, string, bool, time.Duration) {} func (noopWriteCacheMetrics) AddMethodDuration(string, string, string, string, bool, time.Duration) {}
func (noopWriteCacheMetrics) SetActualCount(string, string, string, uint64) {} func (noopWriteCacheMetrics) SetActualCount(string, string, string, uint64) {}
func (noopWriteCacheMetrics) SetEstimateSize(string, string, string, uint64) {} func (noopWriteCacheMetrics) SetEstimateSize(string, string, string, uint64) {}
func (noopWriteCacheMetrics) SetMode(string, string) {} func (noopWriteCacheMetrics) SetMode(string, string) {}
func (noopWriteCacheMetrics) IncOperationCounter(string, string, string, string, metrics.NullBool) {} func (noopWriteCacheMetrics) IncOperationCounter(string, string, string, string, string) {}
func (noopWriteCacheMetrics) Close(string, string) {} func (noopWriteCacheMetrics) Close(string, string) {}
func (noopGCMetrics) AddRunDuration(string, time.Duration, bool) {} func (noopGCMetrics) AddRunDuration(string, time.Duration, bool) {}

View file

@ -56,7 +56,7 @@ func (e *StorageEngine) Put(ctx context.Context, prm PutPrm) (err error) {
attribute.String("address", object.AddressOf(prm.Object).EncodeToString()), attribute.String("address", object.AddressOf(prm.Object).EncodeToString()),
)) ))
defer span.End() defer span.End()
defer elapsed("Put", e.metrics.AddMethodDuration)() defer elapsed("Put", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
err = e.put(ctx, prm) err = e.put(ctx, prm)

View file

@ -72,7 +72,7 @@ func (e *StorageEngine) GetRange(ctx context.Context, prm RngPrm) (res RngRes, e
attribute.String("length", strconv.FormatUint(prm.ln, 10)), attribute.String("length", strconv.FormatUint(prm.ln, 10)),
)) ))
defer span.End() defer span.End()
defer elapsed("GetRange", e.metrics.AddMethodDuration)() defer elapsed("GetRange", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e.getRange(ctx, prm) res, err = e.getRange(ctx, prm)

View file

@ -51,7 +51,7 @@ func (e *StorageEngine) Select(ctx context.Context, prm SelectPrm) (res SelectRe
attribute.String("container_id", prm.cnr.EncodeToString()), attribute.String("container_id", prm.cnr.EncodeToString()),
)) ))
defer span.End() defer span.End()
defer elapsed("Select", e.metrics.AddMethodDuration)() defer elapsed("Select", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e._select(ctx, prm) res, err = e._select(ctx, prm)
@ -98,7 +98,7 @@ func (e *StorageEngine) _select(ctx context.Context, prm SelectPrm) (SelectRes,
// //
// Returns an error if executions are blocked (see BlockExecution). // Returns an error if executions are blocked (see BlockExecution).
func (e *StorageEngine) List(ctx context.Context, limit uint64) (res SelectRes, err error) { func (e *StorageEngine) List(ctx context.Context, limit uint64) (res SelectRes, err error) {
defer elapsed("List", e.metrics.AddMethodDuration)() defer elapsed("List", e.engineMetrics.AddMethodDuration)()
err = e.execIfNotBlocked(func() error { err = e.execIfNotBlocked(func() error {
res, err = e.list(ctx, limit) res, err = e.list(ctx, limit)
return err return err

View file

@ -28,7 +28,7 @@ type hashedShard struct {
type metricsWithID struct { type metricsWithID struct {
id string id string
mw MetricRegister mw EngineMetrics
} }
func (m *metricsWithID) SetShardID(id string) { func (m *metricsWithID) SetShardID(id string) {
@ -116,7 +116,7 @@ func (e *StorageEngine) AddShard(ctx context.Context, opts ...shard.Option) (*sh
return nil, fmt.Errorf("could not add %s shard: %w", sh.ID().String(), err) return nil, fmt.Errorf("could not add %s shard: %w", sh.ID().String(), err)
} }
e.cfg.metrics.SetMode(sh.ID().String(), sh.GetMode()) e.cfg.engineMetrics.SetMode(sh.ID().String(), sh.GetMode())
return sh.ID(), nil return sh.ID(), nil
} }
@ -154,18 +154,18 @@ func (e *StorageEngine) appendMetrics(id *shard.ID, opts []shard.Option) []shard
shard.WithMetricsWriter( shard.WithMetricsWriter(
&metricsWithID{ &metricsWithID{
id: id.String(), id: id.String(),
mw: e.metrics, mw: e.engineMetrics,
}, },
), ),
shard.WithWriteCacheMetrics( shard.WithWriteCacheMetrics(
&writeCacheMetrics{ &writeCacheMetrics{
shardID: id.String(), shardID: id.String(),
metrics: e.metrics.WriteCache(), metrics: e.writeCacheMetrics,
}, },
), ),
shard.WithGCMetrics( shard.WithGCMetrics(
&gcMetrics{ &gcMetrics{
storage: e.metrics.GC(), storage: e.gcMetrics,
shardID: id.String(), shardID: id.String(),
}, },
), ),
@ -217,7 +217,7 @@ func (e *StorageEngine) removeShards(ctx context.Context, ids ...string) {
continue continue
} }
e.metrics.DeleteShardMetrics(id) e.engineMetrics.DeleteShardMetrics(id)
ss = append(ss, sh) ss = append(ss, sh)
delete(e.shards, id) delete(e.shards, id)
@ -318,7 +318,7 @@ func (e *StorageEngine) SetShardMode(ctx context.Context, id *shard.ID, m mode.M
if id.String() == shID { if id.String() == shID {
if resetErrorCounter { if resetErrorCounter {
sh.errorCount.Store(0) sh.errorCount.Store(0)
e.metrics.ClearErrorCounter(shID) e.engineMetrics.ClearErrorCounter(shID)
} }
return sh.SetMode(ctx, m) return sh.SetMode(ctx, m)
} }
@ -422,7 +422,7 @@ func (e *StorageEngine) deleteShards(ctx context.Context, ids []*shard.ID) ([]ha
for _, sh := range ss { for _, sh := range ss {
idStr := sh.ID().String() idStr := sh.ID().String()
e.metrics.DeleteShardMetrics(idStr) e.engineMetrics.DeleteShardMetrics(idStr)
delete(e.shards, idStr) delete(e.shards, idStr)

View file

@ -2,6 +2,7 @@ package engine
import ( import (
"context" "context"
"strconv"
"sync" "sync"
"time" "time"
@ -182,11 +183,11 @@ func (m *writeCacheMetrics) SetActualCounters(count uint64) {
} }
func (m *writeCacheMetrics) Flush(success bool, st writecache.StorageType) { func (m *writeCacheMetrics) Flush(success bool, st writecache.StorageType) {
m.metrics.IncOperationCounter(m.shardID, m.path, st.String(), "Flush", metrics.NullBool{Bool: success, Valid: true}) m.metrics.IncOperationCounter(m.shardID, m.path, st.String(), "Flush", strconv.FormatBool(success))
} }
func (m *writeCacheMetrics) Evict(st writecache.StorageType) { func (m *writeCacheMetrics) Evict(st writecache.StorageType) {
m.metrics.IncOperationCounter(m.shardID, m.path, st.String(), "Evict", metrics.NullBool{}) m.metrics.IncOperationCounter(m.shardID, m.path, st.String(), "Evict", "")
} }
func (m *writeCacheMetrics) Close() { func (m *writeCacheMetrics) Close() {

View file

@ -1,6 +1,7 @@
package metrics package metrics
import ( import (
"strconv"
"time" "time"
metrics_impl "git.frostfs.info/TrueCloudLab/frostfs-node/internal/metrics" metrics_impl "git.frostfs.info/TrueCloudLab/frostfs-node/internal/metrics"
@ -56,40 +57,40 @@ func (m *blobovniczaTreeMetrics) ObjectMoved(d time.Duration) {
} }
func (m *blobovniczaTreeMetrics) Delete(d time.Duration, success, withStorageID bool) { func (m *blobovniczaTreeMetrics) Delete(d time.Duration, success, withStorageID bool) {
m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Delete", d, success, metrics_impl.NullBool{Valid: true, Bool: withStorageID}) m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Delete", d, success, strconv.FormatBool(withStorageID))
} }
func (m *blobovniczaTreeMetrics) Exists(d time.Duration, success, withStorageID bool) { func (m *blobovniczaTreeMetrics) Exists(d time.Duration, success, withStorageID bool) {
m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Exists", d, success, metrics_impl.NullBool{Valid: true, Bool: withStorageID}) m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Exists", d, success, strconv.FormatBool(withStorageID))
} }
func (m *blobovniczaTreeMetrics) GetRange(d time.Duration, size int, success, withStorageID bool) { func (m *blobovniczaTreeMetrics) GetRange(d time.Duration, size int, success, withStorageID bool) {
m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "GetRange", d, success, metrics_impl.NullBool{Valid: true, Bool: withStorageID}) m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "GetRange", d, success, strconv.FormatBool(withStorageID))
if success { if success {
m.m.AddBlobobvnizcaTreeGet(m.shardID, m.path, size) m.m.AddBlobobvnizcaTreeGet(m.shardID, m.path, size)
} }
} }
func (m *blobovniczaTreeMetrics) Get(d time.Duration, size int, success, withStorageID bool) { func (m *blobovniczaTreeMetrics) Get(d time.Duration, size int, success, withStorageID bool) {
m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Get", d, success, metrics_impl.NullBool{Valid: true, Bool: withStorageID}) m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Get", d, success, strconv.FormatBool(withStorageID))
if success { if success {
m.m.AddBlobobvnizcaTreeGet(m.shardID, m.path, size) m.m.AddBlobobvnizcaTreeGet(m.shardID, m.path, size)
} }
} }
func (m *blobovniczaTreeMetrics) Iterate(d time.Duration, success bool) { func (m *blobovniczaTreeMetrics) Iterate(d time.Duration, success bool) {
m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Iterate", d, success, metrics_impl.NullBool{}) m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Iterate", d, success, "")
} }
func (m *blobovniczaTreeMetrics) Put(d time.Duration, size int, success bool) { func (m *blobovniczaTreeMetrics) Put(d time.Duration, size int, success bool) {
m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Put", d, success, metrics_impl.NullBool{}) m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "Put", d, success, "")
if success { if success {
m.m.AddBlobobvnizcaTreePut(m.shardID, m.path, size) m.m.AddBlobobvnizcaTreePut(m.shardID, m.path, size)
} }
} }
func (m *blobovniczaTreeMetrics) ObjectsCount(d time.Duration, success bool) { func (m *blobovniczaTreeMetrics) ObjectsCount(d time.Duration, success bool) {
m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "ObjectsCount", d, success, metrics_impl.NullBool{}) m.m.BlobobvnizcaTreeMethodDuration(m.shardID, m.path, "ObjectsCount", d, success, "")
} }
type blobovniczaMetrics struct { type blobovniczaMetrics struct {

View file

@ -1,6 +1,7 @@
package metrics package metrics
import ( import (
"strconv"
"time" "time"
metrics_impl "git.frostfs.info/TrueCloudLab/frostfs-node/internal/metrics" metrics_impl "git.frostfs.info/TrueCloudLab/frostfs-node/internal/metrics"
@ -32,38 +33,38 @@ func (m *blobstoreMetrics) Close() {
} }
func (m *blobstoreMetrics) Delete(d time.Duration, success, withStorageID bool) { func (m *blobstoreMetrics) Delete(d time.Duration, success, withStorageID bool) {
m.m.MethodDuration(m.shardID, "Delete", d, success, metrics_impl.NullBool{Bool: withStorageID, Valid: true}) m.m.MethodDuration(m.shardID, "Delete", d, success, strconv.FormatBool(withStorageID))
} }
func (m *blobstoreMetrics) Exists(d time.Duration, success, withStorageID bool) { func (m *blobstoreMetrics) Exists(d time.Duration, success, withStorageID bool) {
m.m.MethodDuration(m.shardID, "Exists", d, success, metrics_impl.NullBool{Bool: withStorageID, Valid: true}) m.m.MethodDuration(m.shardID, "Exists", d, success, strconv.FormatBool(withStorageID))
} }
func (m *blobstoreMetrics) GetRange(d time.Duration, size int, success, withStorageID bool) { func (m *blobstoreMetrics) GetRange(d time.Duration, size int, success, withStorageID bool) {
m.m.MethodDuration(m.shardID, "GetRange", d, success, metrics_impl.NullBool{Bool: withStorageID, Valid: true}) m.m.MethodDuration(m.shardID, "GetRange", d, success, strconv.FormatBool(withStorageID))
if success { if success {
m.m.AddGet(m.shardID, size) m.m.AddGet(m.shardID, size)
} }
} }
func (m *blobstoreMetrics) Get(d time.Duration, size int, success, withStorageID bool) { func (m *blobstoreMetrics) Get(d time.Duration, size int, success, withStorageID bool) {
m.m.MethodDuration(m.shardID, "Get", d, success, metrics_impl.NullBool{Bool: withStorageID, Valid: true}) m.m.MethodDuration(m.shardID, "Get", d, success, strconv.FormatBool(withStorageID))
if success { if success {
m.m.AddGet(m.shardID, size) m.m.AddGet(m.shardID, size)
} }
} }
func (m *blobstoreMetrics) Iterate(d time.Duration, success bool) { func (m *blobstoreMetrics) Iterate(d time.Duration, success bool) {
m.m.MethodDuration(m.shardID, "Iterate", d, success, metrics_impl.NullBool{}) m.m.MethodDuration(m.shardID, "Iterate", d, success, "")
} }
func (m *blobstoreMetrics) Put(d time.Duration, size int, success bool) { func (m *blobstoreMetrics) Put(d time.Duration, size int, success bool) {
m.m.MethodDuration(m.shardID, "Put", d, success, metrics_impl.NullBool{}) m.m.MethodDuration(m.shardID, "Put", d, success, "")
if success { if success {
m.m.AddPut(m.shardID, size) m.m.AddPut(m.shardID, size)
} }
} }
func (m *blobstoreMetrics) ObjectsCount(d time.Duration, success bool) { func (m *blobstoreMetrics) ObjectsCount(d time.Duration, success bool) {
m.m.MethodDuration(m.shardID, "ObjectsCount", d, success, metrics_impl.NullBool{}) m.m.MethodDuration(m.shardID, "ObjectsCount", d, success, "")
} }