From faef3c8f8b0928d7aa4ab025fc049986f9bdf024 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Fri, 28 Apr 2023 11:13:10 +0200 Subject: [PATCH 1/9] Added native histogram support for downsampling Signed-off-by: Sebastian Rabenhorst Fixed imports Signed-off-by: Sebastian Rabenhorst Added native histogram support for downsampling Signed-off-by: Sebastian Rabenhorst Improved comment Signed-off-by: Sebastian Rabenhorst Removed unnecessary if Signed-off-by: Sebastian Rabenhorst Fixed native histogram proto conversion in remote engine Signed-off-by: Sebastian Rabenhorst Added downsampling support for native histograms Reverted files from base PR Added changelog Signed-off-by: Sebastian Rabenhorst Reverted error in bucket.go Signed-off-by: Sebastian Rabenhorst trigger tests Signed-off-by: Sebastian Rabenhorst --- CHANGELOG.md | 1 + pkg/compact/downsample/aggr_test.go | 10 +- pkg/compact/downsample/downsample.go | 696 ++++++++++++++++++---- pkg/compact/downsample/downsample_test.go | 516 +++++++++++++--- pkg/compact/downsample/testutils.go | 71 +++ pkg/store/bucket.go | 19 +- pkg/store/bucket_test.go | 53 +- pkg/store/storepb/testutil/series.go | 63 +- pkg/store/tsdb_test.go | 8 +- 9 files changed, 1176 insertions(+), 261 deletions(-) create mode 100644 pkg/compact/downsample/testutils.go diff --git a/CHANGELOG.md b/CHANGELOG.md index c6f012aebe..66bedd98f1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -61,6 +61,7 @@ We use *breaking :warning:* to mark changes that are not backward compatible (re - [#6163](https://github.com/thanos-io/thanos/pull/6163) Receiver: changed max backoff from 30s to 5s for forwarding requests. Can be configured with `--receive-forward-max-backoff`. - [#6327](https://github.com/thanos-io/thanos/pull/6327) *: *breaking :warning:* Use histograms instead of summaries for instrumented handlers. - [#6322](https://github.com/thanos-io/thanos/pull/6322) Logging: Avoid expensive log.Valuer evaluation for disallowed levels. +- [#6350] https://github.com/thanos-io/thanos/pull/6350 Compact: Add native histogram support for downsampling. - [#6358](https://github.com/thanos-io/thanos/pull/6358) Query: Add +Inf bucket to query duration metrics - [#6363](https://github.com/thanos-io/thanos/pull/6363) Store: Check context error when expanding postings. - [#6405](https://github.com/thanos-io/thanos/pull/6405) Index Cache: Change postings cache key to include the encoding format used so that older Thanos versions would not try to decode it during the deployment of a new version. diff --git a/pkg/compact/downsample/aggr_test.go b/pkg/compact/downsample/aggr_test.go index d2b44a461a..a35058e829 100644 --- a/pkg/compact/downsample/aggr_test.go +++ b/pkg/compact/downsample/aggr_test.go @@ -14,11 +14,11 @@ import ( func TestAggrChunk(t *testing.T) { var input [5][]sample - input[AggrCount] = []sample{{100, 30}, {200, 50}, {300, 60}, {400, 67}} - input[AggrSum] = []sample{{100, 130}, {200, 1000}, {300, 2000}, {400, 5555}} - input[AggrMin] = []sample{{100, 0}, {200, -10}, {300, 1000}, {400, -9.5}} + input[AggrCount] = []sample{{t: 100, v: 30}, {t: 200, v: 50}, {t: 300, v: 60}, {t: 400, v: 67}} + input[AggrSum] = []sample{{t: 100, v: 130}, {t: 200, v: 1000}, {t: 300, v: 2000}, {t: 400, v: 5555}} + input[AggrMin] = []sample{{t: 100}, {t: 200, v: -10}, {t: 300, v: 1000}, {t: 400, v: -9.5}} // Maximum is absent. - input[AggrCounter] = []sample{{100, 5}, {200, 10}, {300, 10.1}, {400, 15}, {400, 3}} + input[AggrCounter] = []sample{{t: 100, v: 5}, {t: 200, v: 10}, {t: 300, v: 10.1}, {t: 400, v: 15}, {t: 400, v: 3}} var chks [5]chunkenc.Chunk @@ -41,7 +41,7 @@ func TestAggrChunk(t *testing.T) { for _, at := range []AggrType{AggrCount, AggrSum, AggrMin, AggrMax, AggrCounter} { if c, err := ac.Get(at); err != ErrAggrNotExist { testutil.Ok(t, err) - testutil.Ok(t, expandChunkIterator(c.Iterator(nil), &res[at])) + testutil.Ok(t, expandXorChunkIterator(c.Iterator(nil), &res[at])) } } testutil.Equals(t, input, res) diff --git a/pkg/compact/downsample/downsample.go b/pkg/compact/downsample/downsample.go index 78b615f904..39fe82ff9d 100644 --- a/pkg/compact/downsample/downsample.go +++ b/pkg/compact/downsample/downsample.go @@ -150,7 +150,7 @@ func Downsample( // TODO(bwplotka): We can optimze this further by using in WriteSeries iterators of each chunk instead of // samples. Also ensure 120 sample limit, otherwise we have gigantic chunks. // https://github.com/thanos-io/thanos/issues/2542. - if err := expandChunkIterator(c.Chunk.Iterator(reuseIt), &all); err != nil { + if err := expandChunkIterator(c.Chunk.Iterator(reuseIt), c.Chunk.Encoding(), &all); err != nil { return id, errors.Wrapf(err, "expand chunk %d, series %d", c.Ref, postings.At()) } } @@ -168,7 +168,7 @@ func Downsample( level.Warn(logger).Log("msg", fmt.Sprintf("expected downsampled chunk (*downsample.AggrChunk) got an empty %T instead for series: %d", c.Chunk, postings.At())) continue } else { - if err := expandChunkIterator(c.Chunk.Iterator(reuseIt), &all); err != nil { + if err := expandChunkIterator(c.Chunk.Iterator(reuseIt), c.Chunk.Encoding(), &all); err != nil { return id, errors.Wrapf(err, "expand chunk %d, series %d", c.Ref, postings.At()) } aggrDataChunks := DownsampleRaw(all, ResLevel1) @@ -179,10 +179,8 @@ func Downsample( } } } - } aggrChunks = append(aggrChunks, ac) - } downsampledChunks, err := downsampleAggr( aggrChunks, @@ -208,6 +206,28 @@ func Downsample( return } +func isHistogram(c *AggrChunk) bool { + // If it is an aggregated chunk histogram chunk, the counter will be of the type histogram. + cntr, err := c.Get(AggrCounter) + if err != nil { + return false + } + return cntr.Encoding() == chunkenc.EncHistogram || cntr.Encoding() == chunkenc.EncFloatHistogram +} + +func expandChunkIterator(it chunkenc.Iterator, encoding chunkenc.Encoding, samples *[]sample) error { + switch encoding { + case chunkenc.EncXOR: + return expandXorChunkIterator(it, samples) + case chunkenc.EncFloatHistogram: + return expandFloatHistogramChunkIterator(it, samples) + case chunkenc.EncHistogram: + return expandHistogramChunkIterator(it, samples) + default: + return errors.Errorf("unexpected chunk encoding %s", encoding) + } +} + // currentWindow returns the end timestamp of the window that t falls into. func currentWindow(t, r int64) int64 { // The next timestamp is the next number after s.t that's aligned with window. @@ -243,8 +263,14 @@ func targetChunkCount(mint, maxt, inRes, outRes int64, count int) (x int) { return x } -// aggregator collects cumulative stats for a stream of values. -type aggregator struct { +type sampleAggregator interface { + reset() + add(s sample) + processedSamples() int +} + +// floatAggregator collects cumulative stats for a stream of values. +type floatAggregator struct { total int // Total samples processed. count int // Samples in current window. sum float64 // Value sum of current window. @@ -256,41 +282,121 @@ type aggregator struct { } // reset the stats to start a new aggregation window. -func (a *aggregator) reset() { - a.count = 0 - a.sum = 0 - a.min = math.MaxFloat64 - a.max = -math.MaxFloat64 +func (f *floatAggregator) reset() { + f.count = 0 + f.sum = 0 + f.min = math.MaxFloat64 + f.max = -math.MaxFloat64 } -func (a *aggregator) add(v float64) { - if a.total > 0 { - if v < a.last { +func (f *floatAggregator) add(s sample) { + if f.total > 0 { + if s.v < f.last { // Counter reset, correct the value. - a.counter += v - a.resets++ + f.counter += s.v + f.resets++ } else { // Add delta with last value to the counter. - a.counter += v - a.last + f.counter += s.v - f.last } } else { // First sample sets the counter. - a.counter = v + f.counter = s.v } - a.last = v + f.last = s.v - a.sum += v - a.count++ - a.total++ + f.sum += s.v + f.count++ + f.total++ - if v < a.min { - a.min = v + if s.v < f.min { + f.min = s.v + } + if s.v > f.max { + f.max = s.v } - if v > a.max { - a.max = v +} + +func (f *floatAggregator) processedSamples() int { + return f.total +} + +type histogramAggregator struct { + total int // Total histograms processed. + count int // Histograms in current window. + sum *histogram.FloatHistogram // Value sum of current window (for gauge histograms). + counter *histogram.FloatHistogram // Total counter state since beginning (for counter histograms). + resets int // Number of counter resets since beginning. + previous *histogram.FloatHistogram // Previously added value. + schema int32 // Smallest schema in the batch that's being aggregated. + resetDetected bool // Whether a reset was detected since the beginning. +} + +func newHistogramAggregator(schema int32) *histogramAggregator { + return &histogramAggregator{ + schema: schema, } } +func (h *histogramAggregator) reset() { + h.count = 0 + h.sum = nil +} + +func (h *histogramAggregator) add(s sample) { + fh := s.fh + if fh.Schema < h.schema { + panic("schema should be greater or equal to aggregator schema") + } + + // If schema of the sample is greater than the + // aggregator schema, we need to reduce the resolution. + if fh.Schema > h.schema { + fh = fh.CopyToSchema(h.schema) + } + + if h.total > 0 { + if fh.DetectReset(h.previous) { + // Counter reset, correct the value. + h.counter.Add(fh) + h.resets++ + h.resetDetected = true + } else { + // Add delta with previous value to the counter. + h.counter.Add(fh.Copy().Sub(h.previous)) + } + } else { + // First sample sets the counter. + h.counter = fh.Copy() + } + + if h.sum == nil { + if h.resetDetected { + h.sum = h.counter.Copy() + } else { + h.sum = fh.Copy() + } + // This needs to be h gauge histogram, otherwise reset detection will be triggered + // when appending the aggregated chunk and histogram.count < appender.count. + h.sum.CounterResetHint = histogram.GaugeType + } else { + if h.resetDetected { + h.sum.Add(h.counter) + } else { + h.sum.Add(fh) + } + } + + h.previous = fh + + h.count++ + h.total++ +} + +func (h *histogramAggregator) processedSamples() int { + return h.total +} + // aggrChunkBuilder builds chunks for multiple different aggregates. type aggrChunkBuilder struct { mint, maxt int64 @@ -319,13 +425,46 @@ func newAggrChunkBuilder() *aggrChunkBuilder { return b } -func (b *aggrChunkBuilder) add(t int64, aggr *aggregator) { +func newHistogramAggrChunkBuilder(counterIsGauge bool) *aggrChunkBuilder { + b := &aggrChunkBuilder{ + mint: math.MaxInt64, + maxt: math.MinInt64, + } + + b.chunks[AggrCount] = chunkenc.NewXORChunk() + + // Sum aggregate always needs to be gauge type, otherwise + // append can fail when sum_window_n > sum_window_n+1 (.i.e + // if last window contains fewer samples than previous window). + sumChunk := chunkenc.NewFloatHistogramChunk() + sumChunk.SetCounterResetHeader(chunkenc.GaugeType) + b.chunks[AggrSum] = sumChunk + + counterChunk := chunkenc.NewFloatHistogramChunk() + // For gauge histograms the counter needs to be set to gauge type, + // so we can append to chunk when sum_window_n > sum_window_n+1. + if counterIsGauge { + counterChunk.SetCounterResetHeader(chunkenc.GaugeType) + } + b.chunks[AggrCounter] = counterChunk + + for i, c := range b.chunks { + if c != nil { + b.apps[i], _ = c.Appender() + } + } + return b +} + +func (b *aggrChunkBuilder) add(t int64, a sampleAggregator) { if t < b.mint { b.mint = t } if t > b.maxt { b.maxt = t } + + aggr := mustGetFloatAggregator(a) b.apps[AggrSum].Append(t, aggr.sum) b.apps[AggrMin].Append(t, aggr.min) b.apps[AggrMax].Append(t, aggr.max) @@ -335,6 +474,72 @@ func (b *aggrChunkBuilder) add(t int64, aggr *aggregator) { b.added++ } +func (b *aggrChunkBuilder) addHistogram(t int64, a sampleAggregator) { + if t < b.mint { + b.mint = t + } + if t > b.maxt { + b.maxt = t + } + + aggr := mustGetHistogramAggregator(a) + b.appendFloatHistogram(AggrCounter, t, aggr.counter) + b.appendFloatHistogram(AggrSum, t, aggr.sum) + b.apps[AggrCount].Append(t, float64(aggr.count)) + + b.added++ +} + +// Mostly copied from https://github.com/prometheus/prometheus/blob/fd8992cdbd6bf7274f2a815e4af64a0b8734841b/tsdb/head_append.go#L1235-L1326 +// but we shouldn't have the not appendable case. +func (b *aggrChunkBuilder) appendFloatHistogram(t AggrType, ts int64, fh *histogram.FloatHistogram) { + app := b.apps[t].(*chunkenc.FloatHistogramAppender) + + if app.NumSamples() > 0 { + var ( + pForwardInserts, nForwardInserts []chunkenc.Insert + pBackwardInserts, nBackwardInserts []chunkenc.Insert + pMergedSpans, nMergedSpans []histogram.Span + okToAppend bool + ) + + switch fh.CounterResetHint { + case histogram.GaugeType: + if app != nil { + var () + pForwardInserts, nForwardInserts, + pBackwardInserts, nBackwardInserts, + pMergedSpans, nMergedSpans, + okToAppend = app.AppendableGauge(fh) + } + default: + pForwardInserts, nForwardInserts, okToAppend, _ = app.Appendable(fh) + } + + // If not appendable we did something wrong. + if !okToAppend { + panic("Not appendable") + } + + if pBackwardInserts != nil || nBackwardInserts != nil { + fh.PositiveSpans = pMergedSpans + fh.NegativeSpans = nMergedSpans + app.RecodeHistogramm(fh, pBackwardInserts, nBackwardInserts) + } + + if len(pForwardInserts) > 0 || len(nForwardInserts) > 0 { + rChunk, rApp := app.Recode( + pForwardInserts, nForwardInserts, + fh.PositiveSpans, fh.NegativeSpans, + ) + b.apps[t] = rApp + b.chunks[t] = rChunk + } + } + + b.apps[t].AppendFloatHistogram(ts, fh) +} + func (b *aggrChunkBuilder) encode() chunks.Meta { return chunks.Meta{ MinTime: b.mint, @@ -343,22 +548,55 @@ func (b *aggrChunkBuilder) encode() chunks.Meta { } } +func mustGetFloatAggregator(aggr sampleAggregator) *floatAggregator { + a, ok := aggr.(*floatAggregator) + if !ok { + panic("aggregator is not of type *floatAggregator") + } + return a +} + +func mustGetHistogramAggregator(aggr sampleAggregator) *histogramAggregator { + a, ok := aggr.(*histogramAggregator) + if !ok { + panic("aggregator is not of type *histogramAggregator") + } + return a +} + // DownsampleRaw create a series of aggregation chunks for the given sample data. func DownsampleRaw(data []sample, resolution int64) []chunks.Meta { if len(data) == 0 { return nil } - mint, maxt := data[0].t, data[len(data)-1].t - // We assume a raw resolution of 1 minute. In practice it will often be lower - // but this is sufficient for our heuristic to produce well-sized chunks. - numChunks := targetChunkCount(mint, maxt, 1*60*1000, resolution, len(data)) - return downsampleRawLoop(data, resolution, numChunks) + min := data[0].t + max := data[len(data)-1].t + numChunks := targetChunkCount(min, max, 1*60*1000, resolution, len(data)) + chks := make([]chunks.Meta, 0, numChunks) + + // First sample determines the type of the samples, since we process + // one chunk and all samples of one chunk have the same type. + if data[0].fh != nil { + downsampleRawLoop(data, resolution, numChunks, &chks, downsampleHistogramBatch) + } else { + downsampleRawLoop(data, resolution, numChunks, &chks, downsampleFloatBatch) + } + + return chks } -func downsampleRawLoop(data []sample, resolution int64, numChunks int) []chunks.Meta { +func downsampleRawLoop( + data []sample, + resolution int64, + numChunks int, + chks *[]chunks.Meta, + downsampleBatch func(batch []sample, resolution int64) chunks.Meta, +) { + if len(data) == 0 { + return + } batchSize := (len(data) / numChunks) + 1 - chks := make([]chunks.Meta, 0, numChunks) for len(data) > 0 { j := batchSize @@ -375,38 +613,60 @@ func downsampleRawLoop(data []sample, resolution int64, numChunks int) []chunks. batch := data[:j] data = data[j:] - ab := newAggrChunkBuilder() - - // Encode first raw value; see ApplyCounterResetsSeriesIterator. - ab.apps[AggrCounter].Append(batch[0].t, batch[0].v) + *chks = append(*chks, downsampleBatch(batch, resolution)) + } +} - lastT := downsampleBatch(batch, resolution, ab.add) +func downsampleFloatBatch(batch []sample, resolution int64) chunks.Meta { + ab := newAggrChunkBuilder() + ab.apps[AggrCounter].Append(batch[0].t, batch[0].v) + lastT := downsampleBatch(batch, resolution, &floatAggregator{}, ab.add) + ab.apps[AggrCounter].Append(lastT, batch[len(batch)-1].v) + return ab.encode() +} - // Encode last raw value; see ApplyCounterResetsSeriesIterator. - ab.apps[AggrCounter].Append(lastT, batch[len(batch)-1].v) +func downsampleHistogramBatch(batch []sample, resolution int64) chunks.Meta { + // We need to know the smallest schema in advanced otherwise we might end + // up with a non appendable histogram if histogram.schema < chunk.schema. + schema := minSchema(batch) + ab := newHistogramAggrChunkBuilder(isGaugeSamples(batch)) + downsampleBatch(batch, resolution, newHistogramAggregator(schema), ab.addHistogram) + return ab.encode() +} - chks = append(chks, ab.encode()) +func minSchema(samples []sample) int32 { + schema := int32(math.MaxInt32) + for _, s := range samples { + if s.fh != nil && !value.IsStaleNaN(s.fh.Sum) && s.fh.Schema < schema { + schema = s.fh.Schema + } } + return schema +} - return chks +func isGaugeSamples(samples []sample) bool { + if len(samples) == 0 { + return false + } + return samples[0].fh.CounterResetHint == histogram.GaugeType } // downsampleBatch aggregates the data over the given resolution and calls add each time // the end of a resolution was reached. -func downsampleBatch(data []sample, resolution int64, add func(int64, *aggregator)) int64 { +func downsampleBatch(data []sample, resolution int64, aggr sampleAggregator, add func(int64, sampleAggregator)) int64 { var ( - aggr aggregator nextT = int64(-1) lastT = data[len(data)-1].t ) + // Fill up one aggregate chunk with up to m samples. for _, s := range data { - if value.IsStaleNaN(s.v) { + if isStale(s) { continue } if s.t > nextT { if nextT != -1 { - add(nextT, &aggr) + add(nextT, aggr) } aggr.reset() nextT = currentWindow(s.t, resolution) @@ -418,25 +678,64 @@ func downsampleBatch(data []sample, resolution int64, add func(int64, *aggregato nextT = lastT } } - aggr.add(s.v) + aggr.add(s) + } + + // Add the last sample if any samples were processed. + if aggr.processedSamples() > 0 { + add(nextT, aggr) } - // Add the last sample. - add(nextT, &aggr) return nextT } +func isStale(s sample) bool { + if value.IsStaleNaN(s.v) { + return true + } + + if s.fh != nil && value.IsStaleNaN(s.fh.Sum) { + return true + } + + return false +} + // downsampleAggr downsamples a sequence of aggregation chunks to the given resolution. -func downsampleAggr(chks []*AggrChunk, buf *[]sample, mint, maxt, inRes, outRes int64) ([]chunks.Meta, error) { +func downsampleAggr( + chks []*AggrChunk, + buf *[]sample, + mint, maxt, inRes, outRes int64, +) ([]chunks.Meta, error) { + var fChks, hChks []*AggrChunk var numSamples int + for _, c := range chks { - numSamples += c.NumSamples() + if isHistogram(c) { + hChks = append(hChks, c) + numSamples += c.NumSamples() + } else { + fChks = append(fChks, c) + numSamples += c.NumSamples() + } } - numChunks := targetChunkCount(mint, maxt, inRes, outRes, numSamples) - return downsampleAggrLoop(chks, buf, outRes, numChunks) + + if len(fChks) > 0 { + numChunksF := targetChunkCount(mint, maxt, inRes, outRes, numSamples) + return downsampleAggrLoop(fChks, buf, outRes, numChunksF, downsampleAggrBatch) + } + + numChunksH := targetChunkCount(mint, maxt, inRes, outRes, numSamples) + return downsampleAggrLoop(hChks, buf, outRes, numChunksH, downsampleHistogramAggrBatch) } -func downsampleAggrLoop(chks []*AggrChunk, buf *[]sample, resolution int64, numChunks int) ([]chunks.Meta, error) { +func downsampleAggrLoop( + chks []*AggrChunk, + buf *[]sample, + resolution int64, + numChunks int, + downsampleAggr func(chks []*AggrChunk, buf *[]sample, resolution int64) (chk chunks.Meta, err error), +) ([]chunks.Meta, error) { // We downsample aggregates only along chunk boundaries. This is required // for counters to be downsampled correctly since a chunk's first and last // counter values are the true values of the original series. We need @@ -452,7 +751,12 @@ func downsampleAggrLoop(chks []*AggrChunk, buf *[]sample, resolution int64, numC part := chks[:j] chks = chks[j:] - chk, err := downsampleAggrBatch(part, buf, resolution) + chk, err := downsampleAggr(part, buf, resolution) + if chk.MinTime == math.MaxInt64 || chk.MaxTime == math.MinInt64 { + msg := fmt.Sprintf("invalid range for downsampled aggregate chunk: mint=%d maxt=%d", chk.MinTime, chk.MaxTime) + return nil, errors.New(msg) + } + if err != nil { return nil, err } @@ -462,9 +766,9 @@ func downsampleAggrLoop(chks []*AggrChunk, buf *[]sample, resolution int64, numC return res, nil } -// expandChunkIterator reads all samples from the iterator and appends them to buf. +// expandXorChunkIterator reads all samples from the iterator and appends them to buf. // Stale markers and out of order samples are skipped. -func expandChunkIterator(it chunkenc.Iterator, buf *[]sample) error { +func expandXorChunkIterator(it chunkenc.Iterator, buf *[]sample) error { // For safety reasons, we check for each sample that it does not go back in time. // If it does, we skip it. lastT := int64(0) @@ -475,13 +779,96 @@ func expandChunkIterator(it chunkenc.Iterator, buf *[]sample) error { continue } if t >= lastT { - *buf = append(*buf, sample{t, v}) + *buf = append(*buf, sample{t: t, v: v}) + lastT = t + } + } + return it.Err() +} + +// expandHistogramChunkIterator reads all histograms from the iterator and appends them to buf. +func expandHistogramChunkIterator(it chunkenc.Iterator, buf *[]sample) error { + // For safety reasons, we check for each sample that it does not go back in time. + // If it does, we skip it. + lastT := int64(0) + + for it.Next() != chunkenc.ValNone { + t, h := it.AtHistogram() + if value.IsStaleNaN(h.Sum) { + continue + } + if t >= lastT { + *buf = append(*buf, sample{t: t, fh: h.ToFloat()}) + lastT = t + } + } + return it.Err() +} + +// expandHistogramChunkIterator reads all histograms from the iterator and appends them to buf. +func expandFloatHistogramChunkIterator(it chunkenc.Iterator, buf *[]sample) error { + // For safety reasons, we check for each sample that it does not go back in time. + // If it does, we skip it. + lastT := int64(0) + + for it.Next() != chunkenc.ValNone { + t, fh := it.AtFloatHistogram() + if value.IsStaleNaN(fh.Sum) { + continue + } + if t >= lastT { + *buf = append(*buf, sample{t: t, fh: fh}) lastT = t } } return it.Err() } +// genericAggregate does a generic aggregation for count, sum, min, and max aggregates. +// Counters need special treatment. +func genericAggregate( + at AggrType, + chks []*AggrChunk, + buf *[]sample, + ab *aggrChunkBuilder, + resolution int64, + f func(a sampleAggregator) float64, +) (int64, int64, error) { + var mint, maxt int64 = math.MaxInt64, math.MinInt64 + var reuseIt chunkenc.Iterator + *buf = (*buf)[:0] + + // Expand all samples for the aggregate type. + for _, chk := range chks { + c, err := chk.Get(at) + if err == ErrAggrNotExist { + continue + } else if err != nil { + return 0, 0, err + } + if err := expandXorChunkIterator(c.Iterator(reuseIt), buf); err != nil { + return 0, 0, err + } + } + if len(*buf) == 0 { + return 0, 0, nil + } + ab.chunks[at] = chunkenc.NewXORChunk() + ab.apps[at], _ = ab.chunks[at].Appender() + + downsampleBatch(*buf, resolution, &floatAggregator{}, func(t int64, a sampleAggregator) { + if t < mint { + mint = t + } + if t > maxt { + maxt = t + } + ab.apps[at].Append(t, f(a)) + }) + + return mint, maxt, nil +} + func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (chk chunks.Meta, err error) { ab := &aggrChunkBuilder{} mint, maxt := int64(math.MaxInt64), int64(math.MinInt64) @@ -489,55 +876,36 @@ func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (ch // do does a generic aggregation for count, sum, min, and max aggregates. // Counters need special treatment. - do := func(at AggrType, f func(a *aggregator) float64) error { - *buf = (*buf)[:0] - // Expand all samples for the aggregate type. - for _, chk := range chks { - c, err := chk.Get(at) - if err == ErrAggrNotExist { - continue - } else if err != nil { - return err - } - if err := expandChunkIterator(c.Iterator(reuseIt), buf); err != nil { - return err - } + do := func(at AggrType, f func(a sampleAggregator) float64) error { + aggrMint, aggrMaxt, err := genericAggregate(at, chks, buf, ab, resolution, f) + if aggrMint < mint { + mint = aggrMint } - if len(*buf) == 0 { - return nil + if aggrMaxt > maxt { + maxt = aggrMaxt } - ab.chunks[at] = chunkenc.NewXORChunk() - ab.apps[at], _ = ab.chunks[at].Appender() - - downsampleBatch(*buf, resolution, func(t int64, a *aggregator) { - if t < mint { - mint = t - } else if t > maxt { - maxt = t - } - ab.apps[at].Append(t, f(a)) - }) - return nil + return err } - if err := do(AggrCount, func(a *aggregator) float64 { + if err := do(AggrCount, func(a sampleAggregator) float64 { // To get correct count of elements from already downsampled count chunk // we have to sum those values. - return a.sum + aggr := mustGetFloatAggregator(a) + return aggr.sum }); err != nil { return chk, err } - if err = do(AggrSum, func(a *aggregator) float64 { - return a.sum + if err = do(AggrSum, func(a sampleAggregator) float64 { + return mustGetFloatAggregator(a).sum }); err != nil { return chk, err } - if err := do(AggrMin, func(a *aggregator) float64 { - return a.min + if err := do(AggrMin, func(a sampleAggregator) float64 { + return mustGetFloatAggregator(a).min }); err != nil { return chk, err } - if err := do(AggrMax, func(a *aggregator) float64 { - return a.max + if err := do(AggrMax, func(a sampleAggregator) float64 { + return mustGetFloatAggregator(a).max }); err != nil { return chk, err } @@ -556,7 +924,7 @@ func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (ch *buf = (*buf)[:0] it := NewApplyCounterResetsIterator(acs...) - if err := expandChunkIterator(it, buf); err != nil { + if err := expandXorChunkIterator(it, buf); err != nil { return chk, err } if len(*buf) == 0 { @@ -570,13 +938,14 @@ func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (ch // 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) { + lastT := downsampleBatch(*buf, resolution, &floatAggregator{}, func(t int64, a sampleAggregator) { if t < mint { mint = t - } else if t > maxt { + } + if t > maxt { maxt = t } - ab.apps[AggrCounter].Append(t, a.counter) + ab.apps[AggrCounter].Append(t, mustGetFloatAggregator(a).counter) }) // Retain last raw value; see ApplyCounterResetsSeriesIterator. @@ -587,9 +956,89 @@ func downsampleAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (ch return ab.encode(), nil } +func downsampleHistogramAggrBatch(chks []*AggrChunk, buf *[]sample, resolution int64) (chk chunks.Meta, err error) { + mint, maxt := int64(math.MaxInt64), int64(math.MinInt64) + var reuseIt chunkenc.Iterator + + // Expand all samples for the counter aggregate type. + *buf = (*buf)[:0] + for _, achk := range chks { + c, err := achk.Get(AggrCounter) + if err == ErrAggrNotExist { + continue + } else if err != nil { + return chk, err + } + if err := expandFloatHistogramChunkIterator(c.Iterator(reuseIt), buf); err != nil { + return chk, err + } + } + + ab := newHistogramAggrChunkBuilder(isGaugeSamples(*buf)) + + schema := minSchema(*buf) + downsampleBatch(*buf, resolution, newHistogramAggregator(schema), func(t int64, a sampleAggregator) { + if t < mint { + mint = t + } + if t > maxt { + maxt = t + } + ab.appendFloatHistogram(AggrCounter, t, mustGetHistogramAggregator(a).counter) + }) + + // Expand all samples for the sum aggregate. + *buf = (*buf)[:0] + for _, achk := range chks { + c, err := achk.Get(AggrSum) + if err == ErrAggrNotExist { + continue + } else if err != nil { + return chk, err + } + if err := expandFloatHistogramChunkIterator(c.Iterator(reuseIt), buf); err != nil { + return chk, err + } + } + if len(*buf) == 0 { + return chk, nil + } + + schema = minSchema(*buf) + downsampleBatch(*buf, resolution, newHistogramAggregator(schema), func(t int64, a sampleAggregator) { + if t < mint { + mint = t + } + if t > maxt { + maxt = t + } + ab.appendFloatHistogram(AggrSum, t, mustGetHistogramAggregator(a).sum) + }) + + *buf = (*buf)[:0] + batchMint, batchMaxt, err := genericAggregate(AggrCount, chks, buf, ab, resolution, func(a sampleAggregator) float64 { + aggr := mustGetFloatAggregator(a) + return aggr.sum + }) + if err != nil { + return chk, err + } + if batchMint < mint { + mint = batchMint + } + if batchMaxt > maxt { + maxt = batchMaxt + } + + ab.mint = mint + ab.maxt = maxt + return ab.encode(), nil +} + type sample struct { - t int64 - v float64 + t int64 + v float64 + fh *histogram.FloatHistogram } // ApplyCounterResetsSeriesIterator generates monotonically increasing values by iterating @@ -601,7 +1050,7 @@ type sample struct { // Counter aggregation chunks must have the first and last values from their // original raw series: the first raw value should be the first value encoded // in the chunk, and the last raw value is encoded by the duplication of the -// previous sample's timestamp. As iteration occurs between chunks, the +// last sample's timestamp. As iteration occurs between chunks, the // comparison between the last raw value of the earlier chunk and the first raw // value of the later chunk ensures that counter resets between chunks are // recognized and that the correct value delta is calculated. @@ -718,6 +1167,7 @@ type AverageChunkIterator struct { sumIt chunkenc.Iterator t int64 v float64 + fh *histogram.FloatHistogram err error } @@ -725,24 +1175,37 @@ func NewAverageChunkIterator(cnt, sum chunkenc.Iterator) *AverageChunkIterator { return &AverageChunkIterator{cntIt: cnt, sumIt: sum} } -// TODO(rabenhorst): Native histogram support needs to be added, float type is hardcoded. func (it *AverageChunkIterator) Next() chunkenc.ValueType { - cok, sok := it.cntIt.Next(), it.sumIt.Next() - if cok != sok { + ct, st := it.cntIt.Next(), it.sumIt.Next() + + if (ct != chunkenc.ValNone && st == chunkenc.ValNone) || + (ct == chunkenc.ValNone && st != chunkenc.ValNone) { it.err = errors.New("sum and count iterator not aligned") return chunkenc.ValNone } - if cok == chunkenc.ValNone { + + if ct == chunkenc.ValNone { return chunkenc.ValNone } - cntT, cntV := it.cntIt.At() - sumT, sumV := it.sumIt.At() + cntT := it.cntIt.AtT() + sumT := it.sumIt.AtT() if cntT != sumT { it.err = errors.New("sum and count timestamps not aligned") return chunkenc.ValNone } - it.t, it.v = cntT, sumV/cntV + + it.t = cntT + _, cntV := it.cntIt.At() + + if st == chunkenc.ValFloatHistogram { + _, sumV := it.sumIt.AtFloatHistogram() + it.fh = sumV.Scale(1 / cntV) + return chunkenc.ValFloatHistogram + } + + _, sumV := it.sumIt.At() + it.v = sumV / cntV return chunkenc.ValFloat } @@ -752,16 +1215,23 @@ func (it *AverageChunkIterator) Seek(t int64) chunkenc.ValueType { } func (it *AverageChunkIterator) At() (int64, float64) { + if it.fh != nil { + panic("not float iterator") + } return it.t, it.v } -// TODO(rabenhorst): Needs to be implemented for native histogram support. +// This should never be called since the result of an histogram calculation +// is always a float histogram. func (it *AverageChunkIterator) AtHistogram() (int64, *histogram.Histogram) { panic("not implemented") } func (it *AverageChunkIterator) AtFloatHistogram() (int64, *histogram.FloatHistogram) { - panic("not implemented") + if it.fh == nil { + panic("not float histogram iterator") + } + return it.t, it.fh } func (it *AverageChunkIterator) AtT() int64 { @@ -779,9 +1249,9 @@ func (it *AverageChunkIterator) Err() error { } // SamplesFromTSDBSamples converts tsdbutil.Sample slice to samples. -func SamplesFromTSDBSamples(samples []tsdbutil.Sample) []sample { - res := make([]sample, len(samples)) - for i, s := range samples { +func SamplesFromTSDBSamples(spls []tsdbutil.Sample) []sample { + res := make([]sample, len(spls)) + for i, s := range spls { res[i] = sample{t: s.T(), v: s.F()} } return res diff --git a/pkg/compact/downsample/downsample_test.go b/pkg/compact/downsample/downsample_test.go index d1dfaeb9b8..0751b2fc6c 100644 --- a/pkg/compact/downsample/downsample_test.go +++ b/pkg/compact/downsample/downsample_test.go @@ -11,6 +11,8 @@ import ( "testing" "github.com/go-kit/log" + "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" "github.com/pkg/errors" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" @@ -52,7 +54,7 @@ func TestDownsampleCounterBoundaryReset(t *testing.T) { iter := chk.Iterator(nil) for iter.Next() != chunkenc.ValNone { t, v := iter.At() - res = append(res, sample{t, v}) + res = append(res, sample{t: t, v: v}) } } return @@ -131,7 +133,9 @@ func TestDownsampleCounterBoundaryReset(t *testing.T) { doTest := func(t *testing.T, test *test) { // Asking for more chunks than raw samples ensures that downsampleRawLoop // will create chunks with samples from a single window. - cm := downsampleRawLoop(test.raw, test.rawAggrResolution, len(test.raw)+1) + numChunks := len(test.raw) + 1 + cm := make([]chunks.Meta, 0, numChunks) + downsampleRawLoop(test.raw, test.rawAggrResolution, numChunks, &cm, downsampleFloatBatch) testutil.Equals(t, test.expectedRawAggrChunks, len(cm)) rawAggrChunks := toAggrChunks(t, cm) @@ -139,7 +143,7 @@ func TestDownsampleCounterBoundaryReset(t *testing.T) { testutil.Equals(t, test.rawCounterIterate, counterIterate(t, rawAggrChunks)) var buf []sample - acm, err := downsampleAggrLoop(rawAggrChunks, &buf, test.aggrAggrResolution, test.aggrChunks) + acm, err := downsampleAggrLoop(rawAggrChunks, &buf, test.aggrAggrResolution, test.aggrChunks, downsampleAggrBatch) testutil.Ok(t, err) testutil.Equals(t, test.aggrChunks, len(acm)) @@ -157,15 +161,15 @@ func TestExpandChunkIterator(t *testing.T) { // Same timestamps are okay since we use them for counter markers. var res []sample testutil.Ok(t, - expandChunkIterator( + expandXorChunkIterator( newSampleIterator([]sample{ - {100, 1}, {200, 2}, {200, 3}, {201, 4}, {200, 5}, - {300, 6}, {400, math.Float64frombits(value.StaleNaN)}, {500, 5}, + {t: 100, v: 1}, {t: 200, v: 2}, {t: 200, v: 3}, {t: 201, v: 4}, {t: 200, v: 5}, + {t: 300, v: 6}, {t: 400, v: math.Float64frombits(value.StaleNaN)}, {t: 500, v: 5}, }), &res, ), ) - testutil.Equals(t, []sample{{100, 1}, {200, 2}, {200, 3}, {201, 4}, {300, 6}, {500, 5}}, res) + testutil.Equals(t, []sample{{t: 100, v: 1}, {t: 200, v: 2}, {t: 200, v: 3}, {t: 201, v: 4}, {t: 300, v: 6}, {t: 500, v: 5}}, res) } var ( @@ -228,26 +232,26 @@ func TestDownsample(t *testing.T) { { name: "single chunk", inRaw: [][]sample{ - {{20, 1}, {40, 2}, {60, 3}, {80, 1}, {100, 2}, {101, math.Float64frombits(value.StaleNaN)}, {120, 5}, {180, 10}, {250, 1}}, + {{t: 20, v: 1}, {t: 40, v: 2}, {t: 60, v: 3}, {t: 80, v: 1}, {t: 100, v: 2}, {t: 101, v: math.Float64frombits(value.StaleNaN)}, {t: 120, v: 5}, {t: 180, v: 10}, {t: 250, v: 1}}, }, resolution: 100, expected: []map[AggrType][]sample{ { - AggrCount: {{99, 4}, {199, 3}, {250, 1}}, - AggrSum: {{99, 7}, {199, 17}, {250, 1}}, - AggrMin: {{99, 1}, {199, 2}, {250, 1}}, - AggrMax: {{99, 3}, {199, 10}, {250, 1}}, - AggrCounter: {{20, 1}, {99, 4}, {199, 13}, {250, 14}, {250, 1}}, + AggrCount: {{t: 99, v: 4}, {t: 199, v: 3}, {t: 250, v: 1}}, + AggrSum: {{t: 99, v: 7}, {t: 199, v: 17}, {t: 250, v: 1}}, + AggrMin: {{t: 99, v: 1}, {t: 199, v: 2}, {t: 250, v: 1}}, + AggrMax: {{t: 99, v: 3}, {t: 199, v: 10}, {t: 250, v: 1}}, + AggrCounter: {{t: 20, v: 1}, {t: 99, v: 4}, {t: 199, v: 13}, {t: 250, v: 14}, {t: 250, v: 1}}, }, }, }, { name: "three chunks", inRaw: [][]sample{ - {{20, 1}, {40, 2}, {60, 3}, {80, 1}, {100, 2}, {101, math.Float64frombits(value.StaleNaN)}, {120, 5}, {180, 10}, {250, 2}}, - {{260, 1}, {300, 10}, {340, 15}, {380, 25}, {420, 35}}, - {{460, math.Float64frombits(value.StaleNaN)}, {500, 10}, {540, 3}}, + {{t: 20, v: 1}, {t: 40, v: 2}, {t: 60, v: 3}, {t: 80, v: 1}, {t: 100, v: 2}, {t: 101, v: math.Float64frombits(value.StaleNaN)}, {t: 120, v: 5}, {t: 180, v: 10}, {t: 250, v: 2}}, + {{t: 260, v: 1}, {t: 300, v: 10}, {t: 340, v: 15}, {t: 380, v: 25}, {t: 420, v: 35}}, + {{t: 460, v: math.Float64frombits(value.StaleNaN)}, {t: 500, v: 10}, {t: 540, v: 3}}, }, resolution: 100, @@ -264,10 +268,10 @@ func TestDownsample(t *testing.T) { { name: "four chunks, two of them overlapping", inRaw: [][]sample{ - {{20, 1}, {40, 2}, {60, 3}, {80, 1}, {100, 2}, {101, math.Float64frombits(value.StaleNaN)}, {120, 5}, {180, 10}, {250, 2}}, - {{20, 1}, {40, 2}, {60, 3}, {80, 1}, {100, 2}, {101, math.Float64frombits(value.StaleNaN)}, {120, 5}, {180, 10}, {250, 2}}, - {{260, 1}, {300, 10}, {340, 15}, {380, 25}, {420, 35}}, - {{460, math.Float64frombits(value.StaleNaN)}, {500, 10}, {540, 3}}, + {{t: 20, v: 1}, {t: 40, v: 2}, {t: 60, v: 3}, {t: 80, v: 1}, {t: 100, v: 2}, {t: 101, v: math.Float64frombits(value.StaleNaN)}, {t: 120, v: 5}, {t: 180, v: 10}, {t: 250, v: 2}}, + {{t: 20, v: 1}, {t: 40, v: 2}, {t: 60, v: 3}, {t: 80, v: 1}, {t: 100, v: 2}, {t: 101, v: math.Float64frombits(value.StaleNaN)}, {t: 120, v: 5}, {t: 180, v: 10}, {t: 250, v: 2}}, + {{t: 260, v: 1}, {t: 300, v: 10}, {t: 340, v: 15}, {t: 380, v: 25}, {t: 420, v: 35}}, + {{t: 460, v: math.Float64frombits(value.StaleNaN)}, {t: 500, v: 10}, {t: 540, v: 3}}, }, resolution: 100, @@ -287,14 +291,14 @@ func TestDownsample(t *testing.T) { name: "single aggregated chunks", inAggr: []map[AggrType][]sample{ { - AggrCount: {{199, 5}, {299, 1}, {399, 10}, {400, 3}, {499, 10}, {699, 0}, {999, 100}}, - AggrSum: {{199, 5}, {299, 1}, {399, 10}, {400, 3}, {499, 10}, {699, 0}, {999, 100}}, - AggrMin: {{199, 5}, {299, 1}, {399, 10}, {400, -3}, {499, 10}, {699, 0}, {999, 100}}, - AggrMax: {{199, 5}, {299, 1}, {399, 10}, {400, -3}, {499, 10}, {699, 0}, {999, 100}}, + AggrCount: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: 3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, + AggrSum: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: 3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, + AggrMin: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: -3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, + AggrMax: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: -3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, AggrCounter: { - {99, 100}, {299, 150}, {499, 210}, {499, 10}, // Chunk 1. - {599, 20}, {799, 50}, {999, 120}, {999, 50}, // Chunk 2, no reset. - {1099, 40}, {1199, 80}, {1299, 110}, // Chunk 3, reset. + {t: 99, v: 100}, {t: 299, v: 150}, {t: 499, v: 210}, {t: 499, v: 10}, // Chunk 1. + {t: 599, v: 20}, {t: 799, v: 50}, {t: 999, v: 120}, {t: 999, v: 50}, // Chunk 2, no reset. + {t: 1099, v: 40}, {t: 1199, v: 80}, {t: 1299, v: 110}, // Chunk 3, reset. }, }, }, @@ -302,22 +306,22 @@ func TestDownsample(t *testing.T) { expected: []map[AggrType][]sample{ { - AggrCount: {{499, 29}, {999, 100}}, - AggrSum: {{499, 29}, {999, 100}}, - AggrMin: {{499, -3}, {999, 0}}, - AggrMax: {{499, 10}, {999, 100}}, - AggrCounter: {{99, 100}, {499, 210}, {999, 320}, {1299, 430}, {1299, 110}}, + AggrCount: {{t: 499, v: 29}, {t: 999, v: 100}}, + AggrSum: {{t: 499, v: 29}, {t: 999, v: 100}}, + AggrMin: {{t: 499, v: -3}, {t: 999}}, + AggrMax: {{t: 499, v: 10}, {t: 999, v: 100}}, + AggrCounter: {{t: 99, v: 100}, {t: 499, v: 210}, {t: 999, v: 320}, {t: 1299, v: 430}, {t: 1299, v: 110}}, }, }, }, func() *downsampleTestCase { downsample500resolutionChunk := []map[AggrType][]sample{ { - AggrCount: {{499, 29}, {999, 100}}, - AggrSum: {{499, 29}, {999, 100}}, - AggrMin: {{499, -3}, {999, 0}}, - AggrMax: {{499, 10}, {999, 100}}, - AggrCounter: {{99, 100}, {499, 210}, {999, 320}, {1299, 430}, {1299, 110}}, + AggrCount: {{t: 499, v: 29}, {t: 999, v: 100}}, + AggrSum: {{t: 499, v: 29}, {t: 999, v: 100}}, + AggrMin: {{t: 499, v: -3}, {t: 999}}, + AggrMax: {{t: 499, v: 10}, {t: 999, v: 100}}, + AggrCounter: {{t: 99, v: 100}, {t: 499, v: 210}, {t: 999, v: 320}, {t: 1299, v: 430}, {t: 1299, v: 110}}, }, } return &downsampleTestCase{ @@ -333,25 +337,25 @@ func TestDownsample(t *testing.T) { name: "two aggregated chunks", inAggr: []map[AggrType][]sample{ { - AggrCount: {{199, 5}, {299, 1}, {399, 10}, {400, 3}, {499, 10}, {699, 0}, {999, 100}}, - AggrSum: {{199, 5}, {299, 1}, {399, 10}, {400, 3}, {499, 10}, {699, 0}, {999, 100}}, - AggrMin: {{199, 5}, {299, 1}, {399, 10}, {400, -3}, {499, 10}, {699, 0}, {999, 100}}, - AggrMax: {{199, 5}, {299, 1}, {399, 10}, {400, -3}, {499, 10}, {699, 0}, {999, 100}}, + AggrCount: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: 3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, + AggrSum: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: 3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, + AggrMin: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: -3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, + AggrMax: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: -3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, AggrCounter: { - {99, 100}, {299, 150}, {499, 210}, {499, 10}, // Chunk 1. - {599, 20}, {799, 50}, {999, 120}, {999, 50}, // Chunk 2, no reset. - {1099, 40}, {1199, 80}, {1299, 110}, // Chunk 3, reset. + {t: 99, v: 100}, {t: 299, v: 150}, {t: 499, v: 210}, {t: 499, v: 10}, // Chunk 1. + {t: 599, v: 20}, {t: 799, v: 50}, {t: 999, v: 120}, {t: 999, v: 50}, // Chunk 2, no reset. + {t: 1099, v: 40}, {t: 1199, v: 80}, {t: 1299, v: 110}, // Chunk 3, reset. }, }, { - AggrCount: {{1399, 10}, {1400, 3}, {1499, 10}, {1699, 0}, {1999, 100}}, - AggrSum: {{1399, 10}, {1400, 3}, {1499, 10}, {1699, 0}, {1999, 100}}, - AggrMin: {{1399, 10}, {1400, -3}, {1499, 10}, {1699, 0}, {1999, 100}}, - AggrMax: {{1399, 10}, {1400, -3}, {1499, 10}, {1699, 0}, {1999, 100}}, + AggrCount: {{t: 1399, v: 10}, {t: 1400, v: 3}, {t: 1499, v: 10}, {t: 1699}, {t: 1999, v: 100}}, + AggrSum: {{t: 1399, v: 10}, {t: 1400, v: 3}, {t: 1499, v: 10}, {t: 1699}, {t: 1999, v: 100}}, + AggrMin: {{t: 1399, v: 10}, {t: 1400, v: -3}, {t: 1499, v: 10}, {t: 1699}, {t: 1999, v: 100}}, + AggrMax: {{t: 1399, v: 10}, {t: 1400, v: -3}, {t: 1499, v: 10}, {t: 1699}, {t: 1999, v: 100}}, AggrCounter: { - {1499, 210}, {1499, 10}, // Chunk 1. - {1599, 20}, {1799, 50}, {1999, 120}, {1999, 50}, // Chunk 2, no reset. - {2099, 40}, {2199, 80}, {2299, 110}, // Chunk 3, reset. + {t: 1499, v: 210}, {t: 1499, v: 10}, // Chunk 1. + {t: 1599, v: 20}, {t: 1799, v: 50}, {t: 1999, v: 120}, {t: 1999, v: 50}, // Chunk 2, no reset. + {t: 2099, v: 40}, {t: 2199, v: 80}, {t: 2299, v: 110}, // Chunk 3, reset. }, }, }, @@ -371,10 +375,10 @@ func TestDownsample(t *testing.T) { name: "two aggregated, overlapping chunks", inAggr: []map[AggrType][]sample{ { - AggrCount: {{199, 5}, {299, 1}, {399, 10}, {400, 3}, {499, 10}, {699, 0}, {999, 100}}, + AggrCount: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: 3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, }, { - AggrCount: {{199, 5}, {299, 1}, {399, 10}, {400, 3}, {499, 10}, {699, 0}, {999, 100}}, + AggrCount: {{t: 199, v: 5}, {t: 299, v: 1}, {t: 399, v: 10}, {t: 400, v: 3}, {t: 499, v: 10}, {t: 699}, {t: 999, v: 100}}, }, }, resolution: 500, @@ -406,6 +410,28 @@ func TestDownsample(t *testing.T) { }, }, }, + { + name: "reproduce invalid chunk time", + inAggr: []map[AggrType][]sample{ + { + AggrCount: {{t: 1679688941708, v: 1}, {t: 1679689199999, v: 1}, {t: 1679689499999, v: 1}, {t: 1679689799999, v: 1}, {t: 1679689811708, v: 1}}, + AggrSum: {{t: 1679688941708, v: 1}, {t: 1679689199999, v: 1}, {t: 1679689499999, v: 1}, {t: 1679689799999, v: 1}, {t: 1679689811708, v: 1}}, + AggrMin: {{t: 1679688941708, v: 1}, {t: 1679689199999, v: 1}, {t: 1679689499999, v: 1}, {t: 1679689799999, v: 1}, {t: 1679689811708, v: 1}}, + AggrMax: {{t: 1679688941708, v: 1}, {t: 1679689199999, v: 1}, {t: 1679689499999, v: 1}, {t: 1679689799999, v: 1}, {t: 1679689811708, v: 1}}, + AggrCounter: {{t: 1679688941708, v: 1}, {t: 1679689199999, v: 1}, {t: 1679689499999, v: 1}, {t: 1679689799999, v: 1}, {t: 1679689811708, v: 1}}, + }, + }, + resolution: ResLevel2, + expected: []map[AggrType][]sample{ + { + AggrCount: {{t: 1679689811708, v: 5}}, + AggrSum: {{t: 1679689811708, v: 5}}, + AggrMin: {{t: 1679689811708, v: 1}}, + AggrMax: {{t: 1679689811708, v: 1}}, + AggrCounter: {{t: 1679688941708, v: 1}, {t: 1679689811708, v: 1}, {t: 1679689811708, v: 1}}, + }, + }, + }, // TODO(bwplotka): This is not very efficient for further query time, we should produce 2 chunks. Fix it https://github.com/thanos-io/thanos/issues/2542. func() *downsampleTestCase { d := &downsampleTestCase{ @@ -484,6 +510,8 @@ func TestDownsample(t *testing.T) { lset = builder.Labels() testutil.Equals(t, labels.FromStrings("__name__", "a"), lset) + assertValidChunkTime(t, chks) + var got []map[AggrType][]sample for _, c := range chks { chk, err := chunkr.Chunk(c) @@ -498,7 +526,7 @@ func TestDownsample(t *testing.T) { testutil.Ok(t, err) buf := m[at] - testutil.Ok(t, expandChunkIterator(c.Iterator(nil), &buf)) + testutil.Ok(t, expandXorChunkIterator(c.Iterator(nil), &buf)) m[at] = buf } got = append(got, m) @@ -572,11 +600,11 @@ func TestDownsampleAggrAndNonEmptyXORChunks(t *testing.T) { expected := []map[AggrType][]sample{ { - AggrCount: {{1587690005794, 20}, {1587690005794, 20}, {1587690005794, 21}}, - AggrSum: {{1587690005794, 9.276972e+06}, {1587690005794, 9.359861e+06}, {1587690005794, 255788.5}}, - AggrMin: {{1587690005794, 461968}, {1587690005794, 466070}, {1587690005794, 470131}, {1587690005794, 42.5}}, - AggrMax: {{1587690005794, 465870}, {1587690005794, 469951}, {1587690005794, 474726}}, - AggrCounter: {{1587690005791, 461968}, {1587690599999, 469951}, {1587690599999, 469951}}, + AggrCount: {{t: 1587690005794, v: 20}, {t: 1587690005794, v: 20}, {t: 1587690005794, v: 21}}, + AggrSum: {{t: 1587690005794, v: 9.276972e+06}, {t: 1587690005794, v: 9.359861e+06}, {t: 1587690005794, v: 255788.5}}, + AggrMin: {{t: 1587690005794, v: 461968}, {t: 1587690005794, v: 466070}, {t: 1587690005794, v: 470131}, {t: 1587690005794, v: 42.5}}, + AggrMax: {{t: 1587690005794, v: 465870}, {t: 1587690005794, v: 469951}, {t: 1587690005794, v: 474726}}, + AggrCounter: {{t: 1587690005791, v: 461968}, {t: 1587690599999, v: 469951}, {t: 1587690599999, v: 469951}}, }, } @@ -619,7 +647,7 @@ func TestDownsampleAggrAndNonEmptyXORChunks(t *testing.T) { testutil.Ok(t, err) buf := m[at] - testutil.Ok(t, expandChunkIterator(c.Iterator(nil), &buf)) + testutil.Ok(t, expandXorChunkIterator(c.Iterator(nil), &buf)) m[at] = buf } got = append(got, m) @@ -628,6 +656,329 @@ func TestDownsampleAggrAndNonEmptyXORChunks(t *testing.T) { } +type expectedHistogramAggregates struct { + count, sum, counter []sample +} + +func TestDownSampleNativeHistogram(t *testing.T) { + tests := []struct { + name string + samples []sample // Samples per chunk. + expectedReseLevel1 []expectedHistogramAggregates + expectedReseLevel2 []expectedHistogramAggregates + }{ + { + "float histogram with 30 samples with counter resets and 30s scrape interval", + generateFloatHistogramSamples(30_000, 0, 15, 5, 10), + []expectedHistogramAggregates{ + { + count: []sample{ + {t: 299_999, v: 10}, {t: 599_999, v: 10}, {t: 870_000, v: 10}, + }, + counter: []sample{ + { + t: 299_999, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(10), histogram.UnknownCounterReset), + }, + { + t: 599_999, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(15, 5), histogram.NotCounterReset), + }, + { + t: 870_000, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(15, 5, 10), histogram.NotCounterReset), + }, + }, + sum: []sample{ + { + t: 299_999, + fh: expectedFloatHistogramsSum(generateFloatHistograms(10), 0), + }, + { + t: 599_999, + fh: expectedFloatHistogramsSum(generateFloatHistograms(15, 5), 10), + }, + { + t: 870_000, + fh: expectedFloatHistogramsSum(generateFloatHistograms(15, 5, 10), 20), + }, + }, + }, + }, + []expectedHistogramAggregates{ + { + count: []sample{ + {t: 870_000, v: 30}, + }, + counter: []sample{ + { + t: 870_000, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(15, 5, 10), histogram.UnknownCounterReset), + }, + }, + sum: []sample{ + { + t: 870_000, + fh: expectedFloatHistogramsSum(generateFloatHistograms(15, 5, 10), 0), + }, + }, + }, + }, + }, + { + "float histogram with 360 samples and 30s scrape interval", + generateFloatHistogramSamples(30_000, 0, 360), + []expectedHistogramAggregates{}, + []expectedHistogramAggregates{ + { + count: []sample{ + {t: 3_599_999, v: 120}, {t: 7_199_999, v: 120}, {t: 10_770_000, v: 120}, + }, + counter: []sample{ + { + t: 3_599_999, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(120), histogram.UnknownCounterReset), + }, + { + t: 7_199_999, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(240), histogram.NotCounterReset), + }, + { + t: 10_770_000, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(360), histogram.NotCounterReset), + }, + }, + sum: []sample{ + { + t: 3_599_999, + fh: expectedFloatHistogramsSum(generateFloatHistograms(120), 0), + }, + { + t: 7_199_999, + fh: expectedFloatHistogramsSum(generateFloatHistograms(240), 120), + }, + { + t: 10_770_000, + fh: expectedFloatHistogramsSum(generateFloatHistograms(360), 240), + }, + }, + }, + }, + }, + { + "gauge float histogram with 30 samples and 30s scrape interval", + generateGaugeFlotHistogramSamples(30_000, 0, 30), + []expectedHistogramAggregates{}, + []expectedHistogramAggregates{}, + }, + { + "stale float histogram sample", + append(generateFloatHistogramSamples(30_000, 0, 10), sample{ + t: 300_000, + fh: &histogram.FloatHistogram{ + Sum: math.Float64frombits(value.StaleNaN), + }, + }), + []expectedHistogramAggregates{ + { + count: []sample{ + {t: 270_000, v: 10}, + }, + counter: []sample{ + { + t: 270_000, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(10), histogram.UnknownCounterReset), + }, + }, + sum: []sample{ + { + t: 270_000, + fh: expectedFloatHistogramsSum(generateFloatHistograms(10), 0), + }, + }, + }, + }, + []expectedHistogramAggregates{}, + }, + } + + logger := log.NewLogfmtLogger(os.Stderr) + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + dir := t.TempDir() + ser := &series{lset: labels.FromStrings("__name__", "a")} + + mint := tt.samples[0].t + maxt := mint + + chunk, app := newChunk(t, tt.samples[0].fh.CounterResetHint) + + var previous *histogram.FloatHistogram + for i, s := range tt.samples { + if i > 0 && s.fh.DetectReset(previous) { + ser.chunks = append(ser.chunks, chunks.Meta{ + MinTime: mint, + MaxTime: maxt, + Chunk: chunk, + }) + chunk, app = newChunk(t, s.fh.CounterResetHint) + mint = s.t + } + if s.t > maxt { + maxt = s.t + } + app.AppendFloatHistogram(s.t, s.fh) + previous = s.fh + } + + ser.chunks = append(ser.chunks, chunks.Meta{ + MinTime: math.MaxInt64, + MaxTime: math.MinInt64, + Chunk: chunk, + }) + + mb := newMemBlock() + mb.addSeries(ser) + + fakeMeta := &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + MinTime: tt.samples[0].t, + MaxTime: maxt, + }, + } + idResLevel1, err := Downsample(logger, fakeMeta, mb, dir, ResLevel1) + testutil.Ok(t, err) + + meta, chks := GetMetaAndChunks(t, dir, idResLevel1) + + assertValidChunkTime(t, chks) + + if len(tt.expectedReseLevel1) > 0 { + compareAggreggates(t, dir, ResLevel1, idResLevel1.String(), tt.expectedReseLevel1, chks) + } + + blk, err := tsdb.OpenBlock(logger, filepath.Join(dir, idResLevel1.String()), NewPool()) + testutil.Ok(t, err) + idResLevel2, err := Downsample(logger, meta, blk, dir, ResLevel2) + testutil.Ok(t, err) + + _, chks = GetMetaAndChunks(t, dir, idResLevel2) + + assertValidChunkTime(t, chks) + + if len(tt.expectedReseLevel2) > 0 { + compareAggreggates(t, dir, ResLevel2, idResLevel2.String(), tt.expectedReseLevel2, chks) + } + }) + } +} + +func newChunk(t *testing.T, counterResetHint histogram.CounterResetHint) (*chunkenc.FloatHistogramChunk, chunkenc.Appender) { + raw := chunkenc.NewFloatHistogramChunk() + app, err := raw.Appender() + if counterResetHint == histogram.GaugeType { + raw.SetCounterResetHeader(chunkenc.GaugeType) + } + testutil.Ok(t, err) + return raw, app +} + +func generateFloatHistogramSamples(scrapeInterval, startTs int64, n ...int) []sample { + // Generate samples for the float histogram. + ts := startTs + floatHistograms := generateFloatHistograms(n...) + samples := make([]sample, 0, len(floatHistograms)) + for _, fh := range floatHistograms { + samples = append(samples, sample{t: ts, fh: fh}) + ts += scrapeInterval + } + return samples +} + +// generateFloatHistograms generates multiple float histograms slices that can be used +// to simulate counter resets. +func generateFloatHistograms(n ...int) []*histogram.FloatHistogram { + var floatHistograms []*histogram.FloatHistogram + for _, v := range n { + floatHistograms = append(floatHistograms, tsdbutil.GenerateTestFloatHistograms(v)...) + } + return floatHistograms +} + +func generateGaugeFlotHistogramSamples(scrapeInterval, startTs int64, n int) (floatHistogramSamples []sample) { + samples := make([]sample, 0, n) + for i, fh := range tsdbutil.GenerateTestGaugeFloatHistograms(n) { + samples = append(samples, sample{t: startTs + int64(i)*scrapeInterval, fh: fh}) + } + return samples +} + +func expectedFloatHistogramsSum(floatHistograms []*histogram.FloatHistogram, fromIndex int) *histogram.FloatHistogram { + adjustedFloatHistograms := counterResetAdjustFloatHistograms(floatHistograms) + sum := adjustedFloatHistograms[fromIndex] + for _, s := range adjustedFloatHistograms[fromIndex+1:] { + sum.Add(s) + } + sum.CounterResetHint = histogram.GaugeType + return sum +} + +func expectedFloatHistogramsCounter(floatHistograms []*histogram.FloatHistogram, hint histogram.CounterResetHint) *histogram.FloatHistogram { + counter := counterResetAdjustFloatHistograms(floatHistograms)[len(floatHistograms)-1] + counter.CounterResetHint = hint + return counter +} + +func counterResetAdjustFloatHistograms(floatHistograms []*histogram.FloatHistogram) []*histogram.FloatHistogram { + var previous, counter *histogram.FloatHistogram + res := make([]*histogram.FloatHistogram, 0, len(floatHistograms)) + for i, fh := range floatHistograms { + if i == 0 { + counter = fh.Copy() + } else { + if fh.DetectReset(previous) { + counter.Add(fh) + } else { + counter.Add(fh.Copy().Sub(previous)) + } + } + previous = fh + res = append(res, counter.Copy()) + } + return res +} + +func compareAggreggates(t *testing.T, dir string, resLevel int64, blockID string, expected []expectedHistogramAggregates, chks []chunks.Meta) { + chunkr, err := chunks.NewDirReader(filepath.Join(dir, blockID, block.ChunksDirname), NewPool()) + testutil.Ok(t, err) + defer func() { testutil.Ok(t, chunkr.Close()) }() + + testUtilWithOps := testutil.WithGoCmp( + cmpopts.EquateApprox(0, 1e-9), + cmp.AllowUnexported(sample{}), + ) + + for i, c := range chks { + count := GetAggregateFromChunk(t, chunkr, c, AggrCount) + testUtilWithOps.Equals(t, expected[i].count, count, "count mismatch for chunk %d with resolution %d", i, resLevel) + + counter := GetAggregateFromChunk(t, chunkr, c, AggrCounter) + testUtilWithOps.Equals(t, expected[i].counter, counter, "counter mismatch for chunk %d with resolution %d", i, resLevel) + + sum := GetAggregateFromChunk(t, chunkr, c, AggrSum) + testUtilWithOps.Equals(t, expected[i].sum, sum, "sum mismatch for chunk %d with resolution %d", i, resLevel) + } +} + +func assertValidChunkTime(t *testing.T, chks []chunks.Meta) { + t.Helper() + for _, chk := range chks { + testutil.Assert(t, chk.MinTime != math.MaxInt64, "chunk MinTime is not set") + testutil.Assert(t, chk.MaxTime >= chk.MinTime, "chunk MaxTime is not greater equal to MinTime") + } +} + func chunksToSeriesIteratable(t *testing.T, inRaw [][]sample, inAggr []map[AggrType][]sample) *series { if len(inRaw) > 0 && len(inAggr) > 0 { t.Fatalf("test must not have raw and aggregate input data at once") @@ -674,17 +1025,18 @@ func encodeTestAggrSeries(v map[AggrType][]sample) chunks.Meta { return b.encode() } +// TODO(srabenhorst): Add tests for histograms. func TestAverageChunkIterator(t *testing.T) { - sum := []sample{{100, 30}, {200, 40}, {300, 5}, {400, -10}} - cnt := []sample{{100, 1}, {200, 5}, {300, 2}, {400, 10}} - exp := []sample{{100, 30}, {200, 8}, {300, 2.5}, {400, -1}} + sum := []sample{{t: 100, v: 30}, {t: 200, v: 40}, {t: 300, v: 5}, {t: 400, v: -10}} + cnt := []sample{{t: 100, v: 1}, {t: 200, v: 5}, {t: 300, v: 2}, {t: 400, v: 10}} + exp := []sample{{t: 100, v: 30}, {t: 200, v: 8}, {t: 300, v: 2.5}, {t: 400, v: -1}} x := NewAverageChunkIterator(newSampleIterator(cnt), newSampleIterator(sum)) var res []sample for x.Next() != chunkenc.ValNone { t, v := x.At() - res = append(res, sample{t, v}) + res = append(res, sample{t: t, v: v}) } testutil.Ok(t, x.Err()) testutil.Equals(t, exp, res) @@ -728,16 +1080,16 @@ func TestApplyCounterResetsIterator(t *testing.T) { { name: "series with stale marker", chunks: [][]sample{ - {{100, 10}, {200, 20}, {300, 10}, {400, 20}, {400, 5}}, - {{500, 10}, {600, 20}, {700, 30}, {800, 40}, {800, 10}}, // No reset, just downsampling addded sample at the end. - {{900, 5}, {1000, 10}, {1100, 15}}, // Actual reset. - {{1200, 20}, {1250, math.Float64frombits(value.StaleNaN)}, {1300, 40}}, // No special last sample, no reset. - {{1400, 30}, {1500, 30}, {1600, 50}}, // No special last sample, reset. + {{t: 100, v: 10}, {t: 200, v: 20}, {t: 300, v: 10}, {t: 400, v: 20}, {t: 400, v: 5}}, + {{t: 500, v: 10}, {t: 600, v: 20}, {t: 700, v: 30}, {t: 800, v: 40}, {t: 800, v: 10}}, // No reset, just downsampling addded sample at the end. + {{t: 900, v: 5}, {t: 1000, v: 10}, {t: 1100, v: 15}}, // Actual reset. + {{t: 1200, v: 20}, {t: 1250, v: math.Float64frombits(value.StaleNaN)}, {t: 1300, v: 40}}, // No special last sample, no reset. + {{t: 1400, v: 30}, {t: 1500, v: 30}, {t: 1600, v: 50}}, // No special last sample, reset. }, expected: []sample{ - {100, 10}, {200, 20}, {300, 30}, {400, 40}, {500, 45}, - {600, 55}, {700, 65}, {800, 75}, {900, 80}, {1000, 85}, - {1100, 90}, {1200, 95}, {1300, 115}, {1400, 145}, {1500, 145}, {1600, 165}, + {t: 100, v: 10}, {t: 200, v: 20}, {t: 300, v: 30}, {t: 400, v: 40}, {t: 500, v: 45}, + {t: 600, v: 55}, {t: 700, v: 65}, {t: 800, v: 75}, {t: 900, v: 80}, {t: 1000, v: 85}, + {t: 1100, v: 90}, {t: 1200, v: 95}, {t: 1300, v: 115}, {t: 1400, v: 145}, {t: 1500, v: 145}, {t: 1600, v: 165}, }, }, { @@ -785,7 +1137,7 @@ func TestApplyCounterResetsIterator(t *testing.T) { var res []sample for x.Next() != chunkenc.ValNone { t, v := x.At() - res = append(res, sample{t, v}) + res = append(res, sample{t: t, v: v}) } testutil.Ok(t, x.Err()) testutil.Equals(t, tcase.expected, res) @@ -857,11 +1209,11 @@ func TestApplyCounterResetsIteratorHistograms(t *testing.T) { func TestCounterSeriesIteratorSeek(t *testing.T) { chunks := [][]sample{ - {{100, 10}, {200, 20}, {300, 10}, {400, 20}, {400, 5}}, + {{t: 100, v: 10}, {t: 200, v: 20}, {t: 300, v: 10}, {t: 400, v: 20}, {t: 400, v: 5}}, } exp := []sample{ - {200, 20}, {300, 30}, {400, 40}, + {t: 200, v: 20}, {t: 300, v: 30}, {t: 400, v: 40}, } var its []chunkenc.Iterator @@ -877,7 +1229,7 @@ func TestCounterSeriesIteratorSeek(t *testing.T) { testutil.Ok(t, x.Err()) for { ts, v := x.At() - res = append(res, sample{ts, v}) + res = append(res, sample{t: ts, v: v}) if x.Next() == chunkenc.ValNone { break @@ -888,7 +1240,7 @@ func TestCounterSeriesIteratorSeek(t *testing.T) { func TestCounterSeriesIteratorSeekExtendTs(t *testing.T) { chunks := [][]sample{ - {{100, 10}, {200, 20}, {300, 10}, {400, 20}, {400, 5}}, + {{t: 100, v: 10}, {t: 200, v: 20}, {t: 300, v: 10}, {t: 400, v: 20}, {t: 400, v: 5}}, } var its []chunkenc.Iterator @@ -904,10 +1256,10 @@ func TestCounterSeriesIteratorSeekExtendTs(t *testing.T) { func TestCounterSeriesIteratorSeekAfterNext(t *testing.T) { chunks := [][]sample{ - {{100, 10}}, + {{t: 100, v: 10}}, } exp := []sample{ - {100, 10}, + {t: 100, v: 10}, } var its []chunkenc.Iterator @@ -925,7 +1277,7 @@ func TestCounterSeriesIteratorSeekAfterNext(t *testing.T) { testutil.Ok(t, x.Err()) for { ts, v := x.At() - res = append(res, sample{ts, v}) + res = append(res, sample{t: ts, v: v}) if x.Next() == chunkenc.ValNone { break @@ -950,12 +1302,12 @@ func TestSamplesFromTSDBSamples(t *testing.T) { { name: "one sample", input: []tsdbutil.Sample{testSample{1, 1}}, - expected: []sample{{1, 1}}, + expected: []sample{{t: 1, v: 1}}, }, { name: "multiple samples", input: []tsdbutil.Sample{testSample{1, 1}, testSample{2, 2}, testSample{3, 3}, testSample{4, 4}, testSample{5, 5}}, - expected: []sample{{1, 1}, {2, 2}, {3, 3}, {4, 4}, {5, 5}}, + expected: []sample{{t: 1, v: 1}, {t: 2, v: 2}, {t: 3, v: 3}, {t: 4, v: 4}, {t: 5, v: 5}}, }, } { t.Run(tcase.name, func(t *testing.T) { diff --git a/pkg/compact/downsample/testutils.go b/pkg/compact/downsample/testutils.go new file mode 100644 index 0000000000..c72fa1a9a1 --- /dev/null +++ b/pkg/compact/downsample/testutils.go @@ -0,0 +1,71 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package downsample + +import ( + "path/filepath" + "testing" + + "github.com/efficientgo/core/testutil" + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/block/metadata" +) + +func GetMetaAndChunks(t *testing.T, dir string, id ulid.ULID) (*metadata.Meta, []chunks.Meta) { + newMeta, err := metadata.ReadFromDir(filepath.Join(dir, id.String())) + testutil.Ok(t, err) + + indexr, err := index.NewFileReader(filepath.Join(dir, id.String(), block.IndexFilename)) + testutil.Ok(t, err) + defer func() { testutil.Ok(t, indexr.Close()) }() + + pall, err := indexr.Postings(index.AllPostingsKey()) + testutil.Ok(t, err) + + var series []storage.SeriesRef + for pall.Next() { + series = append(series, pall.At()) + } + testutil.Ok(t, pall.Err()) + + var chks []chunks.Meta + var builder labels.ScratchBuilder + testutil.Ok(t, indexr.Series(series[0], &builder, &chks)) + + return newMeta, chks +} + +func GetAggregateFromChunk(t *testing.T, chunkr *chunks.Reader, c chunks.Meta, aggrType AggrType) []sample { + chk, err := chunkr.Chunk(c) + testutil.Ok(t, err) + + ac, ok := chk.(*AggrChunk) + testutil.Assert(t, ok) + + var samples []sample + + subChunk, err := ac.Get(aggrType) + testutil.Ok(t, err) + it := subChunk.Iterator(nil) + for valueType := it.Next(); valueType != chunkenc.ValNone; valueType = it.Next() { + switch valueType { + case chunkenc.ValFloat: + t, v := it.At() + samples = append(samples, sample{t: t, v: v}) + case chunkenc.ValFloatHistogram: + t, fh := it.AtFloatHistogram() + samples = append(samples, sample{t: t, fh: fh}) + default: + t.Fatalf("unexpected value type %v", valueType) + } + } + + return samples +} diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index d2bac75f3e..c533e79a8f 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -1095,7 +1095,7 @@ func populateChunk(out *storepb.AggrChunk, in chunkenc.Chunk, aggrs []storepb.Ag hasher := hashPool.Get().(hash.Hash64) defer hashPool.Put(hasher) - if in.Encoding() == chunkenc.EncXOR || in.Encoding() == chunkenc.EncHistogram { + if in.Encoding() == chunkenc.EncXOR || in.Encoding() == chunkenc.EncHistogram || in.Encoding() == chunkenc.EncFloatHistogram { b, err := save(in.Bytes()) if err != nil { return err @@ -1135,7 +1135,7 @@ func populateChunk(out *storepb.AggrChunk, in chunkenc.Chunk, aggrs []storepb.Ag if err != nil { return err } - out.Sum = &storepb.Chunk{Type: storepb.Chunk_XOR, Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} + out.Sum = &storepb.Chunk{Type: chunkToStoreEncoding(x.Encoding()), Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} case storepb.Aggr_MIN: x, err := ac.Get(downsample.AggrMin) if err != nil { @@ -1165,12 +1165,25 @@ func populateChunk(out *storepb.AggrChunk, in chunkenc.Chunk, aggrs []storepb.Ag if err != nil { return err } - out.Counter = &storepb.Chunk{Type: storepb.Chunk_XOR, Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} + out.Counter = &storepb.Chunk{Type: chunkToStoreEncoding(x.Encoding()), Data: b, Hash: hashChunk(hasher, b, calculateChecksum)} } } return nil } +func chunkToStoreEncoding(in chunkenc.Encoding) storepb.Chunk_Encoding { + switch in { + case chunkenc.EncXOR: + return storepb.Chunk_XOR + case chunkenc.EncHistogram: + return storepb.Chunk_HISTOGRAM + case chunkenc.EncFloatHistogram: + return storepb.Chunk_FLOAT_HISTOGRAM + default: + panic("unknown chunk encoding") + } +} + func hashChunk(hasher hash.Hash64, b []byte, doHash bool) uint64 { if !doHash { return 0 diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index e25a121f56..47f8e8e695 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -1100,7 +1100,7 @@ func uploadTestBlock(t testing.TB, tmpDir string, bkt objstore.Bucket, series in appendTestData(t, h.Appender(context.Background()), series) testutil.Ok(t, os.MkdirAll(filepath.Join(tmpDir, "tmp"), os.ModePerm)) - id := createBlockFromHead(t, filepath.Join(tmpDir, "tmp"), h) + id := storetestutil.CreateBlockFromHead(t, filepath.Join(tmpDir, "tmp"), h) _, err = metadata.InjectThanos(log.NewNopLogger(), filepath.Join(tmpDir, "tmp", id.String()), metadata.Thanos{ Labels: labels.Labels{{Name: "ext1", Value: "1"}}.Map(), @@ -1136,19 +1136,6 @@ func appendTestData(t testing.TB, app storage.Appender, series int) { testutil.Ok(t, app.Commit()) } -func createBlockFromHead(t testing.TB, dir string, head *tsdb.Head) ulid.ULID { - compactor, err := tsdb.NewLeveledCompactor(context.Background(), nil, log.NewNopLogger(), []int64{1000000}, nil, nil) - testutil.Ok(t, err) - - testutil.Ok(t, os.MkdirAll(dir, 0777)) - - // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). - // Because of this block intervals are always +1 than the total samples it includes. - ulid, err := compactor.Write(dir, head, head.MinTime(), head.MaxTime()+1, nil) - testutil.Ok(t, err) - return ulid -} - // Very similar benchmark to ths: https://github.com/prometheus/prometheus/blob/1d1732bc25cc4b47f513cb98009a4eb91879f175/tsdb/querier_bench_test.go#L82, // but with postings results check when run as test. func benchmarkExpandedPostings( @@ -1352,7 +1339,7 @@ func benchBucketSeries(t testutil.TB, sampleType chunkenc.ValueType, skipChunk b SkipChunks: t.IsBenchmark() || skipChunk, SampleType: sampleType, }) - id := createBlockFromHead(t, blockDir, head) + id := storetestutil.CreateBlockFromHead(t, blockDir, head) testutil.Ok(t, head.Close()) // Histogram chunks are represented differently in memory and on disk. In order to @@ -1553,7 +1540,7 @@ func TestBucketSeries_OneBlock_InMemIndexCacheSegfault(t *testing.T) { testutil.Ok(t, app.Commit()) blockDir := filepath.Join(tmpDir, "tmp") - id := createBlockFromHead(t, blockDir, h) + id := storetestutil.CreateBlockFromHead(t, blockDir, h) meta, err := metadata.InjectThanos(log.NewNopLogger(), filepath.Join(blockDir, id.String()), thanosMeta, nil) testutil.Ok(t, err) @@ -1594,7 +1581,7 @@ func TestBucketSeries_OneBlock_InMemIndexCacheSegfault(t *testing.T) { testutil.Ok(t, app.Commit()) blockDir := filepath.Join(tmpDir, "tmp2") - id := createBlockFromHead(t, blockDir, h) + id := storetestutil.CreateBlockFromHead(t, blockDir, h) meta, err := metadata.InjectThanos(log.NewNopLogger(), filepath.Join(blockDir, id.String()), thanosMeta, nil) testutil.Ok(t, err) @@ -1886,7 +1873,7 @@ func TestSeries_BlockWithMultipleChunks(t *testing.T) { testutil.Ok(t, app.Commit()) } - blk := createBlockFromHead(t, headOpts.ChunkDirRoot, h) + blk := storetestutil.CreateBlockFromHead(t, headOpts.ChunkDirRoot, h) thanosMeta := metadata.Thanos{ Labels: labels.Labels{{Name: "ext1", Value: "1"}}.Map(), @@ -2035,7 +2022,7 @@ func createBlockWithOneSeriesWithStep(t testutil.TB, dir string, lbls labels.Lab } testutil.Ok(t, app.Commit()) - return createBlockFromHead(t, dir, h) + return storetestutil.CreateBlockFromHead(t, dir, h) } func setupStoreForHintsTest(t *testing.T) (testutil.TB, *BucketStore, []*storepb.Series, []*storepb.Series, ulid.ULID, ulid.ULID, func()) { @@ -2072,7 +2059,7 @@ func setupStoreForHintsTest(t *testing.T) (testutil.TB, *BucketStore, []*storepb PrependLabels: extLset, Random: random, }) - block1 := createBlockFromHead(t, bktDir, head) + block1 := storetestutil.CreateBlockFromHead(t, bktDir, head) testutil.Ok(t, head.Close()) head2, seriesSet2 := storetestutil.CreateHeadWithSeries(t, 1, storetestutil.HeadGenOptions{ TSDBDir: filepath.Join(tmpDir, "1"), @@ -2081,7 +2068,7 @@ func setupStoreForHintsTest(t *testing.T) (testutil.TB, *BucketStore, []*storepb PrependLabels: extLset, Random: random, }) - block2 := createBlockFromHead(t, bktDir, head2) + block2 := storetestutil.CreateBlockFromHead(t, bktDir, head2) testutil.Ok(t, head2.Close()) for _, blockID := range []ulid.ULID{block1, block2} { @@ -2284,7 +2271,7 @@ func TestSeries_ChunksHaveHashRepresentation(t *testing.T) { } testutil.Ok(t, app.Commit()) - blk := createBlockFromHead(t, headOpts.ChunkDirRoot, h) + blk := storetestutil.CreateBlockFromHead(t, headOpts.ChunkDirRoot, h) thanosMeta := metadata.Thanos{ Labels: labels.Labels{{Name: "ext1", Value: "1"}}.Map(), @@ -2448,7 +2435,7 @@ func BenchmarkBucketBlock_readChunkRange(b *testing.B) { } func BenchmarkBlockSeries(b *testing.B) { - blk, blockMeta := prepareBucket(b, compact.ResolutionLevelRaw) + blk, blockMeta := prepareBucket(b, compact.ResolutionLevelRaw, chunkenc.ValFloat) aggrs := []storepb.Aggr{storepb.Aggr_RAW} for _, concurrency := range []int{1, 2, 4, 8, 16, 32} { @@ -2458,7 +2445,7 @@ func BenchmarkBlockSeries(b *testing.B) { } } -func prepareBucket(b *testing.B, resolutionLevel compact.ResolutionLevel) (*bucketBlock, *metadata.Meta) { +func prepareBucket(b testing.TB, resolutionLevel compact.ResolutionLevel, sampleType chunkenc.ValueType) (*bucketBlock, *metadata.Meta) { var ( ctx = context.Background() logger = log.NewNopLogger() @@ -2481,8 +2468,9 @@ func prepareBucket(b *testing.B, resolutionLevel compact.ResolutionLevel) (*buck PrependLabels: nil, Random: rand.New(rand.NewSource(120)), SkipChunks: true, + SampleType: sampleType, }) - blockID := createBlockFromHead(b, tmpDir, head) + blockID := storetestutil.CreateBlockFromHead(b, tmpDir, head) // Upload the block to the bucket. thanosMeta := metadata.Thanos{ @@ -2591,7 +2579,7 @@ func benchmarkBlockSeriesWithConcurrency(b *testing.B, concurrency int, blockMet } func BenchmarkDownsampledBlockSeries(b *testing.B) { - blk, blockMeta := prepareBucket(b, compact.ResolutionLevel5m) + blk, blockMeta := prepareBucket(b, compact.ResolutionLevel5m, chunkenc.ValFloat) aggrs := []storepb.Aggr{} for i := 1; i < int(storepb.Aggr_COUNTER); i++ { aggrs = append(aggrs, storepb.Aggr(i)) @@ -2613,3 +2601,16 @@ func TestExpandPostingsWithContextCancel(t *testing.T) { testutil.Equals(t, context.Canceled, err) testutil.Equals(t, []storage.SeriesRef(nil), res) } + +func BenchmarkDownsampledBlockSeries_NativeHistogram(b *testing.B) { + blk, blockMeta := prepareBucket(b, compact.ResolutionLevel5m, chunkenc.ValHistogram) + aggrs := []storepb.Aggr{} + + // Native histograms only have aggregates for COUNT, COUNTER and SUM. + aggrsTypes := []storepb.Aggr{storepb.Aggr_COUNT, storepb.Aggr_COUNTER, storepb.Aggr_SUM} + for _, concurrency := range []int{1, 2, 4, 8, 16, 32} { + b.Run(fmt.Sprintf("aggregates: %v, concurrency: %d", aggrs, concurrency), func(b *testing.B) { + benchmarkBlockSeriesWithConcurrency(b, concurrency, blockMeta, blk, aggrsTypes) + }) + } +} diff --git a/pkg/store/storepb/testutil/series.go b/pkg/store/storepb/testutil/series.go index cb8abb607c..34448cd9b2 100644 --- a/pkg/store/storepb/testutil/series.go +++ b/pkg/store/storepb/testutil/series.go @@ -17,14 +17,16 @@ import ( "github.com/cespare/xxhash" "github.com/efficientgo/core/testutil" + "github.com/go-kit/log" "github.com/gogo/protobuf/types" - "github.com/prometheus/prometheus/model/histogram" + "github.com/oklog/ulid" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/index" + "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/prometheus/prometheus/tsdb/wlog" "go.uber.org/atomic" @@ -51,9 +53,11 @@ type HeadGenOptions struct { ScrapeInterval time.Duration WithWAL bool + AppendLabels labels.Labels PrependLabels labels.Labels SkipChunks bool // Skips chunks in returned slice (not in generated head!). SampleType chunkenc.ValueType + IncludeName bool Random *rand.Rand } @@ -123,17 +127,15 @@ func ReadSeriesFromBlock(t testing.TB, h tsdb.BlockReader, extLabels labels.Labe defer func() { testutil.Ok(t, ir.Close()) }() var ( - lset labels.Labels + builder labels.ScratchBuilder chunkMetas []chunks.Meta expected = make([]*storepb.Series, 0) ) - var builder labels.ScratchBuilder - all := allPostings(t, ir) for all.Next() { testutil.Ok(t, ir.Series(all.At(), &builder, &chunkMetas)) - lset = builder.Labels() + lset := builder.Labels() expected = append(expected, &storepb.Series{Labels: labelpb.ZLabelsFromPromLabels(append(extLabels.Copy(), lset...))}) if skipChunks { @@ -164,13 +166,25 @@ func ReadSeriesFromBlock(t testing.TB, h tsdb.BlockReader, extLabels labels.Labe return expected } +func CreateBlockFromHead(t testing.TB, dir string, head *tsdb.Head) ulid.ULID { + compactor, err := tsdb.NewLeveledCompactor(context.Background(), nil, log.NewNopLogger(), []int64{1000000}, nil, nil) + testutil.Ok(t, err) + + testutil.Ok(t, os.MkdirAll(dir, 0777)) + + // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). + // Because of this block intervals are always +1 than the total samples it includes. + ulid, err := compactor.Write(dir, head, head.MinTime(), head.MaxTime()+1, nil) + testutil.Ok(t, err) + return ulid +} + func appendFloatSamples(t testing.TB, app storage.Appender, tsLabel int, opts HeadGenOptions) { - ref, err := app.Append( - 0, - labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix), "j", fmt.Sprintf("%v", tsLabel)), - int64(tsLabel)*opts.ScrapeInterval.Milliseconds(), - opts.Random.Float64(), - ) + lblSet := labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix)) + if opts.IncludeName { + lblSet = append(lblSet, labels.Label{Name: "__name__", Value: "test_float_metric"}) + } + ref, err := app.Append(0, lblSet, int64(tsLabel)*opts.ScrapeInterval.Milliseconds(), opts.Random.Float64()) testutil.Ok(t, err) for is := 1; is < opts.SamplesPerSeries; is++ { @@ -180,30 +194,23 @@ func appendFloatSamples(t testing.TB, app storage.Appender, tsLabel int, opts He } func appendHistogramSamples(t testing.TB, app storage.Appender, tsLabel int, opts HeadGenOptions) { - sample := &histogram.Histogram{ - Schema: 0, - Count: 9, - Sum: -3.1415, - ZeroCount: 12, - ZeroThreshold: 0.001, - NegativeSpans: []histogram.Span{ - {Offset: 0, Length: 4}, - {Offset: 1, Length: 1}, - }, - NegativeBuckets: []int64{1, 2, -2, 1, -1}, - } + histograms := tsdbutil.GenerateTestHistograms(opts.SamplesPerSeries) + lblSet := labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix)) + if opts.IncludeName { + lblSet = append(lblSet, labels.Label{Name: "__name__", Value: "test_metric"}) + } ref, err := app.AppendHistogram( 0, - labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix), "j", fmt.Sprintf("%v", tsLabel)), + lblSet, int64(tsLabel)*opts.ScrapeInterval.Milliseconds(), - sample, + histograms[0], nil, ) testutil.Ok(t, err) - for is := 1; is < opts.SamplesPerSeries; is++ { - _, err := app.AppendHistogram(ref, nil, int64(tsLabel+is)*opts.ScrapeInterval.Milliseconds(), sample, nil) + for i, h := range histograms[1:] { + _, err := app.AppendHistogram(ref, nil, int64(tsLabel+i+1)*opts.ScrapeInterval.Milliseconds(), h, nil) testutil.Ok(t, err) } } @@ -311,7 +318,7 @@ func TestServerSeries(t testutil.TB, store storepb.StoreServer, cases ...*Series } // Huge responses can produce unreadable diffs - make it more human readable. - if len(c.ExpectedSeries) > 4 { + if len(c.ExpectedSeries) > 1 { for j := range c.ExpectedSeries { testutil.Equals(t, c.ExpectedSeries[j].Labels, srv.SeriesSet[j].Labels, "%v series chunks mismatch", j) diff --git a/pkg/store/tsdb_test.go b/pkg/store/tsdb_test.go index 08d3c703aa..380aa7e60d 100644 --- a/pkg/store/tsdb_test.go +++ b/pkg/store/tsdb_test.go @@ -13,12 +13,12 @@ import ( "testing" "github.com/cespare/xxhash" + "github.com/efficientgo/core/testutil" "github.com/go-kit/log" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" - "github.com/efficientgo/core/testutil" "github.com/thanos-io/thanos/pkg/component" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" @@ -285,7 +285,7 @@ func TestTSDBStore_SeriesAccessWithDelegateClosing(t *testing.T) { Random: random, SkipChunks: true, }) - _ = createBlockFromHead(t, tmpDir, head) + _ = storetestutil.CreateBlockFromHead(t, tmpDir, head) testutil.Ok(t, head.Close()) head, _ = storetestutil.CreateHeadWithSeries(t, 1, storetestutil.HeadGenOptions{ @@ -454,7 +454,7 @@ func TestTSDBStore_SeriesAccessWithoutDelegateClosing(t *testing.T) { Random: random, SkipChunks: true, }) - _ = createBlockFromHead(t, tmpDir, head) + _ = storetestutil.CreateBlockFromHead(t, tmpDir, head) testutil.Ok(t, head.Close()) head, _ = storetestutil.CreateHeadWithSeries(t, 1, storetestutil.HeadGenOptions{ @@ -594,7 +594,7 @@ func benchTSDBStoreSeries(t testutil.TB, totalSamples, totalSeries int) { resps[j] = append(resps[j], storepb.NewSeriesResponse(created[i])) } - _ = createBlockFromHead(t, tmpDir, head) + _ = storetestutil.CreateBlockFromHead(t, tmpDir, head) t.Cleanup(func() { testutil.Ok(t, head.Close()) }) From b0fe416823464cf2aa8a6a8b01ce061d9be4ad68 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Tue, 13 Jun 2023 13:13:35 +0200 Subject: [PATCH 2/9] Fixed tests, removed unused code Signed-off-by: Sebastian Rabenhorst Fixed tests, removed unused code Signed-off-by: Sebastian Rabenhorst Fixed tests, removed unused code Signed-off-by: Sebastian Rabenhorst --- pkg/compact/downsample/downsample.go | 2 +- pkg/store/storepb/testutil/series.go | 13 +++---------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/pkg/compact/downsample/downsample.go b/pkg/compact/downsample/downsample.go index 39fe82ff9d..47a9b77fef 100644 --- a/pkg/compact/downsample/downsample.go +++ b/pkg/compact/downsample/downsample.go @@ -1200,7 +1200,7 @@ func (it *AverageChunkIterator) Next() chunkenc.ValueType { if st == chunkenc.ValFloatHistogram { _, sumV := it.sumIt.AtFloatHistogram() - it.fh = sumV.Scale(1 / cntV) + it.fh = sumV.Mul(1 / cntV) return chunkenc.ValFloatHistogram } diff --git a/pkg/store/storepb/testutil/series.go b/pkg/store/storepb/testutil/series.go index 34448cd9b2..a5d21b96e3 100644 --- a/pkg/store/storepb/testutil/series.go +++ b/pkg/store/storepb/testutil/series.go @@ -57,7 +57,6 @@ type HeadGenOptions struct { PrependLabels labels.Labels SkipChunks bool // Skips chunks in returned slice (not in generated head!). SampleType chunkenc.ValueType - IncludeName bool Random *rand.Rand } @@ -180,10 +179,7 @@ func CreateBlockFromHead(t testing.TB, dir string, head *tsdb.Head) ulid.ULID { } func appendFloatSamples(t testing.TB, app storage.Appender, tsLabel int, opts HeadGenOptions) { - lblSet := labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix)) - if opts.IncludeName { - lblSet = append(lblSet, labels.Label{Name: "__name__", Value: "test_float_metric"}) - } + lblSet := labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix), "j", fmt.Sprintf("%v", tsLabel)) ref, err := app.Append(0, lblSet, int64(tsLabel)*opts.ScrapeInterval.Milliseconds(), opts.Random.Float64()) testutil.Ok(t, err) @@ -196,10 +192,7 @@ func appendFloatSamples(t testing.TB, app storage.Appender, tsLabel int, opts He func appendHistogramSamples(t testing.TB, app storage.Appender, tsLabel int, opts HeadGenOptions) { histograms := tsdbutil.GenerateTestHistograms(opts.SamplesPerSeries) - lblSet := labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix)) - if opts.IncludeName { - lblSet = append(lblSet, labels.Label{Name: "__name__", Value: "test_metric"}) - } + lblSet := labels.FromStrings("foo", "bar", "i", fmt.Sprintf("%07d%s", tsLabel, LabelLongSuffix), "j", fmt.Sprintf("%v", tsLabel)) ref, err := app.AppendHistogram( 0, lblSet, @@ -318,7 +311,7 @@ func TestServerSeries(t testutil.TB, store storepb.StoreServer, cases ...*Series } // Huge responses can produce unreadable diffs - make it more human readable. - if len(c.ExpectedSeries) > 1 { + if len(c.ExpectedSeries) > 4 { for j := range c.ExpectedSeries { testutil.Equals(t, c.ExpectedSeries[j].Labels, srv.SeriesSet[j].Labels, "%v series chunks mismatch", j) From 0415cc45a764576daa280a6e9cdca462e4b1f6a3 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Mon, 24 Jul 2023 11:34:23 +0200 Subject: [PATCH 3/9] Added test for mixed types and gauge reset fix Signed-off-by: Sebastian Rabenhorst --- pkg/compact/downsample/downsample.go | 2 +- pkg/compact/downsample/downsample_test.go | 293 +++++++++++++++++++--- 2 files changed, 258 insertions(+), 37 deletions(-) diff --git a/pkg/compact/downsample/downsample.go b/pkg/compact/downsample/downsample.go index 47a9b77fef..71aaaaf787 100644 --- a/pkg/compact/downsample/downsample.go +++ b/pkg/compact/downsample/downsample.go @@ -356,7 +356,7 @@ func (h *histogramAggregator) add(s sample) { } if h.total > 0 { - if fh.DetectReset(h.previous) { + if fh.CounterResetHint != histogram.GaugeType && fh.DetectReset(h.previous) { // Counter reset, correct the value. h.counter.Add(fh) h.resets++ diff --git a/pkg/compact/downsample/downsample_test.go b/pkg/compact/downsample/downsample_test.go index 0751b2fc6c..9ed723bd0b 100644 --- a/pkg/compact/downsample/downsample_test.go +++ b/pkg/compact/downsample/downsample_test.go @@ -800,6 +800,57 @@ func TestDownSampleNativeHistogram(t *testing.T) { }, []expectedHistogramAggregates{}, }, + { + "gauge sum aggregate no counter reset", + generateFloatHistogramSamples(30_000, 0, 11), + []expectedHistogramAggregates{ + { + count: []sample{ + {t: 299_999, v: 10}, + {t: 300_000, v: 1}, + }, + counter: []sample{ + { + t: 299_999, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(10), histogram.UnknownCounterReset), + }, + { + t: 300_000, + fh: withCounterResetHint(tsdbutil.GenerateTestFloatHistogram(10), histogram.NotCounterReset), + }, + }, + sum: []sample{ + { + t: 299_999, + fh: expectedFloatHistogramsSum(generateFloatHistograms(10), 0), + }, + { + t: 300_000, + fh: withCounterResetHint(tsdbutil.GenerateTestFloatHistogram(10), histogram.GaugeType), + }, + }, + }, + }, + []expectedHistogramAggregates{ + { + count: []sample{ + {t: 300_000, v: 11}, + }, + counter: []sample{ + { + t: 300_000, + fh: expectedFloatHistogramsCounter(generateFloatHistograms(11), histogram.UnknownCounterReset), + }, + }, + sum: []sample{ + { + t: 300_000, + fh: expectedFloatHistogramsSum(generateFloatHistograms(11), 0), + }, + }, + }, + }, + }, } logger := log.NewLogfmtLogger(os.Stderr) @@ -807,44 +858,12 @@ func TestDownSampleNativeHistogram(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { dir := t.TempDir() - ser := &series{lset: labels.FromStrings("__name__", "a")} - - mint := tt.samples[0].t - maxt := mint - - chunk, app := newChunk(t, tt.samples[0].fh.CounterResetHint) - - var previous *histogram.FloatHistogram - for i, s := range tt.samples { - if i > 0 && s.fh.DetectReset(previous) { - ser.chunks = append(ser.chunks, chunks.Meta{ - MinTime: mint, - MaxTime: maxt, - Chunk: chunk, - }) - chunk, app = newChunk(t, s.fh.CounterResetHint) - mint = s.t - } - if s.t > maxt { - maxt = s.t - } - app.AppendFloatHistogram(s.t, s.fh) - previous = s.fh - } - - ser.chunks = append(ser.chunks, chunks.Meta{ - MinTime: math.MaxInt64, - MaxTime: math.MinInt64, - Chunk: chunk, - }) - - mb := newMemBlock() - mb.addSeries(ser) + mb := blockFromChunks(chunksFromHistogramSamples(t, tt.samples)) fakeMeta := &metadata.Meta{ BlockMeta: tsdb.BlockMeta{ MinTime: tt.samples[0].t, - MaxTime: maxt, + MaxTime: tt.samples[len(tt.samples)-1].t, }, } idResLevel1, err := Downsample(logger, fakeMeta, mb, dir, ResLevel1) @@ -874,7 +893,208 @@ func TestDownSampleNativeHistogram(t *testing.T) { } } -func newChunk(t *testing.T, counterResetHint histogram.CounterResetHint) (*chunkenc.FloatHistogramChunk, chunkenc.Appender) { +func TestDownsampleMixedChunkTypes(t *testing.T) { + var ( + ts int64 + samples []sample + rawChks []chunks.Meta + ) + + for _, fh := range tsdbutil.GenerateTestFloatHistograms(20) { + ts += 15_000 + samples = append(samples, sample{ + t: ts, + fh: fh, + }) + } + + rawChks = append(rawChks, chunksFromHistogramSamples(t, samples)...) + + samples = samples[:0] + for i := 0; i < 20; i++ { + ts += 15_000 + samples = append(samples, sample{ + t: ts, + v: float64(i), + }) + } + + rawChks = append(rawChks, chunkFromSamples(t, samples)) + + dir := t.TempDir() + mb := blockFromChunks(rawChks) + fakeMeta := &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + MinTime: 15_000, + MaxTime: ts, + }, + } + + logger := log.NewLogfmtLogger(os.Stderr) + + idResLevel1, err := Downsample(logger, fakeMeta, mb, dir, ResLevel1) + testutil.Ok(t, err) + + meta, chks := GetMetaAndChunks(t, dir, idResLevel1) + + assertValidChunkTime(t, chks) + + compareAggreggates(t, dir, ResLevel1, idResLevel1.String(), []expectedHistogramAggregates{ + { + count: []sample{ + {t: 299_999, v: 19}, + {t: 300_000, v: 1}, + }, + counter: []sample{ + {t: 299_999, fh: tsdbutil.GenerateTestFloatHistogram(18)}, + {t: 300_000, fh: withCounterResetHint(tsdbutil.GenerateTestFloatHistogram(19), histogram.NotCounterReset)}, + }, + sum: []sample{ + { + t: 299_999, + fh: expectedFloatHistogramsSum(tsdbutil.GenerateTestFloatHistograms(19), 0), + }, + { + t: 300_000, + fh: withCounterResetHint(tsdbutil.GenerateTestFloatHistogram(19), histogram.GaugeType), + }, + }, + }, + { + count: []sample{ + {t: 599_999, v: 19}, + {t: 600_000, v: 1}, + }, + counter: []sample{ + {t: 315000, v: 0}, + {t: 599_999, v: 18}, + {t: 600_000, v: 19}, + {t: 600_000, v: 19}, + }, + sum: []sample{ + {t: 599_999, v: 171}, + {t: 600_000, v: 19}, + }, + }, + }, chks) + + blk, err := tsdb.OpenBlock(logger, filepath.Join(dir, idResLevel1.String()), NewPool()) + testutil.Ok(t, err) + idResLevel2, err := Downsample(logger, meta, blk, dir, ResLevel2) + testutil.Ok(t, err) + + _, chks = GetMetaAndChunks(t, dir, idResLevel2) + + assertValidChunkTime(t, chks) + + compareAggreggates(t, dir, ResLevel2, idResLevel2.String(), []expectedHistogramAggregates{ + { + count: []sample{ + {t: 300_000, v: 20}, + }, + counter: []sample{ + { + t: 300_000, + fh: tsdbutil.GenerateTestFloatHistogram(19), + }, + }, + sum: []sample{ + { + t: 300_000, + fh: expectedFloatHistogramsSum(tsdbutil.GenerateTestFloatHistograms(20), 0), + }, + }, + }, + { + count: []sample{ + {t: 600000, v: 20}, + }, + counter: []sample{ + {t: 315000, v: 0}, + {t: 600_000, v: 19}, + {t: 600_000, v: 19}, + }, + sum: []sample{ + {t: 600_000, v: 190}, + }, + }, + }, chks) +} + +func withCounterResetHint(fh *histogram.FloatHistogram, hint histogram.CounterResetHint) *histogram.FloatHistogram { + fh.CounterResetHint = hint + return fh +} + +func blockFromChunks(chks []chunks.Meta) *memBlock { + ser := &series{ + lset: labels.FromStrings("__name__", "a"), + chunks: chks, + } + mb := newMemBlock() + mb.addSeries(ser) + return mb +} + +func chunkFromSamples(t *testing.T, samples []sample) chunks.Meta { + if len(samples) == 0 { + return chunks.Meta{} + } + + chk := chunkenc.NewXORChunk() + app, err := chk.Appender() + testutil.Ok(t, err) + + for _, s := range samples { + app.Append(s.t, s.v) + } + + return chunks.Meta{ + MinTime: samples[0].t, + MaxTime: samples[len(samples)-1].t, + Chunk: chk, + } +} + +func chunksFromHistogramSamples(t *testing.T, samples []sample) []chunks.Meta { + if len(samples) == 0 { + return nil + } + + var ( + mint = samples[0].t + maxt = samples[0].t + chks []chunks.Meta + ) + + chk, app := newHistogramChunk(t, samples[0].fh.CounterResetHint) + + var previous *histogram.FloatHistogram + for i, s := range samples { + if i > 0 && s.fh.DetectReset(previous) { + chks = append(chks, chunks.Meta{ + MinTime: mint, + MaxTime: maxt, + Chunk: chk, + }) + chk, app = newHistogramChunk(t, s.fh.CounterResetHint) + mint = s.t + } + app.AppendFloatHistogram(s.t, s.fh) + maxt = s.t + previous = s.fh + } + + chks = append(chks, chunks.Meta{ + MinTime: mint, + MaxTime: maxt, + Chunk: chk, + }) + + return chks +} + +func newHistogramChunk(t *testing.T, counterResetHint histogram.CounterResetHint) (*chunkenc.FloatHistogramChunk, chunkenc.Appender) { raw := chunkenc.NewFloatHistogramChunk() app, err := raw.Appender() if counterResetHint == histogram.GaugeType { @@ -884,6 +1104,7 @@ func newChunk(t *testing.T, counterResetHint histogram.CounterResetHint) (*chunk return raw, app } +//nolint:unparam func generateFloatHistogramSamples(scrapeInterval, startTs int64, n ...int) []sample { // Generate samples for the float histogram. ts := startTs @@ -956,7 +1177,7 @@ func compareAggreggates(t *testing.T, dir string, resLevel int64, blockID string testUtilWithOps := testutil.WithGoCmp( cmpopts.EquateApprox(0, 1e-9), - cmp.AllowUnexported(sample{}), + cmp.AllowUnexported(sample{}, histogram.FloatHistogram{}), ) for i, c := range chks { From 34155dca7a8a8a5da21e95c6e4c73f49c31f047f Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Thu, 27 Jul 2023 19:07:53 +0200 Subject: [PATCH 4/9] Add downsampling v2 (wip) --- pkg/compact/downsamplev2/downsample.go | 265 ++++++++++++++++++++ pkg/compact/downsamplev2/downsample_test.go | 56 +++++ 2 files changed, 321 insertions(+) create mode 100644 pkg/compact/downsamplev2/downsample.go create mode 100644 pkg/compact/downsamplev2/downsample_test.go diff --git a/pkg/compact/downsamplev2/downsample.go b/pkg/compact/downsamplev2/downsample.go new file mode 100644 index 0000000000..94fc2cd2ce --- /dev/null +++ b/pkg/compact/downsamplev2/downsample.go @@ -0,0 +1,265 @@ +package downsamplev2 + +import ( + "context" + "fmt" + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/promql/parser" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + v1 "github.com/prometheus/prometheus/web/api/v1" + "github.com/thanos-io/promql-engine/engine" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/compact/downsample" + "github.com/thanos-io/thanos/pkg/errutil" + "github.com/thanos-io/thanos/pkg/runutil" + "math" + "math/rand" + "os" + "path/filepath" + "strings" + "time" +) + +const ( + maxSamplesPerChunk = 120 +) + +var ( + promAggregators = map[downsample.AggrType][2]string{ + downsample.AggrCount: {"count_over_time", "5m"}, + downsample.AggrSum: {"sum", ""}, + downsample.AggrMin: {"min", ""}, + downsample.AggrMax: {"max", ""}, + downsample.AggrCounter: {"increase", "5m"}, + } +) + +type blockQuerier struct { + q storage.Querier +} + +func (b *blockQuerier) Querier(context.Context, int64, int64) (storage.Querier, error) { + return b.q, nil +} + +func Downsample( + ctx context.Context, + logger log.Logger, + origMeta *metadata.Meta, + b tsdb.BlockReader, + dir string, + resolution int64, +) (id ulid.ULID, err error) { + if origMeta.Thanos.Downsample.Resolution >= resolution { + return id, errors.New("target resolution not lower than existing one") + } + + indexr, err := b.Index() + if err != nil { + return id, errors.Wrap(err, "open index reader") + } + defer runutil.CloseWithErrCapture(&err, indexr, "downsample index reader") + + // Generate new block id. + uid := ulid.MustNew(ulid.Now(), rand.New(rand.NewSource(time.Now().UnixNano()))) + + // Create block directory to populate with chunks, meta and index files into. + blockDir := filepath.Join(dir, uid.String()) + if err := os.MkdirAll(blockDir, 0750); err != nil { + return id, errors.Wrap(err, "mkdir block dir") + } + + // Remove blockDir in case of errors. + defer func() { + if err != nil { + var merr errutil.MultiError + merr.Add(err) + merr.Add(os.RemoveAll(blockDir)) + err = merr.Err() + } + }() + + // Copy original meta to the new one. Update downsampling resolution and ULID for a new block. + newMeta := *origMeta + newMeta.Thanos.Downsample.Resolution = resolution + newMeta.ULID = uid + + // Writes downsampled chunks right into the files, avoiding excess memory allocation. + // Flushes index and meta data after aggregations. + streamedBlockWriter, err := downsample.NewStreamedBlockWriter(blockDir, indexr, logger, newMeta) + if err != nil { + return id, errors.Wrap(err, "get streamed block writer") + } + defer runutil.CloseWithErrCapture(&err, streamedBlockWriter, "close stream block writer") + + if origMeta.Thanos.Downsample.Resolution == 0 { + q, err := tsdb.NewBlockQuerier(b, b.Meta().MinTime, b.Meta().MaxTime) + if err != nil { + return id, errors.Wrap(err, "create querier") + } + + // Get all series from the block. + seriesSet := q.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, labels.MetricName, ".+")) + + // Create new promql engine. + opts := promql.EngineOpts{ + Logger: logger, + Reg: nil, + MaxSamples: math.MaxInt32, + Timeout: 30 * time.Second, + LookbackDelta: 5 * time.Minute, + } + ng := engine.New(engine.Opts{EngineOpts: opts}) + + for seriesSet.Next() { + lset := seriesSet.At().Labels() + chks, err := downsampleRawSeries(ctx, &blockQuerier{q: q}, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime)) + if err != nil { + return id, errors.Wrapf(err, "downsample series: %v", lset.String()) + } + if err := streamedBlockWriter.WriteSeries(lset, chks); err != nil { + return id, errors.Wrapf(err, "write series: %v", lset.String()) + } + } + + } else { + return id, errors.New("downsampling of already downsampled block is not supported yet") + } + return uid, nil +} + +func downsampleRawSeries( + ctx context.Context, + q storage.Queryable, + ng v1.QueryEngine, + lset labels.Labels, + bMint, bMaxt time.Time, +) ([]chunks.Meta, error) { + var ( + aggrSeries [5]*promql.Series + err error + ) + for i := 0; i <= int(downsample.AggrCounter); i++ { + aggrSeries[i], err = querySingleSeries(ctx, ng, q, queryString(promAggregators[downsample.AggrType(i)][0], promAggregators[downsample.AggrType(i)][1], lset), bMint, bMaxt, 5*time.Minute) + if err != nil { + return nil, err + } + // TODO: query first value of counter + if i == int(downsample.AggrCounter) { + aggrSeries[i].Floats = append([]promql.FPoint{{T: 0, F: 0}}, aggrSeries[i].Floats...) + } + } + + if !aligned(aggrSeries) { + return nil, errors.New("series aggregates are not aligned") + } + + var aggrChunks []chunks.Meta + + for i := 0; i < len(aggrSeries[downsample.AggrCounter].Floats); i += 120 { + to := i + 119 + if to > len(aggrSeries[downsample.AggrCounter].Floats) { + to = len(aggrSeries[downsample.AggrCounter].Floats) + } + aggrChunks = append(aggrChunks, downsampleFloatBatch(aggrSeries, int64(i), int64(to))) + } + + return aggrChunks, nil +} + +func queryString(aggregator string, vectorRange string, series labels.Labels) string { + builder := strings.Builder{} + builder.WriteString(aggregator) + builder.WriteString("({") + for i, l := range series { + builder.WriteString(fmt.Sprintf("%s=\"%s\"", l.Name, l.Value)) + if i != len(series)-1 { + builder.WriteString(",") + } + } + builder.WriteString("}") + if vectorRange != "" { + builder.WriteString("[") + builder.WriteString(vectorRange) + builder.WriteString("]") + } + builder.WriteString(")") + return builder.String() +} + +func querySingleSeries(ctx context.Context, ng v1.QueryEngine, q storage.Queryable, qs string, mint, maxt time.Time, step time.Duration) (*promql.Series, error) { + query, err := ng.NewRangeQuery(ctx, q, &promql.QueryOpts{}, qs, mint, maxt, step) + if err != nil { + return nil, err + } + sqres := query.Exec(ctx) + if sqres.Err != nil { + return nil, sqres.Err + } + var res promql.Series + switch sqres.Value.Type() { + case parser.ValueTypeMatrix: + matrix := sqres.Value.(promql.Matrix) + if len(matrix) > 1 { + return nil, errors.New("more than one series returned") + } + res = matrix[0] + default: + return nil, errors.New("unknown result type") + } + return &res, nil +} + +func downsampleFloatBatch(aggrSeries [5]*promql.Series, from, to int64) chunks.Meta { + var ( + aggrChunks [5]chunkenc.Chunk + aggrApps [5]chunkenc.Appender + ) + + aggrChunks[downsample.AggrCount] = chunkenc.NewXORChunk() + aggrChunks[downsample.AggrSum] = chunkenc.NewXORChunk() + aggrChunks[downsample.AggrMin] = chunkenc.NewXORChunk() + aggrChunks[downsample.AggrMax] = chunkenc.NewXORChunk() + aggrChunks[downsample.AggrCounter] = chunkenc.NewXORChunk() + + for i, c := range aggrChunks { + if c != nil { + aggrApps[i], _ = c.Appender() + } + } + + // A panic here means that series aggregates are not aligned. + for i := from; i < to; i++ { + for i := 0; i <= int(downsample.AggrCounter); i++ { + aggrApps[i].Append(aggrSeries[i].Floats[i].T, aggrSeries[i].Floats[i].F) + } + } + + return chunks.Meta{ + MinTime: aggrSeries[downsample.AggrCounter].Floats[from].T, + MaxTime: aggrSeries[downsample.AggrCounter].Floats[to-1].T, + Chunk: downsample.EncodeAggrChunk(aggrChunks), + } +} + +func aligned(series [5]*promql.Series) bool { + for _, s := range series { + if len(s.Floats) != len(series[0].Floats) { + return false + } + if s.Floats[0].T != series[0].Floats[0].T { + return false + } + if s.Floats[len(s.Floats)-1].T != series[0].Floats[len(series[0].Floats)-1].T { + return false + } + } + return true +} diff --git a/pkg/compact/downsamplev2/downsample_test.go b/pkg/compact/downsamplev2/downsample_test.go new file mode 100644 index 0000000000..c73074d9be --- /dev/null +++ b/pkg/compact/downsamplev2/downsample_test.go @@ -0,0 +1,56 @@ +package downsamplev2 + +import ( + "context" + "fmt" + "os" + "testing" + "time" + + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/compact/downsample" + "github.com/thanos-io/thanos/pkg/testutil/e2eutil" + + "github.com/efficientgo/core/testutil" + "github.com/go-kit/log" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" +) + +func TestDownsample(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + + dir := t.TempDir() + logger := log.NewLogfmtLogger(os.Stderr) + + id, err := e2eutil.CreateBlock( + ctx, + dir, + []labels.Labels{ + {{Name: labels.MetricName, Value: "a"}, {Name: "a", Value: "1"}}, + {{Name: labels.MetricName, Value: "a"}, {Name: "a", Value: "2"}}, + }, + 9600, 0, downsample.ResLevel1DownsampleRange+1, // Pass the minimum ResLevel1DownsampleRange check. + labels.Labels{{Name: "e1", Value: "1"}}, + downsample.ResLevel0, metadata.NoneFunc) + testutil.Ok(t, err) + + bdir := fmt.Sprintf("%v/%v", dir, id.String()) + pool := chunkenc.NewPool() + b, err := tsdb.OpenBlock(logger, bdir, pool) + testutil.Ok(t, err) + meta, err := metadata.ReadFromDir(bdir) + testutil.Ok(t, err) + + ctx = context.Background() + uid, err := Downsample(ctx, logger, meta, b, dir, downsample.ResLevel1) + testutil.Ok(t, err) + fmt.Println(uid) +} + +func TestQueryString(t *testing.T) { + qs := queryString("increase", "5m", labels.FromStrings(labels.MetricName, "a", "a", "1")) + testutil.Equals(t, `increase({__name__="a",a="1"}[5m])`, qs) +} From 59f33519c08763e9b6bff3f20b1a94df2406ddfc Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Thu, 27 Jul 2023 19:35:55 +0200 Subject: [PATCH 5/9] Improved downsampling --- pkg/compact/downsamplev2/downsample.go | 26 +++++++++++++++----------- 1 file changed, 15 insertions(+), 11 deletions(-) diff --git a/pkg/compact/downsamplev2/downsample.go b/pkg/compact/downsamplev2/downsample.go index 94fc2cd2ce..5d893e4013 100644 --- a/pkg/compact/downsamplev2/downsample.go +++ b/pkg/compact/downsamplev2/downsample.go @@ -163,12 +163,16 @@ func downsampleRawSeries( var aggrChunks []chunks.Meta - for i := 0; i < len(aggrSeries[downsample.AggrCounter].Floats); i += 120 { - to := i + 119 - if to > len(aggrSeries[downsample.AggrCounter].Floats) { - to = len(aggrSeries[downsample.AggrCounter].Floats) + for i := 0; i < len(aggrSeries[downsample.AggrCounter].Floats); i += maxSamplesPerChunk { + var aggrPoints [5][]promql.FPoint + high := i + maxSamplesPerChunk + if high > len(aggrSeries[downsample.AggrCounter].Floats) { + high = len(aggrSeries[downsample.AggrCounter].Floats) } - aggrChunks = append(aggrChunks, downsampleFloatBatch(aggrSeries, int64(i), int64(to))) + for j := 0; j <= int(downsample.AggrCounter); j++ { + aggrPoints[j] = aggrSeries[j].Floats[i:high] + } + aggrChunks = append(aggrChunks, downsampleFloatBatch(aggrPoints)) } return aggrChunks, nil @@ -217,7 +221,7 @@ func querySingleSeries(ctx context.Context, ng v1.QueryEngine, q storage.Queryab return &res, nil } -func downsampleFloatBatch(aggrSeries [5]*promql.Series, from, to int64) chunks.Meta { +func downsampleFloatBatch(aggrPoints [5][]promql.FPoint) chunks.Meta { var ( aggrChunks [5]chunkenc.Chunk aggrApps [5]chunkenc.Appender @@ -236,15 +240,15 @@ func downsampleFloatBatch(aggrSeries [5]*promql.Series, from, to int64) chunks.M } // A panic here means that series aggregates are not aligned. - for i := from; i < to; i++ { - for i := 0; i <= int(downsample.AggrCounter); i++ { - aggrApps[i].Append(aggrSeries[i].Floats[i].T, aggrSeries[i].Floats[i].F) + for i := 0; i < len(aggrPoints[downsample.AggrCounter]); i++ { + for j := 0; j <= int(downsample.AggrCounter); j++ { + aggrApps[j].Append(aggrPoints[j][i].T, aggrPoints[j][i].F) } } return chunks.Meta{ - MinTime: aggrSeries[downsample.AggrCounter].Floats[from].T, - MaxTime: aggrSeries[downsample.AggrCounter].Floats[to-1].T, + MinTime: aggrPoints[downsample.AggrCounter][0].T, + MaxTime: aggrPoints[downsample.AggrCounter][len(aggrPoints[downsample.AggrCounter])-1].T, Chunk: downsample.EncodeAggrChunk(aggrChunks), } } From 8eab858b45335ffc739d41da0796c074002d5aa9 Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Thu, 27 Jul 2023 20:00:00 +0200 Subject: [PATCH 6/9] Add aggregate chunk builder --- pkg/compact/downsamplev2/downsample.go | 75 ++++++++++++++++---------- 1 file changed, 46 insertions(+), 29 deletions(-) diff --git a/pkg/compact/downsamplev2/downsample.go b/pkg/compact/downsamplev2/downsample.go index 5d893e4013..1f0e1336c9 100644 --- a/pkg/compact/downsamplev2/downsample.go +++ b/pkg/compact/downsamplev2/downsample.go @@ -163,17 +163,18 @@ func downsampleRawSeries( var aggrChunks []chunks.Meta - for i := 0; i < len(aggrSeries[downsample.AggrCounter].Floats); i += maxSamplesPerChunk { - var aggrPoints [5][]promql.FPoint - high := i + maxSamplesPerChunk - if high > len(aggrSeries[downsample.AggrCounter].Floats) { - high = len(aggrSeries[downsample.AggrCounter].Floats) + counter := 0 + ab := newAggrChunkBuilder() + for i := range aggrSeries[downsample.AggrCounter].Floats { + if counter >= maxSamplesPerChunk { + aggrChunks = append(aggrChunks, ab.encode()) + ab = newAggrChunkBuilder() + counter = 0 } - for j := 0; j <= int(downsample.AggrCounter); j++ { - aggrPoints[j] = aggrSeries[j].Floats[i:high] - } - aggrChunks = append(aggrChunks, downsampleFloatBatch(aggrPoints)) + ab.add(int64(i), aggrSeries) + counter++ } + aggrChunks = append(aggrChunks, newAggrChunkBuilder().encode()) return aggrChunks, nil } @@ -221,35 +222,51 @@ func querySingleSeries(ctx context.Context, ng v1.QueryEngine, q storage.Queryab return &res, nil } -func downsampleFloatBatch(aggrPoints [5][]promql.FPoint) chunks.Meta { - var ( - aggrChunks [5]chunkenc.Chunk - aggrApps [5]chunkenc.Appender - ) +type aggrChunkBuilder struct { + mint, maxt int64 - aggrChunks[downsample.AggrCount] = chunkenc.NewXORChunk() - aggrChunks[downsample.AggrSum] = chunkenc.NewXORChunk() - aggrChunks[downsample.AggrMin] = chunkenc.NewXORChunk() - aggrChunks[downsample.AggrMax] = chunkenc.NewXORChunk() - aggrChunks[downsample.AggrCounter] = chunkenc.NewXORChunk() + chunks [5]chunkenc.Chunk + apps [5]chunkenc.Appender +} + +func newAggrChunkBuilder() *aggrChunkBuilder { + b := &aggrChunkBuilder{ + mint: math.MaxInt64, + maxt: math.MinInt64, + } + b.chunks[downsample.AggrCount] = chunkenc.NewXORChunk() + b.chunks[downsample.AggrSum] = chunkenc.NewXORChunk() + b.chunks[downsample.AggrMin] = chunkenc.NewXORChunk() + b.chunks[downsample.AggrMax] = chunkenc.NewXORChunk() + b.chunks[downsample.AggrCounter] = chunkenc.NewXORChunk() - for i, c := range aggrChunks { + for i, c := range b.chunks { if c != nil { - aggrApps[i], _ = c.Appender() + b.apps[i], _ = c.Appender() } } + return b +} - // A panic here means that series aggregates are not aligned. - for i := 0; i < len(aggrPoints[downsample.AggrCounter]); i++ { - for j := 0; j <= int(downsample.AggrCounter); j++ { - aggrApps[j].Append(aggrPoints[j][i].T, aggrPoints[j][i].F) - } +func (b *aggrChunkBuilder) add(i int64, aggrSeries [5]*promql.Series) { + t := aggrSeries[downsample.AggrCounter].Floats[i].T + if t < b.mint { + b.mint = t + } + if t > b.maxt { + b.maxt = t } + for j := 0; j <= int(downsample.AggrCounter); j++ { + // Panic here means misaligned series. + b.apps[j].Append(aggrSeries[j].Floats[i].T, aggrSeries[j].Floats[i].F) + } +} +func (b *aggrChunkBuilder) encode() chunks.Meta { return chunks.Meta{ - MinTime: aggrPoints[downsample.AggrCounter][0].T, - MaxTime: aggrPoints[downsample.AggrCounter][len(aggrPoints[downsample.AggrCounter])-1].T, - Chunk: downsample.EncodeAggrChunk(aggrChunks), + MinTime: b.mint, + MaxTime: b.maxt, + Chunk: downsample.EncodeAggrChunk(b.chunks), } } From 84bc9fb4ad916f787ff1750be1b23cf80271084f Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Fri, 28 Jul 2023 15:34:14 +0200 Subject: [PATCH 7/9] Added aggregate aggregation --- pkg/compact/downsamplev2/downsample.go | 101 ++++++++++++++++++-- pkg/compact/downsamplev2/downsample_test.go | 32 +++++-- pkg/store/bucket.go | 4 +- pkg/store/bucket_test.go | 2 +- 4 files changed, 122 insertions(+), 17 deletions(-) diff --git a/pkg/compact/downsamplev2/downsample.go b/pkg/compact/downsamplev2/downsample.go index 1f0e1336c9..0d2e927e85 100644 --- a/pkg/compact/downsamplev2/downsample.go +++ b/pkg/compact/downsamplev2/downsample.go @@ -14,11 +14,15 @@ import ( "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" v1 "github.com/prometheus/prometheus/web/api/v1" + "github.com/thanos-io/objstore" + "github.com/thanos-io/objstore/providers/filesystem" "github.com/thanos-io/promql-engine/engine" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/compact/downsample" "github.com/thanos-io/thanos/pkg/errutil" + "github.com/thanos-io/thanos/pkg/query" "github.com/thanos-io/thanos/pkg/runutil" + "github.com/thanos-io/thanos/pkg/store" "math" "math/rand" "os" @@ -33,11 +37,11 @@ const ( var ( promAggregators = map[downsample.AggrType][2]string{ - downsample.AggrCount: {"count_over_time", "5m"}, + downsample.AggrCount: {"count_over_time", "true"}, downsample.AggrSum: {"sum", ""}, downsample.AggrMin: {"min", ""}, downsample.AggrMax: {"max", ""}, - downsample.AggrCounter: {"increase", "5m"}, + downsample.AggrCounter: {"increase", "true"}, } ) @@ -99,6 +103,7 @@ func Downsample( } defer runutil.CloseWithErrCapture(&err, streamedBlockWriter, "close stream block writer") + // Downsample raw block. if origMeta.Thanos.Downsample.Resolution == 0 { q, err := tsdb.NewBlockQuerier(b, b.Meta().MinTime, b.Meta().MaxTime) if err != nil { @@ -120,7 +125,7 @@ func Downsample( for seriesSet.Next() { lset := seriesSet.At().Labels() - chks, err := downsampleRawSeries(ctx, &blockQuerier{q: q}, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime)) + chks, err := downsampleRawSeries(ctx, &blockQuerier{q: q}, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime), 5*time.Minute) if err != nil { return id, errors.Wrapf(err, "downsample series: %v", lset.String()) } @@ -128,9 +133,40 @@ func Downsample( return id, errors.Wrapf(err, "write series: %v", lset.String()) } } - } else { - return id, errors.New("downsampling of already downsampled block is not supported yet") + q, err := aggrChunksBlockQueryable(ctx, logger, dir, origMeta) + if err != nil { + return id, errors.Wrap(err, "create queryable for aggregated block") + } + + // Create new promql engine. + opts := promql.EngineOpts{ + Logger: logger, + Reg: nil, + MaxSamples: math.MaxInt32, + Timeout: 30 * time.Second, + LookbackDelta: time.Hour, + } + ng := engine.New(engine.Opts{EngineOpts: opts}) + + qq, err := q.Querier(ctx, b.Meta().MinTime, b.Meta().MaxTime) + if err != nil { + return id, errors.Wrap(err, "create querier for aggregated block") + } + + seriesSet := qq.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, labels.MetricName, ".+")) + + for seriesSet.Next() { + lset := seriesSet.At().Labels() + chks, err := downsampleRawSeries(ctx, q, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime), time.Hour) + if err != nil { + return id, errors.Wrapf(err, "downsample series: %v", lset.String()) + } + if err := streamedBlockWriter.WriteSeries(lset, chks); err != nil { + return id, errors.Wrapf(err, "write series: %v", lset.String()) + } + } + } return uid, nil } @@ -141,13 +177,18 @@ func downsampleRawSeries( ng v1.QueryEngine, lset labels.Labels, bMint, bMaxt time.Time, + resolution time.Duration, ) ([]chunks.Meta, error) { var ( aggrSeries [5]*promql.Series err error ) + vectorRange := "5m" + if resolution > 5*time.Minute { + vectorRange = "1h" + } for i := 0; i <= int(downsample.AggrCounter); i++ { - aggrSeries[i], err = querySingleSeries(ctx, ng, q, queryString(promAggregators[downsample.AggrType(i)][0], promAggregators[downsample.AggrType(i)][1], lset), bMint, bMaxt, 5*time.Minute) + aggrSeries[i], err = querySingleSeries(ctx, ng, q, queryString(promAggregators[downsample.AggrType(i)][0], promAggregators[downsample.AggrType(i)][1] != "", vectorRange, lset), bMint, bMaxt, resolution) if err != nil { return nil, err } @@ -179,7 +220,7 @@ func downsampleRawSeries( return aggrChunks, nil } -func queryString(aggregator string, vectorRange string, series labels.Labels) string { +func queryString(aggregator string, needsVectorRange bool, vectorRange string, series labels.Labels) string { builder := strings.Builder{} builder.WriteString(aggregator) builder.WriteString("({") @@ -190,7 +231,7 @@ func queryString(aggregator string, vectorRange string, series labels.Labels) st } } builder.WriteString("}") - if vectorRange != "" { + if needsVectorRange { builder.WriteString("[") builder.WriteString(vectorRange) builder.WriteString("]") @@ -270,6 +311,7 @@ func (b *aggrChunkBuilder) encode() chunks.Meta { } } +// Checks if all series have the same length and time range. func aligned(series [5]*promql.Series) bool { for _, s := range series { if len(s.Floats) != len(series[0].Floats) { @@ -284,3 +326,46 @@ func aligned(series [5]*promql.Series) bool { } return true } + +// This is should be replaced by a queryable that can be directly created from an aggregated chunks block. +func aggrChunksBlockQueryable(ctx context.Context, logger log.Logger, dir string, meta *metadata.Meta) (storage.Queryable, error) { + bkt, err := filesystem.NewBucket(dir) + if err != nil { + return nil, err + } + + bs, err := store.NewBucketStore( + objstore.WithNoopInstr(bkt), + nil, + "", + store.NewChunksLimiterFactory(10000/store.MaxSamplesPerChunk), + store.NewSeriesLimiterFactory(0), + store.NewBytesLimiterFactory(0), + store.NewGapBasedPartitioner(store.PartitionerMaxGapSize), + 10, + false, + store.DefaultPostingOffsetInMemorySampling, + true, + false, + 0, + ) + if err != nil { + return nil, err + } + + if err := bs.AddBlock(ctx, meta); err != nil { + return nil, err + } + + return query.NewQueryableCreator(logger, nil, bs, 2, 30*time.Second)( + false, + nil, + nil, + 9999999, + false, + false, + false, + nil, + query.NoopSeriesStatsReporter, + ), nil +} diff --git a/pkg/compact/downsamplev2/downsample_test.go b/pkg/compact/downsamplev2/downsample_test.go index c73074d9be..12acc68dbd 100644 --- a/pkg/compact/downsamplev2/downsample_test.go +++ b/pkg/compact/downsamplev2/downsample_test.go @@ -32,8 +32,8 @@ func TestDownsample(t *testing.T) { {{Name: labels.MetricName, Value: "a"}, {Name: "a", Value: "1"}}, {{Name: labels.MetricName, Value: "a"}, {Name: "a", Value: "2"}}, }, - 9600, 0, downsample.ResLevel1DownsampleRange+1, // Pass the minimum ResLevel1DownsampleRange check. - labels.Labels{{Name: "e1", Value: "1"}}, + 9600, 0, downsample.ResLevel2DownsampleRange+1, // Pass the minimum ResLevel1DownsampleRange check. + labels.Labels{}, downsample.ResLevel0, metadata.NoneFunc) testutil.Ok(t, err) @@ -48,9 +48,29 @@ func TestDownsample(t *testing.T) { uid, err := Downsample(ctx, logger, meta, b, dir, downsample.ResLevel1) testutil.Ok(t, err) fmt.Println(uid) -} -func TestQueryString(t *testing.T) { - qs := queryString("increase", "5m", labels.FromStrings(labels.MetricName, "a", "a", "1")) - testutil.Equals(t, `increase({__name__="a",a="1"}[5m])`, qs) + res1bdir := fmt.Sprintf("%v/%v", dir, uid.String()) + res1b, err := tsdb.OpenBlock(logger, res1bdir, pool) + testutil.Ok(t, err) + res1meta, err := metadata.ReadFromDir(res1bdir) + testutil.Ok(t, err) + + res1uid, err := Downsample(ctx, logger, res1meta, res1b, dir, downsample.ResLevel2) + testutil.Ok(t, err) + fmt.Println(res1uid) + } + +//func TestQueryString(t *testing.T) { +// qs := queryString("increase", "5m", labels.FromStrings(labels.MetricName, "a", "a", "1")) +// testutil.Equals(t, `increase({__name__="a",a="1"}[5m])`, qs) +//} + +//func TestQueryDownsampled(t *testing.T) { +// bs, err := store.NewBucketStore( +// log.NewNopLogger(), +// ) +// +// bs. +// +//} diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index c533e79a8f..21ea982feb 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -558,7 +558,7 @@ func (s *BucketStore) SyncBlocks(ctx context.Context) error { wg.Add(1) go func() { for meta := range blockc { - if err := s.addBlock(ctx, meta); err != nil { + if err := s.AddBlock(ctx, meta); err != nil { continue } } @@ -654,7 +654,7 @@ func (s *BucketStore) getBlock(id ulid.ULID) *bucketBlock { return s.blocks[id] } -func (s *BucketStore) addBlock(ctx context.Context, meta *metadata.Meta) (err error) { +func (s *BucketStore) AddBlock(ctx context.Context, meta *metadata.Meta) (err error) { var dir string if s.dir != "" { dir = filepath.Join(s.dir, meta.ULID.String()) diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index 47f8e8e695..79405ba946 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -345,7 +345,7 @@ func TestBucketBlockSet_addGet(t *testing.T) { mint, maxt int64 window int64 } - // Input is expected to be sorted. It is sorted in addBlock. + // Input is expected to be sorted. It is sorted in AddBlock. input := []resBlock{ // Blocks from 0 to 100 with raw resolution. {window: downsample.ResLevel0, mint: 0, maxt: 100}, From 17564670144e588e9353eda0bcf7ffe422f82b3c Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Fri, 28 Jul 2023 19:03:56 +0200 Subject: [PATCH 8/9] Improved tests, fixed bugs --- pkg/compact/downsamplev2/downsample.go | 110 +++++++++----------- pkg/compact/downsamplev2/downsample_test.go | 74 ++++++++++--- 2 files changed, 107 insertions(+), 77 deletions(-) diff --git a/pkg/compact/downsamplev2/downsample.go b/pkg/compact/downsamplev2/downsample.go index 0d2e927e85..d45767c9b9 100644 --- a/pkg/compact/downsamplev2/downsample.go +++ b/pkg/compact/downsamplev2/downsample.go @@ -36,12 +36,12 @@ const ( ) var ( - promAggregators = map[downsample.AggrType][2]string{ - downsample.AggrCount: {"count_over_time", "true"}, - downsample.AggrSum: {"sum", ""}, - downsample.AggrMin: {"min", ""}, - downsample.AggrMax: {"max", ""}, - downsample.AggrCounter: {"increase", "true"}, + promAggregators = map[downsample.AggrType]string{ + downsample.AggrCount: "count_over_time", + downsample.AggrSum: "sum", + downsample.AggrMin: "min", + downsample.AggrMax: "max", + downsample.AggrCounter: "increase", } ) @@ -103,6 +103,16 @@ func Downsample( } defer runutil.CloseWithErrCapture(&err, streamedBlockWriter, "close stream block writer") + // Create new promql engine. + opts := promql.EngineOpts{ + Logger: logger, + Reg: nil, + MaxSamples: math.MaxInt32, + Timeout: 30 * time.Second, + LookbackDelta: time.Hour, + } + ng := engine.New(engine.Opts{EngineOpts: opts}) + // Downsample raw block. if origMeta.Thanos.Downsample.Resolution == 0 { q, err := tsdb.NewBlockQuerier(b, b.Meta().MinTime, b.Meta().MaxTime) @@ -113,19 +123,9 @@ func Downsample( // Get all series from the block. seriesSet := q.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, labels.MetricName, ".+")) - // Create new promql engine. - opts := promql.EngineOpts{ - Logger: logger, - Reg: nil, - MaxSamples: math.MaxInt32, - Timeout: 30 * time.Second, - LookbackDelta: 5 * time.Minute, - } - ng := engine.New(engine.Opts{EngineOpts: opts}) - for seriesSet.Next() { lset := seriesSet.At().Labels() - chks, err := downsampleRawSeries(ctx, &blockQuerier{q: q}, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime), 5*time.Minute) + chks, err := aggrChks(ctx, &blockQuerier{q: q}, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime), 5*time.Minute) if err != nil { return id, errors.Wrapf(err, "downsample series: %v", lset.String()) } @@ -139,16 +139,6 @@ func Downsample( return id, errors.Wrap(err, "create queryable for aggregated block") } - // Create new promql engine. - opts := promql.EngineOpts{ - Logger: logger, - Reg: nil, - MaxSamples: math.MaxInt32, - Timeout: 30 * time.Second, - LookbackDelta: time.Hour, - } - ng := engine.New(engine.Opts{EngineOpts: opts}) - qq, err := q.Querier(ctx, b.Meta().MinTime, b.Meta().MaxTime) if err != nil { return id, errors.Wrap(err, "create querier for aggregated block") @@ -158,7 +148,7 @@ func Downsample( for seriesSet.Next() { lset := seriesSet.At().Labels() - chks, err := downsampleRawSeries(ctx, q, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime), time.Hour) + chks, err := aggrChks(ctx, q, ng, lset, time.UnixMilli(b.Meta().MinTime), time.UnixMilli(b.Meta().MaxTime), time.Hour) if err != nil { return id, errors.Wrapf(err, "downsample series: %v", lset.String()) } @@ -171,7 +161,7 @@ func Downsample( return uid, nil } -func downsampleRawSeries( +func aggrChks( ctx context.Context, q storage.Queryable, ng v1.QueryEngine, @@ -183,18 +173,15 @@ func downsampleRawSeries( aggrSeries [5]*promql.Series err error ) - vectorRange := "5m" - if resolution > 5*time.Minute { - vectorRange = "1h" - } - for i := 0; i <= int(downsample.AggrCounter); i++ { - aggrSeries[i], err = querySingleSeries(ctx, ng, q, queryString(promAggregators[downsample.AggrType(i)][0], promAggregators[downsample.AggrType(i)][1] != "", vectorRange, lset), bMint, bMaxt, resolution) + + for _, aggrType := range []downsample.AggrType{downsample.AggrCount, downsample.AggrSum, downsample.AggrMin, downsample.AggrMax, downsample.AggrCounter} { + aggrSeries[aggrType], err = queryAggr(ctx, ng, q, lset, aggrType, bMint, bMaxt, resolution) if err != nil { return nil, err } // TODO: query first value of counter - if i == int(downsample.AggrCounter) { - aggrSeries[i].Floats = append([]promql.FPoint{{T: 0, F: 0}}, aggrSeries[i].Floats...) + if aggrType == downsample.AggrCounter { + aggrSeries[aggrType].Floats = append([]promql.FPoint{{T: 0, F: 0}}, aggrSeries[aggrType].Floats...) } } @@ -215,37 +202,17 @@ func downsampleRawSeries( ab.add(int64(i), aggrSeries) counter++ } - aggrChunks = append(aggrChunks, newAggrChunkBuilder().encode()) + aggrChunks = append(aggrChunks, ab.encode()) return aggrChunks, nil } -func queryString(aggregator string, needsVectorRange bool, vectorRange string, series labels.Labels) string { - builder := strings.Builder{} - builder.WriteString(aggregator) - builder.WriteString("({") - for i, l := range series { - builder.WriteString(fmt.Sprintf("%s=\"%s\"", l.Name, l.Value)) - if i != len(series)-1 { - builder.WriteString(",") - } - } - builder.WriteString("}") - if needsVectorRange { - builder.WriteString("[") - builder.WriteString(vectorRange) - builder.WriteString("]") - } - builder.WriteString(")") - return builder.String() -} - -func querySingleSeries(ctx context.Context, ng v1.QueryEngine, q storage.Queryable, qs string, mint, maxt time.Time, step time.Duration) (*promql.Series, error) { - query, err := ng.NewRangeQuery(ctx, q, &promql.QueryOpts{}, qs, mint, maxt, step) +func queryAggr(ctx context.Context, ng v1.QueryEngine, q storage.Queryable, lset labels.Labels, aggrType downsample.AggrType, mint, maxt time.Time, step time.Duration) (*promql.Series, error) { + sq, err := ng.NewRangeQuery(ctx, q, &promql.QueryOpts{}, queryString(aggrType, step, lset), mint, maxt, step) if err != nil { return nil, err } - sqres := query.Exec(ctx) + sqres := sq.Exec(ctx) if sqres.Err != nil { return nil, sqres.Err } @@ -270,6 +237,24 @@ type aggrChunkBuilder struct { apps [5]chunkenc.Appender } +func queryString(aggrType downsample.AggrType, vectorRange time.Duration, series labels.Labels) string { + builder := strings.Builder{} + builder.WriteString(promAggregators[aggrType]) + builder.WriteString("({") + for i, l := range series { + builder.WriteString(fmt.Sprintf("%s=\"%s\"", l.Name, l.Value)) + if i != len(series)-1 { + builder.WriteString(",") + } + } + builder.WriteString("}") + if aggrType == downsample.AggrCounter || aggrType == downsample.AggrCount { + _, _ = fmt.Fprintf(&builder, "[%s]", vectorRange) + } + builder.WriteString(")") + return builder.String() +} + func newAggrChunkBuilder() *aggrChunkBuilder { b := &aggrChunkBuilder{ mint: math.MaxInt64, @@ -327,7 +312,8 @@ func aligned(series [5]*promql.Series) bool { return true } -// This is should be replaced by a queryable that can be directly created from an aggregated chunks block. +// aggrChunksBlockQueryable creates a queryable for an aggregated chunks block in the give directory. +// TODO: Replaced by a queryable that can be directly created from an aggregated chunks block. func aggrChunksBlockQueryable(ctx context.Context, logger log.Logger, dir string, meta *metadata.Meta) (storage.Queryable, error) { bkt, err := filesystem.NewBucket(dir) if err != nil { diff --git a/pkg/compact/downsamplev2/downsample_test.go b/pkg/compact/downsamplev2/downsample_test.go index 12acc68dbd..9a23733f3e 100644 --- a/pkg/compact/downsamplev2/downsample_test.go +++ b/pkg/compact/downsamplev2/downsample_test.go @@ -3,7 +3,10 @@ package downsamplev2 import ( "context" "fmt" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" "os" + "path/filepath" "testing" "time" @@ -18,7 +21,7 @@ import ( "github.com/prometheus/prometheus/tsdb/chunkenc" ) -func TestDownsample(t *testing.T) { +func TestQueryDownsample(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second) defer cancel() @@ -57,20 +60,61 @@ func TestDownsample(t *testing.T) { res1uid, err := Downsample(ctx, logger, res1meta, res1b, dir, downsample.ResLevel2) testutil.Ok(t, err) - fmt.Println(res1uid) + + res1uidBlockDir := filepath.Join(dir, res1uid.String()) + bRes1uid, err := tsdb.OpenBlock(logger, res1uidBlockDir, pool) + testutil.Ok(t, err) + + indexr, err := bRes1uid.Index() + testutil.Ok(t, err) + postings, err := indexr.Postings(index.AllPostingsKey()) + + expectedLabels := []labels.Labels{ + {{Name: labels.MetricName, Value: "a"}, {Name: "a", Value: "1"}}, + {{Name: labels.MetricName, Value: "a"}, {Name: "a", Value: "2"}}, + } + + i := 0 + for postings.Next() { + var ( + chks []chunks.Meta + builder labels.ScratchBuilder + ) + + testutil.Ok(t, indexr.Series(postings.At(), &builder, &chks)) + lset := builder.Labels() + testutil.Equals(t, expectedLabels[i], lset) + testutil.Equals(t, int64(downsample.ResLevel2DownsampleRange), chks[len(chks)-1].MaxTime) + // TODO check expected samples. + i++ + } } -//func TestQueryString(t *testing.T) { -// qs := queryString("increase", "5m", labels.FromStrings(labels.MetricName, "a", "a", "1")) -// testutil.Equals(t, `increase({__name__="a",a="1"}[5m])`, qs) -//} - -//func TestQueryDownsampled(t *testing.T) { -// bs, err := store.NewBucketStore( -// log.NewNopLogger(), -// ) -// -// bs. -// -//} +func TestQueryString(t *testing.T) { + tt := []struct { + aggrType downsample.AggrType + res time.Duration + lset labels.Labels + qs string + }{ + { + aggrType: downsample.AggrCounter, + res: 5 * time.Minute, + lset: labels.FromStrings(labels.MetricName, "a", "a", "1"), + qs: `increase({__name__="a",a="1"}[5m0s])`, + }, { + aggrType: downsample.AggrSum, + res: 5 * time.Minute, + lset: labels.FromStrings(labels.MetricName, "a", "a", "1"), + qs: `sum({__name__="a",a="1"})`, + }, + } + + for _, tc := range tt { + t.Run(tc.qs, func(t *testing.T) { + qs := queryString(tc.aggrType, tc.res, tc.lset) + testutil.Equals(t, tc.qs, qs) + }) + } +} From d6f1d123dc8695dfdba6eb952e42bcca4952696b Mon Sep 17 00:00:00 2001 From: Sebastian Rabenhorst Date: Mon, 31 Jul 2023 11:19:30 +0200 Subject: [PATCH 9/9] Fixed imports --- pkg/compact/downsamplev2/downsample.go | 14 ++++++++------ pkg/compact/downsamplev2/downsample_test.go | 12 ++++++------ 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/pkg/compact/downsamplev2/downsample.go b/pkg/compact/downsamplev2/downsample.go index d45767c9b9..1dfffbbb19 100644 --- a/pkg/compact/downsamplev2/downsample.go +++ b/pkg/compact/downsamplev2/downsample.go @@ -3,6 +3,13 @@ package downsamplev2 import ( "context" "fmt" + "math" + "math/rand" + "os" + "path/filepath" + "strings" + "time" + "github.com/go-kit/log" "github.com/oklog/ulid" "github.com/pkg/errors" @@ -17,18 +24,13 @@ import ( "github.com/thanos-io/objstore" "github.com/thanos-io/objstore/providers/filesystem" "github.com/thanos-io/promql-engine/engine" + "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/compact/downsample" "github.com/thanos-io/thanos/pkg/errutil" "github.com/thanos-io/thanos/pkg/query" "github.com/thanos-io/thanos/pkg/runutil" "github.com/thanos-io/thanos/pkg/store" - "math" - "math/rand" - "os" - "path/filepath" - "strings" - "time" ) const ( diff --git a/pkg/compact/downsamplev2/downsample_test.go b/pkg/compact/downsamplev2/downsample_test.go index 9a23733f3e..a74794ff77 100644 --- a/pkg/compact/downsamplev2/downsample_test.go +++ b/pkg/compact/downsamplev2/downsample_test.go @@ -3,22 +3,22 @@ package downsamplev2 import ( "context" "fmt" - "github.com/prometheus/prometheus/tsdb/chunks" - "github.com/prometheus/prometheus/tsdb/index" "os" "path/filepath" "testing" "time" - "github.com/thanos-io/thanos/pkg/block/metadata" - "github.com/thanos-io/thanos/pkg/compact/downsample" - "github.com/thanos-io/thanos/pkg/testutil/e2eutil" - "github.com/efficientgo/core/testutil" "github.com/go-kit/log" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" + + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/compact/downsample" + "github.com/thanos-io/thanos/pkg/testutil/e2eutil" ) func TestQueryDownsample(t *testing.T) {