Skip to content

Commit

Permalink
querier: Adjust deduplication for counters when querying for PromQL r…
Browse files Browse the repository at this point in the history
…ates.

Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com>
  • Loading branch information
bwplotka committed Apr 30, 2020
1 parent 5139013 commit 7fdb9fd
Show file tree
Hide file tree
Showing 5 changed files with 86 additions and 44 deletions.
26 changes: 13 additions & 13 deletions pkg/compact/downsample/downsample.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
Expand Down Expand Up @@ -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
Expand All @@ -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) {
Expand All @@ -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
Expand All @@ -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
Expand All @@ -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.
Expand All @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
}
Expand Down
12 changes: 6 additions & 6 deletions pkg/compact/downsample/downsample_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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})
Expand Down Expand Up @@ -590,7 +590,7 @@ var (
}
)

func TestCounterAggegationIterator(t *testing.T) {
func TestApplyCounterResetsIterator(t *testing.T) {
defer leaktest.CheckTimeout(t, 10*time.Second)()

for _, tcase := range []struct {
Expand Down Expand Up @@ -655,7 +655,7 @@ func TestCounterAggegationIterator(t *testing.T) {
its = append(its, newSampleIterator(c))
}

x := NewCounterSeriesIterator(its...)
x := NewApplyCounterResetsIterator(its...)

var res []sample
for x.Next() {
Expand Down Expand Up @@ -689,7 +689,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")
Expand All @@ -716,7 +716,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")
Expand All @@ -736,7 +736,7 @@ func TestCounterSeriesIteratorSeekAfterNext(t *testing.T) {
}

var res []sample
x := NewCounterSeriesIterator(its...)
x := NewApplyCounterResetsIterator(its...)

x.Next()

Expand Down
55 changes: 49 additions & 6 deletions pkg/query/iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)}
}
Expand Down Expand Up @@ -376,15 +376,16 @@ func (it *chunkSeriesIterator) Err() error {
type dedupSeriesSet struct {
set storage.SeriesSet
replicaLabels map[string]struct{}
isCounter bool

replicas []storage.Series
lset labels.Labels
peek storage.Series
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()
Expand Down Expand Up @@ -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 {
Expand All @@ -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 {
Expand All @@ -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
}

Expand Down Expand Up @@ -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
}
2 changes: 1 addition & 1 deletion pkg/query/querier.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
35 changes: 17 additions & 18 deletions pkg/query/querier_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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},
}
)

Expand Down Expand Up @@ -729,7 +729,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",
Expand Down Expand Up @@ -763,8 +762,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},
}},
Expand All @@ -785,8 +784,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)
})
Expand Down Expand Up @@ -953,6 +952,7 @@ func TestDedupSeriesSet(t *testing.T) {
input []series
exp []series
dedupLabels map[string]struct{}
isCounter bool
}{
{
// Single dedup label.
Expand Down Expand Up @@ -1094,7 +1094,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"}},
Expand All @@ -1115,9 +1115,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{}{
Expand All @@ -1127,7 +1126,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"}},
Expand Down Expand Up @@ -1197,7 +1196,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())
Expand Down

0 comments on commit 7fdb9fd

Please sign in to comment.