From e6d7cc5fa43716281bdf5d1cf40415f99a0e9623 Mon Sep 17 00:00:00 2001 From: Bartlomiej Plotka Date: Fri, 31 Jul 2020 16:03:02 +0100 Subject: [PATCH] tsdb: Added ChunkQueryable implementations to db; unified MergeSeriesSets and vertical to single struct. (#7069) * tsdb: Added ChunkQueryable implementations to db; unified compactor, querier and fanout block iterating. Chained to https://github.com/prometheus/prometheus/pull/7059 * NewMerge(Chunk)Querier now takies multiple primaries allowing tsdb DB code to use it. * Added single SeriesEntry / ChunkEntry for all series implementations. * Unified all vertical, and non vertical for compact and querying to single merge series / chunk sets by reusing VerticalSeriesMergeFunc for overlapping algorithm (same logic as before) * Added block (Base/Chunk/)Querier for block querying. We then use populateAndTomb(Base/Chunk/) to iterate over chunks or samples. * Refactored endpoint tests and querier tests to include subtests. Signed-off-by: Bartlomiej Plotka * Addressed comments from Brian and Beorn. Signed-off-by: Bartlomiej Plotka * Fixed snapshot test and added chunk iterator support for DBReadOnly. Signed-off-by: Bartlomiej Plotka * Fixed race when iterating over Ats first. Signed-off-by: Bartlomiej Plotka * Fixed tests. Signed-off-by: Bartlomiej Plotka * Fixed populate block tests. Signed-off-by: Bartlomiej Plotka * Fixed endpoints test. Signed-off-by: Bartlomiej Plotka * Fixed test. Signed-off-by: Bartlomiej Plotka * Added test & fixed case of head open chunk. Signed-off-by: Bartlomiej Plotka * Fixed DBReadOnly tests and bug producing 1 sample chunks. Signed-off-by: Bartlomiej Plotka * Added cases for partial block overlap for multiple full chunks. Signed-off-by: Bartlomiej Plotka * Added extra tests for chunk meta after compaction. Signed-off-by: Bartlomiej Plotka * Fixed small vertical merge bug and added more tests for that. Signed-off-by: Bartlomiej Plotka --- prompb/custom.go | 17 + promql/engine_test.go | 1 - promql/promql_test.go | 11 +- storage/fanout.go | 405 +++++---- storage/fanout_test.go | 140 ++- storage/generic.go | 4 +- storage/interface.go | 15 +- storage/lazy.go | 16 +- storage/remote/storage.go | 4 +- storage/secondary.go | 35 +- storage/series.go | 103 ++- storage/series_test.go | 84 -- tsdb/block.go | 7 +- tsdb/block_test.go | 96 +- tsdb/chunks/chunks.go | 79 -- tsdb/compact.go | 543 +++++++----- tsdb/compact_test.go | 390 ++++++++- tsdb/db.go | 102 ++- tsdb/db_test.go | 490 ++--------- tsdb/head.go | 11 +- tsdb/head_test.go | 36 +- tsdb/mocks_test.go | 56 +- tsdb/querier.go | 1051 +++------------------- tsdb/querier_test.go | 1748 +++++++++++++++---------------------- tsdb/tsdbutil/chunks.go | 22 +- web/api/v1/api.go | 29 +- web/api/v1/api_test.go | 59 +- 27 files changed, 2289 insertions(+), 3265 deletions(-) create mode 100644 prompb/custom.go delete mode 100644 storage/series_test.go diff --git a/prompb/custom.go b/prompb/custom.go new file mode 100644 index 000000000..0b3820c4d --- /dev/null +++ b/prompb/custom.go @@ -0,0 +1,17 @@ +// Copyright 2020 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package prompb + +func (m Sample) T() int64 { return m.Timestamp } +func (m Sample) V() float64 { return m.Value } diff --git a/promql/engine_test.go b/promql/engine_test.go index a43715905..185d4d715 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -1062,7 +1062,6 @@ func TestSubquerySelector(t *testing.T) { testutil.Ok(t, err) res := qry.Exec(test.Context()) - testutil.Equals(t, c.Result.Err, res.Err) mat := res.Value.(Matrix) sort.Sort(mat) diff --git a/promql/promql_test.go b/promql/promql_test.go index 6dd637f47..a19474280 100644 --- a/promql/promql_test.go +++ b/promql/promql_test.go @@ -25,11 +25,12 @@ func TestEvaluations(t *testing.T) { testutil.Ok(t, err) for _, fn := range files { - test, err := newTestFromFile(t, fn) - testutil.Ok(t, err) + t.Run(fn, func(t *testing.T) { + test, err := newTestFromFile(t, fn) + testutil.Ok(t, err) + testutil.Ok(t, test.Run()) - testutil.Ok(t, test.Run()) - - test.Close() + test.Close() + }) } } diff --git a/storage/fanout.go b/storage/fanout.go index c7c738201..6aff89171 100644 --- a/storage/fanout.go +++ b/storage/fanout.go @@ -96,7 +96,7 @@ func (f *fanout) Querier(ctx context.Context, mint, maxt int64) (Querier, error) } secondaries = append(secondaries, querier) } - return NewMergeQuerier(primary, secondaries, ChainedSeriesMerge), nil + return NewMergeQuerier([]Querier{primary}, secondaries, ChainedSeriesMerge), nil } func (f *fanout) ChunkQuerier(ctx context.Context, mint, maxt int64) (ChunkQuerier, error) { @@ -119,7 +119,7 @@ func (f *fanout) ChunkQuerier(ctx context.Context, mint, maxt int64) (ChunkQueri } secondaries = append(secondaries, querier) } - return NewMergeChunkQuerier(primary, secondaries, NewCompactingChunkSeriesMerger(ChainedSeriesMerge)), nil + return NewMergeChunkQuerier([]ChunkQuerier{primary}, secondaries, NewCompactingChunkSeriesMerger(ChainedSeriesMerge)), nil } func (f *fanout) Appender(ctx context.Context) Appender { @@ -214,38 +214,50 @@ type mergeGenericQuerier struct { // mergeFn is used when we see series from different queriers Selects with the same labels. mergeFn genericSeriesMergeFunc + + // TODO(bwplotka): Remove once remote queries are asynchronous. False by default. + concurrentSelect bool } -// NewMergeQuerier returns a new Querier that merges results of given primary and slice of secondary queriers. +// NewMergeQuerier returns a new Querier that merges results of given primary and secondary queriers. // See NewFanout commentary to learn more about primary vs secondary differences. // -// In case of overlaps between the data given by primary + secondaries Selects, merge function will be used. -func NewMergeQuerier(primary Querier, secondaries []Querier, mergeFn VerticalSeriesMergeFunc) Querier { - queriers := make([]genericQuerier, 0, len(secondaries)+1) - if primary != nil { - queriers = append(queriers, newGenericQuerierFrom(primary)) +// In case of overlaps between the data given by primaries' and secondaries' Selects, merge function will be used. +func NewMergeQuerier(primaries []Querier, secondaries []Querier, mergeFn VerticalSeriesMergeFunc) Querier { + queriers := make([]genericQuerier, 0, len(primaries)+len(secondaries)) + for _, q := range primaries { + if _, ok := q.(noopQuerier); !ok && q != nil { + queriers = append(queriers, newGenericQuerierFrom(q)) + } } - for _, querier := range secondaries { - if _, ok := querier.(noopQuerier); !ok && querier != nil { - queriers = append(queriers, newSecondaryQuerierFrom(querier)) + for _, q := range secondaries { + if _, ok := q.(noopQuerier); !ok && q != nil { + queriers = append(queriers, newSecondaryQuerierFrom(q)) } } + concurrentSelect := false + if len(secondaries) > 0 { + concurrentSelect = true + } return &querierAdapter{&mergeGenericQuerier{ - mergeFn: (&seriesMergerAdapter{VerticalSeriesMergeFunc: mergeFn}).Merge, - queriers: queriers, + mergeFn: (&seriesMergerAdapter{VerticalSeriesMergeFunc: mergeFn}).Merge, + queriers: queriers, + concurrentSelect: concurrentSelect, }} } -// NewMergeChunkQuerier returns a new ChunkQuerier that merges results of given primary and slice of secondary chunk queriers. +// NewMergeChunkQuerier returns a new Chunk Querier that merges results of given primary and secondary chunk queriers. // See NewFanout commentary to learn more about primary vs secondary differences. // -// In case of overlaps between the data given by primary + secondaries Selects, merge function will be used. +// In case of overlaps between the data given by primaries' and secondaries' Selects, merge function will be used. // TODO(bwplotka): Currently merge will compact overlapping chunks with bigger chunk, without limit. Split it: https://github.com/prometheus/tsdb/issues/670 -func NewMergeChunkQuerier(primary ChunkQuerier, secondaries []ChunkQuerier, mergeFn VerticalChunkSeriesMergeFunc) ChunkQuerier { - queriers := make([]genericQuerier, 0, len(secondaries)+1) - if primary != nil { - queriers = append(queriers, newGenericQuerierFromChunk(primary)) +func NewMergeChunkQuerier(primaries []ChunkQuerier, secondaries []ChunkQuerier, mergeFn VerticalChunkSeriesMergeFunc) ChunkQuerier { + queriers := make([]genericQuerier, 0, len(primaries)+len(secondaries)) + for _, q := range primaries { + if _, ok := q.(noopChunkQuerier); !ok && q != nil { + queriers = append(queriers, newGenericQuerierFromChunk(q)) + } } for _, querier := range secondaries { if _, ok := querier.(noopChunkQuerier); !ok && querier != nil { @@ -253,24 +265,42 @@ func NewMergeChunkQuerier(primary ChunkQuerier, secondaries []ChunkQuerier, merg } } + concurrentSelect := false + if len(secondaries) > 0 { + concurrentSelect = true + } return &chunkQuerierAdapter{&mergeGenericQuerier{ - mergeFn: (&chunkSeriesMergerAdapter{VerticalChunkSeriesMergeFunc: mergeFn}).Merge, - queriers: queriers, + mergeFn: (&chunkSeriesMergerAdapter{VerticalChunkSeriesMergeFunc: mergeFn}).Merge, + queriers: queriers, + concurrentSelect: concurrentSelect, }} } // Select returns a set of series that matches the given label matchers. func (q *mergeGenericQuerier) Select(sortSeries bool, hints *SelectHints, matchers ...*labels.Matcher) genericSeriesSet { + if len(q.queriers) == 0 { + return noopGenericSeriesSet{} + } if len(q.queriers) == 1 { return q.queriers[0].Select(sortSeries, hints, matchers...) } + var seriesSets = make([]genericSeriesSet, 0, len(q.queriers)) + if !q.concurrentSelect { + for _, querier := range q.queriers { + // We need to sort for merge to work. + seriesSets = append(seriesSets, querier.Select(true, hints, matchers...)) + } + return &lazyGenericSeriesSet{init: func() (genericSeriesSet, bool) { + s := newGenericMergeSeriesSet(seriesSets, q.mergeFn) + return s, s.Next() + }} + } + var ( - seriesSets = make([]genericSeriesSet, 0, len(q.queriers)) wg sync.WaitGroup seriesSetChan = make(chan genericSeriesSet) ) - // Schedule all Selects for all queriers we know about. for _, querier := range q.queriers { wg.Add(1) @@ -289,99 +319,83 @@ func (q *mergeGenericQuerier) Select(sortSeries bool, hints *SelectHints, matche for r := range seriesSetChan { seriesSets = append(seriesSets, r) } - return &lazySeriesSet{create: create(seriesSets, q.mergeFn)} + return &lazyGenericSeriesSet{init: func() (genericSeriesSet, bool) { + s := newGenericMergeSeriesSet(seriesSets, q.mergeFn) + return s, s.Next() + }} } -func create(seriesSets []genericSeriesSet, mergeFunc genericSeriesMergeFunc) func() (genericSeriesSet, bool) { - // Returned function gets called with the first call to Next(). - return func() (genericSeriesSet, bool) { - if len(seriesSets) == 1 { - return seriesSets[0], seriesSets[0].Next() - } - var h genericSeriesSetHeap - for _, set := range seriesSets { - if set == nil { - continue - } - if set.Next() { - heap.Push(&h, set) - continue - } - // When primary fails ignore results from secondaries. - // Only the primary querier returns error. - if err := set.Err(); err != nil { - return errorOnlySeriesSet{err}, false - } - } - set := &genericMergeSeriesSet{ - mergeFunc: mergeFunc, - sets: seriesSets, - heap: h, - } - return set, set.Next() - } +type labelGenericQueriers []genericQuerier + +func (l labelGenericQueriers) Len() int { return len(l) } +func (l labelGenericQueriers) Get(i int) LabelQuerier { return l[i] } +func (l labelGenericQueriers) SplitByHalf() (labelGenericQueriers, labelGenericQueriers) { + i := len(l) / 2 + return l[:i], l[i:] } // LabelValues returns all potential values for a label name. func (q *mergeGenericQuerier) LabelValues(name string) ([]string, Warnings, error) { - var ( - results [][]string - warnings Warnings - ) - for _, querier := range q.queriers { - values, wrn, err := querier.LabelValues(name) - if wrn != nil { - // TODO(bwplotka): We could potentially wrap warnings. - warnings = append(warnings, wrn...) - } - if err != nil { - return nil, nil, errors.Wrapf(err, "LabelValues() from Querier for label %s", name) - } - results = append(results, values) + res, ws, err := q.lvals(q.queriers, name) + if err != nil { + return nil, nil, errors.Wrapf(err, "LabelValues() from merge generic querier for label %s", name) } - return mergeStringSlices(results), warnings, nil + return res, ws, nil } -func mergeStringSlices(ss [][]string) []string { - switch len(ss) { - case 0: - return nil - case 1: - return ss[0] - case 2: - return mergeTwoStringSlices(ss[0], ss[1]) - default: - halfway := len(ss) / 2 - return mergeTwoStringSlices( - mergeStringSlices(ss[:halfway]), - mergeStringSlices(ss[halfway:]), - ) +// lvals performs merge sort for LabelValues from multiple queriers. +func (q *mergeGenericQuerier) lvals(lq labelGenericQueriers, n string) ([]string, Warnings, error) { + if lq.Len() == 0 { + return nil, nil, nil } + if lq.Len() == 1 { + return lq.Get(0).LabelValues(n) + } + a, b := lq.SplitByHalf() + + var ws Warnings + s1, w, err := q.lvals(a, n) + ws = append(ws, w...) + if err != nil { + return nil, ws, err + } + s2, ws, err := q.lvals(b, n) + ws = append(ws, w...) + if err != nil { + return nil, ws, err + } + return mergeStrings(s1, s2), ws, nil } -func mergeTwoStringSlices(a, b []string) []string { - i, j := 0, 0 - result := make([]string, 0, len(a)+len(b)) - for i < len(a) && j < len(b) { - switch strings.Compare(a[i], b[j]) { - case 0: - result = append(result, a[i]) - i++ - j++ - case -1: - result = append(result, a[i]) - i++ - case 1: - result = append(result, b[j]) - j++ +func mergeStrings(a, b []string) []string { + maxl := len(a) + if len(b) > len(a) { + maxl = len(b) + } + res := make([]string, 0, maxl*10/9) + + for len(a) > 0 && len(b) > 0 { + d := strings.Compare(a[0], b[0]) + + if d == 0 { + res = append(res, a[0]) + a, b = a[1:], b[1:] + } else if d < 0 { + res = append(res, a[0]) + a = a[1:] + } else if d > 0 { + res = append(res, b[0]) + b = b[1:] } } - result = append(result, a[i:]...) - result = append(result, b[j:]...) - return result + + // Append all remaining elements. + res = append(res, a...) + res = append(res, b...) + return res } -// LabelNames returns all the unique label names present in the block in sorted order. +// LabelNames returns all the unique label names present in all queriers in sorted order. func (q *mergeGenericQuerier) LabelNames() ([]string, Warnings, error) { var ( labelNamesMap = make(map[string]struct{}) @@ -394,7 +408,7 @@ func (q *mergeGenericQuerier) LabelNames() ([]string, Warnings, error) { warnings = append(warnings, wrn...) } if err != nil { - return nil, nil, errors.Wrap(err, "LabelNames() from Querier") + return nil, nil, errors.Wrap(err, "LabelNames() from merge generic querier") } for _, name := range names { labelNamesMap[name] = struct{}{} @@ -412,7 +426,7 @@ func (q *mergeGenericQuerier) LabelNames() ([]string, Warnings, error) { return labelNames, warnings, nil } -// Close releases the resources of the Querier. +// Close releases the resources of the generic querier. func (q *mergeGenericQuerier) Close() error { errs := tsdb_errors.MultiError{} for _, querier := range q.queriers { @@ -483,6 +497,9 @@ func newGenericMergeSeriesSet(sets []genericSeriesSet, mergeFunc genericSeriesMe if set.Next() { heap.Push(&h, set) } + if err := set.Err(); err != nil { + return errorOnlySeriesSet{err} + } } return &genericMergeSeriesSet{ mergeFunc: mergeFunc, @@ -583,35 +600,24 @@ func (h *genericSeriesSetHeap) Pop() interface{} { // with "almost" the same data, e.g. from 2 Prometheus HA replicas. This is fine, since from the Prometheus perspective // this never happens. // -// NOTE: Use this only when you see potentially overlapping series, as this introduces small overhead to handle overlaps -// between series. -func ChainedSeriesMerge(s ...Series) Series { - if len(s) == 0 { +// NOTE: Use this merge function only when you see potentially overlapping series, as this introduces a small overhead +// to handle overlaps between series. +func ChainedSeriesMerge(series ...Series) Series { + if len(series) == 0 { return nil } - return &chainSeries{ - labels: s[0].Labels(), - series: s, + return &SeriesEntry{ + Lset: series[0].Labels(), + SampleIteratorFn: func() chunkenc.Iterator { + iterators := make([]chunkenc.Iterator, 0, len(series)) + for _, s := range series { + iterators = append(iterators, s.Iterator()) + } + return newChainSampleIterator(iterators) + }, } } -type chainSeries struct { - labels labels.Labels - series []Series -} - -func (m *chainSeries) Labels() labels.Labels { - return m.labels -} - -func (m *chainSeries) Iterator() chunkenc.Iterator { - iterators := make([]chunkenc.Iterator, 0, len(m.series)) - for _, s := range m.series { - iterators = append(iterators, s.Iterator()) - } - return newChainSampleIterator(iterators) -} - // chainSampleIterator is responsible to iterate over samples from different iterators of the same time series in timestamps // order. If one or more samples overlap, one sample from random overlapped ones is kept and all others with the same // timestamp are dropped. @@ -710,67 +716,49 @@ func (h *samplesIteratorHeap) Pop() interface{} { return x } -type compactChunkSeriesMerger struct { - mergeFunc VerticalSeriesMergeFunc - - labels labels.Labels - series []ChunkSeries -} - // NewCompactingChunkSeriesMerger returns VerticalChunkSeriesMergeFunc that merges the same chunk series into single chunk series. // In case of the chunk overlaps, it compacts those into one or more time-ordered non-overlapping chunks with merged data. // Samples from overlapped chunks are merged using series vertical merge func. // It expects the same labels for each given series. // -// NOTE: Use this only when you see potentially overlapping series, as this introduces small overhead to handle overlaps -// between series. +// NOTE: Use the returned merge function only when you see potentially overlapping series, as this introduces small a overhead +// to handle overlaps between series. func NewCompactingChunkSeriesMerger(mergeFunc VerticalSeriesMergeFunc) VerticalChunkSeriesMergeFunc { - return func(s ...ChunkSeries) ChunkSeries { - if len(s) == 0 { + return func(series ...ChunkSeries) ChunkSeries { + if len(series) == 0 { return nil } - return &compactChunkSeriesMerger{ - mergeFunc: mergeFunc, - labels: s[0].Labels(), - series: s, + return &ChunkSeriesEntry{ + Lset: series[0].Labels(), + ChunkIteratorFn: func() chunks.Iterator { + iterators := make([]chunks.Iterator, 0, len(series)) + for _, s := range series { + iterators = append(iterators, s.Iterator()) + } + return &compactChunkIterator{ + mergeFunc: mergeFunc, + iterators: iterators, + } + }, } } } -func (s *compactChunkSeriesMerger) Labels() labels.Labels { - return s.labels -} - -func (s *compactChunkSeriesMerger) Iterator() chunks.Iterator { - iterators := make([]chunks.Iterator, 0, len(s.series)) - for _, series := range s.series { - iterators = append(iterators, series.Iterator()) - } - return &compactChunkIterator{ - mergeFunc: s.mergeFunc, - labels: s.labels, - iterators: iterators, - } -} - // compactChunkIterator is responsible to compact chunks from different iterators of the same time series into single chainSeries. // If time-overlapping chunks are found, they are encoded and passed to series merge and encoded again into one bigger chunk. // TODO(bwplotka): Currently merge will compact overlapping chunks with bigger chunk, without limit. Split it: https://github.com/prometheus/tsdb/issues/670 type compactChunkIterator struct { mergeFunc VerticalSeriesMergeFunc - - labels labels.Labels iterators []chunks.Iterator h chunkIteratorHeap + + err error + curr chunks.Meta } func (c *compactChunkIterator) At() chunks.Meta { - if len(c.h) == 0 { - panic("compactChunkIterator.At() called after .Next() returned false.") - } - - return c.h[0].At() + return c.curr } func (c *compactChunkIterator) Next() bool { @@ -780,59 +768,65 @@ func (c *compactChunkIterator) Next() bool { heap.Push(&c.h, iter) } } - return len(c.h) > 0 } - if len(c.h) == 0 { return false } - // Detect overlaps to compact. - // Be smart about it and deduplicate on the fly if chunks are identical. - last := c.At() - var overlapped []Series - for { + iter := heap.Pop(&c.h).(chunks.Iterator) + c.curr = iter.At() + if iter.Next() { + heap.Push(&c.h, iter) + } + + var ( + overlapping []Series + oMaxTime = c.curr.MaxTime + prev = c.curr + ) + // Detect overlaps to compact. Be smart about it and deduplicate on the fly if chunks are identical. + for len(c.h) > 0 { + // Get the next oldest chunk by min, then max time. + next := c.h[0].At() + if next.MinTime > oMaxTime { + // No overlap with current one. + break + } + + if next.MinTime == prev.MinTime && + next.MaxTime == prev.MaxTime && + bytes.Equal(next.Chunk.Bytes(), prev.Chunk.Bytes()) { + // 1:1 duplicates, skip it. + } else { + // We operate on same series, so labels does not matter here. + overlapping = append(overlapping, newChunkToSeriesDecoder(nil, next)) + if next.MaxTime > oMaxTime { + oMaxTime = next.MaxTime + } + prev = next + } + iter := heap.Pop(&c.h).(chunks.Iterator) if iter.Next() { heap.Push(&c.h, iter) } - if len(c.h) == 0 { - break - } - - // Get the current oldest chunk by min, then max time. - next := c.At() - if next.MinTime > last.MaxTime { - // No overlap with last one. - break - } - - if next.MinTime == last.MinTime && - next.MaxTime == last.MaxTime && - bytes.Equal(next.Chunk.Bytes(), last.Chunk.Bytes()) { - // 1:1 duplicates, skip last. - continue - } - - overlapped = append(overlapped, &chunkToSeriesDecoder{ - labels: c.labels, - Meta: last, - }) - last = next + } + if len(overlapping) == 0 { + return true } - if len(overlapped) == 0 { - return len(c.h) > 0 + // Add last as it's not yet included in overlap. We operate on same series, so labels does not matter here. + iter = (&seriesToChunkEncoder{Series: c.mergeFunc(append(overlapping, newChunkToSeriesDecoder(nil, c.curr))...)}).Iterator() + if !iter.Next() { + if c.err = iter.Err(); c.err != nil { + return false + } + panic("unexpected seriesToChunkEncoder lack of iterations") + } + c.curr = iter.At() + if iter.Next() { + heap.Push(&c.h, iter) } - - // Add last, not yet included overlap. - overlapped = append(overlapped, &chunkToSeriesDecoder{ - labels: c.labels, - Meta: c.At(), - }) - - var chkSeries ChunkSeries = &seriesToChunkEncoder{Series: c.mergeFunc(overlapped...)} - heap.Push(&c.h, chkSeries) return true } @@ -843,6 +837,7 @@ func (c *compactChunkIterator) Err() error { errs.Add(err) } } + errs.Add(c.err) return errs.Err() } diff --git a/storage/fanout_test.go b/storage/fanout_test.go index dc68fd740..d839a63d8 100644 --- a/storage/fanout_test.go +++ b/storage/fanout_test.go @@ -28,35 +28,6 @@ import ( "github.com/prometheus/prometheus/util/testutil" ) -func TestMergeStringSlices(t *testing.T) { - for _, tc := range []struct { - input [][]string - expected []string - }{ - {}, - {[][]string{{"foo"}}, []string{"foo"}}, - {[][]string{{"foo"}, {"bar"}}, []string{"bar", "foo"}}, - {[][]string{{"foo"}, {"bar"}, {"baz"}}, []string{"bar", "baz", "foo"}}, - } { - testutil.Equals(t, tc.expected, mergeStringSlices(tc.input)) - } -} - -func TestMergeTwoStringSlices(t *testing.T) { - for _, tc := range []struct { - a, b, expected []string - }{ - {[]string{}, []string{}, []string{}}, - {[]string{"foo"}, nil, []string{"foo"}}, - {nil, []string{"bar"}, []string{"bar"}}, - {[]string{"foo"}, []string{"bar"}, []string{"bar", "foo"}}, - {[]string{"foo"}, []string{"bar", "baz"}, []string{"bar", "baz", "foo"}}, - {[]string{"foo"}, []string{"foo"}, []string{"foo"}}, - } { - testutil.Equals(t, tc.expected, mergeTwoStringSlices(tc.a, tc.b)) - } -} - func TestMergeQuerierWithChainMerger(t *testing.T) { for _, tc := range []struct { name string @@ -215,7 +186,7 @@ func TestMergeQuerierWithChainMerger(t *testing.T) { } qs = append(qs, tc.extraQueriers...) - mergedQuerier := NewMergeQuerier(p, qs, ChainedSeriesMerge).Select(false, nil) + mergedQuerier := NewMergeQuerier([]Querier{p}, qs, ChainedSeriesMerge).Select(false, nil) // Get all merged series upfront to make sure there are no incorrectly retained shared // buffers causing bugs. @@ -230,8 +201,8 @@ func TestMergeQuerierWithChainMerger(t *testing.T) { expectedSeries := tc.expected.At() testutil.Equals(t, expectedSeries.Labels(), actualSeries.Labels()) - expSmpl, expErr := ExpandSamples(expectedSeries.Iterator()) - actSmpl, actErr := ExpandSamples(actualSeries.Iterator()) + expSmpl, expErr := ExpandSamples(expectedSeries.Iterator(), nil) + actSmpl, actErr := ExpandSamples(actualSeries.Iterator(), nil) testutil.Equals(t, expErr, actErr) testutil.Equals(t, expSmpl, actSmpl) } @@ -391,7 +362,7 @@ func TestMergeChunkQuerierWithNoVerticalChunkSeriesMerger(t *testing.T) { } qs = append(qs, tc.extraQueriers...) - merged := NewMergeChunkQuerier(p, qs, NewCompactingChunkSeriesMerger(nil)).Select(false, nil) + merged := NewMergeChunkQuerier([]ChunkQuerier{p}, qs, NewCompactingChunkSeriesMerger(nil)).Select(false, nil) for merged.Next() { testutil.Assert(t, tc.expected.Next(), "Expected Next() to be true") actualSeries := merged.At() @@ -410,6 +381,103 @@ func TestMergeChunkQuerierWithNoVerticalChunkSeriesMerger(t *testing.T) { } } +func TestCompactingChunkSeriesMerger(t *testing.T) { + m := NewCompactingChunkSeriesMerger(ChainedSeriesMerge) + + for _, tc := range []struct { + name string + input []ChunkSeries + expected ChunkSeries + }{ + { + name: "single empty series", + input: []ChunkSeries{ + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), nil), + }, + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), nil), + }, + { + name: "single series", + input: []ChunkSeries{ + 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}, sample{2, 2}}, []tsdbutil.Sample{sample{3, 3}}), + }, + { + name: "two empty series", + input: []ChunkSeries{ + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), nil), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), nil), + }, + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), nil), + }, + { + 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}}), + }, + 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}}), + }, + { + 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}}), + }, + 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}}), + }, + { + 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}}), + }, + expected: NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}, sample{5, 5}}), + }, + { + 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}}), + }, + 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}}), + }, + { + 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, 6}}), + NewListChunkSeriesFromSamples(labels.FromStrings("bar", "baz"), []tsdbutil.Sample{sample{6, 6}, sample{10, 10}}), + }, + 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, 6}, sample{10, 10}}), + }, + { + 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}}), + }, + 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}}, + ), + }, + } { + t.Run(tc.name, func(t *testing.T) { + merged := m(tc.input...) + testutil.Equals(t, tc.expected.Labels(), merged.Labels()) + actChks, actErr := ExpandChunks(merged.Iterator()) + expChks, expErr := ExpandChunks(tc.expected.Iterator()) + + testutil.Equals(t, expErr, actErr) + testutil.Equals(t, expChks, actChks) + }) + } +} + type mockQuerier struct { LabelQuerier @@ -543,7 +611,7 @@ func TestChainSampleIterator(t *testing.T) { }, } { merged := newChainSampleIterator(tc.input) - actual, err := ExpandSamples(merged) + actual, err := ExpandSamples(merged, nil) testutil.Ok(t, err) testutil.Equals(t, tc.expected, actual) } @@ -586,7 +654,7 @@ func TestChainSampleIteratorSeek(t *testing.T) { t, v := merged.At() actual = append(actual, sample{t, v}) } - s, err := ExpandSamples(merged) + s, err := ExpandSamples(merged, nil) testutil.Ok(t, err) actual = append(actual, s...) testutil.Equals(t, tc.expected, actual) @@ -620,7 +688,7 @@ func benchmarkDrain(seriesSet SeriesSet, b *testing.B) { var err error for n := 0; n < b.N; n++ { for seriesSet.Next() { - result, err = ExpandSamples(seriesSet.At().Iterator()) + result, err = ExpandSamples(seriesSet.At().Iterator(), nil) testutil.Ok(b, err) } } diff --git a/storage/generic.go b/storage/generic.go index daa14555a..817122bc1 100644 --- a/storage/generic.go +++ b/storage/generic.go @@ -16,7 +16,9 @@ package storage -import "github.com/prometheus/prometheus/pkg/labels" +import ( + "github.com/prometheus/prometheus/pkg/labels" +) type genericQuerier interface { LabelQuerier diff --git a/storage/interface.go b/storage/interface.go index e01a88ecf..a10178aee 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -20,7 +20,6 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" - "github.com/prometheus/prometheus/tsdb/tombstones" ) // The errors exposed. @@ -161,6 +160,7 @@ type Appender interface { // SeriesSet contains a set of series. type SeriesSet interface { Next() bool + // At returns full series. Returned series should be iteratable even after Next is called. At() Series // The error that iteration as failed with. // When an error occurs, set cannot continue to iterate. @@ -221,6 +221,7 @@ type Series interface { // ChunkSeriesSet contains a set of chunked series. type ChunkSeriesSet interface { Next() bool + // At returns full chunk series. Returned series should be iteratable even after Next is called. At() ChunkSeries // The error that iteration has failed with. // When an error occurs, set cannot continue to iterate. @@ -243,20 +244,14 @@ type Labels interface { } type SampleIteratable interface { - // Iterator returns a new iterator of the data of the series. + // Iterator returns a new, independent iterator of the data of the series. Iterator() chunkenc.Iterator } type ChunkIteratable interface { - // ChunkIterator returns a new iterator that iterates over non-overlapping chunks of the series. + // Iterator returns a new, independent iterator that iterates over potentially overlapping + // chunks of the series, sorted by min time. Iterator() chunks.Iterator } -// TODO(bwplotka): Remove in next Pr. -type DeprecatedChunkSeriesSet interface { - Next() bool - At() (labels.Labels, []chunks.Meta, tombstones.Intervals) - Err() error -} - type Warnings []error diff --git a/storage/lazy.go b/storage/lazy.go index 8eaa8772a..62f76cb6a 100644 --- a/storage/lazy.go +++ b/storage/lazy.go @@ -13,37 +13,37 @@ package storage -type lazySeriesSet struct { - create func() (s genericSeriesSet, ok bool) +// lazyGenericSeriesSet is a wrapped series set that is initialised on first call to Next(). +type lazyGenericSeriesSet struct { + init func() (genericSeriesSet, bool) set genericSeriesSet } -func (c *lazySeriesSet) Next() bool { +func (c *lazyGenericSeriesSet) Next() bool { if c.set != nil { return c.set.Next() } - var ok bool - c.set, ok = c.create() + c.set, ok = c.init() return ok } -func (c *lazySeriesSet) Err() error { +func (c *lazyGenericSeriesSet) Err() error { if c.set != nil { return c.set.Err() } return nil } -func (c *lazySeriesSet) At() Labels { +func (c *lazyGenericSeriesSet) At() Labels { if c.set != nil { return c.set.At() } return nil } -func (c *lazySeriesSet) Warnings() Warnings { +func (c *lazyGenericSeriesSet) Warnings() Warnings { if c.set != nil { return c.set.Warnings() } diff --git a/storage/remote/storage.go b/storage/remote/storage.go index a9d60b196..8f716f942 100644 --- a/storage/remote/storage.go +++ b/storage/remote/storage.go @@ -147,7 +147,7 @@ func (s *Storage) Querier(ctx context.Context, mint, maxt int64) (storage.Querie } queriers = append(queriers, q) } - return storage.NewMergeQuerier(storage.NoopQuerier(), queriers, storage.ChainedSeriesMerge), nil + return storage.NewMergeQuerier(nil, queriers, storage.ChainedSeriesMerge), nil } // ChunkQuerier returns a storage.MergeQuerier combining the remote client queriers @@ -165,7 +165,7 @@ func (s *Storage) ChunkQuerier(ctx context.Context, mint, maxt int64) (storage.C } queriers = append(queriers, q) } - return storage.NewMergeChunkQuerier(storage.NoopChunkedQuerier(), queriers, storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge)), nil + return storage.NewMergeChunkQuerier(nil, queriers, storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge)), nil } // Appender implements storage.Storage. diff --git a/storage/secondary.go b/storage/secondary.go index 7f7093069..9e768b349 100644 --- a/storage/secondary.go +++ b/storage/secondary.go @@ -63,35 +63,37 @@ func (s *secondaryQuerier) LabelNames() ([]string, Warnings, error) { return names, w, nil } -func (s *secondaryQuerier) createFn(asyncSet genericSeriesSet) func() (genericSeriesSet, bool) { - s.asyncSets = append(s.asyncSets, asyncSet) +func (s *secondaryQuerier) Select(sortSeries bool, hints *SelectHints, matchers ...*labels.Matcher) genericSeriesSet { + if s.done { + panic("secondaryQuerier: Select invoked after first Next of any returned SeriesSet was done") + } + + s.asyncSets = append(s.asyncSets, s.genericQuerier.Select(sortSeries, hints, matchers...)) curr := len(s.asyncSets) - 1 - return func() (genericSeriesSet, bool) { + return &lazyGenericSeriesSet{init: func() (genericSeriesSet, bool) { s.once.Do(func() { - // At first create invocation we iterate over all sets and ensure its Next() returns some value without + // At first init invocation we iterate over all async sets and ensure its Next() returns some value without // errors. This is to ensure we support consistent partial failures. for i, set := range s.asyncSets { if set.Next() { continue } ws := set.Warnings() - // Failed set. if err := set.Err(); err != nil { - ws = append([]error{err}, ws...) - // Promote the warnings to the current one. - s.asyncSets[curr] = warningsOnlySeriesSet(ws) - // One of the sets failed, ensure rest of the sets returns nothing. (All or nothing logic). + // One of the sets failed, ensure current one returning errors as warnings, and rest of the sets return nothing. + // (All or nothing logic). + s.asyncSets[curr] = warningsOnlySeriesSet(append([]error{err}, ws...)) for i := range s.asyncSets { - if curr != i { - s.asyncSets[i] = noopGenericSeriesSet{} + if curr == i { + continue } + s.asyncSets[i] = noopGenericSeriesSet{} } break } // Exhausted set. s.asyncSets[i] = warningsOnlySeriesSet(ws) } - s.done = true }) @@ -101,12 +103,5 @@ func (s *secondaryQuerier) createFn(asyncSet genericSeriesSet) func() (genericSe default: return s.asyncSets[curr], true } - } -} - -func (s *secondaryQuerier) Select(sortSeries bool, hints *SelectHints, matchers ...*labels.Matcher) genericSeriesSet { - if s.done { - panic("secondaryQuerier: Select invoked after first Next of any returned SeriesSet was done") - } - return &lazySeriesSet{create: s.createFn(s.genericQuerier.Select(sortSeries, hints, matchers...))} + }} } diff --git a/storage/series.go b/storage/series.go index 237d2e376..a123743ef 100644 --- a/storage/series.go +++ b/storage/series.go @@ -23,6 +23,47 @@ import ( "github.com/prometheus/prometheus/tsdb/tsdbutil" ) +type SeriesEntry struct { + Lset labels.Labels + SampleIteratorFn func() chunkenc.Iterator +} + +func (s *SeriesEntry) Labels() labels.Labels { return s.Lset } +func (s *SeriesEntry) Iterator() chunkenc.Iterator { return s.SampleIteratorFn() } + +type ChunkSeriesEntry struct { + Lset labels.Labels + ChunkIteratorFn func() chunks.Iterator +} + +func (s *ChunkSeriesEntry) Labels() labels.Labels { return s.Lset } +func (s *ChunkSeriesEntry) Iterator() chunks.Iterator { return s.ChunkIteratorFn() } + +// NewListSeries returns series entry with iterator that allows to iterate over provided samples. +func NewListSeries(lset labels.Labels, s []tsdbutil.Sample) *SeriesEntry { + return &SeriesEntry{ + Lset: lset, + SampleIteratorFn: func() chunkenc.Iterator { + return NewListSeriesIterator(samples(s)) + }, + } +} + +// NewListChunkSeriesIterator returns chunk series entry that allows to iterate over provided samples. +// NOTE: It uses inefficient chunks encoding implementation, not caring about chunk size. +func NewListChunkSeriesFromSamples(lset labels.Labels, samples ...[]tsdbutil.Sample) *ChunkSeriesEntry { + return &ChunkSeriesEntry{ + Lset: lset, + ChunkIteratorFn: func() chunks.Iterator { + chks := make([]chunks.Meta, 0, len(samples)) + for _, s := range samples { + chks = append(chks, tsdbutil.ChunkFromSamples(s)) + } + return NewListChunkSeriesIterator(chks...) + }, + } +} + type listSeriesIterator struct { samples Samples idx int @@ -39,7 +80,7 @@ type Samples interface { Len() int } -// NewListSeriesIterator returns listSeriesIterator that allows to iterate over provided samples. Does not handle overlaps. +// NewListSeriesIterator returns listSeriesIterator that allows to iterate over provided samples. func NewListSeriesIterator(samples Samples) chunkenc.Iterator { return &listSeriesIterator{samples: samples, idx: -1} } @@ -71,11 +112,10 @@ func (it *listSeriesIterator) Err() error { return nil } type listChunkSeriesIterator struct { chks []chunks.Meta - - idx int + idx int } -// NewListChunkSeriesIterator returns listChunkSeriesIterator that allows to iterate over provided chunks. Does not handle overlaps. +// NewListChunkSeriesIterator returns listChunkSeriesIterator that allows to iterate over provided chunks. func NewListChunkSeriesIterator(chks ...chunks.Meta) chunks.Iterator { return &listChunkSeriesIterator{chks: chks, idx: -1} } @@ -112,17 +152,16 @@ func (c *chunkSetToSeriesSet) Next() bool { c.sameSeriesChunks = c.sameSeriesChunks[:0] for iter.Next() { - c.sameSeriesChunks = append(c.sameSeriesChunks, &chunkToSeriesDecoder{ - labels: c.ChunkSeriesSet.At().Labels(), - Meta: iter.At(), - }) + c.sameSeriesChunks = append( + c.sameSeriesChunks, + newChunkToSeriesDecoder(c.ChunkSeriesSet.At().Labels(), iter.At()), + ) } if iter.Err() != nil { c.chkIterErr = iter.Err() return false } - return true } @@ -138,17 +177,16 @@ func (c *chunkSetToSeriesSet) Err() error { return c.ChunkSeriesSet.Err() } -type chunkToSeriesDecoder struct { - chunks.Meta - - labels labels.Labels +func newChunkToSeriesDecoder(labels labels.Labels, chk chunks.Meta) Series { + return &SeriesEntry{ + Lset: labels, + SampleIteratorFn: func() chunkenc.Iterator { + // TODO(bwplotka): Can we provide any chunkenc buffer? + return chk.Chunk.Iterator(nil) + }, + } } -func (s *chunkToSeriesDecoder) Labels() labels.Labels { return s.labels } - -// TODO(bwplotka): Can we provide any chunkenc buffer? -func (s *chunkToSeriesDecoder) Iterator() chunkenc.Iterator { return s.Chunk.Iterator(nil) } - type seriesSetToChunkSet struct { SeriesSet } @@ -217,3 +255,32 @@ type errChunksIterator struct { func (e errChunksIterator) At() chunks.Meta { return chunks.Meta{} } func (e errChunksIterator) Next() bool { return false } 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) { + if newSampleFn == nil { + newSampleFn = func(t int64, v float64) tsdbutil.Sample { return sample{t, v} } + } + + 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 + } + result = append(result, newSampleFn(t, v)) + } + return result, iter.Err() +} + +// ExpandChunks iterates over all chunks in the iterator, buffering all in slice. +func ExpandChunks(iter chunks.Iterator) ([]chunks.Meta, error) { + var result []chunks.Meta + for iter.Next() { + result = append(result, iter.At()) + } + return result, iter.Err() +} diff --git a/storage/series_test.go b/storage/series_test.go deleted file mode 100644 index 09fd95141..000000000 --- a/storage/series_test.go +++ /dev/null @@ -1,84 +0,0 @@ -// Copyright 2020 The Prometheus Authors -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package storage - -import ( - "math" - - "github.com/prometheus/prometheus/pkg/labels" - "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/tsdb/chunks" - "github.com/prometheus/prometheus/tsdb/tsdbutil" -) - -type MockSeries struct { - labels labels.Labels - SampleIteratorFn func() chunkenc.Iterator -} - -func NewListSeries(lset labels.Labels, s []tsdbutil.Sample) *MockSeries { - return &MockSeries{ - labels: lset, - SampleIteratorFn: func() chunkenc.Iterator { - return NewListSeriesIterator(samples(s)) - }, - } -} - -func (s *MockSeries) Labels() labels.Labels { return s.labels } -func (s *MockSeries) Iterator() chunkenc.Iterator { return s.SampleIteratorFn() } - -type MockChunkSeries struct { - labels labels.Labels - ChunkIteratorFn func() chunks.Iterator -} - -func NewListChunkSeriesFromSamples(lset labels.Labels, samples ...[]tsdbutil.Sample) *MockChunkSeries { - var chks []chunks.Meta - - return &MockChunkSeries{ - labels: lset, - ChunkIteratorFn: func() chunks.Iterator { - // Inefficient chunks encoding implementation, not caring about chunk size. - for _, s := range samples { - chks = append(chks, tsdbutil.ChunkFromSamples(s)) - } - return NewListChunkSeriesIterator(chks...) - }, - } -} - -func (s *MockChunkSeries) Labels() labels.Labels { return s.labels } -func (s *MockChunkSeries) Iterator() chunks.Iterator { return s.ChunkIteratorFn() } - -func ExpandSamples(iter chunkenc.Iterator) ([]tsdbutil.Sample, error) { - 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 - } - result = append(result, sample{t, v}) - } - return result, iter.Err() -} - -func ExpandChunks(iter chunks.Iterator) ([]chunks.Meta, error) { - var result []chunks.Meta - for iter.Next() { - result = append(result, iter.At()) - } - return result, iter.Err() -} diff --git a/tsdb/block.go b/tsdb/block.go index a4b03f07f..d708bc809 100644 --- a/tsdb/block.go +++ b/tsdb/block.go @@ -44,10 +44,9 @@ type IndexWriter interface { // AddSeries populates the index writer with a series and its offsets // of chunks that the index can reference. - // Implementations may require series to be insert in increasing order by - // their labels. - // The reference numbers are used to resolve entries in postings lists that - // are added later. + // Implementations may require series to be insert in strictly increasing order by + // their labels. The reference numbers are used to resolve entries in postings lists + // that are added later. AddSeries(ref uint64, l labels.Labels, chunks ...chunks.Meta) error // Close writes any finalization and closes the resources associated with diff --git a/tsdb/block_test.go b/tsdb/block_test.go index 17f403244..0ccaa7abd 100644 --- a/tsdb/block_test.go +++ b/tsdb/block_test.go @@ -16,7 +16,6 @@ package tsdb import ( "context" "encoding/binary" - "errors" "hash/crc32" "io/ioutil" @@ -32,6 +31,7 @@ import ( "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/util/testutil" ) @@ -88,21 +88,22 @@ func TestCreateBlock(t *testing.T) { } func TestCorruptedChunk(t *testing.T) { - for name, test := range map[string]struct { + for _, tc := range []struct { + name string corrFunc func(f *os.File) // Func that applies the corruption. openErr error - queryErr error + iterErr error }{ - "invalid header size": { - func(f *os.File) { - err := f.Truncate(1) - testutil.Ok(t, err) + { + name: "invalid header size", + corrFunc: func(f *os.File) { + testutil.Ok(t, f.Truncate(1)) }, - errors.New("invalid segment header in segment 0: invalid size"), - nil, + openErr: errors.New("invalid segment header in segment 0: invalid size"), }, - "invalid magic number": { - func(f *os.File) { + { + name: "invalid magic number", + corrFunc: func(f *os.File) { magicChunksOffset := int64(0) _, err := f.Seek(magicChunksOffset, 0) testutil.Ok(t, err) @@ -114,11 +115,11 @@ func TestCorruptedChunk(t *testing.T) { testutil.Ok(t, err) testutil.Equals(t, chunks.MagicChunksSize, n) }, - errors.New("invalid magic number 0"), - nil, + openErr: errors.New("invalid magic number 0"), }, - "invalid chunk format version": { - func(f *os.File) { + { + name: "invalid chunk format version", + corrFunc: func(f *os.File) { chunksFormatVersionOffset := int64(4) _, err := f.Seek(chunksFormatVersionOffset, 0) testutil.Ok(t, err) @@ -130,31 +131,28 @@ func TestCorruptedChunk(t *testing.T) { testutil.Ok(t, err) testutil.Equals(t, chunks.ChunksFormatVersionSize, n) }, - errors.New("invalid chunk format version 0"), - nil, + openErr: errors.New("invalid chunk format version 0"), }, - "chunk not enough bytes to read the chunk length": { - func(f *os.File) { + { + name: "chunk not enough bytes to read the chunk length", + corrFunc: func(f *os.File) { // Truncate one byte after the segment header. - err := f.Truncate(chunks.SegmentHeaderSize + 1) - testutil.Ok(t, err) + testutil.Ok(t, f.Truncate(chunks.SegmentHeaderSize+1)) }, - nil, - errors.New("segment doesn't include enough bytes to read the chunk size data field - required:13, available:9"), + iterErr: errors.New("cannot populate chunk 8: segment doesn't include enough bytes to read the chunk size data field - required:13, available:9"), }, - "chunk not enough bytes to read the data": { - func(f *os.File) { + { + name: "chunk not enough bytes to read the data", + corrFunc: func(f *os.File) { fi, err := f.Stat() testutil.Ok(t, err) - - err = f.Truncate(fi.Size() - 1) - testutil.Ok(t, err) + testutil.Ok(t, f.Truncate(fi.Size()-1)) }, - nil, - errors.New("segment doesn't include enough bytes to read the chunk - required:26, available:25"), + iterErr: errors.New("cannot populate chunk 8: segment doesn't include enough bytes to read the chunk - required:26, available:25"), }, - "checksum mismatch": { - func(f *os.File) { + { + name: "checksum mismatch", + corrFunc: func(f *os.File) { fi, err := f.Stat() testutil.Ok(t, err) @@ -168,18 +166,17 @@ func TestCorruptedChunk(t *testing.T) { testutil.Ok(t, err) testutil.Equals(t, n, 1) }, - nil, - errors.New("checksum mismatch expected:cfc0526c, actual:34815eae"), + iterErr: errors.New("cannot populate chunk 8: checksum mismatch expected:cfc0526c, actual:34815eae"), }, } { - t.Run(name, func(t *testing.T) { + t.Run(tc.name, func(t *testing.T) { tmpdir, err := ioutil.TempDir("", "test_open_block_chunk_corrupted") testutil.Ok(t, err) defer func() { testutil.Ok(t, os.RemoveAll(tmpdir)) }() - series := newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{sample{1, 1}}) + series := storage.NewListSeries(labels.FromStrings("a", "b"), []tsdbutil.Sample{sample{1, 1}}) blockDir := createBlock(t, tmpdir, []storage.Series{series}) files, err := sequenceFiles(chunkDir(blockDir)) testutil.Ok(t, err) @@ -189,13 +186,13 @@ func TestCorruptedChunk(t *testing.T) { testutil.Ok(t, err) // Apply corruption function. - test.corrFunc(f) + tc.corrFunc(f) testutil.Ok(t, f.Close()) // Check open err. b, err := OpenBlock(nil, blockDir, nil) - if test.openErr != nil { - testutil.Equals(t, test.openErr.Error(), err.Error()) + if tc.openErr != nil { + testutil.Equals(t, tc.openErr.Error(), err.Error()) return } defer func() { testutil.Ok(t, b.Close()) }() @@ -205,9 +202,11 @@ func TestCorruptedChunk(t *testing.T) { defer func() { testutil.Ok(t, querier.Close()) }() set := querier.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) - // Check query err. - testutil.Equals(t, false, set.Next()) - testutil.Equals(t, test.queryErr.Error(), set.Err().Error()) + // Check chunk errors during iter time. + testutil.Assert(t, set.Next(), "") + it := set.At().Iterator() + testutil.Equals(t, false, it.Next()) + testutil.Equals(t, tc.iterErr.Error(), it.Err().Error()) }) } } @@ -306,7 +305,7 @@ func createBlock(tb testing.TB, dir string, series []storage.Series) string { chunkDir, err := ioutil.TempDir("", "chunk_dir") testutil.Ok(tb, err) defer func() { testutil.Ok(tb, os.RemoveAll(chunkDir)) }() - head := createHead(tb, series, chunkDir) + head := createHead(tb, nil, series, chunkDir) defer func() { testutil.Ok(tb, head.Close()) }() return createBlockFromHead(tb, dir, head) } @@ -324,8 +323,8 @@ func createBlockFromHead(tb testing.TB, dir string, head *Head) string { return filepath.Join(dir, ulid.String()) } -func createHead(tb testing.TB, series []storage.Series, chunkDir string) *Head { - head, err := NewHead(nil, nil, nil, 2*60*60*1000, chunkDir, nil, DefaultStripeSize, nil) +func createHead(tb testing.TB, w *wal.WAL, series []storage.Series, chunkDir string) *Head { + head, err := NewHead(nil, nil, w, DefaultBlockDuration, chunkDir, nil, DefaultStripeSize, nil) testutil.Ok(tb, err) app := head.Appender(context.Background()) @@ -345,8 +344,7 @@ func createHead(tb testing.TB, series []storage.Series, chunkDir string) *Head { } testutil.Ok(tb, it.Err()) } - err = app.Commit() - testutil.Ok(tb, err) + testutil.Ok(tb, app.Commit()) return head } @@ -373,7 +371,7 @@ func genSeries(totalSeries, labelCount int, mint, maxt int64) []storage.Series { for t := mint; t < maxt; t++ { samples = append(samples, sample{t: t, v: rand.Float64()}) } - series[i] = newSeries(lbls, samples) + series[i] = storage.NewListSeries(labels.FromMap(lbls), samples) } return series } @@ -393,7 +391,7 @@ func populateSeries(lbls []map[string]string, mint, maxt int64) []storage.Series for t := mint; t <= maxt; t++ { samples = append(samples, sample{t: t, v: rand.Float64()}) } - series = append(series, newSeries(lbl, samples)) + series = append(series, storage.NewListSeries(labels.FromMap(lbl), samples)) } return series } diff --git a/tsdb/chunks/chunks.go b/tsdb/chunks/chunks.go index 5a824d965..7d0c50159 100644 --- a/tsdb/chunks/chunks.go +++ b/tsdb/chunks/chunks.go @@ -283,85 +283,6 @@ func (w *Writer) write(b []byte) error { return err } -// MergeOverlappingChunks removes the samples whose timestamp is overlapping. -// The last appearing sample is retained in case there is overlapping. -// This assumes that `chks []Meta` is sorted w.r.t. MinTime. -func MergeOverlappingChunks(chks []Meta) ([]Meta, error) { - if len(chks) < 2 { - return chks, nil - } - newChks := make([]Meta, 0, len(chks)) // Will contain the merged chunks. - newChks = append(newChks, chks[0]) - last := 0 - for _, c := range chks[1:] { - // We need to check only the last chunk in newChks. - // Reason: (1) newChks[last-1].MaxTime < newChks[last].MinTime (non overlapping) - // (2) As chks are sorted w.r.t. MinTime, newChks[last].MinTime < c.MinTime. - // So never overlaps with newChks[last-1] or anything before that. - if c.MinTime > newChks[last].MaxTime { - newChks = append(newChks, c) - last++ - continue - } - nc := &newChks[last] - if c.MaxTime > nc.MaxTime { - nc.MaxTime = c.MaxTime - } - chk, err := MergeChunks(nc.Chunk, c.Chunk) - if err != nil { - return nil, err - } - nc.Chunk = chk - } - - return newChks, nil -} - -// MergeChunks vertically merges a and b, i.e., if there is any sample -// with same timestamp in both a and b, the sample in a is discarded. -func MergeChunks(a, b chunkenc.Chunk) (*chunkenc.XORChunk, error) { - newChunk := chunkenc.NewXORChunk() - app, err := newChunk.Appender() - if err != nil { - return nil, err - } - ait := a.Iterator(nil) - bit := b.Iterator(nil) - aok, bok := ait.Next(), bit.Next() - for aok && bok { - at, av := ait.At() - bt, bv := bit.At() - if at < bt { - app.Append(at, av) - aok = ait.Next() - } else if bt < at { - app.Append(bt, bv) - bok = bit.Next() - } else { - app.Append(bt, bv) - aok = ait.Next() - bok = bit.Next() - } - } - for aok { - at, av := ait.At() - app.Append(at, av) - aok = ait.Next() - } - for bok { - bt, bv := bit.At() - app.Append(bt, bv) - bok = bit.Next() - } - if ait.Err() != nil { - return nil, ait.Err() - } - if bit.Err() != nil { - return nil, bit.Err() - } - return newChunk, nil -} - // WriteChunks writes as many chunks as possible to the current segment, // cuts a new segment when the current segment is full and // writes the rest of the chunks in the new segment. diff --git a/tsdb/compact.go b/tsdb/compact.go index c3078ed8f..8ce1b3ca7 100644 --- a/tsdb/compact.go +++ b/tsdb/compact.go @@ -575,7 +575,7 @@ func (c *LeveledCompactor) write(dest string, meta *BlockMeta, blocks ...BlockRe closers = append(closers, indexw) if err := c.populateBlock(blocks, meta, indexw, chunkw); err != nil { - return errors.Wrap(err, "write compaction") + return errors.Wrap(err, "populate block") } select { @@ -648,9 +648,9 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, } var ( - set storage.DeprecatedChunkSeriesSet + sets []storage.ChunkSeriesSet symbols index.StringIter - closers = []io.Closer{} + closers []io.Closer overlapping bool ) defer func() { @@ -707,18 +707,13 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, return err } all = indexr.SortedPostings(all) - - s := newCompactionSeriesSet(indexr, chunkr, tombsr, all) + // Blocks meta is half open: [min, max), so subtract 1 to ensure we don't hold samples with exact meta.MaxTime timestamp. + sets = append(sets, newBlockChunkSeriesSet(indexr, chunkr, tombsr, all, meta.MinTime, meta.MaxTime-1)) syms := indexr.Symbols() if i == 0 { - set = s symbols = syms continue } - set, err = newCompactionMerger(set, s) - if err != nil { - return err - } symbols = newMergedStringIter(symbols, syms) } @@ -731,21 +726,34 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, return errors.Wrap(symbols.Err(), "next symbol") } - delIter := &deletedIterator{} - ref := uint64(0) + var ( + ref = uint64(0) + chks []chunks.Meta + ) + + set := sets[0] + if len(sets) > 1 { + // Merge series using compacting chunk series merger. + set = storage.NewMergeChunkSeriesSet(sets, storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge)) + } + + // Iterate over all sorted chunk series. for set.Next() { select { case <-c.ctx.Done(): return c.ctx.Err() default: } - - lset, chks, dranges := set.At() // The chunks here are not fully deleted. - if overlapping { - // If blocks are overlapping, it is possible to have unsorted chunks. - sort.Slice(chks, func(i, j int) bool { - return chks[i].MinTime < chks[j].MinTime - }) + s := set.At() + chksIter := s.Iterator() + chks = chks[:0] + for chksIter.Next() { + // We are not iterating in streaming way over chunk as it's more efficient to do bulk write for index and + // chunk file purposes. + chks = append(chks, chksIter.At()) + } + if chksIter.Err() != nil { + return errors.Wrap(chksIter.Err(), "chunk iter") } // Skip the series with all deleted chunks. @@ -753,85 +761,24 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, continue } - for i, chk := range chks { - // Re-encode head chunks that are still open (being appended to) or - // outside the compacted MaxTime range. - // The chunk.Bytes() method is not safe for open chunks hence the re-encoding. - // This happens when snapshotting the head block. - // - // Block time range is half-open: [meta.MinTime, meta.MaxTime) and - // chunks are closed hence the chk.MaxTime >= meta.MaxTime check. - // - // TODO think how to avoid the typecasting to verify when it is head block. - if _, isHeadChunk := chk.Chunk.(*safeChunk); isHeadChunk && chk.MaxTime >= meta.MaxTime { - dranges = append(dranges, tombstones.Interval{Mint: meta.MaxTime, Maxt: math.MaxInt64}) - - } else - // Sanity check for disk blocks. - // chk.MaxTime == meta.MaxTime shouldn't happen as well, but will brake many users so not checking for that. - if chk.MinTime < meta.MinTime || chk.MaxTime > meta.MaxTime { - return errors.Errorf("found chunk with minTime: %d maxTime: %d outside of compacted minTime: %d maxTime: %d", - chk.MinTime, chk.MaxTime, meta.MinTime, meta.MaxTime) - } - - if len(dranges) > 0 { - // Re-encode the chunk to not have deleted values. - if !chk.OverlapsClosedInterval(dranges[0].Mint, dranges[len(dranges)-1].Maxt) { - continue - } - newChunk := chunkenc.NewXORChunk() - app, err := newChunk.Appender() - if err != nil { - return err - } - - delIter.it = chk.Chunk.Iterator(delIter.it) - delIter.intervals = dranges - - var ( - t int64 - v float64 - ) - for delIter.Next() { - t, v = delIter.At() - app.Append(t, v) - } - if err := delIter.Err(); err != nil { - return errors.Wrap(err, "iterate chunk while re-encoding") - } - - chks[i].Chunk = newChunk - chks[i].MaxTime = t - } - } - - mergedChks := chks - if overlapping { - mergedChks, err = chunks.MergeOverlappingChunks(chks) - if err != nil { - return errors.Wrap(err, "merge overlapping chunks") - } - } - if err := chunkw.WriteChunks(mergedChks...); err != nil { + if err := chunkw.WriteChunks(chks...); err != nil { return errors.Wrap(err, "write chunks") } - - if err := indexw.AddSeries(ref, lset, mergedChks...); err != nil { + if err := indexw.AddSeries(ref, s.Labels(), chks...); err != nil { return errors.Wrap(err, "add series") } - meta.Stats.NumChunks += uint64(len(mergedChks)) + meta.Stats.NumChunks += uint64(len(chks)) meta.Stats.NumSeries++ - for _, chk := range mergedChks { + for _, chk := range chks { meta.Stats.NumSamples += uint64(chk.Chunk.NumSamples()) } - for _, chk := range mergedChks { + for _, chk := range chks { if err := c.chunkPool.Put(chk.Chunk); err != nil { return errors.Wrap(err, "put chunk") } } - ref++ } if set.Err() != nil { @@ -841,166 +788,354 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, return nil } -type compactionSeriesSet struct { +// blockBaseSeriesSet allows to iterate over all series in the single block. +// Iterated series are trimmed with given min and max time as well as tombstones. +// See newBlockSeriesSet and newBlockChunkSeriesSet to use it for either sample or chunk iterating. +type blockBaseSeriesSet struct { p index.Postings index IndexReader chunks ChunkReader tombstones tombstones.Reader + mint, maxt int64 - l labels.Labels - c []chunks.Meta - intervals tombstones.Intervals - err error + currIterFn func() *populateWithDelGenericSeriesIterator + currLabels labels.Labels + + bufChks []chunks.Meta + err error } -func newCompactionSeriesSet(i IndexReader, c ChunkReader, t tombstones.Reader, p index.Postings) *compactionSeriesSet { - return &compactionSeriesSet{ - index: i, - chunks: c, - tombstones: t, - p: p, - } -} +func (b *blockBaseSeriesSet) Next() bool { + var lbls labels.Labels -func (c *compactionSeriesSet) Next() bool { - if !c.p.Next() { - return false - } - var err error + for b.p.Next() { + if err := b.index.Series(b.p.At(), &lbls, &b.bufChks); err != nil { + // Postings may be stale. Skip if no underlying series exists. + if errors.Cause(err) == storage.ErrNotFound { + continue + } + b.err = errors.Wrapf(err, "get series %d", b.p.At()) + return false + } - c.intervals, err = c.tombstones.Get(c.p.At()) - if err != nil { - c.err = errors.Wrap(err, "get tombstones") - return false - } + if len(b.bufChks) == 0 { + continue + } - if err = c.index.Series(c.p.At(), &c.l, &c.c); err != nil { - c.err = errors.Wrapf(err, "get series %d", c.p.At()) - return false - } + intervals, err := b.tombstones.Get(b.p.At()) + if err != nil { + b.err = errors.Wrap(err, "get tombstones") + return false + } - // Remove completely deleted chunks. - if len(c.intervals) > 0 { - chks := make([]chunks.Meta, 0, len(c.c)) - for _, chk := range c.c { - if !(tombstones.Interval{Mint: chk.MinTime, Maxt: chk.MaxTime}.IsSubrange(c.intervals)) { + // NOTE: + // * block time range is half-open: [meta.MinTime, meta.MaxTime). + // * chunks are both closed: [chk.MinTime, chk.MaxTime]. + // * requested time ranges are closed: [req.Start, req.End]. + + var trimFront, trimBack bool + + // Copy chunks as iteratables are reusable. + chks := make([]chunks.Meta, 0, len(b.bufChks)) + + // Prefilter chunks and pick those which are not entirely deleted or totally outside of the requested range. + for _, chk := range b.bufChks { + if chk.MaxTime < b.mint { + continue + } + if chk.MinTime > b.maxt { + continue + } + + if !(tombstones.Interval{Mint: chk.MinTime, Maxt: chk.MaxTime}.IsSubrange(intervals)) { chks = append(chks, chk) } + + // If still not entirely deleted, check if trim is needed based on requested time range. + if chk.MinTime < b.mint { + trimFront = true + } + if chk.MaxTime > b.maxt { + trimBack = true + } } - c.c = chks - } - - for i := range c.c { - chk := &c.c[i] - - chk.Chunk, err = c.chunks.Chunk(chk.Ref) - if err != nil { - c.err = errors.Wrapf(err, "chunk %d not found", chk.Ref) - return false + if len(chks) == 0 { + continue } + + if trimFront { + intervals = intervals.Add(tombstones.Interval{Mint: math.MinInt64, Maxt: b.mint - 1}) + } + if trimBack { + intervals = intervals.Add(tombstones.Interval{Mint: b.maxt + 1, Maxt: math.MaxInt64}) + } + b.currLabels = lbls + b.currIterFn = func() *populateWithDelGenericSeriesIterator { + return newPopulateWithDelGenericSeriesIterator(b.chunks, chks, intervals) + } + return true } - - return true + return false } -func (c *compactionSeriesSet) Err() error { - if c.err != nil { - return c.err +func (b *blockBaseSeriesSet) Err() error { + if b.err != nil { + return b.err } - return c.p.Err() + return b.p.Err() } -func (c *compactionSeriesSet) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) { - return c.l, c.c, c.intervals -} +func (b *blockBaseSeriesSet) Warnings() storage.Warnings { return nil } -type compactionMerger struct { - a, b storage.DeprecatedChunkSeriesSet +// populateWithDelGenericSeriesIterator allows to iterate over given chunk metas. In each iteration it ensures +// that chunks are trimmed based on given tombstones interval if any. +// +// populateWithDelGenericSeriesIterator assumes that chunks that would be fully removed by intervals are filtered out in previous phase. +// +// On each iteration currChkMeta is available. If currDelIter is not nil, it means that chunk iterator in currChkMeta +// is invalid and chunk rewrite is needed, currDelIter should be used. +type populateWithDelGenericSeriesIterator struct { + chunks ChunkReader + // chks are expected to be sorted by minTime and should be related to the same, single series. + chks []chunks.Meta - aok, bok bool - l labels.Labels - c []chunks.Meta + i int + err error + bufIter *deletedIterator intervals tombstones.Intervals + + currDelIter chunkenc.Iterator + currChkMeta chunks.Meta } -// TODO(bwplotka): Move to storage mergers. -func newCompactionMerger(a, b storage.DeprecatedChunkSeriesSet) (*compactionMerger, error) { - c := &compactionMerger{ - a: a, - b: b, +func newPopulateWithDelGenericSeriesIterator( + chunks ChunkReader, + chks []chunks.Meta, + intervals tombstones.Intervals, +) *populateWithDelGenericSeriesIterator { + return &populateWithDelGenericSeriesIterator{ + chunks: chunks, + chks: chks, + i: -1, + bufIter: &deletedIterator{}, + intervals: intervals, } - // Initialize first elements of both sets as Next() needs - // one element look-ahead. - c.aok = c.a.Next() - c.bok = c.b.Next() - - return c, c.Err() } -func (c *compactionMerger) compare() int { - if !c.aok { - return 1 - } - if !c.bok { - return -1 - } - a, _, _ := c.a.At() - b, _, _ := c.b.At() - return labels.Compare(a, b) -} - -func (c *compactionMerger) Next() bool { - if !c.aok && !c.bok || c.Err() != nil { +func (p *populateWithDelGenericSeriesIterator) next() bool { + if p.err != nil || p.i >= len(p.chks)-1 { return false } - // While advancing child iterators the memory used for labels and chunks - // may be reused. When picking a series we have to store the result. - var lset labels.Labels - var chks []chunks.Meta - d := c.compare() - if d > 0 { - lset, chks, c.intervals = c.b.At() - c.l = append(c.l[:0], lset...) - c.c = append(c.c[:0], chks...) + p.i++ + p.currChkMeta = p.chks[p.i] - c.bok = c.b.Next() - } else if d < 0 { - lset, chks, c.intervals = c.a.At() - c.l = append(c.l[:0], lset...) - c.c = append(c.c[:0], chks...) - - c.aok = c.a.Next() - } else { - // Both sets contain the current series. Chain them into a single one. - l, ca, ra := c.a.At() - _, cb, rb := c.b.At() - - for _, r := range rb { - ra = ra.Add(r) - } - - c.l = append(c.l[:0], l...) - c.c = append(append(c.c[:0], ca...), cb...) - c.intervals = ra - - c.aok = c.a.Next() - c.bok = c.b.Next() + p.currChkMeta.Chunk, p.err = p.chunks.Chunk(p.currChkMeta.Ref) + if p.err != nil { + p.err = errors.Wrapf(p.err, "cannot populate chunk %d", p.currChkMeta.Ref) + return false } + p.bufIter.intervals = p.bufIter.intervals[:0] + for _, interval := range p.intervals { + if p.currChkMeta.OverlapsClosedInterval(interval.Mint, interval.Maxt) { + p.bufIter.intervals = p.bufIter.intervals.Add(interval) + } + } + + // Re-encode head chunks that are still open (being appended to) or + // outside the compacted MaxTime range. + // The chunk.Bytes() method is not safe for open chunks hence the re-encoding. + // This happens when snapshotting the head block or just fetching chunks from TSDB. + // + // TODO think how to avoid the typecasting to verify when it is head block. + _, isSafeChunk := p.currChkMeta.Chunk.(*safeChunk) + if len(p.bufIter.intervals) == 0 && !(isSafeChunk && p.currChkMeta.MaxTime == math.MaxInt64) { + // If there are no overlap with deletion intervals AND it's NOT an "open" head chunk, we can take chunk as it is. + p.currDelIter = nil + return true + } + + // We don't want full chunk or it's potentially still opened, take just part of it. + p.bufIter.it = p.currChkMeta.Chunk.Iterator(nil) + p.currDelIter = p.bufIter return true } -func (c *compactionMerger) Err() error { - if c.a.Err() != nil { - return c.a.Err() - } - return c.b.Err() +func (p *populateWithDelGenericSeriesIterator) Err() error { return p.err } + +func (p *populateWithDelGenericSeriesIterator) toSeriesIterator() chunkenc.Iterator { + return &populateWithDelSeriesIterator{populateWithDelGenericSeriesIterator: p} +} +func (p *populateWithDelGenericSeriesIterator) toChunkSeriesIterator() chunks.Iterator { + return &populateWithDelChunkSeriesIterator{populateWithDelGenericSeriesIterator: p} } -func (c *compactionMerger) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) { - return c.l, c.c, c.intervals +// populateWithDelSeriesIterator allows to iterate over samples for the single series. +type populateWithDelSeriesIterator struct { + *populateWithDelGenericSeriesIterator + + curr chunkenc.Iterator +} + +func (p *populateWithDelSeriesIterator) Next() bool { + if p.curr != nil && p.curr.Next() { + return true + } + + for p.next() { + if p.currDelIter != nil { + p.curr = p.currDelIter + } else { + p.curr = p.currChkMeta.Chunk.Iterator(nil) + } + if p.curr.Next() { + return true + } + } + return false +} + +func (p *populateWithDelSeriesIterator) Seek(t int64) bool { + if p.curr != nil && p.curr.Seek(t) { + return true + } + for p.Next() { + if p.curr.Seek(t) { + return true + } + } + return false +} + +func (p *populateWithDelSeriesIterator) At() (int64, float64) { return p.curr.At() } + +func (p *populateWithDelSeriesIterator) Err() error { + if err := p.populateWithDelGenericSeriesIterator.Err(); err != nil { + return err + } + if p.curr != nil { + return p.curr.Err() + } + return nil +} + +type populateWithDelChunkSeriesIterator struct { + *populateWithDelGenericSeriesIterator + + curr chunks.Meta +} + +func (p *populateWithDelChunkSeriesIterator) Next() bool { + if !p.next() { + return false + } + + p.curr = p.currChkMeta + if p.currDelIter == nil { + return true + } + + // Re-encode the chunk if iterator is provider. This means that it has some samples to be deleted or chunk is opened. + newChunk := chunkenc.NewXORChunk() + app, err := newChunk.Appender() + if err != nil { + p.err = err + return false + } + + if !p.currDelIter.Next() { + if err := p.currDelIter.Err(); err != nil { + p.err = errors.Wrap(err, "iterate chunk while re-encoding") + return false + } + + // Empty chunk, this should not happen, as we assume full deletions being filtered before this iterator. + p.err = errors.Wrap(err, "populateWithDelChunkSeriesIterator: unexpected empty chunk found while rewriting chunk") + return false + } + + t, v := p.currDelIter.At() + p.curr.MinTime = t + app.Append(t, v) + + for p.currDelIter.Next() { + t, v = p.currDelIter.At() + app.Append(t, v) + } + if err := p.currDelIter.Err(); err != nil { + p.err = errors.Wrap(err, "iterate chunk while re-encoding") + return false + } + + p.curr.Chunk = newChunk + p.curr.MaxTime = t + return true +} + +func (p *populateWithDelChunkSeriesIterator) At() chunks.Meta { return p.curr } + +// blockSeriesSet allows to iterate over sorted, populated series with applied tombstones. +// Series with all deleted chunks are still present as Series with no samples. +// Samples from chunks are also trimmed to requested min and max time. +type blockSeriesSet struct { + blockBaseSeriesSet +} + +func newBlockSeriesSet(i IndexReader, c ChunkReader, t tombstones.Reader, p index.Postings, mint, maxt int64) storage.SeriesSet { + return &blockSeriesSet{ + blockBaseSeriesSet{ + index: i, + chunks: c, + tombstones: t, + p: p, + mint: mint, + maxt: maxt, + }, + } +} + +func (b *blockSeriesSet) At() storage.Series { + // At can be looped over before iterating, so save the current value locally. + currIterFn := b.currIterFn + return &storage.SeriesEntry{ + Lset: b.currLabels, + SampleIteratorFn: func() chunkenc.Iterator { + return currIterFn().toSeriesIterator() + }, + } +} + +// blockChunkSeriesSet allows to iterate over sorted, populated series with applied tombstones. +// Series with all deleted chunks are still present as Labelled iterator with no chunks. +// Chunks are also trimmed to requested [min and max] (keeping samples with min and max timestamps). +type blockChunkSeriesSet struct { + blockBaseSeriesSet +} + +func newBlockChunkSeriesSet(i IndexReader, c ChunkReader, t tombstones.Reader, p index.Postings, mint, maxt int64) storage.ChunkSeriesSet { + return &blockChunkSeriesSet{ + blockBaseSeriesSet{ + index: i, + chunks: c, + tombstones: t, + p: p, + mint: mint, + maxt: maxt, + }, + } +} + +func (b *blockChunkSeriesSet) At() storage.ChunkSeries { + // At can be looped over before iterating, so save the current value locally. + currIterFn := b.currIterFn + return &storage.ChunkSeriesEntry{ + Lset: b.currLabels, + ChunkIteratorFn: func() chunks.Iterator { + return currIterFn().toChunkSeriesIterator() + }, + } } func newMergedStringIter(a index.StringIter, b index.StringIter) index.StringIter { diff --git a/tsdb/compact_test.go b/tsdb/compact_test.go index 68a8ce166..bec05d18e 100644 --- a/tsdb/compact_test.go +++ b/tsdb/compact_test.go @@ -28,8 +28,10 @@ import ( "github.com/pkg/errors" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/fileutil" + "github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/tombstones" "github.com/prometheus/prometheus/util/testutil" ) @@ -466,8 +468,23 @@ type nopChunkWriter struct{} func (nopChunkWriter) WriteChunks(chunks ...chunks.Meta) error { return nil } func (nopChunkWriter) Close() error { return nil } +func samplesForRange(minTime, maxTime int64, maxSamplesPerChunk int) (ret [][]sample) { + var curr []sample + for i := minTime; i <= maxTime; i++ { + curr = append(curr, sample{t: i}) + if len(curr) >= maxSamplesPerChunk { + ret = append(ret, curr) + curr = []sample{} + } + } + if len(curr) > 0 { + ret = append(ret, curr) + } + return ret +} + func TestCompaction_populateBlock(t *testing.T) { - var populateBlocksCases = []struct { + for _, tc := range []struct { title string inputSeriesSamples [][]seriesSamples compactMinTime int64 @@ -483,11 +500,7 @@ func TestCompaction_populateBlock(t *testing.T) { { // Populate from single block without chunks. We expect these kind of series being ignored. inputSeriesSamples: [][]seriesSamples{ - { - { - lset: map[string]string{"a": "b"}, - }, - }, + {{lset: map[string]string{"a": "b"}}}, }, }, { @@ -546,6 +559,46 @@ func TestCompaction_populateBlock(t *testing.T) { }, }, }, + { + title: "Populate from two blocks; chunks with negative time.", + inputSeriesSamples: [][]seriesSamples{ + { + { + lset: map[string]string{"a": "b"}, + chunks: [][]sample{{{t: 0}, {t: 10}}, {{t: 11}, {t: 20}}}, + }, + { + lset: map[string]string{"a": "c"}, + chunks: [][]sample{{{t: -11}, {t: -9}}, {{t: 10}, {t: 19}}}, + }, + { + // no-chunk series should be dropped. + lset: map[string]string{"a": "empty"}, + }, + }, + { + { + lset: map[string]string{"a": "b"}, + chunks: [][]sample{{{t: 21}, {t: 30}}}, + }, + { + lset: map[string]string{"a": "c"}, + chunks: [][]sample{{{t: 40}, {t: 45}}}, + }, + }, + }, + compactMinTime: -11, + expSeriesSamples: []seriesSamples{ + { + lset: map[string]string{"a": "b"}, + chunks: [][]sample{{{t: 0}, {t: 10}}, {{t: 11}, {t: 20}}, {{t: 21}, {t: 30}}}, + }, + { + lset: map[string]string{"a": "c"}, + chunks: [][]sample{{{t: -11}, {t: -9}}, {{t: 10}, {t: 19}}, {{t: 40}, {t: 45}}}, + }, + }, + }, { title: "Populate from two blocks showing that order is maintained.", inputSeriesSamples: [][]seriesSamples{ @@ -636,8 +689,45 @@ func TestCompaction_populateBlock(t *testing.T) { }, }, }, + { + title: "Populate from two blocks 1:1 duplicated chunks; with negative timestamps.", + inputSeriesSamples: [][]seriesSamples{ + { + { + lset: map[string]string{"a": "1"}, + chunks: [][]sample{{{t: 1}, {t: 2}}, {{t: 3}, {t: 4}}}, + }, + { + lset: map[string]string{"a": "2"}, + chunks: [][]sample{{{t: -3}, {t: -2}}, {{t: 1}, {t: 3}, {t: 4}}, {{t: 5}, {t: 6}}}, + }, + }, + { + { + lset: map[string]string{"a": "1"}, + chunks: [][]sample{{{t: 3}, {t: 4}}}, + }, + { + lset: map[string]string{"a": "2"}, + chunks: [][]sample{{{t: 1}, {t: 3}, {t: 4}}, {{t: 7}, {t: 8}}}, + }, + }, + }, + compactMinTime: -3, + expSeriesSamples: []seriesSamples{ + { + lset: map[string]string{"a": "1"}, + chunks: [][]sample{{{t: 1}, {t: 2}}, {{t: 3}, {t: 4}}}, + }, + { + lset: map[string]string{"a": "2"}, + chunks: [][]sample{{{t: -3}, {t: -2}}, {{t: 1}, {t: 3}, {t: 4}}, {{t: 5}, {t: 6}}, {{t: 7}, {t: 8}}}, + }, + }, + }, { // This should not happened because head block is making sure the chunks are not crossing block boundaries. + // We used to return error, but now chunk is trimmed. title: "Populate from single block containing chunk outside of compact meta time range.", inputSeriesSamples: [][]seriesSamples{ { @@ -649,10 +739,15 @@ func TestCompaction_populateBlock(t *testing.T) { }, compactMinTime: 0, compactMaxTime: 20, - expErr: errors.New("found chunk with minTime: 10 maxTime: 30 outside of compacted minTime: 0 maxTime: 20"), + expSeriesSamples: []seriesSamples{ + { + lset: map[string]string{"a": "b"}, + chunks: [][]sample{{{t: 1}, {t: 2}}, {{t: 10}}}, + }, + }, }, { - // Introduced by https://github.com/prometheus/tsdb/issues/347. + // Introduced by https://github.com/prometheus/tsdb/issues/347. We used to return error, but now chunk is trimmed. title: "Populate from single block containing extra chunk", inputSeriesSamples: [][]seriesSamples{ { @@ -664,7 +759,12 @@ func TestCompaction_populateBlock(t *testing.T) { }, compactMinTime: 0, compactMaxTime: 10, - expErr: errors.New("found chunk with minTime: 10 maxTime: 20 outside of compacted minTime: 0 maxTime: 10"), + expSeriesSamples: []seriesSamples{ + { + lset: map[string]string{"a": "issue347"}, + chunks: [][]sample{{{t: 1}, {t: 2}}}, + }, + }, }, { // Deduplication expected. @@ -693,54 +793,146 @@ func TestCompaction_populateBlock(t *testing.T) { }, { // Introduced by https://github.com/prometheus/tsdb/pull/539. - title: "Populate from three blocks that the last two are overlapping.", + title: "Populate from three overlapping blocks.", inputSeriesSamples: [][]seriesSamples{ { { - lset: map[string]string{"before": "fix"}, - chunks: [][]sample{{{t: 0}, {t: 10}, {t: 11}, {t: 20}}}, - }, - { - lset: map[string]string{"after": "fix"}, - chunks: [][]sample{{{t: 0}, {t: 10}, {t: 11}, {t: 20}}}, - }, - }, - { - { - lset: map[string]string{"before": "fix"}, + lset: map[string]string{"a": "overlap-all"}, chunks: [][]sample{{{t: 19}, {t: 30}}}, }, { - lset: map[string]string{"after": "fix"}, + lset: map[string]string{"a": "overlap-beginning"}, + chunks: [][]sample{{{t: 0}, {t: 5}}}, + }, + { + lset: map[string]string{"a": "overlap-ending"}, chunks: [][]sample{{{t: 21}, {t: 30}}}, }, }, { { - lset: map[string]string{"before": "fix"}, + lset: map[string]string{"a": "overlap-all"}, + chunks: [][]sample{{{t: 0}, {t: 10}, {t: 11}, {t: 20}}}, + }, + { + lset: map[string]string{"a": "overlap-beginning"}, + chunks: [][]sample{{{t: 0}, {t: 10}, {t: 12}, {t: 20}}}, + }, + { + lset: map[string]string{"a": "overlap-ending"}, + chunks: [][]sample{{{t: 0}, {t: 10}, {t: 13}, {t: 20}}}, + }, + }, + { + { + lset: map[string]string{"a": "overlap-all"}, chunks: [][]sample{{{t: 27}, {t: 35}}}, }, { - lset: map[string]string{"after": "fix"}, + lset: map[string]string{"a": "overlap-ending"}, chunks: [][]sample{{{t: 27}, {t: 35}}}, }, }, }, expSeriesSamples: []seriesSamples{ { - lset: map[string]string{"after": "fix"}, - chunks: [][]sample{{{t: 0}, {t: 10}, {t: 11}, {t: 20}}, {{t: 21}, {t: 27}, {t: 30}, {t: 35}}}, + lset: map[string]string{"a": "overlap-all"}, + chunks: [][]sample{{{t: 0}, {t: 10}, {t: 11}, {t: 19}, {t: 20}, {t: 27}, {t: 30}, {t: 35}}}, }, { - lset: map[string]string{"before": "fix"}, - chunks: [][]sample{{{t: 0}, {t: 10}, {t: 11}, {t: 19}, {t: 20}, {t: 27}, {t: 30}, {t: 35}}}, + lset: map[string]string{"a": "overlap-beginning"}, + chunks: [][]sample{{{t: 0}, {t: 5}, {t: 10}, {t: 12}, {t: 20}}}, + }, + { + lset: map[string]string{"a": "overlap-ending"}, + chunks: [][]sample{{{t: 0}, {t: 10}, {t: 13}, {t: 20}}, {{t: 21}, {t: 27}, {t: 30}, {t: 35}}}, }, }, }, - } - - for _, tc := range populateBlocksCases { - if ok := t.Run(tc.title, func(t *testing.T) { + { + title: "Populate from three partially overlapping blocks with few full chunks.", + inputSeriesSamples: [][]seriesSamples{ + { + { + lset: map[string]string{"a": "1", "b": "1"}, + chunks: samplesForRange(0, 659, 120), // 5 chunks and half. + }, + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: samplesForRange(0, 659, 120), + }, + }, + { + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: samplesForRange(480, 1199, 120), // two chunks overlapping with previous, two non overlapping and two overlapping with next block. + }, + { + lset: map[string]string{"a": "1", "b": "3"}, + chunks: samplesForRange(480, 1199, 120), + }, + }, + { + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: samplesForRange(960, 1499, 120), // 5 chunks and half. + }, + { + lset: map[string]string{"a": "1", "b": "4"}, + chunks: samplesForRange(960, 1499, 120), + }, + }, + }, + expSeriesSamples: []seriesSamples{ + { + lset: map[string]string{"a": "1", "b": "1"}, + chunks: samplesForRange(0, 659, 120), + }, + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: samplesForRange(0, 1499, 120), + }, + { + lset: map[string]string{"a": "1", "b": "3"}, + chunks: samplesForRange(480, 1199, 120), + }, + { + lset: map[string]string{"a": "1", "b": "4"}, + chunks: samplesForRange(960, 1499, 120), + }, + }, + }, + { + title: "Populate from three partially overlapping blocks with chunks that are expected to merge into single big chunks.", + inputSeriesSamples: [][]seriesSamples{ + { + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: [][]sample{{{t: 0}, {t: 6902464}}, {{t: 6961968}, {t: 7080976}}}, + }, + }, + { + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: [][]sample{{{t: 3600000}, {t: 13953696}}, {{t: 14042952}, {t: 14221464}}}, + }, + }, + { + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: [][]sample{{{t: 10800000}, {t: 14251232}}, {{t: 14280984}, {t: 14340488}}}, + }, + }, + }, + expSeriesSamples: []seriesSamples{ + { + lset: map[string]string{"a": "1", "b": "2"}, + chunks: [][]sample{{{t: 0}, {t: 3600000}, {t: 6902464}, {t: 6961968}, {t: 7080976}, {t: 10800000}, {t: 13953696}, {t: 14042952}, {t: 14221464}, {t: 14251232}}, {{t: 14280984}, {t: 14340488}}}, + }, + }, + }, + } { + t.Run(tc.title, func(t *testing.T) { blocks := make([]BlockReader, 0, len(tc.inputSeriesSamples)) for _, b := range tc.inputSeriesSamples { ir, cr, mint, maxt := createIdxChkReaders(t, b) @@ -767,12 +959,39 @@ func TestCompaction_populateBlock(t *testing.T) { } testutil.Ok(t, err) - testutil.Equals(t, tc.expSeriesSamples, iw.series) + // Check if response is expected and chunk is valid. + var raw []seriesSamples + for _, s := range iw.seriesChunks { + ss := seriesSamples{lset: s.l.Map()} + var iter chunkenc.Iterator + for _, chk := range s.chunks { + var ( + samples = make([]sample, 0, chk.Chunk.NumSamples()) + iter = chk.Chunk.Iterator(iter) + firstTs int64 = math.MaxInt64 + s sample + ) + for iter.Next() { + s.t, s.v = iter.At() + if firstTs == math.MaxInt64 { + firstTs = s.t + } + samples = append(samples, s) + } + + // Check if chunk has correct min, max times. + testutil.Equals(t, firstTs, chk.MinTime, "chunk Meta %v does not match the first encoded sample timestamp: %v", chk, firstTs) + testutil.Equals(t, s.t, chk.MaxTime, "chunk Meta %v does not match the last encoded sample timestamp %v", chk, s.t) + + testutil.Ok(t, iter.Err()) + ss.chunks = append(ss.chunks, samples) + } + raw = append(raw, ss) + } + testutil.Equals(t, tc.expSeriesSamples, raw) // Check if stats are calculated properly. - s := BlockStats{ - NumSeries: uint64(len(tc.expSeriesSamples)), - } + s := BlockStats{NumSeries: uint64(len(tc.expSeriesSamples))} for _, series := range tc.expSeriesSamples { s.NumChunks += uint64(len(series.chunks)) for _, chk := range series.chunks { @@ -780,9 +999,7 @@ func TestCompaction_populateBlock(t *testing.T) { } } testutil.Equals(t, s, meta.Stats) - }); !ok { - return - } + }) } } @@ -1090,3 +1307,98 @@ func TestDeleteCompactionBlockAfterFailedReload(t *testing.T) { }) } } + +func TestBlockBaseSeriesSet(t *testing.T) { + type refdSeries struct { + lset labels.Labels + chunks []chunks.Meta + + ref uint64 + } + + cases := []struct { + series []refdSeries + // Postings should be in the sorted order of the series + postings []uint64 + + expIdxs []int + }{ + { + series: []refdSeries{ + { + lset: labels.New([]labels.Label{{Name: "a", Value: "a"}}...), + chunks: []chunks.Meta{ + {Ref: 29}, {Ref: 45}, {Ref: 245}, {Ref: 123}, {Ref: 4232}, {Ref: 5344}, + {Ref: 121}, + }, + ref: 12, + }, + { + lset: labels.New([]labels.Label{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}...), + chunks: []chunks.Meta{ + {Ref: 82}, {Ref: 23}, {Ref: 234}, {Ref: 65}, {Ref: 26}, + }, + ref: 10, + }, + { + lset: labels.New([]labels.Label{{Name: "b", Value: "c"}}...), + chunks: []chunks.Meta{{Ref: 8282}}, + ref: 1, + }, + { + lset: labels.New([]labels.Label{{Name: "b", Value: "b"}}...), + chunks: []chunks.Meta{ + {Ref: 829}, {Ref: 239}, {Ref: 2349}, {Ref: 659}, {Ref: 269}, + }, + ref: 108, + }, + }, + postings: []uint64{12, 13, 10, 108}, // 13 doesn't exist and should just be skipped over. + expIdxs: []int{0, 1, 3}, + }, + { + series: []refdSeries{ + { + lset: labels.New([]labels.Label{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}...), + chunks: []chunks.Meta{ + {Ref: 82}, {Ref: 23}, {Ref: 234}, {Ref: 65}, {Ref: 26}, + }, + ref: 10, + }, + { + lset: labels.New([]labels.Label{{Name: "b", Value: "c"}}...), + chunks: []chunks.Meta{{Ref: 8282}}, + ref: 3, + }, + }, + postings: []uint64{}, + expIdxs: []int{}, + }, + } + + for _, tc := range cases { + mi := newMockIndex() + for _, s := range tc.series { + testutil.Ok(t, mi.AddSeries(s.ref, s.lset, s.chunks...)) + } + + bcs := &blockBaseSeriesSet{ + p: index.NewListPostings(tc.postings), + index: mi, + tombstones: tombstones.NewMemTombstones(), + } + + i := 0 + for bcs.Next() { + chks := bcs.currIterFn().chks + idx := tc.expIdxs[i] + + testutil.Equals(t, tc.series[idx].lset, bcs.currLabels) + testutil.Equals(t, tc.series[idx].chunks, chks) + + i++ + } + testutil.Equals(t, len(tc.expIdxs), i) + testutil.Ok(t, bcs.Err()) + } +} diff --git a/tsdb/db.go b/tsdb/db.go index be05c84d8..86704501d 100644 --- a/tsdb/db.go +++ b/tsdb/db.go @@ -321,7 +321,7 @@ func (db *DBReadOnly) FlushWAL(dir string) (returnErr error) { if err != nil { return err } - head, err := NewHead(nil, db.logger, w, 1, db.dir, nil, DefaultStripeSize, nil) + head, err := NewHead(nil, db.logger, w, DefaultBlockDuration, db.dir, nil, DefaultStripeSize, nil) if err != nil { return err } @@ -359,9 +359,7 @@ func (db *DBReadOnly) FlushWAL(dir string) (returnErr error) { return errors.Wrap(err, "writing WAL") } -// Querier loads the wal and returns a new querier over the data partition for the given time range. -// Current implementation doesn't support multiple Queriers. -func (db *DBReadOnly) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { +func (db *DBReadOnly) loadDataAsQueryable(maxt int64) (storage.SampleAndChunkQueryable, error) { select { case <-db.closed: return nil, ErrClosed @@ -380,7 +378,7 @@ func (db *DBReadOnly) Querier(ctx context.Context, mint, maxt int64) (storage.Qu blocks[i] = b } - head, err := NewHead(nil, db.logger, nil, 1, db.dir, nil, DefaultStripeSize, nil) + head, err := NewHead(nil, db.logger, nil, DefaultBlockDuration, db.dir, nil, DefaultStripeSize, nil) if err != nil { return nil, err } @@ -398,7 +396,7 @@ func (db *DBReadOnly) Querier(ctx context.Context, mint, maxt int64) (storage.Qu if err != nil { return nil, err } - head, err = NewHead(nil, db.logger, w, 1, db.dir, nil, DefaultStripeSize, nil) + head, err = NewHead(nil, db.logger, w, DefaultBlockDuration, db.dir, nil, DefaultStripeSize, nil) if err != nil { return nil, err } @@ -413,24 +411,32 @@ func (db *DBReadOnly) Querier(ctx context.Context, mint, maxt int64) (storage.Qu } db.closers = append(db.closers, head) - - // TODO: Refactor so that it is possible to obtain a Querier without initializing a writable DB instance. - // Option 1: refactor DB to have the Querier implementation using the DBReadOnly.Querier implementation not the opposite. - // Option 2: refactor Querier to use another independent func which - // can than be used by a read only and writable db instances without any code duplication. - dbWritable := &DB{ + return &DB{ dir: db.dir, logger: db.logger, blocks: blocks, head: head, - } - - return dbWritable.Querier(ctx, mint, maxt) + }, nil } -func (db *DBReadOnly) ChunkQuerier(context.Context, int64, int64) (storage.ChunkQuerier, error) { - // TODO(bwplotka): Implement in next PR. - return nil, errors.New("not implemented") +// Querier loads the blocks and wal and returns a new querier over the data partition for the given time range. +// Current implementation doesn't support multiple Queriers. +func (db *DBReadOnly) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { + q, err := db.loadDataAsQueryable(maxt) + if err != nil { + return nil, err + } + return q.Querier(ctx, mint, maxt) +} + +// ChunkQuerier loads blocks and the wal and returns a new chunk querier over the data partition for the given time range. +// Current implementation doesn't support multiple ChunkQueriers. +func (db *DBReadOnly) ChunkQuerier(ctx context.Context, mint, maxt int64) (storage.ChunkQuerier, error) { + q, err := db.loadDataAsQueryable(maxt) + if err != nil { + return nil, err + } + return q.ChunkQuerier(ctx, mint, maxt) } // Blocks returns a slice of block readers for persisted blocks. @@ -1330,10 +1336,8 @@ func (db *DB) Snapshot(dir string, withHead bool) error { } // Querier returns a new querier over the data partition for the given time range. -// A goroutine must not handle more than one open Querier. func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, error) { var blocks []BlockReader - var blockMetas []BlockMeta db.mtx.RLock() defer db.mtx.RUnlock() @@ -1341,7 +1345,6 @@ func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, err for _, b := range db.blocks { if b.OverlapsClosedInterval(mint, maxt) { blocks = append(blocks, b) - blockMetas = append(blockMetas, b.Meta()) } } if maxt >= db.head.MinTime() { @@ -1361,27 +1364,50 @@ func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, err } // If we fail, all previously opened queriers must be closed. for _, q := range blockQueriers { - q.Close() + // TODO(bwplotka): Handle error. + _ = q.Close() + } + return nil, errors.Wrapf(err, "open querier for block %s", b) + } + return storage.NewMergeQuerier(blockQueriers, nil, storage.ChainedSeriesMerge), nil +} + +// ChunkQuerier returns a new chunk querier over the data partition for the given time range. +func (db *DB) ChunkQuerier(_ context.Context, mint, maxt int64) (storage.ChunkQuerier, error) { + var blocks []BlockReader + + db.mtx.RLock() + defer db.mtx.RUnlock() + + for _, b := range db.blocks { + if b.OverlapsClosedInterval(mint, maxt) { + blocks = append(blocks, b) + } + } + if maxt >= db.head.MinTime() { + blocks = append(blocks, &RangeHead{ + head: db.head, + mint: mint, + maxt: maxt, + }) + } + + blockQueriers := make([]storage.ChunkQuerier, 0, len(blocks)) + for _, b := range blocks { + q, err := NewBlockChunkQuerier(b, mint, maxt) + if err == nil { + blockQueriers = append(blockQueriers, q) + continue + } + // If we fail, all previously opened queriers must be closed. + for _, q := range blockQueriers { + // TODO(bwplotka): Handle error. + _ = q.Close() } return nil, errors.Wrapf(err, "open querier for block %s", b) } - if len(OverlappingBlocks(blockMetas)) > 0 { - return &verticalQuerier{ - querier: querier{ - blocks: blockQueriers, - }, - }, nil - } - - return &querier{ - blocks: blockQueriers, - }, nil -} - -func (db *DB) ChunkQuerier(context.Context, int64, int64) (storage.ChunkQuerier, error) { - // TODO(bwplotka): Implement in next PR. - return nil, errors.New("not implemented") + return storage.NewMergeChunkQuerier(blockQueriers, nil, storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge)), nil } func rangeForTimestamp(t int64, width int64) (maxt int64) { diff --git a/tsdb/db_test.go b/tsdb/db_test.go index 763502484..8d175f256 100644 --- a/tsdb/db_test.go +++ b/tsdb/db_test.go @@ -104,6 +104,36 @@ func query(t testing.TB, q storage.Querier, matchers ...*labels.Matcher) map[str return result } +// queryChunks runs a matcher query against the querier and fully expands its data. +func queryChunks(t testing.TB, q storage.ChunkQuerier, matchers ...*labels.Matcher) map[string][]chunks.Meta { + ss := q.Select(false, nil, matchers...) + defer func() { + testutil.Ok(t, q.Close()) + }() + + result := map[string][]chunks.Meta{} + for ss.Next() { + series := ss.At() + + chks := []chunks.Meta{} + it := series.Iterator() + for it.Next() { + chks = append(chks, it.At()) + } + testutil.Ok(t, it.Err()) + + if len(chks) == 0 { + continue + } + + name := series.Labels().String() + result[name] = chks + } + testutil.Ok(t, ss.Err()) + testutil.Equals(t, 0, len(ss.Warnings())) + return result +} + // Ensure that blocks are held in memory in their time order // and not in ULID order as they are read from the directory. func TestDB_reloadOrder(t *testing.T) { @@ -387,7 +417,7 @@ Outer: } expss := newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{"a": "b"}, expSamples), + storage.NewListSeries(labels.FromStrings("a", "b"), expSamples), }) for { @@ -403,8 +433,8 @@ Outer: testutil.Equals(t, sexp.Labels(), sres.Labels()) - smplExp, errExp := expandSeriesIterator(sexp.Iterator()) - smplRes, errRes := expandSeriesIterator(sres.Iterator()) + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(), nil) testutil.Equals(t, errExp, errRes) testutil.Equals(t, smplExp, smplRes) @@ -683,7 +713,7 @@ Outer: } expss := newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{"a": "b"}, expSamples), + storage.NewListSeries(labels.FromStrings("a", "b"), expSamples), }) if len(expSamples) == 0 { @@ -704,8 +734,8 @@ Outer: testutil.Equals(t, sexp.Labels(), sres.Labels()) - smplExp, errExp := expandSeriesIterator(sexp.Iterator()) - smplRes, errRes := expandSeriesIterator(sres.Iterator()) + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(), nil) testutil.Equals(t, errExp, errRes) testutil.Equals(t, smplExp, smplRes) @@ -854,7 +884,7 @@ func TestDB_e2e(t *testing.T) { for ss.Next() { x := ss.At() - smpls, err := expandSeriesIterator(x.Iterator()) + smpls, err := storage.ExpandSamples(x.Iterator(), newSample) testutil.Ok(t, err) if len(smpls) > 0 { @@ -1033,7 +1063,7 @@ func TestTombstoneClean(t *testing.T) { } expss := newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{"a": "b"}, expSamples), + storage.NewListSeries(labels.FromStrings("a", "b"), expSamples), }) if len(expSamples) == 0 { @@ -1053,8 +1083,8 @@ func TestTombstoneClean(t *testing.T) { testutil.Equals(t, sexp.Labels(), sres.Labels()) - smplExp, errExp := expandSeriesIterator(sexp.Iterator()) - smplRes, errRes := expandSeriesIterator(sres.Iterator()) + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(), nil) testutil.Equals(t, errExp, errRes) testutil.Equals(t, smplExp, smplRes) @@ -1550,6 +1580,8 @@ func TestQuerierWithBoundaryChunks(t *testing.T) { for i := int64(0); i < 5; i++ { _, err := app.Add(label, i*blockRange, 0) testutil.Ok(t, err) + _, err = app.Add(labels.FromStrings("blockID", strconv.FormatInt(i, 10)), i*blockRange, 0) + testutil.Ok(t, err) } err := app.Commit() @@ -1564,9 +1596,11 @@ func TestQuerierWithBoundaryChunks(t *testing.T) { testutil.Ok(t, err) defer q.Close() - // The requested interval covers 2 blocks, so the querier should contain 2 blocks. - count := len(q.(*querier).blocks) - testutil.Assert(t, count == 2, "expected 2 blocks in querier, got %d", count) + // The requested interval covers 2 blocks, so the querier's label values for blockID should give us 2 values, one from each block. + b, ws, err := q.LabelValues("blockID") + testutil.Ok(t, err) + testutil.Equals(t, storage.Warnings(nil), ws) + testutil.Equals(t, []string{"1", "2"}, b) } // TestInitializeHeadTimestamp ensures that the h.minTime is set properly. @@ -1930,371 +1964,6 @@ func TestCorrectNumTombstones(t *testing.T) { testutil.Equals(t, uint64(3), db.blocks[0].meta.Stats.NumTombstones) } -func TestVerticalCompaction(t *testing.T) { - cases := []struct { - blockSeries [][]storage.Series - expSeries map[string][]tsdbutil.Sample - expBlockNum int - expOverlappingBlocks int - }{ - // Case 0 - // |--------------| - // |----------------| - { - blockSeries: [][]storage.Series{ - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{7, 0}, sample{8, 0}, sample{9, 0}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, sample{10, 99}, sample{11, 99}, - sample{12, 99}, sample{13, 99}, sample{14, 99}, - }), - }, - }, - expSeries: map[string][]tsdbutil.Sample{`{a="b"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{3, 99}, - sample{4, 0}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, sample{10, 99}, sample{11, 99}, - sample{12, 99}, sample{13, 99}, sample{14, 99}, - }}, - expBlockNum: 1, - expOverlappingBlocks: 1, - }, - // Case 1 - // |-------------------------------| - // |----------------| - { - blockSeries: [][]storage.Series{ - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{7, 0}, sample{8, 0}, sample{9, 0}, - sample{11, 0}, sample{13, 0}, sample{17, 0}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, sample{10, 99}, - }), - }, - }, - expSeries: map[string][]tsdbutil.Sample{`{a="b"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{3, 99}, - sample{4, 0}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, sample{10, 99}, sample{11, 0}, - sample{13, 0}, sample{17, 0}, - }}, - expBlockNum: 1, - expOverlappingBlocks: 1, - }, - // Case 2 - // |-------------------------------| - // |------------| - // |--------------------| - { - blockSeries: [][]storage.Series{ - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{7, 0}, sample{8, 0}, sample{9, 0}, - sample{11, 0}, sample{13, 0}, sample{17, 0}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{14, 59}, sample{15, 59}, sample{17, 59}, sample{20, 59}, - sample{21, 59}, sample{22, 59}, - }), - }, - }, - expSeries: map[string][]tsdbutil.Sample{`{a="b"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{3, 99}, - sample{4, 0}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, sample{11, 0}, sample{13, 0}, - sample{14, 59}, sample{15, 59}, sample{17, 59}, sample{20, 59}, - sample{21, 59}, sample{22, 59}, - }}, - expBlockNum: 1, - expOverlappingBlocks: 1, - }, - // Case 3 - // |-------------------| - // |--------------------| - // |----------------| - { - blockSeries: [][]storage.Series{ - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{8, 0}, sample{9, 0}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{14, 59}, sample{15, 59}, sample{17, 59}, sample{20, 59}, - sample{21, 59}, sample{22, 59}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{5, 99}, sample{6, 99}, sample{7, 99}, sample{8, 99}, - sample{9, 99}, sample{10, 99}, sample{13, 99}, sample{15, 99}, - sample{16, 99}, sample{17, 99}, - }), - }, - }, - expSeries: map[string][]tsdbutil.Sample{`{a="b"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 99}, sample{6, 99}, sample{7, 99}, sample{8, 99}, - sample{9, 99}, sample{10, 99}, sample{13, 99}, sample{14, 59}, - sample{15, 59}, sample{16, 99}, sample{17, 59}, sample{20, 59}, - sample{21, 59}, sample{22, 59}, - }}, - expBlockNum: 1, - expOverlappingBlocks: 1, - }, - // Case 4 - // |-------------------------------------| - // |------------| - // |-------------------------| - { - blockSeries: [][]storage.Series{ - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{8, 0}, sample{9, 0}, sample{10, 0}, - sample{13, 0}, sample{15, 0}, sample{16, 0}, sample{17, 0}, - sample{20, 0}, sample{22, 0}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{7, 59}, sample{8, 59}, sample{9, 59}, sample{10, 59}, - sample{11, 59}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{8, 99}, - sample{9, 99}, sample{10, 99}, sample{13, 99}, sample{15, 99}, - sample{16, 99}, sample{17, 99}, - }), - }, - }, - expSeries: map[string][]tsdbutil.Sample{`{a="b"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{3, 99}, - sample{4, 0}, sample{5, 99}, sample{6, 99}, sample{7, 59}, - sample{8, 59}, sample{9, 59}, sample{10, 59}, sample{11, 59}, - sample{13, 99}, sample{15, 99}, sample{16, 99}, sample{17, 99}, - sample{20, 0}, sample{22, 0}, - }}, - expBlockNum: 1, - expOverlappingBlocks: 1, - }, - // Case 5: series are merged properly when there are multiple series. - // |-------------------------------------| - // |------------| - // |-------------------------| - { - blockSeries: [][]storage.Series{ - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{8, 0}, sample{9, 0}, sample{10, 0}, - sample{13, 0}, sample{15, 0}, sample{16, 0}, sample{17, 0}, - sample{20, 0}, sample{22, 0}, - }), - newSeries(map[string]string{"b": "c"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{8, 0}, sample{9, 0}, sample{10, 0}, - sample{13, 0}, sample{15, 0}, sample{16, 0}, sample{17, 0}, - sample{20, 0}, sample{22, 0}, - }), - newSeries(map[string]string{"c": "d"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{8, 0}, sample{9, 0}, sample{10, 0}, - sample{13, 0}, sample{15, 0}, sample{16, 0}, sample{17, 0}, - sample{20, 0}, sample{22, 0}, - }), - }, - { - newSeries(map[string]string{"__name__": "a"}, []tsdbutil.Sample{ - sample{7, 59}, sample{8, 59}, sample{9, 59}, sample{10, 59}, - sample{11, 59}, - }), - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{7, 59}, sample{8, 59}, sample{9, 59}, sample{10, 59}, - sample{11, 59}, - }), - newSeries(map[string]string{"aa": "bb"}, []tsdbutil.Sample{ - sample{7, 59}, sample{8, 59}, sample{9, 59}, sample{10, 59}, - sample{11, 59}, - }), - newSeries(map[string]string{"c": "d"}, []tsdbutil.Sample{ - sample{7, 59}, sample{8, 59}, sample{9, 59}, sample{10, 59}, - sample{11, 59}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{8, 99}, - sample{9, 99}, sample{10, 99}, sample{13, 99}, sample{15, 99}, - sample{16, 99}, sample{17, 99}, - }), - newSeries(map[string]string{"aa": "bb"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{8, 99}, - sample{9, 99}, sample{10, 99}, sample{13, 99}, sample{15, 99}, - sample{16, 99}, sample{17, 99}, - }), - newSeries(map[string]string{"c": "d"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{8, 99}, - sample{9, 99}, sample{10, 99}, sample{13, 99}, sample{15, 99}, - sample{16, 99}, sample{17, 99}, - }), - }, - }, - expSeries: map[string][]tsdbutil.Sample{ - `{__name__="a"}`: { - sample{7, 59}, sample{8, 59}, sample{9, 59}, sample{10, 59}, - sample{11, 59}, - }, - `{a="b"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{3, 99}, - sample{4, 0}, sample{5, 99}, sample{6, 99}, sample{7, 59}, - sample{8, 59}, sample{9, 59}, sample{10, 59}, sample{11, 59}, - sample{13, 99}, sample{15, 99}, sample{16, 99}, sample{17, 99}, - sample{20, 0}, sample{22, 0}, - }, - `{aa="bb"}`: { - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{7, 59}, - sample{8, 59}, sample{9, 59}, sample{10, 59}, sample{11, 59}, - sample{13, 99}, sample{15, 99}, sample{16, 99}, sample{17, 99}, - }, - `{b="c"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{8, 0}, sample{9, 0}, sample{10, 0}, - sample{13, 0}, sample{15, 0}, sample{16, 0}, sample{17, 0}, - sample{20, 0}, sample{22, 0}, - }, - `{c="d"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{3, 99}, - sample{4, 0}, sample{5, 99}, sample{6, 99}, sample{7, 59}, - sample{8, 59}, sample{9, 59}, sample{10, 59}, sample{11, 59}, - sample{13, 99}, sample{15, 99}, sample{16, 99}, sample{17, 99}, - sample{20, 0}, sample{22, 0}, - }, - }, - expBlockNum: 1, - expOverlappingBlocks: 1, - }, - // Case 6 - // |--------------| - // |----------------| - // |--------------| - // |----------------| - { - blockSeries: [][]storage.Series{ - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, - sample{5, 0}, sample{7, 0}, sample{8, 0}, sample{9, 0}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{3, 99}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, sample{10, 99}, sample{11, 99}, - sample{12, 99}, sample{13, 99}, sample{14, 99}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{20, 0}, sample{21, 0}, sample{22, 0}, sample{24, 0}, - sample{25, 0}, sample{27, 0}, sample{28, 0}, sample{29, 0}, - }), - }, - { - newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ - sample{23, 99}, sample{25, 99}, sample{26, 99}, sample{27, 99}, - sample{28, 99}, sample{29, 99}, sample{30, 99}, sample{31, 99}, - }), - }, - }, - expSeries: map[string][]tsdbutil.Sample{`{a="b"}`: { - sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{3, 99}, - sample{4, 0}, sample{5, 99}, sample{6, 99}, sample{7, 99}, - sample{8, 99}, sample{9, 99}, sample{10, 99}, sample{11, 99}, - sample{12, 99}, sample{13, 99}, sample{14, 99}, - sample{20, 0}, sample{21, 0}, sample{22, 0}, sample{23, 99}, - sample{24, 0}, sample{25, 99}, sample{26, 99}, sample{27, 99}, - sample{28, 99}, sample{29, 99}, sample{30, 99}, sample{31, 99}, - }}, - expBlockNum: 2, - expOverlappingBlocks: 2, - }, - } - - defaultMatcher := labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*") - for _, c := range cases { - if ok := t.Run("", func(t *testing.T) { - - tmpdir, err := ioutil.TempDir("", "data") - testutil.Ok(t, err) - defer func() { - testutil.Ok(t, os.RemoveAll(tmpdir)) - }() - - for _, series := range c.blockSeries { - createBlock(t, tmpdir, series) - } - opts := DefaultOptions() - opts.AllowOverlappingBlocks = true - db, err := Open(tmpdir, nil, nil, opts) - testutil.Ok(t, err) - defer func() { - testutil.Ok(t, db.Close()) - }() - db.DisableCompactions() - testutil.Assert(t, len(db.blocks) == len(c.blockSeries), "Wrong number of blocks [before compact].") - - // Vertical Query Merging test. - querier, err := db.Querier(context.TODO(), 0, 100) - testutil.Ok(t, err) - actSeries := query(t, querier, defaultMatcher) - testutil.Equals(t, c.expSeries, actSeries) - - // Vertical compaction. - lc := db.compactor.(*LeveledCompactor) - testutil.Equals(t, 0, int(prom_testutil.ToFloat64(lc.metrics.overlappingBlocks)), "overlapping blocks count should be still 0 here") - err = db.Compact() - testutil.Ok(t, err) - testutil.Equals(t, c.expBlockNum, len(db.Blocks()), "Wrong number of blocks [after compact]") - - testutil.Equals(t, c.expOverlappingBlocks, int(prom_testutil.ToFloat64(lc.metrics.overlappingBlocks)), "overlapping blocks count mismatch") - - // Query test after merging the overlapping blocks. - querier, err = db.Querier(context.TODO(), 0, 100) - testutil.Ok(t, err) - actSeries = query(t, querier, defaultMatcher) - testutil.Equals(t, c.expSeries, actSeries) - }); !ok { - return - } - } -} - // TestBlockRanges checks the following use cases: // - No samples can be added with timestamps lower than the last block maxt. // - The compactor doesn't create overlapping blocks @@ -2396,14 +2065,14 @@ func TestBlockRanges(t *testing.T) { // It also checks that the API calls return equivalent results as a normal db.Open() mode. func TestDBReadOnly(t *testing.T) { var ( - dbDir string - logger = log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) - expBlocks []*Block - expSeries map[string][]tsdbutil.Sample - expSeriesCount int - expDBHash []byte - matchAll = labels.MustNewMatcher(labels.MatchEqual, "", "") - err error + dbDir string + logger = log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) + expBlocks []*Block + expSeries map[string][]tsdbutil.Sample + expChunks map[string][]chunks.Meta + expDBHash []byte + matchAll = labels.MustNewMatcher(labels.MatchEqual, "", "") + err error ) // Bootstrap the db. @@ -2416,15 +2085,21 @@ func TestDBReadOnly(t *testing.T) { }() dbBlocks := []*BlockMeta{ - {MinTime: 10, MaxTime: 11}, - {MinTime: 11, MaxTime: 12}, - {MinTime: 12, MaxTime: 13}, + // Create three 2-sample blocks. + {MinTime: 10, MaxTime: 12}, + {MinTime: 12, MaxTime: 14}, + {MinTime: 14, MaxTime: 16}, } for _, m := range dbBlocks { - createBlock(t, dbDir, genSeries(1, 1, m.MinTime, m.MaxTime)) + _ = createBlock(t, dbDir, genSeries(1, 1, m.MinTime, m.MaxTime)) } - expSeriesCount++ + + // Add head to test DBReadOnly WAL reading capabilities. + w, err := wal.New(logger, nil, filepath.Join(dbDir, "wal"), true) + testutil.Ok(t, err) + h := createHead(t, w, genSeries(1, 1, 16, 18), dbDir) + testutil.Ok(t, h.Close()) } // Open a normal db to use for a comparison. @@ -2439,7 +2114,6 @@ func TestDBReadOnly(t *testing.T) { _, err = app.Add(labels.FromStrings("foo", "bar"), dbWritable.Head().MaxTime()+1, 0) testutil.Ok(t, err) testutil.Ok(t, app.Commit()) - expSeriesCount++ expBlocks = dbWritable.Blocks() expDbSize, err := fileutil.DirSize(dbWritable.Dir()) @@ -2449,35 +2123,49 @@ func TestDBReadOnly(t *testing.T) { q, err := dbWritable.Querier(context.TODO(), math.MinInt64, math.MaxInt64) testutil.Ok(t, err) expSeries = query(t, q, matchAll) + cq, err := dbWritable.ChunkQuerier(context.TODO(), math.MinInt64, math.MaxInt64) + testutil.Ok(t, err) + expChunks = queryChunks(t, cq, matchAll) testutil.Ok(t, dbWritable.Close()) // Close here to allow getting the dir hash for windows. expDBHash = testutil.DirHash(t, dbWritable.Dir()) } // Open a read only db and ensure that the API returns the same result as the normal DB. - { - dbReadOnly, err := OpenDBReadOnly(dbDir, logger) - testutil.Ok(t, err) - defer func() { - testutil.Ok(t, dbReadOnly.Close()) - }() + dbReadOnly, err := OpenDBReadOnly(dbDir, logger) + testutil.Ok(t, err) + defer func() { testutil.Ok(t, dbReadOnly.Close()) }() + + t.Run("blocks", func(t *testing.T) { blocks, err := dbReadOnly.Blocks() testutil.Ok(t, err) testutil.Equals(t, len(expBlocks), len(blocks)) - for i, expBlock := range expBlocks { testutil.Equals(t, expBlock.Meta(), blocks[i].Meta(), "block meta mismatch") } + }) + t.Run("querier", func(t *testing.T) { + // Open a read only db and ensure that the API returns the same result as the normal DB. q, err := dbReadOnly.Querier(context.TODO(), math.MinInt64, math.MaxInt64) testutil.Ok(t, err) readOnlySeries := query(t, q, matchAll) readOnlyDBHash := testutil.DirHash(t, dbDir) - testutil.Equals(t, expSeriesCount, len(readOnlySeries), "total series mismatch") + testutil.Equals(t, len(expSeries), len(readOnlySeries), "total series mismatch") testutil.Equals(t, expSeries, readOnlySeries, "series mismatch") testutil.Equals(t, expDBHash, readOnlyDBHash, "after all read operations the db hash should remain the same") - } + }) + t.Run("chunk querier", func(t *testing.T) { + cq, err := dbReadOnly.ChunkQuerier(context.TODO(), math.MinInt64, math.MaxInt64) + testutil.Ok(t, err) + readOnlySeries := queryChunks(t, cq, matchAll) + readOnlyDBHash := testutil.DirHash(t, dbDir) + + testutil.Equals(t, len(expChunks), len(readOnlySeries), "total series mismatch") + testutil.Equals(t, expChunks, readOnlySeries, "series chunks mismatch") + testutil.Equals(t, expDBHash, readOnlyDBHash, "after all read operations the db hash should remain the same") + }) } // TestDBReadOnlyClosing ensures that after closing the db diff --git a/tsdb/head.go b/tsdb/head.go index a013f4b30..e0cfa698e 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -37,6 +37,7 @@ import ( "github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/record" "github.com/prometheus/prometheus/tsdb/tombstones" + "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/prometheus/prometheus/tsdb/wal" "go.uber.org/atomic" ) @@ -1928,13 +1929,9 @@ type sample struct { v float64 } -func (s sample) T() int64 { - return s.t -} - -func (s sample) V() float64 { - return s.v -} +func newSample(t int64, v float64) tsdbutil.Sample { return sample{t, v} } +func (s sample) T() int64 { return s.t } +func (s sample) V() float64 { return s.v } // 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. diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 622bd146e..1291966ab 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -577,7 +577,7 @@ func TestHeadDeleteSimple(t *testing.T) { actSeriesSet := q.Select(false, nil, labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value)) testutil.Ok(t, q.Close()) expSeriesSet := newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{lblDefault.Name: lblDefault.Value}, func() []tsdbutil.Sample { + storage.NewListSeries(labels.Labels{lblDefault}, func() []tsdbutil.Sample { ss := make([]tsdbutil.Sample, 0, len(c.smplsExp)) for _, s := range c.smplsExp { ss = append(ss, s) @@ -602,8 +602,8 @@ func TestHeadDeleteSimple(t *testing.T) { testutil.Equals(t, expSeries.Labels(), actSeries.Labels()) - smplExp, errExp := expandSeriesIterator(expSeries.Iterator()) - smplRes, errRes := expandSeriesIterator(actSeries.Iterator()) + smplExp, errExp := storage.ExpandSamples(expSeries.Iterator(), nil) + smplRes, errRes := storage.ExpandSamples(actSeries.Iterator(), nil) testutil.Equals(t, errExp, errRes) testutil.Equals(t, smplExp, smplRes) @@ -655,7 +655,7 @@ func TestDeleteUntilCurMax(t *testing.T) { testutil.Assert(t, res.Next(), "series don't exist") exps := res.At() it = exps.Iterator() - resSamples, err := expandSeriesIterator(it) + resSamples, err := storage.ExpandSamples(it, newSample) testutil.Ok(t, err) testutil.Equals(t, []tsdbutil.Sample{sample{11, 1}}, resSamples) for res.Next() { @@ -828,12 +828,9 @@ func TestDelete_e2e(t *testing.T) { smpls = deletedSamples(smpls, del.drange) // Only append those series for which samples exist as mockSeriesSet // doesn't skip series with no samples. - // TODO: But sometimes SeriesSet returns an empty SeriesIterator + // TODO: But sometimes SeriesSet returns an empty chunkenc.Iterator if len(smpls) > 0 { - matchedSeries = append(matchedSeries, newSeries( - m.Map(), - smpls, - )) + matchedSeries = append(matchedSeries, storage.NewListSeries(m, smpls)) } } expSs := newMockSeriesSet(matchedSeries) @@ -856,8 +853,8 @@ func TestDelete_e2e(t *testing.T) { sexp := expSs.At() sres := ss.At() testutil.Equals(t, sexp.Labels(), sres.Labels()) - smplExp, errExp := expandSeriesIterator(sexp.Iterator()) - smplRes, errRes := expandSeriesIterator(sres.Iterator()) + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(), nil) testutil.Equals(t, errExp, errRes) testutil.Equals(t, smplExp, smplRes) } @@ -1417,7 +1414,6 @@ func TestAddDuplicateLabelName(t *testing.T) { func TestMemSeriesIsolation(t *testing.T) { // Put a series, select it. GC it and then access it. - lastValue := func(h *Head, maxAppendID uint64) int { idx, err := h.Index() @@ -1428,12 +1424,16 @@ func TestMemSeriesIsolation(t *testing.T) { chunks, err := h.chunksRange(math.MinInt64, math.MaxInt64, iso) testutil.Ok(t, err) - querier := &blockQuerier{ - mint: 0, - maxt: 10000, - index: idx, - chunks: chunks, - tombstones: tombstones.NewMemTombstones(), + // Hm.. here direct block chunk querier might be required? + querier := blockQuerier{ + blockBaseQuerier: &blockBaseQuerier{ + index: idx, + chunks: chunks, + tombstones: tombstones.NewMemTombstones(), + + mint: 0, + maxt: 10000, + }, } testutil.Ok(t, err) diff --git a/tsdb/mocks_test.go b/tsdb/mocks_test.go index 913061e83..5294115d9 100644 --- a/tsdb/mocks_test.go +++ b/tsdb/mocks_test.go @@ -14,6 +14,7 @@ package tsdb import ( + "github.com/pkg/errors" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" @@ -21,48 +22,35 @@ import ( ) type mockIndexWriter struct { - series []seriesSamples + seriesChunks []series } -func (mockIndexWriter) AddSymbol(sym string) error { return nil } -func (m *mockIndexWriter) AddSeries(ref uint64, l labels.Labels, chunks ...chunks.Meta) error { - i := -1 - for j, s := range m.series { - if !labels.Equal(labels.FromMap(s.lset), l) { - continue +func copyChunk(c chunkenc.Chunk) (chunkenc.Chunk, error) { + b := c.Bytes() + nb := make([]byte, len(b)) + copy(nb, b) + return chunkenc.FromData(c.Encoding(), nb) +} + +func (mockIndexWriter) AddSymbol(string) error { return nil } +func (m *mockIndexWriter) AddSeries(_ uint64, l labels.Labels, chks ...chunks.Meta) error { + // Copy chunks as their bytes are pooled. + chksNew := make([]chunks.Meta, len(chks)) + for i, chk := range chks { + c, err := copyChunk(chk.Chunk) + if err != nil { + return errors.Wrap(err, "mockIndexWriter: copy chunk") } - i = j - break - } - if i == -1 { - m.series = append(m.series, seriesSamples{ - lset: l.Map(), - }) - i = len(m.series) - 1 + chksNew[i] = chunks.Meta{MaxTime: chk.MaxTime, MinTime: chk.MinTime, Chunk: c} } - var iter chunkenc.Iterator - for _, chk := range chunks { - samples := make([]sample, 0, chk.Chunk.NumSamples()) - - iter = chk.Chunk.Iterator(iter) - for iter.Next() { - s := sample{} - s.t, s.v = iter.At() - - samples = append(samples, s) - } - if err := iter.Err(); err != nil { - return err - } - - m.series[i].chunks = append(m.series[i].chunks, samples) - } + // We don't combine multiple same series together, by design as `AddSeries` requires full series to be saved. + m.seriesChunks = append(m.seriesChunks, series{l: l, chunks: chksNew}) return nil } -func (mockIndexWriter) WriteLabelIndex(names []string, values []string) error { return nil } -func (mockIndexWriter) Close() error { return nil } +func (mockIndexWriter) WriteLabelIndex([]string, []string) error { return nil } +func (mockIndexWriter) Close() error { return nil } type mockBReader struct { ir IndexReader diff --git a/tsdb/querier.go b/tsdb/querier.go index 70a65a570..529302132 100644 --- a/tsdb/querier.go +++ b/tsdb/querier.go @@ -22,214 +22,11 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/tsdb/chunks" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/tombstones" ) -// querier aggregates querying results from time blocks within -// a single partition. -type querier struct { - blocks []storage.Querier -} - -func (q *querier) LabelValues(n string) ([]string, storage.Warnings, error) { - return q.lvals(q.blocks, n) -} - -// LabelNames returns all the unique label names present querier blocks. -func (q *querier) LabelNames() ([]string, storage.Warnings, error) { - labelNamesMap := make(map[string]struct{}) - var ws storage.Warnings - for _, b := range q.blocks { - names, w, err := b.LabelNames() - ws = append(ws, w...) - if err != nil { - return nil, ws, errors.Wrap(err, "LabelNames() from Querier") - } - for _, name := range names { - labelNamesMap[name] = struct{}{} - } - } - - labelNames := make([]string, 0, len(labelNamesMap)) - for name := range labelNamesMap { - labelNames = append(labelNames, name) - } - sort.Strings(labelNames) - - return labelNames, ws, nil -} - -func (q *querier) lvals(qs []storage.Querier, n string) ([]string, storage.Warnings, error) { - if len(qs) == 0 { - return []string{}, nil, nil - } - if len(qs) == 1 { - return qs[0].LabelValues(n) - } - l := len(qs) / 2 - - var ws storage.Warnings - s1, w, err := q.lvals(qs[:l], n) - ws = append(ws, w...) - if err != nil { - return []string{}, ws, err - } - s2, ws, err := q.lvals(qs[l:], n) - ws = append(ws, w...) - if err != nil { - return []string{}, ws, err - } - return mergeStrings(s1, s2), ws, nil -} - -func (q *querier) Select(sortSeries bool, hints *storage.SelectHints, ms ...*labels.Matcher) storage.SeriesSet { - if len(q.blocks) == 0 { - return storage.EmptySeriesSet() - } - if len(q.blocks) == 1 { - // Sorting Head series is slow, and unneeded when only the - // Head is being queried. - return q.blocks[0].Select(sortSeries, hints, ms...) - } - - ss := make([]storage.SeriesSet, len(q.blocks)) - for i, b := range q.blocks { - // We have to sort if blocks > 1 as MergedSeriesSet requires it. - ss[i] = b.Select(true, hints, ms...) - } - - return NewMergedSeriesSet(ss) -} - -func (q *querier) Close() error { - var merr tsdb_errors.MultiError - - for _, bq := range q.blocks { - merr.Add(bq.Close()) - } - return merr.Err() -} - -// verticalQuerier aggregates querying results from time blocks within -// a single partition. The block time ranges can be overlapping. -type verticalQuerier struct { - querier -} - -func (q *verticalQuerier) Select(sortSeries bool, hints *storage.SelectHints, ms ...*labels.Matcher) storage.SeriesSet { - return q.sel(sortSeries, hints, q.blocks, ms) -} - -func (q *verticalQuerier) sel(sortSeries bool, hints *storage.SelectHints, qs []storage.Querier, ms []*labels.Matcher) storage.SeriesSet { - if len(qs) == 0 { - return storage.EmptySeriesSet() - } - if len(qs) == 1 { - return qs[0].Select(sortSeries, hints, ms...) - } - l := len(qs) / 2 - - return newMergedVerticalSeriesSet( - q.sel(sortSeries, hints, qs[:l], ms), - q.sel(sortSeries, hints, qs[l:], ms), - ) -} - -// NewBlockQuerier returns a querier against the reader. -func NewBlockQuerier(b BlockReader, mint, maxt int64) (storage.Querier, error) { - indexr, err := b.Index() - if err != nil { - return nil, errors.Wrapf(err, "open index reader") - } - chunkr, err := b.Chunks() - if err != nil { - indexr.Close() - return nil, errors.Wrapf(err, "open chunk reader") - } - tombsr, err := b.Tombstones() - if err != nil { - indexr.Close() - chunkr.Close() - return nil, errors.Wrapf(err, "open tombstone reader") - } - return &blockQuerier{ - mint: mint, - maxt: maxt, - index: indexr, - chunks: chunkr, - tombstones: tombsr, - }, nil -} - -// blockQuerier provides querying access to a single block database. -type blockQuerier struct { - index IndexReader - chunks ChunkReader - tombstones tombstones.Reader - - closed bool - - mint, maxt int64 -} - -func (q *blockQuerier) Select(sortSeries bool, hints *storage.SelectHints, ms ...*labels.Matcher) storage.SeriesSet { - var base storage.DeprecatedChunkSeriesSet - var err error - - if sortSeries { - base, err = LookupChunkSeriesSorted(q.index, q.tombstones, ms...) - } else { - base, err = LookupChunkSeries(q.index, q.tombstones, ms...) - } - if err != nil { - return storage.ErrSeriesSet(err) - } - - mint := q.mint - maxt := q.maxt - if hints != nil { - mint = hints.Start - maxt = hints.End - } - return &blockSeriesSet{ - set: &populatedChunkSeries{ - set: base, - chunks: q.chunks, - mint: mint, - maxt: maxt, - }, - - mint: mint, - maxt: maxt, - } -} - -func (q *blockQuerier) LabelValues(name string) ([]string, storage.Warnings, error) { - res, err := q.index.SortedLabelValues(name) - return res, nil, err -} - -func (q *blockQuerier) LabelNames() ([]string, storage.Warnings, error) { - res, err := q.index.LabelNames() - return res, nil, err -} - -func (q *blockQuerier) Close() error { - if q.closed { - return errors.New("block querier already closed") - } - - var merr tsdb_errors.MultiError - merr.Add(q.index.Close()) - merr.Add(q.chunks.Close()) - merr.Add(q.tombstones.Close()) - q.closed = true - return merr.Err() -} - // Bitmap used by func isRegexMetaCharacter to check whether a character needs to be escaped. var regexMetaCharacterBytes [16]byte @@ -244,6 +41,129 @@ func init() { } } +type blockBaseQuerier struct { + index IndexReader + chunks ChunkReader + tombstones tombstones.Reader + + closed bool + + mint, maxt int64 +} + +func newBlockBaseQuerier(b BlockReader, mint, maxt int64) (*blockBaseQuerier, error) { + indexr, err := b.Index() + if err != nil { + return nil, errors.Wrap(err, "open index reader") + } + chunkr, err := b.Chunks() + if err != nil { + indexr.Close() + return nil, errors.Wrap(err, "open chunk reader") + } + tombsr, err := b.Tombstones() + if err != nil { + indexr.Close() + chunkr.Close() + return nil, errors.Wrap(err, "open tombstone reader") + } + + if tombsr == nil { + tombsr = tombstones.NewMemTombstones() + } + return &blockBaseQuerier{ + mint: mint, + maxt: maxt, + index: indexr, + chunks: chunkr, + tombstones: tombsr, + }, nil +} + +func (q *blockBaseQuerier) LabelValues(name string) ([]string, storage.Warnings, error) { + res, err := q.index.SortedLabelValues(name) + return res, nil, err +} + +func (q *blockBaseQuerier) LabelNames() ([]string, storage.Warnings, error) { + res, err := q.index.LabelNames() + return res, nil, err +} + +func (q *blockBaseQuerier) Close() error { + if q.closed { + return errors.New("block querier already closed") + } + var merr tsdb_errors.MultiError + merr.Add(q.index.Close()) + merr.Add(q.chunks.Close()) + merr.Add(q.tombstones.Close()) + q.closed = true + return merr.Err() +} + +type blockQuerier struct { + *blockBaseQuerier +} + +// NewBlockQuerier returns a querier against the block reader and requested min and max time range. +func NewBlockQuerier(b BlockReader, mint, maxt int64) (storage.Querier, error) { + q, err := newBlockBaseQuerier(b, mint, maxt) + if err != nil { + return nil, err + } + return &blockQuerier{blockBaseQuerier: q}, nil +} + +func (q *blockQuerier) Select(sortSeries bool, hints *storage.SelectHints, ms ...*labels.Matcher) storage.SeriesSet { + mint := q.mint + maxt := q.maxt + if hints != nil { + mint = hints.Start + maxt = hints.End + } + + p, err := PostingsForMatchers(q.index, ms...) + if err != nil { + return storage.ErrSeriesSet(err) + } + if sortSeries { + p = q.index.SortedPostings(p) + } + return newBlockSeriesSet(q.index, q.chunks, q.tombstones, p, mint, maxt) +} + +// blockChunkQuerier provides chunk querying access to a single block database. +type blockChunkQuerier struct { + *blockBaseQuerier +} + +// NewBlockChunkQuerier returns a chunk querier against the block reader and requested min and max time range. +func NewBlockChunkQuerier(b BlockReader, mint, maxt int64) (storage.ChunkQuerier, error) { + q, err := newBlockBaseQuerier(b, mint, maxt) + if err != nil { + return nil, err + } + return &blockChunkQuerier{blockBaseQuerier: q}, nil +} + +func (q *blockChunkQuerier) Select(sortSeries bool, hints *storage.SelectHints, ms ...*labels.Matcher) storage.ChunkSeriesSet { + mint := q.mint + maxt := q.maxt + if hints != nil { + mint = hints.Start + maxt = hints.End + } + p, err := PostingsForMatchers(q.index, ms...) + if err != nil { + return storage.ErrChunkSeriesSet(err) + } + if sortSeries { + p = q.index.SortedPostings(p) + } + return newBlockChunkSeriesSet(q.index, q.chunks, q.tombstones, p, mint, maxt) +} + func findSetMatches(pattern string) []string { // Return empty matches if the wrapper from Prometheus is missing. if len(pattern) < 6 || pattern[:4] != "^(?:" || pattern[len(pattern)-2:] != ")$" { @@ -439,731 +359,6 @@ func inversePostingsForMatcher(ix IndexReader, m *labels.Matcher) (index.Posting return ix.Postings(m.Name, res...) } -func mergeStrings(a, b []string) []string { - maxl := len(a) - if len(b) > len(a) { - maxl = len(b) - } - res := make([]string, 0, maxl*10/9) - - for len(a) > 0 && len(b) > 0 { - d := strings.Compare(a[0], b[0]) - - if d == 0 { - res = append(res, a[0]) - a, b = a[1:], b[1:] - } else if d < 0 { - res = append(res, a[0]) - a = a[1:] - } else if d > 0 { - res = append(res, b[0]) - b = b[1:] - } - } - - // Append all remaining elements. - res = append(res, a...) - res = append(res, b...) - return res -} - -// mergedSeriesSet returns a series sets slice as a single series set. The input series sets -// must be sorted and sequential in time. -// TODO(bwplotka): Merge this with merge SeriesSet available in storage package. -type mergedSeriesSet struct { - all []storage.SeriesSet - buf []storage.SeriesSet // A buffer for keeping the order of SeriesSet slice during forwarding the SeriesSet. - ids []int // The indices of chosen SeriesSet for the current run. - done bool - err error - cur storage.Series -} - -// TODO(bwplotka): Merge this with merge SeriesSet available in storage package. -func NewMergedSeriesSet(all []storage.SeriesSet) storage.SeriesSet { - if len(all) == 1 { - return all[0] - } - s := &mergedSeriesSet{all: all} - // Initialize first elements of all sets as Next() needs - // one element look-ahead. - s.nextAll() - if len(s.all) == 0 { - s.done = true - } - - return s -} - -func (s *mergedSeriesSet) At() storage.Series { - return s.cur -} - -func (s *mergedSeriesSet) Err() error { - return s.err -} - -func (s *mergedSeriesSet) Warnings() storage.Warnings { - var ws storage.Warnings - for _, ss := range s.all { - ws = append(ws, ss.Warnings()...) - } - return ws -} - -// nextAll is to call Next() for all SeriesSet. -// Because the order of the SeriesSet slice will affect the results, -// we need to use an buffer slice to hold the order. -func (s *mergedSeriesSet) nextAll() { - s.buf = s.buf[:0] - for _, ss := range s.all { - if ss.Next() { - s.buf = append(s.buf, ss) - continue - } - - if ss.Err() != nil { - s.done = true - s.err = ss.Err() - break - } - } - s.all, s.buf = s.buf, s.all -} - -// nextWithID is to call Next() for the SeriesSet with the indices of s.ids. -// Because the order of the SeriesSet slice will affect the results, -// we need to use an buffer slice to hold the order. -func (s *mergedSeriesSet) nextWithID() { - if len(s.ids) == 0 { - return - } - - s.buf = s.buf[:0] - i1 := 0 - i2 := 0 - for i1 < len(s.all) { - if i2 < len(s.ids) && i1 == s.ids[i2] { - if !s.all[s.ids[i2]].Next() { - if s.all[s.ids[i2]].Err() != nil { - s.done = true - s.err = s.all[s.ids[i2]].Err() - break - } - i2++ - i1++ - continue - } - i2++ - } - s.buf = append(s.buf, s.all[i1]) - i1++ - } - s.all, s.buf = s.buf, s.all -} - -func (s *mergedSeriesSet) Next() bool { - if s.done { - return false - } - - s.nextWithID() - if s.done { - return false - } - s.ids = s.ids[:0] - if len(s.all) == 0 { - s.done = true - return false - } - - // Here we are looking for a set of series sets with the lowest labels, - // and we will cache their indexes in s.ids. - s.ids = append(s.ids, 0) - for i := 1; i < len(s.all); i++ { - cmp := labels.Compare(s.all[s.ids[0]].At().Labels(), s.all[i].At().Labels()) - if cmp > 0 { - s.ids = s.ids[:1] - s.ids[0] = i - } else if cmp == 0 { - s.ids = append(s.ids, i) - } - } - - if len(s.ids) > 1 { - series := make([]storage.Series, len(s.ids)) - for i, idx := range s.ids { - series[i] = s.all[idx].At() - } - s.cur = &chainedSeries{series: series} - } else { - s.cur = s.all[s.ids[0]].At() - } - return true -} - -type mergedVerticalSeriesSet struct { - a, b storage.SeriesSet - cur storage.Series - adone, bdone bool -} - -// NewMergedVerticalSeriesSet takes two series sets as a single series set. -// The input series sets must be sorted and -// the time ranges of the series can be overlapping. -func NewMergedVerticalSeriesSet(a, b storage.SeriesSet) storage.SeriesSet { - return newMergedVerticalSeriesSet(a, b) -} - -func newMergedVerticalSeriesSet(a, b storage.SeriesSet) *mergedVerticalSeriesSet { - s := &mergedVerticalSeriesSet{a: a, b: b} - // Initialize first elements of both sets as Next() needs - // one element look-ahead. - s.adone = !s.a.Next() - s.bdone = !s.b.Next() - - return s -} - -func (s *mergedVerticalSeriesSet) At() storage.Series { - return s.cur -} - -func (s *mergedVerticalSeriesSet) Err() error { - if s.a.Err() != nil { - return s.a.Err() - } - return s.b.Err() -} - -func (s *mergedVerticalSeriesSet) Warnings() storage.Warnings { - var ws storage.Warnings - ws = append(ws, s.a.Warnings()...) - ws = append(ws, s.b.Warnings()...) - return ws -} - -func (s *mergedVerticalSeriesSet) compare() int { - if s.adone { - return 1 - } - if s.bdone { - return -1 - } - return labels.Compare(s.a.At().Labels(), s.b.At().Labels()) -} - -func (s *mergedVerticalSeriesSet) Next() bool { - if s.adone && s.bdone || s.Err() != nil { - return false - } - - d := s.compare() - - // Both sets contain the current series. Chain them into a single one. - if d > 0 { - s.cur = s.b.At() - s.bdone = !s.b.Next() - } else if d < 0 { - s.cur = s.a.At() - s.adone = !s.a.Next() - } else { - s.cur = &verticalChainedSeries{series: []storage.Series{s.a.At(), s.b.At()}} - s.adone = !s.a.Next() - s.bdone = !s.b.Next() - } - return true -} - -// baseChunkSeries loads the label set and chunk references for a postings -// list from an index. It filters out series that have labels set that should be unset. -type baseChunkSeries struct { - p index.Postings - index IndexReader - tombstones tombstones.Reader - - lset labels.Labels - chks []chunks.Meta - intervals tombstones.Intervals - err error -} - -// LookupChunkSeries retrieves all series for the given matchers and returns a ChunkSeriesSet -// over them. It drops chunks based on tombstones in the given reader. -func LookupChunkSeries(ir IndexReader, tr tombstones.Reader, ms ...*labels.Matcher) (storage.DeprecatedChunkSeriesSet, error) { - return lookupChunkSeries(false, ir, tr, ms...) -} - -// LookupChunkSeries retrieves all series for the given matchers and returns a ChunkSeriesSet -// over them. It drops chunks based on tombstones in the given reader. Series will be in order. -func LookupChunkSeriesSorted(ir IndexReader, tr tombstones.Reader, ms ...*labels.Matcher) (storage.DeprecatedChunkSeriesSet, error) { - return lookupChunkSeries(true, ir, tr, ms...) -} - -func lookupChunkSeries(sorted bool, ir IndexReader, tr tombstones.Reader, ms ...*labels.Matcher) (storage.DeprecatedChunkSeriesSet, error) { - if tr == nil { - tr = tombstones.NewMemTombstones() - } - p, err := PostingsForMatchers(ir, ms...) - if err != nil { - return nil, err - } - if sorted { - p = ir.SortedPostings(p) - } - return &baseChunkSeries{ - p: p, - index: ir, - tombstones: tr, - }, nil -} - -func (s *baseChunkSeries) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) { - return s.lset, s.chks, s.intervals -} - -func (s *baseChunkSeries) Err() error { return s.err } - -func (s *baseChunkSeries) Next() bool { - var ( - lset = make(labels.Labels, len(s.lset)) - chkMetas = make([]chunks.Meta, len(s.chks)) - err error - ) - - for s.p.Next() { - ref := s.p.At() - if err := s.index.Series(ref, &lset, &chkMetas); err != nil { - // Postings may be stale. Skip if no underlying series exists. - if errors.Cause(err) == storage.ErrNotFound { - continue - } - s.err = err - return false - } - - s.lset = lset - s.chks = chkMetas - s.intervals, err = s.tombstones.Get(s.p.At()) - if err != nil { - s.err = errors.Wrap(err, "get tombstones") - return false - } - - if len(s.intervals) > 0 { - // Only those chunks that are not entirely deleted. - chks := make([]chunks.Meta, 0, len(s.chks)) - for _, chk := range s.chks { - if !(tombstones.Interval{Mint: chk.MinTime, Maxt: chk.MaxTime}.IsSubrange(s.intervals)) { - chks = append(chks, chk) - } - } - - s.chks = chks - } - - return true - } - if err := s.p.Err(); err != nil { - s.err = err - } - return false -} - -// populatedChunkSeries loads chunk data from a store for a set of series -// with known chunk references. It filters out chunks that do not fit the -// given time range. -type populatedChunkSeries struct { - set storage.DeprecatedChunkSeriesSet - chunks ChunkReader - mint, maxt int64 - - err error - chks []chunks.Meta - lset labels.Labels - intervals tombstones.Intervals -} - -func (s *populatedChunkSeries) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) { - return s.lset, s.chks, s.intervals -} - -func (s *populatedChunkSeries) Err() error { return s.err } - -func (s *populatedChunkSeries) Next() bool { - for s.set.Next() { - lset, chks, dranges := s.set.At() - - for len(chks) > 0 { - if chks[0].MaxTime >= s.mint { - break - } - chks = chks[1:] - } - - // This is to delete in place while iterating. - for i, rlen := 0, len(chks); i < rlen; i++ { - j := i - (rlen - len(chks)) - c := &chks[j] - - // Break out at the first chunk that has no overlap with mint, maxt. - if c.MinTime > s.maxt { - chks = chks[:j] - break - } - - c.Chunk, s.err = s.chunks.Chunk(c.Ref) - if s.err != nil { - // This means that the chunk has be garbage collected. Remove it from the list. - if s.err == storage.ErrNotFound { - s.err = nil - // Delete in-place. - s.chks = append(chks[:j], chks[j+1:]...) - } - return false - } - } - - if len(chks) == 0 { - continue - } - - s.lset = lset - s.chks = chks - s.intervals = dranges - - return true - } - if err := s.set.Err(); err != nil { - s.err = err - } - return false -} - -// blockSeriesSet is a set of series from an inverted index query. -type blockSeriesSet struct { - set storage.DeprecatedChunkSeriesSet - err error - cur storage.Series - - mint, maxt int64 -} - -func (s *blockSeriesSet) Next() bool { - for s.set.Next() { - lset, chunks, dranges := s.set.At() - s.cur = &chunkSeries{ - labels: lset, - chunks: chunks, - mint: s.mint, - maxt: s.maxt, - - intervals: dranges, - } - return true - } - if s.set.Err() != nil { - s.err = s.set.Err() - } - return false -} - -func (s *blockSeriesSet) At() storage.Series { return s.cur } -func (s *blockSeriesSet) Err() error { return s.err } -func (s *blockSeriesSet) Warnings() storage.Warnings { return nil } - -// chunkSeries is a series that is backed by a sequence of chunks holding -// time series data. -type chunkSeries struct { - labels labels.Labels - chunks []chunks.Meta // in-order chunk refs - - mint, maxt int64 - - intervals tombstones.Intervals -} - -func (s *chunkSeries) Labels() labels.Labels { - return s.labels -} - -func (s *chunkSeries) Iterator() chunkenc.Iterator { - return newChunkSeriesIterator(s.chunks, s.intervals, s.mint, s.maxt) -} - -// chainedSeries implements a series for a list of time-sorted series. -// They all must have the same labels. -type chainedSeries struct { - series []storage.Series -} - -func (s *chainedSeries) Labels() labels.Labels { - return s.series[0].Labels() -} - -func (s *chainedSeries) Iterator() chunkenc.Iterator { - return newChainedSeriesIterator(s.series...) -} - -// chainedSeriesIterator implements a series iterator over a list -// of time-sorted, non-overlapping iterators. -type chainedSeriesIterator struct { - series []storage.Series // series in time order - - i int - cur chunkenc.Iterator -} - -func newChainedSeriesIterator(s ...storage.Series) *chainedSeriesIterator { - return &chainedSeriesIterator{ - series: s, - i: 0, - cur: s[0].Iterator(), - } -} - -func (it *chainedSeriesIterator) Seek(t int64) bool { - // We just scan the chained series sequentially as they are already - // pre-selected by relevant time and should be accessed sequentially anyway. - for i, s := range it.series[it.i:] { - cur := s.Iterator() - if !cur.Seek(t) { - continue - } - it.cur = cur - it.i += i - return true - } - return false -} - -func (it *chainedSeriesIterator) Next() bool { - if it.cur.Next() { - return true - } - if err := it.cur.Err(); err != nil { - return false - } - if it.i == len(it.series)-1 { - return false - } - - it.i++ - it.cur = it.series[it.i].Iterator() - - return it.Next() -} - -func (it *chainedSeriesIterator) At() (t int64, v float64) { - return it.cur.At() -} - -func (it *chainedSeriesIterator) Err() error { - return it.cur.Err() -} - -// verticalChainedSeries implements a series for a list of time-sorted, time-overlapping series. -// They all must have the same labels. -type verticalChainedSeries struct { - series []storage.Series -} - -func (s *verticalChainedSeries) Labels() labels.Labels { - return s.series[0].Labels() -} - -func (s *verticalChainedSeries) Iterator() chunkenc.Iterator { - return newVerticalMergeSeriesIterator(s.series...) -} - -// verticalMergeSeriesIterator implements a series iterator over a list -// of time-sorted, time-overlapping iterators. -type verticalMergeSeriesIterator struct { - a, b chunkenc.Iterator - aok, bok, initialized bool - - curT int64 - curV float64 -} - -func newVerticalMergeSeriesIterator(s ...storage.Series) chunkenc.Iterator { - if len(s) == 1 { - return s[0].Iterator() - } else if len(s) == 2 { - return &verticalMergeSeriesIterator{ - a: s[0].Iterator(), - b: s[1].Iterator(), - } - } - return &verticalMergeSeriesIterator{ - a: s[0].Iterator(), - b: newVerticalMergeSeriesIterator(s[1:]...), - } -} - -func (it *verticalMergeSeriesIterator) Seek(t int64) bool { - it.aok, it.bok = it.a.Seek(t), it.b.Seek(t) - it.initialized = true - return it.Next() -} - -func (it *verticalMergeSeriesIterator) Next() bool { - if !it.initialized { - it.aok = it.a.Next() - it.bok = it.b.Next() - it.initialized = true - } - - if !it.aok && !it.bok { - return false - } - - if !it.aok { - it.curT, it.curV = it.b.At() - it.bok = it.b.Next() - return true - } - if !it.bok { - it.curT, it.curV = it.a.At() - it.aok = it.a.Next() - return true - } - - acurT, acurV := it.a.At() - bcurT, bcurV := it.b.At() - if acurT < bcurT { - it.curT, it.curV = acurT, acurV - it.aok = it.a.Next() - } else if acurT > bcurT { - it.curT, it.curV = bcurT, bcurV - it.bok = it.b.Next() - } else { - it.curT, it.curV = bcurT, bcurV - it.aok = it.a.Next() - it.bok = it.b.Next() - } - return true -} - -func (it *verticalMergeSeriesIterator) At() (t int64, v float64) { - return it.curT, it.curV -} - -func (it *verticalMergeSeriesIterator) Err() error { - if it.a.Err() != nil { - return it.a.Err() - } - return it.b.Err() -} - -// chunkSeriesIterator implements a series iterator on top -// of a list of time-sorted, non-overlapping chunks. -type chunkSeriesIterator struct { - chunks []chunks.Meta - - i int - cur chunkenc.Iterator - bufDelIter *deletedIterator - - maxt, mint int64 - - intervals tombstones.Intervals -} - -func newChunkSeriesIterator(cs []chunks.Meta, dranges tombstones.Intervals, mint, maxt int64) *chunkSeriesIterator { - csi := &chunkSeriesIterator{ - chunks: cs, - i: 0, - - mint: mint, - maxt: maxt, - - intervals: dranges, - } - csi.resetCurIterator() - - return csi -} - -func (it *chunkSeriesIterator) resetCurIterator() { - if len(it.intervals) == 0 { - it.cur = it.chunks[it.i].Chunk.Iterator(it.cur) - return - } - if it.bufDelIter == nil { - it.bufDelIter = &deletedIterator{ - intervals: it.intervals, - } - } - it.bufDelIter.it = it.chunks[it.i].Chunk.Iterator(it.bufDelIter.it) - it.cur = it.bufDelIter -} - -func (it *chunkSeriesIterator) Seek(t int64) (ok bool) { - if t > it.maxt { - return false - } - - // Seek to the first valid value after t. - if t < it.mint { - t = it.mint - } - - for ; it.chunks[it.i].MaxTime < t; it.i++ { - if it.i == len(it.chunks)-1 { - return false - } - } - - it.resetCurIterator() - - for it.cur.Next() { - t0, _ := it.cur.At() - if t0 >= t { - return true - } - } - return false -} - -func (it *chunkSeriesIterator) At() (t int64, v float64) { - return it.cur.At() -} - -func (it *chunkSeriesIterator) Next() bool { - if it.cur.Next() { - t, _ := it.cur.At() - - if t < it.mint { - if !it.Seek(it.mint) { - return false - } - t, _ = it.At() - - return t <= it.maxt - } - if t > it.maxt { - return false - } - return true - } - if err := it.cur.Err(); err != nil { - return false - } - if it.i == len(it.chunks)-1 { - return false - } - - it.i++ - it.resetCurIterator() - - return it.Next() -} - -func (it *chunkSeriesIterator) Err() error { - return it.cur.Err() -} - // deletedIterator wraps an Iterator and makes sure any deleted metrics are not // returned. type deletedIterator struct { diff --git a/tsdb/querier_test.go b/tsdb/querier_test.go index da11d399c..cf5c54fbf 100644 --- a/tsdb/querier_test.go +++ b/tsdb/querier_test.go @@ -24,6 +24,7 @@ import ( "sort" "strconv" "testing" + "time" "github.com/pkg/errors" "github.com/prometheus/prometheus/pkg/labels" @@ -36,6 +37,7 @@ import ( "github.com/prometheus/prometheus/util/testutil" ) +// TODO(bwplotka): Replace those mocks with remote.concreteSeriesSet. type mockSeriesSet struct { next func() bool series func() storage.Series @@ -63,134 +65,31 @@ func newMockSeriesSet(list []storage.Series) *mockSeriesSet { } } -func TestMergedSeriesSet(t *testing.T) { - cases := []struct { - // The input sets in order (samples in series in b are strictly - // after those in a). - a, b storage.SeriesSet - // The composition of a and b in the partition series set must yield - // results equivalent to the result series set. - exp storage.SeriesSet - }{ - { - a: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "a": "a", - }, []tsdbutil.Sample{ - sample{t: 1, v: 1}, - }), - }), - b: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "a": "a", - }, []tsdbutil.Sample{ - sample{t: 2, v: 2}, - }), - newSeries(map[string]string{ - "b": "b", - }, []tsdbutil.Sample{ - sample{t: 1, v: 1}, - }), - }), - exp: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "a": "a", - }, []tsdbutil.Sample{ - sample{t: 1, v: 1}, - sample{t: 2, v: 2}, - }), - newSeries(map[string]string{ - "b": "b", - }, []tsdbutil.Sample{ - sample{t: 1, v: 1}, - }), - }), - }, - { - a: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "handler": "prometheus", - "instance": "127.0.0.1:9090", - }, []tsdbutil.Sample{ - sample{t: 1, v: 1}, - }), - newSeries(map[string]string{ - "handler": "prometheus", - "instance": "localhost:9090", - }, []tsdbutil.Sample{ - sample{t: 1, v: 2}, - }), - }), - b: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "handler": "prometheus", - "instance": "127.0.0.1:9090", - }, []tsdbutil.Sample{ - sample{t: 2, v: 1}, - }), - newSeries(map[string]string{ - "handler": "query", - "instance": "localhost:9090", - }, []tsdbutil.Sample{ - sample{t: 2, v: 2}, - }), - }), - exp: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "handler": "prometheus", - "instance": "127.0.0.1:9090", - }, []tsdbutil.Sample{ - sample{t: 1, v: 1}, - sample{t: 2, v: 1}, - }), - newSeries(map[string]string{ - "handler": "prometheus", - "instance": "localhost:9090", - }, []tsdbutil.Sample{ - sample{t: 1, v: 2}, - }), - newSeries(map[string]string{ - "handler": "query", - "instance": "localhost:9090", - }, []tsdbutil.Sample{ - sample{t: 2, v: 2}, - }), - }), - }, - } - -Outer: - for _, c := range cases { - res := NewMergedSeriesSet([]storage.SeriesSet{c.a, c.b}) - - for { - eok, rok := c.exp.Next(), res.Next() - testutil.Equals(t, eok, rok) - - if !eok { - continue Outer - } - sexp := c.exp.At() - sres := res.At() - - testutil.Equals(t, sexp.Labels(), sres.Labels()) - - smplExp, errExp := expandSeriesIterator(sexp.Iterator()) - smplRes, errRes := expandSeriesIterator(sres.Iterator()) - - testutil.Equals(t, errExp, errRes) - testutil.Equals(t, smplExp, smplRes) - } - } +type mockChunkSeriesSet struct { + next func() bool + series func() storage.ChunkSeries + ws func() storage.Warnings + err func() error } -func expandSeriesIterator(it chunkenc.Iterator) (r []tsdbutil.Sample, err error) { - for it.Next() { - t, v := it.At() - r = append(r, sample{t: t, v: v}) - } +func (m *mockChunkSeriesSet) Next() bool { return m.next() } +func (m *mockChunkSeriesSet) At() storage.ChunkSeries { return m.series() } +func (m *mockChunkSeriesSet) Err() error { return m.err() } +func (m *mockChunkSeriesSet) Warnings() storage.Warnings { return m.ws() } - return r, it.Err() +func newMockChunkSeriesSet(list []storage.ChunkSeries) *mockChunkSeriesSet { + i := -1 + return &mockChunkSeriesSet{ + next: func() bool { + i++ + return i < len(list) + }, + series: func() storage.ChunkSeries { + return list[i] + }, + err: func() error { return nil }, + ws: func() storage.Warnings { return nil }, + } } type seriesSamples struct { @@ -238,7 +137,6 @@ func createIdxChkReaders(t *testing.T, tc []seriesSamples) (IndexReader, ChunkRe chkReader[chunkRef] = chunk chunkRef++ } - ls := labels.FromMap(s.lset) testutil.Ok(t, mi.AddSeries(uint64(i), ls, metas...)) @@ -257,125 +155,31 @@ func createIdxChkReaders(t *testing.T, tc []seriesSamples) (IndexReader, ChunkRe testutil.Ok(t, postings.Iter(func(l labels.Label, p index.Postings) error { return mi.WritePostings(l.Name, l.Value, p) })) - return mi, chkReader, blockMint, blockMaxt } -func TestBlockQuerier(t *testing.T) { - newSeries := func(l map[string]string, s []tsdbutil.Sample) storage.Series { - return &mockSeries{ - labels: func() labels.Labels { return labels.FromMap(l) }, - iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) }, - } - } +type blockQuerierTestCase struct { + mint, maxt int64 + ms []*labels.Matcher + exp storage.SeriesSet + expChks storage.ChunkSeriesSet +} - type query struct { - mint, maxt int64 - ms []*labels.Matcher - exp storage.SeriesSet - } +func testBlockQuerier(t *testing.T, c blockQuerierTestCase, ir IndexReader, cr ChunkReader, stones *tombstones.MemTombstones) { + t.Run("sample", func(t *testing.T) { + q := blockQuerier{ + blockBaseQuerier: &blockBaseQuerier{ + index: ir, + chunks: cr, + tombstones: stones, - cases := struct { - data []seriesSamples - - queries []query - }{ - data: []seriesSamples{ - { - lset: map[string]string{ - "a": "a", - }, - chunks: [][]sample{ - { - {1, 2}, {2, 3}, {3, 4}, - }, - { - {5, 2}, {6, 3}, {7, 4}, - }, - }, + mint: c.mint, + maxt: c.maxt, }, - { - lset: map[string]string{ - "a": "a", - "b": "b", - }, - chunks: [][]sample{ - { - {1, 1}, {2, 2}, {3, 3}, - }, - { - {5, 3}, {6, 6}, - }, - }, - }, - { - lset: map[string]string{ - "b": "b", - }, - chunks: [][]sample{ - { - {1, 3}, {2, 2}, {3, 6}, - }, - { - {5, 1}, {6, 7}, {7, 2}, - }, - }, - }, - }, - - queries: []query{ - { - mint: 0, - maxt: 0, - ms: []*labels.Matcher{}, - exp: newMockSeriesSet([]storage.Series{}), - }, - { - mint: 0, - maxt: 0, - ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]storage.Series{}), - }, - { - mint: 1, - maxt: 0, - ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]storage.Series{}), - }, - { - mint: 2, - maxt: 6, - ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "a": "a", - }, - []tsdbutil.Sample{sample{2, 3}, sample{3, 4}, sample{5, 2}, sample{6, 3}}, - ), - newSeries(map[string]string{ - "a": "a", - "b": "b", - }, - []tsdbutil.Sample{sample{2, 2}, sample{3, 3}, sample{5, 3}, sample{6, 6}}, - ), - }), - }, - }, - } - -Outer: - for _, c := range cases.queries { - ir, cr, _, _ := createIdxChkReaders(t, cases.data) - querier := &blockQuerier{ - index: ir, - chunks: cr, - tombstones: tombstones.NewMemTombstones(), - - mint: c.mint, - maxt: c.maxt, } - res := querier.Select(false, nil, c.ms...) + res := q.Select(false, nil, c.ms...) + defer func() { testutil.Ok(t, q.Close()) }() for { eok, rok := c.exp.Next(), res.Next() @@ -383,788 +187,728 @@ Outer: if !eok { testutil.Equals(t, 0, len(res.Warnings())) - continue Outer + break } sexp := c.exp.At() sres := res.At() - testutil.Equals(t, sexp.Labels(), sres.Labels()) - smplExp, errExp := expandSeriesIterator(sexp.Iterator()) - smplRes, errRes := expandSeriesIterator(sres.Iterator()) + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(), nil) testutil.Equals(t, errExp, errRes) testutil.Equals(t, smplExp, smplRes) } + testutil.Ok(t, res.Err()) + }) + + t.Run("chunk", func(t *testing.T) { + q := blockChunkQuerier{ + blockBaseQuerier: &blockBaseQuerier{ + index: ir, + chunks: cr, + tombstones: stones, + + mint: c.mint, + maxt: c.maxt, + }, + } + res := q.Select(false, nil, c.ms...) + defer func() { testutil.Ok(t, q.Close()) }() + + for { + eok, rok := c.expChks.Next(), res.Next() + testutil.Equals(t, eok, rok) + + if !eok { + testutil.Equals(t, 0, len(res.Warnings())) + break + } + sexpChks := c.expChks.At() + sres := res.At() + + testutil.Equals(t, sexpChks.Labels(), sres.Labels()) + + chksExp, errExp := storage.ExpandChunks(sexpChks.Iterator()) + rmChunkRefs(chksExp) + chksRes, errRes := storage.ExpandChunks(sres.Iterator()) + rmChunkRefs(chksRes) + testutil.Equals(t, errExp, errRes) + testutil.Equals(t, chksExp, chksRes) + } + testutil.Ok(t, res.Err()) + }) +} + +func TestBlockQuerier(t *testing.T) { + for _, c := range []blockQuerierTestCase{ + { + mint: 0, + maxt: 0, + ms: []*labels.Matcher{}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: 0, + maxt: 0, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: 1, + maxt: 0, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: math.MinInt64, + maxt: math.MaxInt64, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "x")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: math.MinInt64, + maxt: math.MaxInt64, + 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}}, + ), + 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}}, + ), + 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}}, + ), + }), + 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}}, + ), + 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}}, + ), + 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}}, + ), + }), + }, + { + mint: 2, + maxt: 6, + 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}}, + ), + 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}}, + ), + }), + 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}}, + ), + 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}}, + ), + }), + }, + } { + t.Run("", func(t *testing.T) { + ir, cr, _, _ := createIdxChkReaders(t, testData) + testBlockQuerier(t, c, ir, cr, tombstones.NewMemTombstones()) + }) } } +func TestBlockQuerier_AgainstHeadWithOpenChunks(t *testing.T) { + for _, c := range []blockQuerierTestCase{ + { + mint: 0, + maxt: 0, + ms: []*labels.Matcher{}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: 0, + maxt: 0, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: 1, + maxt: 0, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: math.MinInt64, + maxt: math.MaxInt64, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "x")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), + }, + { + mint: math.MinInt64, + maxt: math.MaxInt64, + 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}}, + ), + 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}}, + ), + 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}}, + ), + }), + 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}}, + ), + 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}}, + ), + 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}}, + ), + }), + }, + { + mint: 2, + maxt: 6, + 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}}, + ), + 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}}, + ), + }), + 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}}, + ), + 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}}, + ), + }), + }, + } { + t.Run("", func(t *testing.T) { + h, err := NewHead(nil, nil, nil, 2*time.Hour.Milliseconds(), "", nil, DefaultStripeSize, nil) + testutil.Ok(t, err) + defer h.Close() + + app := h.Appender() + for _, s := range testData { + for _, chk := range s.chunks { + for _, sample := range chk { + _, err = app.Add(labels.FromMap(s.lset), sample.t, sample.v) + testutil.Ok(t, err) + } + } + } + testutil.Ok(t, app.Commit()) + + hr := NewRangeHead(h, c.mint, c.maxt) + ir, err := hr.Index() + testutil.Ok(t, err) + defer ir.Close() + + cr, err := hr.Chunks() + testutil.Ok(t, err) + defer cr.Close() + + testBlockQuerier(t, c, ir, cr, tombstones.NewMemTombstones()) + }) + } +} + +var testData = []seriesSamples{ + { + lset: map[string]string{"a": "a"}, + chunks: [][]sample{ + {{1, 2}, {2, 3}, {3, 4}}, + {{5, 2}, {6, 3}, {7, 4}}, + }, + }, + { + lset: map[string]string{"a": "a", "b": "b"}, + chunks: [][]sample{ + {{1, 1}, {2, 2}, {3, 3}}, + {{5, 3}, {6, 6}}, + }, + }, + { + lset: map[string]string{"b": "b"}, + chunks: [][]sample{ + {{1, 3}, {2, 2}, {3, 6}}, + {{5, 1}, {6, 7}, {7, 2}}, + }, + }, +} + func TestBlockQuerierDelete(t *testing.T) { - newSeries := func(l map[string]string, s []tsdbutil.Sample) storage.Series { - return &mockSeries{ - labels: func() labels.Labels { return labels.FromMap(l) }, - iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) }, - } - } + stones := tombstones.NewTestMemTombstones([]tombstones.Intervals{ + {{Mint: 1, Maxt: 3}}, + {{Mint: 1, Maxt: 3}, {Mint: 6, Maxt: 10}}, + {{Mint: 6, Maxt: 10}}, + }) - type query struct { - mint, maxt int64 - ms []*labels.Matcher - exp storage.SeriesSet - } - - cases := struct { - data []seriesSamples - - tombstones tombstones.Reader - queries []query - }{ - data: []seriesSamples{ - { - lset: map[string]string{ - "a": "a", - }, - chunks: [][]sample{ - { - {1, 2}, {2, 3}, {3, 4}, - }, - { - {5, 2}, {6, 3}, {7, 4}, - }, - }, - }, - { - lset: map[string]string{ - "a": "a", - "b": "b", - }, - chunks: [][]sample{ - { - {1, 1}, {2, 2}, {3, 3}, - }, - { - {4, 15}, {5, 3}, {6, 6}, - }, - }, - }, - { - lset: map[string]string{ - "b": "b", - }, - chunks: [][]sample{ - { - {1, 3}, {2, 2}, {3, 6}, - }, - { - {5, 1}, {6, 7}, {7, 2}, - }, - }, - }, - }, - tombstones: tombstones.NewTestMemTombstones([]tombstones.Intervals{ - {{Mint: 1, Maxt: 3}}, - {{Mint: 1, Maxt: 3}, {Mint: 6, Maxt: 10}}, - {{Mint: 6, Maxt: 10}}, - }), - queries: []query{ - { - mint: 2, - maxt: 7, - ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "a": "a", - }, - []tsdbutil.Sample{sample{5, 2}, sample{6, 3}, sample{7, 4}}, - ), - newSeries(map[string]string{ - "a": "a", - "b": "b", - }, - []tsdbutil.Sample{sample{4, 15}, sample{5, 3}}, - ), - }), - }, - { - mint: 2, - maxt: 7, - ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "b", "b")}, - exp: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "a": "a", - "b": "b", - }, - []tsdbutil.Sample{sample{4, 15}, sample{5, 3}}, - ), - newSeries(map[string]string{ - "b": "b", - }, - []tsdbutil.Sample{sample{2, 2}, sample{3, 6}, sample{5, 1}}, - ), - }), - }, - { - mint: 1, - maxt: 4, - ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]storage.Series{ - newSeries(map[string]string{ - "a": "a", - "b": "b", - }, - []tsdbutil.Sample{sample{4, 15}}, - ), - }), - }, - { - mint: 1, - maxt: 3, - ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]storage.Series{}), - }, - }, - } - -Outer: - for _, c := range cases.queries { - ir, cr, _, _ := createIdxChkReaders(t, cases.data) - querier := &blockQuerier{ - index: ir, - chunks: cr, - tombstones: cases.tombstones, - - mint: c.mint, - maxt: c.maxt, - } - - res := querier.Select(false, nil, c.ms...) - - for { - eok, rok := c.exp.Next(), res.Next() - testutil.Equals(t, eok, rok) - - if !eok { - testutil.Equals(t, 0, len(res.Warnings())) - continue Outer - } - sexp := c.exp.At() - sres := res.At() - - testutil.Equals(t, sexp.Labels(), sres.Labels()) - - smplExp, errExp := expandSeriesIterator(sexp.Iterator()) - smplRes, errRes := expandSeriesIterator(sres.Iterator()) - - testutil.Equals(t, errExp, errRes) - testutil.Equals(t, smplExp, smplRes) - } - } -} - -func TestBaseChunkSeries(t *testing.T) { - type refdSeries struct { - lset labels.Labels - chunks []chunks.Meta - - ref uint64 - } - - cases := []struct { - series []refdSeries - // Postings should be in the sorted order of the series - postings []uint64 - - expIdxs []int - }{ + for _, c := range []blockQuerierTestCase{ { - series: []refdSeries{ - { - lset: labels.New([]labels.Label{{Name: "a", Value: "a"}}...), - chunks: []chunks.Meta{ - {Ref: 29}, {Ref: 45}, {Ref: 245}, {Ref: 123}, {Ref: 4232}, {Ref: 5344}, - {Ref: 121}, - }, - ref: 12, - }, - { - lset: labels.New([]labels.Label{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}...), - chunks: []chunks.Meta{ - {Ref: 82}, {Ref: 23}, {Ref: 234}, {Ref: 65}, {Ref: 26}, - }, - ref: 10, - }, - { - lset: labels.New([]labels.Label{{Name: "b", Value: "c"}}...), - chunks: []chunks.Meta{{Ref: 8282}}, - ref: 1, - }, - { - lset: labels.New([]labels.Label{{Name: "b", Value: "b"}}...), - chunks: []chunks.Meta{ - {Ref: 829}, {Ref: 239}, {Ref: 2349}, {Ref: 659}, {Ref: 269}, - }, - ref: 108, - }, - }, - postings: []uint64{12, 13, 10, 108}, // 13 doesn't exist and should just be skipped over. - expIdxs: []int{0, 1, 3}, + mint: 0, + maxt: 0, + ms: []*labels.Matcher{}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), }, { - series: []refdSeries{ - { - lset: labels.New([]labels.Label{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}...), - chunks: []chunks.Meta{ - {Ref: 82}, {Ref: 23}, {Ref: 234}, {Ref: 65}, {Ref: 26}, - }, - ref: 10, - }, - { - lset: labels.New([]labels.Label{{Name: "b", Value: "c"}}...), - chunks: []chunks.Meta{{Ref: 8282}}, - ref: 3, - }, - }, - postings: []uint64{}, - expIdxs: []int{}, - }, - } - - for _, tc := range cases { - mi := newMockIndex() - for _, s := range tc.series { - testutil.Ok(t, mi.AddSeries(s.ref, s.lset, s.chunks...)) - } - - bcs := &baseChunkSeries{ - p: index.NewListPostings(tc.postings), - index: mi, - tombstones: tombstones.NewMemTombstones(), - } - - i := 0 - for bcs.Next() { - lset, chks, _ := bcs.At() - - idx := tc.expIdxs[i] - - testutil.Equals(t, tc.series[idx].lset, lset) - testutil.Equals(t, tc.series[idx].chunks, chks) - - i++ - } - testutil.Equals(t, len(tc.expIdxs), i) - testutil.Ok(t, bcs.Err()) - } -} - -type itSeries struct { - si chunkenc.Iterator -} - -func (s itSeries) Iterator() chunkenc.Iterator { return s.si } -func (s itSeries) Labels() labels.Labels { return labels.Labels{} } - -func TestSeriesIterator(t *testing.T) { - itcases := []struct { - a, b, c []tsdbutil.Sample - exp []tsdbutil.Sample - - mint, maxt int64 - }{ - { - a: []tsdbutil.Sample{}, - b: []tsdbutil.Sample{}, - c: []tsdbutil.Sample{}, - - exp: []tsdbutil.Sample{}, - - mint: math.MinInt64, - maxt: math.MaxInt64, + mint: 0, + maxt: 0, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), }, { - a: []tsdbutil.Sample{ - sample{1, 2}, - sample{2, 3}, - sample{3, 5}, - sample{6, 1}, - }, - b: []tsdbutil.Sample{}, - c: []tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, - }, - - exp: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, - }, - mint: math.MinInt64, - maxt: math.MaxInt64, + mint: 1, + maxt: 0, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), }, { - a: []tsdbutil.Sample{}, - b: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, - }, - c: []tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, - }, - - exp: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, - }, - mint: 2, - maxt: 8, - }, - { - a: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, - }, - b: []tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, - }, - c: []tsdbutil.Sample{ - sample{10, 22}, sample{203, 3493}, - }, - - exp: []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}, - }, - mint: 6, - maxt: 10, - }, - } - - seekcases := []struct { - a, b, c []tsdbutil.Sample - - seek int64 - success bool - exp []tsdbutil.Sample - - mint, maxt int64 - }{ - { - a: []tsdbutil.Sample{}, - b: []tsdbutil.Sample{}, - c: []tsdbutil.Sample{}, - - seek: 0, - success: false, - exp: nil, - }, - { - a: []tsdbutil.Sample{ - sample{2, 3}, - }, - b: []tsdbutil.Sample{}, - c: []tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, - }, - - seek: 10, - success: false, - exp: nil, mint: math.MinInt64, maxt: math.MaxInt64, + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "x")}, + exp: newMockSeriesSet([]storage.Series{}), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{}), }, { - a: []tsdbutil.Sample{}, - b: []tsdbutil.Sample{ - sample{1, 2}, sample{3, 5}, sample{6, 1}, + mint: math.MinInt64, + maxt: math.MaxInt64, + 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}}, + ), + storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, + []tsdbutil.Sample{sample{5, 3}}, + ), + storage.NewListSeries(labels.Labels{{Name: "b", Value: "b"}}, + []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}, sample{5, 1}}, + ), + }), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ + storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, + []tsdbutil.Sample{sample{5, 2}, sample{6, 3}, sample{7, 4}}, + ), + storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, + []tsdbutil.Sample{sample{5, 3}}, + ), + storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "b", Value: "b"}}, + []tsdbutil.Sample{sample{1, 3}, sample{2, 2}, sample{3, 6}}, []tsdbutil.Sample{sample{5, 1}}, + ), + }), + }, + { + mint: 2, + maxt: 6, + 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}}, + ), + storage.NewListSeries(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, + []tsdbutil.Sample{sample{5, 3}}, + ), + }), + expChks: newMockChunkSeriesSet([]storage.ChunkSeries{ + storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}}, + []tsdbutil.Sample{sample{5, 2}, sample{6, 3}}, + ), + storage.NewListChunkSeriesFromSamples(labels.Labels{{Name: "a", Value: "a"}, {Name: "b", Value: "b"}}, + []tsdbutil.Sample{sample{5, 3}}, + ), + }), + }, + } { + t.Run("", func(t *testing.T) { + ir, cr, _, _ := createIdxChkReaders(t, testData) + testBlockQuerier(t, c, ir, cr, stones) + }) + } +} + +type fakeChunksReader struct { + ChunkReader + chks map[uint64]chunkenc.Chunk +} + +func createFakeReaderAndNotPopulatedChunks(s ...[]tsdbutil.Sample) (*fakeChunksReader, []chunks.Meta) { + f := &fakeChunksReader{ + chks: map[uint64]chunkenc.Chunk{}, + } + chks := make([]chunks.Meta, 0, len(s)) + + for ref, samples := range s { + chk := tsdbutil.ChunkFromSamples(samples) + f.chks[uint64(ref)] = chk.Chunk + + chks = append(chks, chunks.Meta{ + Ref: uint64(ref), + MinTime: chk.MinTime, + MaxTime: chk.MaxTime, + }) + } + return f, chks +} + +func (r *fakeChunksReader) Chunk(ref uint64) (chunkenc.Chunk, error) { + chk, ok := r.chks[ref] + if !ok { + return nil, errors.Errorf("chunk not found at ref %v", ref) + } + return chk, nil +} + +func TestPopulateWithTombSeriesIterators(t *testing.T) { + cases := []struct { + name string + chks [][]tsdbutil.Sample + + expected []tsdbutil.Sample + expectedChks []chunks.Meta + + intervals tombstones.Intervals + + // Seek being zero means do not test seek. + seek int64 + seekSuccess bool + }{ + { + name: "no chunk", + chks: [][]tsdbutil.Sample{}, + }, + { + name: "one empty chunk", // This should never happen. + chks: [][]tsdbutil.Sample{{}}, + + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{}), }, - c: []tsdbutil.Sample{ - sample{7, 89}, sample{9, 8}, + }, + { + name: "three empty chunks", // This should never happen. + chks: [][]tsdbutil.Sample{{}, {}, {}}, + + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{}), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{}), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{}), + }, + }, + { + name: "one chunk", + chks: [][]tsdbutil.Sample{ + {sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}}, }, - seek: 2, - success: true, - exp: []tsdbutil.Sample{ + expected: []tsdbutil.Sample{ + sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + }, + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + }), + }, + }, + { + 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}}, + }, + + expected: []tsdbutil.Sample{ + sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, + }, + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + }), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{7, 89}, sample{9, 8}, + }), + }, + }, + { + 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}}, + }, + + 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}, + }, + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, + }), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{7, 89}, sample{9, 8}, + }), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{10, 22}, sample{203, 3493}, + }), + }, + }, + // Seek cases. + { + name: "three empty chunks and seek", // This should never happen. + chks: [][]tsdbutil.Sample{{}, {}, {}}, + seek: 1, + + seekSuccess: false, + }, + { + 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}}, + }, + seek: 10, + + seekSuccess: false, + }, + { + 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}}, + }, + seek: 2, + + seekSuccess: true, + expected: []tsdbutil.Sample{ sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, }, - mint: 5, - maxt: 8, }, { - a: []tsdbutil.Sample{ - sample{6, 1}, - }, - b: []tsdbutil.Sample{ - sample{9, 8}, - }, - c: []tsdbutil.Sample{ - sample{10, 22}, sample{203, 3493}, + 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}}, }, + seek: -32, - seek: 10, - success: true, - exp: []tsdbutil.Sample{ - sample{10, 22}, sample{203, 3493}, + seekSuccess: true, + expected: []tsdbutil.Sample{ + sample{1, 2}, sample{3, 5}, sample{6, 1}, sample{7, 89}, sample{9, 8}, }, - mint: 10, - maxt: 203, + }, + // Deletion / Trim cases. + { + name: "no chunk with deletion interval", + chks: [][]tsdbutil.Sample{}, + intervals: tombstones.Intervals{{Mint: 20, Maxt: 21}}, }, { - a: []tsdbutil.Sample{ - sample{6, 1}, - }, - b: []tsdbutil.Sample{ - sample{9, 8}, - }, - c: []tsdbutil.Sample{ - sample{10, 22}, sample{203, 3493}, + 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}}, }, + intervals: tombstones.Intervals{{Mint: math.MinInt64, Maxt: 2}}.Add(tombstones.Interval{Mint: 9, Maxt: math.MaxInt64}), - seek: 203, - success: true, - exp: []tsdbutil.Sample{ - sample{203, 3493}, + expected: []tsdbutil.Sample{ + sample{3, 5}, sample{6, 1}, sample{7, 89}, + }, + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{3, 5}, sample{6, 1}, + }), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{7, 89}, + }), + }, + }, + { + 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}}, + }, + intervals: tombstones.Intervals{{Mint: 2, Maxt: 3}}, + + expected: []tsdbutil.Sample{ + sample{1, 2}, sample{6, 1}, sample{7, 89}, sample{9, 8}, + }, + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{1, 2}, sample{6, 1}, + }), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{7, 89}, sample{9, 8}, + }), + }, + }, + { + 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}}, + }, + intervals: tombstones.Intervals{{Mint: 6, Maxt: 7}}, + + expected: []tsdbutil.Sample{ + sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{9, 8}, + }, + expectedChks: []chunks.Meta{ + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{1, 2}, sample{2, 3}, sample{3, 5}, + }), + tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + sample{9, 8}, + }), + }, + }, + // Deletion with seek. + { + 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}}, + }, + 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}, }, - mint: 7, - maxt: 203, }, } + for _, tc := range cases { + t.Run(tc.name, func(t *testing.T) { + t.Run("sample", func(t *testing.T) { + f, chkMetas := createFakeReaderAndNotPopulatedChunks(tc.chks...) + it := newPopulateWithDelGenericSeriesIterator(f, chkMetas, tc.intervals).toSeriesIterator() - t.Run("Chunk", func(t *testing.T) { - for _, tc := range itcases { - chkMetas := []chunks.Meta{ - tsdbutil.ChunkFromSamples(tc.a), - tsdbutil.ChunkFromSamples(tc.b), - tsdbutil.ChunkFromSamples(tc.c), - } - res := newChunkSeriesIterator(chkMetas, nil, tc.mint, tc.maxt) + var r []tsdbutil.Sample + if tc.seek != 0 { + testutil.Equals(t, tc.seekSuccess, it.Seek(tc.seek)) + testutil.Equals(t, tc.seekSuccess, it.Seek(tc.seek)) // Next one should be noop. - smplValid := make([]tsdbutil.Sample, 0) - for _, s := range tc.exp { - if s.T() >= tc.mint && s.T() <= tc.maxt { - smplValid = append(smplValid, tsdbutil.Sample(s)) - } - } - exp := newListSeriesIterator(smplValid) - - smplExp, errExp := expandSeriesIterator(exp) - smplRes, errRes := expandSeriesIterator(res) - - testutil.Equals(t, errExp, errRes) - testutil.Equals(t, smplExp, smplRes) - } - - t.Run("Seek", func(t *testing.T) { - extra := []struct { - a, b, c []tsdbutil.Sample - - seek int64 - success bool - exp []tsdbutil.Sample - - mint, maxt int64 - }{ - { - a: []tsdbutil.Sample{ - sample{6, 1}, - }, - b: []tsdbutil.Sample{ - sample{9, 8}, - }, - c: []tsdbutil.Sample{ - sample{10, 22}, sample{203, 3493}, - }, - - seek: 203, - success: false, - exp: nil, - mint: 2, - maxt: 202, - }, - { - a: []tsdbutil.Sample{ - sample{6, 1}, - }, - b: []tsdbutil.Sample{ - sample{9, 8}, - }, - c: []tsdbutil.Sample{ - sample{10, 22}, sample{203, 3493}, - }, - - seek: 5, - success: true, - exp: []tsdbutil.Sample{sample{10, 22}}, - mint: 10, - maxt: 202, - }, - } - - seekcases2 := append(seekcases, extra...) - - for _, tc := range seekcases2 { - chkMetas := []chunks.Meta{ - tsdbutil.ChunkFromSamples(tc.a), - tsdbutil.ChunkFromSamples(tc.b), - tsdbutil.ChunkFromSamples(tc.c), - } - res := newChunkSeriesIterator(chkMetas, nil, tc.mint, tc.maxt) - - smplValid := make([]tsdbutil.Sample, 0) - for _, s := range tc.exp { - if s.T() >= tc.mint && s.T() <= tc.maxt { - smplValid = append(smplValid, tsdbutil.Sample(s)) + if tc.seekSuccess { + // After successful seek iterator is ready. Grab the value. + t, v := it.At() + r = append(r, sample{t: t, v: v}) } } - exp := newListSeriesIterator(smplValid) + expandedResult, err := storage.ExpandSamples(it, newSample) + testutil.Ok(t, err) + r = append(r, expandedResult...) + testutil.Equals(t, tc.expected, r) + }) + t.Run("chunk", func(t *testing.T) { + f, chkMetas := createFakeReaderAndNotPopulatedChunks(tc.chks...) + it := newPopulateWithDelGenericSeriesIterator(f, chkMetas, tc.intervals).toChunkSeriesIterator() - testutil.Equals(t, tc.success, res.Seek(tc.seek)) - - if tc.success { - // Init the list and then proceed to check. - remaining := exp.Next() - testutil.Assert(t, remaining == true, "") - - for remaining { - sExp, eExp := exp.At() - sRes, eRes := res.At() - testutil.Equals(t, eExp, eRes) - testutil.Equals(t, sExp, sRes) - - remaining = exp.Next() - testutil.Equals(t, remaining, res.Next()) - } + if tc.seek != 0 { + // Chunk iterator does not have Seek method. + return } - } + expandedResult, err := storage.ExpandChunks(it) + testutil.Ok(t, err) + + // We don't care about ref IDs for comparision, only chunk's samples matters. + rmChunkRefs(expandedResult) + rmChunkRefs(tc.expectedChks) + testutil.Equals(t, tc.expectedChks, expandedResult) + }) }) - }) + } +} - t.Run("Chain", func(t *testing.T) { - // Extra cases for overlapping series. - itcasesExtra := []struct { - a, b, c []tsdbutil.Sample - exp []tsdbutil.Sample - mint, maxt int64 - }{ - { - a: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{3, 5}, sample{6, 1}, - }, - b: []tsdbutil.Sample{ - sample{5, 49}, sample{7, 89}, sample{9, 8}, - }, - c: []tsdbutil.Sample{ - sample{2, 33}, sample{4, 44}, sample{10, 3}, - }, - - exp: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 33}, sample{3, 5}, sample{4, 44}, sample{5, 49}, sample{6, 1}, sample{7, 89}, sample{9, 8}, sample{10, 3}, - }, - mint: math.MinInt64, - maxt: math.MaxInt64, - }, - { - a: []tsdbutil.Sample{ - sample{1, 2}, sample{2, 3}, sample{9, 5}, sample{13, 1}, - }, - b: []tsdbutil.Sample{}, - c: []tsdbutil.Sample{ - sample{1, 23}, sample{2, 342}, sample{3, 25}, sample{6, 11}, - }, - - exp: []tsdbutil.Sample{ - sample{1, 23}, sample{2, 342}, sample{3, 25}, sample{6, 11}, sample{9, 5}, sample{13, 1}, - }, - mint: math.MinInt64, - maxt: math.MaxInt64, - }, - } - - for _, tc := range itcases { - a, b, c := itSeries{newListSeriesIterator(tc.a)}, - itSeries{newListSeriesIterator(tc.b)}, - itSeries{newListSeriesIterator(tc.c)} - - res := newChainedSeriesIterator(a, b, c) - exp := newListSeriesIterator([]tsdbutil.Sample(tc.exp)) - - smplExp, errExp := expandSeriesIterator(exp) - smplRes, errRes := expandSeriesIterator(res) - - testutil.Equals(t, errExp, errRes) - testutil.Equals(t, smplExp, smplRes) - } - - for _, tc := range append(itcases, itcasesExtra...) { - a, b, c := itSeries{newListSeriesIterator(tc.a)}, - itSeries{newListSeriesIterator(tc.b)}, - itSeries{newListSeriesIterator(tc.c)} - - res := newVerticalMergeSeriesIterator(a, b, c) - exp := newListSeriesIterator([]tsdbutil.Sample(tc.exp)) - - smplExp, errExp := expandSeriesIterator(exp) - smplRes, errRes := expandSeriesIterator(res) - - testutil.Equals(t, errExp, errRes) - testutil.Equals(t, smplExp, smplRes) - } - - t.Run("Seek", func(t *testing.T) { - for _, tc := range seekcases { - ress := []chunkenc.Iterator{ - newChainedSeriesIterator( - itSeries{newListSeriesIterator(tc.a)}, - itSeries{newListSeriesIterator(tc.b)}, - itSeries{newListSeriesIterator(tc.c)}, - ), - newVerticalMergeSeriesIterator( - itSeries{newListSeriesIterator(tc.a)}, - itSeries{newListSeriesIterator(tc.b)}, - itSeries{newListSeriesIterator(tc.c)}, - ), - } - - for _, res := range ress { - exp := newListSeriesIterator(tc.exp) - - testutil.Equals(t, tc.success, res.Seek(tc.seek)) - - if tc.success { - // Init the list and then proceed to check. - remaining := exp.Next() - testutil.Assert(t, remaining == true, "") - - for remaining { - sExp, eExp := exp.At() - sRes, eRes := res.At() - testutil.Equals(t, eExp, eRes) - testutil.Equals(t, sExp, sRes) - - remaining = exp.Next() - testutil.Equals(t, remaining, res.Next()) - } - } - } - } - }) - }) +func rmChunkRefs(chks []chunks.Meta) { + for i := range chks { + chks[i].Ref = 0 + } } // Regression for: https://github.com/prometheus/tsdb/pull/97 -func TestChunkSeriesIterator_DoubleSeek(t *testing.T) { - chkMetas := []chunks.Meta{ - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 1}, sample{2, 2}, sample{3, 3}}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{4, 4}, sample{5, 5}}), - } +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}}, + ) - res := newChunkSeriesIterator(chkMetas, nil, 2, 8) - testutil.Assert(t, res.Seek(1) == true, "") - testutil.Assert(t, res.Seek(2) == true, "") - ts, v := res.At() + it := newPopulateWithDelGenericSeriesIterator(f, chkMetas, nil).toSeriesIterator() + testutil.Assert(t, it.Seek(1), "") + testutil.Assert(t, it.Seek(2), "") + testutil.Assert(t, it.Seek(2), "") + ts, v := it.At() testutil.Equals(t, int64(2), ts) testutil.Equals(t, float64(2), v) } // Regression when seeked chunks were still found via binary search and we always // skipped to the end when seeking a value in the current chunk. -func TestChunkSeriesIterator_SeekInCurrentChunk(t *testing.T) { - metas := []chunks.Meta{ - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 2}, sample{3, 4}, sample{5, 6}, sample{7, 8}}), - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{}), - } +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{}, + ) - it := newChunkSeriesIterator(metas, nil, 1, 7) - - testutil.Assert(t, it.Next() == true, "") + it := newPopulateWithDelGenericSeriesIterator(f, chkMetas, nil).toSeriesIterator() + testutil.Assert(t, it.Next(), "") ts, v := it.At() testutil.Equals(t, int64(1), ts) testutil.Equals(t, float64(2), v) - testutil.Assert(t, it.Seek(4) == true, "") + testutil.Assert(t, it.Seek(4), "") ts, v = it.At() testutil.Equals(t, int64(5), ts) testutil.Equals(t, float64(6), v) } -// Regression when calling Next() with a time bounded to fit within two samples. -// Seek gets called and advances beyond the max time, which was just accepted as a valid sample. -func TestChunkSeriesIterator_NextWithMinTime(t *testing.T) { - metas := []chunks.Meta{ - tsdbutil.ChunkFromSamples([]tsdbutil.Sample{sample{1, 6}, sample{5, 6}, sample{7, 8}}), - } - - it := newChunkSeriesIterator(metas, nil, 2, 4) - testutil.Assert(t, it.Next() == false, "") -} - -func TestPopulatedCSReturnsValidChunkSlice(t *testing.T) { - lbls := []labels.Labels{labels.New(labels.Label{Name: "a", Value: "b"})} - chunkMetas := [][]chunks.Meta{ - { - {MinTime: 1, MaxTime: 2, Ref: 1}, - {MinTime: 3, MaxTime: 4, Ref: 2}, - {MinTime: 10, MaxTime: 12, Ref: 3}, - }, - } - - cr := mockChunkReader( - map[uint64]chunkenc.Chunk{ - 1: chunkenc.NewXORChunk(), - 2: chunkenc.NewXORChunk(), - 3: chunkenc.NewXORChunk(), - }, +func TestPopulateWithDelSeriesIterator_SeekWithMinTime(t *testing.T) { + f, chkMetas := createFakeReaderAndNotPopulatedChunks( + []tsdbutil.Sample{sample{1, 6}, sample{5, 6}, sample{6, 8}}, ) - m := &mockChunkSeriesSet{l: lbls, cm: chunkMetas, i: -1} - p := &populatedChunkSeries{ - set: m, - chunks: cr, - - mint: 0, - maxt: 0, - } - - testutil.Assert(t, p.Next() == false, "") - - p.mint = 6 - p.maxt = 9 - testutil.Assert(t, p.Next() == false, "") - - // Test the case where 1 chunk could cause an unpopulated chunk to be returned. - chunkMetas = [][]chunks.Meta{ - { - {MinTime: 1, MaxTime: 2, Ref: 1}, - }, - } - - m = &mockChunkSeriesSet{l: lbls, cm: chunkMetas, i: -1} - p = &populatedChunkSeries{ - set: m, - chunks: cr, - - mint: 10, - maxt: 15, - } - testutil.Assert(t, p.Next() == false, "") + it := newPopulateWithDelGenericSeriesIterator(f, chkMetas, nil).toSeriesIterator() + testutil.Equals(t, false, it.Seek(7)) + testutil.Equals(t, true, it.Seek(3)) } -type mockChunkSeriesSet struct { - l []labels.Labels - cm [][]chunks.Meta +// Regression when calling Next() with a time bounded to fit within two samples. +// 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}}, + ) - i int -} - -func (m *mockChunkSeriesSet) Next() bool { - if len(m.l) != len(m.cm) { - return false - } - m.i++ - return m.i < len(m.l) -} - -func (m *mockChunkSeriesSet) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) { - return m.l[m.i], m.cm[m.i], nil -} - -func (m *mockChunkSeriesSet) Err() error { - return nil + it := newPopulateWithDelGenericSeriesIterator( + f, chkMetas, tombstones.Intervals{{Mint: math.MinInt64, Maxt: 2}}.Add(tombstones.Interval{Mint: 4, Maxt: math.MaxInt64}), + ).toSeriesIterator() + testutil.Equals(t, false, it.Next()) } // Test the cost of merging series sets for different number of merged sets and their size. @@ -1172,7 +916,7 @@ func (m *mockChunkSeriesSet) Err() error { // TODO(bwplotka): Merge with storage merged series set benchmark. func BenchmarkMergedSeriesSet(b *testing.B) { var sel = func(sets []storage.SeriesSet) storage.SeriesSet { - return NewMergedSeriesSet(sets) + return storage.NewMergeSeriesSet(sets, storage.ChainedSeriesMerge) } for _, k := range []int{ @@ -1193,7 +937,7 @@ func BenchmarkMergedSeriesSet(b *testing.B) { for _, l := range lbls { l2 := l for j := range in { - in[j] = append(in[j], &mockSeries{labels: func() labels.Labels { return l2 }}) + in[j] = append(in[j], storage.NewListSeries(l2, nil)) } } @@ -1461,56 +1205,6 @@ func (m mockIndex) LabelNames() ([]string, error) { return l, nil } -type mockSeries struct { - labels func() labels.Labels - iterator func() chunkenc.Iterator -} - -func newSeries(l map[string]string, s []tsdbutil.Sample) storage.Series { - return &mockSeries{ - labels: func() labels.Labels { return labels.FromMap(l) }, - iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) }, - } -} -func (m *mockSeries) Labels() labels.Labels { return m.labels() } -func (m *mockSeries) Iterator() chunkenc.Iterator { return m.iterator() } - -type listSeriesIterator struct { - list []tsdbutil.Sample - idx int -} - -func newListSeriesIterator(list []tsdbutil.Sample) *listSeriesIterator { - return &listSeriesIterator{list: list, idx: -1} -} - -func (it *listSeriesIterator) At() (int64, float64) { - s := it.list[it.idx] - return s.T(), s.V() -} - -func (it *listSeriesIterator) Next() bool { - it.idx++ - return it.idx < len(it.list) -} - -func (it *listSeriesIterator) Seek(t int64) bool { - if it.idx == -1 { - it.idx = 0 - } - // Do binary search between current position and end. - it.idx = sort.Search(len(it.list)-it.idx, func(i int) bool { - s := it.list[i+it.idx] - return s.T() >= t - }) - - return it.idx < len(it.list) -} - -func (it *listSeriesIterator) Err() error { - return nil -} - func BenchmarkQueryIterator(b *testing.B) { cases := []struct { numBlocks int @@ -1562,21 +1256,14 @@ func BenchmarkQueryIterator(b *testing.B) { defer block.Close() } - que := &querier{ - blocks: make([]storage.Querier, 0, len(blocks)), - } + qblocks := make([]storage.Querier, 0, len(blocks)) for _, blk := range blocks { q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) testutil.Ok(b, err) - que.blocks = append(que.blocks, q) + qblocks = append(qblocks, q) } - var sq storage.Querier = que - if overlapPercentage > 0 { - sq = &verticalQuerier{ - querier: *que, - } - } + sq := storage.NewMergeQuerier(qblocks, nil, storage.ChainedSeriesMerge) defer sq.Close() benchQuery(b, c.numSeries, sq, labels.Selector{labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")}) @@ -1636,21 +1323,14 @@ func BenchmarkQuerySeek(b *testing.B) { defer block.Close() } - que := &querier{ - blocks: make([]storage.Querier, 0, len(blocks)), - } + qblocks := make([]storage.Querier, 0, len(blocks)) for _, blk := range blocks { q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) testutil.Ok(b, err) - que.blocks = append(que.blocks, q) + qblocks = append(qblocks, q) } - var sq storage.Querier = que - if overlapPercentage > 0 { - sq = &verticalQuerier{ - querier: *que, - } - } + sq := storage.NewMergeQuerier(qblocks, nil, storage.ChainedSeriesMerge) defer sq.Close() mint := blocks[0].meta.MinTime @@ -1782,22 +1462,22 @@ func BenchmarkSetMatcher(b *testing.B) { defer block.Close() } - que := &querier{ - blocks: make([]storage.Querier, 0, len(blocks)), - } + qblocks := make([]storage.Querier, 0, len(blocks)) for _, blk := range blocks { q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) testutil.Ok(b, err) - que.blocks = append(que.blocks, q) + qblocks = append(qblocks, q) } - defer que.Close() + + sq := storage.NewMergeQuerier(qblocks, nil, storage.ChainedSeriesMerge) + defer sq.Close() benchMsg := fmt.Sprintf("nSeries=%d,nBlocks=%d,cardinality=%d,pattern=\"%s\"", c.numSeries, c.numBlocks, c.cardinality, c.pattern) b.Run(benchMsg, func(b *testing.B) { b.ResetTimer() b.ReportAllocs() for n := 0; n < b.N; n++ { - ss := que.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, "test", c.pattern)) + ss := sq.Select(false, nil, labels.MustNewMatcher(labels.MatchRegexp, "test", c.pattern)) for ss.Next() { } testutil.Ok(b, ss.Err()) @@ -2196,7 +1876,7 @@ func BenchmarkQueries(b *testing.B) { testutil.Ok(b, os.RemoveAll(dir)) }() - series := genSeries(nSeries, 5, 1, int64(nSamples)) + series := genSeries(nSeries, 5, 1, nSamples) // Add some common labels to make the matchers select these series. { @@ -2210,13 +1890,12 @@ func BenchmarkQueries(b *testing.B) { } } for i := range commonLbls { - s := series[i].(*mockSeries) + s := series[i].(*storage.SeriesEntry) allLabels := append(commonLbls, s.Labels()...) - s = &mockSeries{ - labels: func() labels.Labels { return allLabels }, - iterator: s.iterator, - } - series[i] = s + newS := storage.NewListSeries(allLabels, nil) + newS.SampleIteratorFn = s.SampleIteratorFn + + series[i] = newS } } @@ -2228,17 +1907,18 @@ func BenchmarkQueries(b *testing.B) { testutil.Ok(b, err) qs = append(qs, q) } - queryTypes["_1-Block"] = &querier{blocks: qs[:1]} - queryTypes["_3-Blocks"] = &querier{blocks: qs[0:3]} - queryTypes["_10-Blocks"] = &querier{blocks: qs} + + queryTypes["_1-Block"] = storage.NewMergeQuerier(qs[:1], nil, storage.ChainedSeriesMerge) + queryTypes["_3-Blocks"] = storage.NewMergeQuerier(qs[0:3], nil, storage.ChainedSeriesMerge) + queryTypes["_10-Blocks"] = storage.NewMergeQuerier(qs, nil, storage.ChainedSeriesMerge) chunkDir, err := ioutil.TempDir("", "chunk_dir") testutil.Ok(b, err) defer func() { testutil.Ok(b, os.RemoveAll(chunkDir)) }() - head := createHead(b, series, chunkDir) - qHead, err := NewBlockQuerier(head, 1, int64(nSamples)) + head := createHead(b, nil, series, chunkDir) + qHead, err := NewBlockQuerier(head, 1, nSamples) testutil.Ok(b, err) queryTypes["_Head"] = qHead diff --git a/tsdb/tsdbutil/chunks.go b/tsdb/tsdbutil/chunks.go index 8acaae9b5..47760453e 100644 --- a/tsdb/tsdbutil/chunks.go +++ b/tsdb/tsdbutil/chunks.go @@ -18,23 +18,37 @@ import ( "github.com/prometheus/prometheus/tsdb/chunks" ) +type Samples interface { + Get(i int) Sample + Len() int +} + type Sample interface { T() int64 V() float64 } +type SampleSlice []Sample + +func (s SampleSlice) Get(i int) Sample { return s[i] } +func (s SampleSlice) Len() int { return len(s) } + func ChunkFromSamples(s []Sample) chunks.Meta { + return ChunkFromSamplesGeneric(SampleSlice(s)) +} + +func ChunkFromSamplesGeneric(s Samples) chunks.Meta { mint, maxt := int64(0), int64(0) - if len(s) > 0 { - mint, maxt = s[0].T(), s[len(s)-1].T() + if s.Len() > 0 { + mint, maxt = s.Get(0).T(), s.Get(s.Len()-1).T() } c := chunkenc.NewXORChunk() ca, _ := c.Appender() - for _, s := range s { - ca.Append(s.T(), s.V()) + for i := 0; i < s.Len(); i++ { + ca.Append(s.Get(i).T(), s.Get(i).V()) } return chunks.Meta{ MinTime: mint, diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 33cb5a9a3..cc64ab498 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -172,8 +172,7 @@ type TSDBAdminStats interface { // API can register a set of endpoints in a router and handle // them using the provided storage and query engine. type API struct { - // TODO(bwplotka): Change to SampleAndChunkQueryable in next PR. - Queryable storage.Queryable + Queryable storage.SampleAndChunkQueryable QueryEngine *promql.Engine targetRetriever func(context.Context) TargetRetriever @@ -502,6 +501,9 @@ func (api *API) labelNames(r *http.Request) apiFuncResult { if err != nil { return apiFuncResult{nil, &apiError{errorExec, err}, warnings, nil} } + if names == nil { + names = []string{} + } return apiFuncResult{names, nil, warnings, nil} } @@ -542,6 +544,9 @@ func (api *API) labelValues(r *http.Request) (result apiFuncResult) { if err != nil { return apiFuncResult{nil, &apiError{errorExec, err}, warnings, closer} } + if vals == nil { + vals = []string{} + } return apiFuncResult{vals, nil, warnings, closer} } @@ -1218,15 +1223,21 @@ func (api *API) remoteRead(w http.ResponseWriter, r *http.Request) { case prompb.ReadRequest_STREAMED_XOR_CHUNKS: api.remoteReadStreamedXORChunks(ctx, w, req, externalLabels, sortedExternalLabels) default: + // On empty or unknown types in req.AcceptedResponseTypes we default to non streamed, raw samples response. api.remoteReadSamples(ctx, w, req, externalLabels, sortedExternalLabels) } } -func (api *API) remoteReadSamples(ctx context.Context, w http.ResponseWriter, req *prompb.ReadRequest, externalLabels map[string]string, sortedExternalLabels []prompb.Label) { +func (api *API) remoteReadSamples( + ctx context.Context, + w http.ResponseWriter, + req *prompb.ReadRequest, + externalLabels map[string]string, + sortedExternalLabels []prompb.Label, +) { w.Header().Set("Content-Type", "application/x-protobuf") w.Header().Set("Content-Encoding", "snappy") - // On empty or unknown types in req.AcceptedResponseTypes we default to non streamed, raw samples response. resp := prompb.ReadResponse{ Results: make([]*prompb.QueryResult, len(req.Queries)), } @@ -1265,15 +1276,12 @@ func (api *API) remoteReadSamples(ctx context.Context, w http.ResponseWriter, re if err != nil { return err } - for _, w := range ws { level.Warn(api.logger).Log("msg", "Warnings on remote read query", "err", w.Error()) } - for _, ts := range resp.Results[i].Timeseries { ts.Labels = remote.MergeLabels(ts.Labels, sortedExternalLabels) } - return nil }(); err != nil { if httpErr, ok := err.(remote.HTTPError); ok { @@ -1307,14 +1315,13 @@ func (api *API) remoteReadStreamedXORChunks(ctx context.Context, w http.Response return err } - // TODO(bwplotka): Use ChunkQuerier once ready in tsdb package. - querier, err := api.Queryable.Querier(ctx, query.StartTimestampMs, query.EndTimestampMs) + querier, err := api.Queryable.ChunkQuerier(ctx, query.StartTimestampMs, query.EndTimestampMs) if err != nil { return err } defer func() { if err := querier.Close(); err != nil { - level.Warn(api.logger).Log("msg", "Error on chunk querier close", "warnings", err.Error()) + level.Warn(api.logger).Log("msg", "Error on chunk querier close", "err", err.Error()) } }() @@ -1331,7 +1338,7 @@ func (api *API) remoteReadStreamedXORChunks(ctx context.Context, w http.Response } } - ws, err := remote.DeprecatedStreamChunkedReadResponses( + ws, err := remote.StreamChunkedReadResponses( remote.NewChunkedWriter(w, f), int64(i), // The streaming API has to provide the series sorted. diff --git a/web/api/v1/api_test.go b/web/api/v1/api_test.go index 18861f6df..4f2e5ac3a 100644 --- a/web/api/v1/api_test.go +++ b/web/api/v1/api_test.go @@ -26,6 +26,7 @@ import ( "net/url" "os" "reflect" + "runtime" "sort" "strings" "testing" @@ -1702,40 +1703,48 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, testLabelAPI } for i, test := range tests { - for _, method := range methods(test.endpoint) { - // Build a context with the correct request params. - ctx := context.Background() - for p, v := range test.params { - ctx = route.WithParam(ctx, p, v) - } - t.Logf("run %d\t%s\t%q", i, method, test.query.Encode()) + t.Run(fmt.Sprintf("run %d %s %q", i, describeAPIFunc(test.endpoint), test.query.Encode()), func(t *testing.T) { + for _, method := range methods(test.endpoint) { + t.Run(method, func(t *testing.T) { + // Build a context with the correct request params. + ctx := context.Background() + for p, v := range test.params { + ctx = route.WithParam(ctx, p, v) + } - req, err := request(method, test.query) - if err != nil { - t.Fatal(err) - } + req, err := request(method, test.query) + if err != nil { + t.Fatal(err) + } - tr.ResetMetadataStore() - for _, tm := range test.metadata { - tr.SetMetadataStoreForTargets(tm.identifier, &testMetaStore{Metadata: tm.metadata}) - } + tr.ResetMetadataStore() + for _, tm := range test.metadata { + tr.SetMetadataStoreForTargets(tm.identifier, &testMetaStore{Metadata: tm.metadata}) + } - res := test.endpoint(req.WithContext(ctx)) - assertAPIError(t, res.err, test.errType) + res := test.endpoint(req.WithContext(ctx)) + assertAPIError(t, res.err, test.errType) - if test.sorter != nil { - test.sorter(res.data) - } + if test.sorter != nil { + test.sorter(res.data) + } - if test.responseLen != 0 { - assertAPIResponseLength(t, res.data, test.responseLen) - } else { - assertAPIResponse(t, res.data, test.response) + if test.responseLen != 0 { + assertAPIResponseLength(t, res.data, test.responseLen) + } else { + assertAPIResponse(t, res.data, test.response) + } + }) } - } + }) } } +func describeAPIFunc(f apiFunc) string { + name := runtime.FuncForPC(reflect.ValueOf(f).Pointer()).Name() + return strings.Split(name[strings.LastIndex(name, ".")+1:], "-")[0] +} + func assertAPIError(t *testing.T, got *apiError, exp errorType) { t.Helper()