From b698ab3daa63c0dce682e8e4b4a1eedfeaac6f69 Mon Sep 17 00:00:00 2001 From: Dimitar Dimitrov Date: Thu, 23 May 2024 15:44:32 +0200 Subject: [PATCH 1/7] query-frontend: correctly replace `@` modifier for split queries (#8162) * query-frontend: correctly replace `@` modifier for split queries The existing code would only evaluate the at modifier for vector selectors, but subquery expressions also can have it. Signed-off-by: Dimitar Dimitrov * Add CHANGELOG.md entry Signed-off-by: Dimitar Dimitrov * Rename variable Signed-off-by: Dimitar Dimitrov --------- Signed-off-by: Dimitar Dimitrov --- CHANGELOG.md | 1 + .../astmapper/instant_splitting_test.go | 4 ++++ .../querymiddleware/split_and_cache.go | 19 ++++++++++++++----- .../querymiddleware/split_and_cache_test.go | 14 ++++++++++++++ 4 files changed, 33 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c330cbe84fe..6a4bdaa7779 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -69,6 +69,7 @@ * [BUGFIX] Querying: matrix results returned from instant queries were not sorted by series. #8113 * [BUGFIX] Query scheduler: Fix a crash in result marshaling. #8140 * [BUGFIX] Store-gateway: Allow long-running index scans to be interrupted. #8154 +* [BUGFIX] Query-frontend: fix splitting of queries using `@ start()` and `@end()` modifiers on a subquery. Previously the `start()` and `end()` would be evaluated using the start end end of the split query instead of the original query. #8162 ### Mixin diff --git a/pkg/frontend/querymiddleware/astmapper/instant_splitting_test.go b/pkg/frontend/querymiddleware/astmapper/instant_splitting_test.go index 3c8197ac0e0..16b9490d708 100644 --- a/pkg/frontend/querymiddleware/astmapper/instant_splitting_test.go +++ b/pkg/frontend/querymiddleware/astmapper/instant_splitting_test.go @@ -605,6 +605,10 @@ func TestInstantSplitterSkippedQueryReason(t *testing.T) { query: `max_over_time(absent_over_time(deriv(rate(metric_counter[1m])[5m:1m])[2m:])[10m:])`, skippedReason: SkippedReasonSubquery, }, + { + query: `sum by(group_1) (sum_over_time(metric_counter[7d:] @ start()))`, + skippedReason: SkippedReasonSubquery, + }, } { tt := tt diff --git a/pkg/frontend/querymiddleware/split_and_cache.go b/pkg/frontend/querymiddleware/split_and_cache.go index ede4bf86e0a..71bdfb55c25 100644 --- a/pkg/frontend/querymiddleware/split_and_cache.go +++ b/pkg/frontend/querymiddleware/split_and_cache.go @@ -667,14 +667,23 @@ func evaluateAtModifierFunction(query string, start, end int64) (string, error) return "", apierror.New(apierror.TypeBadData, decorateWithParamName(err, "query").Error()) } parser.Inspect(expr, func(n parser.Node, _ []parser.Node) error { - if selector, ok := n.(*parser.VectorSelector); ok { - switch selector.StartOrEnd { + switch exprAt := n.(type) { + case *parser.VectorSelector: + switch exprAt.StartOrEnd { case parser.START: - selector.Timestamp = &start + exprAt.Timestamp = &start case parser.END: - selector.Timestamp = &end + exprAt.Timestamp = &end } - selector.StartOrEnd = 0 + exprAt.StartOrEnd = 0 + case *parser.SubqueryExpr: + switch exprAt.StartOrEnd { + case parser.START: + exprAt.Timestamp = &start + case parser.END: + exprAt.Timestamp = &end + } + exprAt.StartOrEnd = 0 } return nil }) diff --git a/pkg/frontend/querymiddleware/split_and_cache_test.go b/pkg/frontend/querymiddleware/split_and_cache_test.go index 3859a381a5e..0b54e720877 100644 --- a/pkg/frontend/querymiddleware/split_and_cache_test.go +++ b/pkg/frontend/querymiddleware/split_and_cache_test.go @@ -1620,6 +1620,10 @@ func TestSplitQueryByInterval(t *testing.T) { queryFooAtStartExpr, _ := parser.ParseExpr(queryFooAtStart) queryFooAtZero := "foo @ 0.000" queryFooAtZeroExpr, _ := parser.ParseExpr(queryFooAtZero) + queryFooSubqueryAtStart := "sum_over_time(foo[1d:] @ start())" + queryFooSubqueryAtStartExpr, _ := parser.ParseExpr(queryFooSubqueryAtStart) + queryFooSubqueryAtZero := "sum_over_time(foo[1d:] @ 0.000)" + queryFooSubqueryAtZeroExpr, _ := parser.ParseExpr(queryFooSubqueryAtZero) for i, tc := range []struct { input MetricsQueryRequest @@ -1662,6 +1666,14 @@ func TestSplitQueryByInterval(t *testing.T) { }, interval: day, }, + { + input: &PrometheusRangeQueryRequest{minT: -(24 * 3600 * seconds), start: 0, end: 2 * 24 * 3600 * seconds, step: 15 * seconds, queryExpr: queryFooSubqueryAtStartExpr}, + expected: []MetricsQueryRequest{ + &PrometheusRangeQueryRequest{minT: -(24 * 3600 * seconds), start: 0, end: (24 * 3600 * seconds) - (15 * seconds), step: 15 * seconds, queryExpr: queryFooSubqueryAtZeroExpr}, + &PrometheusRangeQueryRequest{minT: -(24 * 3600 * seconds), start: 24 * 3600 * seconds, end: 2 * 24 * 3600 * seconds, step: 15 * seconds, queryExpr: queryFooSubqueryAtZeroExpr}, + }, + interval: day, + }, { input: &PrometheusRangeQueryRequest{start: 0, end: 2 * 3 * 3600 * seconds, step: 15 * seconds, queryExpr: queryFooExpr}, expected: []MetricsQueryRequest{ @@ -1797,6 +1809,8 @@ func Test_evaluateAtModifier(t *testing.T) { {"topk(5, rate(http_requests_total[1h] @ start()))", "topk(5, rate(http_requests_total[1h] @ 1546300.800))", nil}, {"topk(5, rate(http_requests_total[1h] @ 0))", "topk(5, rate(http_requests_total[1h] @ 0.000))", nil}, {"http_requests_total[1h] @ 10.001", "http_requests_total[1h] @ 10.001", nil}, + {"sum_over_time(http_requests_total[1h:] @ start())", "sum_over_time(http_requests_total[1h:] @ 1546300.800)", nil}, + {"sum_over_time((http_requests_total @ end())[1h:] @ start())", "sum_over_time((http_requests_total @ 1646300.800)[1h:] @ 1546300.800)", nil}, { `min_over_time( sum by(cluster) ( From 2a0da30ed761a80c84bfd832633fb2e90b5db639 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 24 May 2024 04:28:55 +1000 Subject: [PATCH 2/7] Don't load series multiple times when streaming chunks from store-gateways and only one batch is needed (#8039) Co-authored-by: Jeanette Tan --- CHANGELOG.md | 1 + pkg/storegateway/bucket.go | 102 +++--- pkg/storegateway/bucket_chunk_reader_test.go | 2 +- pkg/storegateway/series_refs.go | 102 +++--- pkg/storegateway/series_refs_streaming.go | 145 ++++++++ .../series_refs_streaming_test.go | 345 ++++++++++++++++++ pkg/storegateway/series_refs_test.go | 71 +--- 7 files changed, 605 insertions(+), 163 deletions(-) create mode 100644 pkg/storegateway/series_refs_streaming.go create mode 100644 pkg/storegateway/series_refs_streaming_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index 6a4bdaa7779..53e24cc3ebf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -38,6 +38,7 @@ * [ENHANCEMENT] Store-gateway: add `-blocks-storage.bucket-store.index-header.lazy-loading-concurrency-queue-timeout`. When set, loads of index-headers at the store-gateway's index-header lazy load gate will not wait longer than that to execute. If a load reaches the wait timeout, then the querier will retry the blocks on a different store-gateway. If all store-gateways are unavailable, then the query will fail with `err-mimir-store-consistency-check-failed`. #8138 * [ENHANCEMENT] Ingester: Optimize querying with regexp matchers. #8106 * [ENHANCEMENT] Distributor: Introduce `-distributor.max-request-pool-buffer-size` to allow configuring the maximum size of the request pool buffers. #8082 +* [ENHANCEMENT] Store-gateway: improve performance when streaming chunks to queriers is enabled (`-querier.prefer-streaming-chunks-from-store-gateways=true`) and the query selects fewer than `-blocks-storage.bucket-store.batch-series-size` series (defaults to 5000 series). #8039 * [ENHANCEMENT] Ingester: active series are now updated along with owned series. They decrease when series change ownership between ingesters. This helps provide a more accurate total of active series when ingesters are added. This is only enabled when `-ingester.track-ingester-owned-series` or `-ingester.use-ingester-owned-series-for-limits` are enabled. #8084 * [BUGFIX] Rules: improve error handling when querier is local to the ruler. #7567 * [BUGFIX] Querier, store-gateway: Protect against panics raised during snappy encoding. #7520 diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 102448cb046..c5e1118df26 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -606,10 +606,8 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storegatewaypb.Stor defer done() var ( - // If we are streaming the series labels and chunks separately, we don't need to fetch the postings - // twice. So we use these slices to re-use them. Each reuse[i] corresponds to a single block. - reuse []*reusedPostingsAndMatchers - resHints = &hintspb.SeriesResponseHints{} + streamingIterators *streamingSeriesIterators + resHints = &hintspb.SeriesResponseHints{} ) for _, b := range blocks { resHints.AddQueriedBlock(b.meta.ULID) @@ -633,7 +631,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storegatewaypb.Stor seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series")) ) - seriesSet, reuse, err = s.streamingSeriesForBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) + seriesSet, streamingIterators, err = s.createIteratorForChunksStreamingLabelsPhase(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) if err != nil { return err } @@ -661,15 +659,11 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storegatewaypb.Stor start := time.Now() if req.StreamingChunksBatchSize > 0 { - var seriesChunkIt iterator[seriesChunksSet] - seriesChunkIt, err = s.streamingChunksSetForBlocks(ctx, req, blocks, indexReaders, readers, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, reuse) - if err != nil { - return err - } + seriesChunkIt := s.createIteratorForChunksStreamingChunksPhase(ctx, readers, stats, chunksLimiter, seriesLimiter, streamingIterators) err = s.sendStreamingChunks(req, srv, seriesChunkIt, stats, streamingSeriesCount) } else { var seriesSet storepb.SeriesSet - seriesSet, err = s.nonStreamingSeriesSetForBlocks(ctx, req, blocks, indexReaders, readers, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) + seriesSet, err = s.createIteratorForNonChunksStreamingRequest(ctx, req, blocks, indexReaders, readers, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) if err != nil { return err } @@ -1027,8 +1021,8 @@ func chunksSize(chks []storepb.AggrChunk) (size int) { return size } -// nonStreamingSeriesSetForBlocks is used when the streaming feature is not enabled. -func (s *BucketStore) nonStreamingSeriesSetForBlocks( +// createIteratorForNonChunksStreamingRequest is used when the streaming feature is not enabled. +func (s *BucketStore) createIteratorForNonChunksStreamingRequest( ctx context.Context, req *storepb.SeriesRequest, blocks []*bucketBlock, @@ -1036,15 +1030,15 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( chunkReaders *bucketChunkReaders, shardSelector *sharding.ShardSelector, matchers []*labels.Matcher, - chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. - seriesLimiter SeriesLimiter, // Rate limiter for loading series. + chunksLimiter ChunksLimiter, + seriesLimiter SeriesLimiter, stats *safeQueryStats, ) (storepb.SeriesSet, error) { strategy := defaultStrategy if req.SkipChunks { strategy = noChunkRefs } - it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, nil, strategy) + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy, nil) if err != nil { return nil, err } @@ -1059,56 +1053,45 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( return set, nil } -// streamingSeriesForBlocks is used when streaming feature is enabled. +// createIteratorForChunksStreamingLabelsPhase is used when streaming feature is enabled. // It returns a series set that only contains the series labels without any chunks information. -// The returned postings (series ref) and matches should be re-used when getting chunks to save on computation. -func (s *BucketStore) streamingSeriesForBlocks( +// The streamingSeriesIterators should be re-used when getting chunks to save on computation. +func (s *BucketStore) createIteratorForChunksStreamingLabelsPhase( ctx context.Context, req *storepb.SeriesRequest, blocks []*bucketBlock, indexReaders map[ulid.ULID]*bucketIndexReader, shardSelector *sharding.ShardSelector, matchers []*labels.Matcher, - chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. - seriesLimiter SeriesLimiter, // Rate limiter for loading series. + chunksLimiter ChunksLimiter, + seriesLimiter SeriesLimiter, stats *safeQueryStats, -) (storepb.SeriesSet, []*reusedPostingsAndMatchers, error) { - var ( - reuse = make([]*reusedPostingsAndMatchers, len(blocks)) - strategy = noChunkRefs | overlapMintMaxt - ) - for i := range reuse { - reuse[i] = &reusedPostingsAndMatchers{} - } - it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, reuse, strategy) +) (storepb.SeriesSet, *streamingSeriesIterators, error) { + streamingIterators := newStreamingSeriesIterators() + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, overlapMintMaxt, streamingIterators) if err != nil { return nil, nil, err } - return newSeriesSetWithoutChunks(ctx, it, stats), reuse, nil + + return newSeriesSetWithoutChunks(ctx, it, stats), streamingIterators, nil } -// streamingChunksSetForBlocks is used when streaming feature is enabled. -// It returns an iterator to go over the chunks for the series returned in the streamingSeriesForBlocks call. -// It is recommended to pass the reusePostings and reusePendingMatches returned by the streamingSeriesForBlocks call. -func (s *BucketStore) streamingChunksSetForBlocks( +// createIteratorForChunksStreamingChunksPhase is used when streaming feature is enabled. +// It returns an iterator to go over the chunks for the series returned in the createIteratorForChunksStreamingLabelsPhase call. +// It is required to pass the iterators returned by the createIteratorForChunksStreamingLabelsPhase call for reuse. +func (s *BucketStore) createIteratorForChunksStreamingChunksPhase( ctx context.Context, - req *storepb.SeriesRequest, - blocks []*bucketBlock, - indexReaders map[ulid.ULID]*bucketIndexReader, chunkReaders *bucketChunkReaders, - shardSelector *sharding.ShardSelector, - matchers []*labels.Matcher, - chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. - seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, - reuse []*reusedPostingsAndMatchers, // Should come from streamingSeriesForBlocks. -) (iterator[seriesChunksSet], error) { - it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, reuse, defaultStrategy) - if err != nil { - return nil, err - } + chunksLimiter ChunksLimiter, + seriesLimiter SeriesLimiter, + iterators *streamingSeriesIterators, +) iterator[seriesChunksSet] { + preparedIterators := iterators.prepareForChunksStreamingPhase() + it := s.getSeriesIteratorFromPerBlockIterators(preparedIterators, chunksLimiter, seriesLimiter) scsi := newChunksPreloadingIterator(ctx, s.logger, s.userID, *chunkReaders, it, s.maxSeriesPerBatch, stats) - return scsi, nil + + return scsi } func (s *BucketStore) getSeriesIteratorFromBlocks( @@ -1121,8 +1104,8 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, - reuse []*reusedPostingsAndMatchers, // Used if not empty. If not empty, len(reuse) must be len(blocks). strategy seriesIteratorStrategy, + streamingIterators *streamingSeriesIterators, ) (iterator[seriesChunkRefsSet], error) { var ( mtx = sync.Mutex{} @@ -1131,9 +1114,8 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( begin = time.Now() blocksQueriedByBlockMeta = make(map[blockQueriedMeta]int) ) - for i, b := range blocks { + for _, b := range blocks { b := b - i := i // Keep track of queried blocks. indexr := indexReaders[b.meta.ULID] @@ -1144,10 +1126,6 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( if shardSelector != nil { blockSeriesHashCache = s.seriesHashCache.GetBlockCache(b.meta.ULID.String()) } - var r *reusedPostingsAndMatchers - if len(reuse) > 0 { - r = reuse[i] - } g.Go(func() error { part, err := openBlockSeriesChunkRefsSetsIterator( ctx, @@ -1162,8 +1140,8 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( strategy, req.MinTime, req.MaxTime, stats, - r, s.logger, + streamingIterators, ) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) @@ -1192,13 +1170,17 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( stats.streamingSeriesExpandPostingsDuration += time.Since(begin) }) - mergedIterator := mergedSeriesChunkRefsSetIterators(s.maxSeriesPerBatch, batches...) + return s.getSeriesIteratorFromPerBlockIterators(batches, chunksLimiter, seriesLimiter), nil +} + +func (s *BucketStore) getSeriesIteratorFromPerBlockIterators(perBlockIterators []iterator[seriesChunkRefsSet], chunksLimiter ChunksLimiter, seriesLimiter SeriesLimiter) iterator[seriesChunkRefsSet] { + mergedIterator := mergedSeriesChunkRefsSetIterators(s.maxSeriesPerBatch, perBlockIterators...) // Apply limits after the merging, so that if the same series is part of multiple blocks it just gets // counted once towards the limit. mergedIterator = newLimitingSeriesChunkRefsSetIterator(mergedIterator, chunksLimiter, seriesLimiter) - return mergedIterator, nil + return mergedIterator } func (s *BucketStore) recordSeriesCallResult(safeStats *safeQueryStats) { @@ -1467,8 +1449,8 @@ func blockLabelNames(ctx context.Context, indexr *bucketIndexReader, matchers [] noChunkRefs, minTime, maxTime, stats, - nil, logger, + nil, ) if err != nil { return nil, errors.Wrap(err, "fetch series") diff --git a/pkg/storegateway/bucket_chunk_reader_test.go b/pkg/storegateway/bucket_chunk_reader_test.go index 03bce0fc253..ba96344881c 100644 --- a/pkg/storegateway/bucket_chunk_reader_test.go +++ b/pkg/storegateway/bucket_chunk_reader_test.go @@ -44,8 +44,8 @@ func TestBucketChunkReader_refetchChunks(t *testing.T) { block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), - nil, log.NewNopLogger(), + nil, ) require.NoError(t, err) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index df23681acb5..060263ea0d0 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -139,6 +139,13 @@ func (b seriesChunkRefsSet) release() { seriesChunkRefsSetPool.Put(&reuse) } +// makeUnreleasable returns a new seriesChunkRefsSet that cannot be released on a subsequent call to release. +// +// This is useful for scenarios where a set will be used multiple times and so it is not safe for consumers to release it. +func (b seriesChunkRefsSet) makeUnreleasable() seriesChunkRefsSet { + return seriesChunkRefsSet{b.series, false} +} + // seriesChunkRefs holds a series with a list of chunk references. type seriesChunkRefs struct { lset labels.Labels @@ -690,47 +697,58 @@ func openBlockSeriesChunkRefsSetsIterator( ctx context.Context, batchSize int, tenantID string, - indexr *bucketIndexReader, // Index reader for block. + indexr *bucketIndexReader, indexCache indexcache.IndexCache, blockMeta *block.Meta, - matchers []*labels.Matcher, // Series matchers. - shard *sharding.ShardSelector, // Shard selector. + matchers []*labels.Matcher, + shard *sharding.ShardSelector, seriesHasher seriesHasher, strategy seriesIteratorStrategy, - minTime, maxTime int64, // Series must have data in this time range to be returned (ignored if skipChunks=true). + minTime, maxTime int64, stats *safeQueryStats, - reuse *reusedPostingsAndMatchers, // If this is not nil, these posting and matchers are used as it is without fetching new ones. logger log.Logger, + streamingIterators *streamingSeriesIterators, ) (iterator[seriesChunkRefsSet], error) { if batchSize <= 0 { return nil, errors.New("set size must be a positive number") } - var ( - ps []storage.SeriesRef - pendingMatchers []*labels.Matcher - fetchPostings = true - ) - if reuse != nil { - fetchPostings = !reuse.isSet() - ps = reuse.ps - pendingMatchers = reuse.matchers + ps, pendingMatchers, err := indexr.ExpandedPostings(ctx, matchers, stats) + if err != nil { + return nil, errors.Wrap(err, "expanded matching postings") } - if fetchPostings { - var err error - ps, pendingMatchers, err = indexr.ExpandedPostings(ctx, matchers, stats) - if err != nil { - return nil, errors.Wrap(err, "expanded matching postings") - } - if reuse != nil { - reuse.set(ps, pendingMatchers) - } + + iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { + return openBlockSeriesChunkRefsSetsIteratorFromPostings(ctx, tenantID, indexr, indexCache, blockMeta, shard, seriesHasher, strategy, minTime, maxTime, stats, psi, pendingMatchers, logger) } + if streamingIterators == nil { + psi := newPostingsSetsIterator(ps, batchSize) + return iteratorFactory(strategy, psi), nil + } + + return streamingIterators.wrapIterator(strategy, ps, batchSize, iteratorFactory), nil +} + +func openBlockSeriesChunkRefsSetsIteratorFromPostings( + ctx context.Context, + tenantID string, + indexr *bucketIndexReader, + indexCache indexcache.IndexCache, + blockMeta *block.Meta, + shard *sharding.ShardSelector, + seriesHasher seriesHasher, + strategy seriesIteratorStrategy, + minTime, maxTime int64, + stats *safeQueryStats, + postingsSetsIterator *postingsSetsIterator, + pendingMatchers []*labels.Matcher, + logger log.Logger, +) iterator[seriesChunkRefsSet] { var it iterator[seriesChunkRefsSet] it = newLoadingSeriesChunkRefsSetIterator( ctx, - newPostingsSetsIterator(ps, batchSize), + postingsSetsIterator, indexr, indexCache, stats, @@ -743,36 +761,12 @@ func openBlockSeriesChunkRefsSetsIterator( tenantID, logger, ) + if len(pendingMatchers) > 0 { it = newFilteringSeriesChunkRefsSetIterator(pendingMatchers, it, stats) } - return it, nil -} - -// reusedPostings is used to share the postings and matches across function calls for re-use -// in case of streaming series. We have it as a separate struct so that we can give a safe way -// to use it by making a copy where required. You can use it to put items only once. -type reusedPostingsAndMatchers struct { - ps []storage.SeriesRef - matchers []*labels.Matcher - filled bool -} - -func (p *reusedPostingsAndMatchers) set(ps []storage.SeriesRef, matchers []*labels.Matcher) { - if p.filled { - // We already have something here. - return - } - // Postings list can be modified later, so we make a copy here. - p.ps = make([]storage.SeriesRef, len(ps)) - copy(p.ps, ps) - p.matchers = matchers - p.filled = true -} - -func (p *reusedPostingsAndMatchers) isSet() bool { - return p.filled + return it } // seriesIteratorStrategy defines the strategy to use when loading the series and their chunk refs. @@ -812,6 +806,14 @@ func (s seriesIteratorStrategy) isNoChunkRefsAndOverlapMintMaxt() bool { return s.isNoChunkRefs() && s.isOverlapMintMaxt() } +func (s seriesIteratorStrategy) withNoChunkRefs() seriesIteratorStrategy { + return s | noChunkRefs +} + +func (s seriesIteratorStrategy) withChunkRefs() seriesIteratorStrategy { + return s & ^noChunkRefs +} + func newLoadingSeriesChunkRefsSetIterator( ctx context.Context, postingsSetIterator *postingsSetsIterator, diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go new file mode 100644 index 00000000000..b5c5d8bd3bb --- /dev/null +++ b/pkg/storegateway/series_refs_streaming.go @@ -0,0 +1,145 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package storegateway + +import ( + "sync" + + "github.com/prometheus/prometheus/storage" +) + +type seriesChunkRefsIteratorFactory func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] + +// chunksStreamingCachingSeriesChunkRefsSetIterator is an iterator used while streaming chunks from store-gateways to queriers. +// +// It wraps another iterator that does the actual work. If that iterator is expected to produce only a single batch, +// this iterator caches that batch for the chunks streaming phase, to avoid repeating work done during the series label sending phase. +type chunksStreamingCachingSeriesChunkRefsSetIterator struct { + strategy seriesIteratorStrategy + iteratorFactory seriesChunkRefsIteratorFactory + postings []storage.SeriesRef + batchSize int + + it iterator[seriesChunkRefsSet] + + expectSingleBatch bool + inChunksStreamingPhaseForSingleBatch bool + haveCachedBatch bool // It's possible that we expected a single batch to be returned, but the batch was filtered out by the inner iterator. + + currentBatchIndex int // -1 after beginning chunks streaming phase, 0 when on first and only batch, 1 after first and only batch + cachedBatch seriesChunkRefsSet +} + +func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIteratorStrategy, postings []storage.SeriesRef, batchSize int, iteratorFactory seriesChunkRefsIteratorFactory) *chunksStreamingCachingSeriesChunkRefsSetIterator { + expectSingleBatch := len(postings) <= batchSize + var initialStrategy seriesIteratorStrategy + var psi *postingsSetsIterator + + if expectSingleBatch { + initialStrategy = strategy.withChunkRefs() + + // No need to make a copy: we're only going to use the postings once. + psi = newPostingsSetsIterator(postings, batchSize) + } else { + // We'll load chunk refs during the chunks streaming phase. + initialStrategy = strategy.withNoChunkRefs() + + copiedPostings := make([]storage.SeriesRef, len(postings)) + copy(copiedPostings, postings) + psi = newPostingsSetsIterator(copiedPostings, batchSize) + } + + return &chunksStreamingCachingSeriesChunkRefsSetIterator{ + strategy: strategy, + postings: postings, + batchSize: batchSize, + iteratorFactory: iteratorFactory, + it: iteratorFactory(initialStrategy, psi), + expectSingleBatch: expectSingleBatch, + } +} + +func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) Next() bool { + if i.inChunksStreamingPhaseForSingleBatch && i.haveCachedBatch { + i.currentBatchIndex++ + return i.currentBatchIndex < 1 + } + + return i.it.Next() +} + +func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) At() seriesChunkRefsSet { + if i.inChunksStreamingPhaseForSingleBatch { + if i.currentBatchIndex == 0 && i.haveCachedBatch { + // Called Next() once. Return the cached batch. + // If the original batch was releasable or unreleasable, retain that state here. + return i.cachedBatch + } + + // Haven't called Next() yet, or called Next() multiple times and we've advanced past the only batch. + // At() should never be called in either case, so just return nothing if it is. + return seriesChunkRefsSet{} + } + + set := i.it.At() + + if i.expectSingleBatch { + i.cachedBatch = set + i.haveCachedBatch = true + + // Don't allow releasing this batch - we'll need it again later, so releasing it is not safe. + return set.makeUnreleasable() + } + + return set +} + +func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) Err() error { + return i.it.Err() +} + +func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) PrepareForChunksStreamingPhase() { + if i.expectSingleBatch { + i.inChunksStreamingPhaseForSingleBatch = true + i.currentBatchIndex = -1 + } else { + // No need to make a copy of postings here like we do in newChunksStreamingCachingSeriesChunkRefsSetIterator: + // we're not expecting to use them again after this, so we don't care if they're modified. + psi := newPostingsSetsIterator(i.postings, i.batchSize) + i.it = i.iteratorFactory(i.strategy.withChunkRefs(), psi) + } +} + +// streamingSeriesIterators represents a collection of iterators that will be used to handle a +// Series() request that uses chunks streaming. +type streamingSeriesIterators struct { + iterators []*chunksStreamingCachingSeriesChunkRefsSetIterator + mtx *sync.RWMutex +} + +func newStreamingSeriesIterators() *streamingSeriesIterators { + return &streamingSeriesIterators{ + mtx: &sync.RWMutex{}, + } +} + +func (i *streamingSeriesIterators) wrapIterator(strategy seriesIteratorStrategy, ps []storage.SeriesRef, batchSize int, iteratorFactory seriesChunkRefsIteratorFactory) iterator[seriesChunkRefsSet] { + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, ps, batchSize, iteratorFactory) + + i.mtx.Lock() + i.iterators = append(i.iterators, it) + i.mtx.Unlock() + + return it +} + +func (i *streamingSeriesIterators) prepareForChunksStreamingPhase() []iterator[seriesChunkRefsSet] { + prepared := make([]iterator[seriesChunkRefsSet], 0, len(i.iterators)) + + for _, it := range i.iterators { + it.PrepareForChunksStreamingPhase() + prepared = append(prepared, it) + } + + return prepared +} diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go new file mode 100644 index 00000000000..51e8f6bfe80 --- /dev/null +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -0,0 +1,345 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package storegateway + +import ( + "errors" + "fmt" + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/stretchr/testify/require" +) + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_HappyPath(t *testing.T) { + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 + factoryCalls := 0 + var factoryStrategy seriesIteratorStrategy + + firstBatchSeries := []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1")}, + {lset: labels.FromStrings("series", "2")}, + {lset: labels.FromStrings("series", "3")}, + } + + iteratorFactory := func(strategy seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + return newSliceSeriesChunkRefsSetIterator( + nil, + seriesChunkRefsSet{ + series: firstBatchSeries, + releasable: true, + }, + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) + + // Inner iterator should be created with chunk refs enabled. + require.Equal(t, 1, factoryCalls) + require.Equal(t, defaultStrategy.withChunkRefs(), factoryStrategy) + require.NoError(t, it.Err()) + + // During label sending phase, the single batch should be returned and not be releasable. + batches := readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + + unreleasableBatch := seriesChunkRefsSet{ + series: firstBatchSeries, + releasable: false, + } + require.Equal(t, []seriesChunkRefsSet{unreleasableBatch}, batches) + + // Prepare for chunks streaming phase. Inner iterator should not be recreated. + it.PrepareForChunksStreamingPhase() + require.Equal(t, 1, factoryCalls) + require.NoError(t, it.Err()) + + // During chunks streaming phase, the single batch should be returned but be releasable this time. + batches = readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + + releasableBatch := seriesChunkRefsSet{ + series: firstBatchSeries, + releasable: true, + } + require.Equal(t, []seriesChunkRefsSet{releasableBatch}, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_InnerIteratorReturnsUnreleasableSet(t *testing.T) { + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 + factoryCalls := 0 + + unreleasableBatch := seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1")}, + {lset: labels.FromStrings("series", "2")}, + {lset: labels.FromStrings("series", "3")}, + }, + releasable: false, + } + + iteratorFactory := func(_ seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { + factoryCalls++ + return newSliceSeriesChunkRefsSetIterator(nil, unreleasableBatch) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) + + // During label sending phase, the single batch should be returned and not be releasable. + batches := readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Equal(t, []seriesChunkRefsSet{unreleasableBatch}, batches) + + // Prepare for chunks streaming phase. Inner iterator should not be recreated. + it.PrepareForChunksStreamingPhase() + require.Equal(t, 1, factoryCalls) + require.NoError(t, it.Err()) + + // During chunks streaming phase, the single batch should be returned and still not be releasable. + batches = readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Equal(t, []seriesChunkRefsSet{unreleasableBatch}, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_AllBatchesFilteredOut(t *testing.T) { + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 + factoryCalls := 0 + var factoryStrategy seriesIteratorStrategy + + iteratorFactory := func(strategy seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + return newSliceSeriesChunkRefsSetIterator( + nil, + // No batches. + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) + + // Inner iterator should be created with chunk refs enabled. + require.Equal(t, 1, factoryCalls) + require.Equal(t, defaultStrategy.withChunkRefs(), factoryStrategy) + require.NoError(t, it.Err()) + + // Label sending phase. + batches := readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Empty(t, batches) + + // Prepare for chunks streaming phase. Inner iterator should not be recreated. + it.PrepareForChunksStreamingPhase() + require.Equal(t, 1, factoryCalls) + require.NoError(t, it.Err()) + + // Chunks streaming phase. + batches = readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Empty(t, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_IteratorReturnsError(t *testing.T) { + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 + factoryCalls := 0 + iteratorError := errors.New("something went wrong") + + iteratorFactory := func(_ seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { + factoryCalls++ + + return newSliceSeriesChunkRefsSetIterator( + iteratorError, + seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1")}, + {lset: labels.FromStrings("series", "2")}, + {lset: labels.FromStrings("series", "3")}, + }, + releasable: true, + }, + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) + + // During label sending phase, the error should be returned. + _ = readAllSeriesChunkRefsSet(it) + require.Equal(t, iteratorError, it.Err()) + + // Prepare for chunks streaming phase. Inner iterator should not be recreated. + it.PrepareForChunksStreamingPhase() + require.Equal(t, 1, factoryCalls) + + // During chunks streaming phase, the error should be returned. + _ = readAllSeriesChunkRefsSet(it) + require.Equal(t, iteratorError, it.Err()) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_HappyPath(t *testing.T) { + postings := []storage.SeriesRef{1, 2, 3, 4, 5, 6} + batchSize := 3 + factoryCalls := 0 + var factoryStrategy seriesIteratorStrategy + + firstBatchWithNoChunkRefs := seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1", "have_chunk_refs", "no")}, + {lset: labels.FromStrings("series", "2", "have_chunk_refs", "no")}, + {lset: labels.FromStrings("series", "3", "have_chunk_refs", "no")}, + }, + releasable: true, + } + + secondBatchWithNoChunkRefs := seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "4", "have_chunk_refs", "no")}, + {lset: labels.FromStrings("series", "5", "have_chunk_refs", "no")}, + {lset: labels.FromStrings("series", "6", "have_chunk_refs", "no")}, + }, + releasable: true, + } + + firstBatchWithChunkRefs := seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1", "have_chunk_refs", "yes")}, + {lset: labels.FromStrings("series", "2", "have_chunk_refs", "yes")}, + {lset: labels.FromStrings("series", "3", "have_chunk_refs", "yes")}, + }, + releasable: true, + } + + secondBatchWithChunkRefs := seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "4", "have_chunk_refs", "yes")}, + {lset: labels.FromStrings("series", "5", "have_chunk_refs", "yes")}, + {lset: labels.FromStrings("series", "6", "have_chunk_refs", "yes")}, + }, + releasable: true, + } + + iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + require.Equal(t, 0, psi.nextBatchPostingsOffset, "postings set iterator should be at beginning when creating iterator") + + if factoryCalls == 1 { + // Simulate the underlying iterator advancing the postings set iterator to the end, to ensure we get a fresh iterator next time. + for psi.Next() { + // Nothing to do, we just want to advance. + } + + return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs) + } + + return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithChunkRefs, secondBatchWithChunkRefs) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) + + // Inner iterator should be created with chunk refs disabled. + require.Equal(t, 1, factoryCalls) + require.Equal(t, defaultStrategy.withNoChunkRefs(), factoryStrategy) + require.NoError(t, it.Err()) + + // During label sending phase, the batches should be returned as-is. + batches := readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Equal(t, []seriesChunkRefsSet{firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs}, batches) + + // Prepare for chunks streaming phase. Inner iterator should be recreated with chunk refs enabled. + it.PrepareForChunksStreamingPhase() + require.Equal(t, 2, factoryCalls) + require.Equal(t, defaultStrategy.withChunkRefs(), factoryStrategy) + require.NoError(t, it.Err()) + + // During chunks streaming phase, the batches should be returned as-is from the new iterator. + batches = readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Equal(t, []seriesChunkRefsSet{firstBatchWithChunkRefs, secondBatchWithChunkRefs}, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_AllBatchesFilteredOut(t *testing.T) { + postings := []storage.SeriesRef{1, 2, 3, 4, 5, 6} + batchSize := 3 + factoryCalls := 0 + var factoryStrategy seriesIteratorStrategy + + iteratorFactory := func(strategy seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + return newSliceSeriesChunkRefsSetIterator( + nil, + // No batches. + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) + + // Inner iterator should be created with chunk refs disabled. + require.Equal(t, 1, factoryCalls) + require.Equal(t, defaultStrategy.withNoChunkRefs(), factoryStrategy) + require.NoError(t, it.Err()) + + // Label sending phase. + batches := readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Empty(t, batches) + + // Prepare for chunks streaming phase. Inner iterator should be recreated with chunk refs enabled. + it.PrepareForChunksStreamingPhase() + require.Equal(t, 2, factoryCalls) + require.Equal(t, defaultStrategy.withChunkRefs(), factoryStrategy) + require.NoError(t, it.Err()) + + // Chunks streaming phase. + batches = readAllSeriesChunkRefsSet(it) + require.NoError(t, it.Err()) + require.Empty(t, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_IteratorReturnsError(t *testing.T) { + postings := []storage.SeriesRef{1, 2, 3, 4, 5, 6} + batchSize := 3 + factoryCalls := 0 + + iteratorFactory := func(_ seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { + factoryCalls++ + + return newSliceSeriesChunkRefsSetIterator( + fmt.Errorf("error #%v", factoryCalls), + seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1")}, + {lset: labels.FromStrings("series", "2")}, + {lset: labels.FromStrings("series", "3")}, + }, + releasable: true, + }, + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) + require.Equal(t, 1, factoryCalls) + + // During label sending phase, the error from the original iterator should be returned. + _ = readAllSeriesChunkRefsSet(it) + require.EqualError(t, it.Err(), "error #1") + + // Prepare for chunks streaming phase. Inner iterator should be recreated. + it.PrepareForChunksStreamingPhase() + require.Equal(t, 2, factoryCalls) + + // During chunks streaming phase, the error from the new iterator should be returned. + _ = readAllSeriesChunkRefsSet(it) + require.EqualError(t, it.Err(), "error #2") +} diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index ee5070e6cfd..dad02bd2a2c 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1256,7 +1256,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { return []seriesChunkRefsSet{set} }(), }, - "works with many series in many batches batch": { + "works with many series in many batches": { blockFactory: largerTestBlockFactory, minT: 0, maxT: math.MaxInt64, @@ -1466,7 +1466,7 @@ func assertSeriesChunkRefsSetsEqual(t testing.TB, blockID ulid.ULID, blockDir st assert.True(t, uint64(prevChunkRef)+prevChunkLen <= uint64(promChunk.Ref), "estimated length shouldn't extend into the next chunk [%d, %d, %d]", i, j, k) assert.True(t, actualChunk.length <= uint32(tsdb.EstimatedMaxChunkSize), - "chunks can be larger than 16KB, but the estimted length should be capped to 16KB to limit the impact of bugs in estimations [%d, %d, %d]", i, j, k) + "chunks can be larger than 16KB, but the estimated length should be capped to 16KB to limit the impact of bugs in estimations [%d, %d, %d]", i, j, k) prevChunkRef, prevChunkLen = promChunk.Ref, uint64(actualChunk.length) } @@ -1703,8 +1703,8 @@ func TestOpenBlockSeriesChunkRefsSetsIterator(t *testing.T) { minT, maxT, newSafeQueryStats(), - nil, log.NewNopLogger(), + nil, ) require.NoError(t, err) @@ -1804,8 +1804,8 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_pendingMatchers(t *testing.T) { block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), - nil, log.NewNopLogger(), + nil, ) require.NoError(t, err) allSets := readAllSeriesChunkRefsSet(iterator) @@ -1868,8 +1868,8 @@ func BenchmarkOpenBlockSeriesChunkRefsSetsIterator(b *testing.B) { block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), - nil, log.NewNopLogger(), + nil, ) require.NoError(b, err) @@ -2197,8 +2197,8 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { b.meta.MinTime, b.meta.MaxTime, statsColdCache, - nil, log.NewNopLogger(), + nil, ) require.NoError(t, err) @@ -2213,7 +2213,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { cached: testCase.cachedSeriesHashesWithWarmCache, } - statsWarnCache := newSafeQueryStats() + statsWarmCache := newSafeQueryStats() ss, err = openBlockSeriesChunkRefsSetsIterator( context.Background(), batchSize, @@ -2227,15 +2227,15 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { noChunkRefs, b.meta.MinTime, b.meta.MaxTime, - statsWarnCache, - nil, + statsWarmCache, log.NewNopLogger(), + nil, ) require.NoError(t, err) lset = extractLabelsFromSeriesChunkRefsSets(readAllSeriesChunkRefsSet(ss)) require.NoError(t, ss.Err()) assert.Equal(t, testCase.expectedLabelSets, lset) - assert.Equal(t, testCase.expectedSeriesReadFromBlockWithWarmCache, statsWarnCache.export().seriesFetched) + assert.Equal(t, testCase.expectedSeriesReadFromBlockWithWarmCache, statsWarmCache.export().seriesFetched) }) } }) @@ -2291,7 +2291,7 @@ func TestPostingsSetsIterator(t *testing.T) { "empty postings": { postings: []storage.SeriesRef{}, batchSize: 2, - expectedBatches: [][]storage.SeriesRef{}, + expectedBatches: nil, }, } @@ -2305,51 +2305,11 @@ func TestPostingsSetsIterator(t *testing.T) { actualBatches = append(actualBatches, iterator.At()) } - assert.ElementsMatch(t, testCase.expectedBatches, actualBatches) + require.Equal(t, testCase.expectedBatches, actualBatches) }) } } -func TestReusedPostingsAndMatchers(t *testing.T) { - postingsList := [][]storage.SeriesRef{ - nil, - {}, - {1, 2, 3}, - } - matchersList := [][]*labels.Matcher{ - nil, - {}, - {labels.MustNewMatcher(labels.MatchEqual, "a", "b")}, - } - - for _, firstPostings := range postingsList { - for _, firstMatchers := range matchersList { - for _, secondPostings := range postingsList { - for _, secondMatchers := range matchersList { - r := reusedPostingsAndMatchers{} - require.False(t, r.isSet()) - - verify := func() { - r.set(firstPostings, firstMatchers) - require.True(t, r.isSet()) - if firstPostings == nil { - require.Equal(t, []storage.SeriesRef{}, r.ps) - } else { - require.Equal(t, firstPostings, r.ps) - } - require.Equal(t, firstMatchers, r.matchers) - } - verify() - - // This should not overwrite the first set. - r.set(secondPostings, secondMatchers) - verify() - } - } - } - } -} - type mockSeriesHasher struct { cached map[storage.SeriesRef]uint64 hashes map[string]uint64 @@ -2680,3 +2640,10 @@ type mockIndexCacheEntry struct { func (c mockIndexCache) FetchSeriesForPostings(context.Context, string, ulid.ULID, *sharding.ShardSelector, indexcache.PostingsKey) ([]byte, bool) { return c.fetchSeriesForPostingsResponse.contents, c.fetchSeriesForPostingsResponse.cached } + +func TestSeriesIteratorStrategy(t *testing.T) { + require.False(t, defaultStrategy.isNoChunkRefs()) + require.True(t, defaultStrategy.withNoChunkRefs().isNoChunkRefs()) + require.False(t, defaultStrategy.withNoChunkRefs().withChunkRefs().isNoChunkRefs()) + require.False(t, defaultStrategy.withChunkRefs().isNoChunkRefs()) +} From 80b9794c5d1e7b32daa6c628dfc0526d6afc2c8a Mon Sep 17 00:00:00 2001 From: Jon Kartago Lamida Date: Fri, 24 May 2024 16:39:31 +0800 Subject: [PATCH 3/7] Add documentation for helm memberlist cluster label migration (#7961) Signed-off-by: Jon Kartago Lamida Co-authored-by: Dimitar Dimitrov --- .../_index.md | 94 +++++++++++++++++++ 1 file changed, 94 insertions(+) create mode 100644 docs/sources/helm-charts/mimir-distributed/migration-guides/migrate-enable-cluster-label-verification/_index.md diff --git a/docs/sources/helm-charts/mimir-distributed/migration-guides/migrate-enable-cluster-label-verification/_index.md b/docs/sources/helm-charts/mimir-distributed/migration-guides/migrate-enable-cluster-label-verification/_index.md new file mode 100644 index 00000000000..fca09c00fe6 --- /dev/null +++ b/docs/sources/helm-charts/mimir-distributed/migration-guides/migrate-enable-cluster-label-verification/_index.md @@ -0,0 +1,94 @@ +--- +description: + Learn how to configure Helm installed Grafana Mimir's cluster label to prevent the Mimir components to join + different Memberlist cluster. +menuTitle: "Configure a unique Memberlist cluster label" +title: "Configure a unique Grafana Mimir's Memberlist cluster label in the mimir-distributed Helm chart installation" +weight: 110 +--- + +# Configure a unique Grafana Mimir's Memberlist cluster label in the mimir-distributed Helm chart installation + +This document shows the steps to configure cluster label verification in a Grafana Mimir installed by Helm. +Multiple [Memberlist](https://grafana.com/docs/mimir//references/architecture/memberlist-and-the-gossip-protocol/) [gossip ring](https://grafana.com/docs/mimir//references/architecture/hash-ring/) clusters are at risk of merging into one without enabling cluster label verification. +For example, if a Mimir, Tempo or Loki are running in the same Kubernetes cluster, they might communicate with each other without this configuration update. +Once cluster label verification is enabled, before Mimir components communicate with other components, they will verify whether the other components have the same cluster label. +The process to update the configuration will take three rollouts of the whole cluster. + +## Before you begin + +- You have a Grafana Mimir installed by mimir-distributed helm chart with its Memberlist cluster label still set to default value. +- You have `kubectl` and `helm` command line configured to connect to the Kubernetes cluster where your Grafana Mimir is running. + +## Configuration update steps + +There are three steps of the configuration update: + +1. Disable Memberlist cluster label verification +1. Set cluster label on all Mimir components +1. Enable Memberlist cluster label verification again + +### 1. Disable Memberlist cluster label verification + +Cluster label verification flag is enabled by default with cluster label set to an empty string. +Using the default value of cluster label can make different systems that use Memberlist communicate with each other if they also have not updated the default cluster label. +Setting a new cluster label directly to a non-empty string value without first disabling cluster label verification will cause Memberlist to form partition in the Grafana Mimir cluster. +The partition makes some Mimir components have different cluster label values which can prevent the component from communicating. +To disable cluster label verification flag, set the following structured config in mimir-distributed values.yaml configuration. + +```yaml +mimir: + structuredConfig: + memberlist: + cluster_label_verification_disabled: true +``` + +Rollout the installation to apply the configuration changes by running `helm upgrade mimir-distributed -f values.yaml`. +Replace `` with the actual Mimir release name. Wait until all Pods are ready before going to the next step. + +### 2. Set cluster label on all Mimir components + +Set cluster label on all Mimir components by setting the following configuration. +The configuration will set `cluster_label` to the Helm release name and the namespace where the helm release is installed. +Updating a new cluster label after disabling cluster label verification will prevent Memberlist from forming a partition. + +```yaml +mimir: + structuredConfig: + memberlist: + cluster_label_verification_disabled: true + cluster_label: "{{.Release.Name}}-{{.Release.Namespace}}" +``` + +Apply the configuration changes again by running `helm upgrade mimir-distributed -f values.yaml`. +Replace `` with the actual Mimir release name. Wait until all Pods are ready before going to the next step. + +### 3. Enable Memberlist cluster label verification + +Remove `mimir.structuredConfig.memberlist.cluster_label_verification_disabled` from the values.yaml file to re-enable Memberlist cluster label verification. + +```yaml +mimir: + structuredConfig: + memberlist: + cluster_label: "{{.Release.Name}}-{{.Release.Namespace}}" +``` + +Apply the configuration changes by running `helm upgrade mimir-distributed -f values.yaml`. +Replace `` with the actual Mimir release name. Wait until all Pods are ready before verifying that the configuration is applied correctly. + +## Verifying the configuration changes + +Once the rollout is completed, verify the change by looking at the `/memberlist` endpoint in some of Grafana Mimir pods. +Run the following port-forward command on several different Grafana Mimir components. + +```bash + kubectl port-forward pod/ --kube-context= --namespace= 8080:8080 + kubectl port-forward pod/ --kube-context= --namespace= 8081:8080 +``` + +Replace `` and `` with several actual pods from different Mimir components. +Ensure the host port 8080 and 8081 are available, otherwise use different available ports. + +Open the port-forwarded URL in browser to see the Memberlist status http://localhost:8080/memberlist, http://localhost:8081/memberlist and also +few others Grafana Mimir components. The Memberlist page from different pods must show same view of all of their members. From 487a5c94bb9013405b9ac07327e3a9573854c039 Mon Sep 17 00:00:00 2001 From: Dimitar Dimitrov Date: Fri, 24 May 2024 10:41:39 +0200 Subject: [PATCH 4/7] mixin dashbaords: add support for ingest storage replication (#8175) Signed-off-by: Dimitar Dimitrov --- .../metamonitoring/grafana-dashboards.yaml | 42 +++--- .../metamonitoring/mixin-alerts.yaml | 2 +- .../dashboards/mimir-tenants.json | 20 +-- .../dashboards/mimir-top-tenants.json | 8 +- .../dashboards/mimir-writes.json | 14 +- operations/mimir-mixin-compiled/alerts.yaml | 2 +- .../dashboards/mimir-tenants.json | 20 +-- .../dashboards/mimir-top-tenants.json | 8 +- .../dashboards/mimir-writes.json | 14 +- .../dashboards/dashboard-queries.libsonnet | 26 ++++ .../mimir-mixin/dashboards/tenants.libsonnet | 128 +++--------------- .../dashboards/top-tenants.libsonnet | 70 ++++------ .../mimir-mixin/dashboards/writes.libsonnet | 55 +++----- 13 files changed, 156 insertions(+), 253 deletions(-) diff --git a/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml b/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml index f3a034e9a07..7a467e405e0 100644 --- a/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml +++ b/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml @@ -32113,7 +32113,7 @@ data: "span": 4, "targets": [ { - "expr": "sum(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "in-memory", "legendLink": null @@ -32125,19 +32125,19 @@ data: "legendLink": null }, { - "expr": "sum(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active", "legendLink": null }, { - "expr": "sum(\n cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "owned", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active ({{ name }})", "legendLink": null @@ -32352,7 +32352,7 @@ data: "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "series", "legendLink": null @@ -32469,13 +32469,13 @@ data: "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active ({{ name }})", "legendLink": null @@ -32543,13 +32543,13 @@ data: "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "buckets", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "buckets ({{ name }})", "legendLink": null @@ -33235,7 +33235,7 @@ data: "span": 3, "targets": [ { - "expr": "sum(\n rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval])\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval]))\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval]),\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "rate", "legendLink": null @@ -34816,7 +34816,7 @@ data: ], "targets": [ { - "expr": "topk($limit,\n sum by (user) (\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n )\n)\n", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", "format": "table", "instant": true, "legendFormat": "", @@ -34957,7 +34957,7 @@ data: ], "targets": [ { - "expr": "topk($limit, sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} )\n)\n)", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)", "format": "table", "instant": true, "legendFormat": "", @@ -35052,7 +35052,7 @@ data: "span": 12, "targets": [ { - "expr": "sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} )\n)\n\nand\ntopk($limit, sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end())\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end())\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} @ end())\n)\n - sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start())\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start())\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} @ start())\n)\n)\n", + "expr": "(# Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\nand\ntopk($limit,\n (\n # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n\n )\n -\n (\n # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n\n )\n)\n", "format": "time_series", "legendFormat": "{{ user }}", "legendLink": null @@ -35560,7 +35560,7 @@ data: ], "targets": [ { - "expr": "topk($limit,\n sum by (user) (\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n )\n)\n", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", "format": "table", "instant": true, "legendFormat": "", @@ -38584,7 +38584,7 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", - "description": "### In-memory series\nThe number of series not yet flushed to object storage that are held in ingester memory.\n\n", + "description": "### In-memory series\nThe number of series not yet flushed to object storage that are held in ingester memory.\nWith classic storage we the sum of series from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum series of each ingest partition.\n\n", "fill": 1, "format": "short", "id": 4, @@ -38612,7 +38612,7 @@ data: "steppedLine": false, "targets": [ { - "expr": "sum(cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n/ on(cluster, namespace) group_left\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}))\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "instant": true, "refId": "A" @@ -38660,7 +38660,7 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", - "description": "### Exemplars in ingesters\nNumber of TSDB exemplars currently in ingesters' storage.\n\n", + "description": "### Exemplars in ingesters\nNumber of TSDB exemplars currently in ingesters' storage.\nWith classic storage we the sum of exemplars from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum exemplars of each ingest partition.\n\n", "fill": 1, "format": "short", "id": 5, @@ -38688,7 +38688,7 @@ data: "steppedLine": false, "targets": [ { - "expr": "sum(cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n/ on(cluster, namespace) group_left\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}))\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "instant": true, "refId": "A" @@ -41303,7 +41303,7 @@ data: }, { "datasource": "$datasource", - "description": "### Ingester ingested exemplars rate\nThe rate of exemplars ingested in the ingesters.\nEvery exemplar is sent to the replication factor number of ingesters, so the sum of rates from all ingesters is divided by the replication factor.\nThis ingested exemplars rate should match the distributor's received exemplars rate.\n\n", + "description": "### Ingester ingested exemplars rate\nThe rate of exemplars ingested in the ingesters.\nEvery exemplar is replicated to a number of ingesters. With classic storage we the sum of rates from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum rate of each ingest partition.\nThis ingested exemplars rate should match the distributor's received exemplars rate.\n\n", "fieldConfig": { "defaults": { "custom": { @@ -41341,7 +41341,7 @@ data: "span": 3, "targets": [ { - "expr": "sum(\n cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "ingested exemplars", "legendLink": null @@ -41390,7 +41390,7 @@ data: "span": 3, "targets": [ { - "expr": "sum(\n cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "appended exemplars", "legendLink": null diff --git a/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/mixin-alerts.yaml b/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/mixin-alerts.yaml index 6b915c8f751..c7aa2132f04 100644 --- a/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/mixin-alerts.yaml +++ b/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/mixin-alerts.yaml @@ -994,9 +994,9 @@ spec: runbook_url: https://grafana.com/docs/mimir/latest/operators-guide/mimir-runbooks/#mimiringesterstuckprocessingrecordsfromkafka expr: | # Alert if the reader is not processing any records, but there buffered records to process in the Kafka client. - # NOTE: the cortex_ingest_storage_reader_buffered_fetch_records_total metric is a gauge showing the current number of buffered records. (sum by (cluster, namespace, pod) (rate(cortex_ingest_storage_reader_records_total[5m])) == 0) and + # NOTE: the cortex_ingest_storage_reader_buffered_fetch_records_total metric is a gauge showing the current number of buffered records. (sum by (cluster, namespace, pod) (cortex_ingest_storage_reader_buffered_fetch_records_total) > 0) for: 5m labels: diff --git a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json index 257327ddffa..2802ae566f5 100644 --- a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json +++ b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json @@ -116,7 +116,7 @@ "span": 4, "targets": [ { - "expr": "sum(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n)\n,\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "in-memory", "legendLink": null @@ -128,19 +128,19 @@ "legendLink": null }, { - "expr": "sum(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active", "legendLink": null }, { - "expr": "sum(\n cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "owned", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active ({{ name }})", "legendLink": null @@ -355,7 +355,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "series", "legendLink": null @@ -472,13 +472,13 @@ "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active ({{ name }})", "legendLink": null @@ -546,13 +546,13 @@ "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "buckets", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "buckets ({{ name }})", "legendLink": null @@ -1238,7 +1238,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval])\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval]))\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval]),\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "rate", "legendLink": null diff --git a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json index 1d50197d70a..620e58249df 100644 --- a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json +++ b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json @@ -142,7 +142,7 @@ ], "targets": [ { - "expr": "topk($limit,\n sum by (user) (\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n )\n)\n", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", "format": "table", "instant": true, "legendFormat": "", @@ -283,7 +283,7 @@ ], "targets": [ { - "expr": "topk($limit, sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} )\n)\n)", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n)", "format": "table", "instant": true, "legendFormat": "", @@ -378,7 +378,7 @@ "span": 12, "targets": [ { - "expr": "sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} )\n)\n\nand\ntopk($limit, sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end())\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end())\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} @ end())\n)\n - sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start())\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start())\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} @ start())\n)\n)\n", + "expr": "(# Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n)\nand\ntopk($limit,\n (\n # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n)\n,\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n\n )\n -\n (\n # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n)\n,\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n\n )\n)\n", "format": "time_series", "legendFormat": "{{ user }}", "legendLink": null @@ -886,7 +886,7 @@ ], "targets": [ { - "expr": "topk($limit,\n sum by (user) (\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n )\n)\n", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", "format": "table", "instant": true, "legendFormat": "", diff --git a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-writes.json b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-writes.json index 9d36d861030..2dd57015e5d 100644 --- a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-writes.json +++ b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-writes.json @@ -214,7 +214,7 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "description": "### In-memory series\nThe number of series not yet flushed to object storage that are held in ingester memory.\n\n", + "description": "### In-memory series\nThe number of series not yet flushed to object storage that are held in ingester memory.\nWith classic storage we the sum of series from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum series of each ingest partition.\n\n", "fill": 1, "format": "short", "id": 4, @@ -242,7 +242,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n/ on(cluster, namespace) group_left\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}))\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "instant": true, "refId": "A" @@ -290,7 +290,7 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "description": "### Exemplars in ingesters\nNumber of TSDB exemplars currently in ingesters' storage.\n\n", + "description": "### Exemplars in ingesters\nNumber of TSDB exemplars currently in ingesters' storage.\nWith classic storage we the sum of exemplars from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum exemplars of each ingest partition.\n\n", "fill": 1, "format": "short", "id": 5, @@ -318,7 +318,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n/ on(cluster, namespace) group_left\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}))\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "instant": true, "refId": "A" @@ -2933,7 +2933,7 @@ }, { "datasource": "$datasource", - "description": "### Ingester ingested exemplars rate\nThe rate of exemplars ingested in the ingesters.\nEvery exemplar is sent to the replication factor number of ingesters, so the sum of rates from all ingesters is divided by the replication factor.\nThis ingested exemplars rate should match the distributor's received exemplars rate.\n\n", + "description": "### Ingester ingested exemplars rate\nThe rate of exemplars ingested in the ingesters.\nEvery exemplar is replicated to a number of ingesters. With classic storage we the sum of rates from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum rate of each ingest partition.\nThis ingested exemplars rate should match the distributor's received exemplars rate.\n\n", "fieldConfig": { "defaults": { "custom": { @@ -2971,7 +2971,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "ingested exemplars", "legendLink": null @@ -3020,7 +3020,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "appended exemplars", "legendLink": null diff --git a/operations/mimir-mixin-compiled/alerts.yaml b/operations/mimir-mixin-compiled/alerts.yaml index c7c5baf542f..4524457921c 100644 --- a/operations/mimir-mixin-compiled/alerts.yaml +++ b/operations/mimir-mixin-compiled/alerts.yaml @@ -982,9 +982,9 @@ groups: runbook_url: https://grafana.com/docs/mimir/latest/operators-guide/mimir-runbooks/#mimiringesterstuckprocessingrecordsfromkafka expr: | # Alert if the reader is not processing any records, but there buffered records to process in the Kafka client. - # NOTE: the cortex_ingest_storage_reader_buffered_fetch_records_total metric is a gauge showing the current number of buffered records. (sum by (cluster, namespace, pod) (rate(cortex_ingest_storage_reader_records_total[5m])) == 0) and + # NOTE: the cortex_ingest_storage_reader_buffered_fetch_records_total metric is a gauge showing the current number of buffered records. (sum by (cluster, namespace, pod) (cortex_ingest_storage_reader_buffered_fetch_records_total) > 0) for: 5m labels: diff --git a/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json b/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json index 257327ddffa..ddbb1acd3a5 100644 --- a/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json +++ b/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json @@ -116,7 +116,7 @@ "span": 4, "targets": [ { - "expr": "sum(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n - cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "in-memory", "legendLink": null @@ -128,19 +128,19 @@ "legendLink": null }, { - "expr": "sum(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active", "legendLink": null }, { - "expr": "sum(\n cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_owned_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "owned", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active ({{ name }})", "legendLink": null @@ -355,7 +355,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "series", "legendLink": null @@ -472,13 +472,13 @@ "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_native_histogram_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_native_histogram_series_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "active ({{ name }})", "legendLink": null @@ -546,13 +546,13 @@ "span": 3, "targets": [ { - "expr": "sum(\n cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_active_native_histogram_buckets{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "buckets", "legendLink": null }, { - "expr": "sum by (name) (\n cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n) > 0\n", + "expr": "# Classic storage\nsum by (cluster, namespace, name) (cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, name) (\n max by (ingester_id, cluster, namespace, name) (\n label_replace(\n cortex_ingester_active_native_histogram_buckets_custom_tracker{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "buckets ({{ name }})", "legendLink": null @@ -1238,7 +1238,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval])\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval]))\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\", user=\"$user\"}[$__rate_interval]),\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "rate", "legendLink": null diff --git a/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json b/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json index 1d50197d70a..d7f89b03bbc 100644 --- a/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json +++ b/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json @@ -142,7 +142,7 @@ ], "targets": [ { - "expr": "topk($limit,\n sum by (user) (\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n )\n)\n", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", "format": "table", "instant": true, "legendFormat": "", @@ -283,7 +283,7 @@ ], "targets": [ { - "expr": "topk($limit, sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} )\n)\n)", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)", "format": "table", "instant": true, "legendFormat": "", @@ -378,7 +378,7 @@ "span": 12, "targets": [ { - "expr": "sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} )\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} )\n)\n\nand\ntopk($limit, sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end())\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end())\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} @ end())\n)\n - sum by (user) (\n (\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start())\n -\n sum by (user, cluster, namespace) (cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start())\n )\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"} @ start())\n)\n)\n", + "expr": "(# Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\nand\ntopk($limit,\n (\n # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ end()\n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n\n )\n -\n (\n # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} @ start()\n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n\n )\n)\n", "format": "time_series", "legendFormat": "{{ user }}", "legendLink": null @@ -886,7 +886,7 @@ ], "targets": [ { - "expr": "topk($limit,\n sum by (user) (\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n )\n)\n", + "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", "format": "table", "instant": true, "legendFormat": "", diff --git a/operations/mimir-mixin-compiled/dashboards/mimir-writes.json b/operations/mimir-mixin-compiled/dashboards/mimir-writes.json index 756e3018875..ab465dba610 100644 --- a/operations/mimir-mixin-compiled/dashboards/mimir-writes.json +++ b/operations/mimir-mixin-compiled/dashboards/mimir-writes.json @@ -214,7 +214,7 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "description": "### In-memory series\nThe number of series not yet flushed to object storage that are held in ingester memory.\n\n", + "description": "### In-memory series\nThe number of series not yet flushed to object storage that are held in ingester memory.\nWith classic storage we the sum of series from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum series of each ingest partition.\n\n", "fill": 1, "format": "short", "id": 4, @@ -242,7 +242,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n/ on(cluster, namespace) group_left\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}))\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_memory_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "instant": true, "refId": "A" @@ -290,7 +290,7 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "description": "### Exemplars in ingesters\nNumber of TSDB exemplars currently in ingesters' storage.\n\n", + "description": "### Exemplars in ingesters\nNumber of TSDB exemplars currently in ingesters' storage.\nWith classic storage we the sum of exemplars from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum exemplars of each ingest partition.\n\n", "fill": 1, "format": "short", "id": 5, @@ -318,7 +318,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n/ on(cluster, namespace) group_left\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}))\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cortex_ingester_tsdb_exemplar_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "instant": true, "refId": "A" @@ -2933,7 +2933,7 @@ }, { "datasource": "$datasource", - "description": "### Ingester ingested exemplars rate\nThe rate of exemplars ingested in the ingesters.\nEvery exemplar is sent to the replication factor number of ingesters, so the sum of rates from all ingesters is divided by the replication factor.\nThis ingested exemplars rate should match the distributor's received exemplars rate.\n\n", + "description": "### Ingester ingested exemplars rate\nThe rate of exemplars ingested in the ingesters.\nEvery exemplar is replicated to a number of ingesters. With classic storage we the sum of rates from all ingesters is divided by the replication factor.\nWith ingest storage we take the maximum rate of each ingest partition.\nThis ingested exemplars rate should match the distributor's received exemplars rate.\n\n", "fieldConfig": { "defaults": { "custom": { @@ -2971,7 +2971,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cluster_namespace_job:cortex_ingester_ingested_exemplars:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "ingested exemplars", "legendLink": null @@ -3020,7 +3020,7 @@ "span": 3, "targets": [ { - "expr": "sum(\n cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"}\n / on(cluster, namespace) group_left\n max by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\n)\n", + "expr": "# Classic storage\nsum by (cluster, namespace, ) (cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, ) (\n max by (ingester_id, cluster, namespace, ) (\n label_replace(\n cluster_namespace_job:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n", "format": "time_series", "legendFormat": "appended exemplars", "legendLink": null diff --git a/operations/mimir-mixin/dashboards/dashboard-queries.libsonnet b/operations/mimir-mixin/dashboards/dashboard-queries.libsonnet index 70bbeb09c4a..dcd2a4289ec 100644 --- a/operations/mimir-mixin/dashboards/dashboard-queries.libsonnet +++ b/operations/mimir-mixin/dashboards/dashboard-queries.libsonnet @@ -43,6 +43,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; perClusterLabel: $._config.per_cluster_label, recordingRulePrefix: $.recordingRulePrefix($.jobSelector('any')), // The job name does not matter here. groupPrefixJobs: $._config.group_prefix_jobs, + instance: $._config.per_instance_label, }, write_http_routes_regex: 'api_(v1|prom)_push|otlp_v1_metrics', @@ -233,5 +234,30 @@ local utils = import 'mixin-utils/utils.libsonnet'; sum(rate(thanos_objstore_bucket_operations_total{%(namespaceMatcher)s}[$__rate_interval])) ||| % variables, }, + + ingester: { + ingestOrClassicDeduplicatedQuery(perIngesterQuery, groupByLabels=''):: ||| + # Classic storage + sum by (%(groupByCluster)s, %(groupByLabels)s) (%(perIngesterQuery)s) + / on (%(groupByCluster)s) group_left() + max by (%(groupByCluster)s) (cortex_distributor_replication_factor{%(distributor)s}) + or on (%(groupByCluster)s) + # Ingest storage + sum by (%(groupByCluster)s, %(groupByLabels)s) ( + max by (ingester_id, %(groupByCluster)s, %(groupByLabels)s) ( + label_replace( + %(perIngesterQuery)s, + "ingester_id", "$1", "%(instance)s", ".*-([0-9]+)$" + ) + ) + ) + ||| % { + perIngesterQuery: perIngesterQuery, + instance: variables.instance, + groupByLabels: groupByLabels, + groupByCluster: $._config.group_by_cluster, + distributor: variables.distributorMatcher, + }, + }, }, } diff --git a/operations/mimir-mixin/dashboards/tenants.libsonnet b/operations/mimir-mixin/dashboards/tenants.libsonnet index c50005fe109..0e233cfc7c2 100644 --- a/operations/mimir-mixin/dashboards/tenants.libsonnet +++ b/operations/mimir-mixin/dashboards/tenants.libsonnet @@ -1,7 +1,8 @@ local utils = import 'mixin-utils/utils.libsonnet'; local filename = 'mimir-tenants.json'; -(import 'dashboard-utils.libsonnet') { +(import 'dashboard-utils.libsonnet') + +(import 'dashboard-queries.libsonnet') { local user_limits_overrides_query(limit_name) = ||| max(cortex_limits_overrides{%(overrides_exporter)s, limit_name="%(limit_name)s", user="$user"}) or @@ -41,54 +42,19 @@ local filename = 'mimir-tenants.json'; $.timeseriesPanel(title) + $.queryPanel( [ - ||| - sum( - ( - cortex_ingester_memory_series_created_total{%(ingester)s, user="$user"} - - cortex_ingester_memory_series_removed_total{%(ingester)s, user="$user"} - ) - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) + local perIngesterInMemorySeries = ||| + ( + cortex_ingester_memory_series_created_total{%(ingester)s, user="$user"} + - cortex_ingester_memory_series_removed_total{%(ingester)s, user="$user"} ) ||| % { ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + }; + $.queries.ingester.ingestOrClassicDeduplicatedQuery(perIngesterInMemorySeries), user_limits_overrides_query('max_global_series_per_user'), - ||| - sum( - cortex_ingester_active_series{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, - ||| - sum( - cortex_ingester_owned_series{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, - ||| - sum by (name) ( - cortex_ingester_active_series_custom_tracker{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) > 0 - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_active_series{%s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)]), + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_owned_series{%s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)]), + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_active_series_custom_tracker{%s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)], groupByLabels='name'), ], [ 'in-memory', @@ -213,17 +179,7 @@ local filename = 'mimir-tenants.json'; local title = 'Series with exemplars'; $.timeseriesPanel(title) + $.queryPanel( - ||| - sum( - cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{%(ingester)s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)]), 'series', ) + { options+: { legend+: { showLegend: false } } } + @@ -261,28 +217,8 @@ local filename = 'mimir-tenants.json'; $.timeseriesPanel(title) + $.queryPanel( [ - ||| - sum( - cortex_ingester_active_native_histogram_series{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, - ||| - sum by (name) ( - cortex_ingester_active_native_histogram_series_custom_tracker{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) > 0 - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_active_native_histogram_series{%(ingester)s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)]), + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_active_native_histogram_series_custom_tracker{%(ingester)s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)], groupByLabels='name'), ], [ 'active', @@ -303,28 +239,8 @@ local filename = 'mimir-tenants.json'; $.timeseriesPanel(title) + $.queryPanel( [ - ||| - sum( - cortex_ingester_active_native_histogram_buckets{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, - ||| - sum by (name) ( - cortex_ingester_active_native_histogram_buckets_custom_tracker{%(ingester)s, user="$user"} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) > 0 - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_active_native_histogram_buckets{%(ingester)s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)]), + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_active_native_histogram_buckets_custom_tracker{%(ingester)s, user="$user"}' % [$.jobMatcher($._config.job_names.ingester)], groupByLabels='name'), ], [ 'buckets', @@ -562,17 +478,7 @@ local filename = 'mimir-tenants.json'; local title = 'Ingester appended exemplars rate'; $.timeseriesPanel(title) + $.queryPanel( - ||| - sum( - rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{%(ingester)s, user="$user"}[$__rate_interval]) - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + $.queries.ingester.ingestOrClassicDeduplicatedQuery('rate(cortex_ingester_tsdb_exemplar_exemplars_appended_total{%(ingester)s, user="$user"}[$__rate_interval])' % [$.jobMatcher($._config.job_names.ingester)]), 'rate', ) + { options+: { legend+: { showLegend: false } } } + diff --git a/operations/mimir-mixin/dashboards/top-tenants.libsonnet b/operations/mimir-mixin/dashboards/top-tenants.libsonnet index 96eab666278..676e0de394d 100644 --- a/operations/mimir-mixin/dashboards/top-tenants.libsonnet +++ b/operations/mimir-mixin/dashboards/top-tenants.libsonnet @@ -1,23 +1,21 @@ local utils = import 'mixin-utils/utils.libsonnet'; local filename = 'mimir-top-tenants.json'; -(import 'dashboard-utils.libsonnet') { - local in_memory_series_per_user_query(at='') = ||| - sum by (user) ( +(import 'dashboard-utils.libsonnet') + +(import 'dashboard-queries.libsonnet') { + local in_memory_series_per_user_query(at='') = ( + local perIngesterQuery = ||| ( - sum by (user, %(group_by_cluster)s) (cortex_ingester_memory_series_created_total{%(ingester)s} %(at)s) + cortex_ingester_memory_series_created_total{%(ingester)s} %(at)s - - sum by (user, %(group_by_cluster)s) (cortex_ingester_memory_series_removed_total{%(ingester)s} %(at)s) + cortex_ingester_memory_series_removed_total{%(ingester)s} %(at)s ) - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s} %(at)s) - ) - ||| % { - at: at, - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + ||| % { + at: at, + ingester: $.jobMatcher($._config.job_names.ingester), + }; + $.queries.ingester.ingestOrClassicDeduplicatedQuery(perIngesterQuery, groupByLabels='user') + ), [filename]: assert std.md5(filename) == 'bc6e12d4fe540e4a1785b9d3ca0ffdd9' : 'UID of the dashboard has changed, please update references to dashboard.'; @@ -46,18 +44,10 @@ local filename = 'mimir-top-tenants.json'; $.tablePanel( [ ||| - topk($limit, - sum by (user) ( - cortex_ingester_active_series{%(ingester)s} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + topk($limit, %s) + ||| % [ + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_active_series{%s}' % [$.jobMatcher($._config.job_names.ingester)], groupByLabels='user'), + ], ], { user: { alias: 'user', unit: 'string' }, Value: { alias: 'series' }, @@ -89,9 +79,17 @@ local filename = 'mimir-top-tenants.json'; $.timeseriesPanel(title) + $.queryPanel( ||| - %(in_memory_series_per_user)s + (%(in_memory_series_per_user)s) and - topk($limit, %(in_memory_series_per_user_at_end)s - %(in_memory_series_per_user_at_start)s) + topk($limit, + ( + %(in_memory_series_per_user_at_end)s + ) + - + ( + %(in_memory_series_per_user_at_start)s + ) + ) ||| % { in_memory_series_per_user: in_memory_series_per_user_query(), in_memory_series_per_user_at_end: in_memory_series_per_user_query(at='@ end()'), @@ -186,18 +184,10 @@ local filename = 'mimir-top-tenants.json'; $.tablePanel( [ ||| - topk($limit, - sum by (user) ( - cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{%(ingester)s} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ) - ||| % { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, - }, + topk($limit, %s) + ||| % [ + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{%s}' % [$.jobMatcher($._config.job_names.ingester)], groupByLabels='user'), + ], ], { user: { alias: 'user', unit: 'string' }, Value: { alias: 'series' }, diff --git a/operations/mimir-mixin/dashboards/writes.libsonnet b/operations/mimir-mixin/dashboards/writes.libsonnet index 42e17041010..7e20cbd0269 100644 --- a/operations/mimir-mixin/dashboards/writes.libsonnet +++ b/operations/mimir-mixin/dashboards/writes.libsonnet @@ -61,36 +61,32 @@ local filename = 'mimir-writes.json'; .addPanel( local title = 'In-memory series'; $.panel(title) + - $.statPanel(||| - sum(cortex_ingester_memory_series{%(ingester)s} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s})) - ||| % ($._config) { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - }, format='short') + + $.statPanel( + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_memory_series{%s}' % [$.jobMatcher($._config.job_names.ingester)]), + format='short' + ) + $.panelDescription( title, ||| The number of series not yet flushed to object storage that are held in ingester memory. + With classic storage we the sum of series from all ingesters is divided by the replication factor. + With ingest storage we take the maximum series of each ingest partition. ||| ), ) .addPanel( local title = 'Exemplars in ingesters'; $.panel(title) + - $.statPanel(||| - sum(cortex_ingester_tsdb_exemplar_exemplars_in_storage{%(ingester)s} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s})) - ||| % ($._config) { - ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - }, format='short') + + $.statPanel( + $.queries.ingester.ingestOrClassicDeduplicatedQuery('cortex_ingester_tsdb_exemplar_exemplars_in_storage{%s}' % [$.jobMatcher($._config.job_names.ingester)]), + format='short' + ) + $.panelDescription( title, ||| Number of TSDB exemplars currently in ingesters' storage. + With classic storage we the sum of exemplars from all ingesters is divided by the replication factor. + With ingest storage we take the maximum exemplars of each ingest partition. ||| ), ) @@ -649,18 +645,10 @@ local filename = 'mimir-writes.json'; local title = 'Ingester ingested exemplars rate'; $.timeseriesPanel(title) + $.queryPanel( - ||| - sum( - %(group_prefix_jobs)s:cortex_ingester_ingested_exemplars:rate5m{%(ingester)s} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { + $.queries.ingester.ingestOrClassicDeduplicatedQuery('%(group_prefix_jobs)s:cortex_ingester_ingested_exemplars:rate5m{%(ingester)s}' % { ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, group_prefix_jobs: $._config.group_prefix_jobs, - }, + }), 'ingested exemplars', ) + { fieldConfig+: { defaults+: { unit: 'ex/s' } } } + @@ -668,7 +656,8 @@ local filename = 'mimir-writes.json'; title, ||| The rate of exemplars ingested in the ingesters. - Every exemplar is sent to the replication factor number of ingesters, so the sum of rates from all ingesters is divided by the replication factor. + Every exemplar is replicated to a number of ingesters. With classic storage we the sum of rates from all ingesters is divided by the replication factor. + With ingest storage we take the maximum rate of each ingest partition. This ingested exemplars rate should match the distributor's received exemplars rate. ||| ), @@ -677,18 +666,10 @@ local filename = 'mimir-writes.json'; local title = 'Ingester appended exemplars rate'; $.timeseriesPanel(title) + $.queryPanel( - ||| - sum( - %(group_prefix_jobs)s:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{%(ingester)s} - / on(%(group_by_cluster)s) group_left - max by (%(group_by_cluster)s) (cortex_distributor_replication_factor{%(distributor)s}) - ) - ||| % { + $.queries.ingester.ingestOrClassicDeduplicatedQuery('%(group_prefix_jobs)s:cortex_ingester_tsdb_exemplar_exemplars_appended:rate5m{%(ingester)s}' % { ingester: $.jobMatcher($._config.job_names.ingester), - distributor: $.jobMatcher($._config.job_names.distributor), - group_by_cluster: $._config.group_by_cluster, group_prefix_jobs: $._config.group_prefix_jobs, - }, + }), 'appended exemplars', ) + { fieldConfig+: { defaults+: { unit: 'ex/s' } } } + From 8727c7f975c08d603f8d1a5e16e52c81b73cd117 Mon Sep 17 00:00:00 2001 From: Dimitar Dimitrov Date: Fri, 24 May 2024 14:30:31 +0200 Subject: [PATCH 5/7] ingest storage: record latency in replaying records (#8176) Signed-off-by: Dimitar Dimitrov --- pkg/storage/ingest/reader.go | 23 ++++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/pkg/storage/ingest/reader.go b/pkg/storage/ingest/reader.go index df15ba35669..9eb02efd166 100644 --- a/pkg/storage/ingest/reader.go +++ b/pkg/storage/ingest/reader.go @@ -173,7 +173,7 @@ func (r *PartitionReader) run(ctx context.Context) error { } func (r *PartitionReader) processNextFetches(ctx context.Context, delayObserver prometheus.Observer) { - fetches := r.client.PollFetches(ctx) + fetches := r.pollFetches(ctx) r.recordFetchesMetrics(fetches, delayObserver) r.logFetchErrors(fetches) fetches = filterOutErrFetches(fetches) @@ -336,7 +336,9 @@ func (r *PartitionReader) consumeFetches(ctx context.Context, fetches kgo.Fetche }) for boff.Ongoing() { + consumeStart := time.Now() err := r.consumer.consume(ctx, records) + r.metrics.consumeLatency.Observe(time.Since(consumeStart).Seconds()) if err == nil { break } @@ -571,6 +573,13 @@ func (r *PartitionReader) WaitReadConsistency(ctx context.Context) (returnErr er return r.consumedOffsetWatcher.Wait(ctx, lastProducedOffset) } +func (r *PartitionReader) pollFetches(ctx context.Context) kgo.Fetches { + defer func(start time.Time) { + r.metrics.fetchWaitDuration.Observe(time.Since(start).Seconds()) + }(time.Now()) + return r.client.PollFetches(ctx) +} + type partitionCommitter struct { services.Service @@ -707,10 +716,12 @@ type readerMetrics struct { recordsPerFetch prometheus.Histogram fetchesErrors prometheus.Counter fetchesTotal prometheus.Counter + fetchWaitDuration prometheus.Histogram strongConsistencyRequests prometheus.Counter strongConsistencyFailures prometheus.Counter strongConsistencyLatency prometheus.Histogram lastConsumedOffset prometheus.Gauge + consumeLatency prometheus.Histogram kprom *kprom.Metrics } @@ -750,6 +761,16 @@ func newReaderMetrics(partitionID int32, reg prometheus.Registerer) readerMetric Name: "cortex_ingest_storage_reader_fetches_total", Help: "Total number of Kafka fetches received by the consumer.", }), + fetchWaitDuration: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "cortex_ingest_storage_reader_fetch_wait_duration_seconds", + Help: "How long fetching a batch of records from the kafka client took to complete.", + NativeHistogramBucketFactor: 1.1, + }), + consumeLatency: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "cortex_ingest_storage_reader_consume_duration_seconds", + Help: "How long a request spent consuming a record batch from Kafka.", + NativeHistogramBucketFactor: 1.1, + }), strongConsistencyRequests: promauto.With(reg).NewCounter(prometheus.CounterOpts{ Name: "cortex_ingest_storage_strong_consistency_requests_total", Help: "Total number of requests for which strong consistency has been requested.", From 5f01872dcbcbec600ac180b06347ada231a483b1 Mon Sep 17 00:00:00 2001 From: Vladimir Varankin Date: Fri, 24 May 2024 17:05:47 +0200 Subject: [PATCH 6/7] mimir-mixin: update dashboards to not use deprecated "Table (old)" plugin (#8181) * mimir-mixin: update tablePanel from use of deprecated plugin Signed-off-by: Vladimir Varankin * rebuild assets Signed-off-by: Vladimir Varankin * update changelog Signed-off-by: Vladimir Varankin * mimir-mixin-tools: improve run script to serve with Grafana 11 Signed-off-by: Vladimir Varankin --------- Signed-off-by: Vladimir Varankin --- CHANGELOG.md | 5 +- .../metamonitoring/grafana-dashboards.yaml | 1378 +++++++++-------- .../dashboards/mimir-scaling.json | 249 +-- .../dashboards/mimir-tenants.json | 148 +- .../dashboards/mimir-top-tenants.json | 981 ++++++------ .../dashboards/mimir-scaling.json | 249 +-- .../dashboards/mimir-tenants.json | 148 +- .../dashboards/mimir-top-tenants.json | 981 ++++++------ operations/mimir-mixin-tools/serve/run.sh | 9 +- .../dashboards/dashboard-utils.libsonnet | 29 + .../mimir-mixin/dashboards/scaling.libsonnet | 2 +- 11 files changed, 2318 insertions(+), 1861 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 53e24cc3ebf..901b19705f3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -94,8 +94,9 @@ * Overview dashboard, Status panel, `cortex_request_duration_seconds` metric. * [ENHANCEMENT] Alerts: exclude `529` and `598` status codes from failure codes in `MimirRequestsError`. #7889 * [ENHANCEMENT] Dashboards: renamed "TCP Connections" panel to "Ingress TCP Connections" in the networking dashboards. #8092 -* [BUGFIX] Dashboards: Fix regular expression for matching read-path gRPC ingester methods to include querying of exemplars, label-related queries, or active series queries. #7676 -* [BUGFIX] Dashboards: Fix user id abbreviations and column heads for Top Tenants dashboard. #7724 +* [ENHANCEMENT] Dashboards: update the use of deprecated "table (old)" panels to "table". #8181 +* [BUGFIX] Dashboards: fix regular expression for matching read-path gRPC ingester methods to include querying of exemplars, label-related queries, or active series queries. #7676 +* [BUGFIX] Dashboards: fix user id abbreviations and column heads for Top Tenants dashboard. #7724 * [BUGFIX] Dashboards: fix incorrect query used for "queue length" panel on "Ruler" dashboard. #8006 ### Jsonnet diff --git a/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml b/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml index 7a467e405e0..36948547a38 100644 --- a/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml +++ b/operations/helm/tests/metamonitoring-values-generated/mimir-distributed/templates/metamonitoring/grafana-dashboards.yaml @@ -30203,6 +30203,146 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "Required Replicas" + }, + { + "id": "decimals", + "value": 0 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "__name__" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "cluster" + }, + "properties": [ + { + "id": "displayName", + "value": "Cluster" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "deployment" + }, + "properties": [ + { + "id": "displayName", + "value": "Service" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "namespace" + }, + "properties": [ + { + "id": "displayName", + "value": "Namespace" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "reason" + }, + "properties": [ + { + "id": "displayName", + "value": "Reason" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 2, "legend": { @@ -30231,115 +30371,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "Required Replicas", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 0, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Cluster", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "__name__", - "thresholds": [ ], - "type": "hidden", - "unit": "short" - }, - { - "alias": "Cluster", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "cluster", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Service", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "deployment", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Namespace", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "namespace", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Reason", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "reason", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "sort_desc(\n cluster_namespace_deployment_reason:required_replicas:count{cluster=~\"$cluster\", namespace=~\"$namespace\"}\n > ignoring(reason) group_left\n cluster_namespace_deployment:actual_replicas:count{cluster=~\"$cluster\", namespace=~\"$namespace\"}\n)\n", @@ -33603,6 +33634,46 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value #A" + }, + "properties": [ + { + "id": "displayName", + "value": "rules" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 27, "legend": { @@ -33631,40 +33702,6 @@ data: "span": 6, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "rules", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value #A", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group) (cortex_prometheus_rule_group_rules{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\", user=\"$user\"}))", @@ -33717,6 +33754,46 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value #A" + }, + "properties": [ + { + "id": "displayName", + "value": "seconds" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 28, "legend": { @@ -33745,40 +33822,6 @@ data: "span": 6, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "seconds", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value #A", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group) (cortex_prometheus_rule_group_last_duration_seconds{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\", user=\"$user\"}))", @@ -34737,6 +34780,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 2, "legend": { @@ -34765,55 +34868,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", @@ -34878,6 +34932,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 3, "legend": { @@ -34906,55 +35020,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)", @@ -35079,6 +35144,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "samples/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 5, "legend": { @@ -35107,55 +35232,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "samples/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_distributor_received_samples_total{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -35280,6 +35356,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "samples/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 7, "legend": { @@ -35308,55 +35444,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "samples/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_discarded_samples_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*|distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -35481,6 +35568,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 9, "legend": { @@ -35509,55 +35656,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", @@ -35622,6 +35720,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "exemplars/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 10, "legend": { @@ -35650,55 +35808,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "exemplars/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_distributor_received_exemplars_total{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -35763,6 +35872,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "rules" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 11, "legend": { @@ -35791,55 +35960,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "rules", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group, user) (cortex_prometheus_rule_group_rules{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\"}))", @@ -35904,6 +36024,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "seconds" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 12, "legend": { @@ -35932,55 +36112,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "seconds", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group, user) (cortex_prometheus_rule_group_last_duration_seconds{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\"}))", @@ -36045,6 +36176,66 @@ data: "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "Compaction Jobs" + }, + { + "id": "decimals", + "value": 0 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 13, "legend": { @@ -36073,55 +36264,6 @@ data: "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "Compaction Jobs", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 0, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit,\n sum by (user) (cortex_bucket_index_estimated_compaction_jobs{cluster=~\"$cluster\", job=~\"($namespace)/((compactor.*|cortex|mimir|mimir-backend.*))\"})\n and ignoring(user)\n (sum(rate(cortex_bucket_index_estimated_compaction_jobs_errors_total{cluster=~\"$cluster\", job=~\"($namespace)/((compactor.*|cortex|mimir|mimir-backend.*))\"}[$__rate_interval])) == 0)\n)\n", diff --git a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-scaling.json b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-scaling.json index b7b618287a4..22b8f38b947 100644 --- a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-scaling.json +++ b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-scaling.json @@ -62,6 +62,146 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "Required Replicas" + }, + { + "id": "decimals", + "value": 0 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "__name__" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "cluster" + }, + "properties": [ + { + "id": "displayName", + "value": "Cluster" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "deployment" + }, + "properties": [ + { + "id": "displayName", + "value": "Service" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "namespace" + }, + "properties": [ + { + "id": "displayName", + "value": "Namespace" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "reason" + }, + "properties": [ + { + "id": "displayName", + "value": "Reason" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 2, "legend": { @@ -90,115 +230,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "Required Replicas", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 0, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Cluster", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "__name__", - "thresholds": [ ], - "type": "hidden", - "unit": "short" - }, - { - "alias": "Cluster", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "cluster", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Service", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "deployment", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Namespace", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "namespace", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Reason", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "reason", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "sort_desc(\n cluster_namespace_deployment_reason:required_replicas:count{cluster=~\"$cluster\", namespace=~\"$namespace\"}\n > ignoring(reason) group_left\n cluster_namespace_deployment:actual_replicas:count{cluster=~\"$cluster\", namespace=~\"$namespace\"}\n)\n", diff --git a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json index 2802ae566f5..7baa6f30944 100644 --- a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json +++ b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-tenants.json @@ -1606,6 +1606,46 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value #A" + }, + "properties": [ + { + "id": "displayName", + "value": "rules" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 27, "legend": { @@ -1634,40 +1674,6 @@ "span": 6, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "rules", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value #A", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group) (cortex_prometheus_rule_group_rules{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\", user=\"$user\"}))", @@ -1720,6 +1726,46 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value #A" + }, + "properties": [ + { + "id": "displayName", + "value": "seconds" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 28, "legend": { @@ -1748,40 +1794,6 @@ "span": 6, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "seconds", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value #A", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group) (cortex_prometheus_rule_group_last_duration_seconds{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\", user=\"$user\"}))", diff --git a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json index 620e58249df..9ef468344d0 100644 --- a/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json +++ b/operations/mimir-mixin-compiled-baremetal/dashboards/mimir-top-tenants.json @@ -63,6 +63,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 2, "legend": { @@ -91,55 +151,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", @@ -204,6 +215,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 3, "legend": { @@ -232,55 +303,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n)", @@ -405,6 +427,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "samples/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 5, "legend": { @@ -433,55 +515,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "samples/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_distributor_received_samples_total{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -606,6 +639,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "samples/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 7, "legend": { @@ -634,55 +727,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "samples/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_discarded_samples_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*|distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -807,6 +851,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 9, "legend": { @@ -835,55 +939,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"instance\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", @@ -948,6 +1003,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "exemplars/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 10, "legend": { @@ -976,55 +1091,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "exemplars/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_distributor_received_exemplars_total{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -1089,6 +1155,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "rules" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 11, "legend": { @@ -1117,55 +1243,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "rules", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group, user) (cortex_prometheus_rule_group_rules{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\"}))", @@ -1230,6 +1307,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "seconds" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 12, "legend": { @@ -1258,55 +1395,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "seconds", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group, user) (cortex_prometheus_rule_group_last_duration_seconds{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\"}))", @@ -1371,6 +1459,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "Compaction Jobs" + }, + { + "id": "decimals", + "value": 0 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 13, "legend": { @@ -1399,55 +1547,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "Compaction Jobs", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 0, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit,\n sum by (user) (cortex_bucket_index_estimated_compaction_jobs{cluster=~\"$cluster\", job=~\"($namespace)/((compactor.*|cortex|mimir|mimir-backend.*))\"})\n and ignoring(user)\n (sum(rate(cortex_bucket_index_estimated_compaction_jobs_errors_total{cluster=~\"$cluster\", job=~\"($namespace)/((compactor.*|cortex|mimir|mimir-backend.*))\"}[$__rate_interval])) == 0)\n)\n", diff --git a/operations/mimir-mixin-compiled/dashboards/mimir-scaling.json b/operations/mimir-mixin-compiled/dashboards/mimir-scaling.json index b7b618287a4..22b8f38b947 100644 --- a/operations/mimir-mixin-compiled/dashboards/mimir-scaling.json +++ b/operations/mimir-mixin-compiled/dashboards/mimir-scaling.json @@ -62,6 +62,146 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "Required Replicas" + }, + { + "id": "decimals", + "value": 0 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "__name__" + }, + "properties": [ + { + "id": "custom.hidden", + "value": true + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "cluster" + }, + "properties": [ + { + "id": "displayName", + "value": "Cluster" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "deployment" + }, + "properties": [ + { + "id": "displayName", + "value": "Service" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "namespace" + }, + "properties": [ + { + "id": "displayName", + "value": "Namespace" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "reason" + }, + "properties": [ + { + "id": "displayName", + "value": "Reason" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 2, "legend": { @@ -90,115 +230,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "Required Replicas", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 0, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Cluster", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "__name__", - "thresholds": [ ], - "type": "hidden", - "unit": "short" - }, - { - "alias": "Cluster", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "cluster", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Service", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "deployment", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Namespace", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "namespace", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "Reason", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "reason", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "sort_desc(\n cluster_namespace_deployment_reason:required_replicas:count{cluster=~\"$cluster\", namespace=~\"$namespace\"}\n > ignoring(reason) group_left\n cluster_namespace_deployment:actual_replicas:count{cluster=~\"$cluster\", namespace=~\"$namespace\"}\n)\n", diff --git a/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json b/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json index ddbb1acd3a5..66bac4c7b66 100644 --- a/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json +++ b/operations/mimir-mixin-compiled/dashboards/mimir-tenants.json @@ -1606,6 +1606,46 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value #A" + }, + "properties": [ + { + "id": "displayName", + "value": "rules" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 27, "legend": { @@ -1634,40 +1674,6 @@ "span": 6, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "rules", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value #A", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group) (cortex_prometheus_rule_group_rules{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\", user=\"$user\"}))", @@ -1720,6 +1726,46 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value #A" + }, + "properties": [ + { + "id": "displayName", + "value": "seconds" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + } + ] + }, "fill": 1, "id": 28, "legend": { @@ -1748,40 +1794,6 @@ "span": 6, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "seconds", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value #A", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group) (cortex_prometheus_rule_group_last_duration_seconds{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\", user=\"$user\"}))", diff --git a/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json b/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json index d7f89b03bbc..efc266139cc 100644 --- a/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json +++ b/operations/mimir-mixin-compiled/dashboards/mimir-top-tenants.json @@ -63,6 +63,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 2, "legend": { @@ -91,55 +151,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_active_series{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", @@ -204,6 +215,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 3, "legend": { @@ -232,55 +303,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) ((\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n)\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n (\n cortex_ingester_memory_series_created_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n -\n cortex_ingester_memory_series_removed_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"} \n)\n,\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)", @@ -405,6 +427,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "samples/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 5, "legend": { @@ -433,55 +515,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "samples/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_distributor_received_samples_total{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -606,6 +639,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "samples/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 7, "legend": { @@ -634,55 +727,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "samples/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_discarded_samples_total{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*|distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -807,6 +851,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "series" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 9, "legend": { @@ -835,55 +939,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "series", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, # Classic storage\nsum by (cluster, namespace, user) (cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"})\n/ on (cluster, namespace) group_left()\nmax by (cluster, namespace) (cortex_distributor_replication_factor{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"})\nor on (cluster, namespace)\n# Ingest storage\nsum by (cluster, namespace, user) (\n max by (ingester_id, cluster, namespace, user) (\n label_replace(\n cortex_ingester_tsdb_exemplar_series_with_exemplars_in_storage{cluster=~\"$cluster\", job=~\"($namespace)/((ingester.*|cortex|mimir|mimir-write.*))\"},\n \"ingester_id\", \"$1\", \"pod\", \".*-([0-9]+)$\"\n )\n )\n)\n)\n", @@ -948,6 +1003,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "exemplars/s" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 10, "legend": { @@ -976,55 +1091,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "exemplars/s", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (user) (rate(cortex_distributor_received_exemplars_total{cluster=~\"$cluster\", job=~\"($namespace)/((distributor.*|cortex|mimir|mimir-write.*))\"}[5m])))", @@ -1089,6 +1155,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "rules" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 11, "legend": { @@ -1117,55 +1243,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "rules", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group, user) (cortex_prometheus_rule_group_rules{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\"}))", @@ -1230,6 +1307,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "seconds" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 12, "legend": { @@ -1258,55 +1395,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "seconds", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit, sum by (rule_group, user) (cortex_prometheus_rule_group_last_duration_seconds{cluster=~\"$cluster\", job=~\"($namespace)/((ruler|cortex|mimir|mimir-backend.*))\"}))", @@ -1371,6 +1459,66 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", + "fieldConfig": { + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.hidden", + "value": true + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "Value" + }, + "properties": [ + { + "id": "displayName", + "value": "Compaction Jobs" + }, + { + "id": "decimals", + "value": 0 + }, + { + "id": "unit", + "value": "short" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "user" + }, + "properties": [ + { + "id": "displayName", + "value": "user" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "unit", + "value": "string" + } + ] + } + ] + }, "fill": 1, "id": 13, "legend": { @@ -1399,55 +1547,6 @@ "span": 12, "stack": false, "steppedLine": false, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "Compaction Jobs", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 0, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "Value", - "thresholds": [ ], - "type": "number", - "unit": "short" - }, - { - "alias": "user", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": false, - "linkTargetBlank": false, - "linkTooltip": "Drill down", - "linkUrl": "", - "pattern": "user", - "thresholds": [ ], - "type": "number", - "unit": "string" - }, - { - "alias": "", - "colorMode": null, - "colors": [ ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "string", - "unit": "short" - } - ], "targets": [ { "expr": "topk($limit,\n sum by (user) (cortex_bucket_index_estimated_compaction_jobs{cluster=~\"$cluster\", job=~\"($namespace)/((compactor.*|cortex|mimir|mimir-backend.*))\"})\n and ignoring(user)\n (sum(rate(cortex_bucket_index_estimated_compaction_jobs_errors_total{cluster=~\"$cluster\", job=~\"($namespace)/((compactor.*|cortex|mimir|mimir-backend.*))\"}[$__rate_interval])) == 0)\n)\n", diff --git a/operations/mimir-mixin-tools/serve/run.sh b/operations/mimir-mixin-tools/serve/run.sh index c968c063198..7c24cfc3f33 100755 --- a/operations/mimir-mixin-tools/serve/run.sh +++ b/operations/mimir-mixin-tools/serve/run.sh @@ -4,7 +4,9 @@ set -e SCRIPT_DIR=$(cd `dirname $0` && pwd) -DOCKER_CONTAINER_NAME="mixin-serve-grafana" +# Ensure we run recent Grafana. +GRAFANA_VERSION=11.0.0 +DOCKER_CONTAINER_NAME="mixin-serve-grafana-graph" DOCKER_OPTS="" function usage() { @@ -59,8 +61,7 @@ function cleanup() { cleanup trap cleanup EXIT -# Ensure we run on Grafana latest. -docker pull grafana/grafana:latest +docker pull grafana/grafana:${GRAFANA_VERSION} # Run Grafana. echo "Starting Grafana container with name ${DOCKER_CONTAINER_NAME} listening on host port ${GRAFANA_PUBLISHED_PORT}" @@ -83,4 +84,4 @@ docker run \ -v "${SCRIPT_DIR}/provisioning-datasources.yaml:/etc/grafana/provisioning/datasources/provisioning-datasources.yaml" \ --expose 3000 \ --publish "${GRAFANA_PUBLISHED_PORT}:3000" \ - grafana/grafana:latest + grafana/grafana:${GRAFANA_VERSION} diff --git a/operations/mimir-mixin/dashboards/dashboard-utils.libsonnet b/operations/mimir-mixin/dashboards/dashboard-utils.libsonnet index 821511b6074..8be5838b79c 100644 --- a/operations/mimir-mixin/dashboards/dashboard-utils.libsonnet +++ b/operations/mimir-mixin/dashboards/dashboard-utils.libsonnet @@ -892,6 +892,35 @@ local utils = import 'mixin-utils/utils.libsonnet'; }, }, + tablePanel(queries, labelStyles):: + super.tablePanel(queries, labelStyles={}) + { + // Hides styles field, as it makes Grafana 11 use the deprecate "Table (old)" plugin. + styles:: super.styles, + local stylesToProps(s) = + if std.type(s) == 'string' then [ + $.overrideProperty('displayName', s), + $.overrideProperty('decimals', 0), + $.overrideProperty('unit', 'short'), + ] else [ + if std.objectHas(s, 'alias') then $.overrideProperty('displayName', s.alias), + if std.objectHas(s, 'type') && s.type == 'hidden' then $.overrideProperty('custom.hidden', true), + $.overrideProperty('decimals', if std.objectHas(s, 'decimals') then s.decimals else 2), + $.overrideProperty('unit', if std.objectHas(s, 'unit') then s.unit else 'short'), + ], + fieldConfig+: { + overrides+: [ + // Hide time column by default, like jsonnet-lib/grafana-builder does. + $.overrideFieldByName('Time', [ + $.overrideProperty('displayName', 'Time'), + $.overrideProperty('custom.hidden', true), + ]), + ] + [ + $.overrideFieldByName(label, std.prune(stylesToProps(labelStyles[label]))) + for label in std.objectFields(labelStyles) + ], + }, + }, + // Enables stacking of timeseries on top of each. // It overrites the "stack" mixin from jsonnet-lib/grafana-builder, to make it compatible with timeseriesPanel. stack:: { diff --git a/operations/mimir-mixin/dashboards/scaling.libsonnet b/operations/mimir-mixin/dashboards/scaling.libsonnet index 526ccc9e697..ecad60e4670 100644 --- a/operations/mimir-mixin/dashboards/scaling.libsonnet +++ b/operations/mimir-mixin/dashboards/scaling.libsonnet @@ -50,7 +50,7 @@ local filename = 'mimir-scaling.json'; ) ||| % [$._config.alert_aggregation_rule_prefix, $.namespaceMatcher(), $._config.alert_aggregation_rule_prefix, $.namespaceMatcher()], ], { - __name__: { alias: 'Cluster', type: 'hidden' }, + __name__: { type: 'hidden' }, cluster: { alias: 'Cluster' }, namespace: { alias: 'Namespace' }, deployment: { alias: 'Service' }, From cc7c34839215d27718ecc972caacf2bd3ea1e147 Mon Sep 17 00:00:00 2001 From: Ying WANG <74549700+ying-jeanne@users.noreply.github.com> Date: Sun, 26 May 2024 22:01:07 -0600 Subject: [PATCH 7/7] [ENHANCEMENT] Add `mimir-http-prefix` option to introduce a prefix for Mimir URL when use legacy routes. (#8069) * [ENHANCEMENT] Add `mimir-http-prefix` option to add prefix for mimir url when use legacy routes * fix test * Update CHANGELOG.md Co-authored-by: Jon Kartago Lamida --------- Co-authored-by: Jon Kartago Lamida --- CHANGELOG.md | 1 + pkg/mimirtool/client/client.go | 6 +++- pkg/mimirtool/client/rules_test.go | 49 +++++++++++++++++++++--------- pkg/mimirtool/commands/env_var.go | 3 ++ pkg/mimirtool/commands/rules.go | 5 +++ 5 files changed, 48 insertions(+), 16 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 901b19705f3..6e62c773829 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -118,6 +118,7 @@ * [CHANGE] Deprecated `--rule-files` flag in favor of CLI arguments. #7756 * [BUGFIX] Fix panic in `loadgen` subcommand. #7629 +* [ENHANCEMENT] Add `mimir-http-prefix` configuration to set the Mimir URL prefix when using legacy routes. #8069 * [ENHANCEMENT] `mimirtool promql format`: Format PromQL query with Prometheus' string or pretty-print formatter. #7742 * [BUGFIX] `mimirtool rules prepare`: do not add aggregation label to `on()` clause if already present in `group_left()` or `group_right()`. #7839 * [BUGFIX] Analyze Grafana: fix parsing queries with variables. #8062 diff --git a/pkg/mimirtool/client/client.go b/pkg/mimirtool/client/client.go index ebf5bb3fd60..b7e3165b410 100644 --- a/pkg/mimirtool/client/client.go +++ b/pkg/mimirtool/client/client.go @@ -46,6 +46,7 @@ type Config struct { ID string `yaml:"id"` TLS tls.ClientConfig UseLegacyRoutes bool `yaml:"use_legacy_routes"` + MimirHTTPPrefix string `yaml:"mimir_http_prefix"` AuthToken string `yaml:"auth_token"` ExtraHeaders map[string]string `yaml:"extra_headers"` } @@ -97,7 +98,10 @@ func New(cfg Config) (*MimirClient, error) { path := rulerAPIPath if cfg.UseLegacyRoutes { - path = legacyAPIPath + var err error + if path, err = url.JoinPath(cfg.MimirHTTPPrefix, legacyAPIPath); err != nil { + return nil, err + } } return &MimirClient{ diff --git a/pkg/mimirtool/client/rules_test.go b/pkg/mimirtool/client/rules_test.go index 378ac7f53e1..5a187e0f48d 100644 --- a/pkg/mimirtool/client/rules_test.go +++ b/pkg/mimirtool/client/rules_test.go @@ -24,22 +24,13 @@ func TestMimirClient_X(t *testing.T) { })) defer ts.Close() - client, err := New(Config{ - Address: ts.URL, - ID: "my-id", - Key: "my-key", - ExtraHeaders: map[string]string{ - "key1": "value1", - "key2": "value2", - }, - }) - require.NoError(t, err) - for _, tc := range []struct { - test string - namespace string - name string - expURLPath string + test string + namespace string + name string + expURLPath string + useLegacyRoutes bool + mimirHTTPPrefix string }{ { test: "regular-characters", @@ -71,9 +62,37 @@ func TestMimirClient_X(t *testing.T) { name: "last-char-slash/", expURLPath: "/prometheus/config/v1/rules/My%2FNamespace/last-char-slash%2F", }, + { + test: "use legacy routes with mimir-http-prefix", + namespace: "my-namespace", + name: "my-name", + useLegacyRoutes: true, + mimirHTTPPrefix: "/foo", + expURLPath: "/foo/api/v1/rules/my-namespace/my-name", + }, + { + test: "use non legacy routes with mimir-http-prefix ignored", + namespace: "my-namespace", + name: "my-name", + useLegacyRoutes: false, + mimirHTTPPrefix: "/foo", + expURLPath: "/prometheus/config/v1/rules/my-namespace/my-name", + }, } { t.Run(tc.test, func(t *testing.T) { ctx := context.Background() + client, err := New(Config{ + Address: ts.URL, + ID: "my-id", + Key: "my-key", + ExtraHeaders: map[string]string{ + "key1": "value1", + "key2": "value2", + }, + UseLegacyRoutes: tc.useLegacyRoutes, + MimirHTTPPrefix: tc.mimirHTTPPrefix, + }) + require.NoError(t, err) require.NoError(t, client.DeleteRuleGroup(ctx, tc.namespace, tc.name)) req := <-requestCh diff --git a/pkg/mimirtool/commands/env_var.go b/pkg/mimirtool/commands/env_var.go index 2cf00002cfc..3e6187ecdb9 100644 --- a/pkg/mimirtool/commands/env_var.go +++ b/pkg/mimirtool/commands/env_var.go @@ -12,6 +12,7 @@ type EnvVarNames struct { TLSInsecureSkipVerify string TenantID string UseLegacyRoutes string + MimirHTTPPrefix string AuthToken string ExtraHeaders string } @@ -29,6 +30,7 @@ func NewEnvVarsWithPrefix(prefix string) EnvVarNames { useLegacyRoutes = "USE_LEGACY_ROUTES" authToken = "AUTH_TOKEN" extraHeaders = "EXTRA_HEADERS" + mimirHTTPPrefix = "MIMIR_HTTP_PREFIX" ) if len(prefix) > 0 && prefix[len(prefix)-1] != '_' { @@ -47,5 +49,6 @@ func NewEnvVarsWithPrefix(prefix string) EnvVarNames { UseLegacyRoutes: prefix + useLegacyRoutes, AuthToken: prefix + authToken, ExtraHeaders: prefix + extraHeaders, + MimirHTTPPrefix: prefix + mimirHTTPPrefix, } } diff --git a/pkg/mimirtool/commands/rules.go b/pkg/mimirtool/commands/rules.go index 4601e5c730d..1c5d371045a 100644 --- a/pkg/mimirtool/commands/rules.go +++ b/pkg/mimirtool/commands/rules.go @@ -176,6 +176,11 @@ func (r *RuleCommand) Register(app *kingpin.Application, envVars EnvVarNames, re Envar(envVars.UseLegacyRoutes). BoolVar(&r.ClientConfig.UseLegacyRoutes) + c.Flag("mimir-http-prefix", "Used when use-legacy-routes is set to true. The prefix to use for the url when contacting Grafana Mimir; alternatively, set "+envVars.MimirHTTPPrefix+"."). + Default("/prometheus"). + Envar(envVars.MimirHTTPPrefix). + StringVar(&r.ClientConfig.MimirHTTPPrefix) + c.Flag("tls-ca-path", "TLS CA certificate to verify Grafana Mimir API as part of mTLS; alternatively, set "+envVars.TLSCAPath+"."). Default(""). Envar(envVars.TLSCAPath).