From c8b267efd6159c0f76608fffeeacf4f030fa1053 Mon Sep 17 00:00:00 2001 From: Ganesh Vernekar Date: Tue, 2 Nov 2021 20:31:32 +0530 Subject: [PATCH 1/7] Get histograms from TSDB to the rate() function implementation Signed-off-by: Ganesh Vernekar --- promql/engine.go | 63 ++++++++++++++++++++++++--------- promql/engine_test.go | 32 +++++++++++++++++ promql/test_test.go | 10 +++--- promql/value.go | 3 ++ rules/manager.go | 2 +- storage/buffer.go | 77 +++++++++++++++++++++++++++++++---------- storage/series.go | 23 +++++++----- tsdb/compact_test.go | 6 ++-- tsdb/head.go | 23 +++++++++++- tsdb/head_read.go | 41 +++++++++++++++------- tsdb/head_test.go | 31 ++++------------- tsdb/tsdbutil/buffer.go | 5 +++ tsdb/tsdbutil/chunks.go | 4 ++- 13 files changed, 229 insertions(+), 91 deletions(-) diff --git a/promql/engine.go b/promql/engine.go index bd3d836a3..be7913a81 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -40,6 +40,7 @@ import ( "github.com/prometheus/prometheus/pkg/value" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/stats" ) @@ -1735,29 +1736,57 @@ func (ev *evaluator) matrixIterSlice(it *storage.BufferedSeriesIterator, mint, m } buf := it.Buffer() - for buf.Next() { - t, v := buf.At() - if value.IsStaleNaN(v) { - continue - } - // Values in the buffer are guaranteed to be smaller than maxt. - if t >= mint { - if ev.currentSamples >= ev.maxSamples { - ev.error(ErrTooManySamples(env)) + if it.ChunkEncoding() == chunkenc.EncHistogram { + for buf.Next() { + t, h := buf.AtHistogram() + if value.IsStaleNaN(h.Sum) { + continue + } + // Values in the buffer are guaranteed to be smaller than maxt. + if t >= mint { + if ev.currentSamples >= ev.maxSamples { + ev.error(ErrTooManySamples(env)) + } + ev.currentSamples++ + out = append(out, Point{T: t, H: &h}) + } + } + } else { + for buf.Next() { + t, v := buf.At() + if value.IsStaleNaN(v) { + continue + } + // Values in the buffer are guaranteed to be smaller than maxt. + if t >= mint { + if ev.currentSamples >= ev.maxSamples { + ev.error(ErrTooManySamples(env)) + } + ev.currentSamples++ + out = append(out, Point{T: t, V: v}) } - ev.currentSamples++ - out = append(out, Point{T: t, V: v}) } } // The seeked sample might also be in the range. if ok { - t, v := it.Values() - if t == maxt && !value.IsStaleNaN(v) { - if ev.currentSamples >= ev.maxSamples { - ev.error(ErrTooManySamples(env)) + if it.ChunkEncoding() == chunkenc.EncHistogram { + t, h := it.HistogramValues() + if t == maxt && !value.IsStaleNaN(h.Sum) { + if ev.currentSamples >= ev.maxSamples { + ev.error(ErrTooManySamples(env)) + } + out = append(out, Point{T: t, H: &h}) + ev.currentSamples++ + } + } else { + t, v := it.Values() + if t == maxt && !value.IsStaleNaN(v) { + if ev.currentSamples >= ev.maxSamples { + ev.error(ErrTooManySamples(env)) + } + out = append(out, Point{T: t, V: v}) + ev.currentSamples++ } - out = append(out, Point{T: t, V: v}) - ev.currentSamples++ } } return out diff --git a/promql/engine_test.go b/promql/engine_test.go index c4eb07d52..88eed7f8b 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -16,6 +16,7 @@ package promql import ( "context" "errors" + "fmt" "io/ioutil" "os" "sort" @@ -30,6 +31,7 @@ import ( "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" ) func TestMain(m *testing.M) { @@ -2429,3 +2431,33 @@ func TestRangeQuery(t *testing.T) { }) } } + +func TestSparseHistogramRate(t *testing.T) { + // Currently, this test it to only find panics or errors in the engine execution path. + // The panic stack trace will mostly tell you what code path is breaking and needs fixing for + // fetching the raw histograms and passing it rightly upto the rate() function implementation. + // TODO: Check the result for correctness once implementation is ready. + + test, err := NewTest(t, "") + require.NoError(t, err) + defer test.Close() + + seriesName := "sparse_histogram_series" + lbls := labels.FromStrings("__name__", seriesName) + + app := test.Storage().Appender(context.TODO()) + for i, h := range tsdb.GenerateTestHistograms(100) { + _, err := app.AppendHistogram(0, lbls, int64(i)*int64(15*time.Second/time.Millisecond), h) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + require.NoError(t, test.Run()) + engine := test.QueryEngine() + + queryString := fmt.Sprintf("rate(%s[1m])", seriesName) + qry, err := engine.NewInstantQuery(test.Queryable(), queryString, timestamp.Time(int64(5*time.Minute/time.Millisecond))) + require.NoError(t, err) + res := qry.Exec(test.Context()) + require.NoError(t, res.Err) +} diff --git a/promql/test_test.go b/promql/test_test.go index ec50e5753..ec2bac1b1 100644 --- a/promql/test_test.go +++ b/promql/test_test.go @@ -47,7 +47,7 @@ func TestLazyLoader_WithSamplesTill(t *testing.T) { { Metric: labels.FromStrings("__name__", "metric1"), Points: []Point{ - {0, 1}, {10000, 2}, {20000, 3}, {30000, 4}, {40000, 5}, + {0, 1, nil}, {10000, 2, nil}, {20000, 3, nil}, {30000, 4, nil}, {40000, 5, nil}, }, }, }, @@ -58,7 +58,7 @@ func TestLazyLoader_WithSamplesTill(t *testing.T) { { Metric: labels.FromStrings("__name__", "metric1"), Points: []Point{ - {0, 1}, {10000, 2}, {20000, 3}, {30000, 4}, {40000, 5}, + {0, 1, nil}, {10000, 2, nil}, {20000, 3, nil}, {30000, 4, nil}, {40000, 5, nil}, }, }, }, @@ -69,7 +69,7 @@ func TestLazyLoader_WithSamplesTill(t *testing.T) { { Metric: labels.FromStrings("__name__", "metric1"), Points: []Point{ - {0, 1}, {10000, 2}, {20000, 3}, {30000, 4}, {40000, 5}, {50000, 6}, {60000, 7}, + {0, 1, nil}, {10000, 2, nil}, {20000, 3, nil}, {30000, 4, nil}, {40000, 5, nil}, {50000, 6, nil}, {60000, 7, nil}, }, }, }, @@ -89,13 +89,13 @@ func TestLazyLoader_WithSamplesTill(t *testing.T) { { Metric: labels.FromStrings("__name__", "metric1"), Points: []Point{ - {0, 1}, {10000, 1}, {20000, 1}, {30000, 1}, {40000, 1}, {50000, 1}, + {0, 1, nil}, {10000, 1, nil}, {20000, 1, nil}, {30000, 1, nil}, {40000, 1, nil}, {50000, 1, nil}, }, }, { Metric: labels.FromStrings("__name__", "metric2"), Points: []Point{ - {0, 1}, {10000, 2}, {20000, 3}, {30000, 4}, {40000, 5}, {50000, 6}, {60000, 7}, {70000, 8}, + {0, 1, nil}, {10000, 2, nil}, {20000, 3, nil}, {30000, 4, nil}, {40000, 5, nil}, {50000, 6, nil}, {60000, 7, nil}, {70000, 8, nil}, }, }, }, diff --git a/promql/value.go b/promql/value.go index 3a724d053..63590042b 100644 --- a/promql/value.go +++ b/promql/value.go @@ -78,9 +78,12 @@ func (s Series) String() string { } // Point represents a single data point for a given timestamp. +// If H is not nil, then this is a histogram point and only (T, H) is valid. +// If H is nil, then only (T, V) is valid. type Point struct { T int64 V float64 + H *histogram.Histogram } func (p Point) String() string { diff --git a/rules/manager.go b/rules/manager.go index fa8cd6763..a7863a60e 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -197,7 +197,7 @@ func EngineQueryFunc(engine *promql.Engine, q storage.Queryable) QueryFunc { return v, nil case promql.Scalar: return promql.Vector{promql.Sample{ - Point: promql.Point(v), + Point: promql.Point{T: v.T, V: v.V}, Metric: labels.Labels{}, }}, nil default: diff --git a/storage/buffer.go b/storage/buffer.go index 2d06d3fb2..45bfe0645 100644 --- a/storage/buffer.go +++ b/storage/buffer.go @@ -40,8 +40,9 @@ func NewBuffer(delta int64) *BufferedSeriesIterator { // NewBufferIterator returns a new iterator that buffers the values within the // time range of the current element and the duration of delta before. func NewBufferIterator(it chunkenc.Iterator, delta int64) *BufferedSeriesIterator { + // TODO(codesome): based on encoding, allocate different buffer. bit := &BufferedSeriesIterator{ - buf: newSampleRing(delta, 16), + buf: newSampleRing(delta, 16, it.ChunkEncoding()), delta: delta, } bit.Reset(it) @@ -67,8 +68,9 @@ func (b *BufferedSeriesIterator) ReduceDelta(delta int64) bool { // PeekBack returns the nth previous element of the iterator. If there is none buffered, // ok is false. -func (b *BufferedSeriesIterator) PeekBack(n int) (t int64, v float64, ok bool) { - return b.buf.nthLast(n) +func (b *BufferedSeriesIterator) PeekBack(n int) (t int64, v float64, h *histogram.Histogram, ok bool) { + s, ok := b.buf.nthLast(n) + return s.t, s.v, s.h, ok } // Buffer returns an iterator over the buffered data. Invalidates previously @@ -90,7 +92,11 @@ func (b *BufferedSeriesIterator) Seek(t int64) bool { if !b.ok { return false } - b.lastTime, _ = b.Values() + if b.it.ChunkEncoding() == chunkenc.EncHistogram { + b.lastTime, _ = b.HistogramValues() + } else { + b.lastTime, _ = b.Values() + } } if b.lastTime >= t { @@ -112,11 +118,21 @@ func (b *BufferedSeriesIterator) Next() bool { } // Add current element to buffer before advancing. - b.buf.add(b.it.At()) + if b.it.ChunkEncoding() == chunkenc.EncHistogram { + t, h := b.it.AtHistogram() + b.buf.add(sample{t: t, h: &h}) + } else { + t, v := b.it.At() + b.buf.add(sample{t: t, v: v}) + } b.ok = b.it.Next() if b.ok { - b.lastTime, _ = b.Values() + if b.it.ChunkEncoding() == chunkenc.EncHistogram { + b.lastTime, _ = b.HistogramValues() + } else { + b.lastTime, _ = b.Values() + } } return b.ok @@ -127,6 +143,16 @@ func (b *BufferedSeriesIterator) Values() (int64, float64) { return b.it.At() } +// HistogramValues returns the current histogram element of the iterator. +func (b *BufferedSeriesIterator) HistogramValues() (int64, histogram.Histogram) { + return b.it.AtHistogram() +} + +// ChunkEncoding return the chunk encoding of the underlying iterator. +func (b *BufferedSeriesIterator) ChunkEncoding() chunkenc.Encoding { + return b.it.ChunkEncoding() +} + // Err returns the last encountered error. func (b *BufferedSeriesIterator) Err() error { return b.it.Err() @@ -135,6 +161,7 @@ func (b *BufferedSeriesIterator) Err() error { type sample struct { t int64 v float64 + h *histogram.Histogram } func (s sample) T() int64 { @@ -145,9 +172,14 @@ func (s sample) V() float64 { return s.v } +func (s sample) H() *histogram.Histogram { + return s.h +} + type sampleRing struct { delta int64 + enc chunkenc.Encoding buf []sample // lookback buffer i int // position of most recent element in ring buffer f int // position of first element in ring buffer @@ -156,8 +188,8 @@ type sampleRing struct { it sampleRingIterator } -func newSampleRing(delta int64, sz int) *sampleRing { - r := &sampleRing{delta: delta, buf: make([]sample, sz)} +func newSampleRing(delta int64, sz int, enc chunkenc.Encoding) *sampleRing { + r := &sampleRing{delta: delta, buf: make([]sample, sz), enc: enc} r.reset() return r @@ -200,13 +232,12 @@ func (it *sampleRingIterator) At() (int64, float64) { // AtHistogram always returns (0, histogram.Histogram{}) because there is no // support for histogram values yet. -// TODO(beorn7): Fix that for histogram support in PromQL. func (it *sampleRingIterator) AtHistogram() (int64, histogram.Histogram) { - return 0, histogram.Histogram{} + return it.r.atHistogram(it.i) } func (it *sampleRingIterator) ChunkEncoding() chunkenc.Encoding { - return chunkenc.EncXOR + return it.r.enc } func (r *sampleRing) at(i int) (int64, float64) { @@ -215,9 +246,20 @@ func (r *sampleRing) at(i int) (int64, float64) { return s.t, s.v } +func (r *sampleRing) atHistogram(i int) (int64, histogram.Histogram) { + j := (r.f + i) % len(r.buf) + s := r.buf[j] + return s.t, *s.h +} + +func (r *sampleRing) atSample(i int) sample { + j := (r.f + i) % len(r.buf) + return r.buf[j] +} + // add adds a sample to the ring buffer and frees all samples that fall // out of the delta range. -func (r *sampleRing) add(t int64, v float64) { +func (r *sampleRing) add(s sample) { l := len(r.buf) // Grow the ring buffer if it fits no more elements. if l == r.l { @@ -236,11 +278,11 @@ func (r *sampleRing) add(t int64, v float64) { } } - r.buf[r.i] = sample{t: t, v: v} + r.buf[r.i] = s r.l++ // Free head of the buffer of samples that just fell out of the range. - tmin := t - r.delta + tmin := s.t - r.delta for r.buf[r.f].t < tmin { r.f++ if r.f >= l { @@ -276,12 +318,11 @@ func (r *sampleRing) reduceDelta(delta int64) bool { } // nthLast returns the nth most recent element added to the ring. -func (r *sampleRing) nthLast(n int) (int64, float64, bool) { +func (r *sampleRing) nthLast(n int) (sample, bool) { if n > r.l { - return 0, 0, false + return sample{}, false } - t, v := r.at(r.l - n) - return t, v, true + return r.atSample(r.l - n), true } func (r *sampleRing) samples() []sample { diff --git a/storage/series.go b/storage/series.go index 60541164b..d597a3b90 100644 --- a/storage/series.go +++ b/storage/series.go @@ -297,19 +297,26 @@ func (e errChunksIterator) Err() error { return e.err } // ExpandSamples iterates over all samples in the iterator, buffering all in slice. // Optionally it takes samples constructor, useful when you want to compare sample slices with different // sample implementations. if nil, sample type from this package will be used. -func ExpandSamples(iter chunkenc.Iterator, newSampleFn func(t int64, v float64) tsdbutil.Sample) ([]tsdbutil.Sample, error) { +func ExpandSamples(iter chunkenc.Iterator, newSampleFn func(t int64, v float64, h *histogram.Histogram) tsdbutil.Sample) ([]tsdbutil.Sample, error) { if newSampleFn == nil { - newSampleFn = func(t int64, v float64) tsdbutil.Sample { return sample{t, v} } + newSampleFn = func(t int64, v float64, h *histogram.Histogram) tsdbutil.Sample { return sample{t, v, h} } } var result []tsdbutil.Sample - for iter.Next() { - t, v := iter.At() - // NaNs can't be compared normally, so substitute for another value. - if math.IsNaN(v) { - v = -42 + if iter.ChunkEncoding() == chunkenc.EncHistogram { + for iter.Next() { + t, h := iter.AtHistogram() + result = append(result, newSampleFn(t, 0, &h)) + } + } else { + for iter.Next() { + t, v := iter.At() + // NaNs can't be compared normally, so substitute for another value. + if math.IsNaN(v) { + v = -42 + } + result = append(result, newSampleFn(t, v, nil)) } - result = append(result, newSampleFn(t, v)) } return result, iter.Err() } diff --git a/tsdb/compact_test.go b/tsdb/compact_test.go index e9ac42077..5f3d2374b 100644 --- a/tsdb/compact_test.go +++ b/tsdb/compact_test.go @@ -1336,7 +1336,7 @@ func TestHeadCompactionWithHistograms(t *testing.T) { timeStep := DefaultBlockDuration / int64(numHistograms) expHists := make([]timedHist, 0, numHistograms) l := labels.Labels{{Name: "a", Value: "b"}} - for i, h := range generateHistograms(numHistograms) { + for i, h := range GenerateTestHistograms(numHistograms) { _, err := app.AppendHistogram(0, l, int64(i)*timeStep, h) require.NoError(t, err) expHists = append(expHists, timedHist{int64(i) * timeStep, h}) @@ -1715,8 +1715,8 @@ func TestSparseHistogramCompactionAndQuery(t *testing.T) { } expHists := make(map[string][]timedHist) - series1Histograms := generateHistograms(20) - series2Histograms := generateHistograms(20) + series1Histograms := GenerateTestHistograms(20) + series2Histograms := GenerateTestHistograms(20) idx1, idx2 := -1, -1 addNextHists := func(ts int64, app storage.Appender) { lbls1 := labels.Labels{{Name: "a", Value: "b"}} diff --git a/tsdb/head.go b/tsdb/head.go index 989049d27..93e6d568b 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -1496,11 +1496,13 @@ type histogramSample struct { type sample struct { t int64 v float64 + h *histogram.Histogram } -func newSample(t int64, v float64) tsdbutil.Sample { return sample{t, v} } +func newSample(t int64, v float64) tsdbutil.Sample { return sample{t, v, nil} } func (s sample) T() int64 { return s.t } func (s sample) V() float64 { return s.v } +func (s sample) H() *histogram.Histogram { return s.h } // memSeries is the in-memory representation of a series. None of its methods // are goroutine safe and it is the caller's responsibility to lock it. @@ -1658,3 +1660,22 @@ func (h *Head) updateWALReplayStatusRead(current int) { h.stats.WALReplayStatus.Current = current } + +func GenerateTestHistograms(n int) (r []histogram.Histogram) { + for i := 0; i < n; i++ { + r = append(r, histogram.Histogram{ + Count: 5 + uint64(i*4), + ZeroCount: 2 + uint64(i), + ZeroThreshold: 0.001, + Sum: 18.4 * float64(i+1), + Schema: 1, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []int64{int64(i + 1), 1, -1, 0}, + }) + } + + return r +} diff --git a/tsdb/head_read.go b/tsdb/head_read.go index c9d496f78..7c20c43ad 100644 --- a/tsdb/head_read.go +++ b/tsdb/head_read.go @@ -444,6 +444,7 @@ func (s *memSeries) iterator(id int, isoState *isolationState, chunkDiskMapper * msIter.stopAfter = stopAfter msIter.buf = s.sampleBuf msIter.histogramBuf = s.histogramBuf + msIter.histogramSeries = s.histogramSeries return msIter } return &memSafeIterator{ @@ -452,18 +453,20 @@ func (s *memSeries) iterator(id int, isoState *isolationState, chunkDiskMapper * i: -1, stopAfter: stopAfter, }, - total: numSamples, - buf: s.sampleBuf, - histogramBuf: s.histogramBuf, + total: numSamples, + buf: s.sampleBuf, + histogramBuf: s.histogramBuf, + histogramSeries: s.histogramSeries, } } type memSafeIterator struct { stopIterator - total int - buf [4]sample - histogramBuf [4]histogramSample + histogramSeries bool + total int + buf [4]sample + histogramBuf [4]histogramSample } func (it *memSafeIterator) Seek(t int64) bool { @@ -471,15 +474,29 @@ func (it *memSafeIterator) Seek(t int64) bool { return false } - ts, _ := it.At() - - for t > ts || it.i == -1 { - if !it.Next() { - return false - } + var ts int64 + if it.histogramSeries { + ts, _ = it.AtHistogram() + } else { ts, _ = it.At() } + if it.histogramSeries { + for t > ts || it.i == -1 { + if !it.Next() { + return false + } + ts, _ = it.AtHistogram() + } + } else { + for t > ts || it.i == -1 { + if !it.Next() { + return false + } + ts, _ = it.At() + } + } + return true } diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 4eac434d3..7831ad9d8 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -2544,7 +2544,7 @@ func TestAppendHistogram(t *testing.T) { h histogram.Histogram } expHistograms := make([]timedHistogram, 0, numHistograms) - for i, h := range generateHistograms(numHistograms) { + for i, h := range GenerateTestHistograms(numHistograms) { _, err := app.AppendHistogram(0, l, int64(i), h) require.NoError(t, err) expHistograms = append(expHistograms, timedHistogram{int64(i), h}) @@ -2591,7 +2591,7 @@ func TestHistogramInWAL(t *testing.T) { h histogram.Histogram } expHistograms := make([]timedHistogram, 0, numHistograms) - for i, h := range generateHistograms(numHistograms) { + for i, h := range GenerateTestHistograms(numHistograms) { h.NegativeSpans = h.PositiveSpans h.NegativeBuckets = h.PositiveBuckets _, err := app.AppendHistogram(0, l, int64(i), h) @@ -2630,25 +2630,6 @@ func TestHistogramInWAL(t *testing.T) { require.Equal(t, expHistograms, actHistograms) } -func generateHistograms(n int) (r []histogram.Histogram) { - for i := 0; i < n; i++ { - r = append(r, histogram.Histogram{ - Count: 5 + uint64(i*4), - ZeroCount: 2 + uint64(i), - ZeroThreshold: 0.001, - Sum: 18.4 * float64(i+1), - Schema: 1, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 2}, - {Offset: 1, Length: 2}, - }, - PositiveBuckets: []int64{int64(i + 1), 1, -1, 0}, - }) - } - - return r -} - func TestChunkSnapshot(t *testing.T) { head, _ := newTestHead(t, 120*4, false) defer func() { @@ -2962,7 +2943,7 @@ func TestHistogramMetrics(t *testing.T) { for x := 0; x < 5; x++ { expHSeries++ l := labels.Labels{{Name: "a", Value: fmt.Sprintf("b%d", x)}} - for i, h := range generateHistograms(10) { + for i, h := range GenerateTestHistograms(10) { app := head.Appender(context.Background()) _, err := app.AppendHistogram(0, l, int64(i), h) require.NoError(t, err) @@ -3039,7 +3020,7 @@ func TestHistogramStaleSample(t *testing.T) { // Adding stale in the same appender. app := head.Appender(context.Background()) - for _, h := range generateHistograms(numHistograms) { + for _, h := range GenerateTestHistograms(numHistograms) { _, err := app.AppendHistogram(0, l, 100*int64(len(expHistograms)), h) require.NoError(t, err) expHistograms = append(expHistograms, timedHistogram{100 * int64(len(expHistograms)), h}) @@ -3058,7 +3039,7 @@ func TestHistogramStaleSample(t *testing.T) { // Adding stale in different appender and continuing series after a stale sample. app = head.Appender(context.Background()) - for _, h := range generateHistograms(2 * numHistograms)[numHistograms:] { + for _, h := range GenerateTestHistograms(2 * numHistograms)[numHistograms:] { _, err := app.AppendHistogram(0, l, 100*int64(len(expHistograms)), h) require.NoError(t, err) expHistograms = append(expHistograms, timedHistogram{100 * int64(len(expHistograms)), h}) @@ -3112,7 +3093,7 @@ func TestHistogramCounterResetHeader(t *testing.T) { require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunk.chunk.(*chunkenc.HistogramChunk).GetCounterResetHeader()) } - h := generateHistograms(1)[0] + h := GenerateTestHistograms(1)[0] if len(h.NegativeBuckets) == 0 { h.NegativeSpans = append([]histogram.Span{}, h.PositiveSpans...) h.NegativeBuckets = append([]int64{}, h.PositiveBuckets...) diff --git a/tsdb/tsdbutil/buffer.go b/tsdb/tsdbutil/buffer.go index 2c9bbb10b..5dde74835 100644 --- a/tsdb/tsdbutil/buffer.go +++ b/tsdb/tsdbutil/buffer.go @@ -102,6 +102,7 @@ func (b *BufferedSeriesIterator) Err() error { type sample struct { t int64 v float64 + h *histogram.Histogram } func (s sample) T() int64 { @@ -112,6 +113,10 @@ func (s sample) V() float64 { return s.v } +func (s sample) H() *histogram.Histogram { + return s.h +} + type sampleRing struct { delta int64 diff --git a/tsdb/tsdbutil/chunks.go b/tsdb/tsdbutil/chunks.go index 5ae58b0a8..5b4e954ae 100644 --- a/tsdb/tsdbutil/chunks.go +++ b/tsdb/tsdbutil/chunks.go @@ -14,6 +14,7 @@ package tsdbutil import ( + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" ) @@ -26,6 +27,7 @@ type Samples interface { type Sample interface { T() int64 V() float64 + H() *histogram.Histogram } type SampleSlice []Sample @@ -61,7 +63,7 @@ func ChunkFromSamplesGeneric(s Samples) chunks.Meta { func PopulatedChunk(numSamples int, minTime int64) chunks.Meta { samples := make([]Sample, numSamples) for i := 0; i < numSamples; i++ { - samples[i] = sample{minTime + int64(i*1000), 1.0} + samples[i] = sample{t: minTime + int64(i*1000), v: 1.0} } return ChunkFromSamples(samples) } From 8f92c90897f4904f5322448c42b74e7ff8ccea18 Mon Sep 17 00:00:00 2001 From: beorn7 Date: Sun, 7 Nov 2021 17:12:04 +0100 Subject: [PATCH 2/7] Add TODOs and some minor tweaks Signed-off-by: beorn7 --- promql/engine.go | 2 +- promql/value.go | 7 +++---- storage/buffer.go | 2 -- storage/buffer_test.go | 4 ++-- storage/series.go | 5 ++--- tsdb/chunkenc/chunk.go | 3 +++ tsdb/tsdbutil/buffer.go | 1 + tsdb/tsdbutil/buffer_test.go | 6 +++++- 8 files changed, 17 insertions(+), 13 deletions(-) diff --git a/promql/engine.go b/promql/engine.go index be7913a81..f1c48f610 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -1767,7 +1767,7 @@ func (ev *evaluator) matrixIterSlice(it *storage.BufferedSeriesIterator, mint, m } } } - // The seeked sample might also be in the range. + // The sought sample might also be in the range. if ok { if it.ChunkEncoding() == chunkenc.EncHistogram { t, h := it.HistogramValues() diff --git a/promql/value.go b/promql/value.go index 63590042b..4f52b3199 100644 --- a/promql/value.go +++ b/promql/value.go @@ -87,6 +87,7 @@ type Point struct { } func (p Point) String() string { + // TODO(beorn7): Support Histogram. v := strconv.FormatFloat(p.V, 'f', -1, 64) return fmt.Sprintf("%v @[%v]", v, p.T) } @@ -299,11 +300,9 @@ func (ssi *storageSeriesIterator) At() (t int64, v float64) { return p.T, p.V } -// AtHistogram always returns (0, histogram.Histogram{}) because there is no -// support for histogram values yet. -// TODO(beorn7): Fix that for histogram support in PromQL. func (ssi *storageSeriesIterator) AtHistogram() (int64, histogram.Histogram) { - return 0, histogram.Histogram{} + p := ssi.points[ssi.curr] + return p.T, *p.H } func (ssi *storageSeriesIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/storage/buffer.go b/storage/buffer.go index 45bfe0645..93cd9da00 100644 --- a/storage/buffer.go +++ b/storage/buffer.go @@ -230,8 +230,6 @@ func (it *sampleRingIterator) At() (int64, float64) { return it.r.at(it.i) } -// AtHistogram always returns (0, histogram.Histogram{}) because there is no -// support for histogram values yet. func (it *sampleRingIterator) AtHistogram() (int64, histogram.Histogram) { return it.r.atHistogram(it.i) } diff --git a/storage/buffer_test.go b/storage/buffer_test.go index 032c61f07..2d7b3f1e9 100644 --- a/storage/buffer_test.go +++ b/storage/buffer_test.go @@ -216,11 +216,11 @@ func newFakeSeriesIterator(nsamples, step int64) *fakeSeriesIterator { } func (it *fakeSeriesIterator) At() (int64, float64) { - return it.idx * it.step, 123 // value doesn't matter + return it.idx * it.step, 123 // Value doesn't matter. } func (it *fakeSeriesIterator) AtHistogram() (int64, histogram.Histogram) { - return it.idx * it.step, histogram.Histogram{} // value doesn't matter + return it.idx * it.step, histogram.Histogram{} // Value doesn't matter. } func (it *fakeSeriesIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/storage/series.go b/storage/series.go index d597a3b90..6e5b178df 100644 --- a/storage/series.go +++ b/storage/series.go @@ -91,10 +91,9 @@ func (it *listSeriesIterator) At() (int64, float64) { return s.T(), s.V() } -// AtHistogram always returns (0, histogram.Histogram{}) because there is no -// support for histogram values yet. func (it *listSeriesIterator) AtHistogram() (int64, histogram.Histogram) { - return 0, histogram.Histogram{} + s := it.samples.Get(it.idx) + return s.T(), *s.H() } func (it *listSeriesIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/tsdb/chunkenc/chunk.go b/tsdb/chunkenc/chunk.go index c5133dc59..8069b0ecd 100644 --- a/tsdb/chunkenc/chunk.go +++ b/tsdb/chunkenc/chunk.go @@ -89,6 +89,8 @@ type Appender interface { // Iterator iterates over the samples of a time series, in timestamp-increasing order. type Iterator interface { // Next advances the iterator by one. + // TODO(beorn7): Perhaps this should return if the next value is a float or a histogram + // to make it easier calling the right method (At vs AtHistogram)? Next() bool // Seek advances the iterator forward to the first sample with the timestamp equal or greater than t. // If current sample found by previous `Next` or `Seek` operation already has this property, Seek has no effect. @@ -100,6 +102,7 @@ type Iterator interface { At() (int64, float64) // AtHistogram returns the current timestamp/histogram pair. // Before the iterator has advanced AtHistogram behaviour is unspecified. + // TODO(beorn7): Maybe return *histogram.Histogram? It's a fairly large struct. AtHistogram() (int64, histogram.Histogram) // Err returns the current error. It should be used only after iterator is // exhausted, that is `Next` or `Seek` returns false. diff --git a/tsdb/tsdbutil/buffer.go b/tsdb/tsdbutil/buffer.go index 5dde74835..5569c0047 100644 --- a/tsdb/tsdbutil/buffer.go +++ b/tsdb/tsdbutil/buffer.go @@ -166,6 +166,7 @@ func (it *sampleRingIterator) At() (int64, float64) { } func (it *sampleRingIterator) AtHistogram() (int64, histogram.Histogram) { + // TODO(beorn7): Add proper histogram support. return 0, histogram.Histogram{} } diff --git a/tsdb/tsdbutil/buffer_test.go b/tsdb/tsdbutil/buffer_test.go index 0401e6c87..8ff8fa086 100644 --- a/tsdb/tsdbutil/buffer_test.go +++ b/tsdb/tsdbutil/buffer_test.go @@ -153,7 +153,11 @@ func (it *listSeriesIterator) At() (int64, float64) { } func (it *listSeriesIterator) AtHistogram() (int64, histogram.Histogram) { - return 0, histogram.Histogram{} + s := it.list[it.idx] + if s.h == nil { + return s.t, histogram.Histogram{} + } + return s.t, *s.h } func (it *listSeriesIterator) ChunkEncoding() chunkenc.Encoding { From f1065e44a4241097c4c2a53c44793e80d3475cf1 Mon Sep 17 00:00:00 2001 From: beorn7 Date: Tue, 9 Nov 2021 21:45:04 +0100 Subject: [PATCH 3/7] model: String method for histogram.Histogram This includes a regular bucket iterator and a string method for histogram.Bucket. Signed-off-by: beorn7 --- model/histogram/histogram.go | 202 ++++++++++++++++-- model/histogram/histogram_test.go | 330 +++++++++++++++++++++++++----- promql/engine_test.go | 1 + tsdb/compact_test.go | 1 - 4 files changed, 458 insertions(+), 76 deletions(-) diff --git a/model/histogram/histogram.go b/model/histogram/histogram.go index 5a2ddea34..42ce8eb99 100644 --- a/model/histogram/histogram.go +++ b/model/histogram/histogram.go @@ -14,7 +14,9 @@ package histogram import ( + "fmt" "math" + "strings" ) // Histogram encodes a sparse, high-resolution histogram. See the design @@ -65,7 +67,7 @@ type Span struct { } // Copy returns a deep copy of the Histogram. -func (h Histogram) Copy() Histogram { +func (h Histogram) Copy() *Histogram { c := h if h.PositiveSpans != nil { @@ -85,7 +87,61 @@ func (h Histogram) Copy() Histogram { copy(c.NegativeBuckets, h.NegativeBuckets) } - return c + return &c +} + +// String returns a string representation of the Histogram. +func (h Histogram) String() string { + var sb strings.Builder + fmt.Fprintf(&sb, "{count:%d, sum:%g", h.Count, h.Sum) + + var nBuckets []Bucket + for it := h.NegativeBucketIterator(); it.Next(); { + bucket := it.At() + if bucket.Count != 0 { + nBuckets = append(nBuckets, it.At()) + } + } + for i := len(nBuckets) - 1; i >= 0; i-- { + fmt.Fprintf(&sb, ", %s", nBuckets[i].String()) + } + + if h.ZeroCount != 0 { + fmt.Fprintf(&sb, ", %s", h.ZeroBucket().String()) + } + + for it := h.PositiveBucketIterator(); it.Next(); { + bucket := it.At() + if bucket.Count != 0 { + fmt.Fprintf(&sb, ", %s", bucket.String()) + } + } + + sb.WriteRune('}') + return sb.String() +} + +// ZeroBucket returns the zero bucket. +func (h Histogram) ZeroBucket() Bucket { + return Bucket{ + Lower: -h.ZeroThreshold, + Upper: h.ZeroThreshold, + LowerInclusive: true, + UpperInclusive: true, + Count: h.ZeroCount, + } +} + +// PositiveBucketIterator returns a BucketIterator to iterate over all positive +// buckets in ascending order (starting next to the zero bucket and going up). +func (h Histogram) PositiveBucketIterator() BucketIterator { + return newRegularBucketIterator(&h, true) +} + +// NegativeBucketIterator returns a BucketIterator to iterate over all negative +// buckets in descending order (starting next to the zero bucket and going down). +func (h Histogram) NegativeBucketIterator() BucketIterator { + return newRegularBucketIterator(&h, false) } // CumulativeBucketIterator returns a BucketIterator to iterate over a @@ -96,7 +152,7 @@ func (h Histogram) CumulativeBucketIterator() BucketIterator { if len(h.NegativeBuckets) > 0 { panic("CumulativeIterator called on Histogram with negative buckets") } - return &cumulativeBucketIterator{h: h, posSpansIdx: -1} + return &cumulativeBucketIterator{h: &h, posSpansIdx: -1} } // BucketIterator iterates over the buckets of a Histogram, returning decoded @@ -106,26 +162,126 @@ type BucketIterator interface { Next() bool // At returns the current bucket. At() Bucket - // Err returns the current error. It should be used only after iterator is - // exhausted, that is `Next` or `Seek` returns false. - Err() error } -// Bucket represents a bucket (currently only a cumulative one with an upper -// inclusive bound and a cumulative count). +// Bucket represents a bucket with lower and upper limit and the count of +// samples in the bucket. It also specifies if each limit is inclusive or +// not. (Mathematically, inclusive limits create a closed interval, and +// non-inclusive limits an open interval.) +// +// To represent cumulative buckets, Lower is set to -Inf, and the Count is then +// cumulative (including the counts of all buckets for smaller values). type Bucket struct { - Upper float64 - Count uint64 + Lower, Upper float64 + LowerInclusive, UpperInclusive bool + Count uint64 + Index int32 // Index within schema. To easily compare buckets that share the same schema. +} + +// String returns a string representation, using the usual mathematical notation +// of '['/']' for inclusive bounds and '('/')' for non-inclusive bounds. +func (b Bucket) String() string { + var sb strings.Builder + if b.LowerInclusive { + sb.WriteRune('[') + } else { + sb.WriteRune('(') + } + fmt.Fprintf(&sb, "%g,%g", b.Lower, b.Upper) + if b.UpperInclusive { + sb.WriteRune(']') + } else { + sb.WriteRune(')') + } + fmt.Fprintf(&sb, ":%d", b.Count) + return sb.String() +} + +type regularBucketIterator struct { + schema int32 + spans []Span + buckets []int64 + + positive bool // Whether this is for positive buckets. + + spansIdx int // Current span within spans slice. + idxInSpan uint32 // Index in the current span. 0 <= idxInSpan < span.Length. + bucketsIdx int // Current bucket within buckets slice. + + currCount int64 // Count in the current bucket. + currIdx int32 // The actual bucket index. + currLower, currUpper float64 // Limits of the current bucket. + +} + +func newRegularBucketIterator(h *Histogram, positive bool) *regularBucketIterator { + r := ®ularBucketIterator{schema: h.Schema, positive: positive} + if positive { + r.spans = h.PositiveSpans + r.buckets = h.PositiveBuckets + } else { + r.spans = h.NegativeSpans + r.buckets = h.NegativeBuckets + } + return r +} + +func (r *regularBucketIterator) Next() bool { + if r.spansIdx >= len(r.spans) { + return false + } + span := r.spans[r.spansIdx] + // Seed currIdx for the first bucket. + if r.bucketsIdx == 0 { + r.currIdx = span.Offset + } else { + r.currIdx++ + } + for r.idxInSpan >= span.Length { + // We have exhausted the current span and have to find a new + // one. We'll even handle pathologic spans of length 0. + r.idxInSpan = 0 + r.spansIdx++ + if r.spansIdx >= len(r.spans) { + return false + } + span = r.spans[r.spansIdx] + r.currIdx += span.Offset + } + + r.currCount += r.buckets[r.bucketsIdx] + if r.positive { + r.currUpper = getBound(r.currIdx, r.schema) + r.currLower = getBound(r.currIdx-1, r.schema) + } else { + r.currLower = -getBound(r.currIdx, r.schema) + r.currUpper = -getBound(r.currIdx-1, r.schema) + } + + r.idxInSpan++ + r.bucketsIdx++ + return true +} + +func (r *regularBucketIterator) At() Bucket { + return Bucket{ + Count: uint64(r.currCount), + Lower: r.currLower, + Upper: r.currUpper, + LowerInclusive: r.currLower < 0, + UpperInclusive: r.currUpper > 0, + Index: r.currIdx, + } } type cumulativeBucketIterator struct { - h Histogram + h *Histogram posSpansIdx int // Index in h.PositiveSpans we are in. -1 means 0 bucket. posBucketsIdx int // Index in h.PositiveBuckets. idxInSpan uint32 // Index in the current span. 0 <= idxInSpan < span.Length. - initialised bool + initialized bool currIdx int32 // The actual bucket index after decoding from spans. currUpper float64 // The upper boundary of the current bucket. currCount int64 // Current non-cumulative count for the current bucket. Does not apply for empty bucket. @@ -158,24 +314,24 @@ func (c *cumulativeBucketIterator) Next() bool { if c.emptyBucketCount > 0 { // We are traversing through empty buckets at the moment. - c.currUpper = getUpper(c.currIdx, c.h.Schema) + c.currUpper = getBound(c.currIdx, c.h.Schema) c.currIdx++ c.emptyBucketCount-- return true } span := c.h.PositiveSpans[c.posSpansIdx] - if c.posSpansIdx == 0 && !c.initialised { + if c.posSpansIdx == 0 && !c.initialized { // Initialising. c.currIdx = span.Offset - // The first bucket is absolute value and not a delta with Zero bucket. + // The first bucket is an absolute value and not a delta with Zero bucket. c.currCount = 0 - c.initialised = true + c.initialized = true } c.currCount += c.h.PositiveBuckets[c.posBucketsIdx] c.currCumulativeCount += uint64(c.currCount) - c.currUpper = getUpper(c.currIdx, c.h.Schema) + c.currUpper = getBound(c.currIdx, c.h.Schema) c.posBucketsIdx++ c.idxInSpan++ @@ -191,15 +347,19 @@ func (c *cumulativeBucketIterator) Next() bool { return true } + func (c *cumulativeBucketIterator) At() Bucket { return Bucket{ - Upper: c.currUpper, - Count: c.currCumulativeCount, + Upper: c.currUpper, + Lower: math.Inf(-1), + UpperInclusive: true, + LowerInclusive: true, + Count: c.currCumulativeCount, + Index: c.currIdx - 1, } } -func (c *cumulativeBucketIterator) Err() error { return nil } -func getUpper(idx, schema int32) float64 { +func getBound(idx, schema int32) float64 { if schema < 0 { return math.Ldexp(1, int(idx)<<(-schema)) } diff --git a/model/histogram/histogram_test.go b/model/histogram/histogram_test.go index 2a33fdb85..8ef9da69b 100644 --- a/model/histogram/histogram_test.go +++ b/model/histogram/histogram_test.go @@ -15,15 +15,72 @@ package histogram import ( "fmt" + "math" "testing" "github.com/stretchr/testify/require" ) +func TestHistogramString(t *testing.T) { + cases := []struct { + histogram Histogram + expectedString string + }{ + { + histogram: Histogram{ + Schema: 0, + }, + expectedString: "{count:0, sum:0}", + }, + { + histogram: Histogram{ + Schema: 0, + Count: 9, + Sum: -3.1415, + ZeroCount: 12, + ZeroThreshold: 0.001, + NegativeSpans: []Span{ + {Offset: 0, Length: 5}, + {Offset: 1, Length: 1}, + }, + NegativeBuckets: []int64{1, 2, -2, 1, -1, 0}, + }, + expectedString: "{count:9, sum:-3.1415, [-64,-32):1, [-16,-8):1, [-8,-4):2, [-4,-2):1, [-2,-1):3, [-1,-0.5):1, [-0.001,0.001]:12}", + }, + { + histogram: Histogram{ + Schema: 0, + Count: 19, + Sum: 2.7, + PositiveSpans: []Span{ + {Offset: 0, Length: 4}, + {Offset: 0, Length: 0}, + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []int64{1, 2, -2, 1, -1, 0, 0}, + NegativeSpans: []Span{ + {Offset: 0, Length: 5}, + {Offset: 1, Length: 0}, + {Offset: 0, Length: 1}, + }, + NegativeBuckets: []int64{1, 2, -2, 1, -1, 0}, + }, + expectedString: "{count:19, sum:2.7, [-64,-32):1, [-16,-8):1, [-8,-4):2, [-4,-2):1, [-2,-1):3, [-1,-0.5):1, (0.5,1]:1, (1,2]:3, (2,4]:1, (4,8]:2, (8,16]:1, (16,32]:1, (32,64]:1}", + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + actualString := c.histogram.String() + require.Equal(t, c.expectedString, actualString) + }) + } +} + func TestCumulativeBucketIterator(t *testing.T) { cases := []struct { - histogram Histogram - expectedCumulativeBuckets []Bucket + histogram Histogram + expectedBuckets []Bucket }{ { histogram: Histogram{ @@ -34,14 +91,14 @@ func TestCumulativeBucketIterator(t *testing.T) { }, PositiveBuckets: []int64{1, 1, -1, 0}, }, - expectedCumulativeBuckets: []Bucket{ - {Upper: 1, Count: 1}, - {Upper: 2, Count: 3}, + expectedBuckets: []Bucket{ + {Lower: math.Inf(-1), Upper: 1, Count: 1, LowerInclusive: true, UpperInclusive: true, Index: 0}, + {Lower: math.Inf(-1), Upper: 2, Count: 3, LowerInclusive: true, UpperInclusive: true, Index: 1}, - {Upper: 4, Count: 3}, + {Lower: math.Inf(-1), Upper: 4, Count: 3, LowerInclusive: true, UpperInclusive: true, Index: 2}, - {Upper: 8, Count: 4}, - {Upper: 16, Count: 5}, + {Lower: math.Inf(-1), Upper: 8, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: 3}, + {Lower: math.Inf(-1), Upper: 16, Count: 5, LowerInclusive: true, UpperInclusive: true, Index: 4}, }, }, { @@ -53,16 +110,16 @@ func TestCumulativeBucketIterator(t *testing.T) { }, PositiveBuckets: []int64{1, 2, -2, 1, -1, 0}, }, - expectedCumulativeBuckets: []Bucket{ - {Upper: 1, Count: 1}, - {Upper: 2, Count: 4}, - {Upper: 4, Count: 5}, - {Upper: 8, Count: 7}, + expectedBuckets: []Bucket{ + {Lower: math.Inf(-1), Upper: 1, Count: 1, LowerInclusive: true, UpperInclusive: true, Index: 0}, + {Lower: math.Inf(-1), Upper: 2, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: 1}, + {Lower: math.Inf(-1), Upper: 4, Count: 5, LowerInclusive: true, UpperInclusive: true, Index: 2}, + {Lower: math.Inf(-1), Upper: 8, Count: 7, LowerInclusive: true, UpperInclusive: true, Index: 3}, - {Upper: 16, Count: 8}, + {Lower: math.Inf(-1), Upper: 16, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 4}, - {Upper: 32, Count: 8}, - {Upper: 64, Count: 9}, + {Lower: math.Inf(-1), Upper: 32, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 5}, + {Lower: math.Inf(-1), Upper: 64, Count: 9, LowerInclusive: true, UpperInclusive: true, Index: 6}, }, }, { @@ -73,14 +130,14 @@ func TestCumulativeBucketIterator(t *testing.T) { }, PositiveBuckets: []int64{1, 2, -2, 1, -1, 0, 0}, }, - expectedCumulativeBuckets: []Bucket{ - {Upper: 1, Count: 1}, - {Upper: 2, Count: 4}, - {Upper: 4, Count: 5}, - {Upper: 8, Count: 7}, - {Upper: 16, Count: 8}, - {Upper: 32, Count: 9}, - {Upper: 64, Count: 10}, + expectedBuckets: []Bucket{ + {Lower: math.Inf(-1), Upper: 1, Count: 1, LowerInclusive: true, UpperInclusive: true, Index: 0}, + {Lower: math.Inf(-1), Upper: 2, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: 1}, + {Lower: math.Inf(-1), Upper: 4, Count: 5, LowerInclusive: true, UpperInclusive: true, Index: 2}, + {Lower: math.Inf(-1), Upper: 8, Count: 7, LowerInclusive: true, UpperInclusive: true, Index: 3}, + {Lower: math.Inf(-1), Upper: 16, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 4}, + {Lower: math.Inf(-1), Upper: 32, Count: 9, LowerInclusive: true, UpperInclusive: true, Index: 5}, + {Lower: math.Inf(-1), Upper: 64, Count: 10, LowerInclusive: true, UpperInclusive: true, Index: 6}, }, }, { @@ -93,22 +150,22 @@ func TestCumulativeBucketIterator(t *testing.T) { }, PositiveBuckets: []int64{1, 2, -2, 1, -1, 0, 3}, }, - expectedCumulativeBuckets: []Bucket{ - {Upper: 0.6484197773255048, Count: 1}, // -5 - {Upper: 0.7071067811865475, Count: 4}, // -4 + expectedBuckets: []Bucket{ + {Lower: math.Inf(-1), Upper: 0.6484197773255048, Count: 1, LowerInclusive: true, UpperInclusive: true, Index: -5}, + {Lower: math.Inf(-1), Upper: 0.7071067811865475, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: -4}, - {Upper: 0.7711054127039704, Count: 4}, // -3 - {Upper: 0.8408964152537144, Count: 4}, // -2 + {Lower: math.Inf(-1), Upper: 0.7711054127039704, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: -3}, + {Lower: math.Inf(-1), Upper: 0.8408964152537144, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: -2}, - {Upper: 0.9170040432046711, Count: 5}, // -1 - {Upper: 1, Count: 7}, // 1 - {Upper: 1.0905077326652577, Count: 8}, // 0 + {Lower: math.Inf(-1), Upper: 0.9170040432046711, Count: 5, LowerInclusive: true, UpperInclusive: true, Index: -1}, + {Lower: math.Inf(-1), Upper: 1, Count: 7, LowerInclusive: true, UpperInclusive: true, Index: 0}, + {Lower: math.Inf(-1), Upper: 1.0905077326652577, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 1}, - {Upper: 1.189207115002721, Count: 8}, // 1 - {Upper: 1.2968395546510096, Count: 8}, // 2 + {Lower: math.Inf(-1), Upper: 1.189207115002721, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 2}, + {Lower: math.Inf(-1), Upper: 1.2968395546510096, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 3}, - {Upper: 1.414213562373095, Count: 9}, // 3 - {Upper: 1.5422108254079407, Count: 13}, // 4 + {Lower: math.Inf(-1), Upper: 1.414213562373095, Count: 9, LowerInclusive: true, UpperInclusive: true, Index: 4}, + {Lower: math.Inf(-1), Upper: 1.5422108254079407, Count: 13, LowerInclusive: true, UpperInclusive: true, Index: 5}, }, }, { @@ -120,17 +177,17 @@ func TestCumulativeBucketIterator(t *testing.T) { }, PositiveBuckets: []int64{1, 2, -2, 1, -1, 0}, }, - expectedCumulativeBuckets: []Bucket{ - {Upper: 0.00390625, Count: 1}, // -2 - {Upper: 0.0625, Count: 4}, // -1 - {Upper: 1, Count: 5}, // 0 - {Upper: 16, Count: 7}, // 1 + expectedBuckets: []Bucket{ + {Lower: math.Inf(-1), Upper: 0.00390625, Count: 1, LowerInclusive: true, UpperInclusive: true, Index: -2}, + {Lower: math.Inf(-1), Upper: 0.0625, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: -1}, + {Lower: math.Inf(-1), Upper: 1, Count: 5, LowerInclusive: true, UpperInclusive: true, Index: 0}, + {Lower: math.Inf(-1), Upper: 16, Count: 7, LowerInclusive: true, UpperInclusive: true, Index: 1}, - {Upper: 256, Count: 7}, // 2 - {Upper: 4096, Count: 7}, // 3 + {Lower: math.Inf(-1), Upper: 256, Count: 7, LowerInclusive: true, UpperInclusive: true, Index: 2}, + {Lower: math.Inf(-1), Upper: 4096, Count: 7, LowerInclusive: true, UpperInclusive: true, Index: 3}, - {Upper: 65536, Count: 8}, // 4 - {Upper: 1048576, Count: 9}, // 5 + {Lower: math.Inf(-1), Upper: 65536, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 4}, + {Lower: math.Inf(-1), Upper: 1048576, Count: 9, LowerInclusive: true, UpperInclusive: true, Index: 5}, }, }, { @@ -141,12 +198,12 @@ func TestCumulativeBucketIterator(t *testing.T) { }, PositiveBuckets: []int64{1, 2, -2, 1, -1}, }, - expectedCumulativeBuckets: []Bucket{ - {Upper: 0.0625, Count: 1}, // -2 - {Upper: 0.25, Count: 4}, // -1 - {Upper: 1, Count: 5}, // 0 - {Upper: 4, Count: 7}, // 1 - {Upper: 16, Count: 8}, // 2 + expectedBuckets: []Bucket{ + {Lower: math.Inf(-1), Upper: 0.0625, Count: 1, LowerInclusive: true, UpperInclusive: true, Index: -2}, + {Lower: math.Inf(-1), Upper: 0.25, Count: 4, LowerInclusive: true, UpperInclusive: true, Index: -1}, + {Lower: math.Inf(-1), Upper: 1, Count: 5, LowerInclusive: true, UpperInclusive: true, Index: 0}, + {Lower: math.Inf(-1), Upper: 4, Count: 7, LowerInclusive: true, UpperInclusive: true, Index: 1}, + {Lower: math.Inf(-1), Upper: 16, Count: 8, LowerInclusive: true, UpperInclusive: true, Index: 2}, }, }, } @@ -154,12 +211,177 @@ func TestCumulativeBucketIterator(t *testing.T) { for i, c := range cases { t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { it := c.histogram.CumulativeBucketIterator() - actualBuckets := make([]Bucket, 0, len(c.expectedCumulativeBuckets)) + actualBuckets := make([]Bucket, 0, len(c.expectedBuckets)) for it.Next() { actualBuckets = append(actualBuckets, it.At()) } - require.NoError(t, it.Err()) - require.Equal(t, c.expectedCumulativeBuckets, actualBuckets) + require.Equal(t, c.expectedBuckets, actualBuckets) + }) + } +} + +func TestRegularBucketIterator(t *testing.T) { + cases := []struct { + histogram Histogram + expectedPositiveBuckets []Bucket + expectedNegativeBuckets []Bucket + }{ + { + histogram: Histogram{ + Schema: 0, + }, + expectedPositiveBuckets: []Bucket{}, + expectedNegativeBuckets: []Bucket{}, + }, + { + histogram: Histogram{ + Schema: 0, + PositiveSpans: []Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []int64{1, 1, -1, 0}, + }, + expectedPositiveBuckets: []Bucket{ + {Lower: 0.5, Upper: 1, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 0}, + {Lower: 1, Upper: 2, Count: 2, LowerInclusive: false, UpperInclusive: true, Index: 1}, + + {Lower: 4, Upper: 8, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 3}, + {Lower: 8, Upper: 16, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 4}, + }, + expectedNegativeBuckets: []Bucket{}, + }, + { + histogram: Histogram{ + Schema: 0, + NegativeSpans: []Span{ + {Offset: 0, Length: 5}, + {Offset: 1, Length: 1}, + }, + NegativeBuckets: []int64{1, 2, -2, 1, -1, 0}, + }, + expectedPositiveBuckets: []Bucket{}, + expectedNegativeBuckets: []Bucket{ + {Lower: -1, Upper: -0.5, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 0}, + {Lower: -2, Upper: -1, Count: 3, LowerInclusive: true, UpperInclusive: false, Index: 1}, + {Lower: -4, Upper: -2, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 2}, + {Lower: -8, Upper: -4, Count: 2, LowerInclusive: true, UpperInclusive: false, Index: 3}, + {Lower: -16, Upper: -8, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 4}, + + {Lower: -64, Upper: -32, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 6}, + }, + }, + { + histogram: Histogram{ + Schema: 0, + PositiveSpans: []Span{ + {Offset: 0, Length: 4}, + {Offset: 0, Length: 0}, + {Offset: 0, Length: 3}, + }, + PositiveBuckets: []int64{1, 2, -2, 1, -1, 0, 0}, + NegativeSpans: []Span{ + {Offset: 0, Length: 5}, + {Offset: 1, Length: 0}, + {Offset: 0, Length: 1}, + }, + NegativeBuckets: []int64{1, 2, -2, 1, -1, 0}, + }, + expectedPositiveBuckets: []Bucket{ + {Lower: 0.5, Upper: 1, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 0}, + {Lower: 1, Upper: 2, Count: 3, LowerInclusive: false, UpperInclusive: true, Index: 1}, + {Lower: 2, Upper: 4, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 2}, + {Lower: 4, Upper: 8, Count: 2, LowerInclusive: false, UpperInclusive: true, Index: 3}, + {Lower: 8, Upper: 16, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 4}, + {Lower: 16, Upper: 32, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 5}, + {Lower: 32, Upper: 64, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 6}, + }, + expectedNegativeBuckets: []Bucket{ + {Lower: -1, Upper: -0.5, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 0}, + {Lower: -2, Upper: -1, Count: 3, LowerInclusive: true, UpperInclusive: false, Index: 1}, + {Lower: -4, Upper: -2, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 2}, + {Lower: -8, Upper: -4, Count: 2, LowerInclusive: true, UpperInclusive: false, Index: 3}, + {Lower: -16, Upper: -8, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 4}, + + {Lower: -64, Upper: -32, Count: 1, LowerInclusive: true, UpperInclusive: false, Index: 6}, + }, + }, + { + histogram: Histogram{ + Schema: 3, + PositiveSpans: []Span{ + {Offset: -5, Length: 2}, // -5 -4 + {Offset: 2, Length: 3}, // -1 0 1 + {Offset: 2, Length: 2}, // 4 5 + }, + PositiveBuckets: []int64{1, 2, -2, 1, -1, 0, 3}, + }, + expectedPositiveBuckets: []Bucket{ + {Lower: 0.5946035575013605, Upper: 0.6484197773255048, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: -5}, + {Lower: 0.6484197773255048, Upper: 0.7071067811865475, Count: 3, LowerInclusive: false, UpperInclusive: true, Index: -4}, + + {Lower: 0.8408964152537144, Upper: 0.9170040432046711, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: -1}, + {Lower: 0.9170040432046711, Upper: 1, Count: 2, LowerInclusive: false, UpperInclusive: true, Index: 0}, + {Lower: 1, Upper: 1.0905077326652577, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 1}, + + {Lower: 1.2968395546510096, Upper: 1.414213562373095, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 4}, + {Lower: 1.414213562373095, Upper: 1.5422108254079407, Count: 4, LowerInclusive: false, UpperInclusive: true, Index: 5}, + }, + expectedNegativeBuckets: []Bucket{}, + }, + { + histogram: Histogram{ + Schema: -2, + PositiveSpans: []Span{ + {Offset: -2, Length: 4}, // -2 -1 0 1 + {Offset: 2, Length: 2}, // 4 5 + }, + PositiveBuckets: []int64{1, 2, -2, 1, -1, 0}, + }, + expectedPositiveBuckets: []Bucket{ + {Lower: 0.000244140625, Upper: 0.00390625, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: -2}, + {Lower: 0.00390625, Upper: 0.0625, Count: 3, LowerInclusive: false, UpperInclusive: true, Index: -1}, + {Lower: 0.0625, Upper: 1, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 0}, + {Lower: 1, Upper: 16, Count: 2, LowerInclusive: false, UpperInclusive: true, Index: 1}, + + {Lower: 4096, Upper: 65536, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 4}, + {Lower: 65536, Upper: 1048576, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 5}, + }, + expectedNegativeBuckets: []Bucket{}, + }, + { + histogram: Histogram{ + Schema: -1, + PositiveSpans: []Span{ + {Offset: -2, Length: 5}, // -2 -1 0 1 2 + }, + PositiveBuckets: []int64{1, 2, -2, 1, -1}, + }, + expectedPositiveBuckets: []Bucket{ + {Lower: 0.015625, Upper: 0.0625, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: -2}, + {Lower: 0.0625, Upper: 0.25, Count: 3, LowerInclusive: false, UpperInclusive: true, Index: -1}, + {Lower: 0.25, Upper: 1, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 0}, + {Lower: 1, Upper: 4, Count: 2, LowerInclusive: false, UpperInclusive: true, Index: 1}, + {Lower: 4, Upper: 16, Count: 1, LowerInclusive: false, UpperInclusive: true, Index: 2}, + }, + expectedNegativeBuckets: []Bucket{}, + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("%d", i), func(t *testing.T) { + it := c.histogram.PositiveBucketIterator() + actualPositiveBuckets := make([]Bucket, 0, len(c.expectedPositiveBuckets)) + for it.Next() { + actualPositiveBuckets = append(actualPositiveBuckets, it.At()) + } + require.Equal(t, c.expectedPositiveBuckets, actualPositiveBuckets) + it = c.histogram.NegativeBucketIterator() + actualNegativeBuckets := make([]Bucket, 0, len(c.expectedNegativeBuckets)) + for it.Next() { + actualNegativeBuckets = append(actualNegativeBuckets, it.At()) + } + require.Equal(t, c.expectedNegativeBuckets, actualNegativeBuckets) }) } } diff --git a/promql/engine_test.go b/promql/engine_test.go index 88eed7f8b..941250a41 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -2460,4 +2460,5 @@ func TestSparseHistogramRate(t *testing.T) { require.NoError(t, err) res := qry.Exec(test.Context()) require.NoError(t, res.Err) + fmt.Println(res) } diff --git a/tsdb/compact_test.go b/tsdb/compact_test.go index 5f3d2374b..b0df493f7 100644 --- a/tsdb/compact_test.go +++ b/tsdb/compact_test.go @@ -1533,7 +1533,6 @@ func TestSparseHistogramSpaceSavings(t *testing.T) { require.NoError(t, err) itIdx++ } - require.NoError(t, it.Err()) // _count metric. countLbls := ah.baseLabels.Copy() countLbls[0].Value = countLbls[0].Value + "_count" From 4c28d9fac7342853d52c873b2dd977c6509e6b30 Mon Sep 17 00:00:00 2001 From: beorn7 Date: Fri, 12 Nov 2021 19:07:41 +0100 Subject: [PATCH 4/7] Move to histogram.Histogram pointers This is to avoid copying the many fields of a histogram.Histogram all the time. This also fixes a bunch of formerly broken tests. Signed-off-by: beorn7 --- cmd/prometheus/main.go | 2 +- cmd/promtool/unittest.go | 2 +- pkg/textparse/interface.go | 2 +- pkg/textparse/openmetricsparse.go | 8 +- pkg/textparse/promparse.go | 8 +- pkg/textparse/protobufparse.go | 6 +- pkg/textparse/protobufparse_test.go | 4 +- promql/engine.go | 4 +- promql/engine_test.go | 2 +- promql/value.go | 4 +- scrape/helpers_test.go | 6 +- scrape/scrape.go | 2 +- storage/buffer.go | 10 +- storage/buffer_test.go | 14 +- storage/fanout.go | 2 +- storage/interface.go | 2 +- storage/merge.go | 2 +- storage/merge_test.go | 264 +++++++++++++-------------- storage/remote/codec.go | 8 +- storage/remote/write.go | 2 +- storage/remote/write_handler_test.go | 2 +- storage/series.go | 15 +- tsdb/block_test.go | 12 +- tsdb/chunkenc/chunk.go | 9 +- tsdb/chunkenc/histogram.go | 14 +- tsdb/chunkenc/histogram_test.go | 12 +- tsdb/chunkenc/xor.go | 6 +- tsdb/compact_test.go | 34 ++-- tsdb/db_test.go | 36 ++-- tsdb/head.go | 15 +- tsdb/head_append.go | 12 +- tsdb/head_read.go | 2 +- tsdb/head_test.go | 33 ++-- tsdb/querier.go | 8 +- tsdb/querier_test.go | 164 ++++++++--------- tsdb/record/record.go | 4 +- tsdb/tsdbutil/buffer.go | 4 +- tsdb/tsdbutil/buffer_test.go | 7 +- web/federate.go | 4 +- 39 files changed, 375 insertions(+), 372 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index ce49a9dd9..b405ef670 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -1190,7 +1190,7 @@ func (n notReadyAppender) AppendExemplar(ref uint64, l labels.Labels, e exemplar return 0, tsdb.ErrNotReady } -func (n notReadyAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, sh histogram.Histogram) (uint64, error) { +func (n notReadyAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, h *histogram.Histogram) (uint64, error) { return 0, tsdb.ErrNotReady } diff --git a/cmd/promtool/unittest.go b/cmd/promtool/unittest.go index 3ddec3152..4c197d074 100644 --- a/cmd/promtool/unittest.go +++ b/cmd/promtool/unittest.go @@ -460,7 +460,7 @@ func query(ctx context.Context, qs string, t time.Time, engine *promql.Engine, q return v, nil case promql.Scalar: return promql.Vector{promql.Sample{ - Point: promql.Point(v), + Point: promql.Point{T: v.T, V: v.V}, Metric: labels.Labels{}, }}, nil default: diff --git a/pkg/textparse/interface.go b/pkg/textparse/interface.go index e55e628b6..25f0ec9ab 100644 --- a/pkg/textparse/interface.go +++ b/pkg/textparse/interface.go @@ -30,7 +30,7 @@ type Parser interface { // Histogram returns the bytes of a series with a sparse histogram as a // value, the timestamp if set, and the histogram in the current sample. - Histogram() ([]byte, *int64, histogram.Histogram) + Histogram() ([]byte, *int64, *histogram.Histogram) // Help returns the metric name and help text in the current entry. // Must only be called after Next returned a help entry. diff --git a/pkg/textparse/openmetricsparse.go b/pkg/textparse/openmetricsparse.go index 265ca22b1..b51321ff4 100644 --- a/pkg/textparse/openmetricsparse.go +++ b/pkg/textparse/openmetricsparse.go @@ -114,10 +114,10 @@ func (p *OpenMetricsParser) Series() ([]byte, *int64, float64) { return p.series, nil, p.val } -// Histogram always returns (nil, nil, histogram.Histogram{}) because -// OpenMetrics does not support sparse histograms. -func (p *OpenMetricsParser) Histogram() ([]byte, *int64, histogram.Histogram) { - return nil, nil, histogram.Histogram{} +// Histogram always returns (nil, nil, nil) because OpenMetrics does not support +// sparse histograms. +func (p *OpenMetricsParser) Histogram() ([]byte, *int64, *histogram.Histogram) { + return nil, nil, nil } // Help returns the metric name and help text in the current entry. diff --git a/pkg/textparse/promparse.go b/pkg/textparse/promparse.go index c7059a2b1..0190ef6eb 100644 --- a/pkg/textparse/promparse.go +++ b/pkg/textparse/promparse.go @@ -169,10 +169,10 @@ func (p *PromParser) Series() ([]byte, *int64, float64) { return p.series, nil, p.val } -// Histogram always returns (nil, nil, histogram.Histogram{}) because the -// Prometheus text format does not support sparse histograms. -func (p *PromParser) Histogram() ([]byte, *int64, histogram.Histogram) { - return nil, nil, histogram.Histogram{} +// Histogram always returns (nil, nil, nil) because the Prometheus text format +// does not support sparse histograms. +func (p *PromParser) Histogram() ([]byte, *int64, *histogram.Histogram) { + return nil, nil, nil } // Help returns the metric name and help text in the current entry. diff --git a/pkg/textparse/protobufparse.go b/pkg/textparse/protobufparse.go index 5a43108f1..b0bcf7ed0 100644 --- a/pkg/textparse/protobufparse.go +++ b/pkg/textparse/protobufparse.go @@ -135,7 +135,7 @@ func (p *ProtobufParser) Series() ([]byte, *int64, float64) { // Histogram returns the bytes of a series with a sparse histogram as a // value, the timestamp if set, and the sparse histogram in the current // sample. -func (p *ProtobufParser) Histogram() ([]byte, *int64, histogram.Histogram) { +func (p *ProtobufParser) Histogram() ([]byte, *int64, *histogram.Histogram) { var ( m = p.mf.GetMetric()[p.metricPos] ts = m.GetTimestampMs() @@ -161,12 +161,12 @@ func (p *ProtobufParser) Histogram() ([]byte, *int64, histogram.Histogram) { sh.NegativeSpans[i].Length = span.GetLength() } if ts != 0 { - return p.metricBytes.Bytes(), &ts, sh + return p.metricBytes.Bytes(), &ts, &sh } // Nasty hack: Assume that ts==0 means no timestamp. That's not true in // general, but proto3 has no distinction between unset and // default. Need to avoid in the final format. - return p.metricBytes.Bytes(), nil, sh + return p.metricBytes.Bytes(), nil, &sh } // Help returns the metric name and help text in the current entry. diff --git a/pkg/textparse/protobufparse_test.go b/pkg/textparse/protobufparse_test.go index 9df8c4e63..c92c148fe 100644 --- a/pkg/textparse/protobufparse_test.go +++ b/pkg/textparse/protobufparse_test.go @@ -266,7 +266,7 @@ metric: < help string unit string comment string - shs histogram.Histogram + shs *histogram.Histogram e []exemplar.Exemplar }{ { @@ -332,7 +332,7 @@ metric: < { m: "test_histogram", t: 1234568, - shs: histogram.Histogram{ + shs: &histogram.Histogram{ Count: 175, ZeroCount: 2, Sum: 0.0008280461746287094, diff --git a/promql/engine.go b/promql/engine.go index f1c48f610..afe870f2d 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -1748,7 +1748,7 @@ func (ev *evaluator) matrixIterSlice(it *storage.BufferedSeriesIterator, mint, m ev.error(ErrTooManySamples(env)) } ev.currentSamples++ - out = append(out, Point{T: t, H: &h}) + out = append(out, Point{T: t, H: h}) } } } else { @@ -1775,7 +1775,7 @@ func (ev *evaluator) matrixIterSlice(it *storage.BufferedSeriesIterator, mint, m if ev.currentSamples >= ev.maxSamples { ev.error(ErrTooManySamples(env)) } - out = append(out, Point{T: t, H: &h}) + out = append(out, Point{T: t, H: h}) ev.currentSamples++ } } else { diff --git a/promql/engine_test.go b/promql/engine_test.go index 941250a41..fe1d017bc 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -2460,5 +2460,5 @@ func TestSparseHistogramRate(t *testing.T) { require.NoError(t, err) res := qry.Exec(test.Context()) require.NoError(t, res.Err) - fmt.Println(res) + // fmt.Println(res) } diff --git a/promql/value.go b/promql/value.go index 4f52b3199..5e3863379 100644 --- a/promql/value.go +++ b/promql/value.go @@ -300,9 +300,9 @@ func (ssi *storageSeriesIterator) At() (t int64, v float64) { return p.T, p.V } -func (ssi *storageSeriesIterator) AtHistogram() (int64, histogram.Histogram) { +func (ssi *storageSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { p := ssi.points[ssi.curr] - return p.T, *p.H + return p.T, p.H } func (ssi *storageSeriesIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/scrape/helpers_test.go b/scrape/helpers_test.go index 16eb8e272..a46350a6e 100644 --- a/scrape/helpers_test.go +++ b/scrape/helpers_test.go @@ -35,7 +35,7 @@ func (a nopAppender) Append(uint64, labels.Labels, int64, float64) (uint64, erro func (a nopAppender) AppendExemplar(uint64, labels.Labels, exemplar.Exemplar) (uint64, error) { return 0, nil } -func (a nopAppender) AppendHistogram(uint64, labels.Labels, int64, histogram.Histogram) (uint64, error) { +func (a nopAppender) AppendHistogram(uint64, labels.Labels, int64, *histogram.Histogram) (uint64, error) { return 0, nil } func (a nopAppender) Commit() error { return nil } @@ -49,7 +49,7 @@ type sample struct { type histogramSample struct { t int64 - h histogram.Histogram + h *histogram.Histogram } // collectResultAppender records all samples that were added through the appender. @@ -96,7 +96,7 @@ func (a *collectResultAppender) AppendExemplar(ref uint64, l labels.Labels, e ex return a.next.AppendExemplar(ref, l, e) } -func (a *collectResultAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, h histogram.Histogram) (uint64, error) { +func (a *collectResultAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, h *histogram.Histogram) (uint64, error) { a.pendingHistograms = append(a.pendingHistograms, histogramSample{h: h, t: t}) if a.next == nil { return 0, nil diff --git a/scrape/scrape.go b/scrape/scrape.go index 8ffb98742..a44ad0c85 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -1440,7 +1440,7 @@ loop: met []byte parsedTimestamp *int64 val float64 - h histogram.Histogram + h *histogram.Histogram ) if et, err = p.Next(); err != nil { if err == io.EOF { diff --git a/storage/buffer.go b/storage/buffer.go index 93cd9da00..cad7e6653 100644 --- a/storage/buffer.go +++ b/storage/buffer.go @@ -120,7 +120,7 @@ func (b *BufferedSeriesIterator) Next() bool { // Add current element to buffer before advancing. if b.it.ChunkEncoding() == chunkenc.EncHistogram { t, h := b.it.AtHistogram() - b.buf.add(sample{t: t, h: &h}) + b.buf.add(sample{t: t, h: h}) } else { t, v := b.it.At() b.buf.add(sample{t: t, v: v}) @@ -144,7 +144,7 @@ func (b *BufferedSeriesIterator) Values() (int64, float64) { } // HistogramValues returns the current histogram element of the iterator. -func (b *BufferedSeriesIterator) HistogramValues() (int64, histogram.Histogram) { +func (b *BufferedSeriesIterator) HistogramValues() (int64, *histogram.Histogram) { return b.it.AtHistogram() } @@ -230,7 +230,7 @@ func (it *sampleRingIterator) At() (int64, float64) { return it.r.at(it.i) } -func (it *sampleRingIterator) AtHistogram() (int64, histogram.Histogram) { +func (it *sampleRingIterator) AtHistogram() (int64, *histogram.Histogram) { return it.r.atHistogram(it.i) } @@ -244,10 +244,10 @@ func (r *sampleRing) at(i int) (int64, float64) { return s.t, s.v } -func (r *sampleRing) atHistogram(i int) (int64, histogram.Histogram) { +func (r *sampleRing) atHistogram(i int) (int64, *histogram.Histogram) { j := (r.f + i) % len(r.buf) s := r.buf[j] - return s.t, *s.h + return s.t, s.h } func (r *sampleRing) atSample(i int) sample { diff --git a/storage/buffer_test.go b/storage/buffer_test.go index 2d7b3f1e9..5cb8aeab8 100644 --- a/storage/buffer_test.go +++ b/storage/buffer_test.go @@ -55,7 +55,7 @@ func TestSampleRing(t *testing.T) { }, } for _, c := range cases { - r := newSampleRing(c.delta, c.size) + r := newSampleRing(c.delta, c.size, chunkenc.EncNone) input := []sample{} for _, t := range c.input { @@ -66,7 +66,7 @@ func TestSampleRing(t *testing.T) { } for i, s := range input { - r.add(s.t, s.v) + r.add(s) buffered := r.samples() for _, sold := range input[:i] { @@ -106,7 +106,7 @@ func TestBufferedSeriesIterator(t *testing.T) { require.Equal(t, ev, v, "value mismatch") } prevSampleEq := func(ets int64, ev float64, eok bool) { - ts, v, ok := it.PeekBack(1) + ts, v, _, ok := it.PeekBack(1) require.Equal(t, eok, ok, "exist mismatch") require.Equal(t, ets, ts, "timestamp mismatch") require.Equal(t, ev, v, "value mismatch") @@ -196,8 +196,8 @@ type mockSeriesIterator struct { func (m *mockSeriesIterator) Seek(t int64) bool { return m.seek(t) } func (m *mockSeriesIterator) At() (int64, float64) { return m.at() } -func (m *mockSeriesIterator) AtHistogram() (int64, histogram.Histogram) { - return 0, histogram.Histogram{} +func (m *mockSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { + return 0, nil } func (m *mockSeriesIterator) ChunkEncoding() chunkenc.Encoding { return chunkenc.EncXOR @@ -219,8 +219,8 @@ func (it *fakeSeriesIterator) At() (int64, float64) { return it.idx * it.step, 123 // Value doesn't matter. } -func (it *fakeSeriesIterator) AtHistogram() (int64, histogram.Histogram) { - return it.idx * it.step, histogram.Histogram{} // Value doesn't matter. +func (it *fakeSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { + return it.idx * it.step, &histogram.Histogram{} // Value doesn't matter. } func (it *fakeSeriesIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/storage/fanout.go b/storage/fanout.go index ccb123895..672756d96 100644 --- a/storage/fanout.go +++ b/storage/fanout.go @@ -173,7 +173,7 @@ func (f *fanoutAppender) AppendExemplar(ref uint64, l labels.Labels, e exemplar. return ref, nil } -func (f *fanoutAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, h histogram.Histogram) (uint64, error) { +func (f *fanoutAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, h *histogram.Histogram) (uint64, error) { ref, err := f.primary.AppendHistogram(ref, l, t, h) if err != nil { return ref, err diff --git a/storage/interface.go b/storage/interface.go index be55a5bb2..f15b1c00d 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -222,7 +222,7 @@ type HistogramAppender interface { // numbers are ephemeral and may be rejected in calls to Append() at any // point. Adding the sample via Append() returns a new reference number. // If the reference is 0 it must not be used for caching. - AppendHistogram(ref uint64, l labels.Labels, t int64, h histogram.Histogram) (uint64, error) + AppendHistogram(ref uint64, l labels.Labels, t int64, h *histogram.Histogram) (uint64, error) } // SeriesSet contains a set of series. diff --git a/storage/merge.go b/storage/merge.go index 7bae3a8d9..daa12108a 100644 --- a/storage/merge.go +++ b/storage/merge.go @@ -486,7 +486,7 @@ func (c *chainSampleIterator) At() (t int64, v float64) { return c.curr.At() } -func (c *chainSampleIterator) AtHistogram() (int64, histogram.Histogram) { +func (c *chainSampleIterator) AtHistogram() (int64, *histogram.Histogram) { if c.curr == nil { panic("chainSampleIterator.AtHistogram() called before first .Next() or after .Next() returned false.") } diff --git a/storage/merge_test.go b/storage/merge_test.go index 23eab0f70..62aa4376e 100644 --- a/storage/merge_test.go +++ b/storage/merge_test.go @@ -62,116 +62,116 @@ func TestMergeQuerierWithChainMerger(t *testing.T) { { name: "one querier, two series", querierSeries: [][]Series{{ - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }}, expected: NewMockSeriesSet( - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), ), }, { name: "two queriers, one different series each", querierSeries: [][]Series{{ - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), }, { - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }}, expected: NewMockSeriesSet( - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), ), }, { name: "two time unsorted queriers, two series each", querierSeries: [][]Series{{ - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5}, sample{6, 6}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5, nil}, sample{6, 6, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }, { - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3}, sample{4, 4}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3, nil}, sample{4, 4, nil}}), }}, expected: NewMockSeriesSet( NewListSeries( labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}, sample{6, 6, nil}}, ), NewListSeries( labels.FromStrings("foo", "bar"), - []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{4, 4}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}}, ), ), }, { name: "five queriers, only two queriers have two time unsorted series each", querierSeries: [][]Series{{}, {}, { - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5}, sample{6, 6}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5, nil}, sample{6, 6, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }, { - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3}, sample{4, 4}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3, nil}, sample{4, 4, nil}}), }, {}}, expected: NewMockSeriesSet( NewListSeries( labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}, sample{6, 6, nil}}, ), NewListSeries( labels.FromStrings("foo", "bar"), - []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{4, 4}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}}, ), ), }, { name: "two queriers, only two queriers have two time unsorted series each, with 3 noop and one nil querier together", querierSeries: [][]Series{{}, {}, { - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5}, sample{6, 6}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5, nil}, sample{6, 6, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }, { - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3}, sample{4, 4}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3, nil}, sample{4, 4, nil}}), }, {}}, extraQueriers: []Querier{NoopQuerier(), NoopQuerier(), nil, NoopQuerier()}, expected: NewMockSeriesSet( NewListSeries( labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}, sample{6, 6, nil}}, ), NewListSeries( labels.FromStrings("foo", "bar"), - []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{4, 4}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}}, ), ), }, { name: "two queriers, with two series, one is overlapping", querierSeries: [][]Series{{}, {}, { - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 21}, sample{3, 31}, sample{5, 5}, sample{6, 6}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 21, nil}, sample{3, 31, nil}, sample{5, 5, nil}, sample{6, 6, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }, { - NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 22}, sample{3, 32}}), - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3}, sample{4, 4}}), + NewListSeries(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 22, nil}, sample{3, 32, nil}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3, nil}, sample{4, 4, nil}}), }, {}}, expected: NewMockSeriesSet( NewListSeries( labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{1, 1}, sample{2, 21}, sample{3, 31}, sample{5, 5}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 21, nil}, sample{3, 31, nil}, sample{5, 5, nil}, sample{6, 6, nil}}, ), NewListSeries( labels.FromStrings("foo", "bar"), - []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{4, 4}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}}, ), ), }, { name: "two queries, one with NaN samples series", querierSeries: [][]Series{{ - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN()}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN(), nil}}), }, { - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{1, 1}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{1, 1, nil}}), }}, expected: NewMockSeriesSet( - NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN()}, sample{1, 1}}), + NewListSeries(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN(), nil}, sample{1, 1, nil}}), ), }, } { @@ -245,108 +245,108 @@ func TestMergeChunkQuerierWithNoVerticalChunkSeriesMerger(t *testing.T) { { name: "one querier, two series", chkQuerierSeries: [][]ChunkSeries{{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, []tsdbutil.Sample{sample{2, 2}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, []tsdbutil.Sample{sample{2, 2, nil}}), }}, expected: NewMockChunkSeriesSet( - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, []tsdbutil.Sample{sample{2, 2}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, []tsdbutil.Sample{sample{2, 2, nil}}), ), }, { name: "two secondaries, one different series each", chkQuerierSeries: [][]ChunkSeries{{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), }, { - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, []tsdbutil.Sample{sample{2, 2}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, []tsdbutil.Sample{sample{2, 2, nil}}), }}, expected: NewMockChunkSeriesSet( - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, []tsdbutil.Sample{sample{2, 2}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, []tsdbutil.Sample{sample{2, 2, nil}}), ), }, { name: "two secondaries, two not in time order series each", chkQuerierSeries: [][]ChunkSeries{{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5}}, []tsdbutil.Sample{sample{6, 6}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, []tsdbutil.Sample{sample{2, 2}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5, nil}}, []tsdbutil.Sample{sample{6, 6, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, []tsdbutil.Sample{sample{2, 2, nil}}), }, { - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3}}, []tsdbutil.Sample{sample{4, 4}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3, nil}}, []tsdbutil.Sample{sample{4, 4, nil}}), }}, expected: NewMockChunkSeriesSet( NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, - []tsdbutil.Sample{sample{3, 3}}, - []tsdbutil.Sample{sample{5, 5}}, - []tsdbutil.Sample{sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, + []tsdbutil.Sample{sample{3, 3, nil}}, + []tsdbutil.Sample{sample{5, 5, nil}}, + []tsdbutil.Sample{sample{6, 6, nil}}, ), NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), - []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, - []tsdbutil.Sample{sample{2, 2}}, - []tsdbutil.Sample{sample{3, 3}}, - []tsdbutil.Sample{sample{4, 4}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, + []tsdbutil.Sample{sample{2, 2, nil}}, + []tsdbutil.Sample{sample{3, 3, nil}}, + []tsdbutil.Sample{sample{4, 4, nil}}, ), ), }, { name: "five secondaries, only two have two not in time order series each", chkQuerierSeries: [][]ChunkSeries{{}, {}, { - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5}}, []tsdbutil.Sample{sample{6, 6}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, []tsdbutil.Sample{sample{2, 2}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5, nil}}, []tsdbutil.Sample{sample{6, 6, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, []tsdbutil.Sample{sample{2, 2, nil}}), }, { - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3}}, []tsdbutil.Sample{sample{4, 4}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3, nil}}, []tsdbutil.Sample{sample{4, 4, nil}}), }, {}}, expected: NewMockChunkSeriesSet( NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, - []tsdbutil.Sample{sample{3, 3}}, - []tsdbutil.Sample{sample{5, 5}}, - []tsdbutil.Sample{sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, + []tsdbutil.Sample{sample{3, 3, nil}}, + []tsdbutil.Sample{sample{5, 5, nil}}, + []tsdbutil.Sample{sample{6, 6, nil}}, ), NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), - []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, - []tsdbutil.Sample{sample{2, 2}}, - []tsdbutil.Sample{sample{3, 3}}, - []tsdbutil.Sample{sample{4, 4}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, + []tsdbutil.Sample{sample{2, 2, nil}}, + []tsdbutil.Sample{sample{3, 3, nil}}, + []tsdbutil.Sample{sample{4, 4, nil}}, ), ), }, { name: "two secondaries, with two not in time order series each, with 3 noop queries and one nil together", chkQuerierSeries: [][]ChunkSeries{{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5}}, []tsdbutil.Sample{sample{6, 6}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, []tsdbutil.Sample{sample{2, 2}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{5, 5, nil}}, []tsdbutil.Sample{sample{6, 6, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, []tsdbutil.Sample{sample{2, 2, nil}}), }, { - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3}}, []tsdbutil.Sample{sample{4, 4}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{3, 3, nil}}, []tsdbutil.Sample{sample{4, 4, nil}}), }}, extraQueriers: []ChunkQuerier{NoopChunkedQuerier(), NoopChunkedQuerier(), nil, NoopChunkedQuerier()}, expected: NewMockChunkSeriesSet( NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, - []tsdbutil.Sample{sample{3, 3}}, - []tsdbutil.Sample{sample{5, 5}}, - []tsdbutil.Sample{sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, + []tsdbutil.Sample{sample{3, 3, nil}}, + []tsdbutil.Sample{sample{5, 5, nil}}, + []tsdbutil.Sample{sample{6, 6, nil}}, ), NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), - []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, - []tsdbutil.Sample{sample{2, 2}}, - []tsdbutil.Sample{sample{3, 3}}, - []tsdbutil.Sample{sample{4, 4}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, + []tsdbutil.Sample{sample{2, 2, nil}}, + []tsdbutil.Sample{sample{3, 3, nil}}, + []tsdbutil.Sample{sample{4, 4, nil}}, ), ), }, { name: "two queries, one with NaN samples series", chkQuerierSeries: [][]ChunkSeries{{ - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN()}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN(), nil}}), }, { - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{1, 1}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{1, 1, nil}}), }}, expected: NewMockChunkSeriesSet( - NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN()}}, []tsdbutil.Sample{sample{1, 1}}), + NewListChunkSeriesFromSamples(labels.FromStrings("foo", "bar"), []tsdbutil.Sample{sample{0, math.NaN(), nil}}, []tsdbutil.Sample{sample{1, 1, nil}}), ), }, } { @@ -399,9 +399,9 @@ func TestCompactingChunkSeriesMerger(t *testing.T) { { name: "single series", input: []ChunkSeries{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), }, - expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}}), }, { name: "two empty series", @@ -414,55 +414,55 @@ func TestCompactingChunkSeriesMerger(t *testing.T) { { name: "two non overlapping", input: []ChunkSeries{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}, sample{5, 5}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{7, 7}, sample{9, 9}}, []tsdbutil.Sample{sample{10, 10}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}, sample{5, 5, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{7, 7, nil}, sample{9, 9, nil}}, []tsdbutil.Sample{sample{10, 10, nil}}), }, - expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}, sample{5, 5}}, []tsdbutil.Sample{sample{7, 7}, sample{9, 9}}, []tsdbutil.Sample{sample{10, 10}}), + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}, sample{5, 5, nil}}, []tsdbutil.Sample{sample{7, 7, nil}, sample{9, 9, nil}}, []tsdbutil.Sample{sample{10, 10, nil}}), }, { name: "two overlapping", input: []ChunkSeries{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}, sample{8, 8}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{7, 7}, sample{9, 9}}, []tsdbutil.Sample{sample{10, 10}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}, sample{8, 8, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{7, 7, nil}, sample{9, 9, nil}}, []tsdbutil.Sample{sample{10, 10, nil}}), }, - expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}, sample{7, 7}, sample{8, 8}, sample{9, 9}}, []tsdbutil.Sample{sample{10, 10}}), + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, []tsdbutil.Sample{sample{3, 3, nil}, sample{7, 7, nil}, sample{8, 8, nil}, sample{9, 9, nil}}, []tsdbutil.Sample{sample{10, 10, nil}}), }, { name: "two duplicated", input: []ChunkSeries{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 2}, sample{3, 3}, sample{5, 5}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}}), }, - expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}}), + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}}), }, { name: "three overlapping", input: []ChunkSeries{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 2}, sample{3, 3}, sample{6, 6}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{0, 0}, sample{4, 4}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}, sample{6, 6, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{0, 0, nil}, sample{4, 4, nil}}), }, - expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{4, 4}, sample{5, 5}, sample{6, 6}}), + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}, sample{5, 5, nil}, sample{6, 6, nil}}), }, { name: "three in chained overlap", input: []ChunkSeries{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{4, 4}, sample{6, 66}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{6, 6}, sample{10, 10}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 5, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{4, 4, nil}, sample{6, 66, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{6, 6, nil}, sample{10, 10, nil}}), }, - expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{4, 4}, sample{5, 5}, sample{6, 66}, sample{10, 10}}), + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}, sample{5, 5, nil}, sample{6, 66, nil}, sample{10, 10, nil}}), }, { name: "three in chained overlap complex", input: []ChunkSeries{ - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{0, 0}, sample{5, 5}}, []tsdbutil.Sample{sample{10, 10}, sample{15, 15}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 2}, sample{20, 20}}, []tsdbutil.Sample{sample{25, 25}, sample{30, 30}}), - NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{18, 18}, sample{26, 26}}, []tsdbutil.Sample{sample{31, 31}, sample{35, 35}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{0, 0, nil}, sample{5, 5, nil}}, []tsdbutil.Sample{sample{10, 10, nil}, sample{15, 15, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{2, 2, nil}, sample{20, 20, nil}}, []tsdbutil.Sample{sample{25, 25, nil}, sample{30, 30, nil}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{18, 18, nil}, sample{26, 26, nil}}, []tsdbutil.Sample{sample{31, 31, nil}, sample{35, 35, nil}}), }, expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), - []tsdbutil.Sample{sample{0, 0}, sample{2, 2}, sample{5, 5}, sample{10, 10}, sample{15, 15}, sample{18, 18}, sample{20, 20}, sample{25, 25}, sample{26, 26}, sample{30, 30}}, - []tsdbutil.Sample{sample{31, 31}, sample{35, 35}}, + []tsdbutil.Sample{sample{0, 0, nil}, sample{2, 2, nil}, sample{5, 5, nil}, sample{10, 10, nil}, sample{15, 15, nil}, sample{18, 18, nil}, sample{20, 20, nil}, sample{25, 25, nil}, sample{26, 26, nil}, sample{30, 30, nil}}, + []tsdbutil.Sample{sample{31, 31, nil}, sample{35, 35, nil}}, ), }, { @@ -598,37 +598,37 @@ func TestChainSampleIterator(t *testing.T) { }{ { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{1, 1}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{1, 1, nil}}), }, - expected: []tsdbutil.Sample{sample{0, 0}, sample{1, 1}}, + expected: []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}}, }, { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{1, 1}}), - NewListSeriesIterator(samples{sample{2, 2}, sample{3, 3}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{1, 1, nil}}), + NewListSeriesIterator(samples{sample{2, 2, nil}, sample{3, 3, nil}}), }, - expected: []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}}, + expected: []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}, }, { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{3, 3}}), - NewListSeriesIterator(samples{sample{1, 1}, sample{4, 4}}), - NewListSeriesIterator(samples{sample{2, 2}, sample{5, 5}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{3, 3, nil}}), + NewListSeriesIterator(samples{sample{1, 1, nil}, sample{4, 4, nil}}), + NewListSeriesIterator(samples{sample{2, 2, nil}, sample{5, 5, nil}}), }, expected: []tsdbutil.Sample{ - sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{4, 4}, sample{5, 5}}, + sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}, sample{5, 5, nil}}, }, // Overlap. { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{1, 1}}), - NewListSeriesIterator(samples{sample{0, 0}, sample{2, 2}}), - NewListSeriesIterator(samples{sample{2, 2}, sample{3, 3}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{1, 1, nil}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{2, 2, nil}}), + NewListSeriesIterator(samples{sample{2, 2, nil}, sample{3, 3, nil}}), NewListSeriesIterator(samples{}), NewListSeriesIterator(samples{}), NewListSeriesIterator(samples{}), }, - expected: []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}}, + expected: []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}, }, } { merged := NewChainSampleIterator(tc.input) @@ -646,42 +646,42 @@ func TestChainSampleIteratorSeek(t *testing.T) { }{ { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }, seek: 1, - expected: []tsdbutil.Sample{sample{1, 1}, sample{2, 2}}, + expected: []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}}, }, { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{1, 1}}), - NewListSeriesIterator(samples{sample{2, 2}, sample{3, 3}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{1, 1, nil}}), + NewListSeriesIterator(samples{sample{2, 2, nil}, sample{3, 3, nil}}), }, seek: 2, - expected: []tsdbutil.Sample{sample{2, 2}, sample{3, 3}}, + expected: []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}}, }, { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{3, 3}}), - NewListSeriesIterator(samples{sample{1, 1}, sample{4, 4}}), - NewListSeriesIterator(samples{sample{2, 2}, sample{5, 5}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{3, 3, nil}}), + NewListSeriesIterator(samples{sample{1, 1, nil}, sample{4, 4, nil}}), + NewListSeriesIterator(samples{sample{2, 2, nil}, sample{5, 5, nil}}), }, seek: 2, - expected: []tsdbutil.Sample{sample{2, 2}, sample{3, 3}, sample{4, 4}, sample{5, 5}}, + expected: []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}, sample{4, 4, nil}, sample{5, 5, nil}}, }, { input: []chunkenc.Iterator{ - NewListSeriesIterator(samples{sample{0, 0}, sample{2, 2}, sample{3, 3}}), - NewListSeriesIterator(samples{sample{0, 0}, sample{1, 1}, sample{2, 2}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{2, 2, nil}, sample{3, 3, nil}}), + NewListSeriesIterator(samples{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}}), }, seek: 0, - expected: []tsdbutil.Sample{sample{0, 0}, sample{1, 1}, sample{2, 2}, sample{3, 3}}, + expected: []tsdbutil.Sample{sample{0, 0, nil}, sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}, }, } { merged := NewChainSampleIterator(tc.input) actual := []tsdbutil.Sample{} if merged.Seek(tc.seek) { t, v := merged.At() - actual = append(actual, sample{t, v}) + actual = append(actual, sample{t, v, nil}) } s, err := ExpandSamples(merged, nil) require.NoError(t, err) diff --git a/storage/remote/codec.go b/storage/remote/codec.go index 9e414ad83..666e847f8 100644 --- a/storage/remote/codec.go +++ b/storage/remote/codec.go @@ -370,11 +370,11 @@ func (c *concreteSeriesIterator) At() (t int64, v float64) { return s.Timestamp, s.Value } -// AtHistogram always returns (0, histogram.Histogram{}) because there is no -// support for histogram values yet. +// AtHistogram always returns (0, nil) because there is no support for histogram +// values yet. // TODO(beorn7): Fix that for histogram support in remote storage. -func (c *concreteSeriesIterator) AtHistogram() (int64, histogram.Histogram) { - return 0, histogram.Histogram{} +func (c *concreteSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { + return 0, nil } func (c *concreteSeriesIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/storage/remote/write.go b/storage/remote/write.go index 69fe36361..c47bc0e98 100644 --- a/storage/remote/write.go +++ b/storage/remote/write.go @@ -241,7 +241,7 @@ func (t *timestampTracker) AppendExemplar(_ uint64, _ labels.Labels, _ exemplar. return 0, nil } -func (t *timestampTracker) AppendHistogram(_ uint64, _ labels.Labels, ts int64, _ histogram.Histogram) (uint64, error) { +func (t *timestampTracker) AppendHistogram(_ uint64, _ labels.Labels, ts int64, _ *histogram.Histogram) (uint64, error) { t.histograms++ if ts > t.highestTimestamp { t.highestTimestamp = ts diff --git a/storage/remote/write_handler_test.go b/storage/remote/write_handler_test.go index 72603f297..2d91d4f41 100644 --- a/storage/remote/write_handler_test.go +++ b/storage/remote/write_handler_test.go @@ -188,7 +188,7 @@ func (m *mockAppendable) AppendExemplar(_ uint64, l labels.Labels, e exemplar.Ex return 0, nil } -func (*mockAppendable) AppendHistogram(ref uint64, l labels.Labels, t int64, h histogram.Histogram) (uint64, error) { +func (*mockAppendable) AppendHistogram(ref uint64, l labels.Labels, t int64, h *histogram.Histogram) (uint64, error) { // TODO(beorn7): Noop until we implement sparse histograms over remote write. return 0, nil } diff --git a/storage/series.go b/storage/series.go index 6e5b178df..cd3531efb 100644 --- a/storage/series.go +++ b/storage/series.go @@ -91,9 +91,9 @@ func (it *listSeriesIterator) At() (int64, float64) { return s.T(), s.V() } -func (it *listSeriesIterator) AtHistogram() (int64, histogram.Histogram) { +func (it *listSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { s := it.samples.Get(it.idx) - return s.T(), *s.H() + return s.T(), s.H() } func (it *listSeriesIterator) ChunkEncoding() chunkenc.Encoding { @@ -302,13 +302,12 @@ func ExpandSamples(iter chunkenc.Iterator, newSampleFn func(t int64, v float64, } var result []tsdbutil.Sample - if iter.ChunkEncoding() == chunkenc.EncHistogram { - for iter.Next() { + for iter.Next() { + // Only after Next() returned true, it is safe to ask for the ChunkEncoding. + if iter.ChunkEncoding() == chunkenc.EncHistogram { t, h := iter.AtHistogram() - result = append(result, newSampleFn(t, 0, &h)) - } - } else { - for iter.Next() { + result = append(result, newSampleFn(t, 0, h)) + } else { t, v := iter.At() // NaNs can't be compared normally, so substitute for another value. if math.IsNaN(v) { diff --git a/tsdb/block_test.go b/tsdb/block_test.go index 54cfbc2c4..31ba76ea6 100644 --- a/tsdb/block_test.go +++ b/tsdb/block_test.go @@ -179,7 +179,7 @@ func TestCorruptedChunk(t *testing.T) { require.NoError(t, os.RemoveAll(tmpdir)) }() - series := storage.NewListSeries(labels.FromStrings("a", "b"), []tsdbutil.Sample{sample{1, 1}}) + series := storage.NewListSeries(labels.FromStrings("a", "b"), []tsdbutil.Sample{sample{1, 1, nil}}) blockDir := createBlock(t, tmpdir, []storage.Series{series}) files, err := sequenceFiles(chunkDir(blockDir)) require.NoError(t, err) @@ -226,7 +226,7 @@ func TestLabelValuesWithMatchers(t *testing.T) { seriesEntries = append(seriesEntries, storage.NewListSeries(labels.Labels{ {Name: "unique", Value: fmt.Sprintf("value%d", i)}, {Name: "tens", Value: fmt.Sprintf("value%d", i/10)}, - }, []tsdbutil.Sample{sample{100, 0}})) + }, []tsdbutil.Sample{sample{100, 0, nil}})) } blockDir := createBlock(t, tmpdir, seriesEntries) @@ -389,7 +389,7 @@ func BenchmarkLabelValuesWithMatchers(b *testing.B) { {Name: "unique", Value: fmt.Sprintf("value%d", i)}, {Name: "tens", Value: fmt.Sprintf("value%d", i/(metricCount/10))}, {Name: "ninety", Value: fmt.Sprintf("value%d", i/(metricCount/10)/9)}, // "0" for the first 90%, then "1" - }, []tsdbutil.Sample{sample{100, 0}})) + }, []tsdbutil.Sample{sample{100, 0, nil}})) } blockDir := createBlock(b, tmpdir, seriesEntries) @@ -427,13 +427,13 @@ func TestLabelNamesWithMatchers(t *testing.T) { for i := 0; i < 100; i++ { seriesEntries = append(seriesEntries, storage.NewListSeries(labels.Labels{ {Name: "unique", Value: fmt.Sprintf("value%d", i)}, - }, []tsdbutil.Sample{sample{100, 0}})) + }, []tsdbutil.Sample{sample{100, 0, nil}})) if i%10 == 0 { seriesEntries = append(seriesEntries, storage.NewListSeries(labels.Labels{ {Name: "unique", Value: fmt.Sprintf("value%d", i)}, {Name: "tens", Value: fmt.Sprintf("value%d", i/10)}, - }, []tsdbutil.Sample{sample{100, 0}})) + }, []tsdbutil.Sample{sample{100, 0, nil}})) } if i%20 == 0 { @@ -441,7 +441,7 @@ func TestLabelNamesWithMatchers(t *testing.T) { {Name: "unique", Value: fmt.Sprintf("value%d", i)}, {Name: "tens", Value: fmt.Sprintf("value%d", i/10)}, {Name: "twenties", Value: fmt.Sprintf("value%d", i/20)}, - }, []tsdbutil.Sample{sample{100, 0}})) + }, []tsdbutil.Sample{sample{100, 0, nil}})) } } diff --git a/tsdb/chunkenc/chunk.go b/tsdb/chunkenc/chunk.go index 8069b0ecd..5fe8c08f7 100644 --- a/tsdb/chunkenc/chunk.go +++ b/tsdb/chunkenc/chunk.go @@ -82,7 +82,7 @@ type Chunk interface { // Appender adds sample pairs to a chunk. type Appender interface { Append(int64, float64) - AppendHistogram(t int64, h histogram.Histogram) + AppendHistogram(t int64, h *histogram.Histogram) } // Iterator is a simple iterator that can only get the next value. @@ -102,8 +102,7 @@ type Iterator interface { At() (int64, float64) // AtHistogram returns the current timestamp/histogram pair. // Before the iterator has advanced AtHistogram behaviour is unspecified. - // TODO(beorn7): Maybe return *histogram.Histogram? It's a fairly large struct. - AtHistogram() (int64, histogram.Histogram) + AtHistogram() (int64, *histogram.Histogram) // Err returns the current error. It should be used only after iterator is // exhausted, that is `Next` or `Seek` returns false. Err() error @@ -120,8 +119,8 @@ type nopIterator struct{} func (nopIterator) Seek(int64) bool { return false } func (nopIterator) At() (int64, float64) { return math.MinInt64, 0 } -func (nopIterator) AtHistogram() (int64, histogram.Histogram) { - return math.MinInt64, histogram.Histogram{} +func (nopIterator) AtHistogram() (int64, *histogram.Histogram) { + return math.MinInt64, nil } func (nopIterator) Next() bool { return false } func (nopIterator) Err() error { return nil } diff --git a/tsdb/chunkenc/histogram.go b/tsdb/chunkenc/histogram.go index cff7d5cc6..c7f94795c 100644 --- a/tsdb/chunkenc/histogram.go +++ b/tsdb/chunkenc/histogram.go @@ -245,7 +245,7 @@ func (a *HistogramAppender) Append(int64, float64) { // The method returns an additional boolean set to true if it is not appendable // because of a counter reset. If the given sample is stale, it is always ok to // append. If counterReset is true, okToAppend is always false. -func (a *HistogramAppender) Appendable(h histogram.Histogram) ( +func (a *HistogramAppender) Appendable(h *histogram.Histogram) ( positiveInterjections, negativeInterjections []Interjection, okToAppend bool, counterReset bool, ) { @@ -369,14 +369,14 @@ func counterResetInAnyBucket(oldBuckets, newBuckets []int64, oldSpans, newSpans // the histogram is properly structured, e.g. the number of buckets used // corresponds to the number conveyed by the span structures. First call // Appendable() and act accordingly! -func (a *HistogramAppender) AppendHistogram(t int64, h histogram.Histogram) { +func (a *HistogramAppender) AppendHistogram(t int64, h *histogram.Histogram) { var tDelta, cntDelta, zCntDelta int64 num := binary.BigEndian.Uint16(a.b.bytes()) if value.IsStaleNaN(h.Sum) { // Emptying out other fields to write no buckets, and an empty // layout in case of first histogram in the chunk. - h = histogram.Histogram{Sum: h.Sum} + h = &histogram.Histogram{Sum: h.Sum} } switch num { @@ -401,7 +401,7 @@ func (a *HistogramAppender) AppendHistogram(t int64, h histogram.Histogram) { // Now store the actual data. putVarbitInt(a.b, t) putVarbitUint(a.b, h.Count) - putVarbitUint(a.b, h.ZeroCount) // + putVarbitUint(a.b, h.ZeroCount) a.b.writeBits(math.Float64bits(h.Sum), 64) for _, b := range h.PositiveBuckets { putVarbitInt(a.b, b) @@ -582,11 +582,11 @@ func (it *histogramIterator) ChunkEncoding() Encoding { return EncHistogram } -func (it *histogramIterator) AtHistogram() (int64, histogram.Histogram) { +func (it *histogramIterator) AtHistogram() (int64, *histogram.Histogram) { if value.IsStaleNaN(it.sum) { - return it.t, histogram.Histogram{Sum: it.sum} + return it.t, &histogram.Histogram{Sum: it.sum} } - return it.t, histogram.Histogram{ + return it.t, &histogram.Histogram{ Count: it.cnt, ZeroCount: it.zCnt, Sum: it.sum, diff --git a/tsdb/chunkenc/histogram_test.go b/tsdb/chunkenc/histogram_test.go index ca5ea7239..e4a9cbe64 100644 --- a/tsdb/chunkenc/histogram_test.go +++ b/tsdb/chunkenc/histogram_test.go @@ -30,7 +30,7 @@ func TestHistogramChunkSameBuckets(t *testing.T) { require.Equal(t, 0, c.NumSamples()) ts := int64(1234567890) - h := histogram.Histogram{ + h := &histogram.Histogram{ Count: 5, ZeroCount: 2, Sum: 18.4, @@ -48,6 +48,7 @@ func TestHistogramChunkSameBuckets(t *testing.T) { // Add an updated histogram. ts += 16 + h = h.Copy() h.Count += 9 h.ZeroCount++ h.Sum = 24.4 @@ -61,6 +62,7 @@ func TestHistogramChunkSameBuckets(t *testing.T) { require.NoError(t, err) ts += 14 + h = h.Copy() h.Count += 13 h.ZeroCount += 2 h.Sum = 24.4 @@ -113,7 +115,7 @@ func TestHistogramChunkSameBuckets(t *testing.T) { type res struct { t int64 - h histogram.Histogram + h *histogram.Histogram } // Mimics the scenario described for compareSpans(). @@ -126,7 +128,7 @@ func TestHistogramChunkBucketChanges(t *testing.T) { require.Equal(t, 0, c.NumSamples()) ts1 := int64(1234567890) - h1 := histogram.Histogram{ + h1 := &histogram.Histogram{ Count: 5, ZeroCount: 2, Sum: 18.4, @@ -147,7 +149,7 @@ func TestHistogramChunkBucketChanges(t *testing.T) { // Add a new histogram that has expanded buckets. ts2 := ts1 + 16 - h2 := h1 + h2 := h1.Copy() h2.PositiveSpans = []histogram.Span{ {Offset: 0, Length: 3}, {Offset: 1, Length: 1}, @@ -202,7 +204,7 @@ func TestHistoChunkAppendable(t *testing.T) { require.Equal(t, 0, c.NumSamples()) ts := int64(1234567890) - h1 := histogram.Histogram{ + h1 := &histogram.Histogram{ Count: 5, ZeroCount: 2, Sum: 18.4, diff --git a/tsdb/chunkenc/xor.go b/tsdb/chunkenc/xor.go index 9b52ed57a..62cf4396a 100644 --- a/tsdb/chunkenc/xor.go +++ b/tsdb/chunkenc/xor.go @@ -150,7 +150,7 @@ type xorAppender struct { trailing uint8 } -func (a *xorAppender) AppendHistogram(t int64, h histogram.Histogram) { +func (a *xorAppender) AppendHistogram(t int64, h *histogram.Histogram) { panic("appended a histogram to an xor chunk") } @@ -253,8 +253,8 @@ func (it *xorIterator) At() (int64, float64) { return it.t, it.val } -func (it *xorIterator) AtHistogram() (int64, histogram.Histogram) { - panic("cannot call xorIterator.AtHistogram().") +func (it *xorIterator) AtHistogram() (int64, *histogram.Histogram) { + panic("cannot call xorIterator.AtHistogram") } func (it *xorIterator) ChunkEncoding() Encoding { diff --git a/tsdb/compact_test.go b/tsdb/compact_test.go index b0df493f7..dd9e426a5 100644 --- a/tsdb/compact_test.go +++ b/tsdb/compact_test.go @@ -1326,20 +1326,20 @@ func TestHeadCompactionWithHistograms(t *testing.T) { require.NoError(t, head.Init(0)) app := head.Appender(context.Background()) - type timedHist struct { + type timedHistogram struct { t int64 - h histogram.Histogram + h *histogram.Histogram } // Ingest samples. numHistograms := 120 * 4 timeStep := DefaultBlockDuration / int64(numHistograms) - expHists := make([]timedHist, 0, numHistograms) + expHists := make([]timedHistogram, 0, numHistograms) l := labels.Labels{{Name: "a", Value: "b"}} for i, h := range GenerateTestHistograms(numHistograms) { _, err := app.AppendHistogram(0, l, int64(i)*timeStep, h) require.NoError(t, err) - expHists = append(expHists, timedHist{int64(i) * timeStep, h}) + expHists = append(expHists, timedHistogram{int64(i) * timeStep, h}) } require.NoError(t, app.Commit()) @@ -1372,10 +1372,10 @@ func TestHeadCompactionWithHistograms(t *testing.T) { require.False(t, ss.Next()) it := s.Iterator() - actHists := make([]timedHist, 0, len(expHists)) + actHists := make([]timedHistogram, 0, len(expHists)) for it.Next() { t, h := it.AtHistogram() - actHists = append(actHists, timedHist{t, h.Copy()}) + actHists = append(actHists, timedHistogram{t, h.Copy()}) } require.Equal(t, expHists, actHists) @@ -1455,7 +1455,7 @@ func TestSparseHistogramSpaceSavings(t *testing.T) { var allSparseSeries []struct { baseLabels labels.Labels - hists []histogram.Histogram + hists []*histogram.Histogram } for sid, schema := range allSchemas { @@ -1467,7 +1467,7 @@ func TestSparseHistogramSpaceSavings(t *testing.T) { } allSparseSeries = append(allSparseSeries, struct { baseLabels labels.Labels - hists []histogram.Histogram + hists []*histogram.Histogram }{baseLabels: lbls, hists: generateCustomHistograms(numHistograms, c.numBuckets, c.numSpans, c.gapBetweenSpans, schema)}) } } @@ -1613,9 +1613,9 @@ Savings: Index=%.2f%%, Chunks=%.2f%%, Total=%.2f%% } } -func generateCustomHistograms(numHists, numBuckets, numSpans, gapBetweenSpans, schema int) (r []histogram.Histogram) { +func generateCustomHistograms(numHists, numBuckets, numSpans, gapBetweenSpans, schema int) (r []*histogram.Histogram) { // First histogram with all the settings. - h := histogram.Histogram{ + h := &histogram.Histogram{ Sum: 1000 * rand.Float64(), Schema: int32(schema), } @@ -1708,11 +1708,11 @@ func TestSparseHistogramCompactionAndQuery(t *testing.T) { }) db.DisableCompactions() - type timedHist struct { + type timedHistogram struct { t int64 - h histogram.Histogram + h *histogram.Histogram } - expHists := make(map[string][]timedHist) + expHists := make(map[string][]timedHistogram) series1Histograms := GenerateTestHistograms(20) series2Histograms := GenerateTestHistograms(20) @@ -1728,8 +1728,8 @@ func TestSparseHistogramCompactionAndQuery(t *testing.T) { require.NoError(t, err) l1, l2 := lbls1.String(), lbls2.String() - expHists[l1] = append(expHists[l1], timedHist{t: ts, h: series1Histograms[idx1]}) - expHists[l2] = append(expHists[l2], timedHist{t: ts, h: series2Histograms[idx2]}) + expHists[l1] = append(expHists[l1], timedHistogram{t: ts, h: series1Histograms[idx1]}) + expHists[l2] = append(expHists[l2], timedHistogram{t: ts, h: series2Histograms[idx2]}) } testQuery := func() { @@ -1740,13 +1740,13 @@ func TestSparseHistogramCompactionAndQuery(t *testing.T) { }() ss := q.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, "a", ".*")) - actHists := make(map[string][]timedHist) + actHists := make(map[string][]timedHistogram) for ss.Next() { s := ss.At() it := s.Iterator() for it.Next() { ts, h := it.AtHistogram() - actHists[s.Labels().String()] = append(actHists[s.Labels().String()], timedHist{ts, h.Copy()}) + actHists[s.Labels().String()] = append(actHists[s.Labels().String()], timedHistogram{ts, h.Copy()}) } require.NoError(t, it.Err()) } diff --git a/tsdb/db_test.go b/tsdb/db_test.go index 26ead3fc3..baa8dd4ea 100644 --- a/tsdb/db_test.go +++ b/tsdb/db_test.go @@ -420,7 +420,7 @@ Outer: expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) for _, ts := range c.remaint { - expSamples = append(expSamples, sample{ts, smpls[ts]}) + expSamples = append(expSamples, sample{ts, smpls[ts], nil}) } expss := newMockSeriesSet([]storage.Series{ @@ -536,7 +536,7 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) { ssMap := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) require.Equal(t, map[string][]tsdbutil.Sample{ - labels.New(labels.Label{Name: "a", Value: "b"}).String(): {sample{0, 1}}, + labels.New(labels.Label{Name: "a", Value: "b"}).String(): {sample{0, 1, nil}}, }, ssMap) // Append Out of Order Value. @@ -553,7 +553,7 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) { ssMap = query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) require.Equal(t, map[string][]tsdbutil.Sample{ - labels.New(labels.Label{Name: "a", Value: "b"}).String(): {sample{0, 1}, sample{10, 3}}, + labels.New(labels.Label{Name: "a", Value: "b"}).String(): {sample{0, 1, nil}, sample{10, 3, nil}}, }, ssMap) } @@ -716,7 +716,7 @@ Outer: expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) for _, ts := range c.remaint { - expSamples = append(expSamples, sample{ts, smpls[ts]}) + expSamples = append(expSamples, sample{ts, smpls[ts], nil}) } expss := newMockSeriesSet([]storage.Series{ @@ -821,7 +821,7 @@ func TestDB_e2e(t *testing.T) { for i := 0; i < numDatapoints; i++ { v := rand.Float64() - series = append(series, sample{ts, v}) + series = append(series, sample{ts, v, nil}) _, err := app.Append(0, lset, ts, v) require.NoError(t, err) @@ -1066,7 +1066,7 @@ func TestTombstoneClean(t *testing.T) { expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) for _, ts := range c.remaint { - expSamples = append(expSamples, sample{ts, smpls[ts]}) + expSamples = append(expSamples, sample{ts, smpls[ts], nil}) } expss := newMockSeriesSet([]storage.Series{ @@ -2541,11 +2541,11 @@ func TestDBQueryDoesntSeeAppendsAfterCreation(t *testing.T) { // TestChunkWriter_ReadAfterWrite ensures that chunk segment are cut at the set segment size and // that the resulted segments includes the expected chunks data. func TestChunkWriter_ReadAfterWrite(t *testing.T) { - chk1 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 1}}) - chk2 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 2}}) - chk3 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 3}}) - chk4 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 4}}) - chk5 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 5}}) + chk1 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 1, nil}}) + chk2 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 2, nil}}) + chk3 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 3, nil}}) + chk4 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 4, nil}}) + chk5 := tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 5, nil}}) chunkSize := len(chk1.Chunk.Bytes()) + chunks.MaxChunkLengthFieldSize + chunks.ChunkEncodingSize + crc32.Size tests := []struct { @@ -2746,11 +2746,11 @@ func TestRangeForTimestamp(t *testing.T) { // Regression test for https://github.com/prometheus/prometheus/pull/6514. func TestChunkReader_ConcurrentReads(t *testing.T) { chks := []chunks.Meta{ - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 1}}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 2}}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 3}}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 4}}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 5}}), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 1, nil}}), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 2, nil}}), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 3, nil}}), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 4, nil}}), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 5, nil}}), } tempDir, err := ioutil.TempDir("", "test_chunk_writer") @@ -2815,7 +2815,7 @@ func TestCompactHead(t *testing.T) { val := rand.Float64() _, err := app.Append(0, labels.FromStrings("a", "b"), int64(i), val) require.NoError(t, err) - expSamples = append(expSamples, sample{int64(i), val}) + expSamples = append(expSamples, sample{int64(i), val, nil}) } require.NoError(t, app.Commit()) @@ -2842,7 +2842,7 @@ func TestCompactHead(t *testing.T) { series := seriesSet.At().Iterator() for series.Next() { time, val := series.At() - actSamples = append(actSamples, sample{int64(time), val}) + actSamples = append(actSamples, sample{int64(time), val, nil}) } require.NoError(t, series.Err()) } diff --git a/tsdb/head.go b/tsdb/head.go index 93e6d568b..7e95b4bf3 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -1490,7 +1490,7 @@ func (s *stripeSeries) getOrSet(hash uint64, lset labels.Labels, createSeries fu type histogramSample struct { t int64 - h histogram.Histogram + h *histogram.Histogram } type sample struct { @@ -1499,10 +1499,11 @@ type sample struct { h *histogram.Histogram } -func newSample(t int64, v float64) tsdbutil.Sample { return sample{t, v, nil} } -func (s sample) T() int64 { return s.t } -func (s sample) V() float64 { return s.v } -func (s sample) H() *histogram.Histogram { return s.h } +func newSample(t int64, v float64, h *histogram.Histogram) tsdbutil.Sample { return sample{t, v, h} } + +func (s sample) T() int64 { return s.t } +func (s sample) V() float64 { return s.v } +func (s sample) H() *histogram.Histogram { return s.h } // memSeries is the in-memory representation of a series. None of its methods // are goroutine safe and it is the caller's responsibility to lock it. @@ -1661,9 +1662,9 @@ func (h *Head) updateWALReplayStatusRead(current int) { h.stats.WALReplayStatus.Current = current } -func GenerateTestHistograms(n int) (r []histogram.Histogram) { +func GenerateTestHistograms(n int) (r []*histogram.Histogram) { for i := 0; i < n; i++ { - r = append(r, histogram.Histogram{ + r = append(r, &histogram.Histogram{ Count: 5 + uint64(i*4), ZeroCount: 2 + uint64(i), ZeroThreshold: 0.001, diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 6ff989f5e..14a7db77d 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -67,7 +67,7 @@ func (a *initAppender) AppendExemplar(ref uint64, l labels.Labels, e exemplar.Ex return a.app.AppendExemplar(ref, l, e) } -func (a *initAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, h histogram.Histogram) (uint64, error) { +func (a *initAppender) AppendHistogram(ref uint64, l labels.Labels, t int64, h *histogram.Histogram) (uint64, error) { if a.app != nil { return a.app.AppendHistogram(ref, l, t, h) } @@ -270,7 +270,7 @@ func (a *headAppender) Append(ref uint64, lset labels.Labels, t int64, v float64 } if value.IsStaleNaN(v) && s.histogramSeries { - return a.AppendHistogram(ref, lset, t, histogram.Histogram{Sum: v}) + return a.AppendHistogram(ref, lset, t, &histogram.Histogram{Sum: v}) } s.Lock() @@ -322,7 +322,7 @@ func (s *memSeries) appendable(t int64, v float64) error { } // appendableHistogram checks whether the given sample is valid for appending to the series. -func (s *memSeries) appendableHistogram(t int64, sh histogram.Histogram) error { +func (s *memSeries) appendableHistogram(t int64, h *histogram.Histogram) error { c := s.head() if c == nil { return nil @@ -334,7 +334,7 @@ func (s *memSeries) appendableHistogram(t int64, sh histogram.Histogram) error { if t < c.maxTime { return storage.ErrOutOfOrderSample } - // TODO: do it for histogram. + // TODO(beorn7): do it for histogram. // We are allowing exact duplicates as we can encounter them in valid cases // like federation and erroring out at that time would be extremely noisy. //if math.Float64bits(s.sampleBuf[3].v) != math.Float64bits(v) { @@ -372,7 +372,7 @@ func (a *headAppender) AppendExemplar(ref uint64, _ labels.Labels, e exemplar.Ex return s.ref, nil } -func (a *headAppender) AppendHistogram(ref uint64, lset labels.Labels, t int64, h histogram.Histogram) (uint64, error) { +func (a *headAppender) AppendHistogram(ref uint64, lset labels.Labels, t int64, h *histogram.Histogram) (uint64, error) { if t < a.minValidTime { a.head.metrics.outOfBoundSamples.Inc() return 0, storage.ErrOutOfBounds @@ -606,7 +606,7 @@ func (s *memSeries) append(t int64, v float64, appendID uint64, chunkDiskMapper // appendHistogram adds the histogram. // It is unsafe to call this concurrently with s.iterator(...) without holding the series lock. -func (s *memSeries) appendHistogram(t int64, h histogram.Histogram, appendID uint64, chunkDiskMapper *chunks.ChunkDiskMapper) (sampleInOrder, chunkCreated bool) { +func (s *memSeries) appendHistogram(t int64, h *histogram.Histogram, appendID uint64, chunkDiskMapper *chunks.ChunkDiskMapper) (sampleInOrder, chunkCreated bool) { // Head controls the execution of recoding, so that we own the proper chunk reference afterwards. // We check for Appendable before appendPreprocessor because in case it ends up creating a new chunk, // we need to know if there was also a counter reset or not to set the meta properly. diff --git a/tsdb/head_read.go b/tsdb/head_read.go index 7c20c43ad..48a294ade 100644 --- a/tsdb/head_read.go +++ b/tsdb/head_read.go @@ -519,7 +519,7 @@ func (it *memSafeIterator) At() (int64, float64) { return s.t, s.v } -func (it *memSafeIterator) AtHistogram() (int64, histogram.Histogram) { +func (it *memSafeIterator) AtHistogram() (int64, *histogram.Histogram) { if it.total-it.i > 4 { return it.Iterator.AtHistogram() } diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 7831ad9d8..a8cb10dc4 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -331,11 +331,11 @@ func TestHead_ReadWAL(t *testing.T) { require.NoError(t, c.Err()) return x } - require.Equal(t, []sample{{100, 2}, {101, 5}}, expandChunk(s10.iterator(0, nil, head.chunkDiskMapper, nil))) - require.Equal(t, []sample{{101, 6}}, expandChunk(s50.iterator(0, nil, head.chunkDiskMapper, nil))) + require.Equal(t, []sample{{100, 2, nil}, {101, 5, nil}}, expandChunk(s10.iterator(0, nil, head.chunkDiskMapper, nil))) + require.Equal(t, []sample{{101, 6, nil}}, expandChunk(s50.iterator(0, nil, head.chunkDiskMapper, nil))) // The samples before the new series record should be discarded since a duplicate record // is only possible when old samples were compacted. - require.Equal(t, []sample{{101, 7}}, expandChunk(s100.iterator(0, nil, head.chunkDiskMapper, nil))) + require.Equal(t, []sample{{101, 7, nil}}, expandChunk(s100.iterator(0, nil, head.chunkDiskMapper, nil))) q, err := head.ExemplarQuerier(context.Background()) require.NoError(t, err) @@ -401,8 +401,8 @@ func TestHead_WALMultiRef(t *testing.T) { // The samples before the new ref should be discarded since Head truncation // happens only after compacting the Head. require.Equal(t, map[string][]tsdbutil.Sample{`{foo="bar"}`: { - sample{1700, 3}, - sample{2000, 4}, + sample{1700, 3, nil}, + sample{2000, 4, nil}, }}, series) } @@ -798,7 +798,7 @@ func TestDeleteUntilCurMax(t *testing.T) { it = exps.Iterator() resSamples, err := storage.ExpandSamples(it, newSample) require.NoError(t, err) - require.Equal(t, []tsdbutil.Sample{sample{11, 1}}, resSamples) + require.Equal(t, []tsdbutil.Sample{sample{11, 1, nil}}, resSamples) for res.Next() { } require.NoError(t, res.Err()) @@ -913,7 +913,7 @@ func TestDelete_e2e(t *testing.T) { v := rand.Float64() _, err := app.Append(0, ls, ts, v) require.NoError(t, err) - series = append(series, sample{ts, v}) + series = append(series, sample{ts, v, nil}) ts += rand.Int63n(timeInterval) + 1 } seriesMap[labels.New(l...).String()] = series @@ -2397,7 +2397,7 @@ func TestDataMissingOnQueryDuringCompaction(t *testing.T) { ref, err = app.Append(ref, labels.FromStrings("a", "b"), ts, float64(i)) require.NoError(t, err) maxt = ts - expSamples = append(expSamples, sample{ts, float64(i)}) + expSamples = append(expSamples, sample{ts, float64(i), nil}) } require.NoError(t, app.Commit()) @@ -2541,7 +2541,7 @@ func TestAppendHistogram(t *testing.T) { type timedHistogram struct { t int64 - h histogram.Histogram + h *histogram.Histogram } expHistograms := make([]timedHistogram, 0, numHistograms) for i, h := range GenerateTestHistograms(numHistograms) { @@ -2588,7 +2588,7 @@ func TestHistogramInWAL(t *testing.T) { type timedHistogram struct { t int64 - h histogram.Histogram + h *histogram.Histogram } expHistograms := make([]timedHistogram, 0, numHistograms) for i, h := range GenerateTestHistograms(numHistograms) { @@ -2728,7 +2728,7 @@ func TestChunkSnapshot(t *testing.T) { // 240 samples should m-map at least 1 chunk. for ts := int64(1); ts <= 240; ts++ { val := rand.Float64() - expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val}) + expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil}) ref, err := app.Append(0, lbls, ts, val) require.NoError(t, err) @@ -2788,7 +2788,7 @@ func TestChunkSnapshot(t *testing.T) { // 240 samples should m-map at least 1 chunk. for ts := int64(241); ts <= 480; ts++ { val := rand.Float64() - expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val}) + expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil}) ref, err := app.Append(0, lbls, ts, val) require.NoError(t, err) @@ -2977,7 +2977,7 @@ func TestHistogramStaleSample(t *testing.T) { type timedHistogram struct { t int64 - h histogram.Histogram + h *histogram.Histogram } expHistograms := make([]timedHistogram, 0, numHistograms) @@ -3011,6 +3011,7 @@ func TestHistogramStaleSample(t *testing.T) { require.True(t, value.IsStaleNaN(ah.h.Sum)) // To make require.Equal work. ah.h.Sum = 0 + eh.h = eh.h.Copy() eh.h.Sum = 0 } require.Equal(t, eh, ah) @@ -3028,7 +3029,7 @@ func TestHistogramStaleSample(t *testing.T) { // +1 so that delta-of-delta is not 0. _, err := app.Append(0, l, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN)) require.NoError(t, err) - expHistograms = append(expHistograms, timedHistogram{100*int64(len(expHistograms)) + 1, histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}}) + expHistograms = append(expHistograms, timedHistogram{100*int64(len(expHistograms)) + 1, &histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}}) require.NoError(t, app.Commit()) // Only 1 chunk in the memory, no m-mapped chunk. @@ -3050,7 +3051,7 @@ func TestHistogramStaleSample(t *testing.T) { // +1 so that delta-of-delta is not 0. _, err = app.Append(0, l, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN)) require.NoError(t, err) - expHistograms = append(expHistograms, timedHistogram{100*int64(len(expHistograms)) + 1, histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}}) + expHistograms = append(expHistograms, timedHistogram{100*int64(len(expHistograms)) + 1, &histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}}) require.NoError(t, app.Commit()) // Total 2 chunks, 1 m-mapped. @@ -3069,7 +3070,7 @@ func TestHistogramCounterResetHeader(t *testing.T) { require.NoError(t, head.Init(0)) ts := int64(0) - appendHistogram := func(h histogram.Histogram) { + appendHistogram := func(h *histogram.Histogram) { ts++ app := head.Appender(context.Background()) _, err := app.AppendHistogram(0, l, ts, h) diff --git a/tsdb/querier.go b/tsdb/querier.go index ec6310da5..5739fbdc5 100644 --- a/tsdb/querier.go +++ b/tsdb/querier.go @@ -649,9 +649,11 @@ func (p *populateWithDelSeriesIterator) Seek(t int64) bool { } func (p *populateWithDelSeriesIterator) At() (int64, float64) { return p.curr.At() } -func (p *populateWithDelSeriesIterator) AtHistogram() (int64, histogram.Histogram) { + +func (p *populateWithDelSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { return p.curr.AtHistogram() } + func (p *populateWithDelSeriesIterator) ChunkEncoding() chunkenc.Encoding { return p.curr.ChunkEncoding() } @@ -714,7 +716,7 @@ func (p *populateWithDelChunkSeriesIterator) Next() bool { var ( t int64 v float64 - h histogram.Histogram + h *histogram.Histogram ) if p.currDelIter.ChunkEncoding() == chunkenc.EncHistogram { if hc, ok := p.currChkMeta.Chunk.(*chunkenc.HistogramChunk); ok { @@ -870,7 +872,7 @@ func (it *DeletedIterator) At() (int64, float64) { return it.Iter.At() } -func (it *DeletedIterator) AtHistogram() (int64, histogram.Histogram) { +func (it *DeletedIterator) AtHistogram() (int64, *histogram.Histogram) { t, h := it.Iter.AtHistogram() return t, h } diff --git a/tsdb/querier_test.go b/tsdb/querier_test.go index 514b1f5b9..9ae34e726 100644 --- a/tsdb/querier_test.go +++ b/tsdb/querier_test.go @@ -277,24 +277,24 @@ func TestBlockQuerier(t *testing.T) { ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "a", ".*")}, exp: newMockSeriesSet([]storage.Series{ storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{1, 2}, sample{2, 3}, sample{3, 4}, sample{5, 2}, sample{6, 3}, sample{7, 4}}, + []tsdbutil.Sample{sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 4, nil}, sample{5, 2, nil}, sample{6, 3, nil}, sample{7, 4, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 3, nil}, sample{6, 6, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}, sample{5, 1}, sample{6, 7}, sample{7, 2}}, + []tsdbutil.Sample{sample{1, 3, nil}, sample{2, 2, nil}, sample{3, 6, nil}, sample{5, 1, nil}, sample{6, 7, nil}, sample{7, 2, nil}}, ), }), expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{1, 2}, sample{2, 3}, sample{3, 4}}, []tsdbutil.Sample{sample{5, 2}, sample{6, 3}, sample{7, 4}}, + []tsdbutil.Sample{sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 4, nil}}, []tsdbutil.Sample{sample{5, 2, nil}, sample{6, 3, nil}, sample{7, 4, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}, []tsdbutil.Sample{sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}, []tsdbutil.Sample{sample{5, 3, nil}, sample{6, 6, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}}, []tsdbutil.Sample{sample{5, 1}, sample{6, 7}, sample{7, 2}}, + []tsdbutil.Sample{sample{1, 3, nil}, sample{2, 2, nil}, sample{3, 6, nil}}, []tsdbutil.Sample{sample{5, 1, nil}, sample{6, 7, nil}, sample{7, 2, nil}}, ), }), }, @@ -304,18 +304,18 @@ func TestBlockQuerier(t *testing.T) { ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, exp: newMockSeriesSet([]storage.Series{ storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{2, 3}, sample{3, 4}, sample{5, 2}, sample{6, 3}}, + []tsdbutil.Sample{sample{2, 3, nil}, sample{3, 4, nil}, sample{5, 2, nil}, sample{6, 3, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{2, 2}, sample{3, 3}, sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 3, nil}, sample{6, 6, nil}}, ), }), expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{2, 3}, sample{3, 4}}, []tsdbutil.Sample{sample{5, 2}, sample{6, 3}}, + []tsdbutil.Sample{sample{2, 3, nil}, sample{3, 4, nil}}, []tsdbutil.Sample{sample{5, 2, nil}, sample{6, 3, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{2, 2}, sample{3, 3}}, []tsdbutil.Sample{sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}}, []tsdbutil.Sample{sample{5, 3, nil}, sample{6, 6, nil}}, ), }), }, @@ -363,24 +363,24 @@ func TestBlockQuerier_AgainstHeadWithOpenChunks(t *testing.T) { ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "a", ".*")}, exp: newMockSeriesSet([]storage.Series{ storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{1, 2}, sample{2, 3}, sample{3, 4}, sample{5, 2}, sample{6, 3}, sample{7, 4}}, + []tsdbutil.Sample{sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 4, nil}, sample{5, 2, nil}, sample{6, 3, nil}, sample{7, 4, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 3, nil}, sample{6, 6, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}, sample{5, 1}, sample{6, 7}, sample{7, 2}}, + []tsdbutil.Sample{sample{1, 3, nil}, sample{2, 2, nil}, sample{3, 6, nil}, sample{5, 1, nil}, sample{6, 7, nil}, sample{7, 2, nil}}, ), }), expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{1, 2}, sample{2, 3}, sample{3, 4}, sample{5, 2}, sample{6, 3}, sample{7, 4}}, + []tsdbutil.Sample{sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 4, nil}, sample{5, 2, nil}, sample{6, 3, nil}, sample{7, 4, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 3, nil}, sample{6, 6, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}, sample{5, 1}, sample{6, 7}, sample{7, 2}}, + []tsdbutil.Sample{sample{1, 3, nil}, sample{2, 2, nil}, sample{3, 6, nil}, sample{5, 1, nil}, sample{6, 7, nil}, sample{7, 2, nil}}, ), }), }, @@ -390,18 +390,18 @@ func TestBlockQuerier_AgainstHeadWithOpenChunks(t *testing.T) { ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, exp: newMockSeriesSet([]storage.Series{ storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{2, 3}, sample{3, 4}, sample{5, 2}, sample{6, 3}}, + []tsdbutil.Sample{sample{2, 3, nil}, sample{3, 4, nil}, sample{5, 2, nil}, sample{6, 3, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{2, 2}, sample{3, 3}, sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 3, nil}, sample{6, 6, nil}}, ), }), expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{2, 3}, sample{3, 4}, sample{5, 2}, sample{6, 3}}, + []tsdbutil.Sample{sample{2, 3, nil}, sample{3, 4, nil}, sample{5, 2, nil}, sample{6, 3, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{2, 2}, sample{3, 3}, sample{5, 3}, sample{6, 6}}, + []tsdbutil.Sample{sample{2, 2, nil}, sample{3, 3, nil}, sample{5, 3, nil}, sample{6, 6, nil}}, ), }), }, @@ -442,22 +442,22 @@ var testData = []seriesSamples{ { lset: map[string]string{"a": "a"}, chunks: [][]sample{ - {{1, 2}, {2, 3}, {3, 4}}, - {{5, 2}, {6, 3}, {7, 4}}, + {{1, 2, nil}, {2, 3, nil}, {3, 4, nil}}, + {{5, 2, nil}, {6, 3, nil}, {7, 4, nil}}, }, }, { lset: map[string]string{"a": "a", "b": "b"}, chunks: [][]sample{ - {{1, 1}, {2, 2}, {3, 3}}, - {{5, 3}, {6, 6}}, + {{1, 1, nil}, {2, 2, nil}, {3, 3, nil}}, + {{5, 3, nil}, {6, 6, nil}}, }, }, { lset: map[string]string{"b": "b"}, chunks: [][]sample{ - {{1, 3}, {2, 2}, {3, 6}}, - {{5, 1}, {6, 7}, {7, 2}}, + {{1, 3, nil}, {2, 2, nil}, {3, 6, nil}}, + {{5, 1, nil}, {6, 7, nil}, {7, 2, nil}}, }, }, } @@ -504,24 +504,24 @@ func TestBlockQuerierDelete(t *testing.T) { ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "a", ".*")}, exp: newMockSeriesSet([]storage.Series{ storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{5, 2}, sample{6, 3}, sample{7, 4}}, + []tsdbutil.Sample{sample{5, 2, nil}, sample{6, 3, nil}, sample{7, 4, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{5, 3}}, + []tsdbutil.Sample{sample{5, 3, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}, sample{5, 1}}, + []tsdbutil.Sample{sample{1, 3, nil}, sample{2, 2, nil}, sample{3, 6, nil}, sample{5, 1, nil}}, ), }), expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{5, 2}, sample{6, 3}, sample{7, 4}}, + []tsdbutil.Sample{sample{5, 2, nil}, sample{6, 3, nil}, sample{7, 4, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{5, 3}}, + []tsdbutil.Sample{sample{5, 3, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}}, []tsdbutil.Sample{sample{5, 1}}, + []tsdbutil.Sample{sample{1, 3, nil}, sample{2, 2, nil}, sample{3, 6, nil}}, []tsdbutil.Sample{sample{5, 1, nil}}, ), }), }, @@ -531,18 +531,18 @@ func TestBlockQuerierDelete(t *testing.T) { ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, exp: newMockSeriesSet([]storage.Series{ storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{5, 2}, sample{6, 3}}, + []tsdbutil.Sample{sample{5, 2, nil}, sample{6, 3, nil}}, ), storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{5, 3}}, + []tsdbutil.Sample{sample{5, 3, nil}}, ), }), expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, - []tsdbutil.Sample{sample{5, 2}, sample{6, 3}}, + []tsdbutil.Sample{sample{5, 2, nil}, sample{6, 3, nil}}, ), storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, - []tsdbutil.Sample{sample{5, 3}}, + []tsdbutil.Sample{sample{5, 3, nil}}, ), }), }, @@ -625,57 +625,57 @@ func TestPopulateWithTombSeriesIterators(t *testing.T) { { name: "one chunk", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, + {sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, }, expected: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}, }, expectedChks: []chunks.Meta{ tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}, }), }, }, { name: "two full chunks", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, expected: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}, sample{7, 89, nil}, sample{9, 8, nil}, }, expectedChks: []chunks.Meta{ tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}, }), tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, + sample{7, 89, nil}, sample{9, 8, nil}, }), }, }, { name: "three full chunks", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, - {sample{10, 22}, sample{203, 3493}}, + {sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, + {sample{10, 22, nil}, sample{203, 3493, nil}}, }, expected: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, sample{10, 22}, sample{203, 3493}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}, sample{7, 89, nil}, sample{9, 8, nil}, sample{10, 22, nil}, sample{203, 3493, nil}, }, expectedChks: []chunks.Meta{ tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}, }), tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, + sample{7, 89, nil}, sample{9, 8, nil}, }), tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{10, 22}, sample{203, 3493}, + sample{10, 22, nil}, sample{203, 3493, nil}, }), }, }, @@ -690,8 +690,8 @@ func TestPopulateWithTombSeriesIterators(t *testing.T) { { name: "two chunks and seek beyond chunks", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, seek: 10, @@ -700,27 +700,27 @@ func TestPopulateWithTombSeriesIterators(t *testing.T) { { name: "two chunks and seek on middle of first chunk", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, seek: 2, seekSuccess: true, expected: []tsdbutil.Sample{ - sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, + sample{3, 5, nil}, sample{6, 1, nil}, sample{7, 89, nil}, sample{9, 8, nil}, }, }, { name: "two chunks and seek before first chunk", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, seek: -32, seekSuccess: true, expected: []tsdbutil.Sample{ - sample{1, 2}, sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, + sample{1, 2, nil}, sample{3, 5, nil}, sample{6, 1, nil}, sample{7, 89, nil}, sample{9, 8, nil}, }, }, // Deletion / Trim cases. @@ -732,60 +732,60 @@ func TestPopulateWithTombSeriesIterators(t *testing.T) { { name: "two chunks with trimmed first and last samples from edge chunks", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, intervals: tombstones.Intervals{{Mint: math.MinInt64, Maxt: 2}}.Add(tombstones.Interval{Mint: 9, Maxt: math.MaxInt64}), expected: []tsdbutil.Sample{ - sample{3, 5}, sample{6, 1}, sample{7, 89}, + sample{3, 5, nil}, sample{6, 1, nil}, sample{7, 89, nil}, }, expectedChks: []chunks.Meta{ tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{3, 5}, sample{6, 1}, + sample{3, 5, nil}, sample{6, 1, nil}, }), tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{7, 89}, + sample{7, 89, nil}, }), }, }, { name: "two chunks with trimmed middle sample of first chunk", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, intervals: tombstones.Intervals{{Mint: 2, Maxt: 3}}, expected: []tsdbutil.Sample{ - sample{1, 2}, sample{6, 1}, sample{7, 89}, sample{9, 8}, + sample{1, 2, nil}, sample{6, 1, nil}, sample{7, 89, nil}, sample{9, 8, nil}, }, expectedChks: []chunks.Meta{ tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{1, 2}, sample{6, 1}, + sample{1, 2, nil}, sample{6, 1, nil}, }), tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, + sample{7, 89, nil}, sample{9, 8, nil}, }), }, }, { name: "two chunks with deletion across two chunks", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, intervals: tombstones.Intervals{{Mint: 6, Maxt: 7}}, expected: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{9, 8}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{9, 8, nil}, }, expectedChks: []chunks.Meta{ tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, + sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, }), tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - sample{9, 8}, + sample{9, 8, nil}, }), }, }, @@ -793,15 +793,15 @@ func TestPopulateWithTombSeriesIterators(t *testing.T) { { name: "two chunks with trimmed first and last samples from edge chunks, seek from middle of first chunk", chks: [][]tsdbutil.Sample{ - {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, - {sample{7, 89}, sample{9, 8}}, + {sample{1, 2, nil}, sample{2, 3, nil}, sample{3, 5, nil}, sample{6, 1, nil}}, + {sample{7, 89, nil}, sample{9, 8, nil}}, }, intervals: tombstones.Intervals{{Mint: math.MinInt64, Maxt: 2}}.Add(tombstones.Interval{Mint: 9, Maxt: math.MaxInt64}), seek: 3, seekSuccess: true, expected: []tsdbutil.Sample{ - sample{3, 5}, sample{6, 1}, sample{7, 89}, + sample{3, 5, nil}, sample{6, 1, nil}, sample{7, 89, nil}, }, }, } @@ -857,8 +857,8 @@ func rmChunkRefs(chks []chunks.Meta) { func TestPopulateWithDelSeriesIterator_DoubleSeek(t *testing.T) { f, chkMetas := createFakeReaderAndNotPopulatedChunks( []tsdbutil.Sample{}, - []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}, - []tsdbutil.Sample{sample{4, 4}, sample{5, 5}}, + []tsdbutil.Sample{sample{1, 1, nil}, sample{2, 2, nil}, sample{3, 3, nil}}, + []tsdbutil.Sample{sample{4, 4, nil}, sample{5, 5, nil}}, ) it := newPopulateWithDelGenericSeriesIterator(f, chkMetas, nil).toSeriesIterator() @@ -875,7 +875,7 @@ func TestPopulateWithDelSeriesIterator_DoubleSeek(t *testing.T) { func TestPopulateWithDelSeriesIterator_SeekInCurrentChunk(t *testing.T) { f, chkMetas := createFakeReaderAndNotPopulatedChunks( []tsdbutil.Sample{}, - []tsdbutil.Sample{sample{1, 2}, sample{3, 4}, sample{5, 6}, sample{7, 8}}, + []tsdbutil.Sample{sample{1, 2, nil}, sample{3, 4, nil}, sample{5, 6, nil}, sample{7, 8, nil}}, []tsdbutil.Sample{}, ) @@ -893,7 +893,7 @@ func TestPopulateWithDelSeriesIterator_SeekInCurrentChunk(t *testing.T) { func TestPopulateWithDelSeriesIterator_SeekWithMinTime(t *testing.T) { f, chkMetas := createFakeReaderAndNotPopulatedChunks( - []tsdbutil.Sample{sample{1, 6}, sample{5, 6}, sample{6, 8}}, + []tsdbutil.Sample{sample{1, 6, nil}, sample{5, 6, nil}, sample{6, 8, nil}}, ) it := newPopulateWithDelGenericSeriesIterator(f, chkMetas, nil).toSeriesIterator() @@ -905,7 +905,7 @@ func TestPopulateWithDelSeriesIterator_SeekWithMinTime(t *testing.T) { // Seek gets called and advances beyond the max time, which was just accepted as a valid sample. func TestPopulateWithDelSeriesIterator_NextWithMinTime(t *testing.T) { f, chkMetas := createFakeReaderAndNotPopulatedChunks( - []tsdbutil.Sample{sample{1, 6}, sample{5, 6}, sample{7, 8}}, + []tsdbutil.Sample{sample{1, 6, nil}, sample{5, 6, nil}, sample{7, 8, nil}}, ) it := newPopulateWithDelGenericSeriesIterator( diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 25c2107b8..0a3e3c2f4 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -74,7 +74,7 @@ type RefExemplar struct { type RefHistogram struct { Ref uint64 T int64 - H histogram.Histogram + H *histogram.Histogram } // Decoder decodes series, sample, and tombstone records. @@ -253,7 +253,7 @@ func (d *Decoder) Histograms(rec []byte, histograms []RefHistogram) ([]RefHistog rh := RefHistogram{ Ref: baseRef + uint64(dref), T: baseTime + dtime, - H: histogram.Histogram{ + H: &histogram.Histogram{ Schema: 0, ZeroThreshold: 0, ZeroCount: 0, diff --git a/tsdb/tsdbutil/buffer.go b/tsdb/tsdbutil/buffer.go index 5569c0047..28cb04a7b 100644 --- a/tsdb/tsdbutil/buffer.go +++ b/tsdb/tsdbutil/buffer.go @@ -165,9 +165,9 @@ func (it *sampleRingIterator) At() (int64, float64) { return it.r.at(it.i) } -func (it *sampleRingIterator) AtHistogram() (int64, histogram.Histogram) { +func (it *sampleRingIterator) AtHistogram() (int64, *histogram.Histogram) { // TODO(beorn7): Add proper histogram support. - return 0, histogram.Histogram{} + return 0, nil } func (it *sampleRingIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/tsdb/tsdbutil/buffer_test.go b/tsdb/tsdbutil/buffer_test.go index 8ff8fa086..de0e40308 100644 --- a/tsdb/tsdbutil/buffer_test.go +++ b/tsdb/tsdbutil/buffer_test.go @@ -152,12 +152,9 @@ func (it *listSeriesIterator) At() (int64, float64) { return s.t, s.v } -func (it *listSeriesIterator) AtHistogram() (int64, histogram.Histogram) { +func (it *listSeriesIterator) AtHistogram() (int64, *histogram.Histogram) { s := it.list[it.idx] - if s.h == nil { - return s.t, histogram.Histogram{} - } - return s.t, *s.h + return s.t, s.h } func (it *listSeriesIterator) ChunkEncoding() chunkenc.Encoding { diff --git a/web/federate.go b/web/federate.go index f97579ad4..8d2af396d 100644 --- a/web/federate.go +++ b/web/federate.go @@ -115,7 +115,8 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) { if ok { t, v = it.Values() } else { - t, v, ok = it.PeekBack(1) + // TODO(beorn7): Handle histograms. + t, v, _, ok = it.PeekBack(1) if !ok { continue } @@ -220,6 +221,7 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) { protMetric.TimestampMs = proto.Int64(s.T) protMetric.Untyped.Value = proto.Float64(s.V) + // TODO(beorn7): Handle histograms. protMetricFam.Metric = append(protMetricFam.Metric, protMetric) } From 9b30ca25985c5eb310dcb4e35aaf1fae37903bf0 Mon Sep 17 00:00:00 2001 From: beorn7 Date: Mon, 15 Nov 2021 20:36:44 +0100 Subject: [PATCH 5/7] promql: Support histogram in value string representation Signed-off-by: beorn7 --- promql/engine_test.go | 5 +++-- promql/value.go | 10 +++++++--- storage/memoized_iterator.go | 18 +++++++++++++++--- 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/promql/engine_test.go b/promql/engine_test.go index fe1d017bc..886d2a92e 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -2455,10 +2455,11 @@ func TestSparseHistogramRate(t *testing.T) { require.NoError(t, test.Run()) engine := test.QueryEngine() - queryString := fmt.Sprintf("rate(%s[1m])", seriesName) + //queryString := fmt.Sprintf("rate(%s[1m])", seriesName) + queryString := fmt.Sprintf("%s", seriesName) qry, err := engine.NewInstantQuery(test.Queryable(), queryString, timestamp.Time(int64(5*time.Minute/time.Millisecond))) require.NoError(t, err) res := qry.Exec(test.Context()) require.NoError(t, res.Err) - // fmt.Println(res) + fmt.Println(res) } diff --git a/promql/value.go b/promql/value.go index 5e3863379..727808423 100644 --- a/promql/value.go +++ b/promql/value.go @@ -87,9 +87,13 @@ type Point struct { } func (p Point) String() string { - // TODO(beorn7): Support Histogram. - v := strconv.FormatFloat(p.V, 'f', -1, 64) - return fmt.Sprintf("%v @[%v]", v, p.T) + var s string + if p.H != nil { + s = p.H.String() + } else { + s = strconv.FormatFloat(p.V, 'f', -1, 64) + } + return fmt.Sprintf("%s @[%v]", s, p.T) } // MarshalJSON implements json.Marshaler. diff --git a/storage/memoized_iterator.go b/storage/memoized_iterator.go index 0c40bb9d0..9d4cc5207 100644 --- a/storage/memoized_iterator.go +++ b/storage/memoized_iterator.go @@ -80,7 +80,11 @@ func (b *MemoizedSeriesIterator) Seek(t int64) bool { if !b.ok { return false } - b.lastTime, _ = b.it.At() + if b.it.ChunkEncoding() == chunkenc.EncHistogram { + b.lastTime, _ = b.it.AtHistogram() + } else { + b.lastTime, _ = b.it.At() + } } if b.lastTime >= t { @@ -102,11 +106,19 @@ func (b *MemoizedSeriesIterator) Next() bool { } // Keep track of the previous element. - b.prevTime, b.prevValue = b.it.At() + if b.it.ChunkEncoding() == chunkenc.EncHistogram { + b.prevTime, b.prev + } else { + b.prevTime, b.prevValue = b.it.At() + } b.ok = b.it.Next() if b.ok { - b.lastTime, _ = b.it.At() + if b.it.ChunkEncoding() == chunkenc.EncHistogram { + b.lastTime, _ = b.it.AtHistogram() + } else { + b.lastTime, _ = b.it.At() + } } return b.ok From 73858d7f82d345aa45ccecaeed26fb640ef3117b Mon Sep 17 00:00:00 2001 From: beorn7 Date: Mon, 15 Nov 2021 21:49:25 +0100 Subject: [PATCH 6/7] storage: histogram support in memoized_iterator Signed-off-by: beorn7 --- model/histogram/histogram.go | 2 +- promql/engine.go | 41 ++++++++-------- promql/engine_test.go | 6 +-- storage/memoized_iterator.go | 77 +++++++++++++++++++++---------- storage/memoized_iterator_test.go | 21 +++++---- 5 files changed, 91 insertions(+), 56 deletions(-) diff --git a/model/histogram/histogram.go b/model/histogram/histogram.go index 42ce8eb99..91c1aa15c 100644 --- a/model/histogram/histogram.go +++ b/model/histogram/histogram.go @@ -47,7 +47,7 @@ type Histogram struct { ZeroCount uint64 // Total number of observations. Count uint64 - // Sum of observations. + // Sum of observations. This is also used as the stale marker. Sum float64 // Spans for positive and negative buckets (see Span below). PositiveSpans, NegativeSpans []Span diff --git a/promql/engine.go b/promql/engine.go index afe870f2d..a5400cb96 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -35,6 +35,7 @@ import ( "github.com/prometheus/common/model" "github.com/uber/jaeger-client-go" + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/pkg/value" @@ -1475,10 +1476,10 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { } for ts := ev.startTimestamp; ts <= ev.endTimestamp; ts += ev.interval { - _, v, ok := ev.vectorSelectorSingle(it, e, ts) + _, v, h, ok := ev.vectorSelectorSingle(it, e, ts) if ok { if ev.currentSamples < ev.maxSamples { - ss.Points = append(ss.Points, Point{V: v, T: ts}) + ss.Points = append(ss.Points, Point{V: v, H: h, T: ts}) ev.currentSamples++ } else { ev.error(ErrTooManySamples(env)) @@ -1601,11 +1602,11 @@ func (ev *evaluator) vectorSelector(node *parser.VectorSelector, ts int64) (Vect for i, s := range node.Series { it.Reset(s.Iterator()) - t, v, ok := ev.vectorSelectorSingle(it, node, ts) + t, v, h, ok := ev.vectorSelectorSingle(it, node, ts) if ok { vec = append(vec, Sample{ Metric: node.Series[i].Labels(), - Point: Point{V: v, T: t}, + Point: Point{V: v, H: h, T: t}, }) ev.currentSamples++ @@ -1618,33 +1619,37 @@ func (ev *evaluator) vectorSelector(node *parser.VectorSelector, ts int64) (Vect return vec, ws } -// vectorSelectorSingle evaluates a instant vector for the iterator of one time series. -func (ev *evaluator) vectorSelectorSingle(it *storage.MemoizedSeriesIterator, node *parser.VectorSelector, ts int64) (int64, float64, bool) { +// vectorSelectorSingle evaluates an instant vector for the iterator of one time series. +func (ev *evaluator) vectorSelectorSingle(it *storage.MemoizedSeriesIterator, node *parser.VectorSelector, ts int64) (int64, float64, *histogram.Histogram, bool) { refTime := ts - durationMilliseconds(node.Offset) var t int64 var v float64 + var h *histogram.Histogram - ok := it.Seek(refTime) - if !ok { + valueType := it.Seek(refTime) + switch valueType { + case storage.ValNone: if it.Err() != nil { ev.error(it.Err()) } - } - - if ok { + case storage.ValFloat: t, v = it.Values() + case storage.ValHistogram: + t, h = it.HistogramValues() + default: + panic(fmt.Errorf("unknown value type %v", valueType)) } - - if !ok || t > refTime { - t, v, ok = it.PeekPrev() + if valueType == storage.ValNone || t > refTime { + var ok bool + t, v, h, ok = it.PeekPrev() if !ok || t < refTime-durationMilliseconds(ev.lookbackDelta) { - return 0, 0, false + return 0, 0, nil, false } } - if value.IsStaleNaN(v) { - return 0, 0, false + if value.IsStaleNaN(v) || (h != nil && value.IsStaleNaN(h.Sum)) { + return 0, 0, nil, false } - return t, v, true + return t, v, h, true } var pointPool = sync.Pool{} diff --git a/promql/engine_test.go b/promql/engine_test.go index 886d2a92e..f484eb1e4 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -2455,11 +2455,11 @@ func TestSparseHistogramRate(t *testing.T) { require.NoError(t, test.Run()) engine := test.QueryEngine() - //queryString := fmt.Sprintf("rate(%s[1m])", seriesName) - queryString := fmt.Sprintf("%s", seriesName) + queryString := fmt.Sprintf("rate(%s[1m])", seriesName) + // TODO(beorn7): "%s" returns a float but "%s[1m]" returns matrix of histograms. qry, err := engine.NewInstantQuery(test.Queryable(), queryString, timestamp.Time(int64(5*time.Minute/time.Millisecond))) require.NoError(t, err) res := qry.Exec(test.Context()) require.NoError(t, res.Err) - fmt.Println(res) + //fmt.Println(res) } diff --git a/storage/memoized_iterator.go b/storage/memoized_iterator.go index 9d4cc5207..7701238cd 100644 --- a/storage/memoized_iterator.go +++ b/storage/memoized_iterator.go @@ -16,20 +16,31 @@ package storage import ( "math" + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/tsdb/chunkenc" ) +// ValueType defines the type of a value in the storage. +type ValueType int + +const ( + ValNone ValueType = iota + ValFloat + ValHistogram +) + // MemoizedSeriesIterator wraps an iterator with a buffer to look back the previous element. type MemoizedSeriesIterator struct { it chunkenc.Iterator delta int64 - lastTime int64 - ok bool + lastTime int64 + valueType ValueType // Keep track of the previously returned value. - prevTime int64 - prevValue float64 + prevTime int64 + prevValue float64 + prevHistogram *histogram.Histogram } // NewMemoizedEmptyIterator is like NewMemoizedIterator but it's initialised with an empty iterator. @@ -53,22 +64,26 @@ func NewMemoizedIterator(it chunkenc.Iterator, delta int64) *MemoizedSeriesItera func (b *MemoizedSeriesIterator) Reset(it chunkenc.Iterator) { b.it = it b.lastTime = math.MinInt64 - b.ok = true b.prevTime = math.MinInt64 it.Next() + if it.ChunkEncoding() == chunkenc.EncHistogram { + b.valueType = ValHistogram + } else { + b.valueType = ValFloat + } } // PeekPrev returns the previous element of the iterator. If there is none buffered, // ok is false. -func (b *MemoizedSeriesIterator) PeekPrev() (t int64, v float64, ok bool) { +func (b *MemoizedSeriesIterator) PeekPrev() (t int64, v float64, h *histogram.Histogram, ok bool) { if b.prevTime == math.MinInt64 { - return 0, 0, false + return 0, 0, nil, false } - return b.prevTime, b.prevValue, true + return b.prevTime, b.prevValue, b.prevHistogram, true } // Seek advances the iterator to the element at time t or greater. -func (b *MemoizedSeriesIterator) Seek(t int64) bool { +func (b *MemoizedSeriesIterator) Seek(t int64) ValueType { t0 := t - b.delta if t0 > b.lastTime { @@ -76,52 +91,61 @@ func (b *MemoizedSeriesIterator) Seek(t int64) bool { // more than the delta. b.prevTime = math.MinInt64 - b.ok = b.it.Seek(t0) - if !b.ok { - return false + ok := b.it.Seek(t0) + if !ok { + b.valueType = ValNone + return ValNone } if b.it.ChunkEncoding() == chunkenc.EncHistogram { + b.valueType = ValHistogram b.lastTime, _ = b.it.AtHistogram() } else { + b.valueType = ValFloat b.lastTime, _ = b.it.At() } } if b.lastTime >= t { - return true + return b.valueType } - for b.Next() { + for b.Next() != ValNone { if b.lastTime >= t { - return true + return b.valueType } } - return false + return ValNone } // Next advances the iterator to the next element. -func (b *MemoizedSeriesIterator) Next() bool { - if !b.ok { - return false +func (b *MemoizedSeriesIterator) Next() ValueType { + if b.valueType == ValNone { + return ValNone } // Keep track of the previous element. if b.it.ChunkEncoding() == chunkenc.EncHistogram { - b.prevTime, b.prev + b.prevTime, b.prevHistogram = b.it.AtHistogram() + b.prevValue = 0 } else { b.prevTime, b.prevValue = b.it.At() + b.prevHistogram = nil } - b.ok = b.it.Next() - if b.ok { + ok := b.it.Next() + if ok { if b.it.ChunkEncoding() == chunkenc.EncHistogram { b.lastTime, _ = b.it.AtHistogram() + b.valueType = ValHistogram + } else { b.lastTime, _ = b.it.At() + b.valueType = ValFloat } + } else { + b.valueType = ValNone } - - return b.ok + return b.valueType } // Values returns the current element of the iterator. @@ -129,6 +153,11 @@ func (b *MemoizedSeriesIterator) Values() (int64, float64) { return b.it.At() } +// Values returns the current element of the iterator. +func (b *MemoizedSeriesIterator) HistogramValues() (int64, *histogram.Histogram) { + return b.it.AtHistogram() +} + // Err returns the last encountered error. func (b *MemoizedSeriesIterator) Err() error { return b.it.Err() diff --git a/storage/memoized_iterator_test.go b/storage/memoized_iterator_test.go index 849256587..ee5155199 100644 --- a/storage/memoized_iterator_test.go +++ b/storage/memoized_iterator_test.go @@ -20,6 +20,7 @@ import ( ) func TestMemoizedSeriesIterator(t *testing.T) { + // TODO(beorn7): Include histograms in testing. var it *MemoizedSeriesIterator sampleEq := func(ets int64, ev float64) { @@ -28,7 +29,7 @@ func TestMemoizedSeriesIterator(t *testing.T) { require.Equal(t, ev, v, "value mismatch") } prevSampleEq := func(ets int64, ev float64, eok bool) { - ts, v, ok := it.PeekPrev() + ts, v, _, ok := it.PeekPrev() require.Equal(t, eok, ok, "exist mismatch") require.Equal(t, ets, ts, "timestamp mismatch") require.Equal(t, ev, v, "value mismatch") @@ -45,29 +46,29 @@ func TestMemoizedSeriesIterator(t *testing.T) { sample{t: 101, v: 10}, }), 2) - require.True(t, it.Seek(-123), "seek failed") + require.Equal(t, it.Seek(-123), ValFloat, "seek failed") sampleEq(1, 2) prevSampleEq(0, 0, false) - require.True(t, it.Next(), "next failed") + require.Equal(t, it.Next(), ValFloat, "next failed") sampleEq(2, 3) prevSampleEq(1, 2, true) - require.True(t, it.Next(), "next failed") - require.True(t, it.Next(), "next failed") - require.True(t, it.Next(), "next failed") + require.Equal(t, it.Next(), ValFloat, "next failed") + require.Equal(t, it.Next(), ValFloat, "next failed") + require.Equal(t, it.Next(), ValFloat, "next failed") sampleEq(5, 6) prevSampleEq(4, 5, true) - require.True(t, it.Seek(5), "seek failed") + require.Equal(t, it.Seek(5), ValFloat, "seek failed") sampleEq(5, 6) prevSampleEq(4, 5, true) - require.True(t, it.Seek(101), "seek failed") + require.Equal(t, it.Seek(101), ValFloat, "seek failed") sampleEq(101, 10) prevSampleEq(100, 9, true) - require.False(t, it.Next(), "next succeeded unexpectedly") + require.Equal(t, it.Next(), ValNone, "next succeeded unexpectedly") } func BenchmarkMemoizedSeriesIterator(b *testing.B) { @@ -78,7 +79,7 @@ func BenchmarkMemoizedSeriesIterator(b *testing.B) { b.ReportAllocs() b.ResetTimer() - for it.Next() { + for it.Next() != ValNone { // scan everything } require.NoError(b, it.Err()) From 9de3ab60df4d220b39c004ce6b9321183a58217e Mon Sep 17 00:00:00 2001 From: beorn7 Date: Tue, 16 Nov 2021 13:20:24 +0100 Subject: [PATCH 7/7] promql: improve histogram support in engine.go Signed-off-by: beorn7 --- promql/engine.go | 6 +++--- promql/engine_test.go | 2 -- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/promql/engine.go b/promql/engine.go index a5400cb96..02acab11e 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -183,7 +183,6 @@ func (q *query) Exec(ctx context.Context) *Result { // Exec query. res, warnings, err := q.ng.exec(ctx, q) - return &Result{Err: err, Value: res, Warnings: warnings} } @@ -616,7 +615,7 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *parser.Eval for i, s := range mat { // Point might have a different timestamp, force it to the evaluation // timestamp as that is when we ran the evaluation. - vector[i] = Sample{Metric: s.Metric, Point: Point{V: s.Points[0].V, T: start}} + vector[i] = Sample{Metric: s.Metric, Point: Point{V: s.Points[0].V, H: s.Points[0].H, T: start}} } return vector, warnings, nil case parser.ValueTypeScalar: @@ -1326,7 +1325,7 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { outVec := call(inArgs, e.Args, enh) enh.Out = outVec[:0] if len(outVec) > 0 { - ss.Points = append(ss.Points, Point{V: outVec[0].Point.V, T: ts}) + ss.Points = append(ss.Points, Point{V: outVec[0].Point.V, H: outVec[0].Point.H, T: ts}) } // Only buffer stepRange milliseconds from the second step on. it.ReduceDelta(stepRange) @@ -1578,6 +1577,7 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { mat[i].Points = append(mat[i].Points, Point{ T: ts, V: mat[i].Points[0].V, + H: mat[i].Points[0].H, }) ev.currentSamples++ if ev.currentSamples > ev.maxSamples { diff --git a/promql/engine_test.go b/promql/engine_test.go index f484eb1e4..88eed7f8b 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -2456,10 +2456,8 @@ func TestSparseHistogramRate(t *testing.T) { engine := test.QueryEngine() queryString := fmt.Sprintf("rate(%s[1m])", seriesName) - // TODO(beorn7): "%s" returns a float but "%s[1m]" returns matrix of histograms. qry, err := engine.NewInstantQuery(test.Queryable(), queryString, timestamp.Time(int64(5*time.Minute/time.Millisecond))) require.NoError(t, err) res := qry.Exec(test.Context()) require.NoError(t, res.Err) - //fmt.Println(res) }