From 897d3b69e7e5fade873c5f824d68ddff97b521c8 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Tue, 19 Mar 2024 11:28:43 +0800 Subject: [PATCH 01/43] Fix outdated comment --- pkg/storegateway/bucket.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 5406757df9f..46afb95e676 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1074,7 +1074,7 @@ func (s *BucketStore) streamingSeriesForBlocks( // 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. +// It is recommended to pass the reusedPostingsAndMatchers returned by the streamingSeriesForBlocks call. func (s *BucketStore) streamingChunksSetForBlocks( ctx context.Context, req *storepb.SeriesRequest, From 254b1dbbf7c9af95c8e0dffe96107ee078e463b9 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Tue, 19 Mar 2024 11:28:43 +0800 Subject: [PATCH 02/43] Add initial support for resetting `loadingSeriesChunkRefsSetIterator` --- pkg/storegateway/series_refs.go | 44 ++++++++++++++++++++++++++-- pkg/storegateway/series_refs_test.go | 16 ++++++++++ 2 files changed, 58 insertions(+), 2 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index df23681acb5..ec2112fa91f 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -682,8 +682,9 @@ type loadingSeriesChunkRefsSetIterator struct { chunkMetasBuffer []chunks.Meta - err error - currentSet seriesChunkRefsSet + err error + currentSet seriesChunkRefsSet + resetToFirstSet bool } func openBlockSeriesChunkRefsSetsIterator( @@ -855,6 +856,18 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { return false } + if s.resetToFirstSet { + // The first set is already loaded as s.currentSet, so nothing more to do. + s.resetToFirstSet = false + + if s.currentSet.len() == 0 { + // If the first batch of postings had series, but they've all been filtered out, then we are done. + return false + } + + return true + } + defer func(startTime time.Time) { spanLog := spanlogger.FromContext(s.ctx, s.logger) spanLog.DebugLog( @@ -1103,6 +1116,24 @@ func (s *loadingSeriesChunkRefsSetIterator) Err() error { return s.err } +func (s *loadingSeriesChunkRefsSetIterator) Reset() { + if s.err != nil { + return + } + + if s.postingsSetIterator.IsFirstAndOnlyBatch() { + s.resetToFirstSet = true + } else { + s.currentSet = seriesChunkRefsSet{} + } + + s.postingsSetIterator.Reset() + + // TODO: handle change in strategy + // - When loading first and only batch, and we know we'll want chunks later for streaming: always load chunk refs + // - Otherwise, when resetting, set flag to load chunk refs +} + // loadSeries returns a for chunks. It is not safe to use the returned []chunks.Meta after calling loadSeries again func (s *loadingSeriesChunkRefsSetIterator) loadSeries(ref storage.SeriesRef, loadedSeries *bucketIndexLoadedSeries, stats *queryStats, lsetPool *pool.SlabPool[symbolizedLabel]) ([]symbolizedLabel, []chunks.Meta, error) { ok, lbls, err := loadedSeries.unsafeLoadSeries(ref, &s.chunkMetasBuffer, s.strategy.isNoChunkRefsOnEntireBlock(), stats, lsetPool) @@ -1403,3 +1434,12 @@ func (s *postingsSetsIterator) Next() bool { func (s *postingsSetsIterator) At() []storage.SeriesRef { return s.currentBatch } + +func (s *postingsSetsIterator) Reset() { + s.currentBatch = nil + s.nextBatchPostingsOffset = 0 +} + +func (s *postingsSetsIterator) IsFirstAndOnlyBatch() bool { + return len(s.postings) > 0 && s.nextBatchPostingsOffset <= s.batchSize && s.nextBatchPostingsOffset >= len(s.postings) +} diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 5ce1f159736..cd92bbdb22f 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1427,6 +1427,12 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { sets := readAllSeriesChunkRefsSet(loadingIterator) assert.NoError(t, loadingIterator.Err()) assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, tc.strategy, tc.expectedSets, sets) + + // Ensure that the iterator behaves correctly after a reset. + loadingIterator.Reset() + setsAfterReset := readAllSeriesChunkRefsSet(loadingIterator) + assert.NoError(t, loadingIterator.Err()) + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, tc.strategy, tc.expectedSets, setsAfterReset) }) } } @@ -2303,9 +2309,19 @@ func TestPostingsSetsIterator(t *testing.T) { var actualBatches [][]storage.SeriesRef for iterator.Next() { actualBatches = append(actualBatches, iterator.At()) + + if len(testCase.expectedBatches) != 1 { + assert.False(t, iterator.IsFirstAndOnlyBatch()) + } } assert.ElementsMatch(t, testCase.expectedBatches, actualBatches) + + if len(testCase.expectedBatches) == 1 { + assert.True(t, iterator.IsFirstAndOnlyBatch()) + } else if len(testCase.expectedBatches) == 0 { + assert.False(t, iterator.IsFirstAndOnlyBatch()) + } }) } } From 4f67084a098381cfc48b9a15aaed42e3470a526f Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Tue, 19 Mar 2024 11:28:43 +0800 Subject: [PATCH 03/43] Add important TODO --- pkg/storegateway/series_refs.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index ec2112fa91f..e8507232262 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -131,6 +131,7 @@ func (b seriesChunkRefsSet) len() int { // // This function is not idempotent. Calling it twice would introduce subtle bugs. func (b seriesChunkRefsSet) release() { + // TODO: how do we ensure that this isn't called on a set that will be reused? if b.series == nil || !b.releasable { return } From 3fa6e3f5cb9fa3e856f81d421f4fa5d3968f2e83 Mon Sep 17 00:00:00 2001 From: Jeanette Tan Date: Tue, 19 Mar 2024 11:28:43 +0800 Subject: [PATCH 04/43] add reset funcs in preparation but don't use them yet --- pkg/storegateway/series_chunks.go | 13 +++++++++++++ pkg/storegateway/series_chunks_test.go | 10 ++++++++++ pkg/storegateway/series_refs.go | 23 +++++++++++++++++++++++ pkg/storegateway/series_refs_test.go | 5 +++++ 4 files changed, 51 insertions(+) diff --git a/pkg/storegateway/series_chunks.go b/pkg/storegateway/series_chunks.go index 04504af57f7..c7e9dfce8d8 100644 --- a/pkg/storegateway/series_chunks.go +++ b/pkg/storegateway/series_chunks.go @@ -60,6 +60,7 @@ type iterator[V any] interface { Next() bool At() V Err() error + Reset() } // seriesChunksSet holds a set of series, each with its own chunks. @@ -287,6 +288,10 @@ func (p *preloadingSetIterator[Set]) Err() error { return p.err } +func (p *preloadingSetIterator[Set]) Reset() { + p.from.Reset() +} + func newPreloadingAndStatsTrackingSetIterator[Set any](ctx context.Context, preloadedSetsCount int, iterator iterator[Set], stats *safeQueryStats) iterator[Set] { // Track the time spent loading batches (including preloading). numBatches := 0 @@ -433,6 +438,10 @@ func (c *loadingSeriesChunksSetIterator) Err() error { return c.err } +func (c *loadingSeriesChunksSetIterator) Reset() { + c.from.Reset() +} + func (c *loadingSeriesChunksSetIterator) recordReturnedChunks(series []seriesChunks) { returnedChunks, returnedChunksBytes := chunkStats(series) @@ -498,3 +507,7 @@ func (m *nextDurationMeasuringIterator[Set]) At() Set { func (m *nextDurationMeasuringIterator[Set]) Err() error { return m.from.Err() } + +func (m *nextDurationMeasuringIterator[Set]) Reset() { + m.from.Reset() +} diff --git a/pkg/storegateway/series_chunks_test.go b/pkg/storegateway/series_chunks_test.go index 5cdfb6cea43..b5eccb326b7 100644 --- a/pkg/storegateway/series_chunks_test.go +++ b/pkg/storegateway/series_chunks_test.go @@ -884,6 +884,12 @@ func (s *sliceSeriesChunksSetIterator) Err() error { return nil } +func (s *sliceSeriesChunksSetIterator) Reset() { + s.current = -1 + s.err = nil + s.errAt = 0 +} + // delayedIterator implements iterator and introduces an artificial delay before returning from Next() and At(). type delayedIterator[S any] struct { wrapped iterator[S] @@ -911,6 +917,10 @@ func (s *delayedIterator[S]) Err() error { return s.wrapped.Err() } +func (s *delayedIterator[S]) Reset() { + s.wrapped.Reset() +} + func generateAggrChunk(num int) []storepb.AggrChunk { out := make([]storepb.AggrChunk, 0, num) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index e8507232262..0fc31397793 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -280,12 +280,18 @@ func (c flattenedSeriesChunkRefsIterator) Err() error { return c.from.Err() } +func (c flattenedSeriesChunkRefsIterator) Reset() { + //TODO(zenador): needed? + c.from.Reset() +} + type emptySeriesChunkRefsSetIterator struct { } func (emptySeriesChunkRefsSetIterator) Next() bool { return false } func (emptySeriesChunkRefsSetIterator) At() seriesChunkRefsSet { return seriesChunkRefsSet{} } func (emptySeriesChunkRefsSetIterator) Err() error { return nil } +func (emptySeriesChunkRefsSetIterator) Reset() {} func mergedSeriesChunkRefsSetIterators(mergedBatchSize int, all ...iterator[seriesChunkRefsSet]) iterator[seriesChunkRefsSet] { switch len(all) { @@ -375,6 +381,11 @@ func (s *mergedSeriesChunkRefsSet) Next() bool { return true } +func (s *mergedSeriesChunkRefsSet) Reset() { + s.a.Reset() + s.b.Reset() +} + func (s *mergedSeriesChunkRefsSet) ensureCursors(curr1, curr2 *seriesChunkRefsIterator, set1, set2 iterator[seriesChunkRefsSet]) error { // When both cursors are empty, we advance their set iterators concurrently to reduce total waiting time for the // IO from underlying set iterators (see grafana/mimir#4596). @@ -612,6 +623,10 @@ func (s *deduplicatingSeriesChunkRefsSetIterator) Next() bool { return true } +func (s *deduplicatingSeriesChunkRefsSetIterator) Reset() { + s.from.Reset() +} + type limitingSeriesChunkRefsSetIterator struct { from iterator[seriesChunkRefsSet] chunksLimiter ChunksLimiter @@ -667,6 +682,10 @@ func (l *limitingSeriesChunkRefsSetIterator) Err() error { return l.err } +func (l *limitingSeriesChunkRefsSetIterator) Reset() { + l.from.Reset() +} + type loadingSeriesChunkRefsSetIterator struct { ctx context.Context postingsSetIterator *postingsSetsIterator @@ -1274,6 +1293,10 @@ func (m *filteringSeriesChunkRefsSetIterator) Err() error { return m.from.Err() } +func (m *filteringSeriesChunkRefsSetIterator) Reset() { + m.from.Reset() +} + // cachedSeriesForPostingsID contains enough information to be able to tell whether a cache entry // is the right cache entry that we are looking for. We store only the postingsKey in the // cache key because the encoded postings are too big. We store the encoded postings within diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index cd92bbdb22f..83ac980a270 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -2415,6 +2415,11 @@ func (s *sliceSeriesChunkRefsSetIterator) Err() error { return nil } +func (s *sliceSeriesChunkRefsSetIterator) Reset() { + s.current = -1 + s.err = nil +} + type staticLimiter struct { limit int msg string From 05e62740d4a053f9ccec800111e45ccbc15de4c9 Mon Sep 17 00:00:00 2001 From: Jeanette Tan Date: Tue, 19 Mar 2024 11:28:43 +0800 Subject: [PATCH 05/43] reuse the iterator, this builds but fails tests with EOF --- pkg/storegateway/bucket.go | 52 ++++++-------------- pkg/storegateway/bucket_chunk_reader_test.go | 1 - pkg/storegateway/series_refs.go | 40 +++------------ pkg/storegateway/series_refs_test.go | 45 ----------------- 4 files changed, 20 insertions(+), 118 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 46afb95e676..8b05fd08dc6 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -618,8 +618,8 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storegatewaypb.Stor 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{} + reuseIterator iterator[seriesChunkRefsSet] + resHints = &hintspb.SeriesResponseHints{} ) for _, b := range blocks { resHints.AddQueriedBlock(b.meta.ULID) @@ -643,7 +643,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, reuseIterator, err = s.streamingSeriesForBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) if err != nil { return err } @@ -672,7 +672,7 @@ 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) + seriesChunkIt, err = s.streamingChunksSetForBlocks(ctx, readers, stats, reuseIterator) if err != nil { return err } @@ -1029,7 +1029,7 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( 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) if err != nil { return nil, err } @@ -1046,7 +1046,7 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( // streamingSeriesForBlocks 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. +// The iterator should be re-used when getting chunks to save on computation. func (s *BucketStore) streamingSeriesForBlocks( ctx context.Context, req *storepb.SeriesRequest, @@ -1057,41 +1057,25 @@ func (s *BucketStore) streamingSeriesForBlocks( chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. seriesLimiter SeriesLimiter, // Rate limiter for loading series. 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, iterator[seriesChunkRefsSet], error) { + var strategy = noChunkRefs | overlapMintMaxt + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy) if err != nil { return nil, nil, err } - return newSeriesSetWithoutChunks(ctx, it, stats), reuse, nil + return newSeriesSetWithoutChunks(ctx, it, stats), it, 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 reusedPostingsAndMatchers returned by the streamingSeriesForBlocks call. +// It is recommended to pass the iterator returned by the streamingSeriesForBlocks call for reuse. func (s *BucketStore) streamingChunksSetForBlocks( 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. + it iterator[seriesChunkRefsSet], // 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 - } + it.Reset() scsi := newChunksPreloadingIterator(ctx, s.logger, s.userID, *chunkReaders, it, s.maxSeriesPerBatch, stats) return scsi, nil } @@ -1106,7 +1090,6 @@ 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, ) (iterator[seriesChunkRefsSet], error) { var ( @@ -1116,9 +1099,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] @@ -1129,10 +1111,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, @@ -1147,7 +1125,6 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( strategy, req.MinTime, req.MaxTime, stats, - r, s.logger, ) if err != nil { @@ -1452,7 +1429,6 @@ func blockLabelNames(ctx context.Context, indexr *bucketIndexReader, matchers [] noChunkRefs, minTime, maxTime, stats, - nil, logger, ) if err != nil { diff --git a/pkg/storegateway/bucket_chunk_reader_test.go b/pkg/storegateway/bucket_chunk_reader_test.go index 8951228b22f..d97dc9a9522 100644 --- a/pkg/storegateway/bucket_chunk_reader_test.go +++ b/pkg/storegateway/bucket_chunk_reader_test.go @@ -44,7 +44,6 @@ func TestBucketChunkReader_refetchChunks(t *testing.T) { block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), - nil, log.NewNopLogger(), ) require.NoError(t, err) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 0fc31397793..91a3fe6a349 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -559,6 +559,10 @@ func (s *seriesChunkRefsSeriesSet) Err() error { return s.from.Err() } +// func (s *seriesChunkRefsSeriesSet) Reset() { +// s.from.Reset() +// } + // deduplicatingSeriesChunkRefsSetIterator merges together consecutive series in the underlying iterator. type deduplicatingSeriesChunkRefsSetIterator struct { batchSize int @@ -720,7 +724,6 @@ func openBlockSeriesChunkRefsSetsIterator( strategy seriesIteratorStrategy, minTime, maxTime int64, // Series must have data in this time range to be returned (ignored if skipChunks=true). 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, ) (iterator[seriesChunkRefsSet], error) { if batchSize <= 0 { @@ -732,20 +735,12 @@ func openBlockSeriesChunkRefsSetsIterator( pendingMatchers []*labels.Matcher fetchPostings = true ) - if reuse != nil { - fetchPostings = !reuse.isSet() - ps = reuse.ps - pendingMatchers = reuse.matchers - } 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) - } } var it iterator[seriesChunkRefsSet] @@ -771,31 +766,6 @@ func openBlockSeriesChunkRefsSetsIterator( 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 -} - // seriesIteratorStrategy defines the strategy to use when loading the series and their chunk refs. // See below for available options. type seriesIteratorStrategy byte @@ -1149,6 +1119,8 @@ func (s *loadingSeriesChunkRefsSetIterator) Reset() { s.postingsSetIterator.Reset() + s.strategy = defaultStrategy + // TODO: handle change in strategy // - When loading first and only batch, and we know we'll want chunks later for streaming: always load chunk refs // - Otherwise, when resetting, set flag to load chunk refs diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 83ac980a270..a00a927cb72 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1709,7 +1709,6 @@ func TestOpenBlockSeriesChunkRefsSetsIterator(t *testing.T) { minT, maxT, newSafeQueryStats(), - nil, log.NewNopLogger(), ) require.NoError(t, err) @@ -1810,7 +1809,6 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_pendingMatchers(t *testing.T) { block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), - nil, log.NewNopLogger(), ) require.NoError(t, err) @@ -1874,7 +1872,6 @@ func BenchmarkOpenBlockSeriesChunkRefsSetsIterator(b *testing.B) { block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), - nil, log.NewNopLogger(), ) require.NoError(b, err) @@ -2203,7 +2200,6 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { b.meta.MinTime, b.meta.MaxTime, statsColdCache, - nil, log.NewNopLogger(), ) @@ -2234,7 +2230,6 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { b.meta.MinTime, b.meta.MaxTime, statsWarnCache, - nil, log.NewNopLogger(), ) require.NoError(t, err) @@ -2326,46 +2321,6 @@ func TestPostingsSetsIterator(t *testing.T) { } } -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 From a15a25e72dcfc33d25c61353bd97bc7688f378e1 Mon Sep 17 00:00:00 2001 From: Jeanette Tan Date: Tue, 19 Mar 2024 11:28:43 +0800 Subject: [PATCH 06/43] try changing strategy up front --- pkg/storegateway/bucket.go | 9 +++++---- pkg/storegateway/bucket_test.go | 2 +- pkg/storegateway/series_refs_test.go | 9 +++++---- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 8b05fd08dc6..827a180c47f 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1027,7 +1027,7 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( ) (storepb.SeriesSet, error) { strategy := defaultStrategy if req.SkipChunks { - strategy = noChunkRefs + // strategy = noChunkRefs } it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy) if err != nil { @@ -1058,7 +1058,8 @@ func (s *BucketStore) streamingSeriesForBlocks( seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, ) (storepb.SeriesSet, iterator[seriesChunkRefsSet], error) { - var strategy = noChunkRefs | overlapMintMaxt + // var strategy = noChunkRefs | overlapMintMaxt + var strategy = overlapMintMaxt it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy) if err != nil { return nil, nil, err @@ -1426,7 +1427,7 @@ func blockLabelNames(ctx context.Context, indexr *bucketIndexReader, matchers [] matchers, nil, cachedSeriesHasher{nil}, - noChunkRefs, + 0, // remove noChunkRefs strategy minTime, maxTime, stats, logger, @@ -1645,7 +1646,7 @@ func labelValuesFromSeries(ctx context.Context, labelName string, seriesPerBatch b.meta, nil, nil, - noChunkRefs, + 0, // remove noChunkRefs strategy b.meta.MinTime, b.meta.MaxTime, b.userID, diff --git a/pkg/storegateway/bucket_test.go b/pkg/storegateway/bucket_test.go index c68bb012d6a..ad1715f3d17 100644 --- a/pkg/storegateway/bucket_test.go +++ b/pkg/storegateway/bucket_test.go @@ -1170,7 +1170,7 @@ func loadSeries(ctx context.Context, tb test.TB, postings []storage.SeriesRef, i indexr.block.meta, nil, nil, - noChunkRefs, + 0, // remove noChunkRefs strategy 0, 0, "", diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index a00a927cb72..3a13801b9fb 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1693,7 +1693,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator(t *testing.T) { strategy := defaultStrategy if testCase.skipChunks { - strategy = noChunkRefs + // strategy = noChunkRefs } iterator, err := openBlockSeriesChunkRefsSetsIterator( ctx, @@ -1805,7 +1805,8 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_pendingMatchers(t *testing.T) { testCase.matchers, nil, cachedSeriesHasher{hashCache}, - noChunkRefs, // skip chunks since we are testing labels filtering + // noChunkRefs, // skip chunks since we are testing labels filtering + 0, // remove noChunkRefs strategy block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), @@ -2196,7 +2197,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { testCase.matchers, testCase.shard, seriesHasher, - noChunkRefs, + 0, // remove noChunkRefs strategy b.meta.MinTime, b.meta.MaxTime, statsColdCache, @@ -2226,7 +2227,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { testCase.matchers, testCase.shard, seriesHasher, - noChunkRefs, + 0, // remove noChunkRefs strategy b.meta.MinTime, b.meta.MaxTime, statsWarnCache, From 17c8360cb2eba0d8f7c09b574314cde0ccc0049d Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 27 Mar 2024 14:27:20 +1100 Subject: [PATCH 07/43] Correctly reset merging and deduplicating iterators --- pkg/storegateway/series_refs.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 91a3fe6a349..0212d86859e 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -384,6 +384,9 @@ func (s *mergedSeriesChunkRefsSet) Next() bool { func (s *mergedSeriesChunkRefsSet) Reset() { s.a.Reset() s.b.Reset() + s.done = false + s.aAt = newSeriesChunkRefsIterator(seriesChunkRefsSet{}) + s.bAt = newSeriesChunkRefsIterator(seriesChunkRefsSet{}) } func (s *mergedSeriesChunkRefsSet) ensureCursors(curr1, curr2 *seriesChunkRefsIterator, set1, set2 iterator[seriesChunkRefsSet]) error { @@ -629,6 +632,7 @@ func (s *deduplicatingSeriesChunkRefsSetIterator) Next() bool { func (s *deduplicatingSeriesChunkRefsSetIterator) Reset() { s.from.Reset() + s.peek = nil } type limitingSeriesChunkRefsSetIterator struct { From 2473ca100ec06a6374ae89b7c500e03e80d54378 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 27 Mar 2024 14:27:53 +1100 Subject: [PATCH 08/43] Avoid counting series and chunks twice in limiting iterator --- pkg/storegateway/series_refs.go | 39 ++++++++++++++++++++++++--------- 1 file changed, 29 insertions(+), 10 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 0212d86859e..dce0d072b8d 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -640,6 +640,11 @@ type limitingSeriesChunkRefsSetIterator struct { chunksLimiter ChunksLimiter seriesLimiter SeriesLimiter + chunksSeen uint64 // Number of chunks seen since last call to Reset() + maxChunksSeenEver uint64 // Maximum number of chunks ever seen across calls to Reset(), used to avoid double-counting after Reset() + seriesSeen uint64 // Number of series seen since last call to Reset() + maxSeriesSeenEver uint64 // Maximum number of chunks ever seen across calls to Reset(), used to avoid double-counting after Reset() + err error currentBatch seriesChunkRefsSet } @@ -663,22 +668,34 @@ func (l *limitingSeriesChunkRefsSetIterator) Next() bool { } l.currentBatch = l.from.At() - err := l.seriesLimiter.Reserve(uint64(l.currentBatch.len())) - if err != nil { - l.err = err - return false + l.seriesSeen += uint64(l.currentBatch.len()) + + if l.seriesSeen > l.maxSeriesSeenEver { + newSeries := l.seriesSeen - l.maxSeriesSeenEver + + if err := l.seriesLimiter.Reserve(newSeries); err != nil { + l.err = err + return false + } + + l.maxSeriesSeenEver = l.seriesSeen } - var totalChunks int for _, s := range l.currentBatch.series { - totalChunks += len(s.refs) + l.chunksSeen += uint64(len(s.refs)) } - err = l.chunksLimiter.Reserve(uint64(totalChunks)) - if err != nil { - l.err = err - return false + if l.chunksSeen > l.maxChunksSeenEver { + newChunks := l.chunksSeen - l.maxChunksSeenEver + + if err := l.chunksLimiter.Reserve(newChunks); err != nil { + l.err = err + return false + } + + l.maxChunksSeenEver = l.chunksSeen } + return true } @@ -691,6 +708,8 @@ func (l *limitingSeriesChunkRefsSetIterator) Err() error { } func (l *limitingSeriesChunkRefsSetIterator) Reset() { + l.chunksSeen = 0 + l.seriesSeen = 0 l.from.Reset() } From 4f4d20be4f21cc3004e86b1d2c1887a28290d5c1 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 3 Apr 2024 15:51:04 +1100 Subject: [PATCH 09/43] Don't allow releasing series chunk refs set that will be reused --- pkg/storegateway/series_refs.go | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index dce0d072b8d..4dee096977d 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -131,7 +131,6 @@ func (b seriesChunkRefsSet) len() int { // // This function is not idempotent. Calling it twice would introduce subtle bugs. func (b seriesChunkRefsSet) release() { - // TODO: how do we ensure that this isn't called on a set that will be reused? if b.series == nil || !b.releasable { return } @@ -1192,8 +1191,11 @@ func (s *loadingSeriesChunkRefsSetIterator) singlePassStringify(symbolizedSet sy } } - // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again. - set := newSeriesChunkRefsSet(len(symbolizedSet.series), true) + // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, + // provided it's not the first and only batch. + // TODO: should we forcibly release the retained first and only batch later? + releaseable := !s.postingsSetIterator.IsFirstAndOnlyBatch() + set := newSeriesChunkRefsSet(len(symbolizedSet.series), releaseable) labelsBuilder := labels.NewScratchBuilder(maxLabelsPerSeries) for _, series := range symbolizedSet.series { @@ -1212,8 +1214,11 @@ func (s *loadingSeriesChunkRefsSetIterator) singlePassStringify(symbolizedSet sy } func (s *loadingSeriesChunkRefsSetIterator) multiLookupStringify(symbolizedSet symbolizedSeriesChunkRefsSet) (seriesChunkRefsSet, error) { - // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again. - set := newSeriesChunkRefsSet(len(symbolizedSet.series), true) + // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, + // provided it's not the first and only batch. + // TODO: should we forcibly release the retained first and only batch later? + releaseable := !s.postingsSetIterator.IsFirstAndOnlyBatch() + set := newSeriesChunkRefsSet(len(symbolizedSet.series), releaseable) labelsBuilder := labels.NewScratchBuilder(16) for _, series := range symbolizedSet.series { From 1542ebe2109adcccb1f1f9201830881974d5e921 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 3 May 2024 12:59:14 +1000 Subject: [PATCH 10/43] Restore previous test setup --- pkg/storegateway/series_refs_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 3a13801b9fb..64836c17498 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1805,8 +1805,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_pendingMatchers(t *testing.T) { testCase.matchers, nil, cachedSeriesHasher{hashCache}, - // noChunkRefs, // skip chunks since we are testing labels filtering - 0, // remove noChunkRefs strategy + noChunkRefs, // skip chunks since we are testing labels filtering block.meta.MinTime, block.meta.MaxTime, newSafeQueryStats(), From 09ddd2e6cff8da611add27bd99610afab0c98688 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 3 May 2024 12:59:46 +1000 Subject: [PATCH 11/43] Fix typo in variable name --- pkg/storegateway/series_refs_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 64836c17498..b7ba7e2cb6f 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -2215,7 +2215,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { cached: testCase.cachedSeriesHashesWithWarmCache, } - statsWarnCache := newSafeQueryStats() + statsWarmCache := newSafeQueryStats() ss, err = openBlockSeriesChunkRefsSetsIterator( context.Background(), batchSize, @@ -2229,14 +2229,14 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { 0, // remove noChunkRefs strategy b.meta.MinTime, b.meta.MaxTime, - statsWarnCache, + statsWarmCache, log.NewNopLogger(), ) 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) }) } }) From fb1322be6006c39013a244647c452d245c0ac7e3 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 3 May 2024 13:00:17 +1000 Subject: [PATCH 12/43] Restore more previous test setup --- pkg/storegateway/series_refs_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index b7ba7e2cb6f..dfd4971fb8f 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -2196,7 +2196,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { testCase.matchers, testCase.shard, seriesHasher, - 0, // remove noChunkRefs strategy + noChunkRefs, b.meta.MinTime, b.meta.MaxTime, statsColdCache, @@ -2226,7 +2226,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { testCase.matchers, testCase.shard, seriesHasher, - 0, // remove noChunkRefs strategy + noChunkRefs, b.meta.MinTime, b.meta.MaxTime, statsWarmCache, From e1eadbd07679d2b7e6a43c248ed32547fd16255e Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 3 May 2024 13:56:30 +1000 Subject: [PATCH 13/43] Restore use of noChunkRefs where it should be used, and remove outdated comments. --- pkg/storegateway/bucket.go | 6 +++--- pkg/storegateway/bucket_test.go | 2 +- pkg/storegateway/series_refs.go | 5 ----- pkg/storegateway/series_refs_test.go | 2 +- 4 files changed, 5 insertions(+), 10 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 827a180c47f..23fd5da8c5a 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1027,7 +1027,7 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( ) (storepb.SeriesSet, error) { strategy := defaultStrategy if req.SkipChunks { - // strategy = noChunkRefs + strategy = noChunkRefs } it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy) if err != nil { @@ -1427,7 +1427,7 @@ func blockLabelNames(ctx context.Context, indexr *bucketIndexReader, matchers [] matchers, nil, cachedSeriesHasher{nil}, - 0, // remove noChunkRefs strategy + noChunkRefs, minTime, maxTime, stats, logger, @@ -1646,7 +1646,7 @@ func labelValuesFromSeries(ctx context.Context, labelName string, seriesPerBatch b.meta, nil, nil, - 0, // remove noChunkRefs strategy + noChunkRefs, b.meta.MinTime, b.meta.MaxTime, b.userID, diff --git a/pkg/storegateway/bucket_test.go b/pkg/storegateway/bucket_test.go index ad1715f3d17..c68bb012d6a 100644 --- a/pkg/storegateway/bucket_test.go +++ b/pkg/storegateway/bucket_test.go @@ -1170,7 +1170,7 @@ func loadSeries(ctx context.Context, tb test.TB, postings []storage.SeriesRef, i indexr.block.meta, nil, nil, - 0, // remove noChunkRefs strategy + noChunkRefs, 0, 0, "", diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 4dee096977d..94708783ba8 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -280,7 +280,6 @@ func (c flattenedSeriesChunkRefsIterator) Err() error { } func (c flattenedSeriesChunkRefsIterator) Reset() { - //TODO(zenador): needed? c.from.Reset() } @@ -561,10 +560,6 @@ func (s *seriesChunkRefsSeriesSet) Err() error { return s.from.Err() } -// func (s *seriesChunkRefsSeriesSet) Reset() { -// s.from.Reset() -// } - // deduplicatingSeriesChunkRefsSetIterator merges together consecutive series in the underlying iterator. type deduplicatingSeriesChunkRefsSetIterator struct { batchSize int diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index dfd4971fb8f..92bae78c710 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1693,7 +1693,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator(t *testing.T) { strategy := defaultStrategy if testCase.skipChunks { - // strategy = noChunkRefs + strategy = noChunkRefs } iterator, err := openBlockSeriesChunkRefsSetsIterator( ctx, From 676349d42a06c0e9bab8dfd78b1f43b34bc6fa20 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 3 May 2024 14:02:08 +1000 Subject: [PATCH 14/43] Fix linting warning. --- pkg/storegateway/series_refs.go | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 94708783ba8..7b5df11ae34 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -867,12 +867,8 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { // The first set is already loaded as s.currentSet, so nothing more to do. s.resetToFirstSet = false - if s.currentSet.len() == 0 { - // If the first batch of postings had series, but they've all been filtered out, then we are done. - return false - } - - return true + // If the first batch of postings had series, but they've all been filtered out, then we are done. + return s.currentSet.len() != 0 } defer func(startTime time.Time) { From c3fb2759dac1cb37207aa5c98b38ab66067cc391 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 3 May 2024 14:13:34 +1000 Subject: [PATCH 15/43] Introduce seriesIteratorStrategy flag for chunks streaming --- pkg/storegateway/bucket.go | 3 +-- pkg/storegateway/series_refs.go | 15 ++++++++++----- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 23fd5da8c5a..31bff10fd8d 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1058,8 +1058,7 @@ func (s *BucketStore) streamingSeriesForBlocks( seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, ) (storepb.SeriesSet, iterator[seriesChunkRefsSet], error) { - // var strategy = noChunkRefs | overlapMintMaxt - var strategy = overlapMintMaxt + var strategy = overlapMintMaxt | forChunksStreaming it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy) if err != nil { return nil, nil, err diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 7b5df11ae34..9ae0642d740 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -798,6 +798,9 @@ const ( // overlapMintMaxt flag is used together with noChunkRefs. With this, only the series whose // chunks overlap with [mint, maxt] are selected. overlapMintMaxt seriesIteratorStrategy = 0b00000010 + // forChunksStreaming flag is used when the iterator is being used for store-gateway to querier chunks + // streaming. It enables optimisations for the case where only a single batch is needed for all series. + forChunksStreaming seriesIteratorStrategy = 0b00000100 ) func (s seriesIteratorStrategy) isNoChunkRefs() bool { @@ -808,6 +811,8 @@ func (s seriesIteratorStrategy) isOverlapMintMaxt() bool { return s&overlapMintMaxt != 0 } +func (s seriesIteratorStrategy) isForChunksStreaming() bool { return s&forChunksStreaming != 0 } + func (s seriesIteratorStrategy) isOnEntireBlock() bool { return !s.isOverlapMintMaxt() } @@ -1183,10 +1188,10 @@ func (s *loadingSeriesChunkRefsSetIterator) singlePassStringify(symbolizedSet sy } // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, - // provided it's not the first and only batch. + // provided it's not the first and only batch used for chunks streaming. // TODO: should we forcibly release the retained first and only batch later? - releaseable := !s.postingsSetIterator.IsFirstAndOnlyBatch() - set := newSeriesChunkRefsSet(len(symbolizedSet.series), releaseable) + isOnlyBatchForChunksStreaming := s.postingsSetIterator.IsFirstAndOnlyBatch() && s.strategy.isForChunksStreaming() + set := newSeriesChunkRefsSet(len(symbolizedSet.series), !isOnlyBatchForChunksStreaming) labelsBuilder := labels.NewScratchBuilder(maxLabelsPerSeries) for _, series := range symbolizedSet.series { @@ -1208,8 +1213,8 @@ func (s *loadingSeriesChunkRefsSetIterator) multiLookupStringify(symbolizedSet s // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, // provided it's not the first and only batch. // TODO: should we forcibly release the retained first and only batch later? - releaseable := !s.postingsSetIterator.IsFirstAndOnlyBatch() - set := newSeriesChunkRefsSet(len(symbolizedSet.series), releaseable) + isOnlyBatchForChunksStreaming := s.postingsSetIterator.IsFirstAndOnlyBatch() && s.strategy.isForChunksStreaming() + set := newSeriesChunkRefsSet(len(symbolizedSet.series), !isOnlyBatchForChunksStreaming) labelsBuilder := labels.NewScratchBuilder(16) for _, series := range symbolizedSet.series { From 8b81435c775600e3dfe5056af77fa8b4f79334c8 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Fri, 3 May 2024 15:37:38 +1000 Subject: [PATCH 16/43] Refactor test to exercise case where chunks streaming is both enabled and disabled --- pkg/storegateway/series_refs_test.go | 106 ++++++++++++++++----------- 1 file changed, 62 insertions(+), 44 deletions(-) diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 92bae78c710..958e9ddfbfb 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1384,55 +1384,73 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { return sortedTestCases[i].name < sortedTestCases[j].name }) + chunksStreamingCases := map[bool]string{ + true: "for chunks streaming", + false: "not for chunks streaming", + } + for _, testCase := range sortedTestCases { testName, tc := testCase.name, testCase.tc t.Run(testName, func(t *testing.T) { - // Setup - blockFactory := defaultTestBlockFactory - if tc.blockFactory != nil { - blockFactory = tc.blockFactory - } - block := blockFactory() - indexr := block.indexReader(selectAllStrategy{}) - postings, _, err := indexr.ExpandedPostings(context.Background(), tc.matchers, newSafeQueryStats()) - require.NoError(t, err) - postingsIterator := newPostingsSetsIterator( - postings, - tc.batchSize, - ) - hasher := tc.seriesHasher - if hasher == nil { - hasher = cachedSeriesHasher{hashcache.NewSeriesHashCache(100).GetBlockCache("")} - } - if tc.strategy == 0 { - tc.strategy = defaultStrategy // the `0` strategy is not usable, so test cases probably meant to not set it - } - loadingIterator := newLoadingSeriesChunkRefsSetIterator( - context.Background(), - postingsIterator, - indexr, - noopCache{}, - newSafeQueryStats(), - block.meta, - tc.shard, - hasher, - tc.strategy, - tc.minT, - tc.maxT, - "t1", - log.NewNopLogger(), - ) + for enableChunksStreaming, name := range chunksStreamingCases { + t.Run(name, func(t *testing.T) { + // Setup + blockFactory := defaultTestBlockFactory + if tc.blockFactory != nil { + blockFactory = tc.blockFactory + } + block := blockFactory() + indexr := block.indexReader(selectAllStrategy{}) + postings, _, err := indexr.ExpandedPostings(context.Background(), tc.matchers, newSafeQueryStats()) + require.NoError(t, err) + postingsIterator := newPostingsSetsIterator( + postings, + tc.batchSize, + ) + hasher := tc.seriesHasher + if hasher == nil { + hasher = cachedSeriesHasher{hashcache.NewSeriesHashCache(100).GetBlockCache("")} + } + + strategy := tc.strategy - // Tests - sets := readAllSeriesChunkRefsSet(loadingIterator) - assert.NoError(t, loadingIterator.Err()) - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, tc.strategy, tc.expectedSets, sets) + if strategy == 0 { + strategy = defaultStrategy // the `0` strategy is not usable, so test cases probably meant to not set it + } - // Ensure that the iterator behaves correctly after a reset. - loadingIterator.Reset() - setsAfterReset := readAllSeriesChunkRefsSet(loadingIterator) - assert.NoError(t, loadingIterator.Err()) - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, tc.strategy, tc.expectedSets, setsAfterReset) + if enableChunksStreaming { + strategy = strategy | forChunksStreaming + } + + loadingIterator := newLoadingSeriesChunkRefsSetIterator( + context.Background(), + postingsIterator, + indexr, + noopCache{}, + newSafeQueryStats(), + block.meta, + tc.shard, + hasher, + strategy, + tc.minT, + tc.maxT, + "t1", + log.NewNopLogger(), + ) + + sets := readAllSeriesChunkRefsSet(loadingIterator) + assert.NoError(t, loadingIterator.Err()) + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, strategy, tc.expectedSets, sets) + + if enableChunksStreaming { + // Ensure that the iterator behaves correctly after a reset. + loadingIterator.Reset() + setsAfterReset := readAllSeriesChunkRefsSet(loadingIterator) + assert.NoError(t, loadingIterator.Err()) + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, strategy, tc.expectedSets, setsAfterReset) + } + }) + } }) } } From 4471f3356280d08add1cd95e7e5c6cb64d594b37 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 13 May 2024 12:18:12 +1000 Subject: [PATCH 17/43] Return reused series set to pool --- pkg/storegateway/series_chunks_test.go | 2 +- pkg/storegateway/series_refs.go | 64 +++++++++++++++++--------- pkg/storegateway/series_refs_test.go | 6 +-- 3 files changed, 47 insertions(+), 25 deletions(-) diff --git a/pkg/storegateway/series_chunks_test.go b/pkg/storegateway/series_chunks_test.go index b5eccb326b7..09bd93a6531 100644 --- a/pkg/storegateway/series_chunks_test.go +++ b/pkg/storegateway/series_chunks_test.go @@ -691,7 +691,7 @@ func BenchmarkLoadingSeriesChunksSetIterator(b *testing.B) { setSeriesOffset := 0 for setIdx := 0; setIdx < numSets; setIdx++ { // This set cannot be released because reused between multiple benchmark runs. - refs := newSeriesChunkRefsSet(numSeriesPerSet, false) + refs := newSeriesChunkRefsSet(numSeriesPerSet, false, false) refs.series = refs.series[:cap(refs.series)] for refIdx := range refs.series { refs.series[refIdx] = testBlk.toSeriesChunkRefs(setSeriesOffset) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 9ae0642d740..b3b62beef03 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -102,11 +102,13 @@ type seriesChunkRefsSet struct { // newSeriesChunkRefsSet creates a new seriesChunkRefsSet with the given capacity. // If releasable is true, then a subsequent call release() will put the internal // series slices to a memory pool for reusing. -func newSeriesChunkRefsSet(capacity int, releasable bool) seriesChunkRefsSet { +// If either releasable or fromPool are true, then the internal series slice is +// taken from a pool. +func newSeriesChunkRefsSet(capacity int, releasable bool, fromPool bool) seriesChunkRefsSet { var prealloc []seriesChunkRefs - // If it's releasable then we try to reuse a slice from the pool. - if releasable { + // If it's releasable or we want to force using the pool, then we try to reuse a slice from the pool. + if releasable || fromPool { if reused := seriesChunkRefsSetPool.Get(); reused != nil { prealloc = *(reused.(*[]seriesChunkRefs)) } @@ -139,6 +141,14 @@ func (b seriesChunkRefsSet) release() { seriesChunkRefsSetPool.Put(&reuse) } +// makeReleasable returns a new seriesChunkRefsSet that can be released on a subsequent call to release. +// +// This is useful for scenarios where a set is used multiple times (eg. during chunks streaming), so initial consumers +// must not release the set, but we know it is safe for the last consumer of the set to release it. +func (b seriesChunkRefsSet) makeReleasable() seriesChunkRefsSet { + return seriesChunkRefsSet{b.series, true} +} + // seriesChunkRefs holds a series with a list of chunk references. type seriesChunkRefs struct { lset labels.Labels @@ -348,7 +358,7 @@ func (s *mergedSeriesChunkRefsSet) Next() bool { // This can be released by the caller because mergedSeriesChunkRefsSet doesn't retain it // after Next() will be called again. - next := newSeriesChunkRefsSet(s.batchSize, true) + next := newSeriesChunkRefsSet(s.batchSize, true, true) for i := 0; i < s.batchSize; i++ { err := s.ensureCursors(s.aAt, s.bAt, s.a, s.b) @@ -598,7 +608,7 @@ func (s *deduplicatingSeriesChunkRefsSetIterator) Next() bool { // This can be released by the caller because deduplicatingSeriesChunkRefsSetIterator doesn't retain it // after Next() will be called again. - nextSet := newSeriesChunkRefsSet(s.batchSize, true) + nextSet := newSeriesChunkRefsSet(s.batchSize, true, true) nextSet.series = append(nextSet.series, firstSeries) var nextSeries seriesChunkRefs @@ -900,7 +910,11 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { if err != nil { level.Warn(s.logger).Log("msg", "could not encode postings for series cache key", "err", err) } else { - if cachedSet, isCached := fetchCachedSeriesForPostings(s.ctx, s.tenantID, s.indexCache, s.blockID, s.shard, cachedSeriesID, s.logger); isCached { + // This set can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, + // unless it's the first and only batch used for chunks streaming. + // If it's the first and only batch used for chunks streaming, we'll make it releasable when Reset() is called. + releaseableSet := !s.isOnlyBatchForChunksStreaming() + if cachedSet, isCached := fetchCachedSeriesForPostings(s.ctx, s.tenantID, s.indexCache, s.blockID, s.shard, cachedSeriesID, releaseableSet, s.logger); isCached { s.currentSet = cachedSet return true } @@ -1129,8 +1143,16 @@ func (s *loadingSeriesChunkRefsSetIterator) Reset() { return } + // Resetting this iterator only makes sense if chunks streaming is enabled. + if !s.strategy.isForChunksStreaming() { + return + } + if s.postingsSetIterator.IsFirstAndOnlyBatch() { s.resetToFirstSet = true + + // We expect that the second iteration through this iterator is the last iteration, so allow releasing the set. + s.currentSet = s.currentSet.makeReleasable() } else { s.currentSet = seriesChunkRefsSet{} } @@ -1187,11 +1209,7 @@ func (s *loadingSeriesChunkRefsSetIterator) singlePassStringify(symbolizedSet sy } } - // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, - // provided it's not the first and only batch used for chunks streaming. - // TODO: should we forcibly release the retained first and only batch later? - isOnlyBatchForChunksStreaming := s.postingsSetIterator.IsFirstAndOnlyBatch() && s.strategy.isForChunksStreaming() - set := newSeriesChunkRefsSet(len(symbolizedSet.series), !isOnlyBatchForChunksStreaming) + set := s.newSeriesChunksRefSet(len(symbolizedSet.series)) labelsBuilder := labels.NewScratchBuilder(maxLabelsPerSeries) for _, series := range symbolizedSet.series { @@ -1210,12 +1228,7 @@ func (s *loadingSeriesChunkRefsSetIterator) singlePassStringify(symbolizedSet sy } func (s *loadingSeriesChunkRefsSetIterator) multiLookupStringify(symbolizedSet symbolizedSeriesChunkRefsSet) (seriesChunkRefsSet, error) { - // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, - // provided it's not the first and only batch. - // TODO: should we forcibly release the retained first and only batch later? - isOnlyBatchForChunksStreaming := s.postingsSetIterator.IsFirstAndOnlyBatch() && s.strategy.isForChunksStreaming() - set := newSeriesChunkRefsSet(len(symbolizedSet.series), !isOnlyBatchForChunksStreaming) - + set := s.newSeriesChunksRefSet(len(symbolizedSet.series)) labelsBuilder := labels.NewScratchBuilder(16) for _, series := range symbolizedSet.series { lset, err := s.indexr.LookupLabelsSymbols(s.ctx, series.lset, &labelsBuilder) @@ -1231,6 +1244,17 @@ func (s *loadingSeriesChunkRefsSetIterator) multiLookupStringify(symbolizedSet s return set, nil } +func (s *loadingSeriesChunkRefsSetIterator) newSeriesChunksRefSet(capacity int) seriesChunkRefsSet { + // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, + // unless it's the first and only batch used for chunks streaming. + // If it's the first and only batch used for chunks streaming, we'll make it releasable when Reset() is called. + return newSeriesChunkRefsSet(capacity, !s.isOnlyBatchForChunksStreaming(), true) +} + +func (s *loadingSeriesChunkRefsSetIterator) isOnlyBatchForChunksStreaming() bool { + return s.postingsSetIterator.IsFirstAndOnlyBatch() && s.strategy.isForChunksStreaming() +} + type filteringSeriesChunkRefsSetIterator struct { stats *safeQueryStats from iterator[seriesChunkRefsSet] @@ -1306,7 +1330,7 @@ func (i cachedSeriesForPostingsID) isSet() bool { return i.postingsKey != "" && len(i.encodedPostings) > 0 } -func fetchCachedSeriesForPostings(ctx context.Context, userID string, indexCache indexcache.IndexCache, blockID ulid.ULID, shard *sharding.ShardSelector, itemID cachedSeriesForPostingsID, logger log.Logger) (seriesChunkRefsSet, bool) { +func fetchCachedSeriesForPostings(ctx context.Context, userID string, indexCache indexcache.IndexCache, blockID ulid.ULID, shard *sharding.ShardSelector, itemID cachedSeriesForPostingsID, releasableSet bool, logger log.Logger) (seriesChunkRefsSet, bool) { data, ok := indexCache.FetchSeriesForPostings(ctx, userID, blockID, shard, itemID.postingsKey) if !ok { return seriesChunkRefsSet{}, false @@ -1327,9 +1351,7 @@ func fetchCachedSeriesForPostings(ctx context.Context, userID string, indexCache return seriesChunkRefsSet{}, false } - // This can be released by the caller because loadingSeriesChunkRefsSetIterator (where this function is called) doesn't retain it - // after Next() will be called again. - res := newSeriesChunkRefsSet(len(entry.Series), true) + res := newSeriesChunkRefsSet(len(entry.Series), releasableSet, true) for _, lset := range entry.Series { res.series = append(res.series, seriesChunkRefs{ lset: mimirpb.FromLabelAdaptersToLabels(lset.Labels), diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 958e9ddfbfb..b0d763c93fc 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1245,7 +1245,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { batchSize: largerTestBlockSeriesCount, matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", ".*")}, expectedSets: func() []seriesChunkRefsSet { - set := newSeriesChunkRefsSet(largerTestBlockSeriesCount, true) + set := newSeriesChunkRefsSet(largerTestBlockSeriesCount, true, true) for i := 0; i < largerTestBlockSeriesCount; i++ { set.series = append(set.series, seriesChunkRefs{lset: labels.FromStrings("l1", fmt.Sprintf("v%d", i))}) } @@ -2442,7 +2442,7 @@ func readAllSeriesChunkRefs(it iterator[seriesChunkRefs]) []seriesChunkRefs { // based on the provided minSeriesID and maxSeriesID (both inclusive). Each series has the ID // incremented by +1. func createSeriesChunkRefsSet(minSeriesID, maxSeriesID int, releasable bool) seriesChunkRefsSet { - set := newSeriesChunkRefsSet(maxSeriesID-minSeriesID+1, releasable) + set := newSeriesChunkRefsSet(maxSeriesID-minSeriesID+1, releasable, releasable) for seriesID := minSeriesID; seriesID <= maxSeriesID; seriesID++ { set.series = append(set.series, seriesChunkRefs{ @@ -2539,7 +2539,7 @@ func BenchmarkFetchCachedSeriesForPostings(b *testing.B) { b.ReportAllocs() b.ResetTimer() for i := 0; i < b.N; i++ { - set, ok := fetchCachedSeriesForPostings(ctx, "tenant-1", mockCache, blockID, testCase.shard, cachedSeriesID, logger) + set, ok := fetchCachedSeriesForPostings(ctx, "tenant-1", mockCache, blockID, testCase.shard, cachedSeriesID, true, logger) assert.Equal(b, testCase.expectedHit, ok) if testCase.expectedHit { assert.NotZero(b, set) From afebea268f8b23abbca3d9b1a8251038e0ace5ea Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 13 May 2024 12:27:35 +1000 Subject: [PATCH 18/43] Fix typo --- pkg/storegateway/series_refs_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index b0d763c93fc..0511215ddd5 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1490,7 +1490,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) } From 5915c1916a999a7072b8f0687e3addd22506b830 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 13 May 2024 13:41:12 +1000 Subject: [PATCH 19/43] Only load chunk refs when they're needed. --- pkg/storegateway/series_refs.go | 27 +++++++++++--- pkg/storegateway/series_refs_test.go | 53 +++++++++++++++++++++++++--- 2 files changed, 70 insertions(+), 10 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index b3b62beef03..bc200fd0c8d 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -835,6 +835,14 @@ func (s seriesIteratorStrategy) isNoChunkRefsAndOverlapMintMaxt() bool { return s.isNoChunkRefs() && s.isOverlapMintMaxt() } +func (s seriesIteratorStrategy) withNoChunkRefs() seriesIteratorStrategy { + return s | noChunkRefs +} + +func (s seriesIteratorStrategy) withoutNoChunkRefs() seriesIteratorStrategy { + return s & ^noChunkRefs +} + func newLoadingSeriesChunkRefsSetIterator( ctx context.Context, postingsSetIterator *postingsSetsIterator, @@ -853,6 +861,11 @@ func newLoadingSeriesChunkRefsSetIterator( if strategy.isOnEntireBlock() { minTime, maxTime = blockMeta.MinTime, blockMeta.MaxTime } + if strategy.isForChunksStreaming() { + // Assume we don't want chunk refs on the first pass through the iterator. + // If it turns out there's only one batch of series, we'll re-enable loading chunk refs in Next() below. + strategy = strategy.withNoChunkRefs() + } return &loadingSeriesChunkRefsSetIterator{ ctx: ctx, postingsSetIterator: postingsSetIterator, @@ -886,6 +899,13 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { return s.currentSet.len() != 0 } + if s.strategy.isForChunksStreaming() { + if s.postingsSetIterator.IsFirstAndOnlyBatch() { + // We must load chunk refs on our first pass. + s.strategy = s.strategy.withoutNoChunkRefs() + } + } + defer func(startTime time.Time) { spanLog := spanlogger.FromContext(s.ctx, s.logger) spanLog.DebugLog( @@ -1159,11 +1179,8 @@ func (s *loadingSeriesChunkRefsSetIterator) Reset() { s.postingsSetIterator.Reset() - s.strategy = defaultStrategy - - // TODO: handle change in strategy - // - When loading first and only batch, and we know we'll want chunks later for streaming: always load chunk refs - // - Otherwise, when resetting, set flag to load chunk refs + // We want to load chunk refs on the second iteration, so that we can send chunks to queriers. + s.strategy = s.strategy.withoutNoChunkRefs() } // loadSeries returns a for chunks. It is not safe to use the returned []chunks.Meta after calling loadSeries again diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 0511215ddd5..589bc1c5a77 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1121,7 +1121,8 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { minT, maxT int64 batchSize int - expectedSets []seriesChunkRefsSet + expectedSets []seriesChunkRefsSet + expectChunkRefsOnFirstAndOnlyBatchWhenStreaming bool } sharedSeriesHasher := cachedSeriesHasher{hashcache.NewSeriesHashCache(1000).GetBlockCache("")} @@ -1191,6 +1192,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { {lset: labels.FromStrings("l1", "v4")}, }}, }, + expectChunkRefsOnFirstAndOnlyBatchWhenStreaming: true, // Some batches will be filtered out, so although only one batch is returned, multiple are created internally. }, "returns no batches when no series are owned by shard": { shard: &sharding.ShardSelector{ShardIndex: 1, ShardCount: 2}, @@ -1222,6 +1224,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { {lset: labels.FromStrings("l1", "v3")}, }}, }, + expectChunkRefsOnFirstAndOnlyBatchWhenStreaming: true, // Some batches will be filtered out, so although only one batch is returned, multiple are created internally. }, "ignores mixT/maxT when skipping chunks": { minT: 0, @@ -1256,7 +1259,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, @@ -1440,14 +1443,31 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { sets := readAllSeriesChunkRefsSet(loadingIterator) assert.NoError(t, loadingIterator.Err()) - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, strategy, tc.expectedSets, sets) if enableChunksStreaming { - // Ensure that the iterator behaves correctly after a reset. + assertionStrategy := strategy + + if len(tc.expectedSets) == 1 && !tc.expectChunkRefsOnFirstAndOnlyBatchWhenStreaming { + // If we expect a single batch, then chunk refs should be present when we iterate through the series the first time. + assertionStrategy = assertionStrategy.withoutNoChunkRefs() + assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t, sets) + } else { + // Otherwise, if multiple batches are expected, then chunk refs should not be present when we iterate through the series the first time. + assertionStrategy = assertionStrategy.withNoChunkRefs() + assertSeriesChunkRefsSetsDoNotHaveChunkRefsPopulated(t, sets) + } + + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, assertionStrategy, tc.expectedSets, sets) + + // Ensure that the iterator behaves correctly after a reset (ie. when streaming chunks to the querier). loadingIterator.Reset() setsAfterReset := readAllSeriesChunkRefsSet(loadingIterator) assert.NoError(t, loadingIterator.Err()) - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, strategy, tc.expectedSets, setsAfterReset) + assertionStrategy = assertionStrategy.withoutNoChunkRefs() // Chunk refs should be present when we iterate through the series after a reset. + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, assertionStrategy, tc.expectedSets, setsAfterReset) + assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t, setsAfterReset) + } else { + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, strategy, tc.expectedSets, sets) } }) } @@ -1502,6 +1522,22 @@ func assertSeriesChunkRefsSetsEqual(t testing.TB, blockID ulid.ULID, blockDir st } } +func assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t *testing.T, sets []seriesChunkRefsSet) { + for _, set := range sets { + for _, s := range set.series { + require.NotEmpty(t, s.refs) + } + } +} + +func assertSeriesChunkRefsSetsDoNotHaveChunkRefsPopulated(t *testing.T, sets []seriesChunkRefsSet) { + for _, set := range sets { + for _, s := range set.series { + require.Empty(t, s.refs) + } + } +} + func assertEqualf[T comparable](t testing.TB, a, b T, msg string, args ...any) { if a != b { t.Helper() @@ -2674,3 +2710,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().withoutNoChunkRefs().isNoChunkRefs()) + require.False(t, defaultStrategy.withoutNoChunkRefs().isNoChunkRefs()) +} From f2dccc712affcf10b621998920c23ec0ab48daeb Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 13 May 2024 13:50:21 +1000 Subject: [PATCH 20/43] Rename variable to make intent clearer --- pkg/storegateway/series_refs_test.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 589bc1c5a77..eccdb052cc7 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1121,8 +1121,8 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { minT, maxT int64 batchSize int - expectedSets []seriesChunkRefsSet - expectChunkRefsOnFirstAndOnlyBatchWhenStreaming bool + expectedSets []seriesChunkRefsSet + expectSomeBatchesFilteredOut bool } sharedSeriesHasher := cachedSeriesHasher{hashcache.NewSeriesHashCache(1000).GetBlockCache("")} @@ -1192,7 +1192,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { {lset: labels.FromStrings("l1", "v4")}, }}, }, - expectChunkRefsOnFirstAndOnlyBatchWhenStreaming: true, // Some batches will be filtered out, so although only one batch is returned, multiple are created internally. + expectSomeBatchesFilteredOut: true, }, "returns no batches when no series are owned by shard": { shard: &sharding.ShardSelector{ShardIndex: 1, ShardCount: 2}, @@ -1224,7 +1224,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { {lset: labels.FromStrings("l1", "v3")}, }}, }, - expectChunkRefsOnFirstAndOnlyBatchWhenStreaming: true, // Some batches will be filtered out, so although only one batch is returned, multiple are created internally. + expectSomeBatchesFilteredOut: true, }, "ignores mixT/maxT when skipping chunks": { minT: 0, @@ -1447,8 +1447,9 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { if enableChunksStreaming { assertionStrategy := strategy - if len(tc.expectedSets) == 1 && !tc.expectChunkRefsOnFirstAndOnlyBatchWhenStreaming { + if len(tc.expectedSets) == 1 && !tc.expectSomeBatchesFilteredOut { // If we expect a single batch, then chunk refs should be present when we iterate through the series the first time. + // We exclude the case where some batches are filtered out internally: in this case, we expect to behave as if there were multiple batches. assertionStrategy = assertionStrategy.withoutNoChunkRefs() assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t, sets) } else { From 103615ed5c0b70d5b51f067c4e26458aab84e268 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 13 May 2024 14:12:43 +1000 Subject: [PATCH 21/43] Add changelog entry --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2b5d392a123..c738f6405a4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ * [CHANGE] Querier: the CLI flag `-querier.minimize-ingester-requests` has been moved from "experimental" to "advanced". #7638 * [ENHANCEMENT] Store-gateway: merge series from different blocks concurrently. #7456 * [ENHANCEMENT] Store-gateway: Add `stage="wait_max_concurrent"` to `cortex_bucket_store_series_request_stage_duration_seconds` which records how long the query had to wait for its turn for `-blocks-storage.bucket-store.max-concurrent`. #7609 +* [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 * [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 * [BUGFIX] Ingester: Prevent timely compaction of empty blocks. #7624 From 9f40abca64bc2c30002b14ab2dda0fca63e41ba0 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Tue, 14 May 2024 15:29:34 +1000 Subject: [PATCH 22/43] Rename `withoutNoChunkRefs` to `withChunkRefs` --- pkg/storegateway/series_refs.go | 6 +++--- pkg/storegateway/series_refs_test.go | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index bc200fd0c8d..8cd091514b8 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -839,7 +839,7 @@ func (s seriesIteratorStrategy) withNoChunkRefs() seriesIteratorStrategy { return s | noChunkRefs } -func (s seriesIteratorStrategy) withoutNoChunkRefs() seriesIteratorStrategy { +func (s seriesIteratorStrategy) withChunkRefs() seriesIteratorStrategy { return s & ^noChunkRefs } @@ -902,7 +902,7 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { if s.strategy.isForChunksStreaming() { if s.postingsSetIterator.IsFirstAndOnlyBatch() { // We must load chunk refs on our first pass. - s.strategy = s.strategy.withoutNoChunkRefs() + s.strategy = s.strategy.withChunkRefs() } } @@ -1180,7 +1180,7 @@ func (s *loadingSeriesChunkRefsSetIterator) Reset() { s.postingsSetIterator.Reset() // We want to load chunk refs on the second iteration, so that we can send chunks to queriers. - s.strategy = s.strategy.withoutNoChunkRefs() + s.strategy = s.strategy.withChunkRefs() } // loadSeries returns a for chunks. It is not safe to use the returned []chunks.Meta after calling loadSeries again diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index bbc40e8ce88..60f1435f991 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1450,7 +1450,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { if len(tc.expectedSets) == 1 && !tc.expectSomeBatchesFilteredOut { // If we expect a single batch, then chunk refs should be present when we iterate through the series the first time. // We exclude the case where some batches are filtered out internally: in this case, we expect to behave as if there were multiple batches. - assertionStrategy = assertionStrategy.withoutNoChunkRefs() + assertionStrategy = assertionStrategy.withChunkRefs() assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t, sets) } else { // Otherwise, if multiple batches are expected, then chunk refs should not be present when we iterate through the series the first time. @@ -1464,7 +1464,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { loadingIterator.Reset() setsAfterReset := readAllSeriesChunkRefsSet(loadingIterator) assert.NoError(t, loadingIterator.Err()) - assertionStrategy = assertionStrategy.withoutNoChunkRefs() // Chunk refs should be present when we iterate through the series after a reset. + assertionStrategy = assertionStrategy.withChunkRefs() // Chunk refs should be present when we iterate through the series after a reset. assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, assertionStrategy, tc.expectedSets, setsAfterReset) assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t, setsAfterReset) } else { @@ -2715,6 +2715,6 @@ func (c mockIndexCache) FetchSeriesForPostings(context.Context, string, ulid.ULI func TestSeriesIteratorStrategy(t *testing.T) { require.False(t, defaultStrategy.isNoChunkRefs()) require.True(t, defaultStrategy.withNoChunkRefs().isNoChunkRefs()) - require.False(t, defaultStrategy.withNoChunkRefs().withoutNoChunkRefs().isNoChunkRefs()) - require.False(t, defaultStrategy.withoutNoChunkRefs().isNoChunkRefs()) + require.False(t, defaultStrategy.withNoChunkRefs().withChunkRefs().isNoChunkRefs()) + require.False(t, defaultStrategy.withChunkRefs().isNoChunkRefs()) } From 3c8e50faab3cc4e256781a849625ed74cc2de2de Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Tue, 14 May 2024 15:30:58 +1000 Subject: [PATCH 23/43] Remove redundant `if` --- pkg/storegateway/series_refs.go | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 8cd091514b8..278b51d9840 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -757,17 +757,9 @@ func openBlockSeriesChunkRefsSetsIterator( return nil, errors.New("set size must be a positive number") } - var ( - ps []storage.SeriesRef - pendingMatchers []*labels.Matcher - fetchPostings = true - ) - if fetchPostings { - var err error - ps, pendingMatchers, err = indexr.ExpandedPostings(ctx, matchers, stats) - if err != nil { - return nil, errors.Wrap(err, "expanded matching postings") - } + ps, pendingMatchers, err := indexr.ExpandedPostings(ctx, matchers, stats) + if err != nil { + return nil, errors.Wrap(err, "expanded matching postings") } var it iterator[seriesChunkRefsSet] From 4f070713126f4fb475d5ad76ecfb2c73ce9bc5f5 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 13:29:35 +1000 Subject: [PATCH 24/43] Add iterator type that wraps an iterator factory to manage the transition between chunks streaming phases. --- pkg/storegateway/series_refs.go | 11 + pkg/storegateway/series_refs_streaming.go | 93 +++++ .../series_refs_streaming_test.go | 346 ++++++++++++++++++ pkg/storegateway/series_refs_test.go | 16 +- 4 files changed, 456 insertions(+), 10 deletions(-) create mode 100644 pkg/storegateway/series_refs_streaming.go create mode 100644 pkg/storegateway/series_refs_streaming_test.go diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 278b51d9840..27c870f8583 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -149,6 +149,13 @@ func (b seriesChunkRefsSet) makeReleasable() seriesChunkRefsSet { return seriesChunkRefsSet{b.series, true} } +// 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 @@ -1491,6 +1498,10 @@ func (s *postingsSetsIterator) Reset() { s.nextBatchPostingsOffset = 0 } +func (s *postingsSetsIterator) HasMultipleBatches() bool { + return len(s.postings) > s.batchSize +} + func (s *postingsSetsIterator) IsFirstAndOnlyBatch() bool { return len(s.postings) > 0 && s.nextBatchPostingsOffset <= s.batchSize && s.nextBatchPostingsOffset >= len(s.postings) } diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go new file mode 100644 index 00000000000..96d952c5005 --- /dev/null +++ b/pkg/storegateway/series_refs_streaming.go @@ -0,0 +1,93 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package storegateway + +type iteratorFactory func(strategy seriesIteratorStrategy) 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 + factory iteratorFactory + 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, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) *chunksStreamingCachingSeriesChunkRefsSetIterator { + expectSingleBatch := !postingsSetsIterator.HasMultipleBatches() + var initialStrategy seriesIteratorStrategy + + if expectSingleBatch { + initialStrategy = strategy.withChunkRefs() + } else { + // We'll load chunk refs during the chunks streaming phase. + initialStrategy = strategy.withNoChunkRefs() + } + + return &chunksStreamingCachingSeriesChunkRefsSetIterator{ + strategy: strategy, + factory: factory, + it: factory(initialStrategy), + 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 { + i.it = i.factory(i.strategy.withChunkRefs()) + } +} + +func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) Reset() { + panic("THIS METHOD SHOULD BE REMOVED FROM THE iterator INTERFACE ALTOGETHER") +} diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go new file mode 100644 index 00000000000..0c69f7632e2 --- /dev/null +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -0,0 +1,346 @@ +// 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) { + psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + + factoryCalls := 0 + var factoryStrategy seriesIteratorStrategy + + firstBatchSeries := []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1")}, + {lset: labels.FromStrings("series", "2")}, + {lset: labels.FromStrings("series", "3")}, + } + + factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + return newSliceSeriesChunkRefsSetIterator( + nil, + seriesChunkRefsSet{ + series: firstBatchSeries, + releasable: true, + }, + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + + // 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 := readAllBatches(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 = readAllBatches(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) { + psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + + factoryCalls := 0 + + unreleasableBatch := seriesChunkRefsSet{ + series: []seriesChunkRefs{ + {lset: labels.FromStrings("series", "1")}, + {lset: labels.FromStrings("series", "2")}, + {lset: labels.FromStrings("series", "3")}, + }, + releasable: false, + } + + factory := func(_ seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + factoryCalls++ + return newSliceSeriesChunkRefsSetIterator(nil, unreleasableBatch) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + + // During label sending phase, the single batch should be returned and not be releasable. + batches := readAllBatches(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 = readAllBatches(it) + require.NoError(t, it.Err()) + require.Equal(t, []seriesChunkRefsSet{unreleasableBatch}, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_AllBatchesFilteredOut(t *testing.T) { + psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + + factoryCalls := 0 + var factoryStrategy seriesIteratorStrategy + + factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + return newSliceSeriesChunkRefsSetIterator( + nil, + // No batches. + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + + // 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 := readAllBatches(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 = readAllBatches(it) + require.NoError(t, it.Err()) + require.Empty(t, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_IteratorReturnsError(t *testing.T) { + psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + factoryCalls := 0 + iteratorError := errors.New("something went wrong") + + factory := func(_ seriesIteratorStrategy) 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, psi, factory) + + // During label sending phase, the error should be returned. + _ = readAllBatches(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. + _ = readAllBatches(it) + require.Equal(t, iteratorError, it.Err()) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_HappyPath(t *testing.T) { + psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 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, + } + + factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + if factoryCalls == 1 { + return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs) + } + + return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithChunkRefs, secondBatchWithChunkRefs) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + + // 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 := readAllBatches(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 = readAllBatches(it) + require.NoError(t, it.Err()) + require.Equal(t, []seriesChunkRefsSet{firstBatchWithChunkRefs, secondBatchWithChunkRefs}, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_AllBatchesFilteredOut(t *testing.T) { + psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) + + factoryCalls := 0 + var factoryStrategy seriesIteratorStrategy + + factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + factoryCalls++ + factoryStrategy = strategy + + return newSliceSeriesChunkRefsSetIterator( + nil, + // No batches. + ) + } + + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + + // 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 := readAllBatches(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 = readAllBatches(it) + require.NoError(t, it.Err()) + require.Empty(t, batches) +} + +func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_IteratorReturnsError(t *testing.T) { + psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) + factoryCalls := 0 + + factory := func(_ seriesIteratorStrategy) 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, psi, factory) + require.Equal(t, 1, factoryCalls) + + // During label sending phase, the error from the original iterator should be returned. + _ = readAllBatches(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. + _ = readAllBatches(it) + require.EqualError(t, it.Err(), "error #2") +} + +func readAllBatches(it iterator[seriesChunkRefsSet]) []seriesChunkRefsSet { + var batches []seriesChunkRefsSet + + for it.Next() { + batches = append(batches, it.At()) + } + + return batches +} diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 60f1435f991..472e4443e4f 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -2356,22 +2356,18 @@ func TestPostingsSetsIterator(t *testing.T) { t.Run(testName, func(t *testing.T) { iterator := newPostingsSetsIterator(testCase.postings, testCase.batchSize) + if len(testCase.expectedBatches) > 1 { + require.True(t, iterator.HasMultipleBatches()) + } else { + require.False(t, iterator.HasMultipleBatches()) + } + var actualBatches [][]storage.SeriesRef for iterator.Next() { actualBatches = append(actualBatches, iterator.At()) - - if len(testCase.expectedBatches) != 1 { - assert.False(t, iterator.IsFirstAndOnlyBatch()) - } } assert.ElementsMatch(t, testCase.expectedBatches, actualBatches) - - if len(testCase.expectedBatches) == 1 { - assert.True(t, iterator.IsFirstAndOnlyBatch()) - } else if len(testCase.expectedBatches) == 0 { - assert.False(t, iterator.IsFirstAndOnlyBatch()) - } }) } } From cc40e1c4496cc5a2827183061fdb8e3c1058b14a Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 13:36:23 +1000 Subject: [PATCH 25/43] Remove `Reset` methods --- pkg/storegateway/series_chunks.go | 13 -- pkg/storegateway/series_chunks_test.go | 10 -- pkg/storegateway/series_refs.go | 89 ++----------- pkg/storegateway/series_refs_streaming.go | 4 - pkg/storegateway/series_refs_test.go | 146 ++++++---------------- 5 files changed, 50 insertions(+), 212 deletions(-) diff --git a/pkg/storegateway/series_chunks.go b/pkg/storegateway/series_chunks.go index c7e9dfce8d8..04504af57f7 100644 --- a/pkg/storegateway/series_chunks.go +++ b/pkg/storegateway/series_chunks.go @@ -60,7 +60,6 @@ type iterator[V any] interface { Next() bool At() V Err() error - Reset() } // seriesChunksSet holds a set of series, each with its own chunks. @@ -288,10 +287,6 @@ func (p *preloadingSetIterator[Set]) Err() error { return p.err } -func (p *preloadingSetIterator[Set]) Reset() { - p.from.Reset() -} - func newPreloadingAndStatsTrackingSetIterator[Set any](ctx context.Context, preloadedSetsCount int, iterator iterator[Set], stats *safeQueryStats) iterator[Set] { // Track the time spent loading batches (including preloading). numBatches := 0 @@ -438,10 +433,6 @@ func (c *loadingSeriesChunksSetIterator) Err() error { return c.err } -func (c *loadingSeriesChunksSetIterator) Reset() { - c.from.Reset() -} - func (c *loadingSeriesChunksSetIterator) recordReturnedChunks(series []seriesChunks) { returnedChunks, returnedChunksBytes := chunkStats(series) @@ -507,7 +498,3 @@ func (m *nextDurationMeasuringIterator[Set]) At() Set { func (m *nextDurationMeasuringIterator[Set]) Err() error { return m.from.Err() } - -func (m *nextDurationMeasuringIterator[Set]) Reset() { - m.from.Reset() -} diff --git a/pkg/storegateway/series_chunks_test.go b/pkg/storegateway/series_chunks_test.go index 09bd93a6531..5201958ea70 100644 --- a/pkg/storegateway/series_chunks_test.go +++ b/pkg/storegateway/series_chunks_test.go @@ -884,12 +884,6 @@ func (s *sliceSeriesChunksSetIterator) Err() error { return nil } -func (s *sliceSeriesChunksSetIterator) Reset() { - s.current = -1 - s.err = nil - s.errAt = 0 -} - // delayedIterator implements iterator and introduces an artificial delay before returning from Next() and At(). type delayedIterator[S any] struct { wrapped iterator[S] @@ -917,10 +911,6 @@ func (s *delayedIterator[S]) Err() error { return s.wrapped.Err() } -func (s *delayedIterator[S]) Reset() { - s.wrapped.Reset() -} - func generateAggrChunk(num int) []storepb.AggrChunk { out := make([]storepb.AggrChunk, 0, num) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 27c870f8583..e06d43ebdec 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -296,17 +296,12 @@ func (c flattenedSeriesChunkRefsIterator) Err() error { return c.from.Err() } -func (c flattenedSeriesChunkRefsIterator) Reset() { - c.from.Reset() -} - type emptySeriesChunkRefsSetIterator struct { } func (emptySeriesChunkRefsSetIterator) Next() bool { return false } func (emptySeriesChunkRefsSetIterator) At() seriesChunkRefsSet { return seriesChunkRefsSet{} } func (emptySeriesChunkRefsSetIterator) Err() error { return nil } -func (emptySeriesChunkRefsSetIterator) Reset() {} func mergedSeriesChunkRefsSetIterators(mergedBatchSize int, all ...iterator[seriesChunkRefsSet]) iterator[seriesChunkRefsSet] { switch len(all) { @@ -396,14 +391,6 @@ func (s *mergedSeriesChunkRefsSet) Next() bool { return true } -func (s *mergedSeriesChunkRefsSet) Reset() { - s.a.Reset() - s.b.Reset() - s.done = false - s.aAt = newSeriesChunkRefsIterator(seriesChunkRefsSet{}) - s.bAt = newSeriesChunkRefsIterator(seriesChunkRefsSet{}) -} - func (s *mergedSeriesChunkRefsSet) ensureCursors(curr1, curr2 *seriesChunkRefsIterator, set1, set2 iterator[seriesChunkRefsSet]) error { // When both cursors are empty, we advance their set iterators concurrently to reduce total waiting time for the // IO from underlying set iterators (see grafana/mimir#4596). @@ -641,21 +628,11 @@ func (s *deduplicatingSeriesChunkRefsSetIterator) Next() bool { return true } -func (s *deduplicatingSeriesChunkRefsSetIterator) Reset() { - s.from.Reset() - s.peek = nil -} - type limitingSeriesChunkRefsSetIterator struct { from iterator[seriesChunkRefsSet] chunksLimiter ChunksLimiter seriesLimiter SeriesLimiter - chunksSeen uint64 // Number of chunks seen since last call to Reset() - maxChunksSeenEver uint64 // Maximum number of chunks ever seen across calls to Reset(), used to avoid double-counting after Reset() - seriesSeen uint64 // Number of series seen since last call to Reset() - maxSeriesSeenEver uint64 // Maximum number of chunks ever seen across calls to Reset(), used to avoid double-counting after Reset() - err error currentBatch seriesChunkRefsSet } @@ -679,32 +656,21 @@ func (l *limitingSeriesChunkRefsSetIterator) Next() bool { } l.currentBatch = l.from.At() - l.seriesSeen += uint64(l.currentBatch.len()) - - if l.seriesSeen > l.maxSeriesSeenEver { - newSeries := l.seriesSeen - l.maxSeriesSeenEver - - if err := l.seriesLimiter.Reserve(newSeries); err != nil { - l.err = err - return false - } - - l.maxSeriesSeenEver = l.seriesSeen + err := l.seriesLimiter.Reserve(uint64(l.currentBatch.len())) + if err != nil { + l.err = err + return false } + var totalChunks int for _, s := range l.currentBatch.series { - l.chunksSeen += uint64(len(s.refs)) + totalChunks += len(s.refs) } - if l.chunksSeen > l.maxChunksSeenEver { - newChunks := l.chunksSeen - l.maxChunksSeenEver - - if err := l.chunksLimiter.Reserve(newChunks); err != nil { - l.err = err - return false - } - - l.maxChunksSeenEver = l.chunksSeen + err = l.chunksLimiter.Reserve(uint64(totalChunks)) + if err != nil { + l.err = err + return false } return true @@ -718,12 +684,6 @@ func (l *limitingSeriesChunkRefsSetIterator) Err() error { return l.err } -func (l *limitingSeriesChunkRefsSetIterator) Reset() { - l.chunksSeen = 0 - l.seriesSeen = 0 - l.from.Reset() -} - type loadingSeriesChunkRefsSetIterator struct { ctx context.Context postingsSetIterator *postingsSetsIterator @@ -1157,31 +1117,6 @@ func (s *loadingSeriesChunkRefsSetIterator) Err() error { return s.err } -func (s *loadingSeriesChunkRefsSetIterator) Reset() { - if s.err != nil { - return - } - - // Resetting this iterator only makes sense if chunks streaming is enabled. - if !s.strategy.isForChunksStreaming() { - return - } - - if s.postingsSetIterator.IsFirstAndOnlyBatch() { - s.resetToFirstSet = true - - // We expect that the second iteration through this iterator is the last iteration, so allow releasing the set. - s.currentSet = s.currentSet.makeReleasable() - } else { - s.currentSet = seriesChunkRefsSet{} - } - - s.postingsSetIterator.Reset() - - // We want to load chunk refs on the second iteration, so that we can send chunks to queriers. - s.strategy = s.strategy.withChunkRefs() -} - // loadSeries returns a for chunks. It is not safe to use the returned []chunks.Meta after calling loadSeries again func (s *loadingSeriesChunkRefsSetIterator) loadSeries(ref storage.SeriesRef, loadedSeries *bucketIndexLoadedSeries, stats *queryStats, lsetPool *pool.SlabPool[symbolizedLabel]) ([]symbolizedLabel, []chunks.Meta, error) { ok, lbls, err := loadedSeries.unsafeLoadSeries(ref, &s.chunkMetasBuffer, s.strategy.isNoChunkRefsOnEntireBlock(), stats, lsetPool) @@ -1329,10 +1264,6 @@ func (m *filteringSeriesChunkRefsSetIterator) Err() error { return m.from.Err() } -func (m *filteringSeriesChunkRefsSetIterator) Reset() { - m.from.Reset() -} - // cachedSeriesForPostingsID contains enough information to be able to tell whether a cache entry // is the right cache entry that we are looking for. We store only the postingsKey in the // cache key because the encoded postings are too big. We store the encoded postings within diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index 96d952c5005..130f94bd303 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -87,7 +87,3 @@ func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) PrepareForChunksStrea i.it = i.factory(i.strategy.withChunkRefs()) } } - -func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) Reset() { - panic("THIS METHOD SHOULD BE REMOVED FROM THE iterator INTERFACE ALTOGETHER") -} diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 472e4443e4f..995179dcf98 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1121,8 +1121,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { minT, maxT int64 batchSize int - expectedSets []seriesChunkRefsSet - expectSomeBatchesFilteredOut bool + expectedSets []seriesChunkRefsSet } sharedSeriesHasher := cachedSeriesHasher{hashcache.NewSeriesHashCache(1000).GetBlockCache("")} @@ -1192,7 +1191,6 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { {lset: labels.FromStrings("l1", "v4")}, }}, }, - expectSomeBatchesFilteredOut: true, }, "returns no batches when no series are owned by shard": { shard: &sharding.ShardSelector{ShardIndex: 1, ShardCount: 2}, @@ -1224,7 +1222,6 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { {lset: labels.FromStrings("l1", "v3")}, }}, }, - expectSomeBatchesFilteredOut: true, }, "ignores mixT/maxT when skipping chunks": { minT: 0, @@ -1387,91 +1384,49 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { return sortedTestCases[i].name < sortedTestCases[j].name }) - chunksStreamingCases := map[bool]string{ - true: "for chunks streaming", - false: "not for chunks streaming", - } - for _, testCase := range sortedTestCases { testName, tc := testCase.name, testCase.tc t.Run(testName, func(t *testing.T) { - for enableChunksStreaming, name := range chunksStreamingCases { - t.Run(name, func(t *testing.T) { - // Setup - blockFactory := defaultTestBlockFactory - if tc.blockFactory != nil { - blockFactory = tc.blockFactory - } - block := blockFactory() - indexr := block.indexReader(selectAllStrategy{}) - postings, _, err := indexr.ExpandedPostings(context.Background(), tc.matchers, newSafeQueryStats()) - require.NoError(t, err) - postingsIterator := newPostingsSetsIterator( - postings, - tc.batchSize, - ) - hasher := tc.seriesHasher - if hasher == nil { - hasher = cachedSeriesHasher{hashcache.NewSeriesHashCache(100).GetBlockCache("")} - } - - strategy := tc.strategy - - if strategy == 0 { - strategy = defaultStrategy // the `0` strategy is not usable, so test cases probably meant to not set it - } - - if enableChunksStreaming { - strategy = strategy | forChunksStreaming - } - - loadingIterator := newLoadingSeriesChunkRefsSetIterator( - context.Background(), - postingsIterator, - indexr, - noopCache{}, - newSafeQueryStats(), - block.meta, - tc.shard, - hasher, - strategy, - tc.minT, - tc.maxT, - "t1", - log.NewNopLogger(), - ) - - sets := readAllSeriesChunkRefsSet(loadingIterator) - assert.NoError(t, loadingIterator.Err()) - - if enableChunksStreaming { - assertionStrategy := strategy - - if len(tc.expectedSets) == 1 && !tc.expectSomeBatchesFilteredOut { - // If we expect a single batch, then chunk refs should be present when we iterate through the series the first time. - // We exclude the case where some batches are filtered out internally: in this case, we expect to behave as if there were multiple batches. - assertionStrategy = assertionStrategy.withChunkRefs() - assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t, sets) - } else { - // Otherwise, if multiple batches are expected, then chunk refs should not be present when we iterate through the series the first time. - assertionStrategy = assertionStrategy.withNoChunkRefs() - assertSeriesChunkRefsSetsDoNotHaveChunkRefsPopulated(t, sets) - } - - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, assertionStrategy, tc.expectedSets, sets) - - // Ensure that the iterator behaves correctly after a reset (ie. when streaming chunks to the querier). - loadingIterator.Reset() - setsAfterReset := readAllSeriesChunkRefsSet(loadingIterator) - assert.NoError(t, loadingIterator.Err()) - assertionStrategy = assertionStrategy.withChunkRefs() // Chunk refs should be present when we iterate through the series after a reset. - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, assertionStrategy, tc.expectedSets, setsAfterReset) - assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t, setsAfterReset) - } else { - assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, strategy, tc.expectedSets, sets) - } - }) + // Setup + blockFactory := defaultTestBlockFactory + if tc.blockFactory != nil { + blockFactory = tc.blockFactory } + block := blockFactory() + indexr := block.indexReader(selectAllStrategy{}) + postings, _, err := indexr.ExpandedPostings(context.Background(), tc.matchers, newSafeQueryStats()) + require.NoError(t, err) + postingsIterator := newPostingsSetsIterator( + postings, + tc.batchSize, + ) + hasher := tc.seriesHasher + if hasher == nil { + hasher = cachedSeriesHasher{hashcache.NewSeriesHashCache(100).GetBlockCache("")} + } + if tc.strategy == 0 { + tc.strategy = defaultStrategy // the `0` strategy is not usable, so test cases probably meant to not set it + } + loadingIterator := newLoadingSeriesChunkRefsSetIterator( + context.Background(), + postingsIterator, + indexr, + noopCache{}, + newSafeQueryStats(), + block.meta, + tc.shard, + hasher, + tc.strategy, + tc.minT, + tc.maxT, + "t1", + log.NewNopLogger(), + ) + + // Tests + sets := readAllSeriesChunkRefsSet(loadingIterator) + assert.NoError(t, loadingIterator.Err()) + assertSeriesChunkRefsSetsEqual(t, block.meta.ULID, block.bkt.(localBucket).dir, tc.minT, tc.maxT, tc.strategy, tc.expectedSets, sets) }) } } @@ -1523,22 +1478,6 @@ func assertSeriesChunkRefsSetsEqual(t testing.TB, blockID ulid.ULID, blockDir st } } -func assertSeriesChunkRefsSetsHaveChunkRefsPopulated(t *testing.T, sets []seriesChunkRefsSet) { - for _, set := range sets { - for _, s := range set.series { - require.NotEmpty(t, s.refs) - } - } -} - -func assertSeriesChunkRefsSetsDoNotHaveChunkRefsPopulated(t *testing.T, sets []seriesChunkRefsSet) { - for _, set := range sets { - for _, s := range set.series { - require.Empty(t, s.refs) - } - } -} - func assertEqualf[T comparable](t testing.TB, a, b T, msg string, args ...any) { if a != b { t.Helper() @@ -2421,11 +2360,6 @@ func (s *sliceSeriesChunkRefsSetIterator) Err() error { return nil } -func (s *sliceSeriesChunkRefsSetIterator) Reset() { - s.current = -1 - s.err = nil -} - type staticLimiter struct { limit int msg string From 61290ed1af12a03c4fa3d8166cf8f2cf2024157e Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 13:37:32 +1000 Subject: [PATCH 26/43] Remove `makeReleasable` --- pkg/storegateway/series_refs.go | 8 -------- 1 file changed, 8 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index e06d43ebdec..c2ff981877d 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -141,14 +141,6 @@ func (b seriesChunkRefsSet) release() { seriesChunkRefsSetPool.Put(&reuse) } -// makeReleasable returns a new seriesChunkRefsSet that can be released on a subsequent call to release. -// -// This is useful for scenarios where a set is used multiple times (eg. during chunks streaming), so initial consumers -// must not release the set, but we know it is safe for the last consumer of the set to release it. -func (b seriesChunkRefsSet) makeReleasable() seriesChunkRefsSet { - return seriesChunkRefsSet{b.series, true} -} - // 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. From 5f0342c236bd81c756126052a721ed3b04e1b5b6 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 13:42:16 +1000 Subject: [PATCH 27/43] Unpick more unnecessary changes --- pkg/storegateway/series_chunks_test.go | 2 +- pkg/storegateway/series_refs.go | 81 ++++++-------------------- pkg/storegateway/series_refs_test.go | 6 +- 3 files changed, 21 insertions(+), 68 deletions(-) diff --git a/pkg/storegateway/series_chunks_test.go b/pkg/storegateway/series_chunks_test.go index 5201958ea70..5cdfb6cea43 100644 --- a/pkg/storegateway/series_chunks_test.go +++ b/pkg/storegateway/series_chunks_test.go @@ -691,7 +691,7 @@ func BenchmarkLoadingSeriesChunksSetIterator(b *testing.B) { setSeriesOffset := 0 for setIdx := 0; setIdx < numSets; setIdx++ { // This set cannot be released because reused between multiple benchmark runs. - refs := newSeriesChunkRefsSet(numSeriesPerSet, false, false) + refs := newSeriesChunkRefsSet(numSeriesPerSet, false) refs.series = refs.series[:cap(refs.series)] for refIdx := range refs.series { refs.series[refIdx] = testBlk.toSeriesChunkRefs(setSeriesOffset) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index c2ff981877d..77612ef2f99 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -102,13 +102,11 @@ type seriesChunkRefsSet struct { // newSeriesChunkRefsSet creates a new seriesChunkRefsSet with the given capacity. // If releasable is true, then a subsequent call release() will put the internal // series slices to a memory pool for reusing. -// If either releasable or fromPool are true, then the internal series slice is -// taken from a pool. -func newSeriesChunkRefsSet(capacity int, releasable bool, fromPool bool) seriesChunkRefsSet { +func newSeriesChunkRefsSet(capacity int, releasable bool) seriesChunkRefsSet { var prealloc []seriesChunkRefs - // If it's releasable or we want to force using the pool, then we try to reuse a slice from the pool. - if releasable || fromPool { + // If it's releasable then we try to reuse a slice from the pool. + if releasable { if reused := seriesChunkRefsSetPool.Get(); reused != nil { prealloc = *(reused.(*[]seriesChunkRefs)) } @@ -352,7 +350,7 @@ func (s *mergedSeriesChunkRefsSet) Next() bool { // This can be released by the caller because mergedSeriesChunkRefsSet doesn't retain it // after Next() will be called again. - next := newSeriesChunkRefsSet(s.batchSize, true, true) + next := newSeriesChunkRefsSet(s.batchSize, true) for i := 0; i < s.batchSize; i++ { err := s.ensureCursors(s.aAt, s.bAt, s.a, s.b) @@ -594,7 +592,7 @@ func (s *deduplicatingSeriesChunkRefsSetIterator) Next() bool { // This can be released by the caller because deduplicatingSeriesChunkRefsSetIterator doesn't retain it // after Next() will be called again. - nextSet := newSeriesChunkRefsSet(s.batchSize, true, true) + nextSet := newSeriesChunkRefsSet(s.batchSize, true) nextSet.series = append(nextSet.series, firstSeries) var nextSeries seriesChunkRefs @@ -692,9 +690,8 @@ type loadingSeriesChunkRefsSetIterator struct { chunkMetasBuffer []chunks.Meta - err error - currentSet seriesChunkRefsSet - resetToFirstSet bool + err error + currentSet seriesChunkRefsSet } func openBlockSeriesChunkRefsSetsIterator( @@ -759,9 +756,6 @@ const ( // overlapMintMaxt flag is used together with noChunkRefs. With this, only the series whose // chunks overlap with [mint, maxt] are selected. overlapMintMaxt seriesIteratorStrategy = 0b00000010 - // forChunksStreaming flag is used when the iterator is being used for store-gateway to querier chunks - // streaming. It enables optimisations for the case where only a single batch is needed for all series. - forChunksStreaming seriesIteratorStrategy = 0b00000100 ) func (s seriesIteratorStrategy) isNoChunkRefs() bool { @@ -772,8 +766,6 @@ func (s seriesIteratorStrategy) isOverlapMintMaxt() bool { return s&overlapMintMaxt != 0 } -func (s seriesIteratorStrategy) isForChunksStreaming() bool { return s&forChunksStreaming != 0 } - func (s seriesIteratorStrategy) isOnEntireBlock() bool { return !s.isOverlapMintMaxt() } @@ -812,11 +804,6 @@ func newLoadingSeriesChunkRefsSetIterator( if strategy.isOnEntireBlock() { minTime, maxTime = blockMeta.MinTime, blockMeta.MaxTime } - if strategy.isForChunksStreaming() { - // Assume we don't want chunk refs on the first pass through the iterator. - // If it turns out there's only one batch of series, we'll re-enable loading chunk refs in Next() below. - strategy = strategy.withNoChunkRefs() - } return &loadingSeriesChunkRefsSetIterator{ ctx: ctx, postingsSetIterator: postingsSetIterator, @@ -842,21 +829,6 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { return false } - if s.resetToFirstSet { - // The first set is already loaded as s.currentSet, so nothing more to do. - s.resetToFirstSet = false - - // If the first batch of postings had series, but they've all been filtered out, then we are done. - return s.currentSet.len() != 0 - } - - if s.strategy.isForChunksStreaming() { - if s.postingsSetIterator.IsFirstAndOnlyBatch() { - // We must load chunk refs on our first pass. - s.strategy = s.strategy.withChunkRefs() - } - } - defer func(startTime time.Time) { spanLog := spanlogger.FromContext(s.ctx, s.logger) spanLog.DebugLog( @@ -881,11 +853,7 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { if err != nil { level.Warn(s.logger).Log("msg", "could not encode postings for series cache key", "err", err) } else { - // This set can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, - // unless it's the first and only batch used for chunks streaming. - // If it's the first and only batch used for chunks streaming, we'll make it releasable when Reset() is called. - releaseableSet := !s.isOnlyBatchForChunksStreaming() - if cachedSet, isCached := fetchCachedSeriesForPostings(s.ctx, s.tenantID, s.indexCache, s.blockID, s.shard, cachedSeriesID, releaseableSet, s.logger); isCached { + if cachedSet, isCached := fetchCachedSeriesForPostings(s.ctx, s.tenantID, s.indexCache, s.blockID, s.shard, cachedSeriesID, s.logger); isCached { s.currentSet = cachedSet return true } @@ -1152,7 +1120,8 @@ func (s *loadingSeriesChunkRefsSetIterator) singlePassStringify(symbolizedSet sy } } - set := s.newSeriesChunksRefSet(len(symbolizedSet.series)) + // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again. + set := newSeriesChunkRefsSet(len(symbolizedSet.series), true) labelsBuilder := labels.NewScratchBuilder(maxLabelsPerSeries) for _, series := range symbolizedSet.series { @@ -1171,7 +1140,9 @@ func (s *loadingSeriesChunkRefsSetIterator) singlePassStringify(symbolizedSet sy } func (s *loadingSeriesChunkRefsSetIterator) multiLookupStringify(symbolizedSet symbolizedSeriesChunkRefsSet) (seriesChunkRefsSet, error) { - set := s.newSeriesChunksRefSet(len(symbolizedSet.series)) + // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again. + set := newSeriesChunkRefsSet(len(symbolizedSet.series), true) + labelsBuilder := labels.NewScratchBuilder(16) for _, series := range symbolizedSet.series { lset, err := s.indexr.LookupLabelsSymbols(s.ctx, series.lset, &labelsBuilder) @@ -1187,17 +1158,6 @@ func (s *loadingSeriesChunkRefsSetIterator) multiLookupStringify(symbolizedSet s return set, nil } -func (s *loadingSeriesChunkRefsSetIterator) newSeriesChunksRefSet(capacity int) seriesChunkRefsSet { - // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it after Next() is called again, - // unless it's the first and only batch used for chunks streaming. - // If it's the first and only batch used for chunks streaming, we'll make it releasable when Reset() is called. - return newSeriesChunkRefsSet(capacity, !s.isOnlyBatchForChunksStreaming(), true) -} - -func (s *loadingSeriesChunkRefsSetIterator) isOnlyBatchForChunksStreaming() bool { - return s.postingsSetIterator.IsFirstAndOnlyBatch() && s.strategy.isForChunksStreaming() -} - type filteringSeriesChunkRefsSetIterator struct { stats *safeQueryStats from iterator[seriesChunkRefsSet] @@ -1269,7 +1229,7 @@ func (i cachedSeriesForPostingsID) isSet() bool { return i.postingsKey != "" && len(i.encodedPostings) > 0 } -func fetchCachedSeriesForPostings(ctx context.Context, userID string, indexCache indexcache.IndexCache, blockID ulid.ULID, shard *sharding.ShardSelector, itemID cachedSeriesForPostingsID, releasableSet bool, logger log.Logger) (seriesChunkRefsSet, bool) { +func fetchCachedSeriesForPostings(ctx context.Context, userID string, indexCache indexcache.IndexCache, blockID ulid.ULID, shard *sharding.ShardSelector, itemID cachedSeriesForPostingsID, logger log.Logger) (seriesChunkRefsSet, bool) { data, ok := indexCache.FetchSeriesForPostings(ctx, userID, blockID, shard, itemID.postingsKey) if !ok { return seriesChunkRefsSet{}, false @@ -1290,7 +1250,9 @@ func fetchCachedSeriesForPostings(ctx context.Context, userID string, indexCache return seriesChunkRefsSet{}, false } - res := newSeriesChunkRefsSet(len(entry.Series), releasableSet, true) + // This can be released by the caller because loadingSeriesChunkRefsSetIterator (where this function is called) doesn't retain it + // after Next() will be called again. + res := newSeriesChunkRefsSet(len(entry.Series), true) for _, lset := range entry.Series { res.series = append(res.series, seriesChunkRefs{ lset: mimirpb.FromLabelAdaptersToLabels(lset.Labels), @@ -1416,15 +1378,6 @@ func (s *postingsSetsIterator) At() []storage.SeriesRef { return s.currentBatch } -func (s *postingsSetsIterator) Reset() { - s.currentBatch = nil - s.nextBatchPostingsOffset = 0 -} - func (s *postingsSetsIterator) HasMultipleBatches() bool { return len(s.postings) > s.batchSize } - -func (s *postingsSetsIterator) IsFirstAndOnlyBatch() bool { - return len(s.postings) > 0 && s.nextBatchPostingsOffset <= s.batchSize && s.nextBatchPostingsOffset >= len(s.postings) -} diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 995179dcf98..974c3ef631c 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1245,7 +1245,7 @@ func TestLoadingSeriesChunkRefsSetIterator(t *testing.T) { batchSize: largerTestBlockSeriesCount, matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "l1", ".*")}, expectedSets: func() []seriesChunkRefsSet { - set := newSeriesChunkRefsSet(largerTestBlockSeriesCount, true, true) + set := newSeriesChunkRefsSet(largerTestBlockSeriesCount, true) for i := 0; i < largerTestBlockSeriesCount; i++ { set.series = append(set.series, seriesChunkRefs{lset: labels.FromStrings("l1", fmt.Sprintf("v%d", i))}) } @@ -2409,7 +2409,7 @@ func readAllSeriesChunkRefs(it iterator[seriesChunkRefs]) []seriesChunkRefs { // based on the provided minSeriesID and maxSeriesID (both inclusive). Each series has the ID // incremented by +1. func createSeriesChunkRefsSet(minSeriesID, maxSeriesID int, releasable bool) seriesChunkRefsSet { - set := newSeriesChunkRefsSet(maxSeriesID-minSeriesID+1, releasable, releasable) + set := newSeriesChunkRefsSet(maxSeriesID-minSeriesID+1, releasable) for seriesID := minSeriesID; seriesID <= maxSeriesID; seriesID++ { set.series = append(set.series, seriesChunkRefs{ @@ -2506,7 +2506,7 @@ func BenchmarkFetchCachedSeriesForPostings(b *testing.B) { b.ReportAllocs() b.ResetTimer() for i := 0; i < b.N; i++ { - set, ok := fetchCachedSeriesForPostings(ctx, "tenant-1", mockCache, blockID, testCase.shard, cachedSeriesID, true, logger) + set, ok := fetchCachedSeriesForPostings(ctx, "tenant-1", mockCache, blockID, testCase.shard, cachedSeriesID, logger) assert.Equal(b, testCase.expectedHit, ok) if testCase.expectedHit { assert.NotZero(b, set) From b24982ddf84097ace25c415f3f8d763e63621ee8 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 13:43:47 +1000 Subject: [PATCH 28/43] Unpick whitespace change --- pkg/storegateway/series_refs.go | 1 - 1 file changed, 1 deletion(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 77612ef2f99..c77caa6b1ef 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -662,7 +662,6 @@ func (l *limitingSeriesChunkRefsSetIterator) Next() bool { l.err = err return false } - return true } From 814d627bea9404c05fc6d118c807ddd228eeb029 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 13:59:20 +1000 Subject: [PATCH 29/43] Extract method --- pkg/storegateway/bucket.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index a6d3c527bbf..66ee63b6317 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1169,13 +1169,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) { From 77ac30b26ea08d1d5f404458027650ab3eb5cdf2 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 16:14:38 +1000 Subject: [PATCH 30/43] Refactor existing code to use `chunksStreamingCachingSeriesChunkRefsSetIterator` --- pkg/storegateway/bucket.go | 74 ++++++++++++++------ pkg/storegateway/bucket_chunk_reader_test.go | 1 + pkg/storegateway/series_refs.go | 34 ++++++++- pkg/storegateway/series_refs_test.go | 5 ++ 4 files changed, 92 insertions(+), 22 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 66ee63b6317..415f601a2c4 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. - reuseIterator iterator[seriesChunkRefsSet] - 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, reuseIterator, err = s.streamingSeriesForBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) + seriesSet, streamingIterators, err = s.streamingSeriesForBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats) if err != nil { return err } @@ -661,11 +659,7 @@ 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, readers, stats, reuseIterator) - if err != nil { - return err - } + seriesChunkIt := s.streamingChunksSetForBlocks(ctx, readers, stats, chunksLimiter, seriesLimiter, streamingIterators) err = s.sendStreamingChunks(req, srv, seriesChunkIt, stats, streamingSeriesCount) } else { var seriesSet storepb.SeriesSet @@ -1044,7 +1038,7 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( if req.SkipChunks { strategy = noChunkRefs } - it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy) + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy, nil) if err != nil { return nil, err } @@ -1061,7 +1055,7 @@ func (s *BucketStore) nonStreamingSeriesSetForBlocks( // streamingSeriesForBlocks is used when streaming feature is enabled. // It returns a series set that only contains the series labels without any chunks information. -// The iterator should be re-used when getting chunks to save on computation. +// The streamingSeriesIterators should be re-used when getting chunks to save on computation. func (s *BucketStore) streamingSeriesForBlocks( ctx context.Context, req *storepb.SeriesRequest, @@ -1072,13 +1066,46 @@ func (s *BucketStore) streamingSeriesForBlocks( chunksLimiter ChunksLimiter, // Rate limiter for loading chunks. seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, -) (storepb.SeriesSet, iterator[seriesChunkRefsSet], error) { - var strategy = overlapMintMaxt | forChunksStreaming - it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, strategy) +) (storepb.SeriesSet, *streamingSeriesIterators, error) { + streamingIterators := newStreamingSeriesIterators() + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, overlapMintMaxt, streamingIterators.iteratorWrapper) if err != nil { return nil, nil, err } - return newSeriesSetWithoutChunks(ctx, it, stats), it, nil + + return newSeriesSetWithoutChunks(ctx, it, stats), streamingIterators, nil +} + +type streamingSeriesIterators struct { + iterators []*chunksStreamingCachingSeriesChunkRefsSetIterator + mtx *sync.RWMutex +} + +func newStreamingSeriesIterators() *streamingSeriesIterators { + return &streamingSeriesIterators{ + mtx: &sync.RWMutex{}, + } +} + +func (i *streamingSeriesIterators) iteratorWrapper(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] { + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIterator, factory) + + 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 } // streamingChunksSetForBlocks is used when streaming feature is enabled. @@ -1088,11 +1115,15 @@ func (s *BucketStore) streamingChunksSetForBlocks( ctx context.Context, chunkReaders *bucketChunkReaders, stats *safeQueryStats, - it iterator[seriesChunkRefsSet], // Should come from streamingSeriesForBlocks. -) (iterator[seriesChunksSet], error) { - it.Reset() + 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( @@ -1106,6 +1137,7 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, strategy seriesIteratorStrategy, + wrapper func(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet], ) (iterator[seriesChunkRefsSet], error) { var ( mtx = sync.Mutex{} @@ -1141,6 +1173,7 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( req.MinTime, req.MaxTime, stats, s.logger, + wrapper, ) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) @@ -1449,6 +1482,7 @@ func blockLabelNames(ctx context.Context, indexr *bucketIndexReader, matchers [] minTime, maxTime, stats, 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 3095df1912a..ba96344881c 100644 --- a/pkg/storegateway/bucket_chunk_reader_test.go +++ b/pkg/storegateway/bucket_chunk_reader_test.go @@ -45,6 +45,7 @@ func TestBucketChunkReader_refetchChunks(t *testing.T) { block.meta.MaxTime, newSafeQueryStats(), log.NewNopLogger(), + nil, ) require.NoError(t, err) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index c77caa6b1ef..20f9f6aa640 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -707,6 +707,7 @@ func openBlockSeriesChunkRefsSetsIterator( minTime, maxTime int64, // Series must have data in this time range to be returned (ignored if skipChunks=true). stats *safeQueryStats, logger log.Logger, + wrapper func(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet], // Optional function called to wrap created iterators. ) (iterator[seriesChunkRefsSet], error) { if batchSize <= 0 { return nil, errors.New("set size must be a positive number") @@ -717,10 +718,38 @@ func openBlockSeriesChunkRefsSetsIterator( return nil, errors.Wrap(err, "expanded matching postings") } + psi := newPostingsSetsIterator(ps, batchSize) + + factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + return openBlockSeriesChunkRefsSetsIteratorFromPostings(ctx, tenantID, indexr, indexCache, blockMeta, shard, seriesHasher, strategy, minTime, maxTime, stats, psi, pendingMatchers, logger) + } + + if wrapper == nil { + return factory(strategy), nil + } + + return wrapper(strategy, psi, factory), nil +} + +func openBlockSeriesChunkRefsSetsIteratorFromPostings( + ctx context.Context, + tenantID string, + indexr *bucketIndexReader, // Index reader for block. + indexCache indexcache.IndexCache, + blockMeta *block.Meta, + shard *sharding.ShardSelector, // Shard selector. + seriesHasher seriesHasher, + strategy seriesIteratorStrategy, + minTime, maxTime int64, // Series must have data in this time range to be returned (ignored if skipChunks=true). + 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, @@ -733,11 +762,12 @@ func openBlockSeriesChunkRefsSetsIterator( tenantID, logger, ) + if len(pendingMatchers) > 0 { it = newFilteringSeriesChunkRefsSetIterator(pendingMatchers, it, stats) } - return it, nil + return it } // seriesIteratorStrategy defines the strategy to use when loading the series and their chunk refs. diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 974c3ef631c..f2b94e5e4fe 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -1704,6 +1704,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator(t *testing.T) { maxT, newSafeQueryStats(), log.NewNopLogger(), + nil, ) require.NoError(t, err) @@ -1804,6 +1805,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_pendingMatchers(t *testing.T) { block.meta.MaxTime, newSafeQueryStats(), log.NewNopLogger(), + nil, ) require.NoError(t, err) allSets := readAllSeriesChunkRefsSet(iterator) @@ -1867,6 +1869,7 @@ func BenchmarkOpenBlockSeriesChunkRefsSetsIterator(b *testing.B) { block.meta.MaxTime, newSafeQueryStats(), log.NewNopLogger(), + nil, ) require.NoError(b, err) @@ -2195,6 +2198,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { b.meta.MaxTime, statsColdCache, log.NewNopLogger(), + nil, ) require.NoError(t, err) @@ -2225,6 +2229,7 @@ func TestOpenBlockSeriesChunkRefsSetsIterator_SeriesCaching(t *testing.T) { b.meta.MaxTime, statsWarmCache, log.NewNopLogger(), + nil, ) require.NoError(t, err) lset = extractLabelsFromSeriesChunkRefsSets(readAllSeriesChunkRefsSet(ss)) From 43115a948a260078848c8a895298dea7aa6f4e28 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 16:33:09 +1000 Subject: [PATCH 31/43] Fix issue where streaming chunks does not work if multiple batches of series are loaded --- pkg/storegateway/series_refs.go | 5 +++++ pkg/storegateway/series_refs_streaming.go | 18 +++++++++++------- pkg/storegateway/series_refs_streaming_test.go | 7 +++++++ pkg/storegateway/series_refs_test.go | 13 +++++++++++-- 4 files changed, 34 insertions(+), 9 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 20f9f6aa640..ed9e0923d29 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -1410,3 +1410,8 @@ func (s *postingsSetsIterator) At() []storage.SeriesRef { func (s *postingsSetsIterator) HasMultipleBatches() bool { return len(s.postings) > s.batchSize } + +func (s *postingsSetsIterator) Reset() { + s.currentBatch = nil + s.nextBatchPostingsOffset = 0 +} diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index 130f94bd303..edca0ce8180 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -9,9 +9,10 @@ type iteratorFactory func(strategy seriesIteratorStrategy) iterator[seriesChunkR // 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 - factory iteratorFactory - it iterator[seriesChunkRefsSet] + strategy seriesIteratorStrategy + postingsSetsIterator *postingsSetsIterator + factory iteratorFactory + it iterator[seriesChunkRefsSet] expectSingleBatch bool inChunksStreamingPhaseForSingleBatch bool @@ -33,10 +34,11 @@ func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIterator } return &chunksStreamingCachingSeriesChunkRefsSetIterator{ - strategy: strategy, - factory: factory, - it: factory(initialStrategy), - expectSingleBatch: expectSingleBatch, + strategy: strategy, + postingsSetsIterator: postingsSetsIterator, + factory: factory, + it: factory(initialStrategy), + expectSingleBatch: expectSingleBatch, } } @@ -80,6 +82,8 @@ func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) Err() error { } func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) PrepareForChunksStreamingPhase() { + i.postingsSetsIterator.Reset() + if i.expectSingleBatch { i.inChunksStreamingPhaseForSingleBatch = true i.currentBatchIndex = -1 diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go index 0c69f7632e2..403e207d4a6 100644 --- a/pkg/storegateway/series_refs_streaming_test.go +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -228,6 +228,8 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch factoryCalls++ factoryStrategy = strategy + require.Equal(t, 0, psi.nextBatchPostingsOffset, "should reset postings set iterator before creating iterator") + if factoryCalls == 1 { return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs) } @@ -247,6 +249,11 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.NoError(t, it.Err()) require.Equal(t, []seriesChunkRefsSet{firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs}, batches) + // Simulate the underlying iterator advancing the postings set iterator to the end. + for psi.Next() { + // Nothing to do, we just want to advance. + } + // Prepare for chunks streaming phase. Inner iterator should be recreated with chunk refs enabled. it.PrepareForChunksStreamingPhase() require.Equal(t, 2, factoryCalls) diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index f2b94e5e4fe..550af19a358 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -2291,7 +2291,7 @@ func TestPostingsSetsIterator(t *testing.T) { "empty postings": { postings: []storage.SeriesRef{}, batchSize: 2, - expectedBatches: [][]storage.SeriesRef{}, + expectedBatches: nil, }, } @@ -2311,7 +2311,16 @@ func TestPostingsSetsIterator(t *testing.T) { actualBatches = append(actualBatches, iterator.At()) } - assert.ElementsMatch(t, testCase.expectedBatches, actualBatches) + require.Equal(t, testCase.expectedBatches, actualBatches) + + iterator.Reset() + + var actualBatchesAfterReset [][]storage.SeriesRef + for iterator.Next() { + actualBatchesAfterReset = append(actualBatchesAfterReset, iterator.At()) + } + + require.Equal(t, testCase.expectedBatches, actualBatchesAfterReset) }) } } From 3c5eefeef9f592fa5f5a08fc2a373ee2dd8116ec Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Wed, 15 May 2024 16:35:06 +1000 Subject: [PATCH 32/43] Rename methods to better reflect their purpose --- pkg/storegateway/bucket.go | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 415f601a2c4..dc18203244d 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -631,7 +631,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storegatewaypb.Stor seriesLimiter = s.seriesLimiterFactory(s.metrics.queriesDropped.WithLabelValues("series")) ) - seriesSet, streamingIterators, 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 } @@ -659,11 +659,11 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storegatewaypb.Stor start := time.Now() if req.StreamingChunksBatchSize > 0 { - seriesChunkIt := s.streamingChunksSetForBlocks(ctx, readers, stats, chunksLimiter, seriesLimiter, streamingIterators) + 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 } @@ -1021,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, @@ -1053,10 +1053,10 @@ 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 streamingSeriesIterators should be re-used when getting chunks to save on computation. -func (s *BucketStore) streamingSeriesForBlocks( +func (s *BucketStore) createIteratorForChunksStreamingLabelsPhase( ctx context.Context, req *storepb.SeriesRequest, blocks []*bucketBlock, @@ -1108,10 +1108,10 @@ func (i *streamingSeriesIterators) prepareForChunksStreamingPhase() []iterator[s return prepared } -// 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 iterator returned by the streamingSeriesForBlocks call for reuse. -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, chunkReaders *bucketChunkReaders, stats *safeQueryStats, From 93b6d9f1224ca25b6fd77599ed1a8d11a728d5f4 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 12:29:45 +1000 Subject: [PATCH 33/43] Move `streamingSeriesIterators` to `series_refs_streaming.go` --- pkg/storegateway/bucket.go | 32 -------------------- pkg/storegateway/series_refs_streaming.go | 36 +++++++++++++++++++++++ 2 files changed, 36 insertions(+), 32 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index dc18203244d..6e285797c30 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1076,38 +1076,6 @@ func (s *BucketStore) createIteratorForChunksStreamingLabelsPhase( return newSeriesSetWithoutChunks(ctx, it, stats), streamingIterators, nil } -type streamingSeriesIterators struct { - iterators []*chunksStreamingCachingSeriesChunkRefsSetIterator - mtx *sync.RWMutex -} - -func newStreamingSeriesIterators() *streamingSeriesIterators { - return &streamingSeriesIterators{ - mtx: &sync.RWMutex{}, - } -} - -func (i *streamingSeriesIterators) iteratorWrapper(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] { - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIterator, factory) - - 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 -} - // 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. diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index edca0ce8180..8dc0e2f410e 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -2,6 +2,8 @@ package storegateway +import "sync" + type iteratorFactory func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] // chunksStreamingCachingSeriesChunkRefsSetIterator is an iterator used while streaming chunks from store-gateways to queriers. @@ -91,3 +93,37 @@ func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) PrepareForChunksStrea i.it = i.factory(i.strategy.withChunkRefs()) } } + +// 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) iteratorWrapper(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] { + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIterator, factory) + + 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 +} From c92b9a3491c22813c88b2269ab8202f15758796a Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 12:30:52 +1000 Subject: [PATCH 34/43] Remove unnecessary comments --- pkg/storegateway/series_refs.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index ed9e0923d29..c12d4def787 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -734,13 +734,13 @@ func openBlockSeriesChunkRefsSetsIterator( func openBlockSeriesChunkRefsSetsIteratorFromPostings( ctx context.Context, tenantID string, - indexr *bucketIndexReader, // Index reader for block. + indexr *bucketIndexReader, indexCache indexcache.IndexCache, blockMeta *block.Meta, - shard *sharding.ShardSelector, // Shard selector. + 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, postingsSetsIterator *postingsSetsIterator, pendingMatchers []*labels.Matcher, From 6ca5bc9925f68302a0ea762207b6882513c61446 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 12:33:13 +1000 Subject: [PATCH 35/43] Remove unnecessary test helper function --- .../series_refs_streaming_test.go | 38 +++++++------------ 1 file changed, 14 insertions(+), 24 deletions(-) diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go index 403e207d4a6..6bd0ed04845 100644 --- a/pkg/storegateway/series_refs_streaming_test.go +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -45,7 +45,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H require.NoError(t, it.Err()) // During label sending phase, the single batch should be returned and not be releasable. - batches := readAllBatches(it) + batches := readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) unreleasableBatch := seriesChunkRefsSet{ @@ -60,7 +60,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H require.NoError(t, it.Err()) // During chunks streaming phase, the single batch should be returned but be releasable this time. - batches = readAllBatches(it) + batches = readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) releasableBatch := seriesChunkRefsSet{ @@ -92,7 +92,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) // During label sending phase, the single batch should be returned and not be releasable. - batches := readAllBatches(it) + batches := readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Equal(t, []seriesChunkRefsSet{unreleasableBatch}, batches) @@ -102,7 +102,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I require.NoError(t, it.Err()) // During chunks streaming phase, the single batch should be returned and still not be releasable. - batches = readAllBatches(it) + batches = readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Equal(t, []seriesChunkRefsSet{unreleasableBatch}, batches) } @@ -131,7 +131,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_A require.NoError(t, it.Err()) // Label sending phase. - batches := readAllBatches(it) + batches := readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Empty(t, batches) @@ -141,7 +141,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_A require.NoError(t, it.Err()) // Chunks streaming phase. - batches = readAllBatches(it) + batches = readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Empty(t, batches) } @@ -170,7 +170,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) // During label sending phase, the error should be returned. - _ = readAllBatches(it) + _ = readAllSeriesChunkRefsSet(it) require.Equal(t, iteratorError, it.Err()) // Prepare for chunks streaming phase. Inner iterator should not be recreated. @@ -178,7 +178,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I require.Equal(t, 1, factoryCalls) // During chunks streaming phase, the error should be returned. - _ = readAllBatches(it) + _ = readAllSeriesChunkRefsSet(it) require.Equal(t, iteratorError, it.Err()) } @@ -245,7 +245,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.NoError(t, it.Err()) // During label sending phase, the batches should be returned as-is. - batches := readAllBatches(it) + batches := readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Equal(t, []seriesChunkRefsSet{firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs}, batches) @@ -261,7 +261,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.NoError(t, it.Err()) // During chunks streaming phase, the batches should be returned as-is from the new iterator. - batches = readAllBatches(it) + batches = readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Equal(t, []seriesChunkRefsSet{firstBatchWithChunkRefs, secondBatchWithChunkRefs}, batches) } @@ -290,7 +290,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.NoError(t, it.Err()) // Label sending phase. - batches := readAllBatches(it) + batches := readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Empty(t, batches) @@ -301,7 +301,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.NoError(t, it.Err()) // Chunks streaming phase. - batches = readAllBatches(it) + batches = readAllSeriesChunkRefsSet(it) require.NoError(t, it.Err()) require.Empty(t, batches) } @@ -330,7 +330,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.Equal(t, 1, factoryCalls) // During label sending phase, the error from the original iterator should be returned. - _ = readAllBatches(it) + _ = readAllSeriesChunkRefsSet(it) require.EqualError(t, it.Err(), "error #1") // Prepare for chunks streaming phase. Inner iterator should be recreated. @@ -338,16 +338,6 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.Equal(t, 2, factoryCalls) // During chunks streaming phase, the error from the new iterator should be returned. - _ = readAllBatches(it) + _ = readAllSeriesChunkRefsSet(it) require.EqualError(t, it.Err(), "error #2") } - -func readAllBatches(it iterator[seriesChunkRefsSet]) []seriesChunkRefsSet { - var batches []seriesChunkRefsSet - - for it.Next() { - batches = append(batches, it.At()) - } - - return batches -} From 17bb03c1a3288532683a57f188053e48a37e3883 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 12:38:57 +1000 Subject: [PATCH 36/43] Introduce `seriesChunkRefsIteratorWrapper` interface instead of passing function handles around, remove more unnecessary comments --- pkg/storegateway/bucket.go | 12 ++++++------ pkg/storegateway/series_refs.go | 12 ++++++------ pkg/storegateway/series_refs_streaming.go | 6 +++++- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 6e285797c30..61f44efdb67 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1030,8 +1030,8 @@ func (s *BucketStore) createIteratorForNonChunksStreamingRequest( 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 @@ -1063,12 +1063,12 @@ func (s *BucketStore) createIteratorForChunksStreamingLabelsPhase( 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, *streamingSeriesIterators, error) { streamingIterators := newStreamingSeriesIterators() - it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, overlapMintMaxt, streamingIterators.iteratorWrapper) + it, err := s.getSeriesIteratorFromBlocks(ctx, req, blocks, indexReaders, shardSelector, matchers, chunksLimiter, seriesLimiter, stats, overlapMintMaxt, streamingIterators) if err != nil { return nil, nil, err } @@ -1105,7 +1105,7 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, strategy seriesIteratorStrategy, - wrapper func(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet], + wrapper seriesChunkRefsIteratorWrapper, ) (iterator[seriesChunkRefsSet], error) { var ( mtx = sync.Mutex{} diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index c12d4def787..c4e0c39394c 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -697,17 +697,17 @@ 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, logger log.Logger, - wrapper func(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet], // Optional function called to wrap created iterators. + wrapper seriesChunkRefsIteratorWrapper, ) (iterator[seriesChunkRefsSet], error) { if batchSize <= 0 { return nil, errors.New("set size must be a positive number") @@ -728,7 +728,7 @@ func openBlockSeriesChunkRefsSetsIterator( return factory(strategy), nil } - return wrapper(strategy, psi, factory), nil + return wrapper.wrapIterator(strategy, psi, factory), nil } func openBlockSeriesChunkRefsSetsIteratorFromPostings( diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index 8dc0e2f410e..3b79893f349 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -107,7 +107,7 @@ func newStreamingSeriesIterators() *streamingSeriesIterators { } } -func (i *streamingSeriesIterators) iteratorWrapper(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] { +func (i *streamingSeriesIterators) wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] { it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIterator, factory) i.mtx.Lock() @@ -127,3 +127,7 @@ func (i *streamingSeriesIterators) prepareForChunksStreamingPhase() []iterator[s return prepared } + +type seriesChunkRefsIteratorWrapper interface { + wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] +} From 81b1975158bc3dc7bfda5e9e8d314b6104ed0b7e Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 12:39:50 +1000 Subject: [PATCH 37/43] Rename type --- pkg/storegateway/series_refs_streaming.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index 3b79893f349..614c47a1b17 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -4,7 +4,7 @@ package storegateway import "sync" -type iteratorFactory func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] +type seriesChunkRefsIteratorFactory func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] // chunksStreamingCachingSeriesChunkRefsSetIterator is an iterator used while streaming chunks from store-gateways to queriers. // @@ -13,7 +13,7 @@ type iteratorFactory func(strategy seriesIteratorStrategy) iterator[seriesChunkR type chunksStreamingCachingSeriesChunkRefsSetIterator struct { strategy seriesIteratorStrategy postingsSetsIterator *postingsSetsIterator - factory iteratorFactory + factory seriesChunkRefsIteratorFactory it iterator[seriesChunkRefsSet] expectSingleBatch bool @@ -24,7 +24,7 @@ type chunksStreamingCachingSeriesChunkRefsSetIterator struct { cachedBatch seriesChunkRefsSet } -func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) *chunksStreamingCachingSeriesChunkRefsSetIterator { +func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory seriesChunkRefsIteratorFactory) *chunksStreamingCachingSeriesChunkRefsSetIterator { expectSingleBatch := !postingsSetsIterator.HasMultipleBatches() var initialStrategy seriesIteratorStrategy @@ -107,7 +107,7 @@ func newStreamingSeriesIterators() *streamingSeriesIterators { } } -func (i *streamingSeriesIterators) wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] { +func (i *streamingSeriesIterators) wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory seriesChunkRefsIteratorFactory) iterator[seriesChunkRefsSet] { it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIterator, factory) i.mtx.Lock() @@ -129,5 +129,5 @@ func (i *streamingSeriesIterators) prepareForChunksStreamingPhase() []iterator[s } type seriesChunkRefsIteratorWrapper interface { - wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory iteratorFactory) iterator[seriesChunkRefsSet] + wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory seriesChunkRefsIteratorFactory) iterator[seriesChunkRefsSet] } From 60f3084df971e0c41859e1dfa34392a4ef63e6ac Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 13:30:05 +1000 Subject: [PATCH 38/43] Make a copy of the postings for each phase. --- pkg/storegateway/series_refs.go | 22 ++++---- pkg/storegateway/series_refs_streaming.go | 43 +++++++-------- .../series_refs_streaming_test.go | 54 +++++++++---------- pkg/storegateway/series_refs_test.go | 9 ---- 4 files changed, 61 insertions(+), 67 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index c4e0c39394c..8c2d5be79d0 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -718,17 +718,24 @@ func openBlockSeriesChunkRefsSetsIterator( return nil, errors.Wrap(err, "expanded matching postings") } - psi := newPostingsSetsIterator(ps, batchSize) - - factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + 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 wrapper == nil { - return factory(strategy), nil + psi := newPostingsSetsIterator(ps, batchSize) + return iteratorFactory(strategy, psi), nil + } + + postingsSetsIteratorFactory := func() *postingsSetsIterator { + // Create a copy of ps so that any modifications aren't persisted for a later chunks streaming phase. + // For example, loadingSeriesChunkRefsSetIterator removes series that don't match the selected shard. + duplicatePS := make([]storage.SeriesRef, len(ps)) + copy(duplicatePS, ps) + return newPostingsSetsIterator(duplicatePS, batchSize) } - return wrapper.wrapIterator(strategy, psi, factory), nil + return wrapper.wrapIterator(strategy, postingsSetsIteratorFactory, iteratorFactory), nil } func openBlockSeriesChunkRefsSetsIteratorFromPostings( @@ -1410,8 +1417,3 @@ func (s *postingsSetsIterator) At() []storage.SeriesRef { func (s *postingsSetsIterator) HasMultipleBatches() bool { return len(s.postings) > s.batchSize } - -func (s *postingsSetsIterator) Reset() { - s.currentBatch = nil - s.nextBatchPostingsOffset = 0 -} diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index 614c47a1b17..decbeb9ea78 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -4,17 +4,22 @@ package storegateway import "sync" -type seriesChunkRefsIteratorFactory func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] +type seriesChunkRefsIteratorFactory func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] +type postingsSetsIteratorFactory func() *postingsSetsIterator + +type seriesChunkRefsIteratorWrapper interface { + wrapIterator(strategy seriesIteratorStrategy, postingsSetsIteratorFactory postingsSetsIteratorFactory, factory seriesChunkRefsIteratorFactory) 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 - postingsSetsIterator *postingsSetsIterator - factory seriesChunkRefsIteratorFactory - it iterator[seriesChunkRefsSet] + strategy seriesIteratorStrategy + postingsSetsIteratorFactory postingsSetsIteratorFactory + iteratorFactory seriesChunkRefsIteratorFactory + it iterator[seriesChunkRefsSet] expectSingleBatch bool inChunksStreamingPhaseForSingleBatch bool @@ -24,8 +29,9 @@ type chunksStreamingCachingSeriesChunkRefsSetIterator struct { cachedBatch seriesChunkRefsSet } -func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory seriesChunkRefsIteratorFactory) *chunksStreamingCachingSeriesChunkRefsSetIterator { - expectSingleBatch := !postingsSetsIterator.HasMultipleBatches() +func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIteratorStrategy, postingsSetsIteratorFactory postingsSetsIteratorFactory, iteratorFactory seriesChunkRefsIteratorFactory) *chunksStreamingCachingSeriesChunkRefsSetIterator { + psi := postingsSetsIteratorFactory() + expectSingleBatch := !psi.HasMultipleBatches() var initialStrategy seriesIteratorStrategy if expectSingleBatch { @@ -36,11 +42,11 @@ func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIterator } return &chunksStreamingCachingSeriesChunkRefsSetIterator{ - strategy: strategy, - postingsSetsIterator: postingsSetsIterator, - factory: factory, - it: factory(initialStrategy), - expectSingleBatch: expectSingleBatch, + strategy: strategy, + postingsSetsIteratorFactory: postingsSetsIteratorFactory, + iteratorFactory: iteratorFactory, + it: iteratorFactory(initialStrategy, psi), + expectSingleBatch: expectSingleBatch, } } @@ -84,13 +90,12 @@ func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) Err() error { } func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) PrepareForChunksStreamingPhase() { - i.postingsSetsIterator.Reset() - if i.expectSingleBatch { i.inChunksStreamingPhaseForSingleBatch = true i.currentBatchIndex = -1 } else { - i.it = i.factory(i.strategy.withChunkRefs()) + psi := i.postingsSetsIteratorFactory() + i.it = i.iteratorFactory(i.strategy.withChunkRefs(), psi) } } @@ -107,8 +112,8 @@ func newStreamingSeriesIterators() *streamingSeriesIterators { } } -func (i *streamingSeriesIterators) wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory seriesChunkRefsIteratorFactory) iterator[seriesChunkRefsSet] { - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIterator, factory) +func (i *streamingSeriesIterators) wrapIterator(strategy seriesIteratorStrategy, postingsSetsIteratorFactory postingsSetsIteratorFactory, iteratorFactory seriesChunkRefsIteratorFactory) iterator[seriesChunkRefsSet] { + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIteratorFactory, iteratorFactory) i.mtx.Lock() i.iterators = append(i.iterators, it) @@ -127,7 +132,3 @@ func (i *streamingSeriesIterators) prepareForChunksStreamingPhase() []iterator[s return prepared } - -type seriesChunkRefsIteratorWrapper interface { - wrapIterator(strategy seriesIteratorStrategy, postingsSetsIterator *postingsSetsIterator, factory seriesChunkRefsIteratorFactory) iterator[seriesChunkRefsSet] -} diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go index 6bd0ed04845..23181a89bdf 100644 --- a/pkg/storegateway/series_refs_streaming_test.go +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -13,7 +13,7 @@ import ( ) func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_HappyPath(t *testing.T) { - psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } factoryCalls := 0 var factoryStrategy seriesIteratorStrategy @@ -24,7 +24,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H {lset: labels.FromStrings("series", "3")}, } - factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ factoryStrategy = strategy @@ -37,7 +37,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) // Inner iterator should be created with chunk refs enabled. require.Equal(t, 1, factoryCalls) @@ -71,7 +71,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_InnerIteratorReturnsUnreleasableSet(t *testing.T) { - psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } factoryCalls := 0 @@ -84,12 +84,12 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I releasable: false, } - factory := func(_ seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + iteratorFactory := func(_ seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ return newSliceSeriesChunkRefsSetIterator(nil, unreleasableBatch) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) // During label sending phase, the single batch should be returned and not be releasable. batches := readAllSeriesChunkRefsSet(it) @@ -108,12 +108,12 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_AllBatchesFilteredOut(t *testing.T) { - psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } factoryCalls := 0 var factoryStrategy seriesIteratorStrategy - factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ factoryStrategy = strategy @@ -123,7 +123,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_A ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) // Inner iterator should be created with chunk refs enabled. require.Equal(t, 1, factoryCalls) @@ -147,11 +147,11 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_A } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_IteratorReturnsError(t *testing.T) { - psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) + postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } factoryCalls := 0 iteratorError := errors.New("something went wrong") - factory := func(_ seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + iteratorFactory := func(_ seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ return newSliceSeriesChunkRefsSetIterator( @@ -167,7 +167,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) // During label sending phase, the error should be returned. _ = readAllSeriesChunkRefsSet(it) @@ -183,7 +183,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_HappyPath(t *testing.T) { - psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) + postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) } factoryCalls := 0 var factoryStrategy seriesIteratorStrategy @@ -224,20 +224,25 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch releasable: true, } - factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ factoryStrategy = strategy - require.Equal(t, 0, psi.nextBatchPostingsOffset, "should reset postings set iterator before creating iterator") + 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. + for psi.Next() { + // Nothing to do, we just want to advance. + } + return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs) } return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithChunkRefs, secondBatchWithChunkRefs) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) // Inner iterator should be created with chunk refs disabled. require.Equal(t, 1, factoryCalls) @@ -249,11 +254,6 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch require.NoError(t, it.Err()) require.Equal(t, []seriesChunkRefsSet{firstBatchWithNoChunkRefs, secondBatchWithNoChunkRefs}, batches) - // Simulate the underlying iterator advancing the postings set iterator to the end. - for psi.Next() { - // Nothing to do, we just want to advance. - } - // Prepare for chunks streaming phase. Inner iterator should be recreated with chunk refs enabled. it.PrepareForChunksStreamingPhase() require.Equal(t, 2, factoryCalls) @@ -267,12 +267,12 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_AllBatchesFilteredOut(t *testing.T) { - psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) + postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) } factoryCalls := 0 var factoryStrategy seriesIteratorStrategy - factory := func(strategy seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ factoryStrategy = strategy @@ -282,7 +282,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) // Inner iterator should be created with chunk refs disabled. require.Equal(t, 1, factoryCalls) @@ -307,10 +307,10 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_IteratorReturnsError(t *testing.T) { - psi := newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) + postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) } factoryCalls := 0 - factory := func(_ seriesIteratorStrategy) iterator[seriesChunkRefsSet] { + iteratorFactory := func(_ seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ return newSliceSeriesChunkRefsSetIterator( @@ -326,7 +326,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, psi, factory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) require.Equal(t, 1, factoryCalls) // During label sending phase, the error from the original iterator should be returned. diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index 550af19a358..c0aa90bd4c8 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -2312,15 +2312,6 @@ func TestPostingsSetsIterator(t *testing.T) { } require.Equal(t, testCase.expectedBatches, actualBatches) - - iterator.Reset() - - var actualBatchesAfterReset [][]storage.SeriesRef - for iterator.Next() { - actualBatchesAfterReset = append(actualBatchesAfterReset, iterator.At()) - } - - require.Equal(t, testCase.expectedBatches, actualBatchesAfterReset) }) } } From c0fb6e9b51e9d8410a1f188a47b3561fb6b43464 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 13:30:48 +1000 Subject: [PATCH 39/43] Clarify comment --- pkg/storegateway/series_refs_streaming_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go index 23181a89bdf..636205b7120 100644 --- a/pkg/storegateway/series_refs_streaming_test.go +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -231,7 +231,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch 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. + // 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. } From e3d16b109f22f2dd8762fdde8e2c041c2eca1200 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 13:33:07 +1000 Subject: [PATCH 40/43] Use concrete type --- pkg/storegateway/bucket.go | 4 ++-- pkg/storegateway/series_refs.go | 6 +++--- pkg/storegateway/series_refs_streaming.go | 4 ---- 3 files changed, 5 insertions(+), 9 deletions(-) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 61f44efdb67..6788dd4b957 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -1105,7 +1105,7 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( seriesLimiter SeriesLimiter, // Rate limiter for loading series. stats *safeQueryStats, strategy seriesIteratorStrategy, - wrapper seriesChunkRefsIteratorWrapper, + streamingIterators *streamingSeriesIterators, ) (iterator[seriesChunkRefsSet], error) { var ( mtx = sync.Mutex{} @@ -1141,7 +1141,7 @@ func (s *BucketStore) getSeriesIteratorFromBlocks( req.MinTime, req.MaxTime, stats, s.logger, - wrapper, + streamingIterators, ) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 8c2d5be79d0..2b263de8e15 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -707,7 +707,7 @@ func openBlockSeriesChunkRefsSetsIterator( minTime, maxTime int64, stats *safeQueryStats, logger log.Logger, - wrapper seriesChunkRefsIteratorWrapper, + streamingIterators *streamingSeriesIterators, ) (iterator[seriesChunkRefsSet], error) { if batchSize <= 0 { return nil, errors.New("set size must be a positive number") @@ -722,7 +722,7 @@ func openBlockSeriesChunkRefsSetsIterator( return openBlockSeriesChunkRefsSetsIteratorFromPostings(ctx, tenantID, indexr, indexCache, blockMeta, shard, seriesHasher, strategy, minTime, maxTime, stats, psi, pendingMatchers, logger) } - if wrapper == nil { + if streamingIterators == nil { psi := newPostingsSetsIterator(ps, batchSize) return iteratorFactory(strategy, psi), nil } @@ -735,7 +735,7 @@ func openBlockSeriesChunkRefsSetsIterator( return newPostingsSetsIterator(duplicatePS, batchSize) } - return wrapper.wrapIterator(strategy, postingsSetsIteratorFactory, iteratorFactory), nil + return streamingIterators.wrapIterator(strategy, postingsSetsIteratorFactory, iteratorFactory), nil } func openBlockSeriesChunkRefsSetsIteratorFromPostings( diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index decbeb9ea78..2783f18f272 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -7,10 +7,6 @@ import "sync" type seriesChunkRefsIteratorFactory func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] type postingsSetsIteratorFactory func() *postingsSetsIterator -type seriesChunkRefsIteratorWrapper interface { - wrapIterator(strategy seriesIteratorStrategy, postingsSetsIteratorFactory postingsSetsIteratorFactory, factory seriesChunkRefsIteratorFactory) 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, From e711d4959ee73d17f3bece8ac715659fd5dbc925 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 14:10:12 +1000 Subject: [PATCH 41/43] Don't make a copy of the postings unnecessarily. --- pkg/storegateway/series_refs.go | 10 +--- pkg/storegateway/series_refs_streaming.go | 48 ++++++++++++------- .../series_refs_streaming_test.go | 40 ++++++++-------- 3 files changed, 54 insertions(+), 44 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 2b263de8e15..f9de23dd2a2 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -727,15 +727,7 @@ func openBlockSeriesChunkRefsSetsIterator( return iteratorFactory(strategy, psi), nil } - postingsSetsIteratorFactory := func() *postingsSetsIterator { - // Create a copy of ps so that any modifications aren't persisted for a later chunks streaming phase. - // For example, loadingSeriesChunkRefsSetIterator removes series that don't match the selected shard. - duplicatePS := make([]storage.SeriesRef, len(ps)) - copy(duplicatePS, ps) - return newPostingsSetsIterator(duplicatePS, batchSize) - } - - return streamingIterators.wrapIterator(strategy, postingsSetsIteratorFactory, iteratorFactory), nil + return streamingIterators.wrapIterator(strategy, ps, batchSize, iteratorFactory), nil } func openBlockSeriesChunkRefsSetsIteratorFromPostings( diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index 2783f18f272..1f8811f84e6 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -2,7 +2,11 @@ package storegateway -import "sync" +import ( + "sync" + + "github.com/prometheus/prometheus/storage" +) type seriesChunkRefsIteratorFactory func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] type postingsSetsIteratorFactory func() *postingsSetsIterator @@ -12,10 +16,12 @@ type postingsSetsIteratorFactory func() *postingsSetsIterator // 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 - postingsSetsIteratorFactory postingsSetsIteratorFactory - iteratorFactory seriesChunkRefsIteratorFactory - it iterator[seriesChunkRefsSet] + strategy seriesIteratorStrategy + iteratorFactory seriesChunkRefsIteratorFactory + postings []storage.SeriesRef + batchSize int + + it iterator[seriesChunkRefsSet] expectSingleBatch bool inChunksStreamingPhaseForSingleBatch bool @@ -25,24 +31,32 @@ type chunksStreamingCachingSeriesChunkRefsSetIterator struct { cachedBatch seriesChunkRefsSet } -func newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy seriesIteratorStrategy, postingsSetsIteratorFactory postingsSetsIteratorFactory, iteratorFactory seriesChunkRefsIteratorFactory) *chunksStreamingCachingSeriesChunkRefsSetIterator { - psi := postingsSetsIteratorFactory() - expectSingleBatch := !psi.HasMultipleBatches() +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, - postingsSetsIteratorFactory: postingsSetsIteratorFactory, - iteratorFactory: iteratorFactory, - it: iteratorFactory(initialStrategy, psi), - expectSingleBatch: expectSingleBatch, + strategy: strategy, + postings: postings, + batchSize: batchSize, + iteratorFactory: iteratorFactory, + it: iteratorFactory(initialStrategy, psi), + expectSingleBatch: expectSingleBatch, } } @@ -90,7 +104,9 @@ func (i *chunksStreamingCachingSeriesChunkRefsSetIterator) PrepareForChunksStrea i.inChunksStreamingPhaseForSingleBatch = true i.currentBatchIndex = -1 } else { - psi := i.postingsSetsIteratorFactory() + // 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) } } @@ -108,8 +124,8 @@ func newStreamingSeriesIterators() *streamingSeriesIterators { } } -func (i *streamingSeriesIterators) wrapIterator(strategy seriesIteratorStrategy, postingsSetsIteratorFactory postingsSetsIteratorFactory, iteratorFactory seriesChunkRefsIteratorFactory) iterator[seriesChunkRefsSet] { - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(strategy, postingsSetsIteratorFactory, iteratorFactory) +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) diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go index 636205b7120..f174abf3648 100644 --- a/pkg/storegateway/series_refs_streaming_test.go +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -13,8 +13,8 @@ import ( ) func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_HappyPath(t *testing.T) { - postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } - + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 factoryCalls := 0 var factoryStrategy seriesIteratorStrategy @@ -37,7 +37,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) // Inner iterator should be created with chunk refs enabled. require.Equal(t, 1, factoryCalls) @@ -71,8 +71,8 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_InnerIteratorReturnsUnreleasableSet(t *testing.T) { - postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } - + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 factoryCalls := 0 unreleasableBatch := seriesChunkRefsSet{ @@ -89,7 +89,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I return newSliceSeriesChunkRefsSetIterator(nil, unreleasableBatch) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) // During label sending phase, the single batch should be returned and not be releasable. batches := readAllSeriesChunkRefsSet(it) @@ -108,8 +108,8 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_AllBatchesFilteredOut(t *testing.T) { - postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } - + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 factoryCalls := 0 var factoryStrategy seriesIteratorStrategy @@ -123,7 +123,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_A ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) // Inner iterator should be created with chunk refs enabled. require.Equal(t, 1, factoryCalls) @@ -147,7 +147,8 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_A } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_IteratorReturnsError(t *testing.T) { - postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3}, 4) } + postings := []storage.SeriesRef{1, 2, 3} + batchSize := 4 factoryCalls := 0 iteratorError := errors.New("something went wrong") @@ -167,7 +168,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) // During label sending phase, the error should be returned. _ = readAllSeriesChunkRefsSet(it) @@ -183,8 +184,8 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_I } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_HappyPath(t *testing.T) { - postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) } - + postings := []storage.SeriesRef{1, 2, 3, 4, 5, 6} + batchSize := 3 factoryCalls := 0 var factoryStrategy seriesIteratorStrategy @@ -242,7 +243,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch return newSliceSeriesChunkRefsSetIterator(nil, firstBatchWithChunkRefs, secondBatchWithChunkRefs) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) // Inner iterator should be created with chunk refs disabled. require.Equal(t, 1, factoryCalls) @@ -267,8 +268,8 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_AllBatchesFilteredOut(t *testing.T) { - postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) } - + postings := []storage.SeriesRef{1, 2, 3, 4, 5, 6} + batchSize := 3 factoryCalls := 0 var factoryStrategy seriesIteratorStrategy @@ -282,7 +283,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) // Inner iterator should be created with chunk refs disabled. require.Equal(t, 1, factoryCalls) @@ -307,7 +308,8 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch } func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatches_IteratorReturnsError(t *testing.T) { - postingsSetsIteratorFactory := func() *postingsSetsIterator { return newPostingsSetsIterator([]storage.SeriesRef{1, 2, 3, 4, 5, 6}, 3) } + postings := []storage.SeriesRef{1, 2, 3, 4, 5, 6} + batchSize := 3 factoryCalls := 0 iteratorFactory := func(_ seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { @@ -326,7 +328,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch ) } - it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postingsSetsIteratorFactory, iteratorFactory) + it := newChunksStreamingCachingSeriesChunkRefsSetIterator(defaultStrategy, postings, batchSize, iteratorFactory) require.Equal(t, 1, factoryCalls) // During label sending phase, the error from the original iterator should be returned. From b2f7f151068d338ba34b9337c28eef4df7fc4c38 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 14:11:00 +1000 Subject: [PATCH 42/43] Fix linting issues --- pkg/storegateway/series_refs_streaming.go | 1 - pkg/storegateway/series_refs_streaming_test.go | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/pkg/storegateway/series_refs_streaming.go b/pkg/storegateway/series_refs_streaming.go index 1f8811f84e6..b5c5d8bd3bb 100644 --- a/pkg/storegateway/series_refs_streaming.go +++ b/pkg/storegateway/series_refs_streaming.go @@ -9,7 +9,6 @@ import ( ) type seriesChunkRefsIteratorFactory func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] -type postingsSetsIteratorFactory func() *postingsSetsIterator // chunksStreamingCachingSeriesChunkRefsSetIterator is an iterator used while streaming chunks from store-gateways to queriers. // diff --git a/pkg/storegateway/series_refs_streaming_test.go b/pkg/storegateway/series_refs_streaming_test.go index f174abf3648..51e8f6bfe80 100644 --- a/pkg/storegateway/series_refs_streaming_test.go +++ b/pkg/storegateway/series_refs_streaming_test.go @@ -24,7 +24,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_H {lset: labels.FromStrings("series", "3")}, } - iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { + iteratorFactory := func(strategy seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ factoryStrategy = strategy @@ -113,7 +113,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingSingleBatch_A factoryCalls := 0 var factoryStrategy seriesIteratorStrategy - iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { + iteratorFactory := func(strategy seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ factoryStrategy = strategy @@ -273,7 +273,7 @@ func TestChunksStreamingCachingSeriesChunkRefsSetIterator_ExpectingMultipleBatch factoryCalls := 0 var factoryStrategy seriesIteratorStrategy - iteratorFactory := func(strategy seriesIteratorStrategy, psi *postingsSetsIterator) iterator[seriesChunkRefsSet] { + iteratorFactory := func(strategy seriesIteratorStrategy, _ *postingsSetsIterator) iterator[seriesChunkRefsSet] { factoryCalls++ factoryStrategy = strategy From 1af591f3dfdfb7b906fad387a026f7333e13c3f8 Mon Sep 17 00:00:00 2001 From: Charles Korn Date: Mon, 20 May 2024 14:12:29 +1000 Subject: [PATCH 43/43] Remove unused method --- pkg/storegateway/series_refs.go | 4 ---- pkg/storegateway/series_refs_test.go | 6 ------ 2 files changed, 10 deletions(-) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index f9de23dd2a2..060263ea0d0 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -1405,7 +1405,3 @@ func (s *postingsSetsIterator) Next() bool { func (s *postingsSetsIterator) At() []storage.SeriesRef { return s.currentBatch } - -func (s *postingsSetsIterator) HasMultipleBatches() bool { - return len(s.postings) > s.batchSize -} diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index c0aa90bd4c8..dad02bd2a2c 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -2300,12 +2300,6 @@ func TestPostingsSetsIterator(t *testing.T) { t.Run(testName, func(t *testing.T) { iterator := newPostingsSetsIterator(testCase.postings, testCase.batchSize) - if len(testCase.expectedBatches) > 1 { - require.True(t, iterator.HasMultipleBatches()) - } else { - require.False(t, iterator.HasMultipleBatches()) - } - var actualBatches [][]storage.SeriesRef for iterator.Next() { actualBatches = append(actualBatches, iterator.At())