From a5ec2c281ce3ee447a90a133ea1bcc862accfaee Mon Sep 17 00:00:00 2001 From: Dimitar Dimitrov Date: Fri, 28 Oct 2022 20:49:02 +0200 Subject: [PATCH 1/7] Reduce newSeriesChunkRefsSet() introducing a memory pool Fixed the optimization for the case series are not duplicated among sets Added TestMergedSeriesChunkRefsSet_Concurrency Signed-off-by: Marco Pracucci --- pkg/storegateway/series_chunks.go | 4 + pkg/storegateway/series_refs.go | 144 ++++++++++++++++++--- pkg/storegateway/series_refs_test.go | 182 ++++++++++++++++++++++++++- 3 files changed, 305 insertions(+), 25 deletions(-) diff --git a/pkg/storegateway/series_chunks.go b/pkg/storegateway/series_chunks.go index 614e2c1ff2..90fe850662 100644 --- a/pkg/storegateway/series_chunks.go +++ b/pkg/storegateway/series_chunks.go @@ -203,6 +203,10 @@ func (c *loadingSeriesChunksSetIterator) Next() bool { } nextUnloaded := c.from.At() + + // This data structure doesn't retain the seriesChunkRefsSet so it can be released once done. + defer nextUnloaded.release() + entries := make([]seriesEntry, nextUnloaded.len()) c.chunkReaders.reset() for i, s := range nextUnloaded.series { diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 8101c2d815..8e9304a70c 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -4,6 +4,7 @@ package storegateway import ( "context" + "sync" "github.com/oklog/ulid" "github.com/pkg/errors" @@ -17,10 +18,22 @@ import ( "github.com/grafana/mimir/pkg/storegateway/storepb" ) +var ( + seriesChunkRefsSetPool = sync.Pool{ + // Intentionally return nil if the pool is empty, so that the caller can preallocate + // the slice with the right size. + New: nil, + } +) + // seriesChunkRefsSetIterator is the interface implemented by an iterator returning a sequence of seriesChunkRefsSet. type seriesChunkRefsSetIterator interface { Next() bool + + // At returns the current seriesChunkRefsSet. The caller should (but NOT must) invoke seriesChunkRefsSet.release() + // on the returned set once it's guaranteed it will not be used anymore. At() seriesChunkRefsSet + Err() error } @@ -35,11 +48,31 @@ type seriesChunkRefsIterator interface { type seriesChunkRefsSet struct { // series sorted by labels. series []seriesChunkRefs + + // releasable holds whether the series slice (but not its content) can be released to a memory pool. + releasable bool } -func newSeriesChunkRefsSet(capacity int) seriesChunkRefsSet { +// 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 { + var prealloc []seriesChunkRefs + + // 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)) + } + } + + if prealloc == nil { + prealloc = make([]seriesChunkRefs, 0, capacity) + } + return seriesChunkRefsSet{ - series: make([]seriesChunkRefs, 0, capacity), + series: prealloc, + releasable: releasable, } } @@ -47,6 +80,20 @@ func (b seriesChunkRefsSet) len() int { return len(b.series) } +// release the internal series slice to a memory pool. This function call has no effect +// if seriesChunkRefsSet was created to be not releasable. +func (b seriesChunkRefsSet) release() { + if b.series == nil || !b.releasable { + return + } + + reuse := b.series[:0] + seriesChunkRefsSetPool.Put(&reuse) + + // TODO this doesn't do what we want, because seriesChunkRefsSet is passed around by copy and not reference + b.series = nil +} + // seriesChunkRefs holds a series with a list of chunk references. type seriesChunkRefs struct { lset labels.Labels @@ -95,11 +142,22 @@ func newSeriesChunkRefsIterator(set seriesChunkRefsSet) *seriesChunkRefsIterator // reset replaces the current set with the provided one. After calling reset() you // must call Next() to advance the iterator to the first element. +// +// This function just reset the internal state and it does NOT invoke release() +// on the previous seriesChunkRefsSet. func (c *seriesChunkRefsIteratorImpl) reset(set seriesChunkRefsSet) { c.set = set c.currentOffset = -1 } +// resetIteratorAndReleasePreviousSet is like reset() but also release the previous seriesChunkRefsSet +// hold internally. Invoke this function if none else except this iterator is holding a +// reference to the previous seriesChunkRefsSet. +func (c *seriesChunkRefsIteratorImpl) resetIteratorAndReleasePreviousSet(set seriesChunkRefsSet) { + c.set.release() + c.reset(set) +} + func (c *seriesChunkRefsIteratorImpl) Next() bool { c.currentOffset++ return !c.Done() @@ -144,10 +202,15 @@ func (c flattenedSeriesChunkRefsIterator) Next() bool { // The current iterator has no more elements. We check if there's another // iterator to fetch and then iterate on. if !c.from.Next() { + // We can safely release the previous set because none retained it except the + // iterator itself (which we're going to reset). + c.iterator.resetIteratorAndReleasePreviousSet(seriesChunkRefsSet{}) return false } - c.iterator.reset(c.from.At()) + // We can safely release the previous set because none retained it except the + // iterator itself (which we're going to reset). + c.iterator.resetIteratorAndReleasePreviousSet(c.from.At()) // We've replaced the current iterator, so can recursively call Next() // to check if there's any item in the new iterator and further advance it if not. @@ -169,19 +232,19 @@ func (emptySeriesChunkRefsSetIterator) Next() bool { return false } func (emptySeriesChunkRefsSetIterator) At() seriesChunkRefsSet { return seriesChunkRefsSet{} } func (emptySeriesChunkRefsSetIterator) Err() error { return nil } -func mergedSeriesChunkRefsSetIterators(mergedSize int, all ...seriesChunkRefsSetIterator) seriesChunkRefsSetIterator { +func mergedSeriesChunkRefsSetIterators(mergedBatchSize int, all ...seriesChunkRefsSetIterator) seriesChunkRefsSetIterator { switch len(all) { case 0: return emptySeriesChunkRefsSetIterator{} case 1: - return newDeduplicatingSeriesChunkRefsSetIterator(mergedSize, all[0]) + return newDeduplicatingSeriesChunkRefsSetIterator(mergedBatchSize, all[0]) } h := len(all) / 2 return newMergedSeriesChunkRefsSet( - mergedSize, - mergedSeriesChunkRefsSetIterators(mergedSize, all[:h]...), - mergedSeriesChunkRefsSetIterators(mergedSize, all[h:]...), + mergedBatchSize, + mergedSeriesChunkRefsSetIterators(mergedBatchSize, all[:h]...), + mergedSeriesChunkRefsSetIterators(mergedBatchSize, all[h:]...), ) } @@ -191,6 +254,7 @@ type mergedSeriesChunkRefsSet struct { a, b seriesChunkRefsSetIterator aAt, bAt *seriesChunkRefsIteratorImpl current seriesChunkRefsSet + done bool } func newMergedSeriesChunkRefsSet(mergedBatchSize int, a, b seriesChunkRefsSetIterator) *mergedSeriesChunkRefsSet { @@ -198,6 +262,7 @@ func newMergedSeriesChunkRefsSet(mergedBatchSize int, a, b seriesChunkRefsSetIte batchSize: mergedBatchSize, a: a, b: b, + done: false, // start iterator on an empty set. It will be reset with a non-empty set when Next() is called aAt: newSeriesChunkRefsIterator(seriesChunkRefsSet{}), bAt: newSeriesChunkRefsIterator(seriesChunkRefsSet{}), @@ -213,17 +278,31 @@ func (s *mergedSeriesChunkRefsSet) Err() error { return nil } +func (s *mergedSeriesChunkRefsSet) At() seriesChunkRefsSet { + return s.current +} + func (s *mergedSeriesChunkRefsSet) Next() bool { - next := newSeriesChunkRefsSet(s.batchSize) + if s.done { + return false + } + + // This can be released by the caller because mergedSeriesChunkRefsSet doesn't retain it + // after Next() will be called again. + next := newSeriesChunkRefsSet(s.batchSize, true) for i := 0; i < s.batchSize; i++ { if err := s.ensureItemAvailableToRead(s.aAt, s.a); err != nil { // Stop iterating on first error encountered. + s.current = seriesChunkRefsSet{} + s.done = true return false } if err := s.ensureItemAvailableToRead(s.bAt, s.b); err != nil { // Stop iterating on first error encountered. + s.current = seriesChunkRefsSet{} + s.done = true return false } @@ -234,8 +313,17 @@ func (s *mergedSeriesChunkRefsSet) Next() bool { next.series = append(next.series, nextSeries) } + // We have reached the end of the iterator and next set is empty, so we can + // directly release it. + if next.len() == 0 { + next.release() + s.current = seriesChunkRefsSet{} + s.done = true + return false + } + s.current = next - return s.current.len() > 0 + return true } // ensureItemAvailableToRead ensures curr has an item available to read, unless we reached the @@ -245,7 +333,8 @@ func (s *mergedSeriesChunkRefsSet) ensureItemAvailableToRead(curr *seriesChunkRe // Ensure curr has an item available, otherwise fetch the next set. for curr.Done() { if set.Next() { - curr.reset(set.At()) + // We can release the previous set because it hasn't been retained by anyone else. + curr.resetIteratorAndReleasePreviousSet(set.At()) // Advance the iterator to the first element. If the iterator is empty, // it will be detected and handled by the outer for loop. @@ -253,6 +342,9 @@ func (s *mergedSeriesChunkRefsSet) ensureItemAvailableToRead(curr *seriesChunkRe continue } + // Release the previous set because won't be accessed anymore. + curr.resetIteratorAndReleasePreviousSet(seriesChunkRefsSet{}) + if set.Err() != nil { // Stop iterating on first error encountered. return set.Err() @@ -333,10 +425,6 @@ Outer: return toReturn, true } -func (s *mergedSeriesChunkRefsSet) At() seriesChunkRefsSet { - return s.current -} - type seriesChunkRefsSeriesSet struct { from seriesChunkRefsSetIterator @@ -362,11 +450,18 @@ func (s *seriesChunkRefsSeriesSet) Next() bool { // The current iterator has no more elements. We check if there's another // iterator to fetch and then iterate on. if !s.from.Next() { + // We can safely release the previous set because none retained it except the + // iterator itself (which we're going to reset). + s.currentIterator.resetIteratorAndReleasePreviousSet(seriesChunkRefsSet{}) + return false } next := s.from.At() - s.currentIterator.reset(next) + + // We can safely release the previous set because none retained it except the + // iterator itself (which we're going to reset). + s.currentIterator.resetIteratorAndReleasePreviousSet(next) // We've replaced the current iterator, so can recursively call Next() // to check if there's any item in the new iterator and further advance it if not. @@ -417,7 +512,10 @@ func (s *deduplicatingSeriesChunkRefsSetIterator) Next() bool { firstSeries = *s.peek s.peek = nil } - nextSet := newSeriesChunkRefsSet(s.batchSize) + + // 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.series = append(nextSet.series, firstSeries) var nextSeries seriesChunkRefs @@ -615,7 +713,9 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { loadStats := &queryStats{} defer s.stats.merge(loadStats) - nextSet := newSeriesChunkRefsSet(len(nextPostings)) + // This can be released by the caller because loadingSeriesChunkRefsSetIterator doesn't retain it + // after Next() will be called again. + nextSet := newSeriesChunkRefsSet(len(nextPostings), true) for _, id := range nextPostings { lset, chks, err := s.loadSeriesForTime(id, loadedSeries, loadStats) @@ -637,9 +737,15 @@ func (s *loadingSeriesChunkRefsSetIterator) Next() bool { chunks: chks, }) } + if nextSet.len() == 0 { - return s.Next() // we didn't find any suitable series in this set, try with the next one + // The next set we attempted to build is empty, so we can directly release it. + nextSet.release() + + // Try with the next set of postings. + return s.Next() } + s.currentSet = nextSet return true } diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index ad786cdd8b..b61f86711d 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -6,6 +6,7 @@ import ( "context" "errors" "fmt" + "math/rand" "sort" "testing" @@ -18,6 +19,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.uber.org/atomic" + "golang.org/x/sync/errgroup" "github.com/grafana/mimir/pkg/storage/sharding" "github.com/grafana/mimir/pkg/util/test" @@ -580,6 +582,148 @@ func TestMergedSeriesChunkRefsSet(t *testing.T) { } } +func TestMergedSeriesChunkRefsSet_Concurrency(t *testing.T) { + const ( + concurrency = 10 + runs = 100 + minIterators = 2 + maxIterators = 20 + minSetsPerIterator = 2 + maxSetsPerIterator = 10 + minSeriesPerSet = 10 + maxSeriesPerSet = 50 + ) + + runTest := func() { + // Randomize the test setup. + var ( + numIterators = minIterators + rand.Intn(maxIterators-minIterators) + numSetsPerIterator = minSetsPerIterator + rand.Intn(maxSetsPerIterator-minSetsPerIterator) + numSeriesPerSet = minSeriesPerSet + rand.Intn(maxSeriesPerSet-minSeriesPerSet) + ) + + // Create the iterators. + iterators := make([]seriesChunkRefsSetIterator, 0, numIterators) + for iteratorIdx := 0; iteratorIdx < numIterators; iteratorIdx++ { + // Create the sets for this iterator. + sets := make([]seriesChunkRefsSet, 0, numSetsPerIterator) + for setIdx := 0; setIdx < numSetsPerIterator; setIdx++ { + minSeriesID := (iteratorIdx * numSetsPerIterator * numSeriesPerSet) + (setIdx * numSeriesPerSet) + maxSeriesID := minSeriesID + numSeriesPerSet - 1 + sets = append(sets, createSeriesChunkRefsSet(minSeriesID, maxSeriesID, true)) + } + + iterators = append(iterators, newSliceSeriesChunkRefsSetIterator(nil, sets...)) + } + + // Run the actual test. + it := mergedSeriesChunkRefsSetIterators(50, iterators...) + + actualSeries := 0 + for it.Next() { + set := it.At() + actualSeries += len(set.series) + set.release() + } + + require.NoError(t, it.Err()) + require.Equal(t, numIterators*numSetsPerIterator*numSeriesPerSet, actualSeries) + } + + g, _ := errgroup.WithContext(context.Background()) + + for c := 0; c < concurrency; c++ { + g.Go(func() error { + for r := 0; r < runs/concurrency; r++ { + runTest() + } + return nil + }) + } + + require.NoError(t, g.Wait()) +} + +func BenchmarkMergedSeriesChunkRefsSetIterators(b *testing.B) { + const ( + numSetsPerIterator = 10 + numSeriesPerSet = 10 + mergedBatchSize = 5000 + ) + + // Creates the series sets, guaranteeing series sorted by labels. + createUnreleasableSets := func(iteratorIdx int) []seriesChunkRefsSet { + sets := make([]seriesChunkRefsSet, 0, numSetsPerIterator) + for setIdx := 0; setIdx < numSetsPerIterator; setIdx++ { + minSeriesID := (iteratorIdx * numSetsPerIterator * numSeriesPerSet) + (setIdx * numSeriesPerSet) + maxSeriesID := minSeriesID + numSeriesPerSet - 1 + + // This set cannot be released because reused between multiple benchmark runs. + set := createSeriesChunkRefsSet(minSeriesID, maxSeriesID, false) + sets = append(sets, set) + } + + return sets + } + + for _, withDuplicatedSeries := range []bool{true, false} { + for numIterators := 1; numIterators <= 64; numIterators *= 2 { + // Create empty iterators that we can reuse in each benchmark run. + iterators := make([]seriesChunkRefsSetIterator, 0, numIterators) + for i := 0; i < numIterators; i++ { + iterators = append(iterators, newSliceSeriesChunkRefsSetIterator(nil)) + } + + // Create the sets for each underlying iterator. These sets cannot be released because + // will be used in multiple benchmark runs. + perIteratorSets := make([][]seriesChunkRefsSet, 0, numIterators) + for iteratorIdx := 0; iteratorIdx < numIterators; iteratorIdx++ { + if withDuplicatedSeries { + perIteratorSets = append(perIteratorSets, createUnreleasableSets(0)) + } else { + perIteratorSets = append(perIteratorSets, createUnreleasableSets(iteratorIdx)) + } + } + + b.Run(fmt.Sprintf("with duplicated series = %t number of iterators = %d", withDuplicatedSeries, numIterators), func(b *testing.B) { + for n := 0; n < b.N; n++ { + // Reset iterators. + for i := 0; i < numIterators; i++ { + iterators[i].(*sliceSeriesChunkRefsSetIterator).reset(perIteratorSets[i]) + } + + // Merge the iterators and run through them. + it := mergedSeriesChunkRefsSetIterators(mergedBatchSize, iterators...) + + actualSeries := 0 + for it.Next() { + set := it.At() + actualSeries += len(set.series) + + set.release() + } + + if err := it.Err(); err != nil { + b.Fatal(it.Err()) + } + + // Ensure each benchmark run go through the same data set. + var expectedSeries int + if withDuplicatedSeries { + expectedSeries = numSetsPerIterator * numSeriesPerSet + } else { + expectedSeries = numIterators * numSetsPerIterator * numSeriesPerSet + } + + if actualSeries != expectedSeries { + b.Fatalf("benchmark iterated through an unexpected number of series (expected: %d got: %d)", expectedSeries, actualSeries) + } + } + }) + } + } +} + func TestSeriesSetWithoutChunks(t *testing.T) { // Generate some chunk fixtures so that we can ensure the right chunks are returned. c := generateSeriesChunkRef(6) @@ -1295,12 +1439,15 @@ type sliceSeriesChunkRefsSetIterator struct { err error } -func newSliceSeriesChunkRefsSetIterator(err error, sets ...seriesChunkRefsSet) seriesChunkRefsSetIterator { - return &sliceSeriesChunkRefsSetIterator{ - current: -1, - sets: sets, - err: err, - } +func newSliceSeriesChunkRefsSetIterator(err error, sets ...seriesChunkRefsSet) *sliceSeriesChunkRefsSetIterator { + s := &sliceSeriesChunkRefsSetIterator{err: err} + s.reset(sets) + return s +} + +func (s *sliceSeriesChunkRefsSetIterator) reset(sets []seriesChunkRefsSet) { + s.current = -1 + s.sets = sets } func (s *sliceSeriesChunkRefsSetIterator) Next() bool { @@ -1361,3 +1508,26 @@ func readAllSeriesChunkRefs(it seriesChunkRefsIterator) []seriesChunkRefs { } return out } + +// createSeriesChunkRefsSet creates a seriesChunkRefsSet with series whose name is generated +// 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) + + for seriesID := minSeriesID; seriesID <= maxSeriesID; seriesID++ { + set.series = append(set.series, seriesChunkRefs{ + lset: labels.New(labels.Label{Name: labels.MetricName, Value: fmt.Sprintf("metric_%06d", seriesID)}), + }) + } + + return set +} + +func TestCreateSeriesChunkRefsSet(t *testing.T) { + set := createSeriesChunkRefsSet(5, 7, true) + require.Len(t, set.series, 3) + assert.Equal(t, seriesChunkRefs{lset: labels.New(labels.Label{Name: labels.MetricName, Value: "metric_000005"})}, set.series[0]) + assert.Equal(t, seriesChunkRefs{lset: labels.New(labels.Label{Name: labels.MetricName, Value: "metric_000006"})}, set.series[1]) + assert.Equal(t, seriesChunkRefs{lset: labels.New(labels.Label{Name: labels.MetricName, Value: "metric_000007"})}, set.series[2]) +} From 53eb868828002afb82e7f34d63f3e7f9d99fc1cd Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Mon, 12 Dec 2022 18:33:11 +0100 Subject: [PATCH 2/7] Added TestBucket_Series_Concurrency Signed-off-by: Marco Pracucci --- pkg/storegateway/bucket_test.go | 150 ++++++++++++++++++++++++++++++-- 1 file changed, 145 insertions(+), 5 deletions(-) diff --git a/pkg/storegateway/bucket_test.go b/pkg/storegateway/bucket_test.go index 543f2530ff..c827306066 100644 --- a/pkg/storegateway/bucket_test.go +++ b/pkg/storegateway/bucket_test.go @@ -20,6 +20,7 @@ import ( "runtime" "sort" "strconv" + "strings" "sync" "testing" "time" @@ -942,21 +943,21 @@ func benchmarkExpandedPostings( } } -func TestBucketSeries(t *testing.T) { +func TestBucket_Series(t *testing.T) { tb := test.NewTB(t) runSeriesInterestingCases(tb, 10000, 10000, func(t test.TB, samplesPerSeries, series int) { benchBucketSeries(t, false, samplesPerSeries, series, 1) }) } -func TestBucketSkipChunksSeries(t *testing.T) { +func TestBucket_Series_WithSkipChunks(t *testing.T) { tb := test.NewTB(t) runSeriesInterestingCases(tb, 10000, 10000, func(t test.TB, samplesPerSeries, series int) { benchBucketSeries(t, true, samplesPerSeries, series, 1) }) } -func BenchmarkBucketSeries(b *testing.B) { +func BenchmarkBucket_Series(b *testing.B) { tb := test.NewTB(b) // 10e6 samples = ~1736 days with 15s scrape runSeriesInterestingCases(tb, 10e6, 10e5, func(t test.TB, samplesPerSeries, series int) { @@ -964,7 +965,7 @@ func BenchmarkBucketSeries(b *testing.B) { }) } -func BenchmarkBucketSkipChunksSeries(b *testing.B) { +func BenchmarkBucket_Series_WithSkipChunks(b *testing.B) { tb := test.NewTB(b) // 10e6 samples = ~1736 days with 15s scrape runSeriesInterestingCases(tb, 10e6, 10e5, func(t test.TB, samplesPerSeries, series int) { @@ -1119,7 +1120,146 @@ func benchBucketSeries(t test.TB, skipChunk bool, samplesPerSeries, totalSeries runTestWithStore(t, st) }) } +} + +func TestBucket_Series_Concurrency(t *testing.T) { + const ( + numWorkers = 10 + numRequestsPerWorker = 100 + numBlocks = 4 + numSeriesPerBlock = 100 + numSamplesPerSeries = 200 + ) + + var ( + ctx = context.Background() + logger = log.NewNopLogger() + expectedSeries []*storepb.Series + expectedBlockIDs []string + random = rand.New(rand.NewSource(120)) + tmpDir = t.TempDir() + ) + + test.VerifyNoLeak(t) + + // Create a filesystem-based bucket. + bucket, err := filesystem.NewBucket(filepath.Join(tmpDir, "bkt")) + assert.NoError(t, err) + defer func() { assert.NoError(t, bucket.Close()) }() + instrumentedBucket := objstore.WithNoopInstr(bucket) + + // Generate some blocks. + t.Log("generating test blocks") + blockDir := filepath.Join(tmpDir, "tmp") + for b := 0; b < numBlocks; b++ { + head, blockSeries := createHeadWithSeries(t, b, headGenOptions{ + TSDBDir: filepath.Join(tmpDir, fmt.Sprintf("%d", b)), + SamplesPerSeries: numSamplesPerSeries, + Series: numSeriesPerBlock, + PrependLabels: labels.FromStrings(labels.MetricName, "test_metric", "zzz_block_id", strconv.Itoa(b)), + Random: random, + }) + + blockID := createBlockFromHead(t, blockDir, head) + assert.NoError(t, head.Close()) + + expectedSeries = append(expectedSeries, blockSeries...) + expectedBlockIDs = append(expectedBlockIDs, blockID.String()) + require.NoError(t, mimir_tsdb.UploadBlock(ctx, logger, bucket, filepath.Join(blockDir, blockID.String()), nil)) + } + t.Log("generated test blocks") + + // Prepare a request to query all series. + hints := &hintspb.SeriesRequestHints{ + BlockMatchers: []storepb.LabelMatcher{ + { + Type: storepb.LabelMatcher_RE, + Name: block.BlockIDLabel, + Value: strings.Join(expectedBlockIDs, "|"), + }, + }, + } + + marshalledHints, err := types.MarshalAny(hints) + require.NoError(t, err) + + req := &storepb.SeriesRequest{ + MinTime: math.MinInt64, + MaxTime: math.MaxInt64, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_EQ, Name: labels.MetricName, Value: "test_metric"}, + }, + Hints: marshalledHints, + } + + runRequest := func(t *testing.T, store *BucketStore) { + srv := newBucketStoreSeriesServer(ctx) + require.NoError(t, store.Series(req, srv)) + require.Equal(t, 0, len(srv.Warnings), "%v", srv.Warnings) + require.Equal(t, len(expectedSeries), len(srv.SeriesSet)) + + // Huge responses can produce unreadable diffs - make it more human readable. + for j := range expectedSeries { + require.Equal(t, expectedSeries[j].Labels, srv.SeriesSet[j].Labels, "series labels mismatch at position %d", j) + require.Equal(t, expectedSeries[j].Chunks, srv.SeriesSet[j].Chunks, "series chunks mismatch at position %d", j) + } + } + + // Run the test with different batch sizes. + for _, batchSize := range []int{len(expectedSeries) / 100, len(expectedSeries) * 2} { + t.Run(fmt.Sprintf("batch size: %d", batchSize), func(t *testing.T) { + metaFetcher, err := block.NewRawMetaFetcher(logger, instrumentedBucket) + assert.NoError(t, err) + + chunkPool, err := pool.NewBucketedBytes(chunkBytesPoolMinSize, chunkBytesPoolMaxSize, 2, 1e9) // 1GB. + assert.NoError(t, err) + trackedChunkPool := &mockedPool{parent: chunkPool} + + // Create the bucket store. + store, err := NewBucketStore( + "test-user", + instrumentedBucket, + metaFetcher, + tmpDir, + NewChunksLimiterFactory(0), + NewSeriesLimiterFactory(0), + newGapBasedPartitioner(mimir_tsdb.DefaultPartitionerMaxGapSize, nil), + 1, + mimir_tsdb.DefaultPostingOffsetInMemorySampling, + indexheader.Config{}, + false, // Lazy index-header loading disabled. + 0, + hashcache.NewSeriesHashCache(1024*1024), + NewBucketStoreMetrics(nil), + WithLogger(logger), + WithChunkPool(trackedChunkPool), + WithStreamingSeriesPerBatch(batchSize), + ) + require.NoError(t, err) + require.NoError(t, store.SyncBlocks(ctx)) + + // Run workers. + wg := sync.WaitGroup{} + wg.Add(numWorkers) + + for c := 0; c < numWorkers; c++ { + go func() { + defer wg.Done() + + for r := 0; r < numRequestsPerWorker; r++ { + runRequest(t, store) + } + }() + } + + // Wait until all workers have done. + wg.Wait() + + // Ensure all chunks have been released to the pool. + require.Equal(t, 0, int(trackedChunkPool.balance.Load())) + }) + } } type mockedPool struct { @@ -2190,7 +2330,7 @@ type headGenOptions struct { } // createHeadWithSeries returns head filled with given samples and same series returned in separate list for assertion purposes. -// Returned series list has "ext1"="1" prepended. Each series looks as follows: +// Each series looks as follows: // {foo=bar,i=000001aaaaaaaaaabbbbbbbbbbccccccccccdddddddddd} where number indicate sample number from 0. // Returned series are framed in the same way as remote read would frame them. func createHeadWithSeries(t testing.TB, j int, opts headGenOptions) (*tsdb.Head, []*storepb.Series) { From 8ed6bbf60ce4c71202eab130f515814d9172010a Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Mon, 12 Dec 2022 18:48:27 +0100 Subject: [PATCH 3/7] Added assertions on seriesChunkRefsSetPool balance Signed-off-by: Marco Pracucci --- pkg/storegateway/bucket_test.go | 44 ++++++++++++++++++++++---- pkg/storegateway/series_chunks_test.go | 2 +- pkg/storegateway/series_refs.go | 5 +-- pkg/storegateway/series_refs_test.go | 14 +++++++- pkg/util/pool/pool.go | 8 +++++ 5 files changed, 62 insertions(+), 11 deletions(-) diff --git a/pkg/storegateway/bucket_test.go b/pkg/storegateway/bucket_test.go index c827306066..4bb09a7684 100644 --- a/pkg/storegateway/bucket_test.go +++ b/pkg/storegateway/bucket_test.go @@ -1041,7 +1041,7 @@ func benchBucketSeries(t test.TB, skipChunk bool, samplesPerSeries, totalSeries runTestWithStore := func(t test.TB, st *BucketStore) { if !t.IsBenchmark() { - st.chunkPool = &mockedPool{parent: st.chunkPool} + st.chunkPool = &trackedBytesPool{parent: st.chunkPool} } assert.NoError(t, st.SyncBlocks(context.Background())) @@ -1081,8 +1081,8 @@ func benchBucketSeries(t test.TB, skipChunk bool, samplesPerSeries, totalSeries if !t.IsBenchmark() { if !skipChunk { // TODO(bwplotka): This is wrong negative for large number of samples (1mln). Investigate. - assert.Equal(t, 0, int(st.chunkPool.(*mockedPool).balance.Load())) - st.chunkPool.(*mockedPool).gets.Store(0) + assert.Equal(t, 0, int(st.chunkPool.(*trackedBytesPool).balance.Load())) + st.chunkPool.(*trackedBytesPool).gets.Store(0) } for _, b := range st.blocks { @@ -1209,12 +1209,15 @@ func TestBucket_Series_Concurrency(t *testing.T) { // Run the test with different batch sizes. for _, batchSize := range []int{len(expectedSeries) / 100, len(expectedSeries) * 2} { t.Run(fmt.Sprintf("batch size: %d", batchSize), func(t *testing.T) { + // Reset the memory pool tracker. + seriesChunkRefsSetPool.(*trackedGenericPool).reset() + metaFetcher, err := block.NewRawMetaFetcher(logger, instrumentedBucket) assert.NoError(t, err) chunkPool, err := pool.NewBucketedBytes(chunkBytesPoolMinSize, chunkBytesPoolMaxSize, 2, 1e9) // 1GB. assert.NoError(t, err) - trackedChunkPool := &mockedPool{parent: chunkPool} + trackedChunkPool := &trackedBytesPool{parent: chunkPool} // Create the bucket store. store, err := NewBucketStore( @@ -1258,17 +1261,22 @@ func TestBucket_Series_Concurrency(t *testing.T) { // Ensure all chunks have been released to the pool. require.Equal(t, 0, int(trackedChunkPool.balance.Load())) + + // Ensure the seriesChunkRefsSet memory pool has been used and all slices pulled from + // pool have put back. + assert.Greater(t, seriesChunkRefsSetPool.(*trackedGenericPool).gets.Load(), int64(0)) + assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*trackedGenericPool).balance.Load()) }) } } -type mockedPool struct { +type trackedBytesPool struct { parent pool.Bytes balance atomic.Uint64 gets atomic.Uint64 } -func (m *mockedPool) Get(sz int) (*[]byte, error) { +func (m *trackedBytesPool) Get(sz int) (*[]byte, error) { b, err := m.parent.Get(sz) if err != nil { return nil, err @@ -1278,11 +1286,33 @@ func (m *mockedPool) Get(sz int) (*[]byte, error) { return b, nil } -func (m *mockedPool) Put(b *[]byte) { +func (m *trackedBytesPool) Put(b *[]byte) { m.balance.Sub(uint64(cap(*b))) m.parent.Put(b) } +type trackedGenericPool struct { + parent pool.Generic + balance atomic.Int64 + gets atomic.Int64 +} + +func (p *trackedGenericPool) Get() any { + p.balance.Inc() + p.gets.Inc() + return p.parent.Get() +} + +func (p *trackedGenericPool) Put(x any) { + p.balance.Dec() + p.parent.Put(x) +} + +func (p *trackedGenericPool) reset() { + p.balance.Store(0) + p.gets.Store(0) +} + // Regression test against: https://github.com/thanos-io/thanos/issues/2147. func TestBucketSeries_OneBlock_InMemIndexCacheSegfault(t *testing.T) { tmpDir := t.TempDir() diff --git a/pkg/storegateway/series_chunks_test.go b/pkg/storegateway/series_chunks_test.go index bb057dd659..bf829fb3af 100644 --- a/pkg/storegateway/series_chunks_test.go +++ b/pkg/storegateway/series_chunks_test.go @@ -470,7 +470,7 @@ func TestLoadingSeriesChunksSetIterator(t *testing.T) { t.Run(testName, func(t *testing.T) { t.Parallel() // Setup - bytesPool := &mockedPool{parent: pool.NoopBytes{}} + bytesPool := &trackedBytesPool{parent: pool.NoopBytes{}} readersMap := make(map[ulid.ULID]chunkReader, len(testCase.existingBlocks)) for _, block := range testCase.existingBlocks { readersMap[block.ulid] = newChunkReaderMockWithSeries(block.series, testCase.addLoadErr, testCase.loadErr) diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 8e9304a70c..cb53908039 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -16,14 +16,15 @@ import ( "github.com/grafana/mimir/pkg/storage/sharding" "github.com/grafana/mimir/pkg/storage/tsdb/metadata" "github.com/grafana/mimir/pkg/storegateway/storepb" + "github.com/grafana/mimir/pkg/util/pool" ) var ( - seriesChunkRefsSetPool = sync.Pool{ + seriesChunkRefsSetPool = pool.Generic(&sync.Pool{ // Intentionally return nil if the pool is empty, so that the caller can preallocate // the slice with the right size. New: nil, - } + }) ) // seriesChunkRefsSetIterator is the interface implemented by an iterator returning a sequence of seriesChunkRefsSet. diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index b61f86711d..c2427fb91a 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -25,6 +25,11 @@ import ( "github.com/grafana/mimir/pkg/util/test" ) +func init() { + // Track the balance of gets/puts to seriesChunkRefsSetPool in all tests. + seriesChunkRefsSetPool = &trackedGenericPool{parent: seriesChunkRefsSetPool} +} + func TestSeriesChunkRef_Compare(t *testing.T) { input := []seriesChunkRef{ {blockID: ulid.MustNew(0, nil), minTime: 2, maxTime: 5}, @@ -630,8 +635,10 @@ func TestMergedSeriesChunkRefsSet_Concurrency(t *testing.T) { require.Equal(t, numIterators*numSetsPerIterator*numSeriesPerSet, actualSeries) } - g, _ := errgroup.WithContext(context.Background()) + // Reset the memory pool tracker. + seriesChunkRefsSetPool.(*trackedGenericPool).reset() + g, _ := errgroup.WithContext(context.Background()) for c := 0; c < concurrency; c++ { g.Go(func() error { for r := 0; r < runs/concurrency; r++ { @@ -642,6 +649,11 @@ func TestMergedSeriesChunkRefsSet_Concurrency(t *testing.T) { } require.NoError(t, g.Wait()) + + // Ensure the seriesChunkRefsSet memory pool has been used and all slices pulled from + // pool have put back. + assert.Greater(t, seriesChunkRefsSetPool.(*trackedGenericPool).gets.Load(), int64(0)) + assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*trackedGenericPool).balance.Load()) } func BenchmarkMergedSeriesChunkRefsSetIterators(b *testing.B) { diff --git a/pkg/util/pool/pool.go b/pkg/util/pool/pool.go index 553f9e21fa..74e205c6c0 100644 --- a/pkg/util/pool/pool.go +++ b/pkg/util/pool/pool.go @@ -11,6 +11,14 @@ import ( "github.com/pkg/errors" ) +// Generic memory pool with the same interface of sync.Pool. +type Generic interface { + // Put is sync.Pool.Put(). + Put(x any) + // Get is sync.Pool.Get(). + Get() any +} + // Bytes is a pool of bytes that can be reused. type Bytes interface { // Get returns a new byte slices that fits the given size. From 30e493a4bf909d46a512487a92bbfefed0d2195d Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Mon, 12 Dec 2022 19:19:05 +0100 Subject: [PATCH 4/7] Fixed BenchmarkBucket_Series_WithSkipChunks Signed-off-by: Marco Pracucci --- pkg/storegateway/bucket_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/storegateway/bucket_test.go b/pkg/storegateway/bucket_test.go index 4bb09a7684..a36b53afed 100644 --- a/pkg/storegateway/bucket_test.go +++ b/pkg/storegateway/bucket_test.go @@ -969,7 +969,9 @@ func BenchmarkBucket_Series_WithSkipChunks(b *testing.B) { tb := test.NewTB(b) // 10e6 samples = ~1736 days with 15s scrape runSeriesInterestingCases(tb, 10e6, 10e5, func(t test.TB, samplesPerSeries, series int) { - benchBucketSeries(t, true, samplesPerSeries, series, 1/100e6, 1/10e4, 1) + // Send only requests with 100% ratio because in Mimir we lookup series at block boundaries + // when skip chunks = true. + benchBucketSeries(t, true, samplesPerSeries, series, 1) }) } From 69a31f50a5f888e388e0369556c81893f0460f20 Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Mon, 12 Dec 2022 20:06:10 +0100 Subject: [PATCH 5/7] Removed TODO Signed-off-by: Marco Pracucci --- after.txt | 156 ++++++++++++++++++++++++++++++++ pkg/storegateway/series_refs.go | 5 +- 2 files changed, 158 insertions(+), 3 deletions(-) create mode 100644 after.txt diff --git a/after.txt b/after.txt new file mode 100644 index 0000000000..b0342666c1 --- /dev/null +++ b/after.txt @@ -0,0 +1,156 @@ +goos: darwin +goarch: amd64 +pkg: github.com/grafana/mimir/pkg/storegateway +cpu: Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1of1000000-12 12 105212618 ns/op 65894878 B/op 9718 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1of1000000-12 12 101783084 ns/op 65916123 B/op 9791 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1of1000000-12 12 102742071 ns/op 65902762 B/op 9745 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/10of1000000-12 12 101336812 ns/op 65915603 B/op 9900 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/10of1000000-12 12 100542850 ns/op 65921138 B/op 9919 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/10of1000000-12 12 100741674 ns/op 65912051 B/op 9925 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1610142014 ns/op 1785269440 B/op 18040140 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1498285673 ns/op 1785364144 B/op 18040466 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1538114335 ns/op 1785365968 B/op 18040472 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1of1000000-12 19 69453047 ns/op 84033607 B/op 14334 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1of1000000-12 18 69592116 ns/op 84029653 B/op 14346 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1of1000000-12 18 68726174 ns/op 84027077 B/op 14328 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/10of1000000-12 18 70563126 ns/op 84053538 B/op 14520 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/10of1000000-12 18 68991483 ns/op 84058184 B/op 14523 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/10of1000000-12 16 69645096 ns/op 84045576 B/op 14459 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 1 1309276361 ns/op 1369696280 B/op 17113886 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 1 1276190903 ns/op 1369630272 B/op 17113426 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 1 1283585236 ns/op 1369992456 B/op 17113823 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1of1000000-12 20 59032948 ns/op 59020070 B/op 6209 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1of1000000-12 21 57018934 ns/op 59055768 B/op 6177 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1of1000000-12 20 56512528 ns/op 59021763 B/op 6226 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/10of1000000-12 21 58332122 ns/op 59102731 B/op 6372 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/10of1000000-12 20 57198341 ns/op 59076132 B/op 6347 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/10of1000000-12 22 57534860 ns/op 58960205 B/op 6314 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 1 1212858241 ns/op 1243813928 B/op 17031473 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 1 1172316030 ns/op 1245723552 B/op 17031420 allocs/op +BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 1 1152077241 ns/op 1241707248 B/op 17031345 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 183 5902946 ns/op 5766383 B/op 823 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 202 5911141 ns/op 5755185 B/op 824 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 200 5928218 ns/op 5757599 B/op 821 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 202 5893376 ns/op 5796483 B/op 878 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 200 5854063 ns/op 5762387 B/op 876 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 200 5875823 ns/op 5760151 B/op 873 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 7 151719780 ns/op 168422265 B/op 1703813 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 7 153107110 ns/op 168435073 B/op 1703810 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 7 152092974 ns/op 168441956 B/op 1703816 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/1of10000000-12 206 5631274 ns/op 5039336 B/op 1140 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/1of10000000-12 210 5873516 ns/op 5041745 B/op 1141 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/1of10000000-12 195 5971325 ns/op 5040939 B/op 1141 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/100of10000000-12 199 5768342 ns/op 5045793 B/op 1200 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/100of10000000-12 213 5638804 ns/op 5046069 B/op 1201 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/100of10000000-12 214 5645288 ns/op 5046671 B/op 1202 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 7 158912220 ns/op 210183737 B/op 1805079 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 6 167314699 ns/op 208606137 B/op 1805029 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 7 159058937 ns/op 209523800 B/op 1805046 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 153 7381460 ns/op 8226830 B/op 1645 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 162 7441410 ns/op 8223953 B/op 1643 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 162 7348889 ns/op 8228856 B/op 1646 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 163 7258858 ns/op 8223920 B/op 1697 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 160 7346886 ns/op 8223687 B/op 1698 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 164 7325498 ns/op 8230169 B/op 1698 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 8 138505173 ns/op 173327277 B/op 1712145 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 8 137346493 ns/op 173558787 B/op 1712169 allocs/op +BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 8 138935676 ns/op 173523663 B/op 1712179 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/1of10000000-12 5203 220290 ns/op 216773 B/op 244 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/1of10000000-12 5370 219966 ns/op 216648 B/op 244 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/1of10000000-12 5083 219098 ns/op 216877 B/op 244 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/100of10000000-12 5288 222883 ns/op 216690 B/op 244 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/100of10000000-12 5149 224897 ns/op 216828 B/op 244 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/100of10000000-12 5265 221965 ns/op 216991 B/op 244 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 14 75400969 ns/op 222873250 B/op 334376 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 14 77376612 ns/op 222878426 B/op 334376 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 15 77324473 ns/op 222872757 B/op 334373 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 5157 230146 ns/op 219429 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 4795 230649 ns/op 218710 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 4880 230327 ns/op 219722 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 4953 229773 ns/op 219017 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 4986 231037 ns/op 219063 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 4760 230651 ns/op 218885 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 13 84214928 ns/op 225667629 B/op 334392 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 13 83222970 ns/op 225688360 B/op 334394 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 13 86494644 ns/op 225684052 B/op 334393 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 5092 231296 ns/op 242135 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 5016 231846 ns/op 242343 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 4936 232337 ns/op 241464 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 4928 232404 ns/op 242999 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 4797 233477 ns/op 241992 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 5049 233607 ns/op 242494 B/op 250 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 13 84488245 ns/op 227280200 B/op 334393 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 13 83557445 ns/op 226493801 B/op 334389 allocs/op +BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 13 85308763 ns/op 227015115 B/op 334392 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1237335391 ns/op 1954554480 B/op 11040550 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1216079860 ns/op 1954527768 B/op 11040178 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1175049608 ns/op 1954459160 B/op 11039942 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 2 901306368 ns/op 946180020 B/op 10061967 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 2 914481941 ns/op 946290036 B/op 10061841 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 2 929324603 ns/op 946390348 B/op 10062105 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 2 889820006 ns/op 849389416 B/op 10025880 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 2 903275285 ns/op 850368904 B/op 10025930 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 2 894877436 ns/op 849602768 B/op 10025786 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 10 112541764 ns/op 179408691 B/op 1104667 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 10 110965852 ns/op 179408440 B/op 1104666 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 9 111299456 ns/op 179413842 B/op 1104685 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 12 95640817 ns/op 93578367 B/op 1006675 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 12 95241279 ns/op 93580946 B/op 1006697 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 12 95300976 ns/op 93568014 B/op 1006694 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 10 106613918 ns/op 86317970 B/op 1003030 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 10 107329630 ns/op 86270081 B/op 1003046 allocs/op +BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 10 109322237 ns/op 86537753 B/op 1003037 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 3236 383036 ns/op 708944 B/op 798 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 3004 384284 ns/op 708951 B/op 798 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 3105 385719 ns/op 708940 B/op 798 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 2029 565212 ns/op 718828 B/op 620 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 2056 564602 ns/op 719045 B/op 620 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 2024 569710 ns/op 719093 B/op 620 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 2020 553268 ns/op 863589 B/op 619 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 1998 554960 ns/op 860112 B/op 618 allocs/op +BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 2083 556228 ns/op 864514 B/op 619 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_1-12 496362 2422 ns/op 211 B/op 5 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_1-12 486528 2408 ns/op 212 B/op 5 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_1-12 493654 2415 ns/op 209 B/op 5 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_2-12 141573 8457 ns/op 668 B/op 15 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_2-12 139905 8445 ns/op 672 B/op 15 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_2-12 141340 8440 ns/op 670 B/op 15 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_4-12 59212 20159 ns/op 1589 B/op 35 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_4-12 59539 20240 ns/op 1602 B/op 35 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_4-12 58899 20394 ns/op 1611 B/op 35 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_8-12 26733 44726 ns/op 3455 B/op 75 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_8-12 27036 44470 ns/op 3445 B/op 75 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_8-12 27285 45032 ns/op 3426 B/op 75 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_16-12 13040 96256 ns/op 7137 B/op 155 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_16-12 13161 92098 ns/op 7042 B/op 155 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_16-12 13089 92216 ns/op 7137 B/op 155 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_32-12 6094 186640 ns/op 14343 B/op 315 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_32-12 6469 189337 ns/op 14329 B/op 315 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_32-12 6188 188606 ns/op 14458 B/op 315 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_64-12 3098 380703 ns/op 28760 B/op 635 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_64-12 3138 377290 ns/op 29225 B/op 635 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_64-12 3108 379811 ns/op 29153 B/op 635 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_1-12 496096 2386 ns/op 212 B/op 5 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_1-12 490495 2402 ns/op 211 B/op 5 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_1-12 487287 2388 ns/op 210 B/op 5 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_2-12 104146 11581 ns/op 682 B/op 15 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_2-12 103983 11522 ns/op 687 B/op 15 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_2-12 104235 11640 ns/op 679 B/op 15 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_4-12 34095 35290 ns/op 1646 B/op 35 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_4-12 33603 35492 ns/op 1611 B/op 35 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_4-12 33639 35711 ns/op 1625 B/op 35 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_8-12 10000 100748 ns/op 3542 B/op 75 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_8-12 10000 101554 ns/op 3541 B/op 75 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_8-12 10000 101558 ns/op 3566 B/op 75 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_16-12 4136 278946 ns/op 7465 B/op 155 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_16-12 4251 277353 ns/op 7103 B/op 155 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_16-12 4173 283901 ns/op 7401 B/op 155 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_32-12 1474 779892 ns/op 14936 B/op 315 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_32-12 1509 782160 ns/op 15405 B/op 315 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_32-12 1531 781083 ns/op 15065 B/op 315 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_64-12 528 2252992 ns/op 31262 B/op 636 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_64-12 525 2257777 ns/op 29873 B/op 636 allocs/op +BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_64-12 528 2249254 ns/op 28464 B/op 636 allocs/op +PASS +ok github.com/grafana/mimir/pkg/storegateway 272.287s diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index cb53908039..6a2bb122ed 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -83,6 +83,8 @@ func (b seriesChunkRefsSet) len() int { // release the internal series slice to a memory pool. This function call has no effect // if seriesChunkRefsSet was created to be not releasable. +// +// This function is not idempotent. Calling it twice would introduce subtle bugs. func (b seriesChunkRefsSet) release() { if b.series == nil || !b.releasable { return @@ -90,9 +92,6 @@ func (b seriesChunkRefsSet) release() { reuse := b.series[:0] seriesChunkRefsSetPool.Put(&reuse) - - // TODO this doesn't do what we want, because seriesChunkRefsSet is passed around by copy and not reference - b.series = nil } // seriesChunkRefs holds a series with a list of chunk references. From 2f05d0e2f7a1798b1f4bf1d493bace168fbe5c17 Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Mon, 12 Dec 2022 20:06:53 +0100 Subject: [PATCH 6/7] Delete spurious file Signed-off-by: Marco Pracucci --- after.txt | 156 ------------------------------------------------------ 1 file changed, 156 deletions(-) delete mode 100644 after.txt diff --git a/after.txt b/after.txt deleted file mode 100644 index b0342666c1..0000000000 --- a/after.txt +++ /dev/null @@ -1,156 +0,0 @@ -goos: darwin -goarch: amd64 -pkg: github.com/grafana/mimir/pkg/storegateway -cpu: Intel(R) Core(TM) i7-9750H CPU @ 2.60GHz -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1of1000000-12 12 105212618 ns/op 65894878 B/op 9718 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1of1000000-12 12 101783084 ns/op 65916123 B/op 9791 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1of1000000-12 12 102742071 ns/op 65902762 B/op 9745 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/10of1000000-12 12 101336812 ns/op 65915603 B/op 9900 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/10of1000000-12 12 100542850 ns/op 65921138 B/op 9919 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/10of1000000-12 12 100741674 ns/op 65912051 B/op 9925 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1610142014 ns/op 1785269440 B/op 18040140 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1498285673 ns/op 1785364144 B/op 18040466 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1538114335 ns/op 1785365968 B/op 18040472 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1of1000000-12 19 69453047 ns/op 84033607 B/op 14334 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1of1000000-12 18 69592116 ns/op 84029653 B/op 14346 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1of1000000-12 18 68726174 ns/op 84027077 B/op 14328 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/10of1000000-12 18 70563126 ns/op 84053538 B/op 14520 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/10of1000000-12 18 68991483 ns/op 84058184 B/op 14523 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/10of1000000-12 16 69645096 ns/op 84045576 B/op 14459 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 1 1309276361 ns/op 1369696280 B/op 17113886 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 1 1276190903 ns/op 1369630272 B/op 17113426 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 1 1283585236 ns/op 1369992456 B/op 17113823 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1of1000000-12 20 59032948 ns/op 59020070 B/op 6209 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1of1000000-12 21 57018934 ns/op 59055768 B/op 6177 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1of1000000-12 20 56512528 ns/op 59021763 B/op 6226 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/10of1000000-12 21 58332122 ns/op 59102731 B/op 6372 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/10of1000000-12 20 57198341 ns/op 59076132 B/op 6347 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/10of1000000-12 22 57534860 ns/op 58960205 B/op 6314 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 1 1212858241 ns/op 1243813928 B/op 17031473 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 1 1172316030 ns/op 1245723552 B/op 17031420 allocs/op -BenchmarkBucket_Series/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 1 1152077241 ns/op 1241707248 B/op 17031345 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 183 5902946 ns/op 5766383 B/op 823 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 202 5911141 ns/op 5755185 B/op 824 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 200 5928218 ns/op 5757599 B/op 821 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 202 5893376 ns/op 5796483 B/op 878 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 200 5854063 ns/op 5762387 B/op 876 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 200 5875823 ns/op 5760151 B/op 873 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 7 151719780 ns/op 168422265 B/op 1703813 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 7 153107110 ns/op 168435073 B/op 1703810 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 7 152092974 ns/op 168441956 B/op 1703816 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/1of10000000-12 206 5631274 ns/op 5039336 B/op 1140 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/1of10000000-12 210 5873516 ns/op 5041745 B/op 1141 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/1of10000000-12 195 5971325 ns/op 5040939 B/op 1141 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/100of10000000-12 199 5768342 ns/op 5045793 B/op 1200 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/100of10000000-12 213 5638804 ns/op 5046069 B/op 1201 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/100of10000000-12 214 5645288 ns/op 5046671 B/op 1202 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 7 158912220 ns/op 210183737 B/op 1805079 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 6 167314699 ns/op 208606137 B/op 1805029 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 7 159058937 ns/op 209523800 B/op 1805046 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 153 7381460 ns/op 8226830 B/op 1645 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 162 7441410 ns/op 8223953 B/op 1643 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 162 7348889 ns/op 8228856 B/op 1646 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 163 7258858 ns/op 8223920 B/op 1697 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 160 7346886 ns/op 8223687 B/op 1698 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 164 7325498 ns/op 8230169 B/op 1698 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 8 138505173 ns/op 173327277 B/op 1712145 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 8 137346493 ns/op 173558787 B/op 1712169 allocs/op -BenchmarkBucket_Series/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 8 138935676 ns/op 173523663 B/op 1712179 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/1of10000000-12 5203 220290 ns/op 216773 B/op 244 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/1of10000000-12 5370 219966 ns/op 216648 B/op 244 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/1of10000000-12 5083 219098 ns/op 216877 B/op 244 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/100of10000000-12 5288 222883 ns/op 216690 B/op 244 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/100of10000000-12 5149 224897 ns/op 216828 B/op 244 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/100of10000000-12 5265 221965 ns/op 216991 B/op 244 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 14 75400969 ns/op 222873250 B/op 334376 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 14 77376612 ns/op 222878426 B/op 334376 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 15 77324473 ns/op 222872757 B/op 334373 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 5157 230146 ns/op 219429 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 4795 230649 ns/op 218710 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/1of10000000-12 4880 230327 ns/op 219722 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 4953 229773 ns/op 219017 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 4986 231037 ns/op 219063 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/100of10000000-12 4760 230651 ns/op 218885 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 13 84214928 ns/op 225667629 B/op 334392 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 13 83222970 ns/op 225688360 B/op 334394 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 13 86494644 ns/op 225684052 B/op 334393 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 5092 231296 ns/op 242135 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 5016 231846 ns/op 242343 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/1of10000000-12 4936 232337 ns/op 241464 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 4928 232404 ns/op 242999 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 4797 233477 ns/op 241992 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/100of10000000-12 5049 233607 ns/op 242494 B/op 250 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 13 84488245 ns/op 227280200 B/op 334393 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 13 83557445 ns/op 226493801 B/op 334389 allocs/op -BenchmarkBucket_Series/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 13 85308763 ns/op 227015115 B/op 334392 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1237335391 ns/op 1954554480 B/op 11040550 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1216079860 ns/op 1954527768 B/op 11040178 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_default_options/1000000of1000000-12 1 1175049608 ns/op 1954459160 B/op 11039942 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 2 901306368 ns/op 946180020 B/op 10061967 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 2 914481941 ns/op 946290036 B/op 10061841 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(1K_per_batch)/1000000of1000000-12 2 929324603 ns/op 946390348 B/op 10062105 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 2 889820006 ns/op 849389416 B/op 10025880 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 2 903275285 ns/op 850368904 B/op 10025930 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1000000SeriesWith1Samples/with_series_streaming_(10K_per_batch)/1000000of1000000-12 2 894877436 ns/op 849602768 B/op 10025786 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 10 112541764 ns/op 179408691 B/op 1104667 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 10 110965852 ns/op 179408440 B/op 1104666 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_default_options/10000000of10000000-12 9 111299456 ns/op 179413842 B/op 1104685 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 12 95640817 ns/op 93578367 B/op 1006675 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 12 95241279 ns/op 93580946 B/op 1006697 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 12 95300976 ns/op 93568014 B/op 1006694 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 10 106613918 ns/op 86317970 B/op 1003030 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 10 107329630 ns/op 86270081 B/op 1003046 allocs/op -BenchmarkBucket_Series_WithSkipChunks/100000SeriesWith100Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 10 109322237 ns/op 86537753 B/op 1003037 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 3236 383036 ns/op 708944 B/op 798 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 3004 384284 ns/op 708951 B/op 798 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_default_options/10000000of10000000-12 3105 385719 ns/op 708940 B/op 798 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 2029 565212 ns/op 718828 B/op 620 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 2056 564602 ns/op 719045 B/op 620 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(1K_per_batch)/10000000of10000000-12 2024 569710 ns/op 719093 B/op 620 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 2020 553268 ns/op 863589 B/op 619 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 1998 554960 ns/op 860112 B/op 618 allocs/op -BenchmarkBucket_Series_WithSkipChunks/1SeriesWith10000000Samples/with_series_streaming_(10K_per_batch)/10000000of10000000-12 2083 556228 ns/op 864514 B/op 619 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_1-12 496362 2422 ns/op 211 B/op 5 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_1-12 486528 2408 ns/op 212 B/op 5 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_1-12 493654 2415 ns/op 209 B/op 5 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_2-12 141573 8457 ns/op 668 B/op 15 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_2-12 139905 8445 ns/op 672 B/op 15 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_2-12 141340 8440 ns/op 670 B/op 15 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_4-12 59212 20159 ns/op 1589 B/op 35 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_4-12 59539 20240 ns/op 1602 B/op 35 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_4-12 58899 20394 ns/op 1611 B/op 35 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_8-12 26733 44726 ns/op 3455 B/op 75 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_8-12 27036 44470 ns/op 3445 B/op 75 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_8-12 27285 45032 ns/op 3426 B/op 75 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_16-12 13040 96256 ns/op 7137 B/op 155 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_16-12 13161 92098 ns/op 7042 B/op 155 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_16-12 13089 92216 ns/op 7137 B/op 155 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_32-12 6094 186640 ns/op 14343 B/op 315 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_32-12 6469 189337 ns/op 14329 B/op 315 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_32-12 6188 188606 ns/op 14458 B/op 315 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_64-12 3098 380703 ns/op 28760 B/op 635 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_64-12 3138 377290 ns/op 29225 B/op 635 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_true_number_of_iterators_=_64-12 3108 379811 ns/op 29153 B/op 635 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_1-12 496096 2386 ns/op 212 B/op 5 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_1-12 490495 2402 ns/op 211 B/op 5 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_1-12 487287 2388 ns/op 210 B/op 5 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_2-12 104146 11581 ns/op 682 B/op 15 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_2-12 103983 11522 ns/op 687 B/op 15 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_2-12 104235 11640 ns/op 679 B/op 15 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_4-12 34095 35290 ns/op 1646 B/op 35 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_4-12 33603 35492 ns/op 1611 B/op 35 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_4-12 33639 35711 ns/op 1625 B/op 35 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_8-12 10000 100748 ns/op 3542 B/op 75 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_8-12 10000 101554 ns/op 3541 B/op 75 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_8-12 10000 101558 ns/op 3566 B/op 75 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_16-12 4136 278946 ns/op 7465 B/op 155 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_16-12 4251 277353 ns/op 7103 B/op 155 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_16-12 4173 283901 ns/op 7401 B/op 155 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_32-12 1474 779892 ns/op 14936 B/op 315 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_32-12 1509 782160 ns/op 15405 B/op 315 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_32-12 1531 781083 ns/op 15065 B/op 315 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_64-12 528 2252992 ns/op 31262 B/op 636 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_64-12 525 2257777 ns/op 29873 B/op 636 allocs/op -BenchmarkMergedSeriesChunkRefsSetIterators/with_duplicated_series_=_false_number_of_iterators_=_64-12 528 2249254 ns/op 28464 B/op 636 allocs/op -PASS -ok github.com/grafana/mimir/pkg/storegateway 272.287s From 774ce9c5e5a2078062d1de5e8e099dbe75ad8560 Mon Sep 17 00:00:00 2001 From: Marco Pracucci Date: Tue, 13 Dec 2022 18:25:16 +0100 Subject: [PATCH 7/7] Address review feedback Signed-off-by: Marco Pracucci --- pkg/storegateway/bucket_test.go | 16 ++++++++-------- pkg/storegateway/series_refs.go | 2 +- pkg/storegateway/series_refs_test.go | 16 ++++++++-------- pkg/util/pool/pool.go | 4 ++-- 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/pkg/storegateway/bucket_test.go b/pkg/storegateway/bucket_test.go index a36b53afed..f3b70d4799 100644 --- a/pkg/storegateway/bucket_test.go +++ b/pkg/storegateway/bucket_test.go @@ -1212,7 +1212,7 @@ func TestBucket_Series_Concurrency(t *testing.T) { for _, batchSize := range []int{len(expectedSeries) / 100, len(expectedSeries) * 2} { t.Run(fmt.Sprintf("batch size: %d", batchSize), func(t *testing.T) { // Reset the memory pool tracker. - seriesChunkRefsSetPool.(*trackedGenericPool).reset() + seriesChunkRefsSetPool.(*trackedPool).reset() metaFetcher, err := block.NewRawMetaFetcher(logger, instrumentedBucket) assert.NoError(t, err) @@ -1266,8 +1266,8 @@ func TestBucket_Series_Concurrency(t *testing.T) { // Ensure the seriesChunkRefsSet memory pool has been used and all slices pulled from // pool have put back. - assert.Greater(t, seriesChunkRefsSetPool.(*trackedGenericPool).gets.Load(), int64(0)) - assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*trackedGenericPool).balance.Load()) + assert.Greater(t, seriesChunkRefsSetPool.(*trackedPool).gets.Load(), int64(0)) + assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*trackedPool).balance.Load()) }) } } @@ -1293,24 +1293,24 @@ func (m *trackedBytesPool) Put(b *[]byte) { m.parent.Put(b) } -type trackedGenericPool struct { - parent pool.Generic +type trackedPool struct { + parent pool.Interface balance atomic.Int64 gets atomic.Int64 } -func (p *trackedGenericPool) Get() any { +func (p *trackedPool) Get() any { p.balance.Inc() p.gets.Inc() return p.parent.Get() } -func (p *trackedGenericPool) Put(x any) { +func (p *trackedPool) Put(x any) { p.balance.Dec() p.parent.Put(x) } -func (p *trackedGenericPool) reset() { +func (p *trackedPool) reset() { p.balance.Store(0) p.gets.Store(0) } diff --git a/pkg/storegateway/series_refs.go b/pkg/storegateway/series_refs.go index 6a2bb122ed..958fd4a143 100644 --- a/pkg/storegateway/series_refs.go +++ b/pkg/storegateway/series_refs.go @@ -20,7 +20,7 @@ import ( ) var ( - seriesChunkRefsSetPool = pool.Generic(&sync.Pool{ + seriesChunkRefsSetPool = pool.Interface(&sync.Pool{ // Intentionally return nil if the pool is empty, so that the caller can preallocate // the slice with the right size. New: nil, diff --git a/pkg/storegateway/series_refs_test.go b/pkg/storegateway/series_refs_test.go index c2427fb91a..59d39e4ec8 100644 --- a/pkg/storegateway/series_refs_test.go +++ b/pkg/storegateway/series_refs_test.go @@ -27,7 +27,7 @@ import ( func init() { // Track the balance of gets/puts to seriesChunkRefsSetPool in all tests. - seriesChunkRefsSetPool = &trackedGenericPool{parent: seriesChunkRefsSetPool} + seriesChunkRefsSetPool = &trackedPool{parent: seriesChunkRefsSetPool} } func TestSeriesChunkRef_Compare(t *testing.T) { @@ -636,7 +636,7 @@ func TestMergedSeriesChunkRefsSet_Concurrency(t *testing.T) { } // Reset the memory pool tracker. - seriesChunkRefsSetPool.(*trackedGenericPool).reset() + seriesChunkRefsSetPool.(*trackedPool).reset() g, _ := errgroup.WithContext(context.Background()) for c := 0; c < concurrency; c++ { @@ -652,8 +652,8 @@ func TestMergedSeriesChunkRefsSet_Concurrency(t *testing.T) { // Ensure the seriesChunkRefsSet memory pool has been used and all slices pulled from // pool have put back. - assert.Greater(t, seriesChunkRefsSetPool.(*trackedGenericPool).gets.Load(), int64(0)) - assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*trackedGenericPool).balance.Load()) + assert.Greater(t, seriesChunkRefsSetPool.(*trackedPool).gets.Load(), int64(0)) + assert.Equal(t, int64(0), seriesChunkRefsSetPool.(*trackedPool).balance.Load()) } func BenchmarkMergedSeriesChunkRefsSetIterators(b *testing.B) { @@ -1529,7 +1529,7 @@ func createSeriesChunkRefsSet(minSeriesID, maxSeriesID int, releasable bool) ser for seriesID := minSeriesID; seriesID <= maxSeriesID; seriesID++ { set.series = append(set.series, seriesChunkRefs{ - lset: labels.New(labels.Label{Name: labels.MetricName, Value: fmt.Sprintf("metric_%06d", seriesID)}), + lset: labels.FromStrings(labels.MetricName, fmt.Sprintf("metric_%06d", seriesID)), }) } @@ -1539,7 +1539,7 @@ func createSeriesChunkRefsSet(minSeriesID, maxSeriesID int, releasable bool) ser func TestCreateSeriesChunkRefsSet(t *testing.T) { set := createSeriesChunkRefsSet(5, 7, true) require.Len(t, set.series, 3) - assert.Equal(t, seriesChunkRefs{lset: labels.New(labels.Label{Name: labels.MetricName, Value: "metric_000005"})}, set.series[0]) - assert.Equal(t, seriesChunkRefs{lset: labels.New(labels.Label{Name: labels.MetricName, Value: "metric_000006"})}, set.series[1]) - assert.Equal(t, seriesChunkRefs{lset: labels.New(labels.Label{Name: labels.MetricName, Value: "metric_000007"})}, set.series[2]) + assert.Equal(t, seriesChunkRefs{lset: labels.FromStrings(labels.MetricName, "metric_000005")}, set.series[0]) + assert.Equal(t, seriesChunkRefs{lset: labels.FromStrings(labels.MetricName, "metric_000006")}, set.series[1]) + assert.Equal(t, seriesChunkRefs{lset: labels.FromStrings(labels.MetricName, "metric_000007")}, set.series[2]) } diff --git a/pkg/util/pool/pool.go b/pkg/util/pool/pool.go index 74e205c6c0..b6fc5bf227 100644 --- a/pkg/util/pool/pool.go +++ b/pkg/util/pool/pool.go @@ -11,8 +11,8 @@ import ( "github.com/pkg/errors" ) -// Generic memory pool with the same interface of sync.Pool. -type Generic interface { +// Interface defines the same functions of sync.Pool. +type Interface interface { // Put is sync.Pool.Put(). Put(x any) // Get is sync.Pool.Get().