From f39fc783a9faadf2705a1787ffb5b73ba88e4760 Mon Sep 17 00:00:00 2001 From: Bartlomiej Plotka Date: Thu, 30 Apr 2020 20:09:44 +0100 Subject: [PATCH] querier: Adjust deduplication for counters when querying for PromQL rates. Signed-off-by: Bartlomiej Plotka --- pkg/compact/downsample/downsample.go | 26 +++++------ pkg/compact/downsample/downsample_test.go | 12 ++--- pkg/query/iter.go | 55 ++++++++++++++++++++--- pkg/query/querier.go | 2 +- pkg/query/querier_test.go | 35 +++++++-------- 5 files changed, 86 insertions(+), 44 deletions(-) diff --git a/pkg/compact/downsample/downsample.go b/pkg/compact/downsample/downsample.go index d9c7f99f29..c8c0387001 100644 --- a/pkg/compact/downsample/downsample.go +++ b/pkg/compact/downsample/downsample.go @@ -348,12 +348,12 @@ func downsampleRawLoop(data []sample, resolution int64, numChunks int) []chunks. ab := newAggrChunkBuilder() - // Encode first raw value; see CounterSeriesIterator. + // Encode first raw value; see ApplyCounterResetsSeriesIterator. ab.apps[AggrCounter].Append(batch[0].t, batch[0].v) lastT := downsampleBatch(batch, resolution, ab.add) - // Encode last raw value; see CounterSeriesIterator. + // Encode last raw value; see ApplyCounterResetsSeriesIterator. ab.apps[AggrCounter].Append(lastT, batch[len(batch)-1].v) chks = append(chks, ab.encode()) @@ -525,7 +525,7 @@ func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (ch acs = append(acs, c.Iterator(reuseIt)) } *buf = (*buf)[:0] - it := NewCounterSeriesIterator(acs...) + it := NewApplyCounterResetsIterator(acs...) if err := expandChunkIterator(it, buf); err != nil { return chk, err @@ -538,7 +538,7 @@ func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (ch ab.chunks[AggrCounter] = chunkenc.NewXORChunk() ab.apps[AggrCounter], _ = ab.chunks[AggrCounter].Appender() - // Retain first raw value; see CounterSeriesIterator. + // Retain first raw value; see ApplyCounterResetsSeriesIterator. ab.apps[AggrCounter].Append((*buf)[0].t, (*buf)[0].v) lastT := downsampleBatch(*buf, resolution, func(t int64, a *aggregator) { @@ -550,7 +550,7 @@ func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (ch ab.apps[AggrCounter].Append(t, a.counter) }) - // Retain last raw value; see CounterSeriesIterator. + // Retain last raw value; see ApplyCounterResetsSeriesIterator. ab.apps[AggrCounter].Append(lastT, it.lastV) ab.mint = mint @@ -563,7 +563,7 @@ type sample struct { v float64 } -// CounterSeriesIterator generates monotonically increasing values by iterating +// ApplyCounterResetsSeriesIterator generates monotonically increasing values by iterating // over an ordered sequence of chunks, which should be raw or aggregated chunks // of counter values. The generated samples can be used by PromQL functions // like 'rate' that calculate differences between counter values. Stale Markers @@ -580,7 +580,7 @@ type sample struct { // It handles overlapped chunks (removes overlaps). // NOTE: It is important to deduplicate with care ensuring that you don't hit // issue https://github.com/thanos-io/thanos/issues/2401#issuecomment-621958839. -type CounterSeriesIterator struct { +type ApplyCounterResetsSeriesIterator struct { chks []chunkenc.Iterator i int // Current chunk. total int // Total number of processed samples. @@ -589,11 +589,11 @@ type CounterSeriesIterator struct { totalV float64 // Total counter state since beginning of series. } -func NewCounterSeriesIterator(chks ...chunkenc.Iterator) *CounterSeriesIterator { - return &CounterSeriesIterator{chks: chks} +func NewApplyCounterResetsIterator(chks ...chunkenc.Iterator) *ApplyCounterResetsSeriesIterator { + return &ApplyCounterResetsSeriesIterator{chks: chks} } -func (it *CounterSeriesIterator) Next() bool { +func (it *ApplyCounterResetsSeriesIterator) Next() bool { for { if it.i >= len(it.chks) { return false @@ -637,11 +637,11 @@ func (it *CounterSeriesIterator) Next() bool { } } -func (it *CounterSeriesIterator) At() (t int64, v float64) { +func (it *ApplyCounterResetsSeriesIterator) At() (t int64, v float64) { return it.lastT, it.totalV } -func (it *CounterSeriesIterator) Seek(x int64) bool { +func (it *ApplyCounterResetsSeriesIterator) Seek(x int64) bool { for { if t, _ := it.At(); t >= x { return true @@ -654,7 +654,7 @@ func (it *CounterSeriesIterator) Seek(x int64) bool { } } -func (it *CounterSeriesIterator) Err() error { +func (it *ApplyCounterResetsSeriesIterator) Err() error { if it.i >= len(it.chks) { return nil } diff --git a/pkg/compact/downsample/downsample_test.go b/pkg/compact/downsample/downsample_test.go index 049765faa5..47fe161a9a 100644 --- a/pkg/compact/downsample/downsample_test.go +++ b/pkg/compact/downsample/downsample_test.go @@ -60,7 +60,7 @@ func TestDownsampleCounterBoundaryReset(t *testing.T) { iters = append(iters, chk.Iterator(nil)) } - citer := NewCounterSeriesIterator(iters...) + citer := NewApplyCounterResetsIterator(iters...) for citer.Next() { t, v := citer.At() res = append(res, sample{t: t, v: v}) @@ -592,7 +592,7 @@ var ( } ) -func TestCounterAggegationIterator(t *testing.T) { +func TestApplyCounterResetsIterator(t *testing.T) { defer leaktest.CheckTimeout(t, 10*time.Second)() for _, tcase := range []struct { @@ -657,7 +657,7 @@ func TestCounterAggegationIterator(t *testing.T) { its = append(its, newSampleIterator(c)) } - x := NewCounterSeriesIterator(its...) + x := NewApplyCounterResetsIterator(its...) var res []sample for x.Next() { @@ -691,7 +691,7 @@ func TestCounterSeriesIteratorSeek(t *testing.T) { } var res []sample - x := NewCounterSeriesIterator(its...) + x := NewApplyCounterResetsIterator(its...) ok := x.Seek(150) testutil.Assert(t, ok, "Seek should return true") @@ -718,7 +718,7 @@ func TestCounterSeriesIteratorSeekExtendTs(t *testing.T) { its = append(its, newSampleIterator(c)) } - x := NewCounterSeriesIterator(its...) + x := NewApplyCounterResetsIterator(its...) ok := x.Seek(500) testutil.Assert(t, !ok, "Seek should return false") @@ -738,7 +738,7 @@ func TestCounterSeriesIteratorSeekAfterNext(t *testing.T) { } var res []sample - x := NewCounterSeriesIterator(its...) + x := NewApplyCounterResetsIterator(its...) x.Next() diff --git a/pkg/query/iter.go b/pkg/query/iter.go index 0067d08a3c..29fe7e23a1 100644 --- a/pkg/query/iter.go +++ b/pkg/query/iter.go @@ -210,7 +210,7 @@ func (s *chunkSeries) Iterator() storage.SeriesIterator { for _, c := range s.chunks { its = append(its, getFirstIterator(c.Counter, c.Raw)) } - sit = downsample.NewCounterSeriesIterator(its...) + sit = downsample.NewApplyCounterResetsIterator(its...) default: return errSeriesIterator{err: errors.Errorf("unexpected result aggregate type %v", s.aggrs)} } @@ -376,6 +376,7 @@ func (it *chunkSeriesIterator) Err() error { type dedupSeriesSet struct { set storage.SeriesSet replicaLabels map[string]struct{} + isCounter bool replicas []storage.Series lset labels.Labels @@ -383,8 +384,8 @@ type dedupSeriesSet struct { ok bool } -func newDedupSeriesSet(set storage.SeriesSet, replicaLabels map[string]struct{}) storage.SeriesSet { - s := &dedupSeriesSet{set: set, replicaLabels: replicaLabels} +func newDedupSeriesSet(set storage.SeriesSet, replicaLabels map[string]struct{}, isCounter bool) storage.SeriesSet { + s := &dedupSeriesSet{set: set, replicaLabels: replicaLabels, isCounter: isCounter} s.ok = s.set.Next() if s.ok { s.peek = s.set.At() @@ -447,7 +448,7 @@ func (s *dedupSeriesSet) At() storage.Series { // Clients may store the series, so we must make a copy of the slice before advancing. repl := make([]storage.Series, len(s.replicas)) copy(repl, s.replicas) - return newDedupSeries(s.lset, repl...) + return newDedupSeries(s.lset, repl, s.isCounter) } func (s *dedupSeriesSet) Err() error { @@ -464,10 +465,12 @@ func (s seriesWithLabels) Labels() labels.Labels { return s.lset } type dedupSeries struct { lset labels.Labels replicas []storage.Series + + isCounter bool } -func newDedupSeries(lset labels.Labels, replicas ...storage.Series) *dedupSeries { - return &dedupSeries{lset: lset, replicas: replicas} +func newDedupSeries(lset labels.Labels, replicas []storage.Series, isCounter bool) *dedupSeries { + return &dedupSeries{lset: lset, isCounter: isCounter, replicas: replicas} } func (s *dedupSeries) Labels() labels.Labels { @@ -479,6 +482,9 @@ func (s *dedupSeries) Iterator() (it storage.SeriesIterator) { for _, o := range s.replicas[1:] { it = newDedupSeriesIterator(it, o.Iterator()) } + if s.isCounter { + return newCounterDedupAdjustSeriesIterator(it) + } return it } @@ -587,3 +593,40 @@ func (it *dedupSeriesIterator) Err() error { } return it.b.Err() } + +// counterDedupAdjustSeriesIterator is used when we deduplicate counter. +// It makes sure we always adjust for the latest seen last counter value for all replicas. +// This short-term solution to mitigate https://github.com/thanos-io/thanos/issues/2401. +// TODO(bwplotka): Find better deduplication algorithm that does not require knowledge if the given +// series is counter or not: https://github.com/thanos-io/thanos/issues/2547. +type counterDedupAdjustSeriesIterator struct { + storage.SeriesIterator + + lastV float64 + adjust float64 +} + +func newCounterDedupAdjustSeriesIterator(iter storage.SeriesIterator) storage.SeriesIterator { + return &counterDedupAdjustSeriesIterator{ + SeriesIterator: iter, + lastV: -1 * math.MaxFloat64, + } + +} + +func (it *counterDedupAdjustSeriesIterator) Next() bool { + if it.SeriesIterator.Next() { + _, v := it.SeriesIterator.At() + if it.lastV > v { + it.adjust += it.lastV - v + } + it.lastV = v + return true + } + return false +} + +func (it *counterDedupAdjustSeriesIterator) At() (int64, float64) { + t, v := it.SeriesIterator.At() + return t, v + it.adjust +} diff --git a/pkg/query/querier.go b/pkg/query/querier.go index 791232597d..e437440ed7 100644 --- a/pkg/query/querier.go +++ b/pkg/query/querier.go @@ -228,7 +228,7 @@ func (q *querier) Select(params *storage.SelectParams, ms ...*labels.Matcher) (s // The merged series set assembles all potentially-overlapping time ranges // of the same series into a single one. The series are ordered so that equal series // from different replicas are sequential. We can now deduplicate those. - return newDedupSeriesSet(set, q.replicaLabels), warns, nil + return newDedupSeriesSet(set, q.replicaLabels, len(aggrs) == 1 && aggrs[0] == storepb.Aggr_COUNTER), warns, nil } // sortDedupLabels re-sorts the set so that the same series with different replica diff --git a/pkg/query/querier_test.go b/pkg/query/querier_test.go index 98402d49f5..873fa72e1d 100644 --- a/pkg/query/querier_test.go +++ b/pkg/query/querier_test.go @@ -274,13 +274,13 @@ var ( {t: 1587691805791, v: 488036}, {t: 1587691820791, v: 488241}, {t: 1587691835791, v: 488411}, {t: 1587691850791, v: 488625}, {t: 1587691865791, v: 488868}, {t: 1587691880791, v: 489005}, {t: 1587691895791, v: 489237}, {t: 1587691910791, v: 489545}, {t: 1587691925791, v: 489750}, {t: 1587691940791, v: 489899}, {t: 1587691955791, v: 490048}, {t: 1587691970791, v: 490364}, {t: 1587691985791, v: 490485}, {t: 1587692000791, v: 490722}, {t: 1587692015791, v: 490866}, {t: 1587692030791, v: 491025}, {t: 1587692045791, v: 491286}, {t: 1587692060816, v: 491543}, {t: 1587692075791, v: 491787}, {t: 1587692090791, v: 492065}, {t: 1587692105791, v: 492223}, {t: 1587692120816, v: 492501}, {t: 1587692135791, v: 492767}, {t: 1587692150791, v: 492955}, {t: 1587692165791, v: 493194}, {t: 1587692180792, v: 493402}, {t: 1587692195791, v: 493647}, {t: 1587692210791, v: 493897}, {t: 1587692225791, v: 494117}, {t: 1587692240805, v: 494356}, {t: 1587692255791, v: 494620}, {t: 1587692270791, v: 494762}, {t: 1587692285791, v: 495001}, {t: 1587692300805, v: 495222}, {t: 1587692315791, v: 495393}, {t: 1587692330791, v: 495662}, - {t: 1587692345791, v: 495875}, {t: 1587692360801, v: 496082}, {t: 1587692375791, v: 496196}, {t: 1587692390791, v: 496245}, {t: 1587692405791, v: 496295}, {t: 1587692420791, v: 496365}, {t: 1587692435791, v: 496401}, {t: 1587692450791, v: 496452}, {t: 1587692465791, v: 496491}, {t: 1587692480791, v: 496544}, {t: 1587692542149, v: 496537}, {t: 1587692557139, v: 496633}, - {t: 1587692572139, v: 496844}, {t: 1587692587139, v: 497040}, {t: 1587692602144, v: 497257}, {t: 1587692617139, v: 497522}, {t: 1587692632139, v: 497710}, {t: 1587692647139, v: 497938}, {t: 1587692662154, v: 498172}, {t: 1587692677139, v: 498459}, {t: 1587692692139, v: 498635}, {t: 1587692707139, v: 498832}, {t: 1587692722139, v: 499014}, {t: 1587692737139, v: 499170}, - {t: 1587692752139, v: 499338}, {t: 1587692767139, v: 499511}, {t: 1587692782149, v: 499719}, {t: 1587692797139, v: 499973}, {t: 1587692812139, v: 500189}, {t: 1587692827139, v: 500359}, {t: 1587692842139, v: 500517}, {t: 1587692857139, v: 500727}, {t: 1587692872139, v: 500959}, {t: 1587692887139, v: 501178}, {t: 1587692902139, v: 501246}, {t: 1587692917153, v: 501404}, - {t: 1587692932139, v: 501663}, {t: 1587692947139, v: 501850}, {t: 1587692962139, v: 502103}, {t: 1587692977155, v: 502280}, {t: 1587692992139, v: 502562}, {t: 1587693007139, v: 502742}, {t: 1587693022139, v: 502931}, {t: 1587693037139, v: 503190}, {t: 1587693052139, v: 503428}, {t: 1587693067139, v: 503630}, {t: 1587693082139, v: 503873}, {t: 1587693097139, v: 504027}, - {t: 1587693112139, v: 504179}, {t: 1587693127139, v: 504362}, {t: 1587693142139, v: 504590}, {t: 1587693157139, v: 504741}, {t: 1587693172139, v: 505056}, {t: 1587693187139, v: 505244}, {t: 1587693202139, v: 505436}, {t: 1587693217139, v: 505635}, {t: 1587693232139, v: 505936}, {t: 1587693247155, v: 506088}, {t: 1587693262139, v: 506309}, {t: 1587693277139, v: 506524}, - {t: 1587693292139, v: 506800}, {t: 1587693307139, v: 507010}, {t: 1587693322139, v: 507286}, {t: 1587693337139, v: 507530}, {t: 1587693352139, v: 507781}, {t: 1587693367139, v: 507991}, {t: 1587693382139, v: 508310}, {t: 1587693397139, v: 508570}, {t: 1587693412139, v: 508770}, {t: 1587693427139, v: 508982}, {t: 1587693442163, v: 509274}, {t: 1587693457139, v: 509477}, - {t: 1587693472139, v: 509713}, {t: 1587693487139, v: 509972}, {t: 1587693502139, v: 510182}, {t: 1587693517139, v: 510498}, {t: 1587693532139, v: 510654}, {t: 1587693547139, v: 510859}, {t: 1587693562139, v: 511124}, {t: 1587693577139, v: 511314}, {t: 1587693592139, v: 511488}, + {t: 1587692345791, v: 495875}, {t: 1587692360801, v: 496082}, {t: 1587692375791, v: 496196}, {t: 1587692390791, v: 496245}, {t: 1587692405791, v: 496295}, {t: 1587692420791, v: 496365}, {t: 1587692435791, v: 496401}, {t: 1587692450791, v: 496452}, {t: 1587692465791, v: 496491}, {t: 1587692480791, v: 496544}, {t: 1587692542149, v: 496544}, {t: 1587692557139, v: 496640}, + {t: 1587692572139, v: 496851}, {t: 1587692587139, v: 497047}, {t: 1587692602144, v: 497264}, {t: 1587692617139, v: 497529}, {t: 1587692632139, v: 497717}, {t: 1587692647139, v: 497945}, {t: 1587692662154, v: 498179}, {t: 1587692677139, v: 498466}, {t: 1587692692139, v: 498642}, {t: 1587692707139, v: 498839}, {t: 1587692722139, v: 499021}, {t: 1587692737139, v: 499177}, + {t: 1587692752139, v: 499345}, {t: 1587692767139, v: 499518}, {t: 1587692782149, v: 499726}, {t: 1587692797139, v: 499980}, {t: 1587692812139, v: 500196}, {t: 1587692827139, v: 500366}, {t: 1587692842139, v: 500524}, {t: 1587692857139, v: 500734}, {t: 1587692872139, v: 500966}, {t: 1587692887139, v: 501185}, {t: 1587692902139, v: 501253}, {t: 1587692917153, v: 501411}, + {t: 1587692932139, v: 501670}, {t: 1587692947139, v: 501857}, {t: 1587692962139, v: 502110}, {t: 1587692977155, v: 502287}, {t: 1587692992139, v: 502569}, {t: 1587693007139, v: 502749}, {t: 1587693022139, v: 502938}, {t: 1587693037139, v: 503197}, {t: 1587693052139, v: 503435}, {t: 1587693067139, v: 503637}, {t: 1587693082139, v: 503880}, {t: 1587693097139, v: 504034}, + {t: 1587693112139, v: 504186}, {t: 1587693127139, v: 504369}, {t: 1587693142139, v: 504597}, {t: 1587693157139, v: 504748}, {t: 1587693172139, v: 505063}, {t: 1587693187139, v: 505251}, {t: 1587693202139, v: 505443}, {t: 1587693217139, v: 505642}, {t: 1587693232139, v: 505943}, {t: 1587693247155, v: 506095}, {t: 1587693262139, v: 506316}, {t: 1587693277139, v: 506531}, + {t: 1587693292139, v: 506807}, {t: 1587693307139, v: 507017}, {t: 1587693322139, v: 507293}, {t: 1587693337139, v: 507537}, {t: 1587693352139, v: 507788}, {t: 1587693367139, v: 507998}, {t: 1587693382139, v: 508317}, {t: 1587693397139, v: 508577}, {t: 1587693412139, v: 508777}, {t: 1587693427139, v: 508989}, {t: 1587693442163, v: 509281}, {t: 1587693457139, v: 509484}, + {t: 1587693472139, v: 509720}, {t: 1587693487139, v: 509979}, {t: 1587693502139, v: 510189}, {t: 1587693517139, v: 510505}, {t: 1587693532139, v: 510661}, {t: 1587693547139, v: 510866}, {t: 1587693562139, v: 511131}, {t: 1587693577139, v: 511321}, {t: 1587693592139, v: 511495}, } ) @@ -730,7 +730,6 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { }) // Regression test against https://github.com/thanos-io/thanos/issues/2401. // Rate + dedup can cause incorrectness. - // TODO(bwplotka): To fix in next PR. t.Run("dedup=true", func(t *testing.T) { expectedLset := labels.FromStrings( "action", "widget.json", "controller", "Projects::MergeRequests::ContentController", "env", "gprd", "environment", @@ -764,8 +763,8 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { {T: 1587691100000, V: 15.922807017543859}, {T: 1587691200000, V: 15.63157894736842}, {T: 1587691300000, V: 14.982456140350878}, {T: 1587691400000, V: 14.187259188557551}, {T: 1587691500000, V: 13.828070175438594}, {T: 1587691600000, V: 13.971929824561403}, {T: 1587691700000, V: 15.31994329585807}, {T: 1587691800000, V: 14.30877192982456}, {T: 1587691900000, V: 13.915789473684212}, {T: 1587692000000, V: 13.312280701754384}, {T: 1587692100000, V: 14.136842105263158}, {T: 1587692200000, V: 14.39298245614035}, - {T: 1587692300000, V: 15.014035087719297}, {T: 1587692400000, V: 14.112280701754386}, {T: 1587692500000, V: 9.421065148148147}, {T: 1587692600000, V: 1740.491873127187}, // Suddenly unexpected rate spike. - {T: 1587692700000, V: 1742.229734448992}, {T: 1587692800000, V: 11.918703026416258}, {T: 1587692900000, V: 13.75813610765101}, {T: 1587693000000, V: 13.087719298245615}, + {T: 1587692300000, V: 15.014035087719297}, {T: 1587692400000, V: 14.112280701754386}, {T: 1587692500000, V: 9.421065148148147}, {T: 1587692600000, V: 6.3736754978451735}, + {T: 1587692700000, V: 8.19632056099571}, {T: 1587692800000, V: 11.918703026416258}, {T: 1587692900000, V: 13.75813610765101}, {T: 1587693000000, V: 13.087719298245615}, {T: 1587693100000, V: 13.466666666666667}, {T: 1587693200000, V: 14.028070175438595}, {T: 1587693300000, V: 14.23859649122807}, {T: 1587693400000, V: 15.407017543859647}, {T: 1587693500000, V: 15.915789473684208}, {T: 1587693600000, V: 15.712280701754386}, }}, @@ -786,8 +785,8 @@ func TestQuerierWithDedupUnderstoodByPromQL_Rate(t *testing.T) { {Metric: expectedLset, Points: []promql.Point{ {T: 1587691800000, V: 14.464425770308123}, {T: 1587692300000, V: 14.763025210084033}, - {T: 1587692800000, V: 291.105652426067}, // Suddenly unexpected rate spike. - {T: 1587693300000, V: 290.89236811640285}, + {T: 1587692800000, V: 13.143575607888273}, + {T: 1587693300000, V: 12.930291298224086}, }}, }, vec) }) @@ -955,6 +954,7 @@ func TestDedupSeriesSet(t *testing.T) { input []series exp []series dedupLabels map[string]struct{} + isCounter bool }{ { // Single dedup label. @@ -1096,7 +1096,7 @@ func TestDedupSeriesSet(t *testing.T) { // Now, depending on what replica we look, we can see totally different counter value in total where total means // after accounting for counter resets. We account for that in downsample.CounterSeriesIterator, mainly because // we handle downsample Counter Aggregations specially (for detecting resets between chunks). - // TODO(bwplotka): Fix in next PR. + isCounter: true, input: []series{ { lset: labels.Labels{{Name: "replica", Value: "01"}}, @@ -1117,9 +1117,8 @@ func TestDedupSeriesSet(t *testing.T) { }, exp: []series{ { - lset: labels.Labels{}, - // Outcome for rate: Double counter reset accounted. - samples: []sample{{10000, 8}, {20000, 9}, {45001, 8.5}}, + lset: labels.Labels{}, + samples: []sample{{10000, 8}, {20000, 9}, {45001, 9}}, }, }, dedupLabels: map[string]struct{}{ @@ -1129,7 +1128,7 @@ func TestDedupSeriesSet(t *testing.T) { { // Regression test on real data against https://github.com/thanos-io/thanos/issues/2401. // Real data with stale marker after downsample.CounterSeriesIterator (required for downsampling + rate). - // TODO(bwplotka): Fix in next PR. + isCounter: true, input: []series{ { lset: labels.Labels{{Name: "replica", Value: "01"}}, @@ -1199,7 +1198,7 @@ func TestDedupSeriesSet(t *testing.T) { for _, tcase := range tests { t.Run("", func(t *testing.T) { - dedupSet := newDedupSeriesSet(&mockedSeriesSet{series: tcase.input}, tcase.dedupLabels) + dedupSet := newDedupSeriesSet(&mockedSeriesSet{series: tcase.input}, tcase.dedupLabels, tcase.isCounter) var ats []storage.Series for dedupSet.Next() { ats = append(ats, dedupSet.At())