Skip to content

Commit

Permalink
Standardize index cache metrics (#6523)
Browse files Browse the repository at this point in the history
  • Loading branch information
alanprot authored Jul 12, 2023
1 parent cb2ab1b commit a395c5d
Show file tree
Hide file tree
Showing 10 changed files with 114 additions and 71 deletions.
2 changes: 1 addition & 1 deletion cmd/thanos/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,7 +323,7 @@ func runStore(
if len(indexCacheContentYaml) > 0 {
indexCache, err = storecache.NewIndexCache(logger, indexCacheContentYaml, reg)
} else {
indexCache, err = storecache.NewInMemoryIndexCacheWithConfig(logger, reg, storecache.InMemoryIndexCacheConfig{
indexCache, err = storecache.NewInMemoryIndexCacheWithConfig(logger, nil, reg, storecache.InMemoryIndexCacheConfig{
MaxSize: model.Bytes(conf.indexCacheSizeBytes),
MaxItemSize: storecache.DefaultInMemoryIndexCacheConfig.MaxItemSize,
})
Expand Down
6 changes: 3 additions & 3 deletions pkg/store/bucket_e2e_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -495,7 +495,7 @@ func TestBucketStore_e2e(t *testing.T) {
}

if ok := t.Run("with large, sufficient index cache", func(t *testing.T) {
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(s.logger, nil, storecache.InMemoryIndexCacheConfig{
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(s.logger, nil, nil, storecache.InMemoryIndexCacheConfig{
MaxItemSize: 1e5,
MaxSize: 2e5,
})
Expand All @@ -507,7 +507,7 @@ func TestBucketStore_e2e(t *testing.T) {
}

t.Run("with small index cache", func(t *testing.T) {
indexCache2, err := storecache.NewInMemoryIndexCacheWithConfig(s.logger, nil, storecache.InMemoryIndexCacheConfig{
indexCache2, err := storecache.NewInMemoryIndexCacheWithConfig(s.logger, nil, nil, storecache.InMemoryIndexCacheConfig{
MaxItemSize: 50,
MaxSize: 100,
})
Expand Down Expand Up @@ -540,7 +540,7 @@ func TestBucketStore_ManyParts_e2e(t *testing.T) {

s := prepareStoreWithTestBlocks(t, dir, bkt, true, NewChunksLimiterFactory(0), NewSeriesLimiterFactory(0), NewBytesLimiterFactory(0), emptyRelabelConfig, allowAllFilterConf)

indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(s.logger, nil, storecache.InMemoryIndexCacheConfig{
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(s.logger, nil, nil, storecache.InMemoryIndexCacheConfig{
MaxItemSize: 1e5,
MaxSize: 2e5,
})
Expand Down
12 changes: 6 additions & 6 deletions pkg/store/bucket_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1517,7 +1517,7 @@ func TestBucketSeries_OneBlock_InMemIndexCacheSegfault(t *testing.T) {
chunkPool, err := pool.NewBucketedBytes(chunkBytesPoolMinSize, chunkBytesPoolMaxSize, 2, 100e7)
testutil.Ok(t, err)

indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, storecache.InMemoryIndexCacheConfig{
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, nil, storecache.InMemoryIndexCacheConfig{
MaxItemSize: 3000,
// This is the exact size of cache needed for our *single request*.
// This is limited in order to make sure we test evictions.
Expand Down Expand Up @@ -1820,7 +1820,7 @@ func TestSeries_ErrorUnmarshallingRequestHints(t *testing.T) {
fetcher, err := block.NewMetaFetcher(logger, 10, instrBkt, tmpDir, nil, nil)
testutil.Ok(tb, err)

indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, storecache.InMemoryIndexCacheConfig{})
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, nil, storecache.InMemoryIndexCacheConfig{})
testutil.Ok(tb, err)

store, err := NewBucketStore(
Expand Down Expand Up @@ -1911,7 +1911,7 @@ func TestSeries_BlockWithMultipleChunks(t *testing.T) {
fetcher, err := block.NewMetaFetcher(logger, 10, instrBkt, tmpDir, nil, nil)
testutil.Ok(tb, err)

indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, storecache.InMemoryIndexCacheConfig{})
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, nil, storecache.InMemoryIndexCacheConfig{})
testutil.Ok(tb, err)

store, err := NewBucketStore(
Expand Down Expand Up @@ -2093,7 +2093,7 @@ func setupStoreForHintsTest(t *testing.T) (testutil.TB, *BucketStore, []*storepb
fetcher, err := block.NewMetaFetcher(logger, 10, instrBkt, tmpDir, nil, nil)
testutil.Ok(tb, err)

indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, storecache.InMemoryIndexCacheConfig{})
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, nil, storecache.InMemoryIndexCacheConfig{})
testutil.Ok(tb, err)

store, err := NewBucketStore(
Expand Down Expand Up @@ -2309,7 +2309,7 @@ func TestSeries_ChunksHaveHashRepresentation(t *testing.T) {
fetcher, err := block.NewMetaFetcher(logger, 10, instrBkt, tmpDir, nil, nil)
testutil.Ok(tb, err)

indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, storecache.InMemoryIndexCacheConfig{})
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, nil, storecache.InMemoryIndexCacheConfig{})
testutil.Ok(tb, err)

store, err := NewBucketStore(
Expand Down Expand Up @@ -2516,7 +2516,7 @@ func prepareBucket(b *testing.B, resolutionLevel compact.ResolutionLevel) (*buck
// Create an index header reader.
indexHeaderReader, err := indexheader.NewBinaryReader(ctx, logger, bkt, tmpDir, blockMeta.ULID, DefaultPostingOffsetInMemorySampling)
testutil.Ok(b, err)
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, storecache.DefaultInMemoryIndexCacheConfig)
indexCache, err := storecache.NewInMemoryIndexCacheWithConfig(logger, nil, nil, storecache.DefaultInMemoryIndexCacheConfig)
testutil.Ok(b, err)

// Create a bucket block with only the dependencies we need for the benchmark.
Expand Down
21 changes: 21 additions & 0 deletions pkg/store/cache/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ import (
"strings"

"github.com/oklog/ulid"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/storage"
"golang.org/x/crypto/blake2b"
Expand Down Expand Up @@ -54,6 +56,25 @@ type IndexCache interface {
FetchMultiSeries(ctx context.Context, blockID ulid.ULID, ids []storage.SeriesRef) (hits map[storage.SeriesRef][]byte, misses []storage.SeriesRef)
}

// Common metrics that should be used by all cache implementations.
type commonMetrics struct {
requestTotal *prometheus.CounterVec
hitsTotal *prometheus.CounterVec
}

func newCommonMetrics(reg prometheus.Registerer) *commonMetrics {
return &commonMetrics{
requestTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Name: "thanos_store_index_cache_requests_total",
Help: "Total number of items requests to the cache.",
}, []string{"item_type"}),
hitsTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Name: "thanos_store_index_cache_hits_total",
Help: "Total number of items requests to the cache that were a hit.",
}, []string{"item_type"}),
}
}

type cacheKey struct {
block string
key interface{}
Expand Down
7 changes: 4 additions & 3 deletions pkg/store/cache/factory.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ type IndexCacheConfig struct {
func NewIndexCache(logger log.Logger, confContentYaml []byte, reg prometheus.Registerer) (IndexCache, error) {
level.Info(logger).Log("msg", "loading index cache configuration")
cacheConfig := &IndexCacheConfig{}
cacheMetrics := newCommonMetrics(reg)
if err := yaml.UnmarshalStrict(confContentYaml, cacheConfig); err != nil {
return nil, errors.Wrap(err, "parsing config YAML file")
}
Expand All @@ -46,18 +47,18 @@ func NewIndexCache(logger log.Logger, confContentYaml []byte, reg prometheus.Reg
var cache IndexCache
switch strings.ToUpper(string(cacheConfig.Type)) {
case string(INMEMORY):
cache, err = NewInMemoryIndexCache(logger, reg, backendConfig)
cache, err = NewInMemoryIndexCache(logger, cacheMetrics, reg, backendConfig)
case string(MEMCACHED):
var memcached cacheutil.RemoteCacheClient
memcached, err = cacheutil.NewMemcachedClient(logger, "index-cache", backendConfig, reg)
if err == nil {
cache, err = NewRemoteIndexCache(logger, memcached, reg)
cache, err = NewRemoteIndexCache(logger, memcached, cacheMetrics, reg)
}
case string(REDIS):
var redisCache cacheutil.RemoteCacheClient
redisCache, err = cacheutil.NewRedisClient(logger, "index-cache", backendConfig, reg)
if err == nil {
cache, err = NewRemoteIndexCache(logger, redisCache, reg)
cache, err = NewRemoteIndexCache(logger, redisCache, cacheMetrics, reg)
}
default:
return nil, errors.Errorf("index cache with type %s is not supported", cacheConfig.Type)
Expand Down
29 changes: 29 additions & 0 deletions pkg/store/cache/factory_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
// Copyright (c) The Thanos Authors.
// Licensed under the Apache License 2.0.

package storecache

import (
"testing"

"github.com/efficientgo/core/testutil"
"github.com/go-kit/log"
"github.com/prometheus/client_golang/prometheus"
)

func TestIndexCacheMetrics(t *testing.T) {
reg := prometheus.NewRegistry()
commonMetrics := newCommonMetrics(reg)

memcached := newMockedMemcachedClient(nil)
_, err := NewRemoteIndexCache(log.NewNopLogger(), memcached, commonMetrics, reg)
testutil.Ok(t, err)
conf := []byte(`
max_size: 10MB
max_item_size: 1MB
`)
// Make sure that the in memory cache does not register the same metrics of the remote index cache.
// If so, we should move those metrics to the `commonMetrics`
_, err = NewInMemoryIndexCache(log.NewNopLogger(), commonMetrics, reg, conf)
testutil.Ok(t, err)
}
39 changes: 18 additions & 21 deletions pkg/store/cache/inmemory.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,13 +43,13 @@ type InMemoryIndexCache struct {
curSize uint64

evicted *prometheus.CounterVec
requests *prometheus.CounterVec
hits *prometheus.CounterVec
added *prometheus.CounterVec
current *prometheus.GaugeVec
currentSize *prometheus.GaugeVec
totalCurrentSize *prometheus.GaugeVec
overflow *prometheus.CounterVec

commonMetrics *commonMetrics
}

// InMemoryIndexCacheConfig holds the in-memory index cache config.
Expand All @@ -72,26 +72,31 @@ func parseInMemoryIndexCacheConfig(conf []byte) (InMemoryIndexCacheConfig, error

// NewInMemoryIndexCache creates a new thread-safe LRU cache for index entries and ensures the total cache
// size approximately does not exceed maxBytes.
func NewInMemoryIndexCache(logger log.Logger, reg prometheus.Registerer, conf []byte) (*InMemoryIndexCache, error) {
func NewInMemoryIndexCache(logger log.Logger, commonMetrics *commonMetrics, reg prometheus.Registerer, conf []byte) (*InMemoryIndexCache, error) {
config, err := parseInMemoryIndexCacheConfig(conf)
if err != nil {
return nil, err
}

return NewInMemoryIndexCacheWithConfig(logger, reg, config)
return NewInMemoryIndexCacheWithConfig(logger, commonMetrics, reg, config)
}

// NewInMemoryIndexCacheWithConfig creates a new thread-safe LRU cache for index entries and ensures the total cache
// size approximately does not exceed maxBytes.
func NewInMemoryIndexCacheWithConfig(logger log.Logger, reg prometheus.Registerer, config InMemoryIndexCacheConfig) (*InMemoryIndexCache, error) {
func NewInMemoryIndexCacheWithConfig(logger log.Logger, commonMetrics *commonMetrics, reg prometheus.Registerer, config InMemoryIndexCacheConfig) (*InMemoryIndexCache, error) {
if config.MaxItemSize > config.MaxSize {
return nil, errors.Errorf("max item size (%v) cannot be bigger than overall cache size (%v)", config.MaxItemSize, config.MaxSize)
}

if commonMetrics == nil {
commonMetrics = newCommonMetrics(reg)
}

c := &InMemoryIndexCache{
logger: logger,
maxSizeBytes: uint64(config.MaxSize),
maxItemSizeBytes: uint64(config.MaxItemSize),
commonMetrics: commonMetrics,
}

c.evicted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Expand All @@ -110,13 +115,9 @@ func NewInMemoryIndexCacheWithConfig(logger log.Logger, reg prometheus.Registere
c.added.WithLabelValues(cacheTypeSeries)
c.added.WithLabelValues(cacheTypeExpandedPostings)

c.requests = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Name: "thanos_store_index_cache_requests_total",
Help: "Total number of requests to the cache.",
}, []string{"item_type"})
c.requests.WithLabelValues(cacheTypePostings)
c.requests.WithLabelValues(cacheTypeSeries)
c.requests.WithLabelValues(cacheTypeExpandedPostings)
c.commonMetrics.requestTotal.WithLabelValues(cacheTypePostings)
c.commonMetrics.requestTotal.WithLabelValues(cacheTypeSeries)
c.commonMetrics.requestTotal.WithLabelValues(cacheTypeExpandedPostings)

c.overflow = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Name: "thanos_store_index_cache_items_overflowed_total",
Expand All @@ -126,13 +127,9 @@ func NewInMemoryIndexCacheWithConfig(logger log.Logger, reg prometheus.Registere
c.overflow.WithLabelValues(cacheTypeSeries)
c.overflow.WithLabelValues(cacheTypeExpandedPostings)

c.hits = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
Name: "thanos_store_index_cache_hits_total",
Help: "Total number of requests to the cache that were a hit.",
}, []string{"item_type"})
c.hits.WithLabelValues(cacheTypePostings)
c.hits.WithLabelValues(cacheTypeSeries)
c.hits.WithLabelValues(cacheTypeExpandedPostings)
c.commonMetrics.hitsTotal.WithLabelValues(cacheTypePostings)
c.commonMetrics.hitsTotal.WithLabelValues(cacheTypeSeries)
c.commonMetrics.hitsTotal.WithLabelValues(cacheTypeExpandedPostings)

c.current = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{
Name: "thanos_store_index_cache_items",
Expand Down Expand Up @@ -201,7 +198,7 @@ func (c *InMemoryIndexCache) onEvict(key, val interface{}) {
}

func (c *InMemoryIndexCache) get(typ string, key cacheKey) ([]byte, bool) {
c.requests.WithLabelValues(typ).Inc()
c.commonMetrics.requestTotal.WithLabelValues(typ).Inc()

c.mtx.Lock()
defer c.mtx.Unlock()
Expand All @@ -210,7 +207,7 @@ func (c *InMemoryIndexCache) get(typ string, key cacheKey) ([]byte, bool) {
if !ok {
return nil, false
}
c.hits.WithLabelValues(typ).Inc()
c.commonMetrics.hitsTotal.WithLabelValues(typ).Inc()
return v.([]byte), true
}

Expand Down
32 changes: 16 additions & 16 deletions pkg/store/cache/inmemory_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,14 @@ import (
func TestNewInMemoryIndexCache(t *testing.T) {
// Should return error on invalid YAML config.
conf := []byte("invalid")
cache, err := NewInMemoryIndexCache(log.NewNopLogger(), nil, conf)
cache, err := NewInMemoryIndexCache(log.NewNopLogger(), nil, nil, conf)
testutil.NotOk(t, err)
testutil.Equals(t, (*InMemoryIndexCache)(nil), cache)

// Should instance an in-memory index cache with default config
// on empty YAML config.
conf = []byte{}
cache, err = NewInMemoryIndexCache(log.NewNopLogger(), nil, conf)
cache, err = NewInMemoryIndexCache(log.NewNopLogger(), nil, nil, conf)
testutil.Ok(t, err)
testutil.Equals(t, uint64(DefaultInMemoryIndexCacheConfig.MaxSize), cache.maxSizeBytes)
testutil.Equals(t, uint64(DefaultInMemoryIndexCacheConfig.MaxItemSize), cache.maxItemSizeBytes)
Expand All @@ -42,7 +42,7 @@ func TestNewInMemoryIndexCache(t *testing.T) {
max_size: 1MB
max_item_size: 2KB
`)
cache, err = NewInMemoryIndexCache(log.NewNopLogger(), nil, conf)
cache, err = NewInMemoryIndexCache(log.NewNopLogger(), nil, nil, conf)
testutil.Ok(t, err)
testutil.Equals(t, uint64(1024*1024), cache.maxSizeBytes)
testutil.Equals(t, uint64(2*1024), cache.maxItemSizeBytes)
Expand All @@ -52,7 +52,7 @@ max_item_size: 2KB
max_size: 2KB
max_item_size: 1MB
`)
cache, err = NewInMemoryIndexCache(log.NewNopLogger(), nil, conf)
cache, err = NewInMemoryIndexCache(log.NewNopLogger(), nil, nil, conf)
testutil.NotOk(t, err)
testutil.Equals(t, (*InMemoryIndexCache)(nil), cache)
// testutil.Equals(t, uint64(1024*1024), cache.maxSizeBytes)
Expand All @@ -64,7 +64,7 @@ max_item_size: 1MB

func TestInMemoryIndexCache_AvoidsDeadlock(t *testing.T) {
metrics := prometheus.NewRegistry()
cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{
cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), nil, metrics, InMemoryIndexCacheConfig{
MaxItemSize: sliceHeaderSize + 5,
MaxSize: sliceHeaderSize + 5,
})
Expand Down Expand Up @@ -114,7 +114,7 @@ func TestInMemoryIndexCache_UpdateItem(t *testing.T) {
})

metrics := prometheus.NewRegistry()
cache, err := NewInMemoryIndexCacheWithConfig(log.NewSyncLogger(errorLogger), metrics, InMemoryIndexCacheConfig{
cache, err := NewInMemoryIndexCacheWithConfig(log.NewSyncLogger(errorLogger), nil, metrics, InMemoryIndexCacheConfig{
MaxItemSize: maxSize,
MaxSize: maxSize,
})
Expand Down Expand Up @@ -208,7 +208,7 @@ func TestInMemoryIndexCache_UpdateItem(t *testing.T) {
// This should not happen as we hardcode math.MaxInt, but we still add test to check this out.
func TestInMemoryIndexCache_MaxNumberOfItemsHit(t *testing.T) {
metrics := prometheus.NewRegistry()
cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{
cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), nil, metrics, InMemoryIndexCacheConfig{
MaxItemSize: 2*sliceHeaderSize + 10,
MaxSize: 2*sliceHeaderSize + 10,
})
Expand All @@ -231,15 +231,15 @@ func TestInMemoryIndexCache_MaxNumberOfItemsHit(t *testing.T) {
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.evicted.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(3), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.commonMetrics.requestTotal.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.commonMetrics.requestTotal.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.commonMetrics.hitsTotal.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(0), promtest.ToFloat64(cache.commonMetrics.hitsTotal.WithLabelValues(cacheTypeSeries)))
}

func TestInMemoryIndexCache_Eviction_WithMetrics(t *testing.T) {
metrics := prometheus.NewRegistry()
cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), metrics, InMemoryIndexCacheConfig{
cache, err := NewInMemoryIndexCacheWithConfig(log.NewNopLogger(), nil, metrics, InMemoryIndexCacheConfig{
MaxItemSize: 2*sliceHeaderSize + 5,
MaxSize: 2*sliceHeaderSize + 5,
})
Expand Down Expand Up @@ -429,8 +429,8 @@ func TestInMemoryIndexCache_Eviction_WithMetrics(t *testing.T) {
// Other metrics.
testutil.Equals(t, float64(4), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(1), promtest.ToFloat64(cache.added.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(9), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(2), promtest.ToFloat64(cache.requests.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(5), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(1), promtest.ToFloat64(cache.hits.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(9), promtest.ToFloat64(cache.commonMetrics.requestTotal.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(2), promtest.ToFloat64(cache.commonMetrics.requestTotal.WithLabelValues(cacheTypeSeries)))
testutil.Equals(t, float64(5), promtest.ToFloat64(cache.commonMetrics.hitsTotal.WithLabelValues(cacheTypePostings)))
testutil.Equals(t, float64(1), promtest.ToFloat64(cache.commonMetrics.hitsTotal.WithLabelValues(cacheTypeSeries)))
}
Loading

0 comments on commit a395c5d

Please sign in to comment.